iceberg

package module
v0.0.0-...-62e415d Latest Latest
Warning

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

Go to latest
Published: Dec 11, 2025 License: Apache-2.0 Imports: 35 Imported by: 0

README

Iceberg Golang

Go Reference

iceberg is a Golang implementation of the Iceberg table spec.

Build From Source

Prerequisites
  • Go 1.23 or later
Build
$ git clone https://github.com/apache/iceberg-go.git
$ cd iceberg-go/cmd/iceberg && go build .

Feature Support / Roadmap

FileSystem Support
Filesystem Type Supported
S3 X
Google Cloud Storage X
Azure Blob Storage X
Local Filesystem X
Metadata
Operation Supported
Get Schema X
Get Snapshots X
Get Sort Orders X
Get Partition Specs X
Get Manifests X
Create New Manifests X
Plan Scan x
Plan Scan for Snapshot x
Catalog Support
Operation REST Hive Glue SQL
Load Table X X X
List Tables X X X
Create Table X X X
Register Table X X
Update Current Snapshot X X X
Create New Snapshot X X X
Rename Table X X X
Drop Table X X X
Alter Table X X X
Check Table Exists X X X
Set Table Properties X X X
List Namespaces X X X
Create Namespace X X X
Check Namespace Exists X X X
Drop Namespace X X X
Update Namespace Properties X X X
Create View X X
Load View X
List View X X
Drop View X X
Check View Exists X X
Read/Write Data Support
  • Data can currently be read as an Arrow Table or as a stream of Arrow record batches.
Supported Write Operations

As long as the FileSystem is supported and the Catalog supports altering the table, the following tracks the current write support:

Operation Supported
Append Stream X
Append Data Files X
Rewrite Files
Rewrite manifests
Overwrite Files
Write Pos Delete
Write Eq Delete
Row Delta
CLI Usage

Run go build ./cmd/iceberg from the root of this repository to build the CLI executable, alternately you can run go install github.com/apache/iceberg-go/cmd/iceberg@latest to install it to the bin directory of your GOPATH.

The iceberg CLI usage is very similar to pyiceberg CLI
You can pass the catalog URI with --uri argument.

Example: You can start the Iceberg REST API docker image which runs on default in port 8181

docker pull apache/iceberg-rest-fixture:latest
docker run -p 8181:8181 apache/iceberg-rest-fixture:latest

and run the iceberg CLI pointing to the REST API server.

 ./iceberg --uri http://0.0.0.0:8181 list
┌─────┐
| IDs |
| --- |
└─────┘

Create Namespace

./iceberg --uri http://0.0.0.0:8181 create namespace taxitrips

List Namespace

 ./iceberg --uri http://0.0.0.0:8181 list
┌───────────┐
| IDs       |
| --------- |
| taxitrips |
└───────────┘


Get in Touch

Documentation

Index

Constants

View Source
const (
	PartitionDataIDStart   = 1000
	InitialPartitionSpecID = 0
)

Variables

View Source
var (
	ErrInvalidTypeString       = errors.New("invalid type")
	ErrNotImplemented          = errors.New("not implemented")
	ErrInvalidArgument         = errors.New("invalid argument")
	ErrInvalidFormatVersion    = fmt.Errorf("%w: invalid format version", ErrInvalidArgument)
	ErrInvalidSchema           = errors.New("invalid schema")
	ErrInvalidPartitionSpec    = errors.New("invalid partition spec")
	ErrInvalidTransform        = errors.New("invalid transform syntax")
	ErrType                    = errors.New("type error")
	ErrBadCast                 = errors.New("could not cast value")
	ErrBadLiteral              = errors.New("invalid literal value")
	ErrInvalidBinSerialization = errors.New("invalid binary serialization")
	ErrResolve                 = errors.New("cannot resolve type")
)
View Source
var PositionalDeleteSchema = NewSchema(0,
	NestedField{ID: 2147483546, Type: PrimitiveTypes.String, Name: "file_path", Required: true},
	NestedField{ID: 2147483545, Type: PrimitiveTypes.Int32, Name: "pos", Required: true},
)
View Source
var PrimitiveTypes = struct {
	Bool          PrimitiveType
	Int32         PrimitiveType
	Int64         PrimitiveType
	Float32       PrimitiveType
	Float64       PrimitiveType
	Date          PrimitiveType
	Time          PrimitiveType
	Timestamp     PrimitiveType
	TimestampTz   PrimitiveType
	TimestampNs   PrimitiveType
	TimestampTzNs PrimitiveType
	String        PrimitiveType
	Binary        PrimitiveType
	UUID          PrimitiveType
}{
	Bool:          BooleanType{},
	Int32:         Int32Type{},
	Int64:         Int64Type{},
	Float32:       Float32Type{},
	Float64:       Float64Type{},
	Date:          DateType{},
	Time:          TimeType{},
	Timestamp:     TimestampType{},
	TimestampTz:   TimestampTzType{},
	TimestampNs:   TimestampNsType{},
	TimestampTzNs: TimestampTzNsType{},
	String:        StringType{},
	Binary:        BinaryType{},
	UUID:          UUIDType{},
}
View Source
var UnpartitionedSpec = &PartitionSpec{id: 0}

UnpartitionedSpec is the default unpartitioned spec which can be used for comparisons or to just provide a convenience for referencing the same unpartitioned spec object.

Functions

func ExpressionEvaluator

func ExpressionEvaluator(s *Schema, unbound BooleanExpression, caseSensitive bool) (func(structLike) (bool, error), error)

ExpressionEvaluator returns a function which can be used to evaluate a given expression as long as a structlike value is passed which operates like and matches the passed in schema.

func ExtractFieldIDs

func ExtractFieldIDs(expr BooleanExpression) ([]int, error)

ExtractFieldIDs returns a slice containing the field IDs which are referenced by any terms in the given expression. This enables retrieving exactly which fields are needed for an expression.

func GeneratePartitionFieldName

func GeneratePartitionFieldName(schema *Schema, field PartitionField) (string, error)

GeneratePartitionFieldName returns default partition field name based on field transform type

The default names are aligned with other client implementations https://github.com/apache/iceberg/blob/main/core/src/main/java/org/apache/iceberg/BaseUpdatePartitionSpec.java#L518-L563

func IndexByID

func IndexByID(schema *Schema) (map[int]NestedField, error)

IndexByID performs a post-order traversal of the given schema and returns a mapping from field ID to field.

func IndexByName

func IndexByName(schema *Schema) (map[string]int, error)

IndexByName performs a post-order traversal of the schema and returns a mapping from field name to field ID.

func IndexNameByID

func IndexNameByID(schema *Schema) (map[int]string, error)

IndexNameByID performs a post-order traversal of the schema and returns a mapping from field ID to field name.

func IndexParents

func IndexParents(schema *Schema) (map[int]int, error)

IndexParents generates an index of field IDs to their parent field IDs. Root fields are not indexed

func PreOrderVisit

func PreOrderVisit[T any](sc *Schema, visitor PreOrderSchemaVisitor[T]) (res T, err error)

func Version

func Version() string

func Visit

func Visit[T any](sc *Schema, visitor SchemaVisitor[T]) (res T, err error)

Visit accepts a visitor and performs a post-order traversal of the given schema.

func VisitBoundPredicate

func VisitBoundPredicate[T any](e BoundPredicate, visitor BoundBooleanExprVisitor[T]) T

VisitBoundPredicate uses a BoundBooleanExprVisitor to call the appropriate method based on the type of operation in the predicate. This is a convenience function for implementing the VisitBound method of a BoundBooleanExprVisitor by simply calling iceberg.VisitBoundPredicate(pred, this).

func VisitExpr

func VisitExpr[T any](expr BooleanExpression, visitor BooleanExprVisitor[T]) (res T, err error)

VisitExpr is a convenience function to use a given visitor to visit all parts of a boolean expression in-order. Values returned from the methods are passed to the subsequent methods, effectively "bubbling up" the results.

func VisitMappedFields

func VisitMappedFields[S, T any](fields []MappedField, visitor NameMappingVisitor[S, T]) (res S, err error)

func VisitNameMapping

func VisitNameMapping[S, T any](obj NameMapping, visitor NameMappingVisitor[S, T]) (res S, err error)

func VisitSchemaWithPartner

func VisitSchemaWithPartner[T, P any](sc *Schema, partner P, visitor SchemaWithPartnerVisitor[T, P], accessor PartnerAccessor[P]) (res T, err error)

func WriteManifestList

func WriteManifestList(version int, out io.Writer, snapshotID int64, parentSnapshotID, sequenceNumber *int64, firstRowId int64, files []ManifestFile) error

WriteManifestList writes a list of manifest files to an avro file.

Types

type AboveMaxLiteral

type AboveMaxLiteral interface {
	Literal
	// contains filtered or unexported methods
}

AboveMaxLiteral represents values that are above the maximum for their type such as values > math.MaxInt32 for an Int32Literal

type AfterFieldVisitor

type AfterFieldVisitor interface {
	AfterField(field NestedField)
}

type AfterListElementVisitor

type AfterListElementVisitor interface {
	AfterListElement(elem NestedField)
}

type AfterMapKeyVisitor

type AfterMapKeyVisitor interface {
	AfterMapKey(key NestedField)
}

type AfterMapValueVisitor

type AfterMapValueVisitor interface {
	AfterMapValue(value NestedField)
}

type AlwaysFalse

type AlwaysFalse struct{}

AlwaysFalse is the boolean expression "False"

func (AlwaysFalse) Equals

func (AlwaysFalse) Equals(other BooleanExpression) bool

func (AlwaysFalse) Negate

func (AlwaysFalse) Negate() BooleanExpression

func (AlwaysFalse) Op

func (AlwaysFalse) Op() Operation

func (AlwaysFalse) String

func (AlwaysFalse) String() string

type AlwaysTrue

type AlwaysTrue struct{}

AlwaysTrue is the boolean expression "True"

func (AlwaysTrue) Equals

func (AlwaysTrue) Equals(other BooleanExpression) bool

func (AlwaysTrue) Negate

func (AlwaysTrue) Negate() BooleanExpression

func (AlwaysTrue) Op

func (AlwaysTrue) Op() Operation

func (AlwaysTrue) String

func (AlwaysTrue) String() string

type AndExpr

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

func (AndExpr) Equals

func (a AndExpr) Equals(other BooleanExpression) bool

func (AndExpr) Negate

func (a AndExpr) Negate() BooleanExpression

func (AndExpr) Op

func (AndExpr) Op() Operation

func (AndExpr) String

func (a AndExpr) String() string

type BeforeFieldVisitor

type BeforeFieldVisitor interface {
	BeforeField(field NestedField)
}

type BeforeListElementVisitor

type BeforeListElementVisitor interface {
	BeforeListElement(elem NestedField)
}

type BeforeMapKeyVisitor

type BeforeMapKeyVisitor interface {
	BeforeMapKey(key NestedField)
}

type BeforeMapValueVisitor

type BeforeMapValueVisitor interface {
	BeforeMapValue(value NestedField)
}

type BelowMinLiteral

type BelowMinLiteral interface {
	Literal
	// contains filtered or unexported methods
}

BelowMinLiteral represents values that are below the minimum for their type such as values < math.MinInt32 for an Int32Literal

type BinaryLiteral

type BinaryLiteral []byte

func (BinaryLiteral) Any

func (b BinaryLiteral) Any() any

func (BinaryLiteral) Comparator

func (BinaryLiteral) Comparator() Comparator[[]byte]

func (BinaryLiteral) Equals

func (b BinaryLiteral) Equals(other Literal) bool

func (BinaryLiteral) MarshalBinary

func (b BinaryLiteral) MarshalBinary() (data []byte, err error)

func (BinaryLiteral) String

func (b BinaryLiteral) String() string

func (BinaryLiteral) To

func (b BinaryLiteral) To(typ Type) (Literal, error)

func (BinaryLiteral) Type

func (b BinaryLiteral) Type() Type

func (*BinaryLiteral) UnmarshalBinary

func (b *BinaryLiteral) UnmarshalBinary(data []byte) error

func (BinaryLiteral) Value

func (b BinaryLiteral) Value() []byte

type BinaryType

type BinaryType struct{}

func (BinaryType) Equals

func (BinaryType) Equals(other Type) bool

func (BinaryType) String

func (BinaryType) String() string

func (BinaryType) Type

func (BinaryType) Type() string

type BoolLiteral

type BoolLiteral bool

func (BoolLiteral) Any

func (b BoolLiteral) Any() any

func (BoolLiteral) Comparator

func (BoolLiteral) Comparator() Comparator[bool]

func (BoolLiteral) Equals

func (b BoolLiteral) Equals(l Literal) bool

func (BoolLiteral) MarshalBinary

func (b BoolLiteral) MarshalBinary() (data []byte, err error)

func (BoolLiteral) String

func (b BoolLiteral) String() string

func (BoolLiteral) To

func (b BoolLiteral) To(t Type) (Literal, error)

func (BoolLiteral) Type

func (b BoolLiteral) Type() Type

func (*BoolLiteral) UnmarshalBinary

func (b *BoolLiteral) UnmarshalBinary(data []byte) error

func (BoolLiteral) Value

func (b BoolLiteral) Value() bool

type BooleanExprVisitor

type BooleanExprVisitor[T any] interface {
	VisitTrue() T
	VisitFalse() T
	VisitNot(childResult T) T
	VisitAnd(left, right T) T
	VisitOr(left, right T) T
	VisitUnbound(UnboundPredicate) T
	VisitBound(BoundPredicate) T
}

BooleanExprVisitor is an interface for recursively visiting the nodes of a boolean expression

type BooleanExpression

type BooleanExpression interface {
	fmt.Stringer
	Op() Operation
	Negate() BooleanExpression
	Equals(BooleanExpression) bool
}

BooleanExpression represents a full expression which will evaluate to a boolean value such as GreaterThan or StartsWith, etc.

func BindExpr

func BindExpr(s *Schema, expr BooleanExpression, caseSensitive bool) (BooleanExpression, error)

BindExpr recursively binds each portion of an expression using the provided schema. Because the expression can end up being simplified to just AlwaysTrue/AlwaysFalse, this returns a BooleanExpression.

func IsIn

func IsIn[T LiteralType](t UnboundTerm, vals ...T) BooleanExpression

IsIn is a convenience wrapper for constructing an unbound set predicate for OpIn. It returns a BooleanExpression instead of an UnboundPredicate because depending on the arguments, it can automatically reduce to AlwaysFalse or AlwaysTrue (if given no values for examples). It may also reduce to EqualTo if only one value is provided.

Will panic if t is nil

func NewAnd

func NewAnd(left, right BooleanExpression, addl ...BooleanExpression) BooleanExpression

NewAnd will construct a new AndExpr, allowing the caller to provide potentially more than just two arguments which will be folded to create an appropriate expression tree. i.e. NewAnd(a, b, c, d) becomes AndExpr(a, AndExpr(b, AndExpr(c, d)))

Slight optimizations are performed on creation if either argument is AlwaysFalse or AlwaysTrue by performing reductions. If any argument is AlwaysFalse, then everything will get folded to a return of AlwaysFalse. If an argument is AlwaysTrue, then the other argument will be returned directly rather than creating an AndExpr.

Will panic if any argument is nil

func NewNot

NewNot creates a BooleanExpression representing a "Not" operation on the given argument. It will optimize slightly though:

If the argument is AlwaysTrue or AlwaysFalse, the appropriate inverse expression will be returned directly. If the argument is itself a NotExpr, then the child will be returned rather than NotExpr(NotExpr(child)).

func NewOr

func NewOr(left, right BooleanExpression, addl ...BooleanExpression) BooleanExpression

NewOr will construct a new OrExpr, allowing the caller to provide potentially more than just two arguments which will be folded to create an appropriate expression tree. i.e. NewOr(a, b, c, d) becomes OrExpr(a, OrExpr(b, OrExpr(c, d)))

Slight optimizations are performed on creation if either argument is AlwaysFalse or AlwaysTrue by performing reductions. If any argument is AlwaysTrue, then everything will get folded to a return of AlwaysTrue. If an argument is AlwaysFalse, then the other argument will be returned directly rather than creating an OrExpr.

Will panic if any argument is nil

func NotIn

func NotIn[T LiteralType](t UnboundTerm, vals ...T) BooleanExpression

NotIn is a convenience wrapper for constructing an unbound set predicate for OpNotIn. It returns a BooleanExpression instead of an UnboundPredicate because depending on the arguments, it can automatically reduce to AlwaysFalse or AlwaysTrue (if given no values for examples). It may also reduce to NotEqualTo if only one value is provided.

Will panic if t is nil

func RewriteNotExpr

func RewriteNotExpr(expr BooleanExpression) (BooleanExpression, error)

RewriteNotExpr rewrites a boolean expression to remove "Not" nodes from the expression tree. This is because Projections assume there are no "not" nodes.

Not nodes will be replaced with simply calling `Negate` on the child in the tree.

func SetPredicate

func SetPredicate(op Operation, t UnboundTerm, lits []Literal) BooleanExpression

SetPredicate creates a boolean expression representing a predicate that uses a set of literals as the argument, like In or NotIn. Duplicate literals will be folded into a set, only maintaining the unique literals.

Will panic if op is not a valid Set operation

func TranslateColumnNames

func TranslateColumnNames(expr BooleanExpression, fileSchema *Schema) (BooleanExpression, error)

TranslateColumnNames converts the names of columns in an expression by looking up the field IDs in the file schema. If columns don't exist they are replaced with AlwaysFalse or AlwaysTrue depending on the operator.

type BooleanType

type BooleanType struct{}

func (BooleanType) Equals

func (BooleanType) Equals(other Type) bool

func (BooleanType) String

func (BooleanType) String() string

func (BooleanType) Type

func (BooleanType) Type() string

type BoundBooleanExprVisitor

type BoundBooleanExprVisitor[T any] interface {
	BooleanExprVisitor[T]

	VisitIn(BoundTerm, Set[Literal]) T
	VisitNotIn(BoundTerm, Set[Literal]) T
	VisitIsNan(BoundTerm) T
	VisitNotNan(BoundTerm) T
	VisitIsNull(BoundTerm) T
	VisitNotNull(BoundTerm) T
	VisitEqual(BoundTerm, Literal) T
	VisitNotEqual(BoundTerm, Literal) T
	VisitGreaterEqual(BoundTerm, Literal) T
	VisitGreater(BoundTerm, Literal) T
	VisitLessEqual(BoundTerm, Literal) T
	VisitLess(BoundTerm, Literal) T
	VisitStartsWith(BoundTerm, Literal) T
	VisitNotStartsWith(BoundTerm, Literal) T
}

BoundBooleanExprVisitor builds on BooleanExprVisitor by adding interface methods for visiting bound expressions, because we do casting of literals during binding you can assume that the BoundTerm and the Literal passed to a method have the same type.

type BoundLiteralPredicate

type BoundLiteralPredicate interface {
	BoundPredicate

	Literal() Literal
	AsUnbound(Reference, Literal) UnboundPredicate
}

BoundLiteralPredicate represents a bound boolean expression that utilizes a single literal as an argument, such as Equals or StartsWith.

type BoundPredicate

type BoundPredicate interface {
	BooleanExpression
	Ref() BoundReference
	Term() BoundTerm
}

BoundPredicate is a boolean predicate expression which has been bound to a schema. The underlying reference and term can be retrieved from it.

type BoundReference

type BoundReference interface {
	BoundTerm

	Field() NestedField
	Pos() int
	PosPath() []int
}

BoundReference is a named reference that has been bound to a particular field in a given schema.

type BoundSetPredicate

type BoundSetPredicate interface {
	BoundPredicate

	Literals() Set[Literal]
	AsUnbound(Reference, []Literal) UnboundPredicate
}

BoundSetPredicate is a bound expression that utilizes a set of literals such as In or NotIn

type BoundTerm

type BoundTerm interface {
	Term

	Equals(BoundTerm) bool
	Ref() BoundReference
	Type() Type
	// contains filtered or unexported methods
}

BoundTerm is a simple expression (typically a reference) that evaluates to a value and has been bound to a schema.

type BoundTransform

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

func (*BoundTransform) Equals

func (b *BoundTransform) Equals(other BoundTerm) bool

func (*BoundTransform) Ref

func (b *BoundTransform) Ref() BoundReference

func (*BoundTransform) String

func (b *BoundTransform) String() string

func (*BoundTransform) Type

func (b *BoundTransform) Type() Type

type BoundUnaryPredicate

type BoundUnaryPredicate interface {
	BoundPredicate

	AsUnbound(Reference) UnboundPredicate
}

BoundUnaryPredicate is a bound predicate expression that has no arguments

type BucketTransform

type BucketTransform struct {
	NumBuckets int
}

BucketTransform transforms values into a bucket partition value. It is parameterized by a number of buckets. Bucket partition transforms use a 32-bit hash of the source value to produce a positive value by mod the bucket number.

func (BucketTransform) Apply

func (t BucketTransform) Apply(value Optional[Literal]) Optional[Literal]

func (BucketTransform) CanTransform

func (BucketTransform) CanTransform(t Type) bool

func (BucketTransform) Equals

func (t BucketTransform) Equals(other Transform) bool

func (BucketTransform) MarshalText

func (t BucketTransform) MarshalText() ([]byte, error)

func (BucketTransform) PreservesOrder

func (BucketTransform) PreservesOrder() bool

func (BucketTransform) Project

func (t BucketTransform) Project(name string, pred BoundPredicate) (UnboundPredicate, error)

func (BucketTransform) ResultType

func (BucketTransform) ResultType(Type) Type

func (BucketTransform) String

func (t BucketTransform) String() string

func (BucketTransform) ToHumanStr

func (BucketTransform) ToHumanStr(val any) string

func (BucketTransform) Transformer

func (t BucketTransform) Transformer(src Type) func(any) Optional[int32]

type Comparator

type Comparator[T LiteralType] func(v1, v2 T) int

Comparator is a comparison function for specific literal types:

returns 0 if v1 == v2
returns <0 if v1 < v2
returns >0 if v1 > v2

type DataFile

type DataFile interface {
	// ContentType is the type of the content stored by the data file,
	// either Data, Equality deletes, or Position deletes. All v1 files
	// are Data files.
	ContentType() ManifestEntryContent
	// FilePath is the full URI for the file, complete with FS scheme.
	FilePath() string
	// FileFormat is the format of the data file, AVRO, Orc, or Parquet.
	FileFormat() FileFormat
	// Partition returns a mapping of field id to partition value for
	// each of the partition spec's fields.
	Partition() map[int]any
	// PartitionFieldData returns a mapping of field id to partition value
	// for each of the partition spec's fields.
	Count() int64
	// FileSizeBytes is the total file size in bytes.
	FileSizeBytes() int64
	// ColumnSizes is a mapping from column id to the total size on disk
	// of all regions that store the column. Does not include bytes
	// necessary to read other columns, like footers. Map will be nil for
	// row-oriented formats (avro).
	ColumnSizes() map[int]int64
	// ValueCounts is a mapping from column id to the number of values
	// in the column, including null and NaN values.
	ValueCounts() map[int]int64
	// NullValueCounts is a mapping from column id to the number of
	// null values in the column.
	NullValueCounts() map[int]int64
	// NaNValueCounts is a mapping from column id to the number of NaN
	// values in the column.
	NaNValueCounts() map[int]int64
	// DistictValueCounts is a mapping from column id to the number of
	// distinct values in the column. Distinct counts must be derived
	// using values in the file by counting or using sketches, but not
	// using methods like merging existing distinct counts.
	DistinctValueCounts() map[int]int64
	// LowerBoundValues is a mapping from column id to the lower bounded
	// value of the column, serialized as binary. Each value in the column
	// must be less than or requal to all non-null, non-NaN values in the
	// column for the file.
	LowerBoundValues() map[int][]byte
	// UpperBoundValues is a mapping from column id to the upper bounded
	// value of the column, serialized as binary. Each value in the column
	// must be greater than or equal to all non-null, non-NaN values in
	// the column for the file.
	UpperBoundValues() map[int][]byte
	// KeyMetadata is implementation-specific key metadata for encryption.
	KeyMetadata() []byte
	// SplitOffsets are the split offsets for the data file. For example,
	// all row group offsets in a Parquet file. Must be sorted ascending.
	SplitOffsets() []int64
	// EqualityFieldIDs are used to determine row equality in equality
	// delete files. It is required when the content type is
	// EntryContentEqDeletes.
	EqualityFieldIDs() []int
	// SortOrderID returns the id representing the sort order for this
	// file, or nil if there is no sort order.
	SortOrderID() *int
	// SpecID returns the partition spec id for this data file, inherited
	// from the manifest that the data file was read from
	SpecID() int32
	// FirstRowID returns the first row ID for this data file ( v3+ only )
	FirstRowID() *int64
	// ReferencedDataFile returns the location of the data file that deletion vector reference
	ReferencedDataFile() *string
	// ContentOffset returns the offset in the file where the content starts ( v3+ only )
	ContentOffset() *int64
	// ContentSizeInBytes returns the length of referenced contented stored in the file (v3+ only)
	ContentSizeInBytes() *int64
}

DataFile is the interface for reading the information about a given data file indicated by an entry in a manifest list.

type DataFileBuilder

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

DataFileBuilder is a helper for building a data file struct which will conform to the DataFile interface.

func NewDataFileBuilder

func NewDataFileBuilder(
	spec PartitionSpec,
	content ManifestEntryContent,
	path string,
	format FileFormat,
	fieldIDToPartitionData map[int]any,
	fieldIDToLogicalType map[int]avro.LogicalType,
	fieldIDToFixedSize map[int]int,
	recordCount int64,
	fileSize int64,
) (*DataFileBuilder, error)

NewDataFileBuilder is passed all of the required fields and then allows all of the optional fields to be set by calling the corresponding methods before calling DataFileBuilder.Build to construct the object.

func (*DataFileBuilder) BlockSizeInBytes

func (b *DataFileBuilder) BlockSizeInBytes(size int64) *DataFileBuilder

BlockSizeInBytes sets the block size in bytes for the data file. Deprecated in v2.

func (*DataFileBuilder) Build

func (b *DataFileBuilder) Build() DataFile

func (*DataFileBuilder) ColumnSizes

func (b *DataFileBuilder) ColumnSizes(sizes map[int]int64) *DataFileBuilder

ColumnSizes sets the column sizes for the data file.

func (*DataFileBuilder) ContentOffset

func (b *DataFileBuilder) ContentOffset(offset int64) *DataFileBuilder

func (*DataFileBuilder) ContentSizeInBytes

func (b *DataFileBuilder) ContentSizeInBytes(size int64) *DataFileBuilder

func (*DataFileBuilder) DistinctValueCounts

func (b *DataFileBuilder) DistinctValueCounts(counts map[int]int64) *DataFileBuilder

DistinctValueCounts sets the distinct value counts for the data file.

func (*DataFileBuilder) EqualityFieldIDs

func (b *DataFileBuilder) EqualityFieldIDs(ids []int) *DataFileBuilder

EqualityFieldIDs sets the equality field ids for the data file.

func (*DataFileBuilder) FirstRowID

func (b *DataFileBuilder) FirstRowID(id int64) *DataFileBuilder

func (*DataFileBuilder) KeyMetadata

func (b *DataFileBuilder) KeyMetadata(key []byte) *DataFileBuilder

KeyMetadata sets the key metadata for the data file.

func (*DataFileBuilder) LowerBoundValues

func (b *DataFileBuilder) LowerBoundValues(bounds map[int][]byte) *DataFileBuilder

LowerBoundValues sets the lower bound values for the data file.

func (*DataFileBuilder) NaNValueCounts

func (b *DataFileBuilder) NaNValueCounts(counts map[int]int64) *DataFileBuilder

NaNValueCounts sets the NaN value counts for the data file.

func (*DataFileBuilder) NullValueCounts

func (b *DataFileBuilder) NullValueCounts(counts map[int]int64) *DataFileBuilder

NullValueCounts sets the null value counts for the data file.

func (*DataFileBuilder) ReferencedDataFile

func (b *DataFileBuilder) ReferencedDataFile(path string) *DataFileBuilder

func (*DataFileBuilder) SortOrderID

func (b *DataFileBuilder) SortOrderID(id int) *DataFileBuilder

SortOrderID sets the sort order id for the data file.

func (*DataFileBuilder) SplitOffsets

func (b *DataFileBuilder) SplitOffsets(offsets []int64) *DataFileBuilder

SplitOffsets sets the split offsets for the data file.

func (*DataFileBuilder) UpperBoundValues

func (b *DataFileBuilder) UpperBoundValues(bounds map[int][]byte) *DataFileBuilder

UpperBoundValues sets the upper bound values for the data file.

func (*DataFileBuilder) ValueCounts

func (b *DataFileBuilder) ValueCounts(counts map[int]int64) *DataFileBuilder

ValueCounts sets the value counts for the data file.

type Date

type Date int32

func (Date) ToTime

func (d Date) ToTime() time.Time

type DateLiteral

type DateLiteral Date

func (DateLiteral) Any

func (d DateLiteral) Any() any

func (DateLiteral) Comparator

func (DateLiteral) Comparator() Comparator[Date]

func (DateLiteral) Decrement

func (d DateLiteral) Decrement() Literal

func (DateLiteral) Equals

func (d DateLiteral) Equals(other Literal) bool

func (DateLiteral) Increment

func (d DateLiteral) Increment() Literal

func (DateLiteral) MarshalBinary

func (d DateLiteral) MarshalBinary() (data []byte, err error)

func (DateLiteral) String

func (d DateLiteral) String() string

func (DateLiteral) To

func (d DateLiteral) To(t Type) (Literal, error)

func (DateLiteral) Type

func (d DateLiteral) Type() Type

func (*DateLiteral) UnmarshalBinary

func (d *DateLiteral) UnmarshalBinary(data []byte) error

func (DateLiteral) Value

func (d DateLiteral) Value() Date

type DateType

type DateType struct{}

DateType represents a calendar date without a timezone or time, represented as a 32-bit integer denoting the number of days since the unix epoch.

func (DateType) Equals

func (DateType) Equals(other Type) bool

func (DateType) String

func (DateType) String() string

func (DateType) Type

func (DateType) Type() string

type DayTransform

type DayTransform struct{}

DayTransform transforms a datetime value into a date value.

func (DayTransform) Apply

func (DayTransform) Apply(value Optional[Literal]) (out Optional[Literal])

func (DayTransform) CanTransform

func (t DayTransform) CanTransform(sourceType Type) bool

func (DayTransform) Equals

func (DayTransform) Equals(other Transform) bool

func (DayTransform) MarshalText

func (t DayTransform) MarshalText() ([]byte, error)

func (DayTransform) PreservesOrder

func (DayTransform) PreservesOrder() bool

func (DayTransform) Project

func (t DayTransform) Project(name string, pred BoundPredicate) (UnboundPredicate, error)

func (DayTransform) ResultType

func (DayTransform) ResultType(Type) Type

func (DayTransform) String

func (DayTransform) String() string

func (DayTransform) ToHumanStr

func (DayTransform) ToHumanStr(val any) string

func (DayTransform) Transformer

func (DayTransform) Transformer(src Type) (func(any) Optional[int32], error)

type Decimal

type Decimal struct {
	Val   decimal.Decimal128
	Scale int
}

func (Decimal) String

func (d Decimal) String() string

type DecimalLiteral

type DecimalLiteral Decimal

func (DecimalLiteral) Any

func (d DecimalLiteral) Any() any

func (DecimalLiteral) Comparator

func (DecimalLiteral) Comparator() Comparator[Decimal]

func (DecimalLiteral) Decrement

func (d DecimalLiteral) Decrement() Literal

func (DecimalLiteral) Equals

func (d DecimalLiteral) Equals(other Literal) bool

func (DecimalLiteral) Increment

func (d DecimalLiteral) Increment() Literal

func (DecimalLiteral) MarshalBinary

func (d DecimalLiteral) MarshalBinary() (data []byte, err error)

func (DecimalLiteral) String

func (d DecimalLiteral) String() string

func (DecimalLiteral) To

func (d DecimalLiteral) To(t Type) (Literal, error)

func (DecimalLiteral) Type

func (d DecimalLiteral) Type() Type

func (*DecimalLiteral) UnmarshalBinary

func (d *DecimalLiteral) UnmarshalBinary(data []byte) error

func (DecimalLiteral) Value

func (d DecimalLiteral) Value() Decimal

type DecimalType

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

func DecimalTypeOf

func DecimalTypeOf(prec, scale int) DecimalType

func (DecimalType) Equals

func (d DecimalType) Equals(other Type) bool

func (DecimalType) Precision

func (d DecimalType) Precision() int

func (DecimalType) Scale

func (d DecimalType) Scale() int

func (DecimalType) String

func (d DecimalType) String() string

func (DecimalType) Type

func (d DecimalType) Type() string

type FieldSummary

type FieldSummary struct {
	ContainsNull bool    `avro:"contains_null"`
	ContainsNaN  *bool   `avro:"contains_nan"`
	LowerBound   *[]byte `avro:"lower_bound"`
	UpperBound   *[]byte `avro:"upper_bound"`
}

type FileFormat

type FileFormat string

FileFormat defines constants for the format of data files.

const (
	AvroFile    FileFormat = "AVRO"
	OrcFile     FileFormat = "ORC"
	ParquetFile FileFormat = "PARQUET"
)

type FixedLiteral

type FixedLiteral []byte

func (FixedLiteral) Any

func (f FixedLiteral) Any() any

func (FixedLiteral) Comparator

func (FixedLiteral) Comparator() Comparator[[]byte]

func (FixedLiteral) Equals

func (f FixedLiteral) Equals(other Literal) bool

func (FixedLiteral) MarshalBinary

func (f FixedLiteral) MarshalBinary() (data []byte, err error)

func (FixedLiteral) String

func (f FixedLiteral) String() string

func (FixedLiteral) To

func (f FixedLiteral) To(typ Type) (Literal, error)

func (FixedLiteral) Type

func (f FixedLiteral) Type() Type

func (*FixedLiteral) UnmarshalBinary

func (f *FixedLiteral) UnmarshalBinary(data []byte) error

func (FixedLiteral) Value

func (f FixedLiteral) Value() []byte

type FixedType

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

func FixedTypeOf

func FixedTypeOf(n int) FixedType

func (FixedType) Equals

func (f FixedType) Equals(other Type) bool

func (FixedType) Len

func (f FixedType) Len() int

func (FixedType) String

func (f FixedType) String() string

func (FixedType) Type

func (f FixedType) Type() string

type Float32Literal

type Float32Literal float32

func (Float32Literal) Any

func (f Float32Literal) Any() any

func (Float32Literal) Comparator

func (Float32Literal) Comparator() Comparator[float32]

func (Float32Literal) Equals

func (f Float32Literal) Equals(other Literal) bool

func (Float32Literal) MarshalBinary

func (f Float32Literal) MarshalBinary() (data []byte, err error)

func (Float32Literal) String

func (f Float32Literal) String() string

func (Float32Literal) To

func (f Float32Literal) To(t Type) (Literal, error)

func (Float32Literal) Type

func (f Float32Literal) Type() Type

func (*Float32Literal) UnmarshalBinary

func (f *Float32Literal) UnmarshalBinary(data []byte) error

func (Float32Literal) Value

func (f Float32Literal) Value() float32

type Float32Type

type Float32Type struct{}

Float32Type is the "float" type in the iceberg spec.

func (Float32Type) Equals

func (Float32Type) Equals(other Type) bool

func (Float32Type) String

func (Float32Type) String() string

func (Float32Type) Type

func (Float32Type) Type() string

type Float64Literal

type Float64Literal float64

func (Float64Literal) Any

func (f Float64Literal) Any() any

func (Float64Literal) Comparator

func (Float64Literal) Comparator() Comparator[float64]

func (Float64Literal) Equals

func (f Float64Literal) Equals(other Literal) bool

func (Float64Literal) MarshalBinary

func (f Float64Literal) MarshalBinary() (data []byte, err error)

func (Float64Literal) String

func (f Float64Literal) String() string

func (Float64Literal) To

func (f Float64Literal) To(t Type) (Literal, error)

func (Float64Literal) Type

func (f Float64Literal) Type() Type

func (*Float64Literal) UnmarshalBinary

func (f *Float64Literal) UnmarshalBinary(data []byte) error

func (Float64Literal) Value

func (f Float64Literal) Value() float64

type Float64Type

type Float64Type struct{}

Float64Type represents the "double" type of the iceberg spec.

func (Float64Type) Equals

func (Float64Type) Equals(other Type) bool

func (Float64Type) String

func (Float64Type) String() string

func (Float64Type) Type

func (Float64Type) Type() string

type HourTransform

type HourTransform struct{}

HourTransform transforms a datetime value into an hour value.

func (HourTransform) Apply

func (HourTransform) Apply(value Optional[Literal]) (out Optional[Literal])

func (HourTransform) CanTransform

func (t HourTransform) CanTransform(sourceType Type) bool

func (HourTransform) Equals

func (HourTransform) Equals(other Transform) bool

func (HourTransform) MarshalText

func (t HourTransform) MarshalText() ([]byte, error)

func (HourTransform) PreservesOrder

func (HourTransform) PreservesOrder() bool

func (HourTransform) Project

func (t HourTransform) Project(name string, pred BoundPredicate) (UnboundPredicate, error)

func (HourTransform) ResultType

func (HourTransform) ResultType(Type) Type

func (HourTransform) String

func (HourTransform) String() string

func (HourTransform) ToHumanStr

func (HourTransform) ToHumanStr(val any) string

func (HourTransform) Transformer

func (HourTransform) Transformer(src Type) (func(any) Optional[int32], error)

type IdentityTransform

type IdentityTransform struct{}

IdentityTransform uses the identity function, performing no transformation but instead partitioning on the value itself.

func (IdentityTransform) Apply

func (IdentityTransform) CanTransform

func (IdentityTransform) CanTransform(t Type) bool

func (IdentityTransform) Equals

func (IdentityTransform) Equals(other Transform) bool

func (IdentityTransform) MarshalText

func (t IdentityTransform) MarshalText() ([]byte, error)

func (IdentityTransform) PreservesOrder

func (IdentityTransform) PreservesOrder() bool

func (IdentityTransform) Project

func (IdentityTransform) ResultType

func (IdentityTransform) ResultType(t Type) Type

func (IdentityTransform) String

func (IdentityTransform) String() string

func (IdentityTransform) ToHumanStr

func (IdentityTransform) ToHumanStr(val any) string

type Int32Literal

type Int32Literal int32

func (Int32Literal) Any

func (i Int32Literal) Any() any

func (Int32Literal) Comparator

func (Int32Literal) Comparator() Comparator[int32]

func (Int32Literal) Decrement

func (i Int32Literal) Decrement() Literal

func (Int32Literal) Equals

func (i Int32Literal) Equals(other Literal) bool

func (Int32Literal) Increment

func (i Int32Literal) Increment() Literal

func (Int32Literal) MarshalBinary

func (i Int32Literal) MarshalBinary() (data []byte, err error)

func (Int32Literal) String

func (i Int32Literal) String() string

func (Int32Literal) To

func (i Int32Literal) To(t Type) (Literal, error)

func (Int32Literal) Type

func (i Int32Literal) Type() Type

func (*Int32Literal) UnmarshalBinary

func (i *Int32Literal) UnmarshalBinary(data []byte) error

func (Int32Literal) Value

func (i Int32Literal) Value() int32

type Int32Type

type Int32Type struct{}

Int32Type is the "int"/"integer" type of the iceberg spec.

func (Int32Type) Equals

func (Int32Type) Equals(other Type) bool

func (Int32Type) String

func (Int32Type) String() string

func (Int32Type) Type

func (Int32Type) Type() string

type Int64Literal

type Int64Literal int64

func (Int64Literal) Any

func (i Int64Literal) Any() any

func (Int64Literal) Comparator

func (Int64Literal) Comparator() Comparator[int64]

func (Int64Literal) Decrement

func (i Int64Literal) Decrement() Literal

func (Int64Literal) Equals

func (i Int64Literal) Equals(other Literal) bool

func (Int64Literal) Increment

func (i Int64Literal) Increment() Literal

func (Int64Literal) MarshalBinary

func (i Int64Literal) MarshalBinary() (data []byte, err error)

func (Int64Literal) String

func (i Int64Literal) String() string

func (Int64Literal) To

func (i Int64Literal) To(t Type) (Literal, error)

func (Int64Literal) Type

func (i Int64Literal) Type() Type

func (*Int64Literal) UnmarshalBinary

func (i *Int64Literal) UnmarshalBinary(data []byte) error

func (Int64Literal) Value

func (i Int64Literal) Value() int64

type Int64Type

type Int64Type struct{}

Int64Type is the "long" type of the iceberg spec.

func (Int64Type) Equals

func (Int64Type) Equals(other Type) bool

func (Int64Type) String

func (Int64Type) String() string

func (Int64Type) Type

func (Int64Type) Type() string

type ListType

type ListType struct {
	ElementID       int  `json:"element-id"`
	Element         Type `json:"-"`
	ElementRequired bool `json:"element-required"`
}

func (*ListType) ElementField

func (l *ListType) ElementField() NestedField

func (*ListType) Equals

func (l *ListType) Equals(other Type) bool

func (*ListType) Fields

func (l *ListType) Fields() []NestedField

func (*ListType) MarshalJSON

func (l *ListType) MarshalJSON() ([]byte, error)

func (*ListType) String

func (l *ListType) String() string

func (*ListType) Type

func (*ListType) Type() string

func (*ListType) UnmarshalJSON

func (l *ListType) UnmarshalJSON(b []byte) error

type Literal

type Literal interface {
	fmt.Stringer
	encoding.BinaryMarshaler

	Any() any
	Type() Type
	To(Type) (Literal, error)
	Equals(Literal) bool
}

Literal is a non-null literal value. It can be casted using To and be checked for equality against other literals.

func Float32AboveMaxLiteral

func Float32AboveMaxLiteral() Literal

func Float32BelowMinLiteral

func Float32BelowMinLiteral() Literal

func Float64AboveMaxLiteral

func Float64AboveMaxLiteral() Literal

func Float64BelowMinLiteral

func Float64BelowMinLiteral() Literal

func Int32AboveMaxLiteral

func Int32AboveMaxLiteral() Literal

func Int32BelowMinLiteral

func Int32BelowMinLiteral() Literal

func Int64AboveMaxLiteral

func Int64AboveMaxLiteral() Literal

func Int64BelowMinLiteral

func Int64BelowMinLiteral() Literal

func LiteralFromBytes

func LiteralFromBytes(typ Type, data []byte) (Literal, error)

LiteralFromBytes uses the defined Iceberg spec for how to serialize a value of a the provided type and returns the appropriate Literal value from it.

If you already have a value of the desired Literal type, you could alternatively call UnmarshalBinary on it yourself manually.

This is primarily used for retrieving stat values.

func NewLiteral

func NewLiteral[T LiteralType](val T) Literal

NewLiteral provides a literal based on the type of T

type LiteralType

type LiteralType interface {
	bool | int32 | int64 | float32 | float64 | Date |
		Time | Timestamp | TimestampNano | string | []byte | uuid.UUID | Decimal
}

LiteralType is a generic type constraint for the explicit Go types that we allow for literal values. This represents the actual primitive types that exist in Iceberg

type ManifestBuilder

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

func NewManifestFile

func NewManifestFile(version int, path string, length int64, partitionSpecID int32, addedSnapshotID int64) *ManifestBuilder

func (*ManifestBuilder) AddedFiles

func (b *ManifestBuilder) AddedFiles(cnt int32) *ManifestBuilder

func (*ManifestBuilder) AddedRows

func (b *ManifestBuilder) AddedRows(cnt int64) *ManifestBuilder

func (*ManifestBuilder) Build

func (b *ManifestBuilder) Build() ManifestFile

func (*ManifestBuilder) Content

func (b *ManifestBuilder) Content(content ManifestContent) *ManifestBuilder

func (*ManifestBuilder) DeletedFiles

func (b *ManifestBuilder) DeletedFiles(cnt int32) *ManifestBuilder

func (*ManifestBuilder) DeletedRows

func (b *ManifestBuilder) DeletedRows(cnt int64) *ManifestBuilder

func (*ManifestBuilder) ExistingFiles

func (b *ManifestBuilder) ExistingFiles(cnt int32) *ManifestBuilder

func (*ManifestBuilder) ExistingRows

func (b *ManifestBuilder) ExistingRows(cnt int64) *ManifestBuilder

func (*ManifestBuilder) KeyMetadata

func (b *ManifestBuilder) KeyMetadata(km []byte) *ManifestBuilder

func (*ManifestBuilder) Partitions

func (b *ManifestBuilder) Partitions(p []FieldSummary) *ManifestBuilder

func (*ManifestBuilder) SequenceNum

func (b *ManifestBuilder) SequenceNum(num, minSeqNum int64) *ManifestBuilder

type ManifestContent

type ManifestContent int32

ManifestContent indicates the type of data inside of the files described by a manifest. This will indicate whether the data files contain active data or deleted rows.

const (
	ManifestContentData    ManifestContent = 0
	ManifestContentDeletes ManifestContent = 1
)

func (ManifestContent) String

func (m ManifestContent) String() string

type ManifestEntry

type ManifestEntry interface {
	// Status returns the type of the file tracked by this entry.
	// Deletes are informational only and not used in scans.
	Status() ManifestEntryStatus
	// SnapshotID is the id where the file was added, or deleted,
	// if null it is inherited from the manifest list.
	SnapshotID() int64
	// SequenceNum returns the data sequence number of the file.
	// If it was null and the status is EntryStatusADDED then it
	// is inherited from the manifest list.
	SequenceNum() int64
	// FileSequenceNum returns the file sequence number indicating
	// when the file was added. If it was null and the status is
	// EntryStatusADDED then it is inherited from the manifest list.
	FileSequenceNum() *int64
	// DataFile provides the information about the data file indicated
	// by this manifest entry.
	DataFile() DataFile
	// contains filtered or unexported methods
}

ManifestEntry is an interface for both v1 and v2 manifest entries.

func NewManifestEntry

func NewManifestEntry(status ManifestEntryStatus, snapshotID *int64, seqNum, fileSeqNum *int64, df DataFile) ManifestEntry

func ReadManifest

func ReadManifest(m ManifestFile, f io.Reader, discardDeleted bool) ([]ManifestEntry, error)

ReadManifest reads in an avro list file and returns a slice of manifest entries or an error if one is encountered. If discardDeleted is true, the returned slice omits entries whose status is "deleted".

type ManifestEntryBuilder

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

func NewManifestEntryBuilder

func NewManifestEntryBuilder(status ManifestEntryStatus, snapshotID *int64, data DataFile) *ManifestEntryBuilder

func (*ManifestEntryBuilder) Build

func (*ManifestEntryBuilder) FileSequenceNum

func (b *ManifestEntryBuilder) FileSequenceNum(num int64) *ManifestEntryBuilder

func (*ManifestEntryBuilder) SequenceNum

func (b *ManifestEntryBuilder) SequenceNum(num int64) *ManifestEntryBuilder

type ManifestEntryContent

type ManifestEntryContent int8

ManifestEntryContent defines constants for the type of file contents in the file entries. Data, Position based deletes and equality based deletes.

const (
	EntryContentData       ManifestEntryContent = 0
	EntryContentPosDeletes ManifestEntryContent = 1
	EntryContentEqDeletes  ManifestEntryContent = 2
)

func (ManifestEntryContent) String

func (m ManifestEntryContent) String() string

type ManifestEntryStatus

type ManifestEntryStatus int8

ManifestEntryStatus defines constants for the entry status of existing, added or deleted.

const (
	EntryStatusEXISTING ManifestEntryStatus = 0
	EntryStatusADDED    ManifestEntryStatus = 1
	EntryStatusDELETED  ManifestEntryStatus = 2
)

type ManifestFile

type ManifestFile interface {
	// Version returns the version number of this manifest file.
	// It should be 1 or 2.
	Version() int
	// FilePath is the location URI of this manifest file.
	FilePath() string
	// Length is the length in bytes of the manifest file.
	Length() int64
	// PartitionSpecID is the ID of the partition spec used to write
	// this manifest. It must be listed in the table metadata
	// partition-specs.
	PartitionSpecID() int32
	// ManifestContent is the type of files tracked by this manifest,
	// either data or delete files. All v1 manifests track data files.
	ManifestContent() ManifestContent
	// SnapshotID is the ID of the snapshot where this manifest file
	// was added.
	SnapshotID() int64
	// AddedDataFiles returns the number of entries in the manifest that
	// have the status of EntryStatusADDED.
	AddedDataFiles() int32
	// ExistingDataFiles returns the number of entries in the manifest
	// which have the status of EntryStatusEXISTING.
	ExistingDataFiles() int32
	// DeletedDataFiles returns the number of entries in the manifest
	// which have the status of EntryStatusDELETED.
	DeletedDataFiles() int32
	// AddedRows returns the number of rows in all files of the manifest
	// that have status EntryStatusADDED.
	AddedRows() int64
	// ExistingRows returns the number of rows in all files of the manifest
	// which have status EntryStatusEXISTING.
	ExistingRows() int64
	// DeletedRows returns the number of rows in all files of the manifest
	// which have status EntryStatusDELETED.
	DeletedRows() int64
	// SequenceNum returns the sequence number when this manifest was
	// added to the table. Will be 0 for v1 manifest lists.
	SequenceNum() int64
	// MinSequenceNum is the minimum data sequence number of all live data
	// or delete files in the manifest. Will be 0 for v1 manifest lists.
	MinSequenceNum() int64
	// KeyMetadata returns implementation-specific key metadata for encryption
	// if it exists in the manifest list.
	KeyMetadata() []byte
	// Partitions returns a list of field summaries for each partition
	// field in the spec. Each field in the list corresponds to a field in
	// the manifest file's partition spec.
	Partitions() []FieldSummary

	// HasAddedFiles returns true if AddedDataFiles > 0 or if it was null.
	HasAddedFiles() bool
	// HasExistingFiles returns true if ExistingDataFiles > 0 or if it was null.
	HasExistingFiles() bool
	// FetchEntries reads the manifest list file to fetch the list of
	// manifest entries using the provided file system IO interface.
	// If discardDeleted is true, entries for files containing deleted rows
	// will be skipped.
	FetchEntries(fs iceio.IO, discardDeleted bool) ([]ManifestEntry, error)
	// contains filtered or unexported methods
}

ManifestFile is the interface which covers both V1 and V2 manifest files.

func ReadManifestList

func ReadManifestList(in io.Reader) ([]ManifestFile, error)

ReadManifestList reads in an avro manifest list file and returns a slice of manifest files or an error if one is encountered.

func WriteManifest

func WriteManifest(
	filename string,
	out io.Writer,
	version int,
	spec PartitionSpec,
	schema *Schema,
	snapshotID int64,
	entries []ManifestEntry,
) (ManifestFile, error)

type ManifestListWriter

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

func NewManifestListWriterV1

func NewManifestListWriterV1(out io.Writer, snapshotID int64, parentSnapshot *int64) (*ManifestListWriter, error)

func NewManifestListWriterV2

func NewManifestListWriterV2(out io.Writer, snapshotID, sequenceNumber int64, parentSnapshot *int64) (*ManifestListWriter, error)

func NewManifestListWriterV3

func NewManifestListWriterV3(out io.Writer, snapshotId, sequenceNumber, firstRowID int64, parentSnapshot *int64) (*ManifestListWriter, error)

func (*ManifestListWriter) AddManifests

func (m *ManifestListWriter) AddManifests(files []ManifestFile) error

func (*ManifestListWriter) Close

func (m *ManifestListWriter) Close() error

func (*ManifestListWriter) NextRowID

func (m *ManifestListWriter) NextRowID() *int64

type ManifestReader

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

ManifestReader reads the metadata and data from an avro manifest file. This type is not thread-safe; its methods should not be called from multiple goroutines.

func NewManifestReader

func NewManifestReader(file ManifestFile, in io.Reader) (*ManifestReader, error)

NewManifestReader returns a value that can read the contents of an avro manifest file. If the caller is interested in the manifest entries in the file, it must call [ManifestReader.Entries] before closing the provided reader.

func (*ManifestReader) ManifestContent

func (c *ManifestReader) ManifestContent() ManifestContent

ManifestContent returns the type of content in the manifest file.

func (*ManifestReader) PartitionSpec

func (c *ManifestReader) PartitionSpec() (*PartitionSpec, error)

PartitionSpec returns the partition spec encoded in the avro file's metadata.

func (*ManifestReader) PartitionSpecID

func (c *ManifestReader) PartitionSpecID() (int, error)

PartitionSpecID returns the partition spec ID encoded in the avro file's metadata.

func (*ManifestReader) ReadEntry

func (c *ManifestReader) ReadEntry() (ManifestEntry, error)

ReadEntry reads the next manifest entry in the avro file's data.

func (*ManifestReader) Schema

func (c *ManifestReader) Schema() (*Schema, error)

Schema returns the schema encoded in the avro file's metadata.

func (*ManifestReader) SchemaID

func (c *ManifestReader) SchemaID() (int, error)

SchemaID returns the schema ID encoded in the avro file's metadata.

func (*ManifestReader) Version

func (c *ManifestReader) Version() int

Version returns the file's format version.

type ManifestWriter

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

func NewManifestWriter

func NewManifestWriter(version int, out io.Writer, spec PartitionSpec, schema *Schema, snapshotID int64) (*ManifestWriter, error)

func (*ManifestWriter) Add

func (w *ManifestWriter) Add(entry ManifestEntry) error

func (*ManifestWriter) Close

func (w *ManifestWriter) Close() error

func (*ManifestWriter) Delete

func (w *ManifestWriter) Delete(entry ManifestEntry) error

func (*ManifestWriter) Existing

func (w *ManifestWriter) Existing(entry ManifestEntry) error

func (*ManifestWriter) ToManifestFile

func (w *ManifestWriter) ToManifestFile(location string, length int64) (ManifestFile, error)

type MapType

type MapType struct {
	KeyID         int  `json:"key-id"`
	KeyType       Type `json:"-"`
	ValueID       int  `json:"value-id"`
	ValueType     Type `json:"-"`
	ValueRequired bool `json:"value-required"`
}

func (*MapType) Equals

func (m *MapType) Equals(other Type) bool

func (*MapType) Fields

func (m *MapType) Fields() []NestedField

func (*MapType) KeyField

func (m *MapType) KeyField() NestedField

func (*MapType) MarshalJSON

func (m *MapType) MarshalJSON() ([]byte, error)

func (*MapType) String

func (m *MapType) String() string

func (*MapType) Type

func (*MapType) Type() string

func (*MapType) UnmarshalJSON

func (m *MapType) UnmarshalJSON(b []byte) error

func (*MapType) ValueField

func (m *MapType) ValueField() NestedField

type MappedField

type MappedField struct {
	Names []string `json:"names"`
	// iceberg spec says this is optional, but I don't see any examples
	// of this being left empty. Does pyiceberg need to be updated or should
	// the spec not say field-id is optional?
	FieldID *int          `json:"field-id,omitempty"`
	Fields  []MappedField `json:"fields,omitempty"`
}

func (*MappedField) GetField

func (m *MappedField) GetField(field string) *MappedField

func (*MappedField) ID

func (m *MappedField) ID() int

func (*MappedField) Len

func (m *MappedField) Len() int

func (*MappedField) String

func (m *MappedField) String() string

type MonthTransform

type MonthTransform struct{}

MonthTransform transforms a datetime value into a month value.

func (MonthTransform) Apply

func (MonthTransform) Apply(value Optional[Literal]) (out Optional[Literal])

func (MonthTransform) CanTransform

func (t MonthTransform) CanTransform(sourceType Type) bool

func (MonthTransform) Equals

func (MonthTransform) Equals(other Transform) bool

func (MonthTransform) MarshalText

func (t MonthTransform) MarshalText() ([]byte, error)

func (MonthTransform) PreservesOrder

func (MonthTransform) PreservesOrder() bool

func (MonthTransform) Project

func (t MonthTransform) Project(name string, pred BoundPredicate) (UnboundPredicate, error)

func (MonthTransform) ResultType

func (MonthTransform) ResultType(Type) Type

func (MonthTransform) String

func (MonthTransform) String() string

func (MonthTransform) ToHumanStr

func (t MonthTransform) ToHumanStr(val any) string

func (MonthTransform) Transformer

func (MonthTransform) Transformer(src Type) (func(any) Optional[int32], error)

type NameMapping

type NameMapping []MappedField

func UpdateNameMapping

func UpdateNameMapping(nameMapping NameMapping, updates map[int]NestedField, adds map[int][]NestedField) (NameMapping, error)

UpdateNameMapping performs incremental updates to an existing NameMapping, preserving backward compatibility by maintaining existing field name mappings while adding new ones. This is different from createMappingFromSchema which creates a completely new mapping and loses all historical field name mappings.

For example, when updating a field name:

Original: {FieldID: 1, Names: ["foo"]}
After update: {FieldID: 1, Names: ["foo", "foo_update"]}

This preserves compatibility with existing data files that reference the old field names.

func (NameMapping) String

func (nm NameMapping) String() string

type NameMappingAccessor

type NameMappingAccessor struct{}

func (NameMappingAccessor) FieldPartner

func (n NameMappingAccessor) FieldPartner(partnerStruct *MappedField, _ int, fieldName string) *MappedField

func (NameMappingAccessor) ListElementPartner

func (n NameMappingAccessor) ListElementPartner(partnerList *MappedField) *MappedField

func (NameMappingAccessor) MapKeyPartner

func (n NameMappingAccessor) MapKeyPartner(partnerMap *MappedField) *MappedField

func (NameMappingAccessor) MapValuePartner

func (n NameMappingAccessor) MapValuePartner(partnerMap *MappedField) *MappedField

func (NameMappingAccessor) SchemaPartner

func (NameMappingAccessor) SchemaPartner(partner *MappedField) *MappedField

type NameMappingVisitor

type NameMappingVisitor[S, T any] interface {
	Mapping(nm NameMapping, fieldResults S) S
	Fields(st []MappedField, fieldResults []T) S
	Field(field MappedField, fieldResult S) T
}

type NestedField

type NestedField struct {
	Type `json:"-"`

	ID             int    `json:"id"`
	Name           string `json:"name"`
	Required       bool   `json:"required"`
	Doc            string `json:"doc,omitempty"`
	InitialDefault any    `json:"initial-default,omitempty"`
	WriteDefault   any    `json:"write-default,omitempty"`
}

func (*NestedField) Equals

func (n *NestedField) Equals(other NestedField) bool

func (NestedField) MarshalJSON

func (n NestedField) MarshalJSON() ([]byte, error)

func (NestedField) String

func (n NestedField) String() string

func (*NestedField) UnmarshalJSON

func (n *NestedField) UnmarshalJSON(b []byte) error

type NestedType

type NestedType interface {
	Type
	Fields() []NestedField
}

NestedType is an interface that allows access to the child fields of a nested type such as a list/struct/map type.

type NotExpr

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

func (NotExpr) Equals

func (n NotExpr) Equals(other BooleanExpression) bool

func (NotExpr) Negate

func (n NotExpr) Negate() BooleanExpression

func (NotExpr) Op

func (NotExpr) Op() Operation

func (NotExpr) String

func (n NotExpr) String() string

type NumericLiteral

type NumericLiteral interface {
	Literal
	Increment() Literal
	Decrement() Literal
}

type Operation

type Operation int

Operation is an enum used for constants to define what operation a given expression or predicate is going to execute.

const (
	OpTrue  Operation = iota // True
	OpFalse                  // False
	// unary ops
	OpIsNull  // IsNull
	OpNotNull // NotNull
	OpIsNan   // IsNaN
	OpNotNan  // NotNaN
	// literal ops
	OpLT            // LessThan
	OpLTEQ          // LessThanEqual
	OpGT            // GreaterThan
	OpGTEQ          // GreaterThanEqual
	OpEQ            // Equal
	OpNEQ           // NotEqual
	OpStartsWith    // StartsWith
	OpNotStartsWith // NotStartsWith
	// set ops
	OpIn    // In
	OpNotIn // NotIn
	// boolean ops
	OpNot // Not
	OpAnd // And
	OpOr  // Or
)

func (Operation) FlipLR

func (op Operation) FlipLR() Operation

FlipLR returns the correct operation to use if the left and right operands are flipped.

func (Operation) Negate

func (op Operation) Negate() Operation

Negate returns the inverse operation for a given op

func (Operation) String

func (i Operation) String() string

type Optional

type Optional[T any] struct {
	Val   T
	Valid bool
}

Optional represents a typed value that could be null

type OrExpr

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

func (OrExpr) Equals

func (o OrExpr) Equals(other BooleanExpression) bool

func (OrExpr) Negate

func (o OrExpr) Negate() BooleanExpression

func (OrExpr) Op

func (OrExpr) Op() Operation

func (OrExpr) String

func (o OrExpr) String() string

type PartitionField

type PartitionField struct {
	// SourceID is the source column id of the table's schema
	SourceID int `json:"source-id"`
	// FieldID is the partition field id across all the table partition specs
	FieldID int `json:"field-id"`
	// Name is the name of the partition field itself
	Name string `json:"name"`
	// Transform is the transform used to produce the partition value
	Transform Transform `json:"transform"`
	// contains filtered or unexported fields
}

PartitionField represents how one partition value is derived from the source column by transformation.

func (PartitionField) Equals

func (p PartitionField) Equals(other PartitionField) bool

func (*PartitionField) EscapedName

func (p *PartitionField) EscapedName() string

EscapedName returns the URL-escaped version of the partition field name.

func (*PartitionField) String

func (p *PartitionField) String() string

func (*PartitionField) UnmarshalJSON

func (p *PartitionField) UnmarshalJSON(b []byte) error

type PartitionOption

type PartitionOption func(*PartitionSpec) error

func AddPartitionFieldByName

func AddPartitionFieldByName(sourceName string, targetName string, transform Transform, schema *Schema, fieldID *int) PartitionOption

func AddPartitionFieldBySourceID

func AddPartitionFieldBySourceID(sourceID int, targetName string, transform Transform, schema *Schema, fieldID *int) PartitionOption

func WithSpecID

func WithSpecID(id int) PartitionOption

type PartitionSpec

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

PartitionSpec captures the transformation from table data to partition values

func NewPartitionSpec

func NewPartitionSpec(fields ...PartitionField) PartitionSpec

NewPartitionSpec creates a new PartitionSpec with the given fields.

The fields are not verified against a schema, use NewPartitionSpecOpts if you have to ensure compatibility.

func NewPartitionSpecID

func NewPartitionSpecID(id int, fields ...PartitionField) PartitionSpec

NewPartitionSpecID creates a new PartitionSpec with the given fields and id.

The fields are not verified against a schema, use NewPartitionSpecOpts if you have to ensure compatibility.

func NewPartitionSpecOpts

func NewPartitionSpecOpts(opts ...PartitionOption) (PartitionSpec, error)

func (*PartitionSpec) BindToSchema

func (p *PartitionSpec) BindToSchema(schema *Schema, lastPartitionID *int, newSpecID *int) (PartitionSpec, error)

BindToSchema creates a new PartitionSpec by copying the fields from the existing spec verifying compatibility with the schema.

If newSpecID is not nil, it will be used as the spec id for the new spec. Otherwise, the existing spec id will be used. If a field in the spec is incompatible with the schema, an error will be returned.

func (*PartitionSpec) CompatibleWith

func (ps *PartitionSpec) CompatibleWith(other *PartitionSpec) bool

CompatibleWith returns true if this partition spec is considered compatible with the passed in partition spec. This means that the two specs have equivalent field lists regardless of the spec id.

func (PartitionSpec) Equals

func (ps PartitionSpec) Equals(other PartitionSpec) bool

Equals returns true iff the field lists are the same AND the spec id is the same between this partition spec and the provided one.

func (*PartitionSpec) Field

func (ps *PartitionSpec) Field(i int) PartitionField

func (*PartitionSpec) Fields

func (ps *PartitionSpec) Fields() iter.Seq[PartitionField]

Fields returns a clone of the partition fields in this spec.

func (*PartitionSpec) FieldsBySourceID

func (ps *PartitionSpec) FieldsBySourceID(fieldID int) []PartitionField

func (*PartitionSpec) ID

func (ps *PartitionSpec) ID() int

func (PartitionSpec) IsUnpartitioned

func (ps PartitionSpec) IsUnpartitioned() bool

func (*PartitionSpec) LastAssignedFieldID

func (ps *PartitionSpec) LastAssignedFieldID() int

func (*PartitionSpec) Len

func (p *PartitionSpec) Len() int

func (PartitionSpec) MarshalJSON

func (ps PartitionSpec) MarshalJSON() ([]byte, error)

func (*PartitionSpec) NumFields

func (ps *PartitionSpec) NumFields() int

func (*PartitionSpec) PartitionToPath

func (ps *PartitionSpec) PartitionToPath(data structLike, sc *Schema) string

PartitionToPath produces a proper partition path from the data and schema by converting the values to human readable strings and properly escaping.

The path will be in the form of `name1=value1/name2=value2/...`.

This does not apply the transforms to the data, it is assumed the provided data has already been transformed appropriately.

func (*PartitionSpec) PartitionType

func (ps *PartitionSpec) PartitionType(schema *Schema) *StructType

PartitionType produces a struct of the partition spec.

The partition fields should be optional:

  • All partition transforms are required to produce null if the input value is null. This can happen when the source column is optional.
  • Partition fields may be added later, in which case not all files would have the result field and it may be null.

There is a case where we can guarantee that a partition field in the first and only parittion spec that uses a required source column will never be null, but it doesn't seem worth tracking this case.

func (PartitionSpec) String

func (ps PartitionSpec) String() string

func (*PartitionSpec) UnmarshalJSON

func (ps *PartitionSpec) UnmarshalJSON(b []byte) error

type PartnerAccessor

type PartnerAccessor[P any] interface {
	SchemaPartner(P) P
	FieldPartner(partnerStruct P, fieldID int, fieldName string) P
	ListElementPartner(P) P
	MapKeyPartner(P) P
	MapValuePartner(P) P
}

type PreOrderSchemaVisitor

type PreOrderSchemaVisitor[T any] interface {
	Schema(*Schema, func() T) T
	Struct(StructType, []func() T) T
	Field(NestedField, func() T) T
	List(ListType, func() T) T
	Map(MapType, func() T, func() T) T
	Primitive(PrimitiveType) T
}

type PrimitiveType

type PrimitiveType interface {
	Type
	// contains filtered or unexported methods
}

type Properties

type Properties map[string]string

func (Properties) Get

func (p Properties) Get(key, defVal string) string

Get returns the value of the key if it exists, otherwise it returns the default value.

func (Properties) GetBool

func (p Properties) GetBool(key string, defVal bool) bool

func (Properties) GetInt

func (p Properties) GetInt(key string, defVal int) int

type Reference

type Reference string

Reference is a field name not yet bound to a particular field in a schema

func (Reference) Bind

func (r Reference) Bind(s *Schema, caseSensitive bool) (BoundTerm, error)

func (Reference) Equals

func (r Reference) Equals(other UnboundTerm) bool

func (Reference) String

func (r Reference) String() string

type Schema

type Schema struct {
	ID                 int   `json:"schema-id"`
	IdentifierFieldIDs []int `json:"identifier-field-ids"`
	// contains filtered or unexported fields
}

Schema is an Iceberg table schema, represented as a struct with multiple fields. The fields are only exported via accessor methods rather than exposing the slice directly in order to ensure a schema as immutable.

func ApplyNameMapping

func ApplyNameMapping(schemaWithoutIDs *Schema, nameMapping NameMapping) (*Schema, error)

func AssignFreshSchemaIDs

func AssignFreshSchemaIDs(sc *Schema, nextID func() int) (*Schema, error)

AssignFreshSchemaIDs creates a new schema with fresh field IDs for all of the fields in it. The nextID function is used to iteratively generate the ids, if it is nil then a simple incrementing counter is used starting at 1.

func NewSchema

func NewSchema(id int, fields ...NestedField) *Schema

NewSchema constructs a new schema with the provided ID and list of fields.

func NewSchemaFromJsonFields

func NewSchemaFromJsonFields(id int, jsonFieldsStr string) (*Schema, error)

NewSchemaFromJsonFields constructs a new schema with the provided ID and fields in json form

func NewSchemaWithIdentifiers

func NewSchemaWithIdentifiers(id int, identifierIDs []int, fields ...NestedField) *Schema

NewSchemaWithIdentifiers constructs a new schema with the provided ID and fields, along with a slice of field IDs to be listed as identifier fields.

func PruneColumns

func PruneColumns(schema *Schema, selected map[int]Void, selectFullTypes bool) (*Schema, error)

PruneColumns visits a schema pruning any columns which do not exist in the provided selected set. Parent fields of a selected child will be retained.

func SanitizeColumnNames

func SanitizeColumnNames(sc *Schema) (*Schema, error)

func (*Schema) AsStruct

func (s *Schema) AsStruct() StructType

AsStruct returns a Struct with the same fields as the schema which can then be used as a Type.

func (*Schema) Equals

func (s *Schema) Equals(other *Schema) bool

Equals compares the fields and identifierIDs, but does not compare the schema ID itself.

func (*Schema) Field

func (s *Schema) Field(i int) NestedField

func (*Schema) FieldHasOptionalParent

func (s *Schema) FieldHasOptionalParent(id int) bool

func (*Schema) FieldIDs

func (s *Schema) FieldIDs() []int

func (*Schema) Fields

func (s *Schema) Fields() []NestedField

func (*Schema) FindColumnName

func (s *Schema) FindColumnName(fieldID int) (string, bool)

FindColumnName returns the name of the column identified by the passed in field id. The second return value reports whether or not the field id was found in the schema.

func (*Schema) FindFieldByID

func (s *Schema) FindFieldByID(id int) (NestedField, bool)

FindFieldByID is like *Schema.FindColumnName, but returns the whole field rather than just the field name.

func (*Schema) FindFieldByName

func (s *Schema) FindFieldByName(name string) (NestedField, bool)

FindFieldByName returns the field identified by the name given, the second return value will be false if no field by this name is found.

Note: This search is done in a case sensitive manner. To perform a case insensitive search, use *Schema.FindFieldByNameCaseInsensitive.

func (*Schema) FindFieldByNameCaseInsensitive

func (s *Schema) FindFieldByNameCaseInsensitive(name string) (NestedField, bool)

FindFieldByNameCaseInsensitive is like *Schema.FindFieldByName, but performs a case insensitive search.

func (*Schema) FindTypeByID

func (s *Schema) FindTypeByID(id int) (Type, bool)

FindTypeByID is like *Schema.FindFieldByID, but returns only the data type of the field.

func (*Schema) FindTypeByName

func (s *Schema) FindTypeByName(name string) (Type, bool)

FindTypeByName is a convenience function for calling *Schema.FindFieldByName, and then returning just the type.

func (*Schema) FindTypeByNameCaseInsensitive

func (s *Schema) FindTypeByNameCaseInsensitive(name string) (Type, bool)

FindTypeByNameCaseInsensitive is like *Schema.FindTypeByName but performs a case insensitive search.

func (*Schema) FlatFields

func (s *Schema) FlatFields() (iter.Seq[NestedField], error)

FlatFields returns an iterator over the flattened fields in the schema The fields are returned in arbitrary order.

func (*Schema) HighestFieldID

func (s *Schema) HighestFieldID() int

HighestFieldID returns the value of the numerically highest field ID in this schema.

func (*Schema) MarshalJSON

func (s *Schema) MarshalJSON() ([]byte, error)

func (*Schema) NameMapping

func (s *Schema) NameMapping() NameMapping

func (*Schema) NumFields

func (s *Schema) NumFields() int

func (*Schema) Select

func (s *Schema) Select(caseSensitive bool, names ...string) (*Schema, error)

Select creates a new schema with just the fields identified by name passed in the order they are provided. If caseSensitive is false, then fields will be identified by case insensitive search.

An error is returned if a requested name cannot be found.

func (*Schema) String

func (s *Schema) String() string

func (*Schema) Type

func (s *Schema) Type() string

func (*Schema) UnmarshalJSON

func (s *Schema) UnmarshalJSON(b []byte) error

type SchemaVisitor

type SchemaVisitor[T any] interface {
	Schema(schema *Schema, structResult T) T
	Struct(st StructType, fieldResults []T) T
	Field(field NestedField, fieldResult T) T
	List(list ListType, elemResult T) T
	Map(mapType MapType, keyResult, valueResult T) T
	Primitive(p PrimitiveType) T
}

SchemaVisitor is an interface that can be implemented to allow for easy traversal and processing of a schema.

A SchemaVisitor can also optionally implement the Before/After Field, ListElement, MapKey, or MapValue interfaces to allow them to get called at the appropriate points within schema traversal.

type SchemaVisitorPerPrimitiveType

type SchemaVisitorPerPrimitiveType[T any] interface {
	SchemaVisitor[T]

	VisitFixed(FixedType) T
	VisitDecimal(DecimalType) T
	VisitBoolean() T
	VisitInt32() T
	VisitInt64() T
	VisitFloat32() T
	VisitFloat64() T
	VisitDate() T
	VisitTime() T
	VisitTimestamp() T
	VisitTimestampNs() T
	VisitTimestampTz() T
	VisitTimestampNsTz() T
	VisitString() T
	VisitBinary() T
	VisitUUID() T
}

type SchemaWithPartnerVisitor

type SchemaWithPartnerVisitor[T, P any] interface {
	Schema(sc *Schema, schemaPartner P, structResult T) T
	Struct(st StructType, structPartner P, fieldResults []T) T
	Field(field NestedField, fieldPartner P, fieldResult T) T
	List(l ListType, listPartner P, elemResult T) T
	Map(m MapType, mapPartner P, keyResult, valResult T) T
	Primitive(p PrimitiveType, primitivePartner P) T
}

type Set

type Set[E any] interface {
	Add(...E)
	Contains(E) bool
	Members() []E
	Equals(Set[E]) bool
	Len() int
	All(func(E) bool) bool
}

type StringLiteral

type StringLiteral string

func (StringLiteral) Any

func (s StringLiteral) Any() any

func (StringLiteral) Comparator

func (StringLiteral) Comparator() Comparator[string]

func (StringLiteral) Equals

func (s StringLiteral) Equals(other Literal) bool

func (StringLiteral) MarshalBinary

func (s StringLiteral) MarshalBinary() (data []byte, err error)

func (StringLiteral) String

func (s StringLiteral) String() string

func (StringLiteral) To

func (s StringLiteral) To(typ Type) (Literal, error)

func (StringLiteral) Type

func (s StringLiteral) Type() Type

func (*StringLiteral) UnmarshalBinary

func (s *StringLiteral) UnmarshalBinary(data []byte) error

func (StringLiteral) Value

func (s StringLiteral) Value() string

type StringType

type StringType struct{}

func (StringType) Equals

func (StringType) Equals(other Type) bool

func (StringType) String

func (StringType) String() string

func (StringType) Type

func (StringType) Type() string

type StructType

type StructType struct {
	FieldList []NestedField `json:"fields"`
}

func (*StructType) Equals

func (s *StructType) Equals(other Type) bool

func (*StructType) Fields

func (s *StructType) Fields() []NestedField

func (*StructType) MarshalJSON

func (s *StructType) MarshalJSON() ([]byte, error)

func (*StructType) String

func (s *StructType) String() string

func (*StructType) Type

func (*StructType) Type() string

type Term

type Term interface {
	fmt.Stringer
	// contains filtered or unexported methods
}

A Term is a simple expression that evaluates to a value

type Time

type Time int64

func (Time) ToTime

func (t Time) ToTime() time.Time

type TimeLiteral

type TimeLiteral Time

func (TimeLiteral) Any

func (t TimeLiteral) Any() any

func (TimeLiteral) Comparator

func (TimeLiteral) Comparator() Comparator[Time]

func (TimeLiteral) Equals

func (t TimeLiteral) Equals(other Literal) bool

func (TimeLiteral) MarshalBinary

func (t TimeLiteral) MarshalBinary() (data []byte, err error)

func (TimeLiteral) String

func (t TimeLiteral) String() string

func (TimeLiteral) To

func (t TimeLiteral) To(typ Type) (Literal, error)

func (TimeLiteral) Type

func (t TimeLiteral) Type() Type

func (*TimeLiteral) UnmarshalBinary

func (t *TimeLiteral) UnmarshalBinary(data []byte) error

func (TimeLiteral) Value

func (t TimeLiteral) Value() Time

type TimeTransform

type TimeTransform interface {
	Transform
	Transformer(Type) (func(any) Optional[int32], error)
}

type TimeType

type TimeType struct{}

TimeType represents a number of microseconds since midnight.

func (TimeType) Equals

func (TimeType) Equals(other Type) bool

func (TimeType) String

func (TimeType) String() string

func (TimeType) Type

func (TimeType) Type() string

type Timestamp

type Timestamp int64

func (Timestamp) ToDate

func (t Timestamp) ToDate() Date

func (Timestamp) ToNanos

func (t Timestamp) ToNanos() TimestampNano

func (Timestamp) ToTime

func (t Timestamp) ToTime() time.Time

type TimestampLiteral

type TimestampLiteral Timestamp

func (TimestampLiteral) Any

func (t TimestampLiteral) Any() any

func (TimestampLiteral) Comparator

func (TimestampLiteral) Comparator() Comparator[Timestamp]

func (TimestampLiteral) Decrement

func (t TimestampLiteral) Decrement() Literal

func (TimestampLiteral) Equals

func (t TimestampLiteral) Equals(other Literal) bool

func (TimestampLiteral) Increment

func (t TimestampLiteral) Increment() Literal

func (TimestampLiteral) MarshalBinary

func (t TimestampLiteral) MarshalBinary() (data []byte, err error)

func (TimestampLiteral) String

func (t TimestampLiteral) String() string

func (TimestampLiteral) To

func (t TimestampLiteral) To(typ Type) (Literal, error)

func (TimestampLiteral) Type

func (t TimestampLiteral) Type() Type

func (*TimestampLiteral) UnmarshalBinary

func (t *TimestampLiteral) UnmarshalBinary(data []byte) error

func (TimestampLiteral) Value

func (t TimestampLiteral) Value() Timestamp

type TimestampNano

type TimestampNano int64

func (TimestampNano) ToDate

func (t TimestampNano) ToDate() Date

func (TimestampNano) ToMicros

func (t TimestampNano) ToMicros() Timestamp

func (TimestampNano) ToTime

func (t TimestampNano) ToTime() time.Time

type TimestampNsLiteral

type TimestampNsLiteral TimestampNano

func (TimestampNsLiteral) Any

func (t TimestampNsLiteral) Any() any

func (TimestampNsLiteral) Comparator

func (TimestampNsLiteral) Decrement

func (t TimestampNsLiteral) Decrement() Literal

func (TimestampNsLiteral) Equals

func (t TimestampNsLiteral) Equals(other Literal) bool

func (TimestampNsLiteral) Increment

func (t TimestampNsLiteral) Increment() Literal

func (TimestampNsLiteral) MarshalBinary

func (t TimestampNsLiteral) MarshalBinary() (data []byte, err error)

func (TimestampNsLiteral) String

func (t TimestampNsLiteral) String() string

func (TimestampNsLiteral) To

func (t TimestampNsLiteral) To(typ Type) (Literal, error)

func (TimestampNsLiteral) Type

func (t TimestampNsLiteral) Type() Type

func (*TimestampNsLiteral) UnmarshalBinary

func (t *TimestampNsLiteral) UnmarshalBinary(data []byte) error

func (TimestampNsLiteral) Value

type TimestampNsType

type TimestampNsType struct{}

TimestampNsType represents a timestamp stored as nanoseconds since the unix epoch without regard for timezone. Requires format version 3+.

func (TimestampNsType) Equals

func (TimestampNsType) Equals(other Type) bool

func (TimestampNsType) String

func (TimestampNsType) String() string

func (TimestampNsType) Type

func (TimestampNsType) Type() string

type TimestampType

type TimestampType struct{}

TimestampType represents a number of microseconds since the unix epoch without regard for timezone.

func (TimestampType) Equals

func (TimestampType) Equals(other Type) bool

func (TimestampType) String

func (TimestampType) String() string

func (TimestampType) Type

func (TimestampType) Type() string

type TimestampTzNsType

type TimestampTzNsType struct{}

TimestampTzNsType represents a timestamp stored as UTC with nanoseconds since the unix epoch. Requires format version 3+.

func (TimestampTzNsType) Equals

func (TimestampTzNsType) Equals(other Type) bool

func (TimestampTzNsType) String

func (TimestampTzNsType) String() string

func (TimestampTzNsType) Type

func (TimestampTzNsType) Type() string

type TimestampTzType

type TimestampTzType struct{}

TimestampTzType represents a timestamp stored as UTC representing the number of microseconds since the unix epoch.

func (TimestampTzType) Equals

func (TimestampTzType) Equals(other Type) bool

func (TimestampTzType) String

func (TimestampTzType) String() string

func (TimestampTzType) Type

func (TimestampTzType) Type() string

type Transform

type Transform interface {
	fmt.Stringer
	encoding.TextMarshaler
	CanTransform(t Type) bool
	ResultType(t Type) Type
	PreservesOrder() bool
	Equals(Transform) bool
	Apply(Optional[Literal]) Optional[Literal]
	Project(name string, pred BoundPredicate) (UnboundPredicate, error)

	ToHumanStr(any) string
}

Transform is an interface for the various Transformation types in partition specs. Currently, they do not yet provide actual transformation functions or implementation. That will come later as data reading gets implemented.

func ParseTransform

func ParseTransform(s string) (Transform, error)

ParseTransform takes the string representation of a transform as defined in the iceberg spec, and produces the appropriate Transform object or an error if the string is not a valid transform string.

type TruncateTransform

type TruncateTransform struct {
	Width int
}

TruncateTransform is a transformation for truncating a value to a specified width.

func (TruncateTransform) Apply

func (t TruncateTransform) Apply(value Optional[Literal]) (out Optional[Literal])

func (TruncateTransform) CanTransform

func (TruncateTransform) CanTransform(t Type) bool

func (TruncateTransform) Equals

func (t TruncateTransform) Equals(other Transform) bool

func (TruncateTransform) MarshalText

func (t TruncateTransform) MarshalText() ([]byte, error)

func (TruncateTransform) PreservesOrder

func (TruncateTransform) PreservesOrder() bool

func (TruncateTransform) Project

func (TruncateTransform) ResultType

func (TruncateTransform) ResultType(t Type) Type

func (TruncateTransform) String

func (t TruncateTransform) String() string

func (TruncateTransform) ToHumanStr

func (TruncateTransform) ToHumanStr(val any) string

func (TruncateTransform) Transformer

func (t TruncateTransform) Transformer(src Type) (func(any) any, error)

type Type

type Type interface {
	fmt.Stringer
	Type() string
	Equals(Type) bool
}

Type is an interface representing any of the available iceberg types, such as primitives (int32/int64/etc.) or nested types (list/struct/map).

func PromoteType

func PromoteType(fileType, readType Type) (Type, error)

PromoteType promotes the type being read from a file to a requested read type. fileType is the type from the file being read readType is the requested readType

type TypedLiteral

type TypedLiteral[T LiteralType] interface {
	Literal

	Value() T
	Comparator() Comparator[T]
}

TypedLiteral is a generic interface for Literals so that you can retrieve the value. This is based on the physical representative type, which means that FixedLiteral and BinaryLiteral will both return []byte, etc.

type UUIDLiteral

type UUIDLiteral uuid.UUID

func (UUIDLiteral) Any

func (u UUIDLiteral) Any() any

func (UUIDLiteral) Comparator

func (UUIDLiteral) Comparator() Comparator[uuid.UUID]

func (UUIDLiteral) Equals

func (u UUIDLiteral) Equals(other Literal) bool

func (UUIDLiteral) MarshalBinary

func (u UUIDLiteral) MarshalBinary() (data []byte, err error)

func (UUIDLiteral) String

func (u UUIDLiteral) String() string

func (UUIDLiteral) To

func (u UUIDLiteral) To(typ Type) (Literal, error)

func (UUIDLiteral) Type

func (UUIDLiteral) Type() Type

func (*UUIDLiteral) UnmarshalBinary

func (u *UUIDLiteral) UnmarshalBinary(data []byte) error

func (UUIDLiteral) Value

func (u UUIDLiteral) Value() uuid.UUID

type UUIDType

type UUIDType struct{}

func (UUIDType) Equals

func (UUIDType) Equals(other Type) bool

func (UUIDType) String

func (UUIDType) String() string

func (UUIDType) Type

func (UUIDType) Type() string

type UnboundPredicate

type UnboundPredicate interface {
	BooleanExpression

	Term() UnboundTerm
	// contains filtered or unexported methods
}

An UnboundPredicate represents a boolean predicate expression which has not yet been bound to a schema. Binding it will produce a BooleanExpression.

BooleanExpression is used for the binding result because we may optimize and return AlwaysTrue / AlwaysFalse in some scenarios during binding which are not considered to be "Bound" as they do not have a bound Term or Reference.

func EqualTo

func EqualTo[T LiteralType](t UnboundTerm, v T) UnboundPredicate

EqualTo is a convenience wrapper for calling LiteralPredicate(OpEQ, t, NewLiteral(v))

Will panic if t is nil

func GreaterThan

func GreaterThan[T LiteralType](t UnboundTerm, v T) UnboundPredicate

GreaterThan is a convenience wrapper for calling LiteralPredicate(OpGT, t, NewLiteral(v))

Will panic if t is nil

func GreaterThanEqual

func GreaterThanEqual[T LiteralType](t UnboundTerm, v T) UnboundPredicate

GreaterThanEqual is a convenience wrapper for calling LiteralPredicate(OpGTEQ, t, NewLiteral(v))

Will panic if t is nil

func IsNaN

IsNaN is a convenience wrapper for calling UnaryPredicate(OpIsNan, t)

Will panic if t is nil

func IsNull

func IsNull(t UnboundTerm) UnboundPredicate

IsNull is a convenience wrapper for calling UnaryPredicate(OpIsNull, t)

Will panic if t is nil

func LessThan

func LessThan[T LiteralType](t UnboundTerm, v T) UnboundPredicate

LessThan is a convenience wrapper for calling LiteralPredicate(OpLT, t, NewLiteral(v))

Will panic if t is nil

func LessThanEqual

func LessThanEqual[T LiteralType](t UnboundTerm, v T) UnboundPredicate

LessThanEqual is a convenience wrapper for calling LiteralPredicate(OpLTEQ, t, NewLiteral(v))

Will panic if t is nil

func LiteralPredicate

func LiteralPredicate(op Operation, t UnboundTerm, lit Literal) UnboundPredicate

LiteralPredicate constructs an unbound predicate for an operation that requires a single literal argument, such as LessThan or StartsWith.

Panics if the operation provided is not a valid Literal operation, if the term is nil or if the literal is nil.

func NotEqualTo

func NotEqualTo[T LiteralType](t UnboundTerm, v T) UnboundPredicate

NotEqualTo is a convenience wrapper for calling LiteralPredicate(OpNEQ, t, NewLiteral(v))

Will panic if t is nil

func NotNaN

func NotNaN(t UnboundTerm) UnboundPredicate

NotNaN is a convenience wrapper for calling UnaryPredicate(OpNotNan, t)

Will panic if t is nil

func NotNull

func NotNull(t UnboundTerm) UnboundPredicate

NotNull is a convenience wrapper for calling UnaryPredicate(OpNotNull, t)

Will panic if t is nil

func NotStartsWith

func NotStartsWith(t UnboundTerm, v string) UnboundPredicate

NotStartsWith is a convenience wrapper for calling LiteralPredicate(OpNotStartsWith, t, NewLiteral(v))

Will panic if t is nil

func StartsWith

func StartsWith(t UnboundTerm, v string) UnboundPredicate

StartsWith is a convenience wrapper for calling LiteralPredicate(OpStartsWith, t, NewLiteral(v))

Will panic if t is nil

func UnaryPredicate

func UnaryPredicate(op Operation, t UnboundTerm) UnboundPredicate

UnaryPredicate creates and returns an unbound predicate for the provided unary operation. Will panic if op is not a unary operation.

type UnboundTerm

type UnboundTerm interface {
	Term

	Equals(UnboundTerm) bool
	Bind(schema *Schema, caseSensitive bool) (BoundTerm, error)
}

UnboundTerm is an expression that evaluates to a value that isn't yet bound to a schema, thus it isn't yet known what the type will be.

type Void

type Void = struct{}

type VoidTransform

type VoidTransform struct{}

VoidTransform is a transformation that always returns nil.

func (VoidTransform) Apply

func (VoidTransform) CanTransform

func (VoidTransform) CanTransform(Type) bool

func (VoidTransform) Equals

func (VoidTransform) Equals(other Transform) bool

func (VoidTransform) MarshalText

func (t VoidTransform) MarshalText() ([]byte, error)

func (VoidTransform) PreservesOrder

func (VoidTransform) PreservesOrder() bool

func (VoidTransform) Project

func (VoidTransform) ResultType

func (VoidTransform) ResultType(t Type) Type

func (VoidTransform) String

func (VoidTransform) String() string

func (VoidTransform) ToHumanStr

func (VoidTransform) ToHumanStr(any) string

type YearTransform

type YearTransform struct{}

YearTransform transforms a datetime value into a year value.

func (YearTransform) Apply

func (YearTransform) Apply(value Optional[Literal]) (out Optional[Literal])

func (YearTransform) CanTransform

func (t YearTransform) CanTransform(sourceType Type) bool

func (YearTransform) Equals

func (YearTransform) Equals(other Transform) bool

func (YearTransform) MarshalText

func (t YearTransform) MarshalText() ([]byte, error)

func (YearTransform) PreservesOrder

func (YearTransform) PreservesOrder() bool

func (YearTransform) Project

func (t YearTransform) Project(name string, pred BoundPredicate) (UnboundPredicate, error)

func (YearTransform) ResultType

func (YearTransform) ResultType(Type) Type

func (YearTransform) String

func (YearTransform) String() string

func (YearTransform) ToHumanStr

func (YearTransform) ToHumanStr(val any) string

func (YearTransform) Transformer

func (YearTransform) Transformer(src Type) (func(any) Optional[int32], error)

Directories

Path Synopsis
Package catalog provides an interface for Catalog implementations along with a registry for registering catalog implementations.
Package catalog provides an interface for Catalog implementations along with a registry for registering catalog implementations.
sql
cmd
iceberg command
Package cloud contains a library and tools for open cloud development in Go.
Package cloud contains a library and tools for open cloud development in Go.
aws
Package aws provides fundamental Wire providers for Amazon Web Services (AWS).
Package aws provides fundamental Wire providers for Amazon Web Services (AWS).
aws/awscloud
Package awscloud contains Wire providers for AWS services.
Package awscloud contains Wire providers for AWS services.
aws/rds
Package rds contains Wire providers that are common across RDS.
Package rds contains Wire providers that are common across RDS.
azure/azurecloud
Package azurecloud contains Wire providers for Azure services.
Package azurecloud contains Wire providers for Azure services.
azure/azuredb
Package azuredb contains Wire providers that are common across Azure Database.
Package azuredb contains Wire providers that are common across Azure Database.
blob
Package blob provides an easy and portable way to interact with blobs within a storage location.
Package blob provides an easy and portable way to interact with blobs within a storage location.
blob/azureblob
Package azureblob provides a blob implementation that uses Azure Storage’s BlockBlob.
Package azureblob provides a blob implementation that uses Azure Storage’s BlockBlob.
blob/driver
Package driver defines interfaces to be implemented by blob drivers, which will be used by the blob package to interact with the underlying services.
Package driver defines interfaces to be implemented by blob drivers, which will be used by the blob package to interact with the underlying services.
blob/drivertest
Package drivertest provides a conformance test for implementations of driver.
Package drivertest provides a conformance test for implementations of driver.
blob/fileblob
Package fileblob provides a blob implementation that uses the filesystem.
Package fileblob provides a blob implementation that uses the filesystem.
blob/gcsblob
Package gcsblob provides a blob implementation that uses GCS.
Package gcsblob provides a blob implementation that uses GCS.
blob/memblob
Package memblob provides an in-memory blob implementation.
Package memblob provides an in-memory blob implementation.
docstore
Package docstore provides a portable way of interacting with a document store.
Package docstore provides a portable way of interacting with a document store.
docstore/awsdynamodb
Package awsdynamodb provides a docstore implementation backed by Amazon DynamoDB.
Package awsdynamodb provides a docstore implementation backed by Amazon DynamoDB.
docstore/awsdynamodb/v2
Package awsdynamodb provides a docstore implementation backed by Amazon DynamoDB.
Package awsdynamodb provides a docstore implementation backed by Amazon DynamoDB.
docstore/driver
Package driver defines interfaces to be implemented by docstore drivers, which will be used by the docstore package to interact with the underlying services.
Package driver defines interfaces to be implemented by docstore drivers, which will be used by the docstore package to interact with the underlying services.
docstore/drivertest
Package drivertest provides a conformance test for implementations of driver.
Package drivertest provides a conformance test for implementations of driver.
docstore/gcpfirestore
Package gcpfirestore provides a docstore implementation backed by Google Cloud Firestore.
Package gcpfirestore provides a docstore implementation backed by Google Cloud Firestore.
docstore/internal/fields
Package fields provides a view of the fields of a struct that follows the Go rules, amended to consider tags and case insensitivity.
Package fields provides a view of the fields of a struct that follows the Go rules, amended to consider tags and case insensitivity.
docstore/memdocstore
Package memdocstore provides an in-process in-memory implementation of the docstore API.
Package memdocstore provides an in-process in-memory implementation of the docstore API.
gcerrors
Package gcerrors provides support for getting error codes from errors returned by Go CDK APIs.
Package gcerrors provides support for getting error codes from errors returned by Go CDK APIs.
gcp
Package gcp provides fundamental Wire providers and types for Google Cloud Platform (GCP).
Package gcp provides fundamental Wire providers and types for Google Cloud Platform (GCP).
gcp/cloudsql
Package cloudsql contains Wire providers that are common across Google Cloud SQL.
Package cloudsql contains Wire providers that are common across Google Cloud SQL.
gcp/gcpcloud
Package gcpcloud contains Wire providers for GCP services.
Package gcpcloud contains Wire providers for GCP services.
internal/escape
Package escape includes helpers for escaping and unescaping strings.
Package escape includes helpers for escaping and unescaping strings.
internal/gcerr
Package gcerr provides an error type for Go CDK APIs.
Package gcerr provides an error type for Go CDK APIs.
internal/openurl
Package openurl provides helpers for URLMux and URLOpeners in portable APIs.
Package openurl provides helpers for URLMux and URLOpeners in portable APIs.
internal/otel
Package otel supports OpenTelemetry tracing and metrics for the Go Cloud Development Kit.
Package otel supports OpenTelemetry tracing and metrics for the Go Cloud Development Kit.
internal/releasehelper command
Helper tool for creating new releases of the Go CDK.
Helper tool for creating new releases of the Go CDK.
internal/retry
Package retry provides retry logic.
Package retry provides retry logic.
internal/testing/oteltest
Package oteltest supports testing of OpenTelemetry integrations.
Package oteltest supports testing of OpenTelemetry integrations.
internal/testing/terraform
Package terraform provides a function to read Terraform output.
Package terraform provides a function to read Terraform output.
internal/testing/test-summary command
Summarizes the output of go test.
Summarizes the output of go test.
internal/useragent
Package useragent includes constants and utilitiesfor setting the User-Agent for Go CDK connections to GCP.
Package useragent includes constants and utilitiesfor setting the User-Agent for Go CDK connections to GCP.
mysql
Package mysql provides functions to open MySQL databases with OpenTelemetry instrumentation.
Package mysql provides functions to open MySQL databases with OpenTelemetry instrumentation.
mysql/awsmysql
Package awsmysql provides connections to AWS RDS MySQL instances.
Package awsmysql provides connections to AWS RDS MySQL instances.
mysql/azuremysql
Package azuremysql provides connections to Azure Database for MySQL.
Package azuremysql provides connections to Azure Database for MySQL.
mysql/gcpmysql
Package gcpmysql provides connections to managed MySQL Cloud SQL instances.
Package gcpmysql provides connections to managed MySQL Cloud SQL instances.
postgres
Package postgres provides functions to open PostgreSQL databases with OpenTelemetry instrumentation.
Package postgres provides functions to open PostgreSQL databases with OpenTelemetry instrumentation.
postgres/awspostgres
Package awspostgres provides connections to AWS RDS PostgreSQL instances.
Package awspostgres provides connections to AWS RDS PostgreSQL instances.
postgres/gcppostgres
Package gcppostgres provides connections to managed PostgreSQL Cloud SQL instances.
Package gcppostgres provides connections to managed PostgreSQL Cloud SQL instances.
pubsub
Package pubsub provides an easy and portable way to interact with publish/subscribe systems.
Package pubsub provides an easy and portable way to interact with publish/subscribe systems.
pubsub/awssnssqs
Package awssnssqs provides two implementations of pubsub.Topic, one that sends messages to AWS SNS (Simple Notification Service), and one that sends messages to SQS (Simple Queuing Service).
Package awssnssqs provides two implementations of pubsub.Topic, one that sends messages to AWS SNS (Simple Notification Service), and one that sends messages to SQS (Simple Queuing Service).
pubsub/azuresb
Package azuresb provides an implementation of pubsub using Azure Service Bus Topic and Subscription.
Package azuresb provides an implementation of pubsub using Azure Service Bus Topic and Subscription.
pubsub/batcher
Package batcher supports batching of items.
Package batcher supports batching of items.
pubsub/driver
Package driver defines interfaces to be implemented by pubsub drivers, which will be used by the pubsub package to interact with the underlying services.
Package driver defines interfaces to be implemented by pubsub drivers, which will be used by the pubsub package to interact with the underlying services.
pubsub/drivertest
Package drivertest provides a conformance test for implementations of driver.
Package drivertest provides a conformance test for implementations of driver.
pubsub/gcppubsub
Package gcppubsub provides a pubsub implementation that uses GCP PubSub.
Package gcppubsub provides a pubsub implementation that uses GCP PubSub.
pubsub/mempubsub
Package mempubsub provides an in-memory pubsub implementation.
Package mempubsub provides an in-memory pubsub implementation.
runtimevar
Package runtimevar provides an easy and portable way to watch runtime configuration variables.
Package runtimevar provides an easy and portable way to watch runtime configuration variables.
runtimevar/awsparamstore
Package awsparamstore provides a runtimevar implementation with variables read from AWS Systems Manager Parameter Store (https://docs.aws.amazon.com/systems-manager/latest/userguide/systems-manager-paramstore.html) Use OpenVariable to construct a *runtimevar.Variable.
Package awsparamstore provides a runtimevar implementation with variables read from AWS Systems Manager Parameter Store (https://docs.aws.amazon.com/systems-manager/latest/userguide/systems-manager-paramstore.html) Use OpenVariable to construct a *runtimevar.Variable.
runtimevar/awssecretsmanager
Package awssecretsmanager provides a runtimevar implementation with variables read from AWS Secrets Manager (https://aws.amazon.com/secrets-manager) Use OpenVariable to construct a *runtimevar.Variable.
Package awssecretsmanager provides a runtimevar implementation with variables read from AWS Secrets Manager (https://aws.amazon.com/secrets-manager) Use OpenVariable to construct a *runtimevar.Variable.
runtimevar/blobvar
Package blobvar provides a runtimevar implementation with variables read from a blob.Bucket.
Package blobvar provides a runtimevar implementation with variables read from a blob.Bucket.
runtimevar/constantvar
Package constantvar provides a runtimevar implementation with Variables that never change.
Package constantvar provides a runtimevar implementation with Variables that never change.
runtimevar/driver
Package driver defines interfaces to be implemented by runtimevar drivers, which will be used by the runtimevar package to interact with the underlying services.
Package driver defines interfaces to be implemented by runtimevar drivers, which will be used by the runtimevar package to interact with the underlying services.
runtimevar/drivertest
Package drivertest provides a conformance test for implementations of runtimevar.
Package drivertest provides a conformance test for implementations of runtimevar.
runtimevar/filevar
Package filevar provides a runtimevar implementation with variables backed by the filesystem.
Package filevar provides a runtimevar implementation with variables backed by the filesystem.
runtimevar/gcpruntimeconfig
Package gcpruntimeconfig provides a runtimevar implementation with variables read from GCP Cloud Runtime Configurator (https://cloud.google.com/deployment-manager/runtime-configurator).
Package gcpruntimeconfig provides a runtimevar implementation with variables read from GCP Cloud Runtime Configurator (https://cloud.google.com/deployment-manager/runtime-configurator).
runtimevar/gcpsecretmanager
Package gcpsecretmanager provides a runtimevar implementation with secrets read from GCP Secret Manager (https://cloud.google.com/secret-manager).
Package gcpsecretmanager provides a runtimevar implementation with secrets read from GCP Secret Manager (https://cloud.google.com/secret-manager).
runtimevar/httpvar
Package httpvar provides a runtimevar implementation with variables backed by http endpoint.
Package httpvar provides a runtimevar implementation with variables backed by http endpoint.
secrets
Package secrets provides an easy and portable way to encrypt and decrypt messages.
Package secrets provides an easy and portable way to encrypt and decrypt messages.
secrets/awskms
Package awskms provides a secrets implementation backed by AWS KMS.
Package awskms provides a secrets implementation backed by AWS KMS.
secrets/azurekeyvault
Package azurekeyvault provides a secrets implementation backed by Azure KeyVault.
Package azurekeyvault provides a secrets implementation backed by Azure KeyVault.
secrets/driver
Package driver defines interfaces to be implemented by secrets drivers, which will be used by the secrets package to interact with the underlying services.
Package driver defines interfaces to be implemented by secrets drivers, which will be used by the secrets package to interact with the underlying services.
secrets/drivertest
Package drivertest provides a conformance test for implementations of the secrets driver.
Package drivertest provides a conformance test for implementations of the secrets driver.
secrets/gcpkms
Package gcpkms provides a secrets implementation backed by Google Cloud KMS.
Package gcpkms provides a secrets implementation backed by Google Cloud KMS.
secrets/localsecrets
Package localsecrets provides a secrets implementation using a locally provided symmetric key.
Package localsecrets provides a secrets implementation using a locally provided symmetric key.
server
Package server provides a preconfigured HTTP server with diagnostic hooks.
Package server provides a preconfigured HTTP server with diagnostic hooks.
server/driver
Package driver defines an interface for custom HTTP listeners.
Package driver defines an interface for custom HTTP listeners.
server/health
Package health provides health check handlers.
Package health provides health check handlers.
server/health/sqlhealth
Package sqlhealth provides a health check for a SQL database connection.
Package sqlhealth provides a health check for a SQL database connection.
server/requestlog
Package requestlog provides an http.Handler that logs information about requests.
Package requestlog provides an http.Handler that logs information about requests.
server/sdserver
Package sdserver provides the diagnostic hooks for a server using Stackdriver.
Package sdserver provides the diagnostic hooks for a server using Stackdriver.
server/xrayserver
Package xrayserver provides the diagnostic hooks for a server using ADOT collector.
Package xrayserver provides the diagnostic hooks for a server using ADOT collector.

Jump to

Keyboard shortcuts

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