table

package
v0.4.0 Latest Latest
Warning

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

Go to latest
Published: Oct 9, 2025 License: Apache-2.0 Imports: 43 Imported by: 19

Documentation

Index

Constants

View Source
const (
	ArrowFieldDocKey = "doc"
	// Arrow schemas that are generated from the Parquet library will utilize
	// this key to identify the field id of the source Parquet field.
	// We use this when converting to Iceberg to provide field IDs
	ArrowParquetFieldIDKey = "PARQUET:field_id"
)

constants to look for as Keys in Arrow field metadata

View Source
const (
	WriteDataPathKey                        = "write.data.path"
	WriteMetadataPathKey                    = "write.metadata.path"
	WriteObjectStorePartitionedPathsKey     = "write.object-storage.partitioned-paths"
	WriteObjectStorePartitionedPathsDefault = true
	ObjectStoreEnabledKey                   = "write.object-storage.enabled"
	ObjectStoreEnabledDefault               = false

	DefaultNameMappingKey = "schema.name-mapping.default"

	MetricsModeColumnConfPrefix    = "write.metadata.metrics.column"
	DefaultWriteMetricsModeKey     = "write.metadata.metrics.default"
	DefaultWriteMetricsModeDefault = "truncate(16)"

	ParquetRowGroupSizeBytesKey              = internal.ParquetRowGroupSizeBytesKey
	ParquetRowGroupSizeBytesDefault          = internal.ParquetRowGroupSizeBytesDefault
	ParquetRowGroupLimitKey                  = internal.ParquetRowGroupLimitKey
	ParquetRowGroupLimitDefault              = internal.ParquetRowGroupLimitDefault
	ParquetPageSizeBytesKey                  = internal.ParquetPageSizeBytesKey
	ParquetPageSizeBytesDefault              = internal.ParquetPageSizeBytesDefault
	ParquetPageRowLimitKey                   = internal.ParquetPageRowLimitKey
	ParquetPageRowLimitDefault               = internal.ParquetPageRowLimitDefault
	ParquetDictSizeBytesKey                  = internal.ParquetDictSizeBytesKey
	ParquetDictSizeBytesDefault              = internal.ParquetDictSizeBytesDefault
	ParquetCompressionKey                    = internal.ParquetCompressionKey
	ParquetCompressionDefault                = internal.ParquetCompressionDefault
	ParquetCompressionLevelKey               = internal.ParquetCompressionLevelKey
	ParquetCompressionLevelDefault           = internal.ParquetCompressionLevelDefault
	ParquetBloomFilterMaxBytesKey            = internal.ParquetBloomFilterMaxBytesKey
	ParquetBloomFilterMaxBytesDefault        = internal.ParquetBloomFilterMaxBytesDefault
	ParquetBloomFilterColumnEnabledKeyPrefix = internal.ParquetBloomFilterColumnEnabledKeyPrefix

	ManifestMergeEnabledKey     = "commit.manifest-merge.enabled"
	ManifestMergeEnabledDefault = false

	ManifestTargetSizeBytesKey     = "commit.manifest.target-size-bytes"
	ManifestTargetSizeBytesDefault = 8 * 1024 * 1024 // 8 MB

	ManifestMinMergeCountKey     = "commit.manifest.min-count-to-merge"
	ManifestMinMergeCountDefault = 100

	WritePartitionSummaryLimitKey     = "write.summary.partition-limit"
	WritePartitionSummaryLimitDefault = 0

	MetadataDeleteAfterCommitEnabledKey     = "write.metadata.delete-after-commit.enabled"
	MetadataDeleteAfterCommitEnabledDefault = false

	MetadataPreviousVersionsMaxKey     = "write.metadata.previous-versions-max"
	MetadataPreviousVersionsMaxDefault = 100

	WriteTargetFileSizeBytesKey     = "write.target-file-size-bytes"
	WriteTargetFileSizeBytesDefault = 512 * 1024 * 1024 // 512 MB

	MinSnapshotsToKeepKey     = "min-snapshots-to-keep"
	MinSnapshotsToKeepDefault = math.MaxInt

	MaxSnapshotAgeMsKey     = "max-snapshot-age-ms"
	MaxSnapshotAgeMsDefault = math.MaxInt

	MaxRefAgeMsKey     = "max-ref-age-ms"
	MaxRefAgeMsDefault = math.MaxInt
)
View Source
const (
	PropertyFormatVersion            = "format-version"
	PropertyUuid                     = "uuid"
	PropertySnapshotCount            = "snapshot-count"
	PropertyCurrentSnapshotId        = "current-snapshot-id"
	PropertyCurrentSnapshotSummary   = "current-snapshot-summary"
	PropertyCurrentSnapshotTimestamp = "current-snapshot-timestamp"
	PropertyCurrentSchema            = "current-schema"
	PropertyDefaultPartitionSpec     = "default-partition-spec"
	PropertyDefaultSortOrder         = "default-sort-order"
)

Reserved properties

View Source
const (
	InitialSortOrderID  = 1
	UnsortedSortOrderID = 0
)
View Source
const (
	UpdateAddSpec      = "add-spec"
	UpdateAddSchema    = "add-schema"
	UpdateAddSnapshot  = "add-snapshot"
	UpdateAddSortOrder = "add-sort-order"

	UpdateAssignUUID = "assign-uuid"

	UpdateRemoveProperties  = "remove-properties"
	UpdateRemoveSchemas     = "remove-schemas"
	UpdateRemoveSnapshots   = "remove-snapshots"
	UpdateRemoveSnapshotRef = "remove-snapshot-ref"
	UpdateRemoveSpec        = "remove-partition-specs"

	UpdateSetCurrentSchema    = "set-current-schema"
	UpdateSetDefaultSortOrder = "set-default-sort-order"
	UpdateSetDefaultSpec      = "set-default-spec"
	UpdateSetLocation         = "set-location"
	UpdateSetProperties       = "set-properties"
	UpdateSetSnapshotRef      = "set-snapshot-ref"

	UpdateUpgradeFormatVersion = "upgrade-format-version"
)

These are the various update actions defined in the iceberg spec

View Source
const DefaultFormatVersion = 2
View Source
const MainBranch = "main"
View Source
const ScanNoLimit = -1
View Source
const (
	ScanOptionArrowUseLargeTypes = "arrow.use_large_types"
)

Variables

View Source
var (
	ErrInvalidMetadataFormatVersion = errors.New("invalid or missing format-version in table metadata")
	ErrInvalidMetadata              = errors.New("invalid metadata")
)
View Source
var (
	ErrInvalidOperation = errors.New("invalid operation value")
	ErrMissingOperation = errors.New("missing operation key")
)
View Source
var (
	ErrInvalidSortOrderID   = errors.New("invalid sort order ID")
	ErrInvalidTransform     = errors.New("invalid transform, must be a valid transform string or a transform object")
	ErrInvalidSortDirection = errors.New("invalid sort direction, must be 'asc' or 'desc'")
	ErrInvalidNullOrder     = errors.New("invalid null order, must be 'nulls-first' or 'nulls-last'")
)
View Source
var ErrInvalidRefType = errors.New("invalid snapshot ref type, should be 'branch' or 'tag'")
View Source
var ErrInvalidRequirement = errors.New("invalid requirement")
View Source
var UnsortedSortOrder = SortOrder{/* contains filtered or unexported fields */}

A default Sort Order indicating no sort order at all

Functions

func ArrowSchemaToIceberg

func ArrowSchemaToIceberg(sc *arrow.Schema, downcastNsTimestamp bool, nameMapping iceberg.NameMapping) (*iceberg.Schema, error)

func ArrowSchemaToIcebergWithFreshIDs added in v0.3.0

func ArrowSchemaToIcebergWithFreshIDs(sc *arrow.Schema, downcastNsTimestamp bool) (*iceberg.Schema, error)

func ArrowTypeToIceberg

func ArrowTypeToIceberg(dt arrow.DataType, downcastNsTimestamp bool) (iceberg.Type, error)

func NewAddPartitionSpecUpdate added in v0.2.0

func NewAddPartitionSpecUpdate(spec *iceberg.PartitionSpec, initial bool) *addPartitionSpecUpdate

NewAddPartitionSpecUpdate creates a new update that adds the given partition spec to the table metadata. If the initial flag is set to true, the spec is considered the initial spec of the table, and all other previously added specs in the metadata builder are removed.

func NewAddSchemaUpdate added in v0.2.0

func NewAddSchemaUpdate(schema *iceberg.Schema) *addSchemaUpdate

NewAddSchemaUpdate creates a new update that adds the given schema and updates the lastColumnID based on the schema.

func NewAddSnapshotUpdate added in v0.2.0

func NewAddSnapshotUpdate(snapshot *Snapshot) *addSnapshotUpdate

NewAddSnapshotUpdate creates a new update that adds the given snapshot to the table metadata.

func NewAddSortOrderUpdate added in v0.2.0

func NewAddSortOrderUpdate(sortOrder *SortOrder) *addSortOrderUpdate

NewAddSortOrderUpdate creates a new update that adds the given sort order to the table metadata. If the initial flag is set to true, the sort order is considered the initial sort order of the table, and all previously added sort orders in the metadata builder are removed.

func NewAssignUUIDUpdate added in v0.2.0

func NewAssignUUIDUpdate(uuid uuid.UUID) *assignUUIDUpdate

NewAssignUUIDUpdate creates a new update to assign a UUID to the table metadata.

func NewRemovePropertiesUpdate added in v0.2.0

func NewRemovePropertiesUpdate(removals []string) *removePropertiesUpdate

NewRemovePropertiesUpdate creates a new update that removes properties from the table metadata. The properties are identified by their names, and if a property with the given name does not exist, it is ignored.

func NewRemoveSchemasUpdate added in v0.4.0

func NewRemoveSchemasUpdate(schemaIds []int) *removeSchemasUpdate

NewRemoveSchemasUpdate creates a new Update that removes a list of schemas from the table metadata.

func NewRemoveSnapshotRefUpdate added in v0.2.0

func NewRemoveSnapshotRefUpdate(ref string) *removeSnapshotRefUpdate

NewRemoveSnapshotRefUpdate creates a new update that removes a snapshot reference from the table metadata.

func NewRemoveSnapshotsUpdate added in v0.2.0

func NewRemoveSnapshotsUpdate(ids []int64) *removeSnapshotsUpdate

NewRemoveSnapshotsUpdate creates a new update that removes all snapshots from the table metadata with the given snapshot IDs.

func NewRemoveSpecUpdate added in v0.4.0

func NewRemoveSpecUpdate(specIds []int) *removeSpecUpdate

NewRemoveSpecUpdate creates a new Update that removes a list of partition specs from the table metadata.

func NewSetCurrentSchemaUpdate added in v0.2.0

func NewSetCurrentSchemaUpdate(id int) *setCurrentSchemaUpdate

NewSetCurrentSchemaUpdate creates a new update that sets the current schema of the table metadata to the given schema ID.

func NewSetDefaultSortOrderUpdate added in v0.2.0

func NewSetDefaultSortOrderUpdate(id int) *setDefaultSortOrderUpdate

NewSetDefaultSortOrderUpdate creates a new update that sets the default sort order of the table metadata to the given sort order ID.

func NewSetDefaultSpecUpdate added in v0.2.0

func NewSetDefaultSpecUpdate(id int) *setDefaultSpecUpdate

NewSetDefaultSpecUpdate creates a new update that sets the default partition spec of the table metadata to the given spec ID.

func NewSetPropertiesUpdate added in v0.2.0

func NewSetPropertiesUpdate(updates iceberg.Properties) *setPropertiesUpdate

NewSetPropertiesUpdate creates a new update that sets the given properties in the table metadata.

func NewSetSnapshotRefUpdate added in v0.2.0

func NewSetSnapshotRefUpdate(
	name string,
	snapshotID int64,
	refType RefType,
	maxRefAgeMs, maxSnapshotAgeMs int64,
	minSnapshotsToKeep int,
) *setSnapshotRefUpdate

NewSetSnapshotRefUpdate creates a new update that sets the given snapshot reference as the current snapshot of the table metadata. MaxRefAgeMs, MaxSnapshotAgeMs, and MinSnapshotsToKeep are optional, and any non-positive values are ignored.

func NewUpgradeFormatVersionUpdate added in v0.2.0

func NewUpgradeFormatVersionUpdate(formatVersion int) *upgradeFormatVersionUpdate

NewUpgradeFormatVersionUpdate creates a new update that upgrades the format version of the table metadata to the given formatVersion.

func NewWriterFactory added in v0.4.0

func NewWriterFactory(rootLocation string, args recordWritingArgs, meta *MetadataBuilder, taskSchema *iceberg.Schema, targetFileSize int64) writerFactory

NewWriterFactory creates a new WriterFactory with the specified configuration for managing rolling data writers across partitions.

func SchemaToArrowSchema

func SchemaToArrowSchema(sc *iceberg.Schema, metadata map[string]string, includeFieldIDs, useLargeTypes bool) (*arrow.Schema, error)

SchemaToArrowSchema converts an Iceberg schema to an Arrow schema. If the metadata parameter is non-nil, it will be included as the top-level metadata in the schema. If includeFieldIDs is true, then each field of the schema will contain a metadata key PARQUET:field_id set to the field id from the iceberg schema.

func ToRequestedSchema

func ToRequestedSchema(ctx context.Context, requested, fileSchema *iceberg.Schema, batch arrow.RecordBatch, downcastTimestamp, includeFieldIDs, useLargeTypes bool) (arrow.RecordBatch, error)

ToRequestedSchema will construct a new record batch matching the requested iceberg schema casting columns if necessary as appropriate.

func TypeToArrowType

func TypeToArrowType(t iceberg.Type, includeFieldIDs bool, useLargeTypes bool) (arrow.DataType, error)

TypeToArrowType converts a given iceberg type, into the equivalent Arrow data type. For dealing with nested fields (List, Struct, Map) if includeFieldIDs is true, then the child fields will contain a metadata key PARQUET:field_id set to the field id.

func VisitArrowSchema

func VisitArrowSchema[T any](sc *arrow.Schema, visitor ArrowSchemaVisitor[T]) (res T, err error)

func WithMaxRefAgeMs added in v0.2.0

func WithMaxRefAgeMs(maxRefAgeMs int64) setSnapshotRefOption

func WithMaxSnapshotAgeMs added in v0.2.0

func WithMaxSnapshotAgeMs(maxSnapshotAgeMs int64) setSnapshotRefOption

func WithMinSnapshotsToKeep added in v0.2.0

func WithMinSnapshotsToKeep(minSnapshotsToKeep int) setSnapshotRefOption

Types

type ArrowSchemaVisitor

type ArrowSchemaVisitor[T any] interface {
	Schema(*arrow.Schema, T) T
	Struct(*arrow.StructType, []T) T
	Field(arrow.Field, T) T
	List(arrow.ListLikeType, T) T
	Map(mt *arrow.MapType, keyResult T, valueResult T) T
	Primitive(arrow.DataType) T
}

ArrowSchemaVisitor is an interface that can be implemented and used to call VisitArrowSchema for iterating

type BlobMetadata added in v0.4.0

type BlobMetadata struct {
	Type           BlobType          `json:"type"`
	SnapshotID     int64             `json:"snapshot-id"`
	SequenceNumber int64             `json:"sequence-number"`
	Fields         []int32           `json:"fields"`
	Properties     map[string]string `json:"properties"`
}

BlobMetadata is the metadata of a statistics or indices blob.

type BlobType added in v0.4.0

type BlobType string

BlobType is the type of blob in a Puffin file

const (
	BlobTypeApacheDatasketchesThetaV1 BlobType = "apache-datasketches-theta-v1"
	BlobTypeDeletionVectorV1          BlobType = "deletion-vector-v1"
)

func (*BlobType) IsValid added in v0.4.0

func (bt *BlobType) IsValid() bool

func (*BlobType) UnmarshalJSON added in v0.4.0

func (bt *BlobType) UnmarshalJSON(data []byte) error

type CatalogIO added in v0.2.0

type CatalogIO interface {
	LoadTable(context.Context, Identifier) (*Table, error)
	CommitTable(context.Context, Identifier, []Requirement, []Update) (Metadata, string, error)
}

type ExpireSnapshotsOpt added in v0.4.0

type ExpireSnapshotsOpt func(*expireSnapshotsCfg)

func WithOlderThan added in v0.4.0

func WithOlderThan(t time.Duration) ExpireSnapshotsOpt

func WithRetainLast added in v0.4.0

func WithRetainLast(n int) ExpireSnapshotsOpt

type FSysF added in v0.4.0

type FSysF func(ctx context.Context) (io.IO, error)

type FileScanTask

type FileScanTask struct {
	File          iceberg.DataFile
	DeleteFiles   []iceberg.DataFile
	Start, Length int64
}

type Identifier

type Identifier = []string

type LocationProvider added in v0.2.0

type LocationProvider interface {
	NewDataLocation(dataFileName string) string
	NewTableMetadataFileLocation(newVersion int) (string, error)
	NewMetadataLocation(metadataFileName string) string
}

func LoadLocationProvider added in v0.2.0

func LoadLocationProvider(tableLocation string, tableProps iceberg.Properties) (LocationProvider, error)

type Metadata

type Metadata interface {
	// Version indicates the version of this metadata, 1 for V1, 2 for V2, etc.
	Version() int
	// TableUUID returns a UUID that identifies the table, generated when the
	// table is created. Implementations must throw an exception if a table's
	// UUID does not match the expected UUID after refreshing metadata.
	TableUUID() uuid.UUID
	// Location is the table's base location. This is used by writers to determine
	// where to store data files, manifest files, and table metadata files.
	Location() string
	// LastUpdatedMillis is the timestamp in milliseconds from the unix epoch when
	// the table was last updated. Each table metadata file should update this
	// field just before writing.
	LastUpdatedMillis() int64
	// LastColumnID returns the highest assigned column ID for the table.
	// This is used to ensure fields are always assigned an unused ID when
	// evolving schemas.
	LastColumnID() int
	// Schemas returns the list of schemas, stored as objects with their
	// schema-id.
	Schemas() []*iceberg.Schema
	// CurrentSchema returns the table's current schema.
	CurrentSchema() *iceberg.Schema
	// PartitionSpecs returns the list of all partition specs in the table.
	PartitionSpecs() []iceberg.PartitionSpec
	// PartitionSpec returns the current partition spec that the table is using.
	PartitionSpec() iceberg.PartitionSpec
	// DefaultPartitionSpec is the ID of the current spec that writers should
	// use by default.
	DefaultPartitionSpec() int
	// LastPartitionSpecID is the highest assigned partition field ID across
	// all partition specs for the table. This is used to ensure partition
	// fields are always assigned an unused ID when evolving specs.
	LastPartitionSpecID() *int
	// Snapshots returns the list of valid snapshots. Valid snapshots are
	// snapshots for which all data files exist in the file system. A data
	// file must not be deleted from the file system until the last snapshot
	// in which it was listed is garbage collected.
	Snapshots() []Snapshot
	// SnapshotByID find and return a specific snapshot by its ID. Returns
	// nil if the ID is not found in the list of snapshots.
	SnapshotByID(int64) *Snapshot
	// SnapshotByName searches the list of snapshots for a snapshot with a given
	// ref name. Returns nil if there's no ref with this name for a snapshot.
	SnapshotByName(name string) *Snapshot
	// CurrentSnapshot returns the table's current snapshot.
	CurrentSnapshot() *Snapshot
	// Ref returns the snapshot ref for the main branch.
	Ref() SnapshotRef
	// Refs returns a list of snapshot name/reference pairs.
	Refs() iter.Seq2[string, SnapshotRef]
	// SnapshotLogs returns the list of snapshot logs for the table.
	SnapshotLogs() iter.Seq[SnapshotLogEntry]
	// SortOrder returns the table's current sort order, ie: the one with the
	// ID that matches the default-sort-order-id.
	SortOrder() SortOrder
	// SortOrders returns the list of sort orders in the table.
	SortOrders() []SortOrder
	// DefaultSortOrder returns the ID of the current sort order that writers
	// should use by default.
	DefaultSortOrder() int
	// Properties is a string to string map of table properties. This is used
	// to control settings that affect reading and writing and is not intended
	// to be used for arbitrary metadata. For example, commit.retry.num-retries
	// is used to control the number of commit retries.
	Properties() iceberg.Properties
	// PreviousFiles returns the list of metadata log entries for the table.
	PreviousFiles() iter.Seq[MetadataLogEntry]
	Equals(Metadata) bool

	NameMapping() iceberg.NameMapping

	LastSequenceNumber() int64
	// Statistics returns an optional list of table statistics.
	// Table statistics files are valid Puffin files.
	// StatisticsFile are informational. A reader can choose to ignore statistics information.
	// StatisticsFile support is not required to read the table correctly.
	// A table can contain many statistics files associated with different table snapshots.
	Statistics() iter.Seq[StatisticsFile]
	// PartitionStatistics returns an optional list of partition statistics files.
	// Partition statistics are not required for reading or planning
	// and readers may ignore them. Each table snapshot may be associated
	// with at most one partition statistics file. A writer can optionally
	// write the partition statistics file during each write operation,
	// or it can also be computed on demand.
	PartitionStatistics() iter.Seq[PartitionStatisticsFile]
}

Metadata for an iceberg table as specified in the Iceberg spec

https://iceberg.apache.org/spec/#iceberg-table-spec

func NewMetadata added in v0.2.0

func NewMetadata(sc *iceberg.Schema, partitions *iceberg.PartitionSpec, sortOrder SortOrder, location string, props iceberg.Properties) (Metadata, error)

NewMetadata creates a new table metadata object using the provided schema, information, generating a fresh UUID for the new table metadata. By default, this will generate a V2 table metadata, but this can be modified by adding a "format-version" property to the props map. An error will be returned if the "format-version" property exists and is not a valid version number.

func NewMetadataWithUUID added in v0.2.0

func NewMetadataWithUUID(sc *iceberg.Schema, partitions *iceberg.PartitionSpec, sortOrder SortOrder, location string, props iceberg.Properties, tableUuid uuid.UUID) (Metadata, error)

NewMetadataWithUUID is like NewMetadata, but allows the caller to specify the UUID of the table rather than creating a new one.

func ParseMetadata

func ParseMetadata(r io.Reader) (Metadata, error)

ParseMetadata parses json metadata provided by the passed in reader, returning an error if one is encountered.

func ParseMetadataBytes

func ParseMetadataBytes(b []byte) (Metadata, error)

ParseMetadataBytes is like ParseMetadataString but for a byte slice.

func ParseMetadataString

func ParseMetadataString(s string) (Metadata, error)

ParseMetadataString is like ParseMetadata, but for a string rather than an io.Reader.

func UpdateTableMetadata added in v0.4.0

func UpdateTableMetadata(base Metadata, updates []Update, metadataLoc string) (Metadata, error)

type MetadataBuilder added in v0.2.0

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

func MetadataBuilderFromBase added in v0.2.0

func MetadataBuilderFromBase(metadata Metadata, currentFileLocation string) (*MetadataBuilder, error)

MetadataBuilderFromBase creates a MetadataBuilder from an existing Metadata object. currentFileLocation is the location where the current version of the metadata file is stored. This is used to update the metadata log. If currentFileLocation is empty, the metadata log will not be updated. This should only be used to stage-create tables.

func NewMetadataBuilder added in v0.2.0

func NewMetadataBuilder(formatVersion int) (*MetadataBuilder, error)

func (*MetadataBuilder) AddPartitionSpec added in v0.2.0

func (b *MetadataBuilder) AddPartitionSpec(spec *iceberg.PartitionSpec, initial bool) error

func (*MetadataBuilder) AddSchema added in v0.2.0

func (b *MetadataBuilder) AddSchema(schema *iceberg.Schema) error

func (*MetadataBuilder) AddSnapshot added in v0.2.0

func (b *MetadataBuilder) AddSnapshot(snapshot *Snapshot) error

func (*MetadataBuilder) AddSortOrder added in v0.2.0

func (b *MetadataBuilder) AddSortOrder(sortOrder *SortOrder) error

func (*MetadataBuilder) AppendMetadataLog added in v0.2.0

func (b *MetadataBuilder) AppendMetadataLog(entry MetadataLogEntry) *MetadataBuilder

func (*MetadataBuilder) Build added in v0.2.0

func (b *MetadataBuilder) Build() (Metadata, error)

func (*MetadataBuilder) CurrentSchema added in v0.2.0

func (b *MetadataBuilder) CurrentSchema() *iceberg.Schema

func (*MetadataBuilder) CurrentSpec added in v0.2.0

func (b *MetadataBuilder) CurrentSpec() (*iceberg.PartitionSpec, error)

func (*MetadataBuilder) GetSchemaByID added in v0.2.0

func (b *MetadataBuilder) GetSchemaByID(id int) (*iceberg.Schema, error)

func (*MetadataBuilder) GetSortOrderByID added in v0.2.0

func (b *MetadataBuilder) GetSortOrderByID(id int) (*SortOrder, error)

func (*MetadataBuilder) GetSpecByID added in v0.2.0

func (b *MetadataBuilder) GetSpecByID(id int) (*iceberg.PartitionSpec, error)

func (*MetadataBuilder) HasChanges added in v0.2.0

func (b *MetadataBuilder) HasChanges() bool

func (*MetadataBuilder) LastUpdatedMS added in v0.2.0

func (b *MetadataBuilder) LastUpdatedMS() int64

func (*MetadataBuilder) NameMapping added in v0.2.0

func (b *MetadataBuilder) NameMapping() iceberg.NameMapping

func (*MetadataBuilder) RemovePartitionSpecs added in v0.4.0

func (b *MetadataBuilder) RemovePartitionSpecs(ints []int) error

func (*MetadataBuilder) RemoveProperties added in v0.2.0

func (b *MetadataBuilder) RemoveProperties(keys []string) error

func (*MetadataBuilder) RemoveSchemas added in v0.4.0

func (b *MetadataBuilder) RemoveSchemas(ints []int) error

func (*MetadataBuilder) RemoveSnapshotRef added in v0.4.0

func (b *MetadataBuilder) RemoveSnapshotRef(name string) error

func (*MetadataBuilder) RemoveSnapshots added in v0.4.0

func (b *MetadataBuilder) RemoveSnapshots(snapshotIds []int64) error

func (*MetadataBuilder) SetCurrentSchemaID added in v0.2.0

func (b *MetadataBuilder) SetCurrentSchemaID(currentSchemaID int) error

func (*MetadataBuilder) SetDefaultSortOrderID added in v0.2.0

func (b *MetadataBuilder) SetDefaultSortOrderID(defaultSortOrderID int) error

func (*MetadataBuilder) SetDefaultSpecID added in v0.2.0

func (b *MetadataBuilder) SetDefaultSpecID(defaultSpecID int) error

func (*MetadataBuilder) SetFormatVersion added in v0.2.0

func (b *MetadataBuilder) SetFormatVersion(formatVersion int) error

func (*MetadataBuilder) SetLastUpdatedMS added in v0.2.0

func (b *MetadataBuilder) SetLastUpdatedMS() *MetadataBuilder

func (*MetadataBuilder) SetLoc added in v0.2.0

func (b *MetadataBuilder) SetLoc(loc string) error

func (*MetadataBuilder) SetProperties added in v0.2.0

func (b *MetadataBuilder) SetProperties(props iceberg.Properties) error

func (*MetadataBuilder) SetSnapshotRef added in v0.2.0

func (b *MetadataBuilder) SetSnapshotRef(
	name string,
	snapshotID int64,
	refType RefType,
	options ...setSnapshotRefOption,
) error

func (*MetadataBuilder) SetUUID added in v0.2.0

func (b *MetadataBuilder) SetUUID(uuid uuid.UUID) error

func (*MetadataBuilder) SnapshotByID added in v0.2.0

func (b *MetadataBuilder) SnapshotByID(id int64) (*Snapshot, error)

func (*MetadataBuilder) TrimMetadataLogs added in v0.2.0

func (b *MetadataBuilder) TrimMetadataLogs(maxEntries int) *MetadataBuilder

type MetadataLogEntry

type MetadataLogEntry struct {
	MetadataFile string `json:"metadata-file"`
	TimestampMs  int64  `json:"timestamp-ms"`
}

type NullOrder

type NullOrder string
const (
	NullsFirst NullOrder = "nulls-first"
	NullsLast  NullOrder = "nulls-last"
)

type Operation

type Operation string
const (
	OpAppend    Operation = "append"
	OpReplace   Operation = "replace"
	OpOverwrite Operation = "overwrite"
	OpDelete    Operation = "delete"
)

func ValidOperation

func ValidOperation(s string) (Operation, error)

ValidOperation ensures that a given string is one of the valid operation types: append,replace,overwrite,delete

type OrphanCleanupOption added in v0.4.0

type OrphanCleanupOption func(*orphanCleanupConfig)

func WithDeleteFunc added in v0.4.0

func WithDeleteFunc(deleteFunc func(string) error) OrphanCleanupOption

WithDeleteFunc sets a custom delete function. If not provided, the table's FileIO delete method will be used.

func WithDryRun added in v0.4.0

func WithDryRun(enabled bool) OrphanCleanupOption

func WithEqualAuthorities added in v0.4.0

func WithEqualAuthorities(authorities map[string]string) OrphanCleanupOption

WithEqualAuthorities specifies authorities that should be considered equivalent. For example, map["endpoint1.s3.amazonaws.com,endpoint2.s3.amazonaws.com"] = "s3.amazonaws.com" treats different S3 endpoints as equivalent. The key can be a comma-separated list.

func WithEqualSchemes added in v0.4.0

func WithEqualSchemes(schemes map[string]string) OrphanCleanupOption

WithEqualSchemes specifies schemes that should be considered equivalent. For example, map["s3,s3a,s3n"] = "s3" treats all S3 scheme variants as equivalent. The key can be a comma-separated list of schemes that map to the value scheme.

func WithFilesOlderThan added in v0.4.0

func WithFilesOlderThan(duration time.Duration) OrphanCleanupOption

func WithLocation added in v0.4.0

func WithLocation(location string) OrphanCleanupOption

func WithMaxConcurrency added in v0.4.0

func WithMaxConcurrency(maxWorkers int) OrphanCleanupOption

WithMaxConcurrency sets the maximum number of goroutines for parallel deletion. Defaults to a reasonable number based on the system. Only used when deleteFunc is nil or when the FileIO doesn't support bulk operations.

func WithPrefixMismatchMode added in v0.4.0

func WithPrefixMismatchMode(mode PrefixMismatchMode) OrphanCleanupOption

WithPrefixMismatchMode sets how to handle situations when metadata references files that match listed files except for authority/scheme differences.

type OrphanCleanupResult added in v0.4.0

type OrphanCleanupResult struct {
	OrphanFileLocations []string
	DeletedFiles        []string
	TotalSizeBytes      int64
}

type PartitionStatisticsFile added in v0.4.0

type PartitionStatisticsFile struct {
	SnapshotID      int64  `json:"snapshot-id"`
	StatisticsPath  string `json:"statistics-path"`
	FileSizeInBytes int64  `json:"file-size-in-bytes"`
}

PartitionStatisticsFile represents a partition statistics file that can be used to read table data more efficiently.

Statistics are informational. A reader can choose to ignore statistics information. Statistics support is not required to read the table correctly.

type PrefixMismatchMode added in v0.4.0

type PrefixMismatchMode int

PrefixMismatchMode defines how to handle cases where candidate files have different URI schemes or authorities compared to table location during orphan cleanup. This is useful when files may be referenced using different but equivalent schemes (e.g., s3:// vs s3a:// vs s3n://) or when cleaning up files across different locations.

const (
	// PrefixMismatchError causes cleanup to fail with an error when candidate files
	// have URI schemes/authorities that don't match the table location and are not
	// covered by configured equivalences. This is the safest default behavior.
	PrefixMismatchError PrefixMismatchMode = iota // default

	// PrefixMismatchIgnore skips candidate files that have mismatched URI schemes/authorities
	// without treating it as an error. Files are silently ignored and not considered for deletion.
	PrefixMismatchIgnore

	// PrefixMismatchDelete treats candidate files with mismatched URI schemes/authorities
	// as orphans and includes them for deletion. Use with caution as this may delete
	// files from unexpected locations.
	PrefixMismatchDelete
)

func (PrefixMismatchMode) String added in v0.4.0

func (p PrefixMismatchMode) String() string

type ReassignedIds added in v0.4.0

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

type RefType

type RefType string

RefType will be either a BranchRef or a TagRef

const (
	BranchRef RefType = "branch"
	TagRef    RefType = "tag"
)

type Requirement added in v0.2.0

type Requirement interface {
	// Validate checks that the current table metadata satisfies the requirement.
	Validate(Metadata) error
	GetType() string
}

A Requirement is a validation rule that must be satisfied before attempting to make and commit changes to a table. Requirements are used to ensure that the table is in a valid state before making changes.

func AssertCreate added in v0.2.0

func AssertCreate() Requirement

AssertCreate creates a requirement that the table does not already exist.

func AssertCurrentSchemaID added in v0.2.0

func AssertCurrentSchemaID(id int) Requirement

AssertCurrentSchemaId creates a requirement that the table's current schema ID matches the given id.

func AssertDefaultSortOrderID added in v0.2.0

func AssertDefaultSortOrderID(id int) Requirement

AssertDefaultSortOrderID creates a requirement that the table's default sort order ID matches the given id.

func AssertDefaultSpecID added in v0.2.0

func AssertDefaultSpecID(id int) Requirement

AssertDefaultSpecID creates a requirement that the table's default partition spec ID matches the given id.

func AssertLastAssignedFieldID added in v0.2.0

func AssertLastAssignedFieldID(id int) Requirement

AssertLastAssignedFieldID validates that the table's last assigned column ID matches the given id.

func AssertLastAssignedPartitionID added in v0.2.0

func AssertLastAssignedPartitionID(id int) Requirement

AssertLastAssignedPartitionID creates a requriement that the table's last assigned partition ID matches the given id.

func AssertRefSnapshotID added in v0.2.0

func AssertRefSnapshotID(ref string, id *int64) Requirement

AssertRefSnapshotID creates a requirement which ensures that the table branch or tag identified by the given ref must reference the given snapshot id. If the id is nil, the ref must not already exist.

func AssertTableUUID added in v0.2.0

func AssertTableUUID(uuid uuid.UUID) Requirement

AssertTableUUID creates a requirement that the table UUID matches the given UUID.

func ParseRequirement added in v0.3.0

func ParseRequirement(r io.Reader) (Requirement, error)

ParseRequirement parses json data provided by the reader into a Requirement

func ParseRequirementBytes added in v0.3.0

func ParseRequirementBytes(b []byte) (Requirement, error)

ParseRequirementBytes parses json bytes into a Requirement

func ParseRequirementString added in v0.3.0

func ParseRequirementString(s string) (Requirement, error)

ParseRequirementString parses json string into a Requirement

type Requirements added in v0.4.0

type Requirements []Requirement

func (*Requirements) UnmarshalJSON added in v0.4.0

func (r *Requirements) UnmarshalJSON(data []byte) error

type RollingDataWriter added in v0.4.0

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

RollingDataWriter accumulates Arrow records for a specific partition and flushes them to data files when the target file size is reached, implementing a rolling file strategy to manage file sizes.

func (*RollingDataWriter) Add added in v0.4.0

func (r *RollingDataWriter) Add(record arrow.RecordBatch) error

Add appends a record to the writer's buffer and flushes to a data file if the target file size is reached.

type Scan

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

func (*Scan) PlanFiles

func (scan *Scan) PlanFiles(ctx context.Context) ([]FileScanTask, error)

PlanFiles orchestrates the fetching and filtering of manifests, and then building a list of FileScanTasks that match the current Scan criteria.

func (*Scan) Projection

func (scan *Scan) Projection() (*iceberg.Schema, error)

func (*Scan) Snapshot

func (scan *Scan) Snapshot() *Snapshot

func (*Scan) ToArrowRecords

func (scan *Scan) ToArrowRecords(ctx context.Context) (*arrow.Schema, iter.Seq2[arrow.RecordBatch, error], error)

ToArrowRecords returns the arrow schema of the expected records and an interator that can be used with a range expression to read the records as they are available. If an error is encountered, during the planning and setup then this will return the error directly. If the error occurs while iterating the records, it will be returned by the iterator.

The purpose for returning the schema up front is to handle the case where there are no rows returned. The resulting Arrow Schema of the projection will still be known.

func (*Scan) ToArrowTable

func (scan *Scan) ToArrowTable(ctx context.Context) (arrow.Table, error)

ToArrowTable calls ToArrowRecords and then gathers all of the records together and returns an arrow.Table make from those records.

func (*Scan) UseRef

func (scan *Scan) UseRef(name string) (*Scan, error)

func (*Scan) UseRowLimit

func (scan *Scan) UseRowLimit(n int64) *Scan

type ScanOption

type ScanOption func(*Scan)

func WitMaxConcurrency

func WitMaxConcurrency(n int) ScanOption

WitMaxConcurrency sets the maximum concurrency for table scan and plan operations. When unset it defaults to runtime.GOMAXPROCS.

func WithCaseSensitive

func WithCaseSensitive(b bool) ScanOption

func WithLimit

func WithLimit(n int64) ScanOption

func WithOptions

func WithOptions(opts iceberg.Properties) ScanOption

func WithRowFilter

func WithRowFilter(e iceberg.BooleanExpression) ScanOption

func WithSelectedFields

func WithSelectedFields(fields ...string) ScanOption

func WithSnapshotAsOf added in v0.4.0

func WithSnapshotAsOf(timeStampMs int64) ScanOption

func WithSnapshotID

func WithSnapshotID(n int64) ScanOption

type Snapshot

type Snapshot struct {
	SnapshotID       int64    `json:"snapshot-id"`
	ParentSnapshotID *int64   `json:"parent-snapshot-id,omitempty"`
	SequenceNumber   int64    `json:"sequence-number"`
	TimestampMs      int64    `json:"timestamp-ms"`
	ManifestList     string   `json:"manifest-list,omitempty"`
	Summary          *Summary `json:"summary,omitempty"`
	SchemaID         *int     `json:"schema-id,omitempty"`
}

func (Snapshot) Equals

func (s Snapshot) Equals(other Snapshot) bool

func (Snapshot) Manifests

func (s Snapshot) Manifests(fio iceio.IO) (_ []iceberg.ManifestFile, err error)

func (Snapshot) String

func (s Snapshot) String() string

type SnapshotLogEntry

type SnapshotLogEntry struct {
	SnapshotID  int64 `json:"snapshot-id"`
	TimestampMs int64 `json:"timestamp-ms"`
}

type SnapshotRef

type SnapshotRef struct {
	SnapshotID         int64   `json:"snapshot-id"`
	SnapshotRefType    RefType `json:"type"`
	MinSnapshotsToKeep *int    `json:"min-snapshots-to-keep,omitempty"`
	MaxSnapshotAgeMs   *int64  `json:"max-snapshot-age-ms,omitempty"`
	MaxRefAgeMs        *int64  `json:"max-ref-age-ms,omitempty"`
}

SnapshotRef represents the reference information for a specific snapshot

func (*SnapshotRef) Equals

func (s *SnapshotRef) Equals(rhs SnapshotRef) bool

func (*SnapshotRef) UnmarshalJSON

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

type SnapshotSummaryCollector added in v0.2.0

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

type SortDirection

type SortDirection string
const (
	SortASC  SortDirection = "asc"
	SortDESC SortDirection = "desc"
)

type SortField

type SortField struct {
	// SourceID is the source column id from the table's schema
	SourceID int `json:"source-id"`
	// Transform is the tranformation used to produce values to be
	// sorted on from the source column.
	Transform iceberg.Transform `json:"transform"`
	// Direction is an enum indicating ascending or descending direction.
	Direction SortDirection `json:"direction"`
	// NullOrder describes the order of null values when sorting
	// should be only either nulls-first or nulls-last enum values.
	NullOrder NullOrder `json:"null-order"`
}

SortField describes a field used in a sort order definition.

func (*SortField) MarshalJSON

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

func (*SortField) String

func (s *SortField) String() string

func (*SortField) UnmarshalJSON

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

type SortOrder

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

SortOrder describes how the data is sorted within the table.

Data can be sorted within partitions by columns to gain performance. The order of the sort fields within the list defines the order in which the sort is applied to the data.

func AssignFreshSortOrderIDs added in v0.2.0

func AssignFreshSortOrderIDs(sortOrder SortOrder, old, fresh *iceberg.Schema) (SortOrder, error)

AssignFreshSortOrderIDs updates and reassigns the field source IDs from the old schema to the corresponding fields in the fresh schema, while also giving the Sort Order a fresh ID of 0 (the initial Sort Order ID).

func AssignFreshSortOrderIDsWithID added in v0.2.0

func AssignFreshSortOrderIDsWithID(sortOrder SortOrder, old, fresh *iceberg.Schema, sortOrderID int) (SortOrder, error)

AssignFreshSortOrderIDsWithID is like AssignFreshSortOrderIDs but allows specifying the id of the returned SortOrder.

func NewSortOrder added in v0.4.0

func NewSortOrder(orderID int, fields []SortField) (SortOrder, error)

NewSortOrder creates a new SortOrder.

The orderID must be greater than or equal to 0. If orderID is 0, no fields can be passed, this is equal to UnsortedSortOrder. Fields need to have non-nil Transform, valid Direction and NullOrder values.

func (*SortOrder) CheckCompatibility added in v0.4.0

func (s *SortOrder) CheckCompatibility(schema *iceberg.Schema) error

func (SortOrder) Equals

func (s SortOrder) Equals(rhs SortOrder) bool

func (SortOrder) Fields

func (s SortOrder) Fields() iter.Seq[SortField]

func (SortOrder) IsUnsorted added in v0.4.0

func (s SortOrder) IsUnsorted() bool

func (SortOrder) Len added in v0.4.0

func (s SortOrder) Len() int

func (SortOrder) MarshalJSON added in v0.4.0

func (s SortOrder) MarshalJSON() ([]byte, error)

func (SortOrder) OrderID

func (s SortOrder) OrderID() int

func (SortOrder) String

func (s SortOrder) String() string

func (*SortOrder) UnmarshalJSON

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

type StagedTable added in v0.2.0

type StagedTable struct {
	*Table
}

func (*StagedTable) Refresh added in v0.2.0

func (s *StagedTable) Refresh(ctx context.Context) (*Table, error)

func (*StagedTable) Scan added in v0.2.0

func (s *StagedTable) Scan(opts ...ScanOption) *Scan

type StatisticsFile added in v0.4.0

type StatisticsFile struct {
	SnapshotID            int64          `json:"snapshot-id"`
	StatisticsPath        string         `json:"statistics-path"`
	FileSizeInBytes       int64          `json:"file-size-in-bytes"`
	FileFooterSizeInBytes int64          `json:"file-footer-size-in-bytes"`
	KeyMetadata           *string        `json:"key-metadata,omitempty"`
	BlobMetadata          []BlobMetadata `json:"blob-metadata"`
}

StatisticsFile represents a statistics file in the Puffin format, that can be used to read table data more efficiently.

Statistics are informational. A reader can choose to ignore statistics information. Statistics support is not required to read the table correctly.

type Summary

type Summary struct {
	Operation  Operation
	Properties iceberg.Properties
}

Summary stores the summary information for a snapshot indicating the operation that created the snapshot, and various properties which might exist in the summary.

func (*Summary) Equals

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

func (*Summary) MarshalJSON

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

func (*Summary) String

func (s *Summary) String() string

func (*Summary) UnmarshalJSON

func (s *Summary) UnmarshalJSON(b []byte) (err error)

type Table

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

func New

func New(ident Identifier, meta Metadata, metadataLocation string, fsF FSysF, cat CatalogIO) *Table

func NewFromLocation

func NewFromLocation(
	ctx context.Context,
	ident Identifier,
	metalocation string,
	fsysF FSysF,
	cat CatalogIO,
) (_ *Table, err error)

func (Table) AllManifests added in v0.2.0

func (t Table) AllManifests(ctx context.Context) iter.Seq2[iceberg.ManifestFile, error]

func (Table) Append added in v0.3.0

func (t Table) Append(ctx context.Context, rdr array.RecordReader, snapshotProps iceberg.Properties) (*Table, error)

Append is a shortcut for NewTransaction().Append() and then committing the transaction

func (Table) AppendTable added in v0.3.0

func (t Table) AppendTable(ctx context.Context, tbl arrow.Table, batchSize int64, snapshotProps iceberg.Properties) (*Table, error)

AppendTable is a shortcut for NewTransaction().AppendTable() and then committing the transaction

func (Table) CurrentSnapshot

func (t Table) CurrentSnapshot() *Snapshot

func (Table) DeleteOrphanFiles added in v0.4.0

func (t Table) DeleteOrphanFiles(ctx context.Context, opts ...OrphanCleanupOption) (OrphanCleanupResult, error)

func (Table) Equals

func (t Table) Equals(other Table) bool

func (Table) FS

func (t Table) FS(ctx context.Context) (io.IO, error)

func (Table) Identifier

func (t Table) Identifier() Identifier

func (Table) Location

func (t Table) Location() string

func (Table) LocationProvider added in v0.2.0

func (t Table) LocationProvider() (LocationProvider, error)

func (Table) Metadata

func (t Table) Metadata() Metadata

func (Table) MetadataLocation

func (t Table) MetadataLocation() string

func (Table) NameMapping added in v0.2.0

func (t Table) NameMapping() iceberg.NameMapping

func (Table) NewTransaction added in v0.2.0

func (t Table) NewTransaction() *Transaction

func (Table) Properties

func (t Table) Properties() iceberg.Properties

func (*Table) Refresh added in v0.4.0

func (t *Table) Refresh(ctx context.Context) error

func (Table) Scan

func (t Table) Scan(opts ...ScanOption) *Scan

func (Table) Schema

func (t Table) Schema() *iceberg.Schema

func (Table) Schemas

func (t Table) Schemas() map[int]*iceberg.Schema

func (Table) SnapshotAsOf added in v0.4.0

func (t Table) SnapshotAsOf(timestampMs int64, inclusive bool) *Snapshot

SnapshotAsOf finds the snapshot that was current as of or right before the given timestamp.

func (Table) SnapshotByID

func (t Table) SnapshotByID(id int64) *Snapshot

func (Table) SnapshotByName

func (t Table) SnapshotByName(name string) *Snapshot

func (Table) SortOrder

func (t Table) SortOrder() SortOrder

func (Table) Spec

func (t Table) Spec() iceberg.PartitionSpec

type Transaction added in v0.2.0

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

func (*Transaction) AddFiles added in v0.2.0

func (t *Transaction) AddFiles(ctx context.Context, files []string, snapshotProps iceberg.Properties, ignoreDuplicates bool) error

func (*Transaction) Append added in v0.2.0

func (t *Transaction) Append(ctx context.Context, rdr array.RecordReader, snapshotProps iceberg.Properties) error

func (*Transaction) AppendTable added in v0.3.0

func (t *Transaction) AppendTable(ctx context.Context, tbl arrow.Table, batchSize int64, snapshotProps iceberg.Properties) error

func (*Transaction) Commit added in v0.2.0

func (t *Transaction) Commit(ctx context.Context) (*Table, error)

func (*Transaction) ExpireSnapshots added in v0.4.0

func (t *Transaction) ExpireSnapshots(opts ...ExpireSnapshotsOpt) error

func (*Transaction) ReplaceDataFiles added in v0.3.0

func (t *Transaction) ReplaceDataFiles(ctx context.Context, filesToDelete, filesToAdd []string, snapshotProps iceberg.Properties) error

ReplaceFiles is actually just an overwrite operation with multiple files deleted and added.

TODO: technically, this could be a REPLACE operation but we aren't performing any validation here that there are no changes to the underlying data. A REPLACE operation is only valid if the data is exactly the same as the previous snapshot.

For now, we'll keep using an overwrite operation.

func (*Transaction) Scan added in v0.2.0

func (t *Transaction) Scan(opts ...ScanOption) (*Scan, error)

func (*Transaction) SetProperties added in v0.2.0

func (t *Transaction) SetProperties(props iceberg.Properties) error

func (*Transaction) StagedTable added in v0.2.0

func (t *Transaction) StagedTable() (*StagedTable, error)

func (*Transaction) UpdateSpec added in v0.4.0

func (t *Transaction) UpdateSpec(caseSensitive bool) *UpdateSpec

type Update added in v0.2.0

type Update interface {
	// Action returns the name of the action that the update represents.
	Action() string
	// Apply applies the update to the given metadata builder.
	Apply(*MetadataBuilder) error
	// PostCommit is called after successful commit of the update
	PostCommit(context.Context, *Table, *Table) error
}

Update represents a change to a table's metadata.

func NewSetLocationUpdate added in v0.2.0

func NewSetLocationUpdate(loc string) Update

NewSetLocationUpdate creates a new update that sets the location of the table metadata.

type UpdateSpec added in v0.4.0

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

UpdateSpec implements a builder for evolving a table's partition specification.

It accumulates a sequence of partition spec update operations (e.g., AddField, RemoveField, RenameField) which are applied during BuildUpdates.

Use the builder methods to chain operations, and call BuildUpdates to apply them and produce the final set of partition fields and update requirements, or call Commit to apply the updates in the transaction.

func NewUpdateSpec added in v0.4.0

func NewUpdateSpec(t *Transaction, caseSensitive bool) *UpdateSpec

func (*UpdateSpec) AddField added in v0.4.0

func (us *UpdateSpec) AddField(sourceColName string, transform iceberg.Transform, partitionFieldName string) *UpdateSpec

func (*UpdateSpec) AddIdentity added in v0.4.0

func (us *UpdateSpec) AddIdentity(sourceColName string) *UpdateSpec

func (*UpdateSpec) Apply added in v0.4.0

func (us *UpdateSpec) Apply() (iceberg.PartitionSpec, error)

func (*UpdateSpec) BuildUpdates added in v0.4.0

func (us *UpdateSpec) BuildUpdates() ([]Update, []Requirement, error)

func (*UpdateSpec) Commit added in v0.4.0

func (us *UpdateSpec) Commit() error

func (*UpdateSpec) RemoveField added in v0.4.0

func (us *UpdateSpec) RemoveField(name string) *UpdateSpec

func (*UpdateSpec) RenameField added in v0.4.0

func (us *UpdateSpec) RenameField(name string, newName string) *UpdateSpec

type Updates added in v0.4.0

type Updates []Update

func (*Updates) UnmarshalJSON added in v0.4.0

func (u *Updates) UnmarshalJSON(data []byte) error

type WriteTask added in v0.3.0

type WriteTask struct {
	Uuid        uuid.UUID
	ID          int
	Schema      *iceberg.Schema
	Batches     []arrow.RecordBatch
	SortOrderID int
}

func (WriteTask) GenerateDataFileName added in v0.3.0

func (w WriteTask) GenerateDataFileName(extension string) string

Directories

Path Synopsis

Jump to

Keyboard shortcuts

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