diff --git a/array.go b/array.go index 49e8629..3d54976 100644 --- a/array.go +++ b/array.go @@ -385,70 +385,6 @@ func (a *Array) Set(index uint64, value Value) (Storable, error) { return existingStorable, nil } -// uninlineStorableIfNeeded uninlines given storable if needed, and -// returns uninlined Storable and its ValueID. -// If given storable is a WrapperStorable, this function uninlines -// wrapped storable if needed and returns a new WrapperStorable -// with wrapped uninlined storable and its ValidID. -func uninlineStorableIfNeeded(storage SlabStorage, storable Storable) (Storable, ValueID, bool, error) { - if storable == nil { - return storable, emptyValueID, false, nil - } - - switch s := storable.(type) { - case ArraySlab: // inlined array slab - err := s.Uninline(storage) - if err != nil { - return nil, emptyValueID, false, err - } - - slabID := s.SlabID() - - newStorable := SlabIDStorable(slabID) - valueID := slabIDToValueID(slabID) - - return newStorable, valueID, true, nil - - case MapSlab: // inlined map slab - err := s.Uninline(storage) - if err != nil { - return nil, emptyValueID, false, err - } - - slabID := s.SlabID() - - newStorable := SlabIDStorable(slabID) - valueID := slabIDToValueID(slabID) - - return newStorable, valueID, true, nil - - case SlabIDStorable: // uninlined slab - valueID := slabIDToValueID(SlabID(s)) - - return storable, valueID, false, nil - - case WrapperStorable: - unwrappedStorable := unwrapStorable(s) - - // Uninline wrapped storable if needed. - uninlinedWrappedStorable, valueID, uninlined, err := uninlineStorableIfNeeded(storage, unwrappedStorable) - if err != nil { - return nil, emptyValueID, false, err - } - - if !uninlined { - return storable, valueID, uninlined, nil - } - - // Create a new WrapperStorable with uninlinedWrappedStorable - newStorable := s.WrapAtreeStorable(uninlinedWrappedStorable) - - return newStorable, valueID, uninlined, nil - } - - return storable, emptyValueID, false, nil -} - func (a *Array) set(index uint64, value Value) (Storable, error) { existingStorable, err := a.root.Set(a.Storage, a.Address(), index, value) if err != nil { diff --git a/cbor_tag_nums.go b/cbor_tag_nums.go new file mode 100644 index 0000000..d5cdd45 --- /dev/null +++ b/cbor_tag_nums.go @@ -0,0 +1,78 @@ +/* + * Atree - Scalable Arrays and Ordered Maps + * + * Copyright Flow Foundation + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package atree + +import "fmt" + +const ( + // WARNING: tag numbers defined in here in github.com/onflow/atree + // MUST not overlap with tag numbers used by Cadence internal value encoding. + // As of Aug. 14, 2024, Cadence uses tag numbers from 128 to 230. + // See runtime/interpreter/encode.go at github.com/onflow/cadence. + + // Atree reserves CBOR tag numbers [240, 255] for internal use. + // Applications must use non-overlapping CBOR tag numbers to encode + // elements managed by atree containers. + minInternalCBORTagNumber = 240 + maxInternalCBORTagNumber = 255 + + // Reserved CBOR tag numbers for atree internal use. + + // Replace _ when new tag number is needed (use higher tag numbers first). + // Atree will use higher tag numbers first because Cadence will use lower tag numbers first. + // This approach allows more flexibility in case we need to revisit ranges used by Atree and Cadence. + + _ = 240 + _ = 241 + _ = 242 + _ = 243 + _ = 244 + _ = 245 + + CBORTagTypeInfoRef = 246 + + CBORTagInlinedArrayExtraData = 247 + CBORTagInlinedMapExtraData = 248 + CBORTagInlinedCompactMapExtraData = 249 + + CBORTagInlinedArray = 250 + CBORTagInlinedMap = 251 + CBORTagInlinedCompactMap = 252 + + CBORTagInlineCollisionGroup = 253 + CBORTagExternalCollisionGroup = 254 + + CBORTagSlabID = 255 +) + +// IsCBORTagNumberRangeAvailable returns true if the specified range is not reserved for internal use by atree. +// Applications must only use available (unreserved) CBOR tag numbers to encode elements in atree managed containers. +func IsCBORTagNumberRangeAvailable(minTagNum, maxTagNum uint64) (bool, error) { + if minTagNum > maxTagNum { + return false, NewUserError(fmt.Errorf("min CBOR tag number %d must be <= max CBOR tag number %d", minTagNum, maxTagNum)) + } + + return maxTagNum < minInternalCBORTagNumber || minTagNum > maxInternalCBORTagNumber, nil +} + +// ReservedCBORTagNumberRange returns minTagNum and maxTagNum of the range of CBOR tag numbers +// reserved for internal use by atree. +func ReservedCBORTagNumberRange() (minTagNum, maxTagNum uint64) { + return minInternalCBORTagNumber, maxInternalCBORTagNumber +} diff --git a/compactmap_extradata.go b/compactmap_extradata.go new file mode 100644 index 0000000..65cc4e1 --- /dev/null +++ b/compactmap_extradata.go @@ -0,0 +1,245 @@ +/* + * Atree - Scalable Arrays and Ordered Maps + * + * Copyright Flow Foundation + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package atree + +import ( + "encoding/binary" + "fmt" + "sort" + "strings" + + "github.com/fxamacker/cbor/v2" +) + +// compactMapExtraData is used for inlining compact values. +// compactMapExtraData includes hkeys and keys with map extra data +// because hkeys and keys are the same in order and content for +// all values with the same compact type and map seed. +type compactMapExtraData struct { + mapExtraData *MapExtraData + hkeys []Digest // hkeys is ordered by mapExtraData.Seed + keys []ComparableStorable // keys is ordered by mapExtraData.Seed +} + +var _ ExtraData = &compactMapExtraData{} + +const compactMapExtraDataLength = 3 + +func newCompactMapExtraData( + dec *cbor.StreamDecoder, + decodeTypeInfo TypeInfoDecoder, + decodeStorable StorableDecoder, +) (*compactMapExtraData, error) { + + length, err := dec.DecodeArrayHead() + if err != nil { + return nil, NewDecodingError(err) + } + + if length != compactMapExtraDataLength { + return nil, NewDecodingError( + fmt.Errorf( + "compact extra data has invalid length %d, want %d", + length, + arrayExtraDataLength, + )) + } + + // element 0: map extra data + mapExtraData, err := newMapExtraData(dec, decodeTypeInfo) + if err != nil { + // err is already categorized by newMapExtraData(). + return nil, err + } + + // element 1: digests + digestBytes, err := dec.DecodeBytes() + if err != nil { + return nil, NewDecodingError(err) + } + + if len(digestBytes)%digestSize != 0 { + return nil, NewDecodingError( + fmt.Errorf( + "decoding digests failed: number of bytes %d is not multiple of %d", + len(digestBytes), + digestSize)) + } + + digestCount := len(digestBytes) / digestSize + + // element 2: keys + keyCount, err := dec.DecodeArrayHead() + if err != nil { + return nil, NewDecodingError(err) + } + + if keyCount != uint64(digestCount) { + return nil, NewDecodingError( + fmt.Errorf( + "decoding compact map key failed: number of keys %d is different from number of digests %d", + keyCount, + digestCount)) + } + + hkeys := make([]Digest, digestCount) + for i := 0; i < digestCount; i++ { + hkeys[i] = Digest(binary.BigEndian.Uint64(digestBytes[i*digestSize:])) + } + + keys := make([]ComparableStorable, keyCount) + for i := uint64(0); i < keyCount; i++ { + // Decode compact map key + key, err := decodeStorable(dec, SlabIDUndefined, nil) + if err != nil { + // Wrap err as external error (if needed) because err is returned by StorableDecoder callback. + return nil, wrapErrorfAsExternalErrorIfNeeded(err, "failed to decode key's storable") + } + compactMapKey, ok := key.(ComparableStorable) + if !ok { + return nil, NewDecodingError(fmt.Errorf("failed to decode key's storable: got %T, expect ComparableStorable", key)) + } + keys[i] = compactMapKey + } + + return &compactMapExtraData{mapExtraData: mapExtraData, hkeys: hkeys, keys: keys}, nil +} + +func (c *compactMapExtraData) Encode(enc *Encoder, encodeTypeInfo encodeTypeInfo) error { + err := enc.CBOR.EncodeArrayHead(compactMapExtraDataLength) + if err != nil { + return NewEncodingError(err) + } + + // element 0: map extra data + err = c.mapExtraData.Encode(enc, encodeTypeInfo) + if err != nil { + // err is already categorized by MapExtraData.Encode(). + return err + } + + // element 1: digests + totalDigestSize := len(c.hkeys) * digestSize + + var digests []byte + if totalDigestSize <= len(enc.Scratch) { + digests = enc.Scratch[:totalDigestSize] + } else { + digests = make([]byte, totalDigestSize) + } + + for i := 0; i < len(c.hkeys); i++ { + binary.BigEndian.PutUint64(digests[i*digestSize:], uint64(c.hkeys[i])) + } + + err = enc.CBOR.EncodeBytes(digests) + if err != nil { + return NewEncodingError(err) + } + + // element 2: field names + err = enc.CBOR.EncodeArrayHead(uint64(len(c.keys))) + if err != nil { + return NewEncodingError(err) + } + + for _, key := range c.keys { + err = key.Encode(enc) + if err != nil { + // Wrap err as external error (if needed) because err is returned by ComparableStorable.Encode(). + return wrapErrorfAsExternalErrorIfNeeded(err, "failed to encode key's storable") + } + } + + err = enc.CBOR.Flush() + if err != nil { + return NewEncodingError(err) + } + + return nil +} + +func (c *compactMapExtraData) isExtraData() bool { + return true +} + +func (c *compactMapExtraData) Type() TypeInfo { + return c.mapExtraData.TypeInfo +} + +// makeCompactMapTypeID returns id of concatenated t.ID() with sorted names with "," as separator. +func makeCompactMapTypeID(encodedTypeInfo string, names []ComparableStorable) string { + const separator = "," + + if len(names) == 0 { + return encodedTypeInfo + } + + if len(names) == 1 { + return encodedTypeInfo + separator + names[0].ID() + } + + sorter := newFieldNameSorter(names) + + sort.Sort(sorter) + + return encodedTypeInfo + separator + sorter.join(separator) +} + +// fieldNameSorter sorts names by index (not in place sort). +type fieldNameSorter struct { + names []ComparableStorable + index []int +} + +func newFieldNameSorter(names []ComparableStorable) *fieldNameSorter { + index := make([]int, len(names)) + for i := 0; i < len(names); i++ { + index[i] = i + } + return &fieldNameSorter{ + names: names, + index: index, + } +} + +func (fn *fieldNameSorter) Len() int { + return len(fn.names) +} + +func (fn *fieldNameSorter) Less(i, j int) bool { + i = fn.index[i] + j = fn.index[j] + return fn.names[i].Less(fn.names[j]) +} + +func (fn *fieldNameSorter) Swap(i, j int) { + fn.index[i], fn.index[j] = fn.index[j], fn.index[i] +} + +func (fn *fieldNameSorter) join(sep string) string { + var sb strings.Builder + for i, index := range fn.index { + if i > 0 { + sb.WriteString(sep) + } + sb.WriteString(fn.names[index].ID()) + } + return sb.String() +} diff --git a/decode.go b/decode.go new file mode 100644 index 0000000..80a122e --- /dev/null +++ b/decode.go @@ -0,0 +1,96 @@ +/* + * Atree - Scalable Arrays and Ordered Maps + * + * Copyright Flow Foundation + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package atree + +import "github.com/fxamacker/cbor/v2" + +type StorableDecoder func( + decoder *cbor.StreamDecoder, + storableSlabID SlabID, + inlinedExtraData []ExtraData, +) ( + Storable, + error, +) + +func DecodeSlab( + id SlabID, + data []byte, + decMode cbor.DecMode, + decodeStorable StorableDecoder, + decodeTypeInfo TypeInfoDecoder, +) ( + Slab, + error, +) { + if len(data) < versionAndFlagSize { + return nil, NewDecodingErrorf("data is too short") + } + + h, err := newHeadFromData(data[:versionAndFlagSize]) + if err != nil { + return nil, NewDecodingError(err) + } + + switch h.getSlabType() { + + case slabArray: + + arrayDataType := h.getSlabArrayType() + + switch arrayDataType { + case slabArrayData: + return newArrayDataSlabFromData(id, data, decMode, decodeStorable, decodeTypeInfo) + case slabArrayMeta: + return newArrayMetaDataSlabFromData(id, data, decMode, decodeTypeInfo) + default: + return nil, NewDecodingErrorf("data has invalid head 0x%x", h[:]) + } + + case slabMap: + + mapDataType := h.getSlabMapType() + + switch mapDataType { + case slabMapData: + return newMapDataSlabFromData(id, data, decMode, decodeStorable, decodeTypeInfo) + case slabMapMeta: + return newMapMetaDataSlabFromData(id, data, decMode, decodeTypeInfo) + case slabMapCollisionGroup: + return newMapDataSlabFromData(id, data, decMode, decodeStorable, decodeTypeInfo) + default: + return nil, NewDecodingErrorf("data has invalid head 0x%x", h[:]) + } + + case slabStorable: + cborDec := decMode.NewByteStreamDecoder(data[versionAndFlagSize:]) + storable, err := decodeStorable(cborDec, id, nil) + if err != nil { + // Wrap err as external error (if needed) because err is returned by StorableDecoder callback. + return nil, wrapErrorfAsExternalErrorIfNeeded(err, "failed to decode slab storable") + } + return &StorableSlab{ + slabID: id, + storable: storable, + }, nil + + default: + return nil, NewDecodingErrorf("data has invalid head 0x%x", h[:]) + } +} diff --git a/encode.go b/encode.go index ec5c4b0..5828d10 100644 --- a/encode.go +++ b/encode.go @@ -19,6 +19,7 @@ package atree import ( + "bytes" "io" "math" @@ -57,79 +58,22 @@ func (enc *Encoder) hasInlinedExtraData() bool { return !enc._inlinedExtraData.empty() } -type StorableDecoder func( - decoder *cbor.StreamDecoder, - storableSlabID SlabID, - inlinedExtraData []ExtraData, -) ( - Storable, - error, -) +func EncodeSlab(slab Slab, encMode cbor.EncMode) ([]byte, error) { + var buf bytes.Buffer + enc := NewEncoder(&buf, encMode) -func DecodeSlab( - id SlabID, - data []byte, - decMode cbor.DecMode, - decodeStorable StorableDecoder, - decodeTypeInfo TypeInfoDecoder, -) ( - Slab, - error, -) { - if len(data) < versionAndFlagSize { - return nil, NewDecodingErrorf("data is too short") + err := slab.Encode(enc) + if err != nil { + // Wrap err as external error (if needed) because err is returned by Storable interface. + return nil, wrapErrorfAsExternalErrorIfNeeded(err, "failed to encode storable") } - h, err := newHeadFromData(data[:versionAndFlagSize]) + err = enc.CBOR.Flush() if err != nil { - return nil, NewDecodingError(err) + return nil, NewEncodingError(err) } - switch h.getSlabType() { - - case slabArray: - - arrayDataType := h.getSlabArrayType() - - switch arrayDataType { - case slabArrayData: - return newArrayDataSlabFromData(id, data, decMode, decodeStorable, decodeTypeInfo) - case slabArrayMeta: - return newArrayMetaDataSlabFromData(id, data, decMode, decodeTypeInfo) - default: - return nil, NewDecodingErrorf("data has invalid head 0x%x", h[:]) - } - - case slabMap: - - mapDataType := h.getSlabMapType() - - switch mapDataType { - case slabMapData: - return newMapDataSlabFromData(id, data, decMode, decodeStorable, decodeTypeInfo) - case slabMapMeta: - return newMapMetaDataSlabFromData(id, data, decMode, decodeTypeInfo) - case slabMapCollisionGroup: - return newMapDataSlabFromData(id, data, decMode, decodeStorable, decodeTypeInfo) - default: - return nil, NewDecodingErrorf("data has invalid head 0x%x", h[:]) - } - - case slabStorable: - cborDec := decMode.NewByteStreamDecoder(data[versionAndFlagSize:]) - storable, err := decodeStorable(cborDec, id, nil) - if err != nil { - // Wrap err as external error (if needed) because err is returned by StorableDecoder callback. - return nil, wrapErrorfAsExternalErrorIfNeeded(err, "failed to decode slab storable") - } - return &StorableSlab{ - slabID: id, - storable: storable, - }, nil - - default: - return nil, NewDecodingErrorf("data has invalid head 0x%x", h[:]) - } + return buf.Bytes(), nil } func GetUintCBORSize(n uint64) uint32 { diff --git a/extradata.go b/extradata.go new file mode 100644 index 0000000..3a64d80 --- /dev/null +++ b/extradata.go @@ -0,0 +1,404 @@ +/* + * Atree - Scalable Arrays and Ordered Maps + * + * Copyright Flow Foundation + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package atree + +import ( + "bytes" + "fmt" + "sort" + "sync" + + "github.com/fxamacker/cbor/v2" +) + +type ExtraData interface { + isExtraData() bool + Type() TypeInfo + Encode(enc *Encoder, encodeTypeInfo encodeTypeInfo) error +} + +type InlinedExtraData struct { + extraData []extraDataAndEncodedTypeInfo // Used to encode deduplicated ExtraData in order + compactMapTypeSet map[string]compactMapTypeInfo // Used to deduplicate compactMapExtraData by encoded TypeInfo + sorted field names + arrayExtraDataSet map[string]int // Used to deduplicate arrayExtraData by encoded TypeInfo +} + +type compactMapTypeInfo struct { + index int + keys []ComparableStorable +} + +type extraDataAndEncodedTypeInfo struct { + extraData ExtraData + encodedTypeInfo string // cached encoded type info +} + +func newInlinedExtraDataFromData( + data []byte, + decMode cbor.DecMode, + decodeStorable StorableDecoder, + defaultDecodeTypeInfo TypeInfoDecoder, +) ([]ExtraData, []byte, error) { + + dec := decMode.NewByteStreamDecoder(data) + + count, err := dec.DecodeArrayHead() + if err != nil { + return nil, nil, NewDecodingError(err) + } + + if count != inlinedExtraDataArrayCount { + return nil, nil, NewDecodingError(fmt.Errorf("failed to decode inlined extra data: expect %d elements, got %d elements", inlinedExtraDataArrayCount, count)) + } + + // element 0: array of duplicate type info + typeInfoCount, err := dec.DecodeArrayHead() + if err != nil { + return nil, nil, NewDecodingError(err) + } + + inlinedTypeInfo := make([]TypeInfo, int(typeInfoCount)) + for i := uint64(0); i < typeInfoCount; i++ { + inlinedTypeInfo[i], err = defaultDecodeTypeInfo(dec) + if err != nil { + return nil, nil, wrapErrorfAsExternalErrorIfNeeded(err, "failed to decode typeInfo") + } + } + + decodeTypeInfo := decodeTypeInfoRefIfNeeded(inlinedTypeInfo, defaultDecodeTypeInfo) + + // element 1: array of deduplicated extra data info + extraDataCount, err := dec.DecodeArrayHead() + if err != nil { + return nil, nil, NewDecodingError(err) + } + + if extraDataCount == 0 { + return nil, nil, NewDecodingError(fmt.Errorf("failed to decode inlined extra data: expect at least one inlined extra data")) + } + + inlinedExtraData := make([]ExtraData, extraDataCount) + for i := uint64(0); i < extraDataCount; i++ { + tagNum, err := dec.DecodeTagNumber() + if err != nil { + return nil, nil, NewDecodingError(err) + } + + switch tagNum { + case CBORTagInlinedArrayExtraData: + inlinedExtraData[i], err = newArrayExtraData(dec, decodeTypeInfo) + if err != nil { + // err is already categorized by newArrayExtraData(). + return nil, nil, err + } + + case CBORTagInlinedMapExtraData: + inlinedExtraData[i], err = newMapExtraData(dec, decodeTypeInfo) + if err != nil { + // err is already categorized by newMapExtraData(). + return nil, nil, err + } + + case CBORTagInlinedCompactMapExtraData: + inlinedExtraData[i], err = newCompactMapExtraData(dec, decodeTypeInfo, decodeStorable) + if err != nil { + // err is already categorized by newCompactMapExtraData(). + return nil, nil, err + } + + default: + return nil, nil, NewDecodingError(fmt.Errorf("failed to decode inlined extra data: unsupported tag number %d", tagNum)) + } + } + + return inlinedExtraData, data[dec.NumBytesDecoded():], nil +} + +func newInlinedExtraData() *InlinedExtraData { + // Maps used for deduplication are initialized lazily. + return &InlinedExtraData{} +} + +const inlinedExtraDataArrayCount = 2 + +var typeInfoRefTagHeadAndTagNumber = []byte{0xd8, CBORTagTypeInfoRef} + +// Encode encodes inlined extra data as 2-element array: +// +// +-----------------------+------------------------+ +// | [+ inlined type info] | [+ inlined extra data] | +// +-----------------------+------------------------+ +func (ied *InlinedExtraData) Encode(enc *Encoder) error { + + typeInfos, typeInfoIndexes := ied.findDuplicateTypeInfo() + + var err error + + err = enc.CBOR.EncodeArrayHead(inlinedExtraDataArrayCount) + if err != nil { + return NewEncodingError(err) + } + + // element 0: array of duplicate type info + err = enc.CBOR.EncodeArrayHead(uint64(len(typeInfos))) + if err != nil { + return NewEncodingError(err) + } + + // Encode type info + for _, typeInfo := range typeInfos { + // Encode cached type info as is. + err = enc.CBOR.EncodeRawBytes([]byte(typeInfo)) + if err != nil { + return NewEncodingError(err) + } + } + + // element 1: deduplicated array of extra data + err = enc.CBOR.EncodeArrayHead(uint64(len(ied.extraData))) + if err != nil { + return NewEncodingError(err) + } + + // Encode inlined extra data + for _, extraDataInfo := range ied.extraData { + var tagNum uint64 + + switch extraDataInfo.extraData.(type) { + case *ArrayExtraData: + tagNum = CBORTagInlinedArrayExtraData + + case *MapExtraData: + tagNum = CBORTagInlinedMapExtraData + + case *compactMapExtraData: + tagNum = CBORTagInlinedCompactMapExtraData + + default: + return NewEncodingError(fmt.Errorf("failed to encode unsupported extra data type %T", extraDataInfo.extraData)) + } + + err = enc.CBOR.EncodeTagHead(tagNum) + if err != nil { + return NewEncodingError(err) + } + + err = extraDataInfo.extraData.Encode(enc, func(enc *Encoder, _ TypeInfo) error { + encodedTypeInfo := extraDataInfo.encodedTypeInfo + + index, exist := typeInfoIndexes[encodedTypeInfo] + if !exist { + // typeInfo is not encoded separately, so encode typeInfo as is here. + err = enc.CBOR.EncodeRawBytes([]byte(encodedTypeInfo)) + if err != nil { + return NewEncodingError(err) + } + return nil + } + + err = enc.CBOR.EncodeRawBytes(typeInfoRefTagHeadAndTagNumber) + if err != nil { + return NewEncodingError(err) + } + + err = enc.CBOR.EncodeUint64(uint64(index)) + if err != nil { + return NewEncodingError(err) + } + + return nil + }) + if err != nil { + // err is already categorized by ExtraData.Encode(). + return err + } + } + + err = enc.CBOR.Flush() + if err != nil { + return NewEncodingError(err) + } + + return nil +} + +func (ied *InlinedExtraData) findDuplicateTypeInfo() ([]string, map[string]int) { + if len(ied.extraData) < 2 { + // No duplicate type info + return nil, nil + } + + // Make a copy of encoded type info to sort + encodedTypeInfo := make([]string, len(ied.extraData)) + for i, info := range ied.extraData { + encodedTypeInfo[i] = info.encodedTypeInfo + } + + sort.Strings(encodedTypeInfo) + + // Find duplicate type info + var duplicateTypeInfo []string + var duplicateTypeInfoIndexes map[string]int + + for currentIndex := 1; currentIndex < len(encodedTypeInfo); { + + if encodedTypeInfo[currentIndex-1] != encodedTypeInfo[currentIndex] { + currentIndex++ + continue + } + + // Found duplicate type info at currentIndex + duplicate := encodedTypeInfo[currentIndex] + + // Insert duplicate into duplicate type info list and map + duplicateTypeInfo = append(duplicateTypeInfo, duplicate) + + if duplicateTypeInfoIndexes == nil { + duplicateTypeInfoIndexes = make(map[string]int) + } + duplicateTypeInfoIndexes[duplicate] = len(duplicateTypeInfo) - 1 + + // Skip same duplicate from sorted list + currentIndex++ + for currentIndex < len(encodedTypeInfo) && encodedTypeInfo[currentIndex] == duplicate { + currentIndex++ + } + } + + return duplicateTypeInfo, duplicateTypeInfoIndexes +} + +// addArrayExtraData returns index of deduplicated array extra data. +// Array extra data is deduplicated by array type info ID because array +// extra data only contains type info. +func (ied *InlinedExtraData) addArrayExtraData(data *ArrayExtraData) (int, error) { + encodedTypeInfo, err := getEncodedTypeInfo(data.TypeInfo) + if err != nil { + // err is already categorized by getEncodedTypeInfo(). + return 0, err + } + + if ied.arrayExtraDataSet == nil { + ied.arrayExtraDataSet = make(map[string]int) + } + + index, exist := ied.arrayExtraDataSet[encodedTypeInfo] + if exist { + return index, nil + } + + index = len(ied.extraData) + ied.extraData = append(ied.extraData, extraDataAndEncodedTypeInfo{data, encodedTypeInfo}) + ied.arrayExtraDataSet[encodedTypeInfo] = index + + return index, nil +} + +// addMapExtraData returns index of map extra data. +// Map extra data is not deduplicated because it also contains count and seed. +func (ied *InlinedExtraData) addMapExtraData(data *MapExtraData) (int, error) { + encodedTypeInfo, err := getEncodedTypeInfo(data.TypeInfo) + if err != nil { + // err is already categorized by getEncodedTypeInfo(). + return 0, err + } + + index := len(ied.extraData) + ied.extraData = append(ied.extraData, extraDataAndEncodedTypeInfo{data, encodedTypeInfo}) + return index, nil +} + +// addCompactMapExtraData returns index of deduplicated compact map extra data. +// Compact map extra data is deduplicated by TypeInfo.ID() with sorted field names. +func (ied *InlinedExtraData) addCompactMapExtraData( + data *MapExtraData, + digests []Digest, + keys []ComparableStorable, +) (int, []ComparableStorable, error) { + + encodedTypeInfo, err := getEncodedTypeInfo(data.TypeInfo) + if err != nil { + // err is already categorized by getEncodedTypeInfo(). + return 0, nil, err + } + + if ied.compactMapTypeSet == nil { + ied.compactMapTypeSet = make(map[string]compactMapTypeInfo) + } + + compactMapTypeID := makeCompactMapTypeID(encodedTypeInfo, keys) + info, exist := ied.compactMapTypeSet[compactMapTypeID] + if exist { + return info.index, info.keys, nil + } + + compactMapData := &compactMapExtraData{ + mapExtraData: data, + hkeys: digests, + keys: keys, + } + + index := len(ied.extraData) + ied.extraData = append(ied.extraData, extraDataAndEncodedTypeInfo{compactMapData, encodedTypeInfo}) + + ied.compactMapTypeSet[compactMapTypeID] = compactMapTypeInfo{ + keys: keys, + index: index, + } + + return index, keys, nil +} + +func (ied *InlinedExtraData) empty() bool { + return len(ied.extraData) == 0 +} + +func getEncodedTypeInfo(ti TypeInfo) (string, error) { + b := getTypeIDBuffer() + defer putTypeIDBuffer(b) + + enc := cbor.NewStreamEncoder(b) + err := ti.Encode(enc) + if err != nil { + // Wrap err as external error (if needed) because err is returned by TypeInfo.Encode(). + return "", wrapErrorfAsExternalErrorIfNeeded(err, "failed to encode type info") + } + enc.Flush() + + return b.String(), nil +} + +const defaultTypeIDBufferSize = 256 + +var typeIDBufferPool = sync.Pool{ + New: func() interface{} { + e := new(bytes.Buffer) + e.Grow(defaultTypeIDBufferSize) + return e + }, +} + +func getTypeIDBuffer() *bytes.Buffer { + return typeIDBufferPool.Get().(*bytes.Buffer) +} + +func putTypeIDBuffer(e *bytes.Buffer) { + e.Reset() + typeIDBufferPool.Put(e) +} diff --git a/inline_utils.go b/inline_utils.go new file mode 100644 index 0000000..d716f44 --- /dev/null +++ b/inline_utils.go @@ -0,0 +1,83 @@ +/* + * Atree - Scalable Arrays and Ordered Maps + * + * Copyright Flow Foundation + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package atree + +// uninlineStorableIfNeeded uninlines given storable if needed, and +// returns uninlined Storable and its ValueID. +// If given storable is a WrapperStorable, this function uninlines +// wrapped storable if needed and returns a new WrapperStorable +// with wrapped uninlined storable and its ValidID. +func uninlineStorableIfNeeded(storage SlabStorage, storable Storable) (Storable, ValueID, bool, error) { + if storable == nil { + return storable, emptyValueID, false, nil + } + + switch s := storable.(type) { + case ArraySlab: // inlined array slab + err := s.Uninline(storage) + if err != nil { + return nil, emptyValueID, false, err + } + + slabID := s.SlabID() + + newStorable := SlabIDStorable(slabID) + valueID := slabIDToValueID(slabID) + + return newStorable, valueID, true, nil + + case MapSlab: // inlined map slab + err := s.Uninline(storage) + if err != nil { + return nil, emptyValueID, false, err + } + + slabID := s.SlabID() + + newStorable := SlabIDStorable(slabID) + valueID := slabIDToValueID(slabID) + + return newStorable, valueID, true, nil + + case SlabIDStorable: // uninlined slab + valueID := slabIDToValueID(SlabID(s)) + + return storable, valueID, false, nil + + case WrapperStorable: + unwrappedStorable := unwrapStorable(s) + + // Uninline wrapped storable if needed. + uninlinedWrappedStorable, valueID, uninlined, err := uninlineStorableIfNeeded(storage, unwrappedStorable) + if err != nil { + return nil, emptyValueID, false, err + } + + if !uninlined { + return storable, valueID, uninlined, nil + } + + // Create a new WrapperStorable with uninlinedWrappedStorable + newStorable := s.WrapAtreeStorable(uninlinedWrappedStorable) + + return newStorable, valueID, uninlined, nil + } + + return storable, emptyValueID, false, nil +} diff --git a/slab_id_storable.go b/slab_id_storable.go new file mode 100644 index 0000000..27a8189 --- /dev/null +++ b/slab_id_storable.go @@ -0,0 +1,110 @@ +/* + * Atree - Scalable Arrays and Ordered Maps + * + * Copyright Flow Foundation + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package atree + +import ( + "fmt" + + "github.com/fxamacker/cbor/v2" +) + +type SlabIDStorable SlabID + +var _ ContainerStorable = SlabIDStorable{} + +func (v SlabIDStorable) HasPointer() bool { + return true +} + +func (v SlabIDStorable) ChildStorables() []Storable { + return nil +} + +func (v SlabIDStorable) StoredValue(storage SlabStorage) (Value, error) { + id := SlabID(v) + if err := id.Valid(); err != nil { + // Don't need to wrap error as external error because err is already categorized by SlabID.Valid(). + return nil, err + } + + slab, found, err := storage.Retrieve(id) + if err != nil { + // Wrap err as external error (if needed) because err is returned by SlabStorage interface. + return nil, wrapErrorfAsExternalErrorIfNeeded(err, fmt.Sprintf("failed to retrieve slab %s", id)) + } + if !found { + return nil, NewSlabNotFoundErrorf(id, "slab not found for stored value") + } + value, err := slab.StoredValue(storage) + if err != nil { + // Wrap err as external error (if needed) because err is returned by Storable interface. + return nil, wrapErrorfAsExternalErrorIfNeeded(err, "failed to get storable's stored value") + } + return value, nil +} + +// Encode encodes SlabIDStorable as +// +// cbor.Tag{ +// Number: cborTagSlabID, +// Content: byte(v), +// } +func (v SlabIDStorable) Encode(enc *Encoder) error { + err := enc.CBOR.EncodeRawBytes([]byte{ + // tag number + 0xd8, CBORTagSlabID, + }) + if err != nil { + return NewEncodingError(err) + } + + copy(enc.Scratch[:], v.address[:]) + copy(enc.Scratch[8:], v.index[:]) + + err = enc.CBOR.EncodeBytes(enc.Scratch[:SlabIDLength]) + if err != nil { + return NewEncodingError(err) + } + + return nil +} + +func (v SlabIDStorable) ByteSize() uint32 { + // tag number (2 bytes) + byte string header (1 byte) + slab id (16 bytes) + return 2 + 1 + SlabIDLength +} + +func (v SlabIDStorable) String() string { + return fmt.Sprintf("SlabIDStorable(%d)", v) +} + +func DecodeSlabIDStorable(dec *cbor.StreamDecoder) (Storable, error) { + b, err := dec.DecodeBytes() + if err != nil { + return nil, NewDecodingError(err) + } + + id, err := NewSlabIDFromRawBytes(b) + if err != nil { + // Don't need to wrap error as external error because err is already categorized by NewSlabIDFromRawBytes(). + return nil, err + } + + return SlabIDStorable(id), nil +} diff --git a/storable.go b/storable.go index ab7ce4c..ed17917 100644 --- a/storable.go +++ b/storable.go @@ -18,13 +18,6 @@ package atree -import ( - "bytes" - "fmt" - - "github.com/fxamacker/cbor/v2" -) - type Storable interface { Encode(*Encoder) error @@ -90,166 +83,6 @@ func unwrapStorable(s Storable) Storable { } } -const ( - // WARNING: tag numbers defined in here in github.com/onflow/atree - // MUST not overlap with tag numbers used by Cadence internal value encoding. - // As of Aug. 14, 2024, Cadence uses tag numbers from 128 to 230. - // See runtime/interpreter/encode.go at github.com/onflow/cadence. - - // Atree reserves CBOR tag numbers [240, 255] for internal use. - // Applications must use non-overlapping CBOR tag numbers to encode - // elements managed by atree containers. - minInternalCBORTagNumber = 240 - maxInternalCBORTagNumber = 255 - - // Reserved CBOR tag numbers for atree internal use. - - // Replace _ when new tag number is needed (use higher tag numbers first). - // Atree will use higher tag numbers first because Cadence will use lower tag numbers first. - // This approach allows more flexibility in case we need to revisit ranges used by Atree and Cadence. - - _ = 240 - _ = 241 - _ = 242 - _ = 243 - _ = 244 - _ = 245 - - CBORTagTypeInfoRef = 246 - - CBORTagInlinedArrayExtraData = 247 - CBORTagInlinedMapExtraData = 248 - CBORTagInlinedCompactMapExtraData = 249 - - CBORTagInlinedArray = 250 - CBORTagInlinedMap = 251 - CBORTagInlinedCompactMap = 252 - - CBORTagInlineCollisionGroup = 253 - CBORTagExternalCollisionGroup = 254 - - CBORTagSlabID = 255 -) - -// IsCBORTagNumberRangeAvailable returns true if the specified range is not reserved for internal use by atree. -// Applications must only use available (unreserved) CBOR tag numbers to encode elements in atree managed containers. -func IsCBORTagNumberRangeAvailable(minTagNum, maxTagNum uint64) (bool, error) { - if minTagNum > maxTagNum { - return false, NewUserError(fmt.Errorf("min CBOR tag number %d must be <= max CBOR tag number %d", minTagNum, maxTagNum)) - } - - return maxTagNum < minInternalCBORTagNumber || minTagNum > maxInternalCBORTagNumber, nil -} - -// ReservedCBORTagNumberRange returns minTagNum and maxTagNum of the range of CBOR tag numbers -// reserved for internal use by atree. -func ReservedCBORTagNumberRange() (minTagNum, maxTagNum uint64) { - return minInternalCBORTagNumber, maxInternalCBORTagNumber -} - -type SlabIDStorable SlabID - -var _ ContainerStorable = SlabIDStorable{} - -func (v SlabIDStorable) HasPointer() bool { - return true -} - -func (v SlabIDStorable) ChildStorables() []Storable { - return nil -} - -func (v SlabIDStorable) StoredValue(storage SlabStorage) (Value, error) { - id := SlabID(v) - if err := id.Valid(); err != nil { - // Don't need to wrap error as external error because err is already categorized by SlabID.Valid(). - return nil, err - } - - slab, found, err := storage.Retrieve(id) - if err != nil { - // Wrap err as external error (if needed) because err is returned by SlabStorage interface. - return nil, wrapErrorfAsExternalErrorIfNeeded(err, fmt.Sprintf("failed to retrieve slab %s", id)) - } - if !found { - return nil, NewSlabNotFoundErrorf(id, "slab not found for stored value") - } - value, err := slab.StoredValue(storage) - if err != nil { - // Wrap err as external error (if needed) because err is returned by Storable interface. - return nil, wrapErrorfAsExternalErrorIfNeeded(err, "failed to get storable's stored value") - } - return value, nil -} - -// Encode encodes SlabIDStorable as -// -// cbor.Tag{ -// Number: cborTagSlabID, -// Content: byte(v), -// } -func (v SlabIDStorable) Encode(enc *Encoder) error { - err := enc.CBOR.EncodeRawBytes([]byte{ - // tag number - 0xd8, CBORTagSlabID, - }) - if err != nil { - return NewEncodingError(err) - } - - copy(enc.Scratch[:], v.address[:]) - copy(enc.Scratch[8:], v.index[:]) - - err = enc.CBOR.EncodeBytes(enc.Scratch[:SlabIDLength]) - if err != nil { - return NewEncodingError(err) - } - - return nil -} - -func (v SlabIDStorable) ByteSize() uint32 { - // tag number (2 bytes) + byte string header (1 byte) + slab id (16 bytes) - return 2 + 1 + SlabIDLength -} - -func (v SlabIDStorable) String() string { - return fmt.Sprintf("SlabIDStorable(%d)", v) -} - -func EncodeSlab(slab Slab, encMode cbor.EncMode) ([]byte, error) { - var buf bytes.Buffer - enc := NewEncoder(&buf, encMode) - - err := slab.Encode(enc) - if err != nil { - // Wrap err as external error (if needed) because err is returned by Storable interface. - return nil, wrapErrorfAsExternalErrorIfNeeded(err, "failed to encode storable") - } - - err = enc.CBOR.Flush() - if err != nil { - return nil, NewEncodingError(err) - } - - return buf.Bytes(), nil -} - -func DecodeSlabIDStorable(dec *cbor.StreamDecoder) (Storable, error) { - b, err := dec.DecodeBytes() - if err != nil { - return nil, NewDecodingError(err) - } - - id, err := NewSlabIDFromRawBytes(b) - if err != nil { - // Don't need to wrap error as external error because err is already categorized by NewSlabIDFromRawBytes(). - return nil, err - } - - return SlabIDStorable(id), nil -} - func getLoadedValue(storage SlabStorage, storable Storable) (Value, error) { switch storable := storable.(type) { case SlabIDStorable: diff --git a/typeinfo.go b/typeinfo.go index 944cb2a..762f5dc 100644 --- a/typeinfo.go +++ b/typeinfo.go @@ -20,11 +20,7 @@ package atree import ( "bytes" - "encoding/binary" "fmt" - "sort" - "strings" - "sync" "github.com/fxamacker/cbor/v2" ) @@ -91,596 +87,3 @@ func decodeTypeInfoRefIfNeeded(inlinedTypeInfo []TypeInfo, defaultTypeInfoDecode return defaultTypeInfoDecoder(dec) } } - -type ExtraData interface { - isExtraData() bool - Type() TypeInfo - Encode(enc *Encoder, encodeTypeInfo encodeTypeInfo) error -} - -// compactMapExtraData is used for inlining compact values. -// compactMapExtraData includes hkeys and keys with map extra data -// because hkeys and keys are the same in order and content for -// all values with the same compact type and map seed. -type compactMapExtraData struct { - mapExtraData *MapExtraData - hkeys []Digest // hkeys is ordered by mapExtraData.Seed - keys []ComparableStorable // keys is ordered by mapExtraData.Seed -} - -var _ ExtraData = &compactMapExtraData{} - -const compactMapExtraDataLength = 3 - -func (c *compactMapExtraData) isExtraData() bool { - return true -} - -func (c *compactMapExtraData) Type() TypeInfo { - return c.mapExtraData.TypeInfo -} - -func (c *compactMapExtraData) Encode(enc *Encoder, encodeTypeInfo encodeTypeInfo) error { - err := enc.CBOR.EncodeArrayHead(compactMapExtraDataLength) - if err != nil { - return NewEncodingError(err) - } - - // element 0: map extra data - err = c.mapExtraData.Encode(enc, encodeTypeInfo) - if err != nil { - // err is already categorized by MapExtraData.Encode(). - return err - } - - // element 1: digests - totalDigestSize := len(c.hkeys) * digestSize - - var digests []byte - if totalDigestSize <= len(enc.Scratch) { - digests = enc.Scratch[:totalDigestSize] - } else { - digests = make([]byte, totalDigestSize) - } - - for i := 0; i < len(c.hkeys); i++ { - binary.BigEndian.PutUint64(digests[i*digestSize:], uint64(c.hkeys[i])) - } - - err = enc.CBOR.EncodeBytes(digests) - if err != nil { - return NewEncodingError(err) - } - - // element 2: field names - err = enc.CBOR.EncodeArrayHead(uint64(len(c.keys))) - if err != nil { - return NewEncodingError(err) - } - - for _, key := range c.keys { - err = key.Encode(enc) - if err != nil { - // Wrap err as external error (if needed) because err is returned by ComparableStorable.Encode(). - return wrapErrorfAsExternalErrorIfNeeded(err, "failed to encode key's storable") - } - } - - err = enc.CBOR.Flush() - if err != nil { - return NewEncodingError(err) - } - - return nil -} - -func newCompactMapExtraData( - dec *cbor.StreamDecoder, - decodeTypeInfo TypeInfoDecoder, - decodeStorable StorableDecoder, -) (*compactMapExtraData, error) { - - length, err := dec.DecodeArrayHead() - if err != nil { - return nil, NewDecodingError(err) - } - - if length != compactMapExtraDataLength { - return nil, NewDecodingError( - fmt.Errorf( - "compact extra data has invalid length %d, want %d", - length, - arrayExtraDataLength, - )) - } - - // element 0: map extra data - mapExtraData, err := newMapExtraData(dec, decodeTypeInfo) - if err != nil { - // err is already categorized by newMapExtraData(). - return nil, err - } - - // element 1: digests - digestBytes, err := dec.DecodeBytes() - if err != nil { - return nil, NewDecodingError(err) - } - - if len(digestBytes)%digestSize != 0 { - return nil, NewDecodingError( - fmt.Errorf( - "decoding digests failed: number of bytes %d is not multiple of %d", - len(digestBytes), - digestSize)) - } - - digestCount := len(digestBytes) / digestSize - - // element 2: keys - keyCount, err := dec.DecodeArrayHead() - if err != nil { - return nil, NewDecodingError(err) - } - - if keyCount != uint64(digestCount) { - return nil, NewDecodingError( - fmt.Errorf( - "decoding compact map key failed: number of keys %d is different from number of digests %d", - keyCount, - digestCount)) - } - - hkeys := make([]Digest, digestCount) - for i := 0; i < digestCount; i++ { - hkeys[i] = Digest(binary.BigEndian.Uint64(digestBytes[i*digestSize:])) - } - - keys := make([]ComparableStorable, keyCount) - for i := uint64(0); i < keyCount; i++ { - // Decode compact map key - key, err := decodeStorable(dec, SlabIDUndefined, nil) - if err != nil { - // Wrap err as external error (if needed) because err is returned by StorableDecoder callback. - return nil, wrapErrorfAsExternalErrorIfNeeded(err, "failed to decode key's storable") - } - compactMapKey, ok := key.(ComparableStorable) - if !ok { - return nil, NewDecodingError(fmt.Errorf("failed to decode key's storable: got %T, expect ComparableStorable", key)) - } - keys[i] = compactMapKey - } - - return &compactMapExtraData{mapExtraData: mapExtraData, hkeys: hkeys, keys: keys}, nil -} - -type compactMapTypeInfo struct { - index int - keys []ComparableStorable -} - -type extraDataAndEncodedTypeInfo struct { - extraData ExtraData - encodedTypeInfo string // cached encoded type info -} - -type InlinedExtraData struct { - extraData []extraDataAndEncodedTypeInfo // Used to encode deduplicated ExtraData in order - compactMapTypeSet map[string]compactMapTypeInfo // Used to deduplicate compactMapExtraData by encoded TypeInfo + sorted field names - arrayExtraDataSet map[string]int // Used to deduplicate arrayExtraData by encoded TypeInfo -} - -func newInlinedExtraData() *InlinedExtraData { - // Maps used for deduplication are initialized lazily. - return &InlinedExtraData{} -} - -const inlinedExtraDataArrayCount = 2 - -var typeInfoRefTagHeadAndTagNumber = []byte{0xd8, CBORTagTypeInfoRef} - -// Encode encodes inlined extra data as 2-element array: -// -// +-----------------------+------------------------+ -// | [+ inlined type info] | [+ inlined extra data] | -// +-----------------------+------------------------+ -func (ied *InlinedExtraData) Encode(enc *Encoder) error { - - typeInfos, typeInfoIndexes := ied.findDuplicateTypeInfo() - - var err error - - err = enc.CBOR.EncodeArrayHead(inlinedExtraDataArrayCount) - if err != nil { - return NewEncodingError(err) - } - - // element 0: array of duplicate type info - err = enc.CBOR.EncodeArrayHead(uint64(len(typeInfos))) - if err != nil { - return NewEncodingError(err) - } - - // Encode type info - for _, typeInfo := range typeInfos { - // Encode cached type info as is. - err = enc.CBOR.EncodeRawBytes([]byte(typeInfo)) - if err != nil { - return NewEncodingError(err) - } - } - - // element 1: deduplicated array of extra data - err = enc.CBOR.EncodeArrayHead(uint64(len(ied.extraData))) - if err != nil { - return NewEncodingError(err) - } - - // Encode inlined extra data - for _, extraDataInfo := range ied.extraData { - var tagNum uint64 - - switch extraDataInfo.extraData.(type) { - case *ArrayExtraData: - tagNum = CBORTagInlinedArrayExtraData - - case *MapExtraData: - tagNum = CBORTagInlinedMapExtraData - - case *compactMapExtraData: - tagNum = CBORTagInlinedCompactMapExtraData - - default: - return NewEncodingError(fmt.Errorf("failed to encode unsupported extra data type %T", extraDataInfo.extraData)) - } - - err = enc.CBOR.EncodeTagHead(tagNum) - if err != nil { - return NewEncodingError(err) - } - - err = extraDataInfo.extraData.Encode(enc, func(enc *Encoder, _ TypeInfo) error { - encodedTypeInfo := extraDataInfo.encodedTypeInfo - - index, exist := typeInfoIndexes[encodedTypeInfo] - if !exist { - // typeInfo is not encoded separately, so encode typeInfo as is here. - err = enc.CBOR.EncodeRawBytes([]byte(encodedTypeInfo)) - if err != nil { - return NewEncodingError(err) - } - return nil - } - - err = enc.CBOR.EncodeRawBytes(typeInfoRefTagHeadAndTagNumber) - if err != nil { - return NewEncodingError(err) - } - - err = enc.CBOR.EncodeUint64(uint64(index)) - if err != nil { - return NewEncodingError(err) - } - - return nil - }) - if err != nil { - // err is already categorized by ExtraData.Encode(). - return err - } - } - - err = enc.CBOR.Flush() - if err != nil { - return NewEncodingError(err) - } - - return nil -} - -func (ied *InlinedExtraData) findDuplicateTypeInfo() ([]string, map[string]int) { - if len(ied.extraData) < 2 { - // No duplicate type info - return nil, nil - } - - // Make a copy of encoded type info to sort - encodedTypeInfo := make([]string, len(ied.extraData)) - for i, info := range ied.extraData { - encodedTypeInfo[i] = info.encodedTypeInfo - } - - sort.Strings(encodedTypeInfo) - - // Find duplicate type info - var duplicateTypeInfo []string - var duplicateTypeInfoIndexes map[string]int - - for currentIndex := 1; currentIndex < len(encodedTypeInfo); { - - if encodedTypeInfo[currentIndex-1] != encodedTypeInfo[currentIndex] { - currentIndex++ - continue - } - - // Found duplicate type info at currentIndex - duplicate := encodedTypeInfo[currentIndex] - - // Insert duplicate into duplicate type info list and map - duplicateTypeInfo = append(duplicateTypeInfo, duplicate) - - if duplicateTypeInfoIndexes == nil { - duplicateTypeInfoIndexes = make(map[string]int) - } - duplicateTypeInfoIndexes[duplicate] = len(duplicateTypeInfo) - 1 - - // Skip same duplicate from sorted list - currentIndex++ - for currentIndex < len(encodedTypeInfo) && encodedTypeInfo[currentIndex] == duplicate { - currentIndex++ - } - } - - return duplicateTypeInfo, duplicateTypeInfoIndexes -} - -func newInlinedExtraDataFromData( - data []byte, - decMode cbor.DecMode, - decodeStorable StorableDecoder, - defaultDecodeTypeInfo TypeInfoDecoder, -) ([]ExtraData, []byte, error) { - - dec := decMode.NewByteStreamDecoder(data) - - count, err := dec.DecodeArrayHead() - if err != nil { - return nil, nil, NewDecodingError(err) - } - - if count != inlinedExtraDataArrayCount { - return nil, nil, NewDecodingError(fmt.Errorf("failed to decode inlined extra data: expect %d elements, got %d elements", inlinedExtraDataArrayCount, count)) - } - - // element 0: array of duplicate type info - typeInfoCount, err := dec.DecodeArrayHead() - if err != nil { - return nil, nil, NewDecodingError(err) - } - - inlinedTypeInfo := make([]TypeInfo, int(typeInfoCount)) - for i := uint64(0); i < typeInfoCount; i++ { - inlinedTypeInfo[i], err = defaultDecodeTypeInfo(dec) - if err != nil { - return nil, nil, wrapErrorfAsExternalErrorIfNeeded(err, "failed to decode typeInfo") - } - } - - decodeTypeInfo := decodeTypeInfoRefIfNeeded(inlinedTypeInfo, defaultDecodeTypeInfo) - - // element 1: array of deduplicated extra data info - extraDataCount, err := dec.DecodeArrayHead() - if err != nil { - return nil, nil, NewDecodingError(err) - } - - if extraDataCount == 0 { - return nil, nil, NewDecodingError(fmt.Errorf("failed to decode inlined extra data: expect at least one inlined extra data")) - } - - inlinedExtraData := make([]ExtraData, extraDataCount) - for i := uint64(0); i < extraDataCount; i++ { - tagNum, err := dec.DecodeTagNumber() - if err != nil { - return nil, nil, NewDecodingError(err) - } - - switch tagNum { - case CBORTagInlinedArrayExtraData: - inlinedExtraData[i], err = newArrayExtraData(dec, decodeTypeInfo) - if err != nil { - // err is already categorized by newArrayExtraData(). - return nil, nil, err - } - - case CBORTagInlinedMapExtraData: - inlinedExtraData[i], err = newMapExtraData(dec, decodeTypeInfo) - if err != nil { - // err is already categorized by newMapExtraData(). - return nil, nil, err - } - - case CBORTagInlinedCompactMapExtraData: - inlinedExtraData[i], err = newCompactMapExtraData(dec, decodeTypeInfo, decodeStorable) - if err != nil { - // err is already categorized by newCompactMapExtraData(). - return nil, nil, err - } - - default: - return nil, nil, NewDecodingError(fmt.Errorf("failed to decode inlined extra data: unsupported tag number %d", tagNum)) - } - } - - return inlinedExtraData, data[dec.NumBytesDecoded():], nil -} - -// addArrayExtraData returns index of deduplicated array extra data. -// Array extra data is deduplicated by array type info ID because array -// extra data only contains type info. -func (ied *InlinedExtraData) addArrayExtraData(data *ArrayExtraData) (int, error) { - encodedTypeInfo, err := getEncodedTypeInfo(data.TypeInfo) - if err != nil { - // err is already categorized by getEncodedTypeInfo(). - return 0, err - } - - if ied.arrayExtraDataSet == nil { - ied.arrayExtraDataSet = make(map[string]int) - } - - index, exist := ied.arrayExtraDataSet[encodedTypeInfo] - if exist { - return index, nil - } - - index = len(ied.extraData) - ied.extraData = append(ied.extraData, extraDataAndEncodedTypeInfo{data, encodedTypeInfo}) - ied.arrayExtraDataSet[encodedTypeInfo] = index - - return index, nil -} - -// addMapExtraData returns index of map extra data. -// Map extra data is not deduplicated because it also contains count and seed. -func (ied *InlinedExtraData) addMapExtraData(data *MapExtraData) (int, error) { - encodedTypeInfo, err := getEncodedTypeInfo(data.TypeInfo) - if err != nil { - // err is already categorized by getEncodedTypeInfo(). - return 0, err - } - - index := len(ied.extraData) - ied.extraData = append(ied.extraData, extraDataAndEncodedTypeInfo{data, encodedTypeInfo}) - return index, nil -} - -// addCompactMapExtraData returns index of deduplicated compact map extra data. -// Compact map extra data is deduplicated by TypeInfo.ID() with sorted field names. -func (ied *InlinedExtraData) addCompactMapExtraData( - data *MapExtraData, - digests []Digest, - keys []ComparableStorable, -) (int, []ComparableStorable, error) { - - encodedTypeInfo, err := getEncodedTypeInfo(data.TypeInfo) - if err != nil { - // err is already categorized by getEncodedTypeInfo(). - return 0, nil, err - } - - if ied.compactMapTypeSet == nil { - ied.compactMapTypeSet = make(map[string]compactMapTypeInfo) - } - - compactMapTypeID := makeCompactMapTypeID(encodedTypeInfo, keys) - info, exist := ied.compactMapTypeSet[compactMapTypeID] - if exist { - return info.index, info.keys, nil - } - - compactMapData := &compactMapExtraData{ - mapExtraData: data, - hkeys: digests, - keys: keys, - } - - index := len(ied.extraData) - ied.extraData = append(ied.extraData, extraDataAndEncodedTypeInfo{compactMapData, encodedTypeInfo}) - - ied.compactMapTypeSet[compactMapTypeID] = compactMapTypeInfo{ - keys: keys, - index: index, - } - - return index, keys, nil -} - -func (ied *InlinedExtraData) empty() bool { - return len(ied.extraData) == 0 -} - -// makeCompactMapTypeID returns id of concatenated t.ID() with sorted names with "," as separator. -func makeCompactMapTypeID(encodedTypeInfo string, names []ComparableStorable) string { - const separator = "," - - if len(names) == 0 { - return encodedTypeInfo - } - - if len(names) == 1 { - return encodedTypeInfo + separator + names[0].ID() - } - - sorter := newFieldNameSorter(names) - - sort.Sort(sorter) - - return encodedTypeInfo + separator + sorter.join(separator) -} - -// fieldNameSorter sorts names by index (not in place sort). -type fieldNameSorter struct { - names []ComparableStorable - index []int -} - -func newFieldNameSorter(names []ComparableStorable) *fieldNameSorter { - index := make([]int, len(names)) - for i := 0; i < len(names); i++ { - index[i] = i - } - return &fieldNameSorter{ - names: names, - index: index, - } -} - -func (fn *fieldNameSorter) Len() int { - return len(fn.names) -} - -func (fn *fieldNameSorter) Less(i, j int) bool { - i = fn.index[i] - j = fn.index[j] - return fn.names[i].Less(fn.names[j]) -} - -func (fn *fieldNameSorter) Swap(i, j int) { - fn.index[i], fn.index[j] = fn.index[j], fn.index[i] -} - -func (fn *fieldNameSorter) join(sep string) string { - var sb strings.Builder - for i, index := range fn.index { - if i > 0 { - sb.WriteString(sep) - } - sb.WriteString(fn.names[index].ID()) - } - return sb.String() -} - -func getEncodedTypeInfo(ti TypeInfo) (string, error) { - b := getTypeIDBuffer() - defer putTypeIDBuffer(b) - - enc := cbor.NewStreamEncoder(b) - err := ti.Encode(enc) - if err != nil { - // Wrap err as external error (if needed) because err is returned by TypeInfo.Encode(). - return "", wrapErrorfAsExternalErrorIfNeeded(err, "failed to encode type info") - } - enc.Flush() - - return b.String(), nil -} - -const defaultTypeIDBufferSize = 256 - -var typeIDBufferPool = sync.Pool{ - New: func() interface{} { - e := new(bytes.Buffer) - e.Grow(defaultTypeIDBufferSize) - return e - }, -} - -func getTypeIDBuffer() *bytes.Buffer { - return typeIDBufferPool.Get().(*bytes.Buffer) -} - -func putTypeIDBuffer(e *bytes.Buffer) { - e.Reset() - typeIDBufferPool.Put(e) -}