variant

package
v18.5.0 Latest Latest
Warning

This package is not in the latest version of its module.

Go to latest
Published: Dec 9, 2025 License: Apache-2.0, BSD-3-Clause Imports: 24 Imported by: 0

Documentation

Overview

Package variant provides an implementation of the Apache Parquet Variant data type.

The Variant type is a flexible binary format designed to represent complex nested data structures with minimal overhead. It supports a wide range of primitive types as well as nested arrays and objects (similar to JSON). The format uses a memory-efficient binary representation with a separate metadata section for dictionary encoding of keys.

Key Components

- Value: The primary type representing a variant value - Metadata: Contains information about the dictionary of keys - Builder: Used to construct variant values

Format Overview

The variant format consists of two parts:

  1. Metadata: A dictionary of keys used in objects
  2. Value: The actual data payload

Values can be one of the following types:

  • Primitive values (null, bool, int8/16/32/64, float32/64, etc.)
  • Short strings (less than 64 bytes)
  • Long strings and binary data
  • Date, time and timestamp values
  • Decimal values (4, 8, or 16 bytes)
  • Arrays of any variant value
  • Objects with key-value pairs

Working with Variants

To create a variant value, use the Builder:

var b variant.Builder
b.Append(map[string]any{
    "id": 123,
    "name": "example",
    "data": []any{1, 2, 3},
})
value, err := b.Build()

To parse an existing variant value:

v, err := variant.New(metadataBytes, valueBytes)

You can access the data using the Value.Value method which returns the appropriate Go type:

switch v.Type() {
case variant.Object:
    obj := v.Value().(variant.ObjectValue)
    field, err := obj.ValueByKey("name")
case variant.Array:
    arr := v.Value().(variant.ArrayValue)
    elem, err := arr.Value(0)
case variant.String:
    s := v.Value().(string)
case variant.Int64:
    i := v.Value().(int64)
}

You can also switch on the type of the result value from the Value.Value method:

switch val := v.Value().(type) {
case nil:
  // ...
case int32:
  // ...
case string:
  // ...
case variant.ArrayValue:
  for i, item := range val.Values() {
    // item is a variant.Value
  }
case variant.ObjectValue:
  for k, item := range val.Values() {
    // k is the field key
    // item is a variant.Value for that field
  }
}

Values can also be converted to JSON:

jsonBytes, err := json.Marshal(v)

Low-level Construction

For direct construction of complex nested structures, you can use the low-level methods:

var b variant.Builder
// Start an object
start := b.Offset()
fields := make([]variant.FieldEntry, 0)

// Add a field
fields = append(fields, b.NextField(start, "key"))
b.AppendString("value")

// Finish the object
b.FinishObject(start, fields)

value, err := b.Build()

Using Struct Tags

When appending Go structs, you can use struct tags to control field names and encoding options:

type Person struct {
    ID        int       `variant:"id"`
    Name      string    `variant:"name"`
    CreatedAt time.Time `variant:"timestamp,nanos,utc"`
    Internal  string    `variant:"-"` // Ignored field
}

Reusing Builders

When reusing a Builder for multiple values, use Reset() to clear it:

var b variant.Builder
v1, _ := b.Append(data1).Build()
v1 = v1.Clone() // Clone before reset if you need to keep the value
b.Reset()
v2, _ := b.Append(data2).Build()

Index

Constants

This section is empty.

Variables

View Source
var (
	// EmptyMetadataBytes contains a minimal valid metadata section with no dictionary entries.
	EmptyMetadataBytes = [3]byte{0x1, 0, 0}

	ErrInvalidMetadata = errors.New("invalid variant metadata")
)
View Source
var NullValue = Value{/* contains filtered or unexported fields */}

Functions

func Encode

func Encode[T variantPrimitiveType](v T, opt ...AppendOpt) ([]byte, error)

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.

Types

type AppendOpt

type AppendOpt int16

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

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
)

type ArrayValue

type ArrayValue struct {
	// contains filtered or unexported fields
}

ArrayValue represents an array of variant values.

func (ArrayValue) Len

func (v ArrayValue) Len() uint32

Len returns the number of elements in the array.

func (ArrayValue) MarshalJSON

func (v ArrayValue) MarshalJSON() ([]byte, error)

MarshalJSON implements the json.Marshaler interface for ArrayValue.

func (ArrayValue) Value

func (v ArrayValue) Value(i uint32) (Value, error)

Value returns the Value at the specified index. Returns an error if the index is out of range.

func (ArrayValue) Values

func (v ArrayValue) Values() iter.Seq[Value]

Values returns an iterator for the elements in the array, allowing for lazy evaluation of the offsets (for the situation where not all elements are iterated).

type BasicType

type BasicType int

BasicType represents the fundamental type category of a variant value.

const (
	BasicUndefined   BasicType = iota - 1 // Unknown
	BasicPrimitive                        // Primitive
	BasicShortString                      // ShortString
	BasicObject                           // Object
	BasicArray                            // Array
)

func (BasicType) String

func (i BasicType) String() string

type Builder

type Builder struct {
	// contains filtered or unexported fields
}

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.

func NewBuilderFromMeta

func NewBuilderFromMeta(m Metadata) *Builder

func (*Builder) AddKey

func (b *Builder) AddKey(key string) (id uint32)

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) Append

func (b *Builder) Append(v any, opts ...AppendOpt) error

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) AppendBinary

func (b *Builder) AppendBinary(v []byte) error

AppendBinary appends a binary value to the builder.

func (*Builder) AppendBool

func (b *Builder) AppendBool(v bool) error

AppendBool appends a boolean value to the builder.

func (*Builder) AppendDate

func (b *Builder) AppendDate(v arrow.Date32) error

AppendDate appends a date value to the builder.

func (*Builder) AppendDecimal16

func (b *Builder) AppendDecimal16(scale uint8, v decimal.Decimal128) error

AppendDecimal16 appends a 16-byte decimal value with the specified scale to the builder.

func (*Builder) AppendDecimal4

func (b *Builder) AppendDecimal4(scale uint8, v decimal.Decimal32) error

AppendDecimal4 appends a 4-byte decimal value with the specified scale to the builder.

func (*Builder) AppendDecimal8

func (b *Builder) AppendDecimal8(scale uint8, v decimal.Decimal64) error

AppendDecimal8 appends a 8-byte decimal value with the specified scale to the builder.

func (*Builder) AppendFloat32

func (b *Builder) AppendFloat32(v float32) error

AppendFloat32 appends a 32-bit floating point value to the builder.

func (*Builder) AppendFloat64

func (b *Builder) AppendFloat64(v float64) error

AppendFloat64 appends a 64-bit floating point value to the builder.

func (*Builder) AppendInt

func (b *Builder) AppendInt(v int64) error

AppendInt appends an integer value to the builder, using the smallest possible integer representation based on the value's range.

func (*Builder) AppendNull

func (b *Builder) AppendNull() error

AppendNull appends a null value to the builder.

func (*Builder) AppendString

func (b *Builder) AppendString(v string) error

AppendString appends a string value to the builder. Small strings are encoded using the short string representation if small enough.

func (*Builder) AppendTimeMicro

func (b *Builder) AppendTimeMicro(v arrow.Time64) error

AppendTimeMicro appends a time value with microsecond precision to the builder.

func (*Builder) AppendTimestamp

func (b *Builder) AppendTimestamp(v arrow.Timestamp, useMicros, useUTC bool) error

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) AppendUUID

func (b *Builder) AppendUUID(v uuid.UUID) error

AppendUUID appends a UUID value to the builder.

func (*Builder) Build

func (b *Builder) Build() (Value, error)

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) BuildWithoutMeta

func (b *Builder) BuildWithoutMeta() []byte

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) FinishArray

func (b *Builder) FinishArray(start int, offsets []int) error

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) FinishObject

func (b *Builder) FinishObject(start int, fields []FieldEntry) error

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) NextElement

func (b *Builder) NextElement(start int) int

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) NextField

func (b *Builder) NextField(start int, key string) FieldEntry

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) Offset

func (b *Builder) Offset() int

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) Reset

func (b *Builder) Reset()

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) SetAllowDuplicates

func (b *Builder) SetAllowDuplicates(allow bool)

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) UnsafeAppendEncoded

func (b *Builder) UnsafeAppendEncoded(v []byte) error

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.

type DecimalValue

type DecimalValue[T decimal.DecimalTypes] struct {
	Scale uint8
	Value decimal.Num[T]
}

DecimalValue represents a decimal number with a specified scale. The generic parameter T can be any supported variant decimal type (Decimal32, Decimal64, Decimal128).

func (DecimalValue[T]) MarshalJSON

func (v DecimalValue[T]) MarshalJSON() ([]byte, error)

MarshalJSON implements the json.Marshaler interface for DecimalValue.

type FieldEntry

type FieldEntry struct {
	Key    string
	ID     uint32
	Offset int
}

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 Metadata

type Metadata struct {
	// contains filtered or unexported fields
}

Metadata represents the dictionary part of a variant value, which stores the keys used in object values.

func NewMetadata

func NewMetadata(data []byte) (Metadata, error)

NewMetadata creates a Metadata instance from a raw byte slice. It validates the metadata format and loads the key dictionary.

func (Metadata) Bytes

func (m Metadata) Bytes() []byte

Bytes returns the raw byte representation of the metadata.

func (*Metadata) Clone

func (m *Metadata) Clone() Metadata

Clone creates a deep copy of the metadata.

func (Metadata) DictionarySize

func (m Metadata) DictionarySize() uint32

DictionarySize returns the number of keys in the metadata dictionary.

func (Metadata) IdFor

func (m Metadata) IdFor(key string) []uint32

IdFor returns the dictionary IDs for the given key. If the metadata is sorted and unique, this performs a binary search. Otherwise, it performs a linear search.

If the metadata is not sorted and unique, then it's possible that multiple IDs will be returned for the same key.

func (Metadata) KeyAt

func (m Metadata) KeyAt(id uint32) (string, error)

KeyAt returns the string key at the given dictionary ID. Returns an error if the ID is out of range.

func (Metadata) OffsetSize

func (m Metadata) OffsetSize() uint8

OffsetSize returns the size in bytes used to store offsets in the metadata.

func (Metadata) SortedAndUnique

func (m Metadata) SortedAndUnique() bool

SortedAndUnique returns whether the keys in the metadata dictionary are sorted and unique.

func (Metadata) Version

func (m Metadata) Version() uint8

Version returns the metadata format version.

type ObjectField

type ObjectField struct {
	Key   string
	Value Value
}

ObjectField represents a key-value pair in an object.

type ObjectValue

type ObjectValue struct {
	// contains filtered or unexported fields
}

ObjectValue represents an object (map/dictionary) of key-value pairs.

func (ObjectValue) FieldAt

func (v ObjectValue) FieldAt(i uint32) (ObjectField, error)

FieldAt returns the field at the specified index. Returns an error if the index is out of range.

func (ObjectValue) MarshalJSON

func (v ObjectValue) MarshalJSON() ([]byte, error)

MarshalJSON implements the json.Marshaler interface for ObjectValue.

func (ObjectValue) NumElements

func (v ObjectValue) NumElements() uint32

NumElements returns the number of fields in the object.

func (ObjectValue) ValueByKey

func (v ObjectValue) ValueByKey(key string) (ObjectField, error)

ValueByKey returns the field with the specified key. Returns arrow.ErrNotFound if the key doesn't exist.

func (ObjectValue) Values

func (v ObjectValue) Values() iter.Seq2[string, Value]

Values returns an iterator over all key-value pairs in the object.

type PrimitiveType

type PrimitiveType int

PrimitiveType represents specific primitive data types within the variant format.

const (
	PrimitiveInvalid            PrimitiveType = iota - 1 // Unknown
	PrimitiveNull                                        // Null
	PrimitiveBoolTrue                                    // BoolTrue
	PrimitiveBoolFalse                                   // BoolFalse
	PrimitiveInt8                                        // Int8
	PrimitiveInt16                                       // Int16
	PrimitiveInt32                                       // Int32
	PrimitiveInt64                                       // Int64
	PrimitiveDouble                                      // Double
	PrimitiveDecimal4                                    // Decimal32
	PrimitiveDecimal8                                    // Decimal64
	PrimitiveDecimal16                                   // Decimal128
	PrimitiveDate                                        // Date
	PrimitiveTimestampMicros                             // Timestamp(micros)
	PrimitiveTimestampMicrosNTZ                          // TimestampNTZ(micros)
	PrimitiveFloat                                       // Float
	PrimitiveBinary                                      // Binary
	PrimitiveString                                      // String
	PrimitiveTimeMicrosNTZ                               // TimeNTZ(micros)
	PrimitiveTimestampNanos                              // Timestamp(nanos)
	PrimitiveTimestampNanosNTZ                           // TimestampNTZ(nanos)
	PrimitiveUUID                                        // UUID
)

func (PrimitiveType) String

func (i PrimitiveType) String() string

type Type

type Type int

Type represents the high-level variant data type. This is what applications typically use to identify the type of a variant value.

const (
	Object Type = iota
	Array
	Null
	Bool
	Int8
	Int16
	Int32
	Int64
	String
	Double
	Decimal4
	Decimal8
	Decimal16
	Date
	TimestampMicros
	TimestampMicrosNTZ
	Float
	Binary
	Time
	TimestampNanos
	TimestampNanosNTZ
	UUID
)

type Value

type Value struct {
	// contains filtered or unexported fields
}

Value represents a variant value of any type.

func New

func New(meta, value []byte) (Value, error)

New creates a Value by parsing both the metadata and value bytes.

func NewWithMetadata

func NewWithMetadata(meta Metadata, value []byte) (Value, error)

NewWithMetadata creates a Value with the provided metadata and value bytes.

func Of added in v18.4.1

func Of[T variantPrimitiveType](v T, opt ...AppendOpt) (Value, error)

func ParseJSON

func ParseJSON(data string, allowDuplicateKeys bool) (Value, error)

func ParseJSONBytes

func ParseJSONBytes(data []byte, allowDuplicateKeys bool) (Value, error)

func Unmarshal

func Unmarshal(dec *json.Decoder, allowDuplicateKeys bool) (Value, error)

func (Value) BasicType

func (v Value) BasicType() BasicType

BasicType returns the fundamental type category of the value.

func (Value) Bytes

func (v Value) Bytes() []byte

Bytes returns the raw byte representation of the value (excluding metadata).

func (Value) Clone

func (v Value) Clone() Value

Clone creates a deep copy of the value including its metadata.

func (Value) MarshalJSON

func (v Value) MarshalJSON() ([]byte, error)

MarshalJSON implements the json.Marshaler interface for Value.

func (Value) Metadata

func (v Value) Metadata() Metadata

Metadata returns the metadata associated with the value.

func (Value) String

func (v Value) String() string

func (Value) Type

func (v Value) Type() Type

Type returns the specific data type of the value.

func (Value) Value

func (v Value) Value() any

Value returns the Go value representation of the variant. The returned type depends on the variant type:

  • Null: nil
  • Bool: bool
  • Int8/16/32/64: corresponding int type
  • Float/Double: float32/float64
  • String: string
  • Binary: []byte
  • Decimal: DecimalValue
  • Date: arrow.Date32
  • Time: arrow.Time64
  • Timestamp: arrow.Timestamp
  • UUID: uuid.UUID
  • Object: ObjectValue
  • Array: ArrayValue

Jump to

Keyboard shortcuts

? : This menu
/ : Search site
f or F : Jump to
y or Y : Canonical URL