table

package
v0.0.0-...-e13d839 Latest Latest
Warning

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

Go to latest
Published: Apr 30, 2024 License: Apache-2.0 Imports: 21 Imported by: 1

Documentation

Index

Constants

View Source
const (
	InitialSortOrderID  = 1
	UnsortedSortOrderID = 0
)
View Source
const MainBranch = "main"

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 (
	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 UnsortedSortOrder = SortOrder{OrderID: UnsortedSortOrderID, Fields: []SortField{}}

A default Sort Order indicating no sort order at all

Functions

This section is empty.

Types

type Identifier

type Identifier = []string

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
	// SchemaID returns the ID of the current schema.
	SchemaID() int
	// 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
	// SnapshotID returns the ID of the current snapshot.
	SnapshotID() *int64
	// 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
	// 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
	// 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

	// GetSnapshotLog is a list of snapshot log entries. This is used to track
	GetSnapshotLog() []SnapshotLogEntry
	// GetMetadataLog is a list of metadata log entries. This is used to track
	GetMetadataLog() []MetadataLogEntry
	// SortOrderID returns the ID of the current sort order.
	SortOrderID() int
	// SnapshotRefs is a map of snapshot ref names to snapshot refs. This is used to
	SnapshotRefs() map[string]SnapshotRef
}

Metadata for an iceberg table as specified in the Iceberg spec

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

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.

type MetadataLogEntry

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

type MetadataV1

type MetadataV1 struct {
	Schema    *iceberg.Schema          `json:"schema"`
	Partition []iceberg.PartitionField `json:"partition-spec"`
	// contains filtered or unexported fields
}

func (*MetadataV1) CurrentSchema

func (c *MetadataV1) CurrentSchema() *iceberg.Schema

func (*MetadataV1) CurrentSnapshot

func (c *MetadataV1) CurrentSnapshot() *Snapshot

func (*MetadataV1) DefaultPartitionSpec

func (c *MetadataV1) DefaultPartitionSpec() int

func (*MetadataV1) GetMetadataLog

func (c *MetadataV1) GetMetadataLog() []MetadataLogEntry

func (*MetadataV1) GetSnapshotLog

func (c *MetadataV1) GetSnapshotLog() []SnapshotLogEntry

func (*MetadataV1) LastColumnID

func (c *MetadataV1) LastColumnID() int

func (*MetadataV1) LastPartitionSpecID

func (c *MetadataV1) LastPartitionSpecID() *int

func (*MetadataV1) LastUpdatedMillis

func (c *MetadataV1) LastUpdatedMillis() int64

func (*MetadataV1) Location

func (c *MetadataV1) Location() string

func (*MetadataV1) PartitionSpec

func (c *MetadataV1) PartitionSpec() iceberg.PartitionSpec

func (*MetadataV1) PartitionSpecs

func (c *MetadataV1) PartitionSpecs() []iceberg.PartitionSpec

func (*MetadataV1) Properties

func (c *MetadataV1) Properties() iceberg.Properties

func (*MetadataV1) SchemaID

func (c *MetadataV1) SchemaID() int

func (*MetadataV1) Schemas

func (c *MetadataV1) Schemas() []*iceberg.Schema

func (*MetadataV1) SnapshotByID

func (c *MetadataV1) SnapshotByID(id int64) *Snapshot

func (*MetadataV1) SnapshotByName

func (c *MetadataV1) SnapshotByName(name string) *Snapshot

func (*MetadataV1) SnapshotID

func (c *MetadataV1) SnapshotID() *int64

func (*MetadataV1) SnapshotRefs

func (c *MetadataV1) SnapshotRefs() map[string]SnapshotRef

func (*MetadataV1) Snapshots

func (c *MetadataV1) Snapshots() []Snapshot

func (*MetadataV1) SortOrder

func (c *MetadataV1) SortOrder() SortOrder

func (*MetadataV1) SortOrderID

func (c *MetadataV1) SortOrderID() int

func (*MetadataV1) SortOrders

func (c *MetadataV1) SortOrders() []SortOrder

func (*MetadataV1) TableUUID

func (c *MetadataV1) TableUUID() uuid.UUID

func (*MetadataV1) ToV2

func (m *MetadataV1) ToV2() MetadataV2

func (*MetadataV1) UnmarshalJSON

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

func (*MetadataV1) Version

func (c *MetadataV1) Version() int

type MetadataV1Builder

type MetadataV1Builder struct {
	*MetadataV1
}

func CloneMetadataV1

func CloneMetadataV1(m Metadata) *MetadataV1Builder

func NewMetadataV1Builder

func NewMetadataV1Builder(
	location string,
	schema *iceberg.Schema,
	lastUpdatesMs int64,
	lastColumnId int,
) *MetadataV1Builder

NewMetadataV1Builder returns a new MetadataV1Builder.

func (*MetadataV1Builder) Build

func (b *MetadataV1Builder) Build() Metadata

func (MetadataV1Builder) CurrentSnapshot

func (c MetadataV1Builder) CurrentSnapshot() *Snapshot

func (MetadataV1Builder) DefaultPartitionSpec

func (c MetadataV1Builder) DefaultPartitionSpec() int

func (MetadataV1Builder) GetMetadataLog

func (c MetadataV1Builder) GetMetadataLog() []MetadataLogEntry

func (MetadataV1Builder) GetSnapshotLog

func (c MetadataV1Builder) GetSnapshotLog() []SnapshotLogEntry

func (MetadataV1Builder) LastColumnID

func (c MetadataV1Builder) LastColumnID() int

func (MetadataV1Builder) LastPartitionSpecID

func (c MetadataV1Builder) LastPartitionSpecID() *int

func (MetadataV1Builder) LastUpdatedMillis

func (c MetadataV1Builder) LastUpdatedMillis() int64

func (MetadataV1Builder) Location

func (c MetadataV1Builder) Location() string

func (MetadataV1Builder) PartitionSpec

func (c MetadataV1Builder) PartitionSpec() iceberg.PartitionSpec

func (MetadataV1Builder) PartitionSpecs

func (c MetadataV1Builder) PartitionSpecs() []iceberg.PartitionSpec

func (MetadataV1Builder) Properties

func (c MetadataV1Builder) Properties() iceberg.Properties

func (MetadataV1Builder) SchemaID

func (c MetadataV1Builder) SchemaID() int

func (MetadataV1Builder) Schemas

func (c MetadataV1Builder) Schemas() []*iceberg.Schema

func (MetadataV1Builder) SnapshotByID

func (c MetadataV1Builder) SnapshotByID(id int64) *Snapshot

func (MetadataV1Builder) SnapshotByName

func (c MetadataV1Builder) SnapshotByName(name string) *Snapshot

func (MetadataV1Builder) SnapshotID

func (c MetadataV1Builder) SnapshotID() *int64

func (MetadataV1Builder) SnapshotRefs

func (c MetadataV1Builder) SnapshotRefs() map[string]SnapshotRef

func (MetadataV1Builder) Snapshots

func (c MetadataV1Builder) Snapshots() []Snapshot

func (MetadataV1Builder) SortOrder

func (c MetadataV1Builder) SortOrder() SortOrder

func (MetadataV1Builder) SortOrderID

func (c MetadataV1Builder) SortOrderID() int

func (MetadataV1Builder) SortOrders

func (c MetadataV1Builder) SortOrders() []SortOrder

func (MetadataV1Builder) TableUUID

func (c MetadataV1Builder) TableUUID() uuid.UUID

func (MetadataV1Builder) Version

func (c MetadataV1Builder) Version() int

func (*MetadataV1Builder) WithCurrentSchemaID

func (b *MetadataV1Builder) WithCurrentSchemaID(currentSchemaID int) *MetadataV1Builder

WithCurrentSchemaID sets the optional current-schema-id field of the metadata.

func (*MetadataV1Builder) WithCurrentSnapshotID

func (b *MetadataV1Builder) WithCurrentSnapshotID(currentSnapshotID int64) *MetadataV1Builder

WithCurrentSnapshotID sets the optional current-snapshot-id field of the metadata.

func (*MetadataV1Builder) WithDefaultSortOrderID

func (b *MetadataV1Builder) WithDefaultSortOrderID(defaultSortOrderID int) *MetadataV1Builder

WithDefaultSortOrderID sets the optional default-sort-order-id field of the metadata.

func (*MetadataV1Builder) WithDefaultSpecID

func (b *MetadataV1Builder) WithDefaultSpecID(defaultSpecID int) *MetadataV1Builder

WithDefaultSpecID sets the optional default-spec-id field of the metadata.

func (*MetadataV1Builder) WithFormatVersion

func (b *MetadataV1Builder) WithFormatVersion(version int) *MetadataV1Builder

func (*MetadataV1Builder) WithLastPartitionID

func (b *MetadataV1Builder) WithLastPartitionID(lastPartitionID int) *MetadataV1Builder

WithLastPartitionID sets the optional last-partition-id field of the metadata.

func (*MetadataV1Builder) WithLastUpdatedMs

func (b *MetadataV1Builder) WithLastUpdatedMs(lastUpdatedMs int64) *MetadataV1Builder

func (*MetadataV1Builder) WithLocation

func (b *MetadataV1Builder) WithLocation(location string) *MetadataV1Builder

func (*MetadataV1Builder) WithMetadataLog

func (b *MetadataV1Builder) WithMetadataLog(metadataLog []MetadataLogEntry) *MetadataV1Builder

WithMetadataLog sets the optional metadata-log field of the metadata.

func (*MetadataV1Builder) WithPartitionSpecs

func (b *MetadataV1Builder) WithPartitionSpecs(specs []iceberg.PartitionSpec) *MetadataV1Builder

WithProperties sets the optional partition-specs field of the metadata.

func (*MetadataV1Builder) WithProperties

func (b *MetadataV1Builder) WithProperties(properties iceberg.Properties) *MetadataV1Builder

WithProperties sets the optional properties field of the metadata.

func (*MetadataV1Builder) WithSchema

func (b *MetadataV1Builder) WithSchema(schema *iceberg.Schema) *MetadataV1Builder

func (*MetadataV1Builder) WithSchemas

func (b *MetadataV1Builder) WithSchemas(schemas []*iceberg.Schema) *MetadataV1Builder

WithSchemas sets the optional schemas field of the metadata.

func (*MetadataV1Builder) WithSnapshotLog

func (b *MetadataV1Builder) WithSnapshotLog(snapshotLog []SnapshotLogEntry) *MetadataV1Builder

WithSnapshotLog sets the optional snapshot-log field of the metadata.

func (*MetadataV1Builder) WithSnapshots

func (b *MetadataV1Builder) WithSnapshots(snapshots []Snapshot) *MetadataV1Builder

WithSnapshots sets the optional snapshots field of the metadata.

func (*MetadataV1Builder) WithSortOrders

func (b *MetadataV1Builder) WithSortOrders(sortOrders []SortOrder) *MetadataV1Builder

WithSortOrders sets the optional sort-orders field of the metadata.

func (*MetadataV1Builder) WithTableUUID

func (b *MetadataV1Builder) WithTableUUID(id uuid.UUID) *MetadataV1Builder

WithTableUUID sets the optional table-uuid field of the metadata.

type MetadataV2

type MetadataV2 struct {
	LastSequenceNumber int `json:"last-sequence-number"`
	// contains filtered or unexported fields
}

func (*MetadataV2) CurrentSchema

func (c *MetadataV2) CurrentSchema() *iceberg.Schema

func (*MetadataV2) CurrentSnapshot

func (c *MetadataV2) CurrentSnapshot() *Snapshot

func (*MetadataV2) DefaultPartitionSpec

func (c *MetadataV2) DefaultPartitionSpec() int

func (*MetadataV2) GetMetadataLog

func (c *MetadataV2) GetMetadataLog() []MetadataLogEntry

func (*MetadataV2) GetSnapshotLog

func (c *MetadataV2) GetSnapshotLog() []SnapshotLogEntry

func (*MetadataV2) LastColumnID

func (c *MetadataV2) LastColumnID() int

func (*MetadataV2) LastPartitionSpecID

func (c *MetadataV2) LastPartitionSpecID() *int

func (*MetadataV2) LastUpdatedMillis

func (c *MetadataV2) LastUpdatedMillis() int64

func (*MetadataV2) Location

func (c *MetadataV2) Location() string

func (*MetadataV2) PartitionSpec

func (c *MetadataV2) PartitionSpec() iceberg.PartitionSpec

func (*MetadataV2) PartitionSpecs

func (c *MetadataV2) PartitionSpecs() []iceberg.PartitionSpec

func (*MetadataV2) Properties

func (c *MetadataV2) Properties() iceberg.Properties

func (*MetadataV2) SchemaID

func (c *MetadataV2) SchemaID() int

func (*MetadataV2) Schemas

func (c *MetadataV2) Schemas() []*iceberg.Schema

func (*MetadataV2) SnapshotByID

func (c *MetadataV2) SnapshotByID(id int64) *Snapshot

func (*MetadataV2) SnapshotByName

func (c *MetadataV2) SnapshotByName(name string) *Snapshot

func (*MetadataV2) SnapshotID

func (c *MetadataV2) SnapshotID() *int64

func (*MetadataV2) SnapshotRefs

func (c *MetadataV2) SnapshotRefs() map[string]SnapshotRef

func (*MetadataV2) Snapshots

func (c *MetadataV2) Snapshots() []Snapshot

func (*MetadataV2) SortOrder

func (c *MetadataV2) SortOrder() SortOrder

func (*MetadataV2) SortOrderID

func (c *MetadataV2) SortOrderID() int

func (*MetadataV2) SortOrders

func (c *MetadataV2) SortOrders() []SortOrder

func (*MetadataV2) TableUUID

func (c *MetadataV2) TableUUID() uuid.UUID

func (*MetadataV2) UnmarshalJSON

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

func (*MetadataV2) Version

func (c *MetadataV2) Version() int

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 ReadOnlyTable

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

func (ReadOnlyTable) Bucket

func (t ReadOnlyTable) Bucket() objstore.Bucket

func (ReadOnlyTable) CurrentSnapshot

func (t ReadOnlyTable) CurrentSnapshot() *Snapshot

func (ReadOnlyTable) Equals

func (t ReadOnlyTable) Equals(other Table) bool

func (ReadOnlyTable) Identifier

func (t ReadOnlyTable) Identifier() Identifier

func (ReadOnlyTable) Location

func (t ReadOnlyTable) Location() string

func (ReadOnlyTable) Metadata

func (t ReadOnlyTable) Metadata() Metadata

func (ReadOnlyTable) MetadataLocation

func (t ReadOnlyTable) MetadataLocation() string

func (ReadOnlyTable) Properties

func (t ReadOnlyTable) Properties() iceberg.Properties

func (ReadOnlyTable) Schema

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

func (ReadOnlyTable) Schemas

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

func (ReadOnlyTable) SnapshotByID

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

func (ReadOnlyTable) SnapshotByName

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

func (*ReadOnlyTable) SnapshotWriter

func (r *ReadOnlyTable) SnapshotWriter(options ...WriterOption) (SnapshotWriter, error)

func (ReadOnlyTable) SortOrder

func (t ReadOnlyTable) SortOrder() SortOrder

func (ReadOnlyTable) Spec

func (t ReadOnlyTable) Spec() iceberg.PartitionSpec

type RefType

type RefType string

RefType will be either a BranchRef or a TagRef

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

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(bucket objstore.Bucket) ([]iceberg.ManifestFile, 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) UnmarshalJSON

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

type SnapshotWriter

type SnapshotWriter interface {
	// Append accepts a ReaderAt object that should read the Parquet file that is to be added to the snapshot.
	Append(ctx context.Context, r io.Reader) error

	// Close writes the new snapshot to the table and closes the writer. It is an error to call Append after Close.
	Close(ctx context.Context) error
}

SnapshotWriter is an interface for writing a new snapshot to a table.

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 {
	OrderID int         `json:"order-id"`
	Fields  []SortField `json:"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 (SortOrder) String

func (s SortOrder) String() string

func (*SortOrder) UnmarshalJSON

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

type Summary

type Summary struct {
	Operation  Operation
	Properties map[string]string
}

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 interface {
	Identifier() Identifier
	Metadata() Metadata
	MetadataLocation() string
	Bucket() objstore.Bucket
	Schema() *iceberg.Schema
	Spec() iceberg.PartitionSpec
	SortOrder() SortOrder
	Properties() iceberg.Properties
	Location() string
	CurrentSnapshot() *Snapshot
	SnapshotByID(id int64) *Snapshot
	SnapshotByName(name string) *Snapshot
	Schemas() map[int]*iceberg.Schema
	Equals(other Table) bool

	SnapshotWriter(options ...WriterOption) (SnapshotWriter, error)
}

func New

func New(ident Identifier, meta Metadata, location string, bucket objstore.Bucket) Table

func NewFromLocation

func NewFromLocation(ident Identifier, metalocation string, bucket objstore.Bucket) (Table, error)

func NewHDFSTable

func NewHDFSTable(ver int, ident Identifier, meta Metadata, location string, bucket objstore.Bucket) Table

type WriterOption

type WriterOption func(*writerOptions)

func WithExpireSnapshotsOlderThan

func WithExpireSnapshotsOlderThan(d time.Duration) WriterOption

func WithFastAppend

func WithFastAppend() WriterOption

func WithManifestSizeBytes

func WithManifestSizeBytes(size int) WriterOption

func WithMergeSchema

func WithMergeSchema() WriterOption

func WithMetadataDeleteAfterCommit

func WithMetadataDeleteAfterCommit() WriterOption

func WithMetadataPreviousVersionsMax

func WithMetadataPreviousVersionsMax(n int) WriterOption

Jump to

Keyboard shortcuts

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