// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements.  See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership.  The ASF licenses this file
// to you 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 variant

import (
	
	
	
	
	
	
	
	
	
	
	
	

	
	
	
	
	
)

// Builder is used to construct Variant values by appending data of various types.
// It manages an internal buffer for the value data and a dictionary for field keys.
type Builder struct {
	buf             bytes.Buffer
	dict            map[string]uint32
	dictKeys        [][]byte
	totalDictSize   int
	allowDuplicates bool
}

func ( Metadata) *Builder {
	 := new(Builder)

	.dictKeys = .keys
	.dict = make(map[string]uint32)
	for ,  := range .keys {
		.dict[string()] = uint32()
		.totalDictSize += len()
	}

	return 
}

// SetAllowDuplicates controls whether duplicate keys are allowed in objects.
// When true, the last value for a key is used. When false, an error is returned
// if a duplicate key is detected.
func ( *Builder) ( bool) {
	.allowDuplicates = 
}

// AddKey adds a key to the builder's dictionary and returns its ID.
// If the key already exists in the dictionary, its existing ID is returned.
func ( *Builder) ( string) ( uint32) {
	if .dict == nil {
		.dict = make(map[string]uint32)
		.dictKeys = make([][]byte, 0, 16)
	}

	var  bool
	if ,  = .dict[];  {
		return 
	}

	 = uint32(len(.dictKeys))
	.dict[] = 
	.dictKeys = append(.dictKeys, unsafe.Slice(unsafe.StringData(), len()))
	.totalDictSize += len()

	return 
}

// AppendOpt represents options for appending time-related values. These are only
// used when using the generic Append method that takes an interface{}.
type AppendOpt int16

const (
	// OptTimestampNano specifies that timestamps should use nanosecond precision,
	// otherwise microsecond precision is used.
	OptTimestampNano AppendOpt = 1 << iota
	// OptTimestampUTC specifies that timestamps should be in UTC timezone, otherwise
	// no time zone (NTZ) is used.
	OptTimestampUTC
	// OptTimeAsDate specifies that time.Time values should be encoded as dates
	OptTimeAsDate
	// OptTimeAsTime specifies that time.Time values should be encoded as a time value
	OptTimeAsTime
)

func extractFieldInfo( reflect.StructField) ( string,  AppendOpt) {
	 := .Tag.Get("variant")
	if  == "" {
		return .Name, 0
	}

	 := strings.Split(, ",")
	if len() == 1 {
		return [0], 0
	}

	 = [0]
	if  == "" {
		 = .Name
	}

	for ,  := range [1:] {
		switch strings.ToLower() {
		case "nanos":
			 |= OptTimestampNano
		case "utc":
			 |= OptTimestampUTC
		case "date":
			 |= OptTimeAsDate
		case "time":
			 |= OptTimeAsTime
		}
	}

	return , 
}

// Append adds a value of any supported type to the builder.
//
// Any basic primitive type is supported, the AppendOpt options are used to control how
// timestamps are appended (e.g., as microseconds or nanoseconds and timezone). The options
// also control how a [time.Time] value is appended (e.g., as a date, timestamp, or time).
//
// Appending a value with type `[]any` will construct an array appropriately, appending
// each element. Calling with a map[string]any will construct an object, recursively calling
// Append for each value, propagating the options.
//
// For other types (arbitrary slices, arrays, maps and structs), reflection is used to determine
// the type and whether we can append it. A nil pointer will append a null, while a non-nil
// pointer will append the value that it points to.
//
// For structs, field tags can be used to control the field names and options. Only exported
// fields are considered, with the field name being used as the key. A struct tag of `variant`
// can be used with the following format and options:
//
//			type MyStruct struct {
//				Field1    string    `variant:"key"`           // Use "key" instead of "Field1" as the field name
//				Field2    time.Time `variant:"day,date"`      // Use "day" instead of "Field2" as the field name
//		                                                      // append this value as a "date" value
//	         Time      time.Time `variant:",time"`         // Use "Time" as the field name, append the value as
//		                                                      // a "time" value
//	         Field3    int       `variant:"-"`             // Ignore this field
//	         Timestamp time.Time `variant:"ts"`            // Use "ts" as the field name, append value as a
//		                                                      // timestamp(UTC=false,MICROS)
//			    Ts2       time.Time `variant:"ts2,nanos,utc"` // Use "ts2" as the field name, append value as a
//															  // timestamp(UTC=true,NANOS)
//			}
//
// There is only one case where options can conflict currently: If both [OptTimeAsDate] and
// [OptTimeAsTime] are set, then [OptTimeAsDate] will take precedence.
//
// Options specified in the struct tags will be OR'd with any options passed to the original call
// to Append. As a result, if a Struct field tag sets [OptTimeAsTime], but the call to Append
// passes [OptTimeAsDate], then the value will be appended as a date since that option takes
// precedence.
func ( *Builder) ( any,  ...AppendOpt) error {
	var  AppendOpt
	for ,  := range  {
		 |= 
	}

	return .append(, )
}

func ( *Builder) ( any,  AppendOpt) error {
	switch v := .(type) {
	case nil:
		return .AppendNull()
	case bool:
		return .AppendBool()
	case int8:
		return .AppendInt(int64())
	case uint8:
		return .AppendInt(int64())
	case int16:
		return .AppendInt(int64())
	case uint16:
		return .AppendInt(int64())
	case int32:
		return .AppendInt(int64())
	case uint32:
		return .AppendInt(int64())
	case int64:
		return .AppendInt()
	case int:
		return .AppendInt(int64())
	case uint:
		return .AppendInt(int64())
	case float32:
		return .AppendFloat32()
	case float64:
		return .AppendFloat64()
	case arrow.Date32:
		return .AppendDate()
	case arrow.Time64:
		return .AppendTimeMicro()
	case arrow.Timestamp:
		return .AppendTimestamp(, &OptTimestampNano == 0, &OptTimestampUTC != 0)
	case []byte:
		return .AppendBinary()
	case string:
		return .AppendString()
	case uuid.UUID:
		return .AppendUUID()
	case time.Time:
		switch {
		case &OptTimeAsDate != 0:
			return .AppendDate(arrow.Date32FromTime())
		case &OptTimeAsTime != 0:
			 := .Sub(.Truncate(24 * time.Hour))
			return .AppendTimeMicro(arrow.Time64(.Microseconds()))
		default:
			 := arrow.Microsecond
			if &OptTimestampNano != 0 {
				 = arrow.Nanosecond
			}

			if &OptTimestampUTC != 0 {
				 = .UTC()
			}

			,  := arrow.TimestampFromTime(, )
			if  != nil {
				return 
			}

			return .AppendTimestamp(, &OptTimestampNano == 0, &OptTimestampUTC != 0)
		}
	case DecimalValue[decimal.Decimal32]:
		return .AppendDecimal4(.Scale, .Value.(decimal.Decimal32))
	case DecimalValue[decimal.Decimal64]:
		return .AppendDecimal8(.Scale, .Value.(decimal.Decimal64))
	case DecimalValue[decimal.Decimal128]:
		return .AppendDecimal16(.Scale, .Value.(decimal.Decimal128))
	case []any:
		,  := .Offset(), make([]int, 0, len())
		for ,  := range  {
			 = append(, .NextElement())
			if  := .(, );  != nil {
				return 
			}
		}
		return .FinishArray(, )
	case map[string]any:
		,  := .Offset(), make([]FieldEntry, 0, len())
		for ,  := range  {
			 = append(, .NextField(, ))
			if  := .(, );  != nil {
				return 
			}
		}
		return .FinishObject(, )
	default:
		// attempt to use reflection before we give up!
		 := reflect.ValueOf()
		switch .Kind() {
		case reflect.Pointer, reflect.Interface:
			if .IsNil() {
				return .AppendNull()
			}
			return .(.Elem().Interface(), )
		case reflect.Array, reflect.Slice:
			,  := .Offset(), make([]int, 0, .Len())
			for ,  := range .Seq2() {
				 = append(, .NextElement())
				if  := .(.Interface(), );  != nil {
					return 
				}
			}
			return .FinishArray(, )
		case reflect.Map:
			if .Type().Key().Kind() != reflect.String {
				return fmt.Errorf("unsupported map key type: %s", .Type().Key())
			}

			,  := .Offset(), make([]FieldEntry, 0, .Len())
			for ,  := range .Seq2() {
				 = append(, .NextField(, .String()))
				if  := .(.Interface(), );  != nil {
					return 
				}
			}
			return .FinishObject(, )
		case reflect.Struct:
			,  := .Offset(), make([]FieldEntry, 0, .NumField())

			 := .Type()
			for  := range .NumField() {
				 := .Field()
				if !.IsExported() {
					continue
				}

				,  := extractFieldInfo()
				if  == "-" {
					continue
				}

				 = append(, .NextField(, ))
				if  := .(.Field().Interface(), |);  != nil {
					return 
				}
			}
			return .FinishObject(, )
		}
	}
	return fmt.Errorf("cannot append unsupported type to variant: %T", )
}

// AppendNull appends a null value to the builder.
func ( *Builder) () error {
	return .buf.WriteByte(primitiveHeader(PrimitiveNull))
}

// AppendBool appends a boolean value to the builder.
func ( *Builder) ( bool) error {
	var  PrimitiveType
	if  {
		 = PrimitiveBoolTrue
	} else {
		 = PrimitiveBoolFalse
	}

	return .buf.WriteByte(primitiveHeader())
}

type primitiveNumeric interface {
	int8 | int16 | int32 | int64 | float32 | float64 |
		arrow.Date32 | arrow.Time64
}

type buffer interface {
	io.Writer
	io.ByteWriter
}

func writeBinary[ string | []byte]( buffer,  ) error {
	var  PrimitiveType
	switch any().(type) {
	case string:
		 = PrimitiveString
	case []byte:
		 = PrimitiveBinary
	}

	if  := .WriteByte(primitiveHeader());  != nil {
		return 
	}

	if  := binary.Write(, binary.LittleEndian, uint32(len()));  != nil {
		return 
	}

	,  := .Write([]byte())
	return 
}

func writeNumeric[ primitiveNumeric]( buffer,  ) error {
	var  PrimitiveType
	switch any().(type) {
	case int8:
		 = PrimitiveInt8
	case int16:
		 = PrimitiveInt16
	case int32:
		 = PrimitiveInt32
	case int64:
		 = PrimitiveInt64
	case float32:
		 = PrimitiveFloat
	case float64:
		 = PrimitiveDouble
	case arrow.Date32:
		 = PrimitiveDate
	case arrow.Time64:
		 = PrimitiveTimeMicrosNTZ
	}

	if  := .WriteByte(primitiveHeader());  != nil {
		return 
	}

	return binary.Write(, binary.LittleEndian, )
}

// AppendInt appends an integer value to the builder, using the smallest
// possible integer representation based on the value's range.
func ( *Builder) ( int64) error {
	.buf.Grow(9)
	switch {
	case  >= math.MinInt8 &&  <= math.MaxInt8:
		return writeNumeric(&.buf, int8())
	case  >= math.MinInt16 &&  <= math.MaxInt16:
		return writeNumeric(&.buf, int16())
	case  >= math.MinInt32 &&  <= math.MaxInt32:
		return writeNumeric(&.buf, int32())
	default:
		return writeNumeric(&.buf, )
	}
}

// AppendFloat32 appends a 32-bit floating point value to the builder.
func ( *Builder) ( float32) error {
	.buf.Grow(5)
	return writeNumeric(&.buf, )
}

// AppendFloat64 appends a 64-bit floating point value to the builder.
func ( *Builder) ( float64) error {
	.buf.Grow(9)
	return writeNumeric(&.buf, )
}

// AppendDate appends a date value to the builder.
func ( *Builder) ( arrow.Date32) error {
	.buf.Grow(5)
	return writeNumeric(&.buf, )
}

// AppendTimeMicro appends a time value with microsecond precision to the builder.
func ( *Builder) ( arrow.Time64) error {
	.buf.Grow(9)
	return writeNumeric(&.buf, )
}

// AppendTimestamp appends a timestamp value to the builder.
// The useMicros parameter controls whether microsecond or nanosecond precision is used.
// The useUTC parameter controls whether the timestamp is in UTC timezone or has no time zone (NTZ).
func ( *Builder) ( arrow.Timestamp, ,  bool) error {
	.buf.Grow(9)
	var  PrimitiveType
	if  {
		 = PrimitiveTimestampMicrosNTZ
	} else {
		 = PrimitiveTimestampNanosNTZ
	}

	if  {
		--
	}

	if  := .buf.WriteByte(primitiveHeader());  != nil {
		return 
	}

	return binary.Write(&.buf, binary.LittleEndian, )
}

// AppendBinary appends a binary value to the builder.
func ( *Builder) ( []byte) error {
	.buf.Grow(5 + len())
	return writeBinary(&.buf, )
}

// AppendString appends a string value to the builder.
// Small strings are encoded using the short string representation if small enough.
func ( *Builder) ( string) error {
	if len() > maxShortStringSize {
		.buf.Grow(5 + len())
		return writeBinary(&.buf, )
	}

	.buf.Grow(1 + len())
	if  := .buf.WriteByte(shortStrHeader(len()));  != nil {
		return 
	}

	,  := .buf.WriteString()
	return 
}

// AppendUUID appends a UUID value to the builder.
func ( *Builder) ( uuid.UUID) error {
	.buf.Grow(17)
	if  := .buf.WriteByte(primitiveHeader(PrimitiveUUID));  != nil {
		return 
	}

	,  := .MarshalBinary()
	,  := .buf.Write()
	return 
}

// AppendDecimal4 appends a 4-byte decimal value with the specified scale to the builder.
func ( *Builder) ( uint8,  decimal.Decimal32) error {
	.buf.Grow(6)
	if  := .buf.WriteByte(primitiveHeader(PrimitiveDecimal4));  != nil {
		return 
	}

	if  := .buf.WriteByte();  != nil {
		return 
	}

	return binary.Write(&.buf, binary.LittleEndian, int32())
}

// AppendDecimal8 appends a 8-byte decimal value with the specified scale to the builder.
func ( *Builder) ( uint8,  decimal.Decimal64) error {
	.buf.Grow(10)
	return errors.Join(
		.buf.WriteByte(primitiveHeader(PrimitiveDecimal8)),
		.buf.WriteByte(),
		binary.Write(&.buf, binary.LittleEndian, int64()),
	)
}

// AppendDecimal16 appends a 16-byte decimal value with the specified scale to the builder.
func ( *Builder) ( uint8,  decimal.Decimal128) error {
	.buf.Grow(18)
	return errors.Join(
		.buf.WriteByte(primitiveHeader(PrimitiveDecimal16)),
		.buf.WriteByte(),
		binary.Write(&.buf, binary.LittleEndian, .LowBits()),
		binary.Write(&.buf, binary.LittleEndian, .HighBits()),
	)
}

// Offset returns the current offset in the builder's buffer. Generally used for
// grabbing a starting point for building an array or object.
func ( *Builder) () int {
	return .buf.Len()
}

// NextElement returns the offset of the next element relative to the start position.
// Use when building arrays to track element positions. The following creates a variant
// equivalent to `[5, 10]`.
//
//	var b variant.Builder
//	start, offsets := b.Offset(), make([]int, 0)
//	offsets = append(offsets, b.NextElement(start))
//	b.Append(5)
//	offsets = append(offsets, b.NextElement(start))
//	b.Append(10)
//	b.FinishArray(start, offsets)
//
// The value returned by this is equivalent to `b.Offset() - start`, as offsets are all
// relative to the start position. This allows for creating nested arrays, the following
// creates a variant equivalent to `[5, [10, 20], 30]`.
//
//	var b variant.Builder
//	start, offsets := b.Offset(), make([]int, 0)
//	offsets = append(offsets, b.NextElement(start))
//	b.Append(5)
//	offsets = append(offsets, b.NextElement(start))
//
//	nestedStart, nestedOffsets := b.Offset(), make([]int, 0)
//	nestedOffsets = append(nestedOffsets, b.NextElement(nestedStart))
//	b.Append(10)
//	nestedOffsets = append(nestedOffsets, b.NextElement(nestedStart))
//	b.Append(20)
//	b.FinishArray(nestedStart, nestedOffsets)
//
//	offsets = append(offsets, b.NextElement(start))
//	b.Append(30)
//	b.FinishArray(start, offsets)
func ( *Builder) ( int) int {
	return .Offset() - 
}

// FinishArray finalizes an array value in the builder.
// The start parameter is the offset where the array begins.
// The offsets parameter contains the offsets of each element in the array. See [Builder.NextElement]
// for examples of how to use this.
func ( *Builder) ( int,  []int) error {
	var (
		,  = .buf.Len() - , len()
		      =  > math.MaxUint8
		    = 1
	)

	if  {
		 = 4
	}

	if  < 0 {
		return errors.New("invalid array size")
	}

	 := intSize()
	 := 1 +  + (+1)*int()

	// shift the just written data to make room for the header section
	.buf.Grow()
	 := .buf.AvailableBuffer()
	if ,  := .buf.Write([:]);  != nil {
		return 
	}

	 := .buf.Bytes()
	copy([+:], [:+])

	// populate the header
	[] = arrayHeader(, )
	writeOffset([+1:], , uint8())

	 :=  + 1 + 
	for ,  := range  {
		writeOffset([+*int():], , )
	}
	writeOffset([+*int():], , )

	return nil
}

// FieldEntry represents a field in an object, with its key, ID, and offset.
// Usually constructed by using [Builder.NextField] and then passed to [Builder.FinishObject].
type FieldEntry struct {
	Key    string
	ID     uint32
	Offset int
}

// NextField creates a new field entry for an object with the given key.
// The start parameter is the offset where the object begins. The following example would
// construct a variant equivalent to `{"key1": 5, "key2": 10}`.
//
//	var b variant.Builder
//	start, fields := b.Offset(), make([]variant.FieldEntry, 0)
//	fields = append(fields, b.NextField(start, "key1"))
//	b.Append(5)
//	fields = append(fields, b.NextField(start, "key2"))
//	b.Append(10)
//	b.FinishObject(start, fields)
//
// This allows for creating nested objects, the following example would create a variant
// equivalent to `{"key1": 5, "key2": {"key3": 10, "key4": 20}, "key5": 30}`.
//
//	var b variant.Builder
//	start, fields := b.Offset(), make([]variant.FieldEntry, 0)
//	fields = append(fields, b.NextField(start, "key1"))
//	b.Append(5)
//	fields = append(fields, b.NextField(start, "key2"))
//	nestedStart, nestedFields := b.Offset(), make([]variant.FieldEntry, 0)
//	nestedFields = append(nestedFields, b.NextField(nestedStart, "key3"))
//	b.Append(10)
//	nestedFields = append(nestedFields, b.NextField(nestedStart, "key4"))
//	b.Append(20)
//	b.FinishObject(nestedStart, nestedFields)
//	fields = append(fields, b.NextField(start, "key5"))
//	b.Append(30)
//	b.FinishObject(start, fields)
//
// The offset value returned by this is equivalent to `b.Offset() - start`, as offsets are all
// relative to the start position. The key provided will be passed to the [Builder.AddKey] method
// to ensure that the key is added to the dictionary and an ID is assigned. It will re-use existing
// IDs if the key already exists in the dictionary.
func ( *Builder) ( int,  string) FieldEntry {
	 := .AddKey()
	return FieldEntry{
		Key:    ,
		ID:     ,
		Offset: .Offset() - ,
	}
}

// FinishObject finalizes an object value in the builder.
// The start parameter is the offset where the object begins.
// The fields parameter contains the entries for each field in the object. See [Builder.NextField]
// for examples of how to use this.
//
// The fields are sorted by key before finalizing the object. If duplicate keys are found,
// the last value for a key is kept if [Builder.SetAllowDuplicates] is set to true. If false,
// an error is returned.
func ( *Builder) ( int,  []FieldEntry) error {
	slices.SortFunc(, func(,  FieldEntry) int {
		return cmp.Compare(.Key, .Key)
	})

	 := len()
	var  uint32
	if  > 0 {
		 = [0].ID
	}

	// if a duplicate key is found, one of two things happens:
	// - if allowDuplicates is true, then the field with the greatest
	//    offset value (the last appended field) is kept.
	// - if allowDuplicates is false, then an error is returned
	if .allowDuplicates {
		 := 0
		// maintain a list of distinct keys in-place
		for  := 1;  < ; ++ {
			 = max(, [].ID)
			if [].ID == [-1].ID {
				// found a duplicate key. keep the
				// field with a greater offset
				if [].Offset < [].Offset {
					[].Offset = [].Offset
				}
			} else {
				// found distinct key, add field to the list
				++
				[] = []
			}
		}

		if +1 < len() {
			 =  + 1
			// resize fields to size
			 = [:]
			// sort the fields by offsets so that we can move the value
			// data of each field to the new offset without overwriting the
			// fields after it.
			slices.SortFunc(, func(,  FieldEntry) int {
				return cmp.Compare(.Offset, .Offset)
			})

			 := .buf.Bytes()
			 := 0
			for  := range  {
				 := [].Offset
				 := valueSize([+:])
				copy([+:], [+:++])
				[].Offset = 
				 += 
			}
			.buf.Truncate( + )
			// change back to sort order by field keys to meet variant spec
			slices.SortFunc(, func(,  FieldEntry) int {
				return cmp.Compare(.Key, .Key)
			})
		}
	} else {
		for  := 1;  < ; ++ {
			 = max(, [].ID)
			if [].Key == [-1].Key {
				return fmt.Errorf("disallowed duplicate key found: %s", [].Key)
			}
		}
	}

	var (
		  = .buf.Len() - 
		   =  > math.MaxUint8
		 = 1
	)

	if  {
		 = 4
	}

	if  < 0 {
		return errors.New("invalid object size")
	}

	,  := intSize(int()), intSize()
	 := 1 +  + *int() + (+1)*int()
	// shift the just written data to make room for the header section
	.buf.Grow()
	 := .buf.AvailableBuffer()
	if ,  := .buf.Write([:]);  != nil {
		return 
	}

	 := .buf.Bytes()
	copy([+:], [:+])

	// populate the header
	[] = objectHeader(, , )
	writeOffset([+1:], , uint8())

	 :=  + 1 + 
	 :=  + *int()
	for ,  := range  {
		writeOffset([+*int():], int(.ID), )
		writeOffset([+*int():], .Offset, )
	}
	writeOffset([+*int():], , )
	return nil
}

// UnsafeAppendEncoded is a special case where we directly append a pre-encoded variant
// value. Its keys must already be in the dictionary and v must already be
// a properly encoded variant value. No checking is performed here currently, so
// be careful as this can easily lead to an invalid variant result.
func ( *Builder) ( []byte) error {
	// this is a special case where we append a pre-encoded value.
	// the value must be a valid variant value, so it must start with
	// a primitive header byte.
	,  := .buf.Write()
	return 
}

// Reset truncates the builder's buffer and clears the dictionary while re-using the
// underlying storage where possible. This allows for reusing the builder while keeping
// the total memory usage low. The caveat to this is that any variant value returned
// by calling [Builder.Build] must be cloned with [Value.Clone] before calling this
// method. Otherwise, the byte slice used by the value will be invalidated upon calling
// this method.
//
// For trivial cases where the builder is not reused, this method never needs to be called,
// and the variant built by the builder gets to avoid having to copy the buffer, just referring
// to it directly.
func ( *Builder) () {
	.buf.Reset()
	.dict = make(map[string]uint32)
	for  := range .dictKeys {
		.dictKeys[] = nil
	}
	.dictKeys = .dictKeys[:0]
}

// BuildWithoutMeta returns just the raw variant bytes that were built without
// constructing metadata at all. This is useful for the case where we're building
// the remainder of a shredded variant and don't need to re-construct the metadata
// for the result.
func ( *Builder) () []byte {
	return .buf.Bytes()
}

// Build creates a Variant Value from the builder's current state.
// The returned Value includes both the value data and the metadata (dictionary).
//
// Importantly, the value data is the returned variant value is not copied here. This will
// return the raw buffer data owned by the builder's buffer. If you wish to reuse a builder,
// then the [Value.Clone] method must be called on the returned value to copy the data before
// calling [Builder.Reset]. This enables trivial cases that don't reuse the builder to avoid
// performing this copy.
func ( *Builder) () (Value, error) {
	 := len(.dictKeys)

	// determine the number of bytes required per offset entry.
	// the largest offset is the one-past-the-end value, the total size.
	// It's very unlikely that the number of keys could be larger, but
	// incorporate that into the calculation in case of pathological data.
	 := max(.totalDictSize, )
	if  > metadataMaxSizeLimit {
		return Value{}, fmt.Errorf("metadata size too large: %d", )
	}

	 := intSize(int())
	 := 1 + 
	 := int() + (+1)*int()
	 :=  + .totalDictSize

	if  > metadataMaxSizeLimit {
		return Value{}, fmt.Errorf("metadata size too large: %d", )
	}

	 := make([]byte, )

	[0] = supportedVersion | (( - 1) << 6)
	if  > 0 && slices.IsSortedFunc(.dictKeys, bytes.Compare) {
		[0] |= 1 << 4
	}
	writeOffset([1:], , )

	 := 0
	for ,  := range .dictKeys {
		writeOffset([int()+*int():], , )
		 += copy([+:], )
	}
	writeOffset([int()+*int():], , )

	return Value{
		value: .buf.Bytes(),
		meta: Metadata{
			data: ,
			keys: .dictKeys,
		},
	}, nil
}

type variantPrimitiveType interface {
	constraints.Integer | constraints.Float | string | []byte |
		arrow.Date32 | arrow.Time64 | arrow.Timestamp | bool |
		uuid.UUID | DecimalValue[decimal.Decimal32] | time.Time |
		DecimalValue[decimal.Decimal64] | DecimalValue[decimal.Decimal128]
}

// Encode is a convenience function that produces the encoded bytes for a primitive
// variant value. At the moment this is just delegating to the [Builder.Append] method,
// but in the future it will be optimized to avoid the extra overhead and reduce allocations.
func [ variantPrimitiveType]( ,  ...AppendOpt) ([]byte, error) {
	,  := Of(, ...)
	if  != nil {
		return nil, fmt.Errorf("failed to encode variant value: %w", )
	}
	return .value, nil
}

func [ variantPrimitiveType]( ,  ...AppendOpt) (Value, error) {
	var  Builder
	if  := .Append(, ...);  != nil {
		return Value{}, fmt.Errorf("failed to append value: %w", )
	}

	,  := .Build()
	if  != nil {
		return Value{}, fmt.Errorf("failed to build variant value: %w", )
	}

	return , nil
}

func ( string,  bool) (Value, error) {
	var  Builder
	.SetAllowDuplicates()

	 := json.NewDecoder(strings.NewReader())
	.UseNumber() // to handle JSON numbers as json.Number

	if  := .buildJSON();  != nil {
		return Value{}, fmt.Errorf("failed to parse JSON: %w", )
	}

	return .Build()
}

func ( []byte,  bool) (Value, error) {
	var  Builder
	.SetAllowDuplicates()

	 := json.NewDecoder(bytes.NewReader())
	.UseNumber() // to handle JSON numbers as json.Number

	if  := .buildJSON();  != nil {
		return Value{}, fmt.Errorf("failed to parse JSON: %w", )
	}

	return .Build()
}

func ( *json.Decoder,  bool) (Value, error) {
	var  Builder
	.SetAllowDuplicates()

	if  := .buildJSON();  != nil {
		return Value{}, fmt.Errorf("failed to unmarshal JSON: %w", )
	}

	return .Build()
}

func ( *Builder) ( *json.Decoder) error {
	,  := .Token()
	if  != nil {
		if errors.Is(, io.EOF) {
			return fmt.Errorf("unexpected end of JSON input")
		}
		return fmt.Errorf("failed to decode JSON token: %w", )
	}

	switch v := .(type) {
	case json.Delim:
		switch  {
		case '{':
			,  := .Offset(), make([]FieldEntry, 0)
			for .More() {
				,  := .Token()
				if  != nil {
					if errors.Is(, io.EOF) {
						return fmt.Errorf("unexpected end of JSON input")
					}
					return fmt.Errorf("failed to decode JSON key: %w", )
				}

				switch key := .(type) {
				case string:
					 = append(, .NextField(, ))
					if  := .();  != nil {
						return 
					}
				default:
					return fmt.Errorf("expected string key in JSON object, got %T", )
				}
			}
			,  = .Token()
			if  != nil {
				return fmt.Errorf("failed to decode JSON object end: %w", )
			}
			if  != json.Delim('}') {
				return fmt.Errorf("expected end of JSON object, got %v", )
			}
			return .FinishObject(, )
		case '[':
			,  := .Offset(), make([]int, 0)
			for .More() {
				 = append(, .NextElement())
				if  := .();  != nil {
					return 
				}
			}
			,  = .Token()
			if  != nil {
				return fmt.Errorf("failed to decode JSON array end: %w", )
			}
			if  != json.Delim(']') {
				return fmt.Errorf("expected end of JSON array, got %v", )
			}
			return .FinishArray(, )
		default:
			return fmt.Errorf("unexpected JSON delimiter: %v", )
		}
	case float64:
		return .AppendFloat64()
	case string:
		return .AppendString()
	case bool:
		return .AppendBool()
	case nil:
		return .AppendNull()
	case json.Number:
		,  := .Int64()
		if  == nil {
			return .AppendInt()
		}

		if !.tryParseDecimal(.String()) {
			,  := .Float64()
			if  == nil {
				return .AppendFloat64()
			}
			return fmt.Errorf("failed to parse JSON number: %w", )
		}

		return nil
	default:
		return fmt.Errorf("unexpected JSON token type: %T", )
	}
}

func ( *Builder) ( string) bool {
	, ,  := decimal.PrecScaleFromString()
	if  != nil {
		return false
	}

	,  := decimal.Decimal128FromString(, , )
	if  != nil {
		return false
	}

	return .AppendDecimal16(uint8(), ) == nil
}