From f9febaf04f17a5e73eae22b38707a495039e456b Mon Sep 17 00:00:00 2001 From: Jonas Weile Date: Mon, 9 Sep 2024 11:51:12 +0000 Subject: [PATCH 01/28] table implementations --- partitions.go | 4 + table/metadata.go | 558 +++++++++++++++++- ...data_test.go => metadata_internal_test.go} | 59 +- table/requirements.go | 212 +++++++ table/table_test.go | 4 +- table/updates.go | 301 ++++++++++ 6 files changed, 1080 insertions(+), 58 deletions(-) rename table/{metadata_test.go => metadata_internal_test.go} (93%) create mode 100644 table/requirements.go create mode 100644 table/updates.go diff --git a/partitions.go b/partitions.go index 321af2e..f39df67 100644 --- a/partitions.go +++ b/partitions.go @@ -117,6 +117,10 @@ func (ps PartitionSpec) Equals(other PartitionSpec) bool { return ps.id == other.id && slices.Equal(ps.fields, other.fields) } +func (ps *PartitionSpec) Fields() []PartitionField { + return ps.fields +} + func (ps PartitionSpec) MarshalJSON() ([]byte, error) { if ps.fields == nil { ps.fields = []PartitionField{} diff --git a/table/metadata.go b/table/metadata.go index 47b3ffe..7faeab5 100644 --- a/table/metadata.go +++ b/table/metadata.go @@ -24,12 +24,18 @@ import ( "io" "maps" "slices" + "time" "github.com/apache/iceberg-go" "github.com/google/uuid" ) +const ( + PARTITION_FIELD_ID_START = 1000 + SUPPORTED_TABLE_FORMAT_VERSION = 2 +) + // Metadata for an iceberg table as specified in the Iceberg spec // // https://iceberg.apache.org/spec/#iceberg-table-spec @@ -80,20 +86,512 @@ type Metadata interface { 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 map of snapshot refs by name. + Refs() map[string]SnapshotRef + // SnapshotLogs returns the list of snapshot logs for the table. + SnapshotLogs() []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() []MetadataLogEntry Equals(Metadata) bool } +type MetadataBuilder struct { + base Metadata + updates []Update + + // common fields + formatVersion int + uuid uuid.UUID + loc string + lastUpdatedMS int64 + lastColumnId int + schemaList []*iceberg.Schema + currentSchemaID int + specs []iceberg.PartitionSpec + defaultSpecID int + lastPartitionID *int + props iceberg.Properties + snapshotList []Snapshot + currentSnapshotID *int64 + snapshotLog []SnapshotLogEntry + metadataLog []MetadataLogEntry + sortOrderList []SortOrder + defaultSortOrderID int + refs map[string]SnapshotRef + + // V2 specific + lastSequenceNumber *int64 +} + +func NewMetadataBuilder() (*MetadataBuilder, error) { + return &MetadataBuilder{ + updates: make([]Update, 0), + schemaList: make([]*iceberg.Schema, 0), + specs: make([]iceberg.PartitionSpec, 0), + props: make(iceberg.Properties), + snapshotList: make([]Snapshot, 0), + snapshotLog: make([]SnapshotLogEntry, 0), + metadataLog: make([]MetadataLogEntry, 0), + sortOrderList: make([]SortOrder, 0), + refs: make(map[string]SnapshotRef), + }, nil +} + +func MetadataBuilderFromBase(metadata Metadata) (*MetadataBuilder, error) { + b := &MetadataBuilder{} + b.base = metadata + + b.formatVersion = metadata.Version() + b.uuid = metadata.TableUUID() + b.loc = metadata.Location() + b.lastUpdatedMS = metadata.LastUpdatedMillis() + b.lastColumnId = metadata.LastColumnID() + b.schemaList = metadata.Schemas() + b.currentSchemaID = metadata.CurrentSchema().ID + b.specs = metadata.PartitionSpecs() + b.defaultSpecID = metadata.DefaultPartitionSpec() + b.lastPartitionID = metadata.LastPartitionSpecID() + b.props = metadata.Properties() + b.snapshotList = metadata.Snapshots() + b.currentSnapshotID = &metadata.CurrentSnapshot().SnapshotID + b.sortOrderList = metadata.SortOrders() + b.defaultSortOrderID = metadata.DefaultSortOrder() + b.refs = metadata.Refs() + b.snapshotLog = metadata.SnapshotLogs() + b.metadataLog = metadata.PreviousFiles() + + return b, nil +} + +func (b *MetadataBuilder) AddSchema(schema *iceberg.Schema, newLastColumnID int) (*MetadataBuilder, error) { + if newLastColumnID < b.lastColumnId { + return nil, fmt.Errorf("invalid last column id %d, must be >= %d", + newLastColumnID, b.lastColumnId) + } + + b.updates = append(b.updates, NewAddSchemaUpdate(schema, newLastColumnID)) + b.lastColumnId = newLastColumnID + b.schemaList = append(b.schemaList, schema) + + return b, nil +} + +func (b *MetadataBuilder) AddPartitionSpec(spec *iceberg.PartitionSpec, initial bool) (*MetadataBuilder, error) { + for _, s := range b.specs { + if s.ID() == spec.ID() && !initial { + return nil, fmt.Errorf("partition spec with id %d already exists", spec.ID()) + } + } + + var maxFieldID int + if len(spec.Fields()) > 0 { + maxField := slices.MaxFunc(spec.Fields(), func(a, b iceberg.PartitionField) int { + return a.FieldID - b.FieldID + }) + maxFieldID = maxField.FieldID + } + + prev := PARTITION_FIELD_ID_START - 1 + if b.lastPartitionID != nil { + prev = *b.lastPartitionID + } + + lastPartitionID := max(maxFieldID, prev) + b.lastPartitionID = &lastPartitionID + b.specs = append(b.specs, *spec) + b.updates = append(b.updates, NewAddPartitionSpecUpdate(spec, initial)) + + return b, nil +} + +func (b *MetadataBuilder) AddSnapshot(snapshot *Snapshot) (*MetadataBuilder, error) { + if len(b.schemaList) == 0 { + return nil, errors.New("can't add snapshot with no added schemas") + } else if len(b.specs) == 0 { + return nil, errors.New("can't add snapshot with no added partition specs") + } else if len(b.sortOrderList) == 0 { + return nil, errors.New("can't add snapshot with no added sort orders") + } else if s, _ := b.SnapshotByID(snapshot.SnapshotID); s != nil { + return nil, fmt.Errorf("can't add snapshot with id %d, already exists", snapshot.SnapshotID) + } else if b.formatVersion == 2 && + snapshot.SequenceNumber > 0 && + snapshot.SequenceNumber <= *b.lastSequenceNumber && + snapshot.ParentSnapshotID != nil { + return nil, fmt.Errorf("can't add snapshot with sequence number %d, must be > than last sequence number %d", + snapshot.SequenceNumber, b.lastSequenceNumber) + } + + b.updates = append(b.updates, NewAddSnapshotUpdate(snapshot)) + b.lastUpdatedMS = snapshot.TimestampMs + b.lastSequenceNumber = &snapshot.SequenceNumber + b.snapshotList = append(b.snapshotList, *snapshot) + return b, nil +} + +func (b *MetadataBuilder) AddSortOrder(sortOrder *SortOrder) (*MetadataBuilder, error) { + b.updates = append(b.updates, NewAddSortOrderUpdate(sortOrder)) + b.sortOrderList = append(b.sortOrderList, *sortOrder) + return b, nil +} + +func (b *MetadataBuilder) RemoveProperties(keys []string) (*MetadataBuilder, error) { + if len(keys) == 0 { + return b, nil + } + + b.updates = append(b.updates, NewRemovePropertiesUpdate(keys)) + for _, key := range keys { + delete(b.props, key) + } + + return b, nil +} + +func (b *MetadataBuilder) SetCurrentSchemaID(currentSchemaID int) (*MetadataBuilder, error) { + if currentSchemaID == -1 { + currentSchemaID = b.MaxSchemaID() + if !b.isAddedSchemaID(currentSchemaID) { + return nil, errors.New("can't set current schema to last added schema, no schema has been added") + } + } + + if currentSchemaID == b.currentSchemaID { + return b, nil + } + + _, err := b.GetSchemaByID(currentSchemaID) + if err != nil { + return nil, fmt.Errorf("can't set current schema to schema with id %d: %w", currentSchemaID, err) + } + + b.updates = append(b.updates, NewSetCurrentSchemaUpdate(currentSchemaID)) + b.currentSchemaID = currentSchemaID + return b, nil +} + +func (b *MetadataBuilder) SetDefaultSortOrderID(defaultSortOrderID int) (*MetadataBuilder, error) { + if defaultSortOrderID == -1 { + defaultSortOrderID = b.MaxSortOrderID() + if !b.isAddedSortOrder(defaultSortOrderID) { + return nil, fmt.Errorf("can't set default sort order to last added with no added sort orders") + } + } + + if defaultSortOrderID == b.defaultSortOrderID { + return b, nil + } + + if _, err := b.GetSortOrderByID(defaultSortOrderID); err != nil { + return nil, fmt.Errorf("can't set default sort order to sort order with id %d: %w", defaultSortOrderID, err) + } + + b.updates = append(b.updates, NewSetDefaultSortOrderUpdate(defaultSortOrderID)) + b.defaultSortOrderID = defaultSortOrderID + return b, nil +} + +func (b *MetadataBuilder) SetDefaultSpecID(defaultSpecID int) (*MetadataBuilder, error) { + if defaultSpecID == -1 { + defaultSpecID = b.MaxSpecID() + if !b.isAddedSpecID(defaultSpecID) { + return nil, fmt.Errorf("can't set default spec to last added with no added partition specs") + } + } + + if defaultSpecID == b.defaultSpecID { + return b, nil + } + + if _, err := b.GetSpecByID(defaultSpecID); err != nil { + return nil, fmt.Errorf("can't set default spec to spec with id %d: %w", defaultSpecID, err) + } + + b.updates = append(b.updates, NewSetDefaultSpecUpdate(defaultSpecID)) + b.defaultSpecID = defaultSpecID + return b, nil +} + +func (b *MetadataBuilder) SetFormatVersion(formatVersion int) (*MetadataBuilder, error) { + if formatVersion < b.formatVersion { + return nil, fmt.Errorf("downgrading format version from %d to %d is not allowed", + b.formatVersion, formatVersion) + } + + if formatVersion > SUPPORTED_TABLE_FORMAT_VERSION { + return nil, fmt.Errorf("unsupported format version %d", formatVersion) + } + + if formatVersion == b.formatVersion { + return b, nil + } + + b.updates = append(b.updates, NewUpgradeFormatVersionUpdate(formatVersion)) + b.formatVersion = formatVersion + return b, nil +} + +func (b *MetadataBuilder) SetLoc(loc string) (*MetadataBuilder, error) { + if b.loc == loc { + return b, nil + } + + b.updates = append(b.updates, NewSetLocationUpdate(loc)) + b.loc = loc + return b, nil +} + +func (b *MetadataBuilder) SetProperties(props iceberg.Properties) (*MetadataBuilder, error) { + if len(props) == 0 { + return b, nil + } + + b.updates = append(b.updates, NewSetPropertiesUpdate(props)) + maps.Copy(b.props, props) + return b, nil +} + +func (b *MetadataBuilder) SetSnapshotRef( + name string, + snapshotID int64, + refType RefType, + maxRefAgeMs, maxSnapshotAgeMs *int64, + minSnapshotsToKeep *int, +) (*MetadataBuilder, error) { + ref := SnapshotRef{ + SnapshotID: snapshotID, + SnapshotRefType: refType, + MinSnapshotsToKeep: minSnapshotsToKeep, + MaxRefAgeMs: maxRefAgeMs, + MaxSnapshotAgeMs: maxSnapshotAgeMs, + } + + if existingRef, ok := b.refs[name]; ok && existingRef.Equals(ref) { + return b, nil + } + + snapshot, err := b.SnapshotByID(snapshotID) + if err != nil { + return nil, fmt.Errorf("can't set snapshot ref %s to unknown snapshot %d: %w", name, snapshotID, err) + } + + b.updates = append(b.updates, NewSetSnapshotRefUpdate(name, snapshotID, refType, maxRefAgeMs, maxSnapshotAgeMs, minSnapshotsToKeep)) + if refType == MainBranch { + b.currentSnapshotID = &snapshotID + b.snapshotLog = append(b.snapshotLog, SnapshotLogEntry{ + SnapshotID: snapshotID, + TimestampMs: snapshot.TimestampMs, + }) + b.lastUpdatedMS = time.Now().Local().UnixMilli() + } + + if b.isAddedSnapshot(snapshotID) { + b.lastUpdatedMS = snapshot.TimestampMs + } + + b.refs[name] = SnapshotRef{ + SnapshotID: snapshotID, + SnapshotRefType: refType, + MinSnapshotsToKeep: minSnapshotsToKeep, + MaxRefAgeMs: maxRefAgeMs, + MaxSnapshotAgeMs: maxSnapshotAgeMs, + } + return b, nil +} + +func (b *MetadataBuilder) SetUUID(uuid uuid.UUID) (*MetadataBuilder, error) { + if b.uuid == uuid { + return b, nil + } + + b.updates = append(b.updates, NewAssignUUIDUpdate(uuid)) + b.uuid = uuid + return b, nil +} + +func (b *MetadataBuilder) buildCommonMetadata() *commonMetadata { + return &commonMetadata{ + FormatVersion: b.formatVersion, + UUID: b.uuid, + Loc: b.loc, + LastUpdatedMS: b.lastUpdatedMS, + LastColumnId: b.lastColumnId, + SchemaList: b.schemaList, + CurrentSchemaID: b.currentSchemaID, + Specs: b.specs, + DefaultSpecID: b.defaultSpecID, + LastPartitionID: b.lastPartitionID, + Props: b.props, + SnapshotList: b.snapshotList, + CurrentSnapshotID: b.currentSnapshotID, + SnapshotLog: b.snapshotLog, + MetadataLog: b.metadataLog, + SortOrderList: b.sortOrderList, + DefaultSortOrderID: b.defaultSortOrderID, + SnapshotRefs: b.refs, + } +} + +func (b *MetadataBuilder) GetSchemaByID(id int) (*iceberg.Schema, error) { + for _, s := range b.schemaList { + if s.ID == id { + return s, nil + } + } + + return nil, fmt.Errorf("schema with id %d not found", id) +} + +func (b *MetadataBuilder) GetSpecByID(id int) (*iceberg.PartitionSpec, error) { + for _, s := range b.specs { + if s.ID() == id { + return &s, nil + } + } + + return nil, fmt.Errorf("partition spec with id %d not found", id) +} + +func (b *MetadataBuilder) GetSortOrderByID(id int) (*SortOrder, error) { + for _, s := range b.sortOrderList { + if s.OrderID == id { + return &s, nil + } + } + + return nil, fmt.Errorf("sort order with id %d not found", id) +} + +func (b *MetadataBuilder) MaxSchemaID() int { + max := 0 + for _, s := range b.schemaList { + if s.ID > max { + max = s.ID + } + } + + return max +} + +func (b *MetadataBuilder) MaxSpecID() int { + max := 0 + for _, s := range b.specs { + if s.ID() > max { + max = s.ID() + } + } + + return max +} + +func (b *MetadataBuilder) MaxSortOrderID() int { + max := 0 + for _, s := range b.sortOrderList { + if s.OrderID > max { + max = s.OrderID + } + } + + return max +} + +func (b *MetadataBuilder) SnapshotByID(id int64) (*Snapshot, error) { + for _, s := range b.snapshotList { + if s.SnapshotID == id { + return &s, nil + } + } + + return nil, fmt.Errorf("snapshot with id %d not found", id) +} + +func (b *MetadataBuilder) isAddedSchemaID(id int) bool { + for _, u := range b.updates { + if u.Action() == "add-schema" && + u.(*AddSchemaUpdate).Schema.ID == id { + return true + } + } + + return false +} + +func (b *MetadataBuilder) isAddedSnapshot(id int64) bool { + for _, u := range b.updates { + if u.Action() == "add-snapshot" && + u.(*AddSnapshotUpdate).Snapshot.SnapshotID == id { + return true + } + } + + return false +} + +func (b *MetadataBuilder) isAddedSpecID(id int) bool { + for _, u := range b.updates { + if u.Action() == "add-partition-spec" && + u.(*AddPartitionSpecUpdate).Spec.ID() == id { + return true + } + + } + + return false +} + +func (b *MetadataBuilder) isAddedSortOrder(id int) bool { + for _, u := range b.updates { + if u.Action() == "add-sort-order" && + u.(*AddSortOrderUpdate).SortOrder.OrderID == id { + return true + } + } + + return false +} + +func (b *MetadataBuilder) Build() (Metadata, error) { + common := b.buildCommonMetadata() + switch b.formatVersion { + case 1: + schema, err := b.GetSchemaByID(b.currentSchemaID) + if err != nil { + return nil, fmt.Errorf("can't build metadata, missing schema for schema ID %d: %w", b.currentSchemaID, err) + } + partition, err := b.GetSpecByID(b.defaultSpecID) + if err != nil { + return nil, fmt.Errorf("can't build metadata, missing partition spec for spec ID %d: %w", b.defaultSpecID, err) + } + return &metadataV1{ + Schema: schema, + Partition: partition.Fields(), + commonMetadata: *common, + }, nil + case 2: + return &metadataV2{ + LastSequenceNumber: *b.lastSequenceNumber, + commonMetadata: *common, + }, nil + default: + panic("unreachable: invalid format version") + } +} + var ( ErrInvalidMetadataFormatVersion = errors.New("invalid or missing format-version in table metadata") ErrInvalidMetadata = errors.New("invalid metadata") @@ -128,9 +626,9 @@ func ParseMetadataBytes(b []byte) (Metadata, error) { var ret Metadata switch ver.FormatVersion { case 1: - ret = &MetadataV1{} + ret = &metadataV1{} case 2: - ret = &MetadataV2{} + ret = &metadataV2{} default: return nil, ErrInvalidMetadataFormatVersion } @@ -163,9 +661,13 @@ type commonMetadata struct { MetadataLog []MetadataLogEntry `json:"metadata-log"` SortOrderList []SortOrder `json:"sort-orders"` DefaultSortOrderID int `json:"default-sort-order-id"` - Refs map[string]SnapshotRef `json:"refs"` + SnapshotRefs map[string]SnapshotRef `json:"refs"` } +func (c *commonMetadata) Ref() SnapshotRef { return c.SnapshotRefs[MainBranch] } +func (c *commonMetadata) Refs() map[string]SnapshotRef { return c.SnapshotRefs } +func (c *commonMetadata) SnapshotLogs() []SnapshotLogEntry { return c.SnapshotLog } +func (c *commonMetadata) PreviousFiles() []MetadataLogEntry { return c.MetadataLog } func (c *commonMetadata) Equals(other *commonMetadata) bool { switch { case c.LastPartitionID == nil && other.LastPartitionID != nil: @@ -187,7 +689,7 @@ func (c *commonMetadata) Equals(other *commonMetadata) bool { fallthrough case !maps.Equal(c.Props, other.Props): fallthrough - case !maps.EqualFunc(c.Refs, other.Refs, func(sr1, sr2 SnapshotRef) bool { return sr1.Equals(sr2) }): + case !maps.EqualFunc(c.SnapshotRefs, other.SnapshotRefs, func(sr1, sr2 SnapshotRef) bool { return sr1.Equals(sr2) }): return false } @@ -245,7 +747,7 @@ func (c *commonMetadata) SnapshotByID(id int64) *Snapshot { } func (c *commonMetadata) SnapshotByName(name string) *Snapshot { - if ref, ok := c.Refs[name]; ok { + if ref, ok := c.SnapshotRefs[name]; ok { return c.SnapshotByID(ref.SnapshotID) } return nil @@ -268,6 +770,10 @@ func (c *commonMetadata) SortOrder() SortOrder { return UnsortedSortOrder } +func (c *commonMetadata) DefaultSortOrder() int { + return c.DefaultSortOrderID +} + func (c *commonMetadata) Properties() iceberg.Properties { return c.Props } @@ -284,8 +790,8 @@ func (c *commonMetadata) preValidate() { } if c.CurrentSnapshotID != nil { - if _, ok := c.Refs[MainBranch]; !ok { - c.Refs[MainBranch] = SnapshotRef{ + if _, ok := c.SnapshotRefs[MainBranch]; !ok { + c.SnapshotRefs[MainBranch] = SnapshotRef{ SnapshotID: *c.CurrentSnapshotID, SnapshotRefType: BranchRef, } @@ -296,8 +802,8 @@ func (c *commonMetadata) preValidate() { c.MetadataLog = []MetadataLogEntry{} } - if c.Refs == nil { - c.Refs = make(map[string]SnapshotRef) + if c.SnapshotRefs == nil { + c.SnapshotRefs = make(map[string]SnapshotRef) } if c.SnapshotLog == nil { @@ -370,26 +876,26 @@ func (c *commonMetadata) validate() error { func (c *commonMetadata) Version() int { return c.FormatVersion } -type MetadataV1 struct { - Schema iceberg.Schema `json:"schema"` +type metadataV1 struct { + Schema *iceberg.Schema `json:"schema"` Partition []iceberg.PartitionField `json:"partition-spec"` commonMetadata } -func (m *MetadataV1) Equals(other Metadata) bool { - rhs, ok := other.(*MetadataV1) +func (m *metadataV1) Equals(other Metadata) bool { + rhs, ok := other.(*metadataV1) if !ok { return false } - return m.Schema.Equals(&rhs.Schema) && slices.Equal(m.Partition, rhs.Partition) && + return m.Schema.Equals(rhs.Schema) && slices.Equal(m.Partition, rhs.Partition) && m.commonMetadata.Equals(&rhs.commonMetadata) } -func (m *MetadataV1) preValidate() { +func (m *metadataV1) preValidate() { if len(m.SchemaList) == 0 { - m.SchemaList = []*iceberg.Schema{&m.Schema} + m.SchemaList = []*iceberg.Schema{m.Schema} } if len(m.Specs) == 0 { @@ -416,8 +922,8 @@ func (m *MetadataV1) preValidate() { m.commonMetadata.preValidate() } -func (m *MetadataV1) UnmarshalJSON(b []byte) error { - type Alias MetadataV1 +func (m *metadataV1) UnmarshalJSON(b []byte) error { + type Alias metadataV1 aux := (*Alias)(m) if err := json.Unmarshal(b, aux); err != nil { @@ -428,24 +934,24 @@ func (m *MetadataV1) UnmarshalJSON(b []byte) error { return m.validate() } -func (m *MetadataV1) ToV2() MetadataV2 { +func (m *metadataV1) ToV2() metadataV2 { commonOut := m.commonMetadata commonOut.FormatVersion = 2 if commonOut.UUID.String() == "" { commonOut.UUID = uuid.New() } - return MetadataV2{commonMetadata: commonOut} + return metadataV2{commonMetadata: commonOut} } -type MetadataV2 struct { - LastSequenceNumber int `json:"last-sequence-number"` +type metadataV2 struct { + LastSequenceNumber int64 `json:"last-sequence-number"` commonMetadata } -func (m *MetadataV2) Equals(other Metadata) bool { - rhs, ok := other.(*MetadataV2) +func (m *metadataV2) Equals(other Metadata) bool { + rhs, ok := other.(*metadataV2) if !ok { return false } @@ -454,8 +960,8 @@ func (m *MetadataV2) Equals(other Metadata) bool { m.commonMetadata.Equals(&rhs.commonMetadata) } -func (m *MetadataV2) UnmarshalJSON(b []byte) error { - type Alias MetadataV2 +func (m *metadataV2) UnmarshalJSON(b []byte) error { + type Alias metadataV2 aux := (*Alias)(m) if err := json.Unmarshal(b, aux); err != nil { diff --git a/table/metadata_test.go b/table/metadata_internal_test.go similarity index 93% rename from table/metadata_test.go rename to table/metadata_internal_test.go index e268d88..95001b8 100644 --- a/table/metadata_test.go +++ b/table/metadata_internal_test.go @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package table_test +package table import ( "encoding/json" @@ -23,7 +23,6 @@ import ( "testing" "github.com/apache/iceberg-go" - "github.com/apache/iceberg-go/table" "github.com/google/uuid" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -112,14 +111,14 @@ const ExampleTableMetadataV1 = `{ }` func TestMetadataV1Parsing(t *testing.T) { - meta, err := table.ParseMetadataBytes([]byte(ExampleTableMetadataV1)) + meta, err := ParseMetadataBytes([]byte(ExampleTableMetadataV1)) require.NoError(t, err) require.NotNil(t, meta) - assert.IsType(t, (*table.MetadataV1)(nil), meta) + assert.IsType(t, (*metadataV1)(nil), meta) assert.Equal(t, 1, meta.Version()) - data := meta.(*table.MetadataV1) + data := meta.(*metadataV1) assert.Equal(t, uuid.MustParse("d20125c8-7284-442c-9aea-15fee620737c"), meta.TableUUID()) assert.Equal(t, "s3://bucket/test/location", meta.Location()) assert.Equal(t, int64(1602638573874), meta.LastUpdatedMillis()) @@ -156,18 +155,18 @@ func TestMetadataV1Parsing(t *testing.T) { assert.Nil(t, meta.SnapshotByID(0)) assert.Nil(t, meta.SnapshotByName("foo")) assert.Zero(t, data.DefaultSortOrderID) - assert.Equal(t, table.UnsortedSortOrder, meta.SortOrder()) + assert.Equal(t, UnsortedSortOrder, meta.SortOrder()) } func TestMetadataV2Parsing(t *testing.T) { - meta, err := table.ParseMetadataBytes([]byte(ExampleTableMetadataV2)) + meta, err := ParseMetadataBytes([]byte(ExampleTableMetadataV2)) require.NoError(t, err) require.NotNil(t, meta) - assert.IsType(t, (*table.MetadataV2)(nil), meta) + assert.IsType(t, (*metadataV2)(nil), meta) assert.Equal(t, 2, meta.Version()) - data := meta.(*table.MetadataV2) + data := meta.(*metadataV2) assert.Equal(t, uuid.MustParse("9c12d441-03fe-4693-9a96-a0705ddf69c1"), data.UUID) assert.Equal(t, "s3://bucket/test/location", data.Location()) assert.Equal(t, 34, data.LastSequenceNumber) @@ -192,7 +191,7 @@ func TestMetadataV2Parsing(t *testing.T) { } func TestParsingCorrectTypes(t *testing.T) { - var meta table.MetadataV2 + var meta metadataV2 require.NoError(t, json.Unmarshal([]byte(ExampleTableMetadataV2), &meta)) assert.IsType(t, &iceberg.Schema{}, meta.SchemaList[0]) @@ -201,7 +200,7 @@ func TestParsingCorrectTypes(t *testing.T) { } func TestSerializeMetadataV1(t *testing.T) { - var meta table.MetadataV1 + var meta metadataV1 require.NoError(t, json.Unmarshal([]byte(ExampleTableMetadataV1), &meta)) data, err := json.Marshal(&meta) @@ -212,7 +211,7 @@ func TestSerializeMetadataV1(t *testing.T) { } func TestSerializeMetadataV2(t *testing.T) { - var meta table.MetadataV2 + var meta metadataV2 require.NoError(t, json.Unmarshal([]byte(ExampleTableMetadataV2), &meta)) data, err := json.Marshal(&meta) @@ -243,9 +242,9 @@ func TestInvalidFormatVersion(t *testing.T) { "snapshots": [] }` - _, err := table.ParseMetadataBytes([]byte(metadataInvalidFormat)) + _, err := ParseMetadataBytes([]byte(metadataInvalidFormat)) assert.Error(t, err) - assert.ErrorIs(t, err, table.ErrInvalidMetadataFormatVersion) + assert.ErrorIs(t, err, ErrInvalidMetadataFormatVersion) } func TestCurrentSchemaNotFound(t *testing.T) { @@ -278,9 +277,9 @@ func TestCurrentSchemaNotFound(t *testing.T) { "snapshots": [] }` - _, err := table.ParseMetadataBytes([]byte(schemaNotFound)) + _, err := ParseMetadataBytes([]byte(schemaNotFound)) assert.Error(t, err) - assert.ErrorIs(t, err, table.ErrInvalidMetadata) + assert.ErrorIs(t, err, ErrInvalidMetadata) assert.ErrorContains(t, err, "current-schema-id 2 can't be found in any schema") } @@ -322,9 +321,9 @@ func TestSortOrderNotFound(t *testing.T) { "snapshots": [] }` - _, err := table.ParseMetadataBytes([]byte(metadataSortOrderNotFound)) + _, err := ParseMetadataBytes([]byte(metadataSortOrderNotFound)) assert.Error(t, err) - assert.ErrorIs(t, err, table.ErrInvalidMetadata) + assert.ErrorIs(t, err, ErrInvalidMetadata) assert.ErrorContains(t, err, "default-sort-order-id 4 can't be found in [3: [\n2 asc nulls-first\nbucket[4](3) desc nulls-last\n]]") } @@ -358,10 +357,10 @@ func TestSortOrderUnsorted(t *testing.T) { "snapshots": [] }` - var meta table.MetadataV2 + var meta metadataV2 require.NoError(t, json.Unmarshal([]byte(sortOrderUnsorted), &meta)) - assert.Equal(t, table.UnsortedSortOrderID, meta.DefaultSortOrderID) + assert.Equal(t, UnsortedSortOrderID, meta.DefaultSortOrderID) assert.Len(t, meta.SortOrderList, 0) } @@ -394,28 +393,28 @@ func TestInvalidPartitionSpecID(t *testing.T) { "last-partition-id": 1000 }` - var meta table.MetadataV2 + var meta metadataV2 err := json.Unmarshal([]byte(invalidSpecID), &meta) - assert.ErrorIs(t, err, table.ErrInvalidMetadata) + assert.ErrorIs(t, err, ErrInvalidMetadata) assert.ErrorContains(t, err, "default-spec-id 1 can't be found") } func TestV2RefCreation(t *testing.T) { - var meta table.MetadataV2 + var meta metadataV2 require.NoError(t, json.Unmarshal([]byte(ExampleTableMetadataV2), &meta)) maxRefAge := int64(10000000) - assert.Equal(t, map[string]table.SnapshotRef{ + assert.Equal(t, map[string]SnapshotRef{ "main": { SnapshotID: 3055729675574597004, - SnapshotRefType: table.BranchRef, + SnapshotRefType: BranchRef, }, "test": { SnapshotID: 3051729675574597004, - SnapshotRefType: table.TagRef, + SnapshotRefType: TagRef, MaxRefAgeMs: &maxRefAge, }, - }, meta.Refs) + }, meta.SnapshotRefs) } func TestV1WriteMetadataToV2(t *testing.T) { @@ -453,11 +452,11 @@ func TestV1WriteMetadataToV2(t *testing.T) { "snapshots": [{"snapshot-id": 1925, "timestamp-ms": 1602638573822}] }` - meta, err := table.ParseMetadataString(minimalV1Example) + meta, err := ParseMetadataString(minimalV1Example) require.NoError(t, err) - assert.IsType(t, (*table.MetadataV1)(nil), meta) + assert.IsType(t, (*metadataV1)(nil), meta) - metaV2 := meta.(*table.MetadataV1).ToV2() + metaV2 := meta.(*metadataV1).ToV2() metaV2Json, err := json.Marshal(metaV2) require.NoError(t, err) diff --git a/table/requirements.go b/table/requirements.go new file mode 100644 index 0000000..9b00636 --- /dev/null +++ b/table/requirements.go @@ -0,0 +1,212 @@ +package table + +import ( + "fmt" + + "github.com/google/uuid" +) + +type Requirement interface { + Validate(Metadata) error +} + +type baseRequirement struct { + Type string `json:"type"` +} + +type AssertCreate struct { + baseRequirement +} + +func NewAssertCreate() *AssertCreate { + return &AssertCreate{ + baseRequirement: baseRequirement{Type: "assert-create"}, + } +} + +func (a *AssertCreate) Validate(meta Metadata) error { + if meta != nil { + return fmt.Errorf("Table already exists") + } + + return nil +} + +type AssertTableUuid struct { + baseRequirement + UUID uuid.UUID `json:"uuid"` +} + +func NewAssertTableUUID(uuid uuid.UUID) *AssertTableUuid { + return &AssertTableUuid{ + baseRequirement: baseRequirement{Type: "assert-table-uuid"}, + UUID: uuid, + } +} + +func (a *AssertTableUuid) Validate(meta Metadata) error { + if meta == nil { + return fmt.Errorf("requirement failed: current table metadata does not exist") + } + + if meta.TableUUID() != a.UUID { + return fmt.Errorf("UUID mismatch: %s != %s", meta.TableUUID(), a.UUID) + } + + return nil +} + +type AssertRefSnapshotID struct { + baseRequirement + Ref string `json:"ref"` + SnapshotID *int64 `json:"snapshot-id"` +} + +func NewAssertRefSnapshotID(ref string, id *int64) *AssertRefSnapshotID { + return &AssertRefSnapshotID{ + baseRequirement: baseRequirement{Type: "assert-ref-snapshot-id"}, + Ref: ref, + SnapshotID: id, + } +} + +func (a *AssertRefSnapshotID) Validate(meta Metadata) error { + if meta == nil { + return fmt.Errorf("requirement failed: current table metadata does not exist") + } + + ref, ok := meta.Refs()[a.Ref] + if !ok { + return fmt.Errorf("requirement failed: branch or tag %s is missing, expected %d", a.Ref, a.SnapshotID) + } + + if a.SnapshotID == nil { + return fmt.Errorf("requirement failed: %s %s was created concurrently", ref.SnapshotRefType, a.Ref) + } + + if ref.SnapshotID != *a.SnapshotID { + return fmt.Errorf("requirement failed: %s %s has changed: expected id %d, found %d", ref.SnapshotRefType, a.Ref, a.SnapshotID, ref.SnapshotID) + } + + return nil +} + +type AssertLastAssignedFieldId struct { + baseRequirement + LastAssignedFieldID int `json:"last-assigned-field-id"` +} + +func NewAssertLastAssignedFieldID(id int) *AssertLastAssignedFieldId { + return &AssertLastAssignedFieldId{ + baseRequirement: baseRequirement{Type: "assert-last-assigned-field-id"}, + LastAssignedFieldID: id, + } +} + +func (a *AssertLastAssignedFieldId) Validate(meta Metadata) error { + if meta == nil { + return fmt.Errorf("requirement failed: current table metadata does not exist") + } + + if meta.LastColumnID() != a.LastAssignedFieldID { + return fmt.Errorf("requirement failed: last assigned field id has changed: expected %d, found %d", a.LastAssignedFieldID, meta.LastColumnID()) + } + + return nil +} + +type AssertCurrentSchemaId struct { + baseRequirement + CurrentSchemaID int `json:"current-schema-id"` +} + +func NewAssertCurrentSchemaID(id int) *AssertCurrentSchemaId { + return &AssertCurrentSchemaId{ + baseRequirement: baseRequirement{Type: "assert-current-schema-id"}, + CurrentSchemaID: id, + } +} + +func (a *AssertCurrentSchemaId) Validate(meta Metadata) error { + if meta == nil { + return fmt.Errorf("requirement failed: current table metadata does not exist") + } + + if meta.CurrentSchema().ID != a.CurrentSchemaID { + return fmt.Errorf("requirement failed: current schema id has changed: expected %d, found %d", a.CurrentSchemaID, meta.CurrentSchema().ID) + } + + return nil +} + +type AssertLastAssignedPartitionId struct { + baseRequirement + LastAssignedPartitionID int `json:"last-assigned-partition-id"` +} + +func NewAssertLastAssignedPartitionID(id int) *AssertLastAssignedPartitionId { + return &AssertLastAssignedPartitionId{ + baseRequirement: baseRequirement{Type: "assert-last-assigned-partition-id"}, + LastAssignedPartitionID: id, + } +} + +func (a *AssertLastAssignedPartitionId) Validate(meta Metadata) error { + if meta == nil { + return fmt.Errorf("requirement failed: current table metadata does not exist") + } + + if *meta.LastPartitionSpecID() != a.LastAssignedPartitionID { + return fmt.Errorf("requirement failed: last assigned partition id has changed: expected %d, found %d", a.LastAssignedPartitionID, *meta.LastPartitionSpecID()) + } + + return nil +} + +type AssertDefaultSpecId struct { + baseRequirement + DefaultSpecID int `json:"default-spec-id"` +} + +func NewAssertDefaultSpecID(id int) *AssertDefaultSpecId { + return &AssertDefaultSpecId{ + baseRequirement: baseRequirement{Type: "assert-default-spec-id"}, + DefaultSpecID: id, + } +} + +func (a *AssertDefaultSpecId) Validate(meta Metadata) error { + if meta == nil { + return fmt.Errorf("requirement failed: current table metadata does not exist") + } + + if meta.DefaultPartitionSpec() != a.DefaultSpecID { + return fmt.Errorf("requirement failed: default spec id has changed: expected %d, found %d", a.DefaultSpecID, meta.DefaultPartitionSpec()) + } + + return nil +} + +type AssertDefaultSortOrderId struct { + baseRequirement + DefaultSortOrderID int `json:"default-sort-order-id"` +} + +func NewAssertDefaultSortOrderID(id int) *AssertDefaultSortOrderId { + return &AssertDefaultSortOrderId{ + baseRequirement: baseRequirement{Type: "assert-default-sort-order-id"}, + DefaultSortOrderID: id, + } +} + +func (a *AssertDefaultSortOrderId) Validate(meta Metadata) error { + if meta == nil { + return fmt.Errorf("requirement failed: current table metadata does not exist") + } + + if meta.DefaultSortOrder() != a.DefaultSortOrderID { + return fmt.Errorf("requirement failed: default sort order id has changed: expected %d, found %d", a.DefaultSortOrderID, meta.DefaultSortOrder()) + } + + return nil +} diff --git a/table/table_test.go b/table/table_test.go index cde94ab..f09054c 100644 --- a/table/table_test.go +++ b/table/table_test.go @@ -41,7 +41,7 @@ func (t *TableTestSuite) SetupSuite() { var mockfs internal.MockFS mockfs.Test(t.T()) mockfs.On("Open", "s3://bucket/test/location/uuid.metadata.json"). - Return(&internal.MockFile{Contents: bytes.NewReader([]byte(ExampleTableMetadataV2))}, nil) + Return(&internal.MockFile{Contents: bytes.NewReader([]byte(table.ExampleTableMetadataV2))}, nil) defer mockfs.AssertExpectations(t.T()) tbl, err := table.NewFromLocation([]string{"foo"}, "s3://bucket/test/location/uuid.metadata.json", &mockfs) @@ -59,7 +59,7 @@ func (t *TableTestSuite) TestNewTableFromReadFile() { var mockfsReadFile internal.MockFSReadFile mockfsReadFile.Test(t.T()) mockfsReadFile.On("ReadFile", "s3://bucket/test/location/uuid.metadata.json"). - Return([]byte(ExampleTableMetadataV2), nil) + Return([]byte(table.ExampleTableMetadataV2), nil) defer mockfsReadFile.AssertExpectations(t.T()) tbl2, err := table.NewFromLocation([]string{"foo"}, "s3://bucket/test/location/uuid.metadata.json", &mockfsReadFile) diff --git a/table/updates.go b/table/updates.go new file mode 100644 index 0000000..28415a8 --- /dev/null +++ b/table/updates.go @@ -0,0 +1,301 @@ +package table + +import ( + "fmt" + + "github.com/apache/iceberg-go" + "github.com/google/uuid" +) + +type Update interface { + Action() string + Apply(*MetadataBuilder) error +} + +type baseUpdate struct { + ActionName string `json:"action"` +} + +func (u *baseUpdate) Action() string { + return u.ActionName +} + +type AssignUUIDUpdate struct { + baseUpdate + UUID uuid.UUID `json:"uuid"` +} + +func NewAssignUUIDUpdate(uuid uuid.UUID) *AssignUUIDUpdate { + return &AssignUUIDUpdate{ + baseUpdate: baseUpdate{ActionName: "assign-uuid"}, + UUID: uuid, + } +} + +func (u *AssignUUIDUpdate) Apply(builder *MetadataBuilder) error { + _, err := builder.SetUUID(u.UUID) + return err +} + +type UpgradeFormatVersionUpdate struct { + baseUpdate + FormatVersion int `json:"format-version"` +} + +func NewUpgradeFormatVersionUpdate(formatVersion int) *UpgradeFormatVersionUpdate { + return &UpgradeFormatVersionUpdate{ + baseUpdate: baseUpdate{ActionName: "upgrade-format-version"}, + FormatVersion: formatVersion, + } +} + +func (u *UpgradeFormatVersionUpdate) Apply(builder *MetadataBuilder) error { + _, err := builder.SetFormatVersion(u.FormatVersion) + return err +} + +type AddSchemaUpdate struct { + baseUpdate + Schema *iceberg.Schema `json:"schema"` + LastColumnID int `json:"last-column-id"` +} + +func NewAddSchemaUpdate(schema *iceberg.Schema, lastColumnID int) *AddSchemaUpdate { + return &AddSchemaUpdate{ + baseUpdate: baseUpdate{ActionName: "add-schema"}, + Schema: schema, + LastColumnID: lastColumnID, + } +} + +func (u *AddSchemaUpdate) Apply(builder *MetadataBuilder) error { + _, err := builder.AddSchema(u.Schema, u.LastColumnID) + return err +} + +type SetCurrentSchemaUpdate struct { + baseUpdate + SchemaID int `json:"schema-id"` +} + +func NewSetCurrentSchemaUpdate(id int) *SetCurrentSchemaUpdate { + return &SetCurrentSchemaUpdate{ + baseUpdate: baseUpdate{ActionName: "set-current-schema"}, + SchemaID: id, + } +} + +func (u *SetCurrentSchemaUpdate) Apply(builder *MetadataBuilder) error { + _, err := builder.SetCurrentSchemaID(u.SchemaID) + return err +} + +type AddPartitionSpecUpdate struct { + baseUpdate + Spec *iceberg.PartitionSpec `json:"spec"` + initialChange bool +} + +func NewAddPartitionSpecUpdate(spec *iceberg.PartitionSpec, initial bool) *AddPartitionSpecUpdate { + return &AddPartitionSpecUpdate{ + baseUpdate: baseUpdate{ActionName: "add-spec"}, + Spec: spec, + initialChange: initial, + } +} + +func (u *AddPartitionSpecUpdate) Apply(builder *MetadataBuilder) error { + _, err := builder.AddPartitionSpec(u.Spec, u.initialChange) + return err +} + +type SetDefaultSpecUpdate struct { + baseUpdate + SpecID int `json:"spec-id"` +} + +func NewSetDefaultSpecUpdate(id int) *SetDefaultSpecUpdate { + return &SetDefaultSpecUpdate{ + baseUpdate: baseUpdate{ActionName: "set-default-spec"}, + SpecID: id, + } +} + +func (u *SetDefaultSpecUpdate) Apply(builder *MetadataBuilder) error { + _, err := builder.SetDefaultSpecID(u.SpecID) + return err +} + +type AddSortOrderUpdate struct { + baseUpdate + SortOrder *SortOrder `json:"sort-order"` +} + +func NewAddSortOrderUpdate(sortOrder *SortOrder) *AddSortOrderUpdate { + return &AddSortOrderUpdate{ + baseUpdate: baseUpdate{ActionName: "add-sort-order"}, + SortOrder: sortOrder, + } +} + +func (u *AddSortOrderUpdate) Apply(builder *MetadataBuilder) error { + _, err := builder.AddSortOrder(u.SortOrder) + return err +} + +type SetDefaultSortOrderUpdate struct { + baseUpdate + SortOrderID int `json:"sort-order-id"` +} + +func NewSetDefaultSortOrderUpdate(id int) *SetDefaultSortOrderUpdate { + return &SetDefaultSortOrderUpdate{ + baseUpdate: baseUpdate{ActionName: "set-default-sort-order"}, + SortOrderID: id, + } +} + +func (u *SetDefaultSortOrderUpdate) Apply(builder *MetadataBuilder) error { + _, err := builder.SetDefaultSortOrderID(u.SortOrderID) + return err +} + +type AddSnapshotUpdate struct { + baseUpdate + Snapshot *Snapshot `json:"snapshot"` +} + +func NewAddSnapshotUpdate(snapshot *Snapshot) *AddSnapshotUpdate { + return &AddSnapshotUpdate{ + baseUpdate: baseUpdate{ActionName: "add-snapshot"}, + Snapshot: snapshot, + } +} + +func (u *AddSnapshotUpdate) Apply(builder *MetadataBuilder) error { + _, err := builder.AddSnapshot(u.Snapshot) + return err +} + +type SetSnapshotRefUpdate struct { + baseUpdate + RefName string `json:"ref-name"` + RefType RefType `json:"type"` + SnapshotID int64 `json:"snapshot-id"` + MaxRefAgeMs *int64 `json:"max-ref-age-ms,omitempty"` + MaxSnapshotAgeMs *int64 `json:"max-snapshot-age-ms,omitempty"` + MinSnapshotsToKeep *int `json:"min-snapshots-to-keep,omitempty"` +} + +func NewSetSnapshotRefUpdate( + name string, + snapshotID int64, + refType RefType, + maxRefAgeMs, maxSnapshotAgeMs *int64, + minSnapshotsToKeep *int, +) *SetSnapshotRefUpdate { + return &SetSnapshotRefUpdate{ + baseUpdate: baseUpdate{ActionName: "set-snapshot-ref"}, + RefName: name, + RefType: refType, + SnapshotID: snapshotID, + MaxRefAgeMs: maxRefAgeMs, + MaxSnapshotAgeMs: maxSnapshotAgeMs, + MinSnapshotsToKeep: minSnapshotsToKeep, + } +} + +func (u *SetSnapshotRefUpdate) Apply(builder *MetadataBuilder) error { + _, err := builder.SetSnapshotRef( + u.RefName, + u.SnapshotID, + u.RefType, + u.MaxRefAgeMs, + u.MaxSnapshotAgeMs, + u.MinSnapshotsToKeep, + ) + return err +} + +type SetLocationUpdate struct { + baseUpdate + Location string `json:"location"` +} + +func NewSetLocationUpdate(loc string) *SetLocationUpdate { + return &SetLocationUpdate{ + baseUpdate: baseUpdate{ActionName: "set-location"}, + Location: loc, + } +} + +func (u *SetLocationUpdate) Apply(builder *MetadataBuilder) error { + _, err := builder.SetLoc(u.Location) + return err +} + +type SetPropertiesUpdate struct { + baseUpdate + Updates iceberg.Properties `json:"updates"` +} + +func NewSetPropertiesUpdate(updates iceberg.Properties) *SetPropertiesUpdate { + return &SetPropertiesUpdate{ + baseUpdate: baseUpdate{ActionName: "set-properties"}, + Updates: updates, + } +} + +func (u *SetPropertiesUpdate) Apply(builder *MetadataBuilder) error { + _, err := builder.SetProperties(u.Updates) + return err +} + +type RemovePropertiesUpdate struct { + baseUpdate + Removals []string `json:"removals"` +} + +func NewRemovePropertiesUpdate(removals []string) *RemovePropertiesUpdate { + return &RemovePropertiesUpdate{ + baseUpdate: baseUpdate{ActionName: "remove-properties"}, + Removals: removals, + } +} + +func (u *RemovePropertiesUpdate) Apply(builder *MetadataBuilder) error { + _, err := builder.RemoveProperties(u.Removals) + return err +} + +type RemoveSnapshotsUpdate struct { + baseUpdate + SnapshotIDs []int64 `json:"snapshot-ids"` +} + +func NewRemoveSnapshotsUpdate(ids []int64) *RemoveSnapshotsUpdate { + return &RemoveSnapshotsUpdate{ + baseUpdate: baseUpdate{ActionName: "remove-snapshots"}, + SnapshotIDs: ids, + } +} + +func (u *RemoveSnapshotsUpdate) Apply(builder *MetadataBuilder) error { + return fmt.Errorf("%w: remove-snapshots", iceberg.ErrNotImplemented) +} + +type RemoveSnapshotRefUpdate struct { + baseUpdate + RefName string `json:"ref-name"` +} + +func NewRemoveSnapshotRefUpdate(ref string) *RemoveSnapshotRefUpdate { + return &RemoveSnapshotRefUpdate{ + baseUpdate: baseUpdate{ActionName: "remove-snapshot-ref"}, + RefName: ref, + } +} + +func (u *RemoveSnapshotRefUpdate) Apply(builder *MetadataBuilder) error { + return fmt.Errorf("%w: remove-snapshot-ref", iceberg.ErrNotImplemented) +} From 3b3a10f157606aa6a515da6cd1d3070102ee9254 Mon Sep 17 00:00:00 2001 From: Jonas Weile Date: Mon, 9 Sep 2024 13:20:36 +0000 Subject: [PATCH 02/28] catalog implementation --- catalog/catalog.go | 5 +- catalog/glue.go | 10 +- catalog/glue_test.go | 2 +- catalog/rest.go | 247 +++++++++++++++++++++++++++++++++++++------ catalog/rest_test.go | 2 +- 5 files changed, 220 insertions(+), 46 deletions(-) diff --git a/catalog/catalog.go b/catalog/catalog.go index d6d7f1e..fede73d 100644 --- a/catalog/catalog.go +++ b/catalog/catalog.go @@ -43,6 +43,7 @@ var ( ErrNoSuchTable = errors.New("table does not exist") ErrNoSuchNamespace = errors.New("namespace does not exist") ErrNamespaceAlreadyExists = errors.New("namespace already exists") + ErrTableAlreadyExists = errors.New("table already exists") ) // WithAwsConfig sets the AWS configuration for the catalog. @@ -146,9 +147,9 @@ type Catalog interface { // identifiers containing the information required to load the table via that catalog. ListTables(ctx context.Context, namespace table.Identifier) ([]table.Identifier, error) // LoadTable loads a table from the catalog and returns a Table with the metadata. - LoadTable(ctx context.Context, identifier table.Identifier, props iceberg.Properties) (*table.Table, error) + LoadTable(ctx context.Context, identifier table.Identifier) (*table.Table, error) // DropTable tells the catalog to drop the table entirely - DropTable(ctx context.Context, identifier table.Identifier) error + DropTable(ctx context.Context, identifier table.Identifier, purge bool) error // RenameTable tells the catalog to rename a given table by the identifiers // provided, and then loads and returns the destination table RenameTable(ctx context.Context, from, to table.Identifier) (*table.Table, error) diff --git a/catalog/glue.go b/catalog/glue.go index b398185..59f0116 100644 --- a/catalog/glue.go +++ b/catalog/glue.go @@ -92,23 +92,19 @@ func (c *GlueCatalog) ListTables(ctx context.Context, namespace table.Identifier // LoadTable loads a table from the catalog table details. // // The identifier should contain the Glue database name, then glue table name. -func (c *GlueCatalog) LoadTable(ctx context.Context, identifier table.Identifier, props iceberg.Properties) (*table.Table, error) { +func (c *GlueCatalog) LoadTable(ctx context.Context, identifier table.Identifier) (*table.Table, error) { database, tableName, err := identifierToGlueTable(identifier) if err != nil { return nil, err } - if props == nil { - props = map[string]string{} - } - location, err := c.getTable(ctx, database, tableName) if err != nil { return nil, err } // TODO: consider providing a way to directly access the S3 iofs to enable testing of the catalog. - iofs, err := io.LoadFS(props, location) + iofs, err := io.LoadFS(nil, location) if err != nil { return nil, fmt.Errorf("failed to load table %s.%s: %w", database, tableName, err) } @@ -125,7 +121,7 @@ func (c *GlueCatalog) CatalogType() CatalogType { return Glue } -func (c *GlueCatalog) DropTable(ctx context.Context, identifier table.Identifier) error { +func (c *GlueCatalog) DropTable(ctx context.Context, identifier table.Identifier, purge bool) error { return fmt.Errorf("%w: [Glue Catalog] drop table", iceberg.ErrNotImplemented) } diff --git a/catalog/glue_test.go b/catalog/glue_test.go index 1d3c42f..aa7e222 100644 --- a/catalog/glue_test.go +++ b/catalog/glue_test.go @@ -142,7 +142,7 @@ func TestGlueLoadTableIntegration(t *testing.T) { catalog := NewGlueCatalog(WithAwsConfig(awscfg)) - table, err := catalog.LoadTable(context.TODO(), []string{os.Getenv("TEST_DATABASE_NAME"), os.Getenv("TEST_TABLE_NAME")}, nil) + table, err := catalog.LoadTable(context.TODO(), []string{os.Getenv("TEST_DATABASE_NAME"), os.Getenv("TEST_TABLE_NAME")}) assert.NoError(err) assert.Equal([]string{os.Getenv("TEST_TABLE_NAME")}, table.Identifier()) } diff --git a/catalog/rest.go b/catalog/rest.go index ef9c332..42bdd27 100644 --- a/catalog/rest.go +++ b/catalog/rest.go @@ -84,6 +84,86 @@ func (e errorResponse) Error() string { return e.Type + ": " + e.Message } +type Identifier struct { + Namespace []string `json:"namespace"` + Name string `json:"name"` +} + +type commitTableResponse struct { + MetadataLoc string `json:"metadata-location"` + RawMetadata json.RawMessage `json:"metadata"` + Metadata table.Metadata `json:"-"` +} + +func (t *commitTableResponse) UnmarshalJSON(b []byte) (err error) { + type Alias commitTableResponse + if err = json.Unmarshal(b, (*Alias)(t)); err != nil { + return err + } + + t.Metadata, err = table.ParseMetadataBytes(t.RawMetadata) + return +} + +type loadTableResponse struct { + MetadataLoc string `json:"metadata-location"` + RawMetadata json.RawMessage `json:"metadata"` + Config iceberg.Properties `json:"config"` + Metadata table.Metadata `json:"-"` +} + +func (t *loadTableResponse) UnmarshalJSON(b []byte) (err error) { + type Alias loadTableResponse + if err = json.Unmarshal(b, (*Alias)(t)); err != nil { + return err + } + + t.Metadata, err = table.ParseMetadataBytes(t.RawMetadata) + return +} + +type createTableOption func(*createTableRequest) + +func WithTableLocation(loc string) createTableOption { + return func(req *createTableRequest) { + req.Location = strings.TrimRight(loc, "/") + } +} + +func WithTablePartitionSpec(spec iceberg.PartitionSpec) createTableOption { + return func(req *createTableRequest) { + req.PartitionSpec = spec + } +} + +func WithTableWriteOrder(order table.SortOrder) createTableOption { + return func(req *createTableRequest) { + req.WriteOrder = order + } +} + +func WithTableStagingCreate() createTableOption { + return func(req *createTableRequest) { + req.StageCreate = true + } +} + +func WithTableProperties(props iceberg.Properties) createTableOption { + return func(req *createTableRequest) { + req.Props = props + } +} + +type createTableRequest struct { + Name string `json:"name"` + Location string `json:"location"` + Schema *iceberg.Schema `json:"schema"` + PartitionSpec iceberg.PartitionSpec `json:"partition-spec"` + WriteOrder table.SortOrder `json:"write-order"` + StageCreate bool `json:"stage-create"` + Props iceberg.Properties `json:"properties"` +} + type oauthTokenResponse struct { AccessToken string `json:"access_token"` TokenType string `json:"token_type"` @@ -537,6 +617,25 @@ func checkValidNamespace(ident table.Identifier) error { return nil } +func (r *RestCatalog) tableFromResponse(identifier []string, metadata table.Metadata, loc string, config iceberg.Properties) (*table.Table, error) { + id := identifier + if r.name != "" { + id = append([]string{r.name}, identifier...) + } + + tblProps := maps.Clone(r.props) + maps.Copy(tblProps, metadata.Properties()) + for k, v := range config { + tblProps[k] = v + } + + iofs, err := iceio.LoadFS(tblProps, loc) + if err != nil { + return nil, err + } + return table.New(id, metadata, loc, iofs), nil +} + func (r *RestCatalog) ListTables(ctx context.Context, namespace table.Identifier) ([]table.Identifier, error) { if err := checkValidNamespace(namespace); err != nil { return nil, err @@ -546,12 +645,8 @@ func (r *RestCatalog) ListTables(ctx context.Context, namespace table.Identifier path := []string{"namespaces", ns, "tables"} type resp struct { - Identifiers []struct { - Namespace []string `json:"namespace"` - Name string `json:"name"` - } `json:"identifiers"` + Identifiers []Identifier `json:"identifiers"` } - rsp, err := doGet[resp](ctx, r.baseURI, path, r.cl, map[int]error{http.StatusNotFound: ErrNoSuchNamespace}) if err != nil { return nil, err @@ -573,64 +668,129 @@ func splitIdentForPath(ident table.Identifier) (string, string, error) { return strings.Join(NamespaceFromIdent(ident), namespaceSeparator), TableNameFromIdent(ident), nil } -type tblResponse struct { - MetadataLoc string `json:"metadata-location"` - RawMetadata json.RawMessage `json:"metadata"` - Config iceberg.Properties `json:"config"` - Metadata table.Metadata `json:"-"` -} +func (r *RestCatalog) CreateTable(ctx context.Context, identifier table.Identifier, schema *iceberg.Schema, opts ...createTableOption) (*table.Table, error) { + ns, tbl, err := splitIdentForPath(identifier) + if err != nil { + return nil, err + } -func (t *tblResponse) UnmarshalJSON(b []byte) (err error) { - type Alias tblResponse - if err = json.Unmarshal(b, (*Alias)(t)); err != nil { - return err + payload := createTableRequest{ + Name: tbl, + Schema: schema, + } + for _, o := range opts { + o(&payload) } - t.Metadata, err = table.ParseMetadataBytes(t.RawMetadata) - return + ret, err := doPost[createTableRequest, loadTableResponse](ctx, r.baseURI, []string{"namespaces", ns, "tables"}, payload, + r.cl, map[int]error{http.StatusNotFound: ErrNoSuchNamespace, http.StatusConflict: ErrTableAlreadyExists}) + if err != nil { + return nil, err + } + + return r.tableFromResponse(identifier, ret.Metadata, ret.MetadataLoc, ret.Config) } -func (r *RestCatalog) LoadTable(ctx context.Context, identifier table.Identifier, props iceberg.Properties) (*table.Table, error) { +func (r *RestCatalog) RegisterTable(ctx context.Context, identifier table.Identifier, metadataLoc string) (*table.Table, error) { ns, tbl, err := splitIdentForPath(identifier) if err != nil { return nil, err } - if props == nil { - props = iceberg.Properties{} + type payload struct { + Name string `json:"name"` + MetadataLoc string `json:"metadata-location"` } - ret, err := doGet[tblResponse](ctx, r.baseURI, []string{"namespaces", ns, "tables", tbl}, - r.cl, map[int]error{http.StatusNotFound: ErrNoSuchTable}) + ret, err := doPost[payload, loadTableResponse](ctx, r.baseURI, []string{"namespaces", ns, "tables", tbl}, + payload{Name: tbl, MetadataLoc: metadataLoc}, r.cl, map[int]error{http.StatusNotFound: ErrNoSuchNamespace, http.StatusConflict: ErrTableAlreadyExists}) if err != nil { return nil, err } - id := identifier - if r.name != "" { - id = append([]string{r.name}, identifier...) + return r.tableFromResponse(identifier, ret.Metadata, ret.MetadataLoc, ret.Config) +} + +func (r *RestCatalog) LoadTable(ctx context.Context, identifier table.Identifier) (*table.Table, error) { + ns, tbl, err := splitIdentForPath(identifier) + if err != nil { + return nil, err } - tblProps := maps.Clone(r.props) - maps.Copy(tblProps, props) - maps.Copy(tblProps, ret.Metadata.Properties()) - for k, v := range ret.Config { - tblProps[k] = v + ret, err := doGet[loadTableResponse](ctx, r.baseURI, []string{"namespaces", ns, "tables", tbl}, + r.cl, map[int]error{http.StatusNotFound: ErrNoSuchTable}) + if err != nil { + return nil, err } - iofs, err := iceio.LoadFS(tblProps, ret.MetadataLoc) + return r.tableFromResponse(identifier, ret.Metadata, ret.MetadataLoc, ret.Config) +} + +func (r *RestCatalog) UpdateTable(ctx context.Context, identifier table.Identifier, requirements []table.Requirement, updates []table.Update) (*table.Table, error) { + ns, tbl, err := splitIdentForPath(identifier) if err != nil { return nil, err } - return table.New(id, ret.Metadata, ret.MetadataLoc, iofs), nil + + ident := Identifier{ + Namespace: NamespaceFromIdent(identifier), + Name: tbl, + } + type payload struct { + Identifier Identifier `json:"identifier"` + Requirements []table.Requirement `json:"requirements"` + Updates []table.Update `json:"updates"` + } + ret, err := doPost[payload, commitTableResponse](ctx, r.baseURI, []string{"namespaces", ns, "tables", tbl}, + payload{Identifier: ident, Requirements: requirements, Updates: updates}, r.cl, + map[int]error{http.StatusNotFound: ErrNoSuchTable, http.StatusConflict: ErrCommitFailed}) + if err != nil { + return nil, err + } + + return r.tableFromResponse(identifier, ret.Metadata, ret.MetadataLoc, nil) } -func (r *RestCatalog) DropTable(ctx context.Context, identifier table.Identifier) error { - return fmt.Errorf("%w: [Rest Catalog] drop table", iceberg.ErrNotImplemented) +func (r *RestCatalog) DropTable(ctx context.Context, identifier table.Identifier, purge bool) error { + ns, tbl, err := splitIdentForPath(identifier) + if err != nil { + return err + } + + uri := r.baseURI.JoinPath("namespaces", ns, "tables", tbl) + if purge { + v := url.Values{} + v.Set("purgeRequested", "true") + uri.RawQuery = v.Encode() + } + + _, err = doDelete[struct{}](ctx, uri, []string{}, r.cl, + map[int]error{http.StatusNotFound: ErrNoSuchTable}) + + return err } func (r *RestCatalog) RenameTable(ctx context.Context, from, to table.Identifier) (*table.Table, error) { - return nil, fmt.Errorf("%w: [Rest Catalog] rename table", iceberg.ErrNotImplemented) + type payload struct { + From Identifier `json:"from"` + To Identifier `json:"to"` + } + f := Identifier{ + Namespace: NamespaceFromIdent(from), + Name: TableNameFromIdent(from), + } + t := Identifier{ + Namespace: NamespaceFromIdent(to), + Name: TableNameFromIdent(to), + } + + _, err := doPost[payload, any](ctx, r.baseURI, []string{"tables", "rename"}, payload{From: f, To: t}, r.cl, + map[int]error{http.StatusNotFound: ErrNoSuchTable}) + if err != nil { + return nil, err + } + + return r.LoadTable(ctx, to) } func (r *RestCatalog) CreateNamespace(ctx context.Context, namespace table.Identifier, props iceberg.Properties) error { @@ -710,3 +870,20 @@ func (r *RestCatalog) UpdateNamespaceProperties(ctx context.Context, namespace t return doPost[payload, PropertiesUpdateSummary](ctx, r.baseURI, []string{"namespaces", ns, "properties"}, payload{Remove: removals, Updates: updates}, r.cl, map[int]error{http.StatusNotFound: ErrNoSuchNamespace}) } + +func (r *RestCatalog) CheckNamespaceExists(ctx context.Context, namespace table.Identifier) (bool, error) { + if err := checkValidNamespace(namespace); err != nil { + return false, err + } + + _, err := doGet[struct{}](ctx, r.baseURI, []string{"namespaces", strings.Join(namespace, namespaceSeparator)}, + r.cl, map[int]error{http.StatusNotFound: ErrNoSuchNamespace}) + if err != nil { + if errors.Is(err, ErrNoSuchNamespace) { + return false, nil + } + return false, err + } + + return true, nil +} diff --git a/catalog/rest_test.go b/catalog/rest_test.go index 618c5e0..ebf8b58 100644 --- a/catalog/rest_test.go +++ b/catalog/rest_test.go @@ -704,7 +704,7 @@ func (r *RestCatalogSuite) TestLoadTable200() { cat, err := catalog.NewRestCatalog("rest", r.srv.URL, catalog.WithOAuthToken(TestToken)) r.Require().NoError(err) - tbl, err := cat.LoadTable(context.Background(), catalog.ToRestIdentifier("fokko", "table"), nil) + tbl, err := cat.LoadTable(context.Background(), catalog.ToRestIdentifier("fokko", "table")) r.Require().NoError(err) r.Equal(catalog.ToRestIdentifier("rest", "fokko", "table"), tbl.Identifier()) From 34161294300d11d45935f59fe85583e26b01ea24 Mon Sep 17 00:00:00 2001 From: Jonas Weile Date: Mon, 9 Sep 2024 13:20:46 +0000 Subject: [PATCH 03/28] main & go files --- cmd/iceberg/main.go | 6 +++--- go.mod | 3 +++ go.sum | 12 ++++++++++-- 3 files changed, 16 insertions(+), 5 deletions(-) diff --git a/cmd/iceberg/main.go b/cmd/iceberg/main.go index fb25618..30c1960 100644 --- a/cmd/iceberg/main.go +++ b/cmd/iceberg/main.go @@ -185,7 +185,7 @@ func main() { os.Exit(1) } case cfg.Table: - err := cat.DropTable(context.Background(), catalog.ToRestIdentifier(cfg.Ident)) + err := cat.DropTable(context.Background(), catalog.ToRestIdentifier(cfg.Ident), false) if err != nil { output.Error(err) os.Exit(1) @@ -241,7 +241,7 @@ func describe(output Output, cat catalog.Catalog, id string, entityType string) } if (entityType == "any" || entityType == "tbl") && len(ident) > 1 { - tbl, err := cat.LoadTable(ctx, ident, nil) + tbl, err := cat.LoadTable(ctx, ident) if err != nil { if !errors.Is(err, catalog.ErrNoSuchTable) || entityType != "any" { output.Error(err) @@ -261,7 +261,7 @@ func describe(output Output, cat catalog.Catalog, id string, entityType string) } func loadTable(output Output, cat catalog.Catalog, id string) *table.Table { - tbl, err := cat.LoadTable(context.Background(), catalog.ToRestIdentifier(id), nil) + tbl, err := cat.LoadTable(context.Background(), catalog.ToRestIdentifier(id)) if err != nil { output.Error(err) os.Exit(1) diff --git a/go.mod b/go.mod index aeccfa1..cee4426 100644 --- a/go.mod +++ b/go.mod @@ -62,6 +62,7 @@ require ( github.com/gookit/color v1.5.4 // indirect github.com/json-iterator/go v1.1.12 // indirect github.com/klauspost/compress v1.17.9 // indirect + github.com/kr/pretty v0.3.1 // indirect github.com/lithammer/fuzzysearch v1.1.8 // indirect github.com/mattn/go-runewidth v0.0.15 // indirect github.com/mitchellh/mapstructure v1.5.0 // indirect @@ -69,6 +70,7 @@ require ( github.com/modern-go/reflect2 v1.0.2 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/rivo/uniseg v0.4.4 // indirect + github.com/rogpeppe/go-internal v1.12.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/xo/terminfo v0.0.0-20220910002029-abceb7e1c41e // indirect golang.org/x/mod v0.19.0 // indirect @@ -79,5 +81,6 @@ require ( golang.org/x/text v0.16.0 // indirect golang.org/x/tools v0.23.0 // indirect golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 // indirect + gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/go.sum b/go.sum index d0c23a9..c205a48 100644 --- a/go.sum +++ b/go.sum @@ -58,6 +58,7 @@ github.com/aws/smithy-go v1.20.4 h1:2HK1zBdPgRbjFOHlfeQZfpC4r72MOb9bZkiFwggKO+4= github.com/aws/smithy-go v1.20.4/go.mod h1:irrKGvNn1InZwb2d7fkIRNucdfwR8R+Ts3wxYa/cJHg= github.com/containerd/console v1.0.3 h1:lIr7SlA5PxZyMV30bDW0MGbiOPXwc63yRuCP0ARubLw= github.com/containerd/console v1.0.3/go.mod h1:7LqA/THxQ86k76b8c/EMSiaJ3h1eZkMkXar0TQ1gf3U= +github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= @@ -87,8 +88,10 @@ github.com/klauspost/cpuid/v2 v2.0.10/go.mod h1:g2LTdtYhdyuGPqyWyv7qRAmj1WBqxuOb github.com/klauspost/cpuid/v2 v2.0.12/go.mod h1:g2LTdtYhdyuGPqyWyv7qRAmj1WBqxuObKfj5c0PQa7c= github.com/klauspost/cpuid/v2 v2.2.7 h1:ZWSB3igEs+d0qvnxR/ZBzXVmxkgt8DdzP6m9pfuVLDM= github.com/klauspost/cpuid/v2 v2.2.7/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws= -github.com/kr/pretty v0.1.0 h1:L/CwN0zerZDmRFUapSPitk6f+Q3+0za1rQkzVuMiMFI= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= +github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= +github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= +github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= @@ -105,6 +108,7 @@ github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M= github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= +github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/pterm/pterm v0.12.27/go.mod h1:PhQ89w4i95rhgE+xedAoqous6K9X+r6aSOI2eFF7DZI= @@ -119,6 +123,9 @@ github.com/pterm/pterm v0.12.79/go.mod h1:1v/gzOF1N0FsjbgTHZ1wVycRkKiatFvJSJC4IG github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= github.com/rivo/uniseg v0.4.4 h1:8TfxU8dW6PdqD27gjM8MVNuicgxIjxpm4K7x4jp8sis= github.com/rivo/uniseg v0.4.4/go.mod h1:FN3SvrM+Zdj16jyLfmOkMNblXMcoc8DfTHruCPUcx88= +github.com/rogpeppe/go-internal v1.9.0/go.mod h1:WtVeX8xhTBvf0smdhujwtBcq4Qrzq/fJaraNFVN+nFs= +github.com/rogpeppe/go-internal v1.12.0 h1:exVL4IDcn6na9z1rAb56Vxr+CgyK3nn3O+epU5NdKM8= +github.com/rogpeppe/go-internal v1.12.0/go.mod h1:E+RYuTGaKKdloAfM02xzb0FW3Paa99yedzYV+kq4uf4= github.com/sergi/go-diff v1.2.0 h1:XU+rvMAioB0UC3q1MFrIQy4Vo5/4VsRDQQXHsEya6xQ= github.com/sergi/go-diff v1.2.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= @@ -195,8 +202,9 @@ golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 h1:+cNy6SZtPcJQH3LJVLOSmiC7MMxXNOb3PU/VUEz+EhU= golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028/go.mod h1:NDW/Ps6MPRej6fsCIbMTohpP40sJ/P/vI1MoTEGwX90= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15 h1:YR8cESwS4TdDjEe65xsg0ogRM/Nc3DYOhEAlW+xobZo= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= From 56c845881cb7996d364e14061f07c495a24f6a96 Mon Sep 17 00:00:00 2001 From: Jonas Weile Date: Mon, 9 Sep 2024 18:13:35 +0000 Subject: [PATCH 04/28] add test --- catalog/rest_test.go | 100 +++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 100 insertions(+) diff --git a/catalog/rest_test.go b/catalog/rest_test.go index ebf8b58..c2e6271 100644 --- a/catalog/rest_test.go +++ b/catalog/rest_test.go @@ -22,6 +22,7 @@ import ( "crypto/tls" "crypto/x509" "encoding/json" + "fmt" "net/http" "net/http/httptest" "net/url" @@ -624,6 +625,105 @@ func (r *RestCatalogSuite) TestUpdateNamespaceProps404() { r.ErrorContains(err, "Namespace does not exist: does_not_exist in warehouse") } +var ( + exampleTableMetadataNoSnapshotV1 = `{ + "format-version": 1, + "table-uuid": "bf289591-dcc0-4234-ad4f-5c3eed811a29", + "location": "s3://warehouse/database/table", + "last-updated-ms": 1657810967051, + "last-column-id": 3, + "schema": { + "type": "struct", + "schema-id": 0, + "identifier-field-ids": [2], + "fields": [ + {"id": 1, "name": "foo", "required": false, "type": "string"}, + {"id": 2, "name": "bar", "required": true, "type": "int"}, + {"id": 3, "name": "baz", "required": false, "type": "boolean"} + ] + }, + "current-schema-id": 0, + "schemas": [ + { + "type": "struct", + "schema-id": 0, + "identifier-field-ids": [2], + "fields": [ + {"id": 1, "name": "foo", "required": false, "type": "string"}, + {"id": 2, "name": "bar", "required": true, "type": "int"}, + {"id": 3, "name": "baz", "required": false, "type": "boolean"} + ] + } + ], + "partition-spec": [], + "default-spec-id": 0, + "last-partition-id": 999, + "default-sort-order-id": 0, + "sort-orders": [{"order-id": 0, "fields": []}], + "properties": { + "write.delete.parquet.compression-codec": "zstd", + "write.metadata.compression-codec": "gzip", + "write.summary.partition-limit": "100", + "write.parquet.compression-codec": "zstd" + }, + "current-snapshot-id": -1, + "refs": {}, + "snapshots": [], + "snapshot-log": [], + "metadata-log": [] +}` + + createTableRestExample = fmt.Sprintf(`{ + "metadata-location": "s3://warehouse/database/table/metadata.json", + "metadata": %s, + "config": { + "client.factory": "io.tabular.iceberg.catalog.TabularAwsClientFactory", + "region": "us-west-2" + } +}`, exampleTableMetadataNoSnapshotV1) + + tableSchemaSimple = iceberg.NewSchemaWithIdentifiers(1, []int{2}, + iceberg.NestedField{ID: 1, Name: "foo", Type: iceberg.StringType{}, Required: false}, + iceberg.NestedField{ID: 2, Name: "bar", Type: iceberg.PrimitiveTypes.Int32, Required: true}, + iceberg.NestedField{ID: 3, Name: "baz", Type: iceberg.PrimitiveTypes.Bool, Required: false}, + ) +) + +func (r *RestCatalogSuite) TestCreateTable200() { + r.mux.HandleFunc("/v1/namespaces/fokko/tables", func(w http.ResponseWriter, req *http.Request) { + r.Require().Equal(http.MethodPost, req.Method) + + for k, v := range TestHeaders { + r.Equal(v, req.Header.Values(k)) + } + + w.Write([]byte(createTableRestExample)) + }) + + t := createTableRestExample + _ = t + cat, err := catalog.NewRestCatalog("rest", r.srv.URL, catalog.WithOAuthToken(TestToken)) + r.Require().NoError(err) + + tbl, err := cat.CreateTable( + context.Background(), + catalog.ToRestIdentifier("fokko", "fokko2"), + tableSchemaSimple, + ) + r.Require().NoError(err) + + r.Equal(catalog.ToRestIdentifier("rest", "fokko", "fokko2"), tbl.Identifier()) + r.Equal("s3://warehouse/database/table/metadata.json", tbl.MetadataLocation()) + r.EqualValues(1, tbl.Metadata().Version()) + r.Equal("bf289591-dcc0-4234-ad4f-5c3eed811a29", tbl.Metadata().TableUUID().String()) + r.EqualValues(1657810967051, tbl.Metadata().LastUpdatedMillis()) + r.Equal(3, tbl.Metadata().LastColumnID()) + r.Zero(tbl.Schema().ID) + r.Zero(tbl.Metadata().DefaultPartitionSpec()) + r.Equal(999, *tbl.Metadata().LastPartitionSpecID()) + r.Equal(table.UnsortedSortOrder, tbl.SortOrder()) +} + func (r *RestCatalogSuite) TestLoadTable200() { r.mux.HandleFunc("/v1/namespaces/fokko/tables/table", func(w http.ResponseWriter, req *http.Request) { r.Require().Equal(http.MethodGet, req.Method) From 4c49f963ac40d314a402c4edde3e6be01fd8ddb6 Mon Sep 17 00:00:00 2001 From: Jonas Weile Date: Wed, 11 Sep 2024 09:14:28 +0000 Subject: [PATCH 05/28] add props back to LoadTable() --- catalog/catalog.go | 2 +- catalog/glue.go | 8 ++++++-- catalog/glue_test.go | 2 +- catalog/rest.go | 43 ++++++++++++++++++++++++++++++------------- catalog/rest_test.go | 2 +- cmd/iceberg/main.go | 4 ++-- 6 files changed, 41 insertions(+), 20 deletions(-) diff --git a/catalog/catalog.go b/catalog/catalog.go index fede73d..0ac5f8a 100644 --- a/catalog/catalog.go +++ b/catalog/catalog.go @@ -147,7 +147,7 @@ type Catalog interface { // identifiers containing the information required to load the table via that catalog. ListTables(ctx context.Context, namespace table.Identifier) ([]table.Identifier, error) // LoadTable loads a table from the catalog and returns a Table with the metadata. - LoadTable(ctx context.Context, identifier table.Identifier) (*table.Table, error) + LoadTable(ctx context.Context, identifier table.Identifier, props iceberg.Properties) (*table.Table, error) // DropTable tells the catalog to drop the table entirely DropTable(ctx context.Context, identifier table.Identifier, purge bool) error // RenameTable tells the catalog to rename a given table by the identifiers diff --git a/catalog/glue.go b/catalog/glue.go index 59f0116..2336b35 100644 --- a/catalog/glue.go +++ b/catalog/glue.go @@ -92,19 +92,23 @@ func (c *GlueCatalog) ListTables(ctx context.Context, namespace table.Identifier // LoadTable loads a table from the catalog table details. // // The identifier should contain the Glue database name, then glue table name. -func (c *GlueCatalog) LoadTable(ctx context.Context, identifier table.Identifier) (*table.Table, error) { +func (c *GlueCatalog) LoadTable(ctx context.Context, identifier table.Identifier, props iceberg.Properties) (*table.Table, error) { database, tableName, err := identifierToGlueTable(identifier) if err != nil { return nil, err } + if props == nil { + props = map[string]string{} + } + location, err := c.getTable(ctx, database, tableName) if err != nil { return nil, err } // TODO: consider providing a way to directly access the S3 iofs to enable testing of the catalog. - iofs, err := io.LoadFS(nil, location) + iofs, err := io.LoadFS(props, location) if err != nil { return nil, fmt.Errorf("failed to load table %s.%s: %w", database, tableName, err) } diff --git a/catalog/glue_test.go b/catalog/glue_test.go index aa7e222..1d3c42f 100644 --- a/catalog/glue_test.go +++ b/catalog/glue_test.go @@ -142,7 +142,7 @@ func TestGlueLoadTableIntegration(t *testing.T) { catalog := NewGlueCatalog(WithAwsConfig(awscfg)) - table, err := catalog.LoadTable(context.TODO(), []string{os.Getenv("TEST_DATABASE_NAME"), os.Getenv("TEST_TABLE_NAME")}) + table, err := catalog.LoadTable(context.TODO(), []string{os.Getenv("TEST_DATABASE_NAME"), os.Getenv("TEST_TABLE_NAME")}, nil) assert.NoError(err) assert.Equal([]string{os.Getenv("TEST_TABLE_NAME")}, table.Identifier()) } diff --git a/catalog/rest.go b/catalog/rest.go index 42bdd27..3e6c1cd 100644 --- a/catalog/rest.go +++ b/catalog/rest.go @@ -623,16 +623,11 @@ func (r *RestCatalog) tableFromResponse(identifier []string, metadata table.Meta id = append([]string{r.name}, identifier...) } - tblProps := maps.Clone(r.props) - maps.Copy(tblProps, metadata.Properties()) - for k, v := range config { - tblProps[k] = v - } - - iofs, err := iceio.LoadFS(tblProps, loc) + iofs, err := iceio.LoadFS(config, loc) if err != nil { return nil, err } + return table.New(id, metadata, loc, iofs), nil } @@ -688,7 +683,13 @@ func (r *RestCatalog) CreateTable(ctx context.Context, identifier table.Identifi return nil, err } - return r.tableFromResponse(identifier, ret.Metadata, ret.MetadataLoc, ret.Config) + config := maps.Clone(r.props) + maps.Copy(config, ret.Metadata.Properties()) + for k, v := range ret.Config { + config[k] = v + } + + return r.tableFromResponse(identifier, ret.Metadata, ret.MetadataLoc, config) } func (r *RestCatalog) RegisterTable(ctx context.Context, identifier table.Identifier, metadataLoc string) (*table.Table, error) { @@ -708,10 +709,16 @@ func (r *RestCatalog) RegisterTable(ctx context.Context, identifier table.Identi return nil, err } - return r.tableFromResponse(identifier, ret.Metadata, ret.MetadataLoc, ret.Config) + config := maps.Clone(r.props) + maps.Copy(config, ret.Metadata.Properties()) + for k, v := range ret.Config { + config[k] = v + } + + return r.tableFromResponse(identifier, ret.Metadata, ret.MetadataLoc, config) } -func (r *RestCatalog) LoadTable(ctx context.Context, identifier table.Identifier) (*table.Table, error) { +func (r *RestCatalog) LoadTable(ctx context.Context, identifier table.Identifier, props iceberg.Properties) (*table.Table, error) { ns, tbl, err := splitIdentForPath(identifier) if err != nil { return nil, err @@ -723,7 +730,14 @@ func (r *RestCatalog) LoadTable(ctx context.Context, identifier table.Identifier return nil, err } - return r.tableFromResponse(identifier, ret.Metadata, ret.MetadataLoc, ret.Config) + config := maps.Clone(r.props) + maps.Copy(config, props) + maps.Copy(config, ret.Metadata.Properties()) + for k, v := range ret.Config { + config[k] = v + } + + return r.tableFromResponse(identifier, ret.Metadata, ret.MetadataLoc, config) } func (r *RestCatalog) UpdateTable(ctx context.Context, identifier table.Identifier, requirements []table.Requirement, updates []table.Update) (*table.Table, error) { @@ -748,7 +762,10 @@ func (r *RestCatalog) UpdateTable(ctx context.Context, identifier table.Identifi return nil, err } - return r.tableFromResponse(identifier, ret.Metadata, ret.MetadataLoc, nil) + config := maps.Clone(r.props) + maps.Copy(config, ret.Metadata.Properties()) + + return r.tableFromResponse(identifier, ret.Metadata, ret.MetadataLoc, config) } func (r *RestCatalog) DropTable(ctx context.Context, identifier table.Identifier, purge bool) error { @@ -790,7 +807,7 @@ func (r *RestCatalog) RenameTable(ctx context.Context, from, to table.Identifier return nil, err } - return r.LoadTable(ctx, to) + return r.LoadTable(ctx, to, nil) } func (r *RestCatalog) CreateNamespace(ctx context.Context, namespace table.Identifier, props iceberg.Properties) error { diff --git a/catalog/rest_test.go b/catalog/rest_test.go index c2e6271..212d5a8 100644 --- a/catalog/rest_test.go +++ b/catalog/rest_test.go @@ -804,7 +804,7 @@ func (r *RestCatalogSuite) TestLoadTable200() { cat, err := catalog.NewRestCatalog("rest", r.srv.URL, catalog.WithOAuthToken(TestToken)) r.Require().NoError(err) - tbl, err := cat.LoadTable(context.Background(), catalog.ToRestIdentifier("fokko", "table")) + tbl, err := cat.LoadTable(context.Background(), catalog.ToRestIdentifier("fokko", "table"), nil) r.Require().NoError(err) r.Equal(catalog.ToRestIdentifier("rest", "fokko", "table"), tbl.Identifier()) diff --git a/cmd/iceberg/main.go b/cmd/iceberg/main.go index 30c1960..6d3f4af 100644 --- a/cmd/iceberg/main.go +++ b/cmd/iceberg/main.go @@ -241,7 +241,7 @@ func describe(output Output, cat catalog.Catalog, id string, entityType string) } if (entityType == "any" || entityType == "tbl") && len(ident) > 1 { - tbl, err := cat.LoadTable(ctx, ident) + tbl, err := cat.LoadTable(ctx, ident, nil) if err != nil { if !errors.Is(err, catalog.ErrNoSuchTable) || entityType != "any" { output.Error(err) @@ -261,7 +261,7 @@ func describe(output Output, cat catalog.Catalog, id string, entityType string) } func loadTable(output Output, cat catalog.Catalog, id string) *table.Table { - tbl, err := cat.LoadTable(context.Background(), catalog.ToRestIdentifier(id)) + tbl, err := cat.LoadTable(context.Background(), catalog.ToRestIdentifier(id), nil) if err != nil { output.Error(err) os.Exit(1) From bded0e6f8e8019e27900dbe79a4f7904783d7a9b Mon Sep 17 00:00:00 2001 From: Jonas Weile Date: Wed, 11 Sep 2024 11:32:16 +0000 Subject: [PATCH 06/28] clone fields --- partitions.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/partitions.go b/partitions.go index f39df67..c5c4b5a 100644 --- a/partitions.go +++ b/partitions.go @@ -117,8 +117,9 @@ func (ps PartitionSpec) Equals(other PartitionSpec) bool { return ps.id == other.id && slices.Equal(ps.fields, other.fields) } +// Fields returns a clone of the partition fields in this spec. func (ps *PartitionSpec) Fields() []PartitionField { - return ps.fields + return slices.Clone(ps.fields) } func (ps PartitionSpec) MarshalJSON() ([]byte, error) { From ca9796bf1241dc9a8607ea99a4ed5dd9b021fab2 Mon Sep 17 00:00:00 2001 From: Jonas Weile Date: Wed, 11 Sep 2024 11:32:26 +0000 Subject: [PATCH 07/28] document purge flag --- catalog/catalog.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/catalog/catalog.go b/catalog/catalog.go index 0ac5f8a..14c30f3 100644 --- a/catalog/catalog.go +++ b/catalog/catalog.go @@ -148,7 +148,8 @@ type Catalog interface { ListTables(ctx context.Context, namespace table.Identifier) ([]table.Identifier, error) // LoadTable loads a table from the catalog and returns a Table with the metadata. LoadTable(ctx context.Context, identifier table.Identifier, props iceberg.Properties) (*table.Table, error) - // DropTable tells the catalog to drop the table entirely + // DropTable tells the catalog to drop the table entirely. + // If the purge flag is set, it is requested to purge the underlying table's data and metadata. DropTable(ctx context.Context, identifier table.Identifier, purge bool) error // RenameTable tells the catalog to rename a given table by the identifiers // provided, and then loads and returns the destination table From 1f9edd7173b2a6cee2f9efe0d522a32cd716b8fa Mon Sep 17 00:00:00 2001 From: Jonas Weile Date: Wed, 11 Sep 2024 11:36:10 +0000 Subject: [PATCH 08/28] check for snapshot == nil --- table/metadata.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/table/metadata.go b/table/metadata.go index 7faeab5..7b2531e 100644 --- a/table/metadata.go +++ b/table/metadata.go @@ -221,6 +221,10 @@ func (b *MetadataBuilder) AddPartitionSpec(spec *iceberg.PartitionSpec, initial } func (b *MetadataBuilder) AddSnapshot(snapshot *Snapshot) (*MetadataBuilder, error) { + if snapshot == nil { + return nil, nil + } + if len(b.schemaList) == 0 { return nil, errors.New("can't add snapshot with no added schemas") } else if len(b.specs) == 0 { From c48301cc099ab79b6db9839455b7131b693eb03d Mon Sep 17 00:00:00 2001 From: Jonas Weile Date: Wed, 11 Sep 2024 11:38:08 +0000 Subject: [PATCH 09/28] update logic for initial updates --- table/metadata.go | 41 +++++++++++++++++++++++++++++++---------- table/updates.go | 24 ++++++++++++++---------- 2 files changed, 45 insertions(+), 20 deletions(-) diff --git a/table/metadata.go b/table/metadata.go index 7b2531e..a176134 100644 --- a/table/metadata.go +++ b/table/metadata.go @@ -179,15 +179,21 @@ func MetadataBuilderFromBase(metadata Metadata) (*MetadataBuilder, error) { return b, nil } -func (b *MetadataBuilder) AddSchema(schema *iceberg.Schema, newLastColumnID int) (*MetadataBuilder, error) { +func (b *MetadataBuilder) AddSchema(schema *iceberg.Schema, newLastColumnID int, initial bool) (*MetadataBuilder, error) { if newLastColumnID < b.lastColumnId { - return nil, fmt.Errorf("invalid last column id %d, must be >= %d", - newLastColumnID, b.lastColumnId) + return nil, fmt.Errorf("%w: newLastColumnID %d, must be >= %d", iceberg.ErrInvalidArgument, newLastColumnID, b.lastColumnId) + } + + var schemas []*iceberg.Schema + if initial { + schemas = []*iceberg.Schema{schema} + } else { + schemas = append(b.schemaList, schema) } - b.updates = append(b.updates, NewAddSchemaUpdate(schema, newLastColumnID)) b.lastColumnId = newLastColumnID - b.schemaList = append(b.schemaList, schema) + b.schemaList = schemas + b.updates = append(b.updates, NewAddSchemaUpdate(schema, newLastColumnID, initial)) return b, nil } @@ -211,10 +217,17 @@ func (b *MetadataBuilder) AddPartitionSpec(spec *iceberg.PartitionSpec, initial if b.lastPartitionID != nil { prev = *b.lastPartitionID } - lastPartitionID := max(maxFieldID, prev) + + var specs []iceberg.PartitionSpec + if initial { + specs = []iceberg.PartitionSpec{*spec} + } else { + specs = append(b.specs, *spec) + } + + b.specs = specs b.lastPartitionID = &lastPartitionID - b.specs = append(b.specs, *spec) b.updates = append(b.updates, NewAddPartitionSpecUpdate(spec, initial)) return b, nil @@ -248,9 +261,17 @@ func (b *MetadataBuilder) AddSnapshot(snapshot *Snapshot) (*MetadataBuilder, err return b, nil } -func (b *MetadataBuilder) AddSortOrder(sortOrder *SortOrder) (*MetadataBuilder, error) { - b.updates = append(b.updates, NewAddSortOrderUpdate(sortOrder)) - b.sortOrderList = append(b.sortOrderList, *sortOrder) +func (b *MetadataBuilder) AddSortOrder(sortOrder *SortOrder, initial bool) (*MetadataBuilder, error) { + var sortOrders []SortOrder + if initial { + sortOrders = []SortOrder{*sortOrder} + } else { + sortOrders = append(b.sortOrderList, *sortOrder) + } + + b.sortOrderList = sortOrders + b.updates = append(b.updates, NewAddSortOrderUpdate(sortOrder, initial)) + return b, nil } diff --git a/table/updates.go b/table/updates.go index 28415a8..88a752a 100644 --- a/table/updates.go +++ b/table/updates.go @@ -58,18 +58,20 @@ type AddSchemaUpdate struct { baseUpdate Schema *iceberg.Schema `json:"schema"` LastColumnID int `json:"last-column-id"` + initial bool } -func NewAddSchemaUpdate(schema *iceberg.Schema, lastColumnID int) *AddSchemaUpdate { +func NewAddSchemaUpdate(schema *iceberg.Schema, lastColumnID int, initial bool) *AddSchemaUpdate { return &AddSchemaUpdate{ baseUpdate: baseUpdate{ActionName: "add-schema"}, Schema: schema, LastColumnID: lastColumnID, + initial: initial, } } func (u *AddSchemaUpdate) Apply(builder *MetadataBuilder) error { - _, err := builder.AddSchema(u.Schema, u.LastColumnID) + _, err := builder.AddSchema(u.Schema, u.LastColumnID, u.initial) return err } @@ -92,20 +94,20 @@ func (u *SetCurrentSchemaUpdate) Apply(builder *MetadataBuilder) error { type AddPartitionSpecUpdate struct { baseUpdate - Spec *iceberg.PartitionSpec `json:"spec"` - initialChange bool + Spec *iceberg.PartitionSpec `json:"spec"` + initial bool } func NewAddPartitionSpecUpdate(spec *iceberg.PartitionSpec, initial bool) *AddPartitionSpecUpdate { return &AddPartitionSpecUpdate{ - baseUpdate: baseUpdate{ActionName: "add-spec"}, - Spec: spec, - initialChange: initial, + baseUpdate: baseUpdate{ActionName: "add-spec"}, + Spec: spec, + initial: initial, } } func (u *AddPartitionSpecUpdate) Apply(builder *MetadataBuilder) error { - _, err := builder.AddPartitionSpec(u.Spec, u.initialChange) + _, err := builder.AddPartitionSpec(u.Spec, u.initial) return err } @@ -129,17 +131,19 @@ func (u *SetDefaultSpecUpdate) Apply(builder *MetadataBuilder) error { type AddSortOrderUpdate struct { baseUpdate SortOrder *SortOrder `json:"sort-order"` + initial bool } -func NewAddSortOrderUpdate(sortOrder *SortOrder) *AddSortOrderUpdate { +func NewAddSortOrderUpdate(sortOrder *SortOrder, initial bool) *AddSortOrderUpdate { return &AddSortOrderUpdate{ baseUpdate: baseUpdate{ActionName: "add-sort-order"}, SortOrder: sortOrder, + initial: initial, } } func (u *AddSortOrderUpdate) Apply(builder *MetadataBuilder) error { - _, err := builder.AddSortOrder(u.SortOrder) + _, err := builder.AddSortOrder(u.SortOrder, u.initial) return err } From c48e9f772d59d1ef1191cbd524f2779207e8aacf Mon Sep 17 00:00:00 2001 From: Jonas Weile Date: Wed, 11 Sep 2024 11:38:58 +0000 Subject: [PATCH 10/28] update SetSnapshotRef logic --- table/metadata.go | 69 ++++++++++++++++++++++++++++++++++++----------- table/updates.go | 25 +++++++++++------ 2 files changed, 71 insertions(+), 23 deletions(-) diff --git a/table/metadata.go b/table/metadata.go index a176134..2c81eea 100644 --- a/table/metadata.go +++ b/table/metadata.go @@ -391,19 +391,64 @@ func (b *MetadataBuilder) SetProperties(props iceberg.Properties) (*MetadataBuil return b, nil } +type setSnapshotRefOption func(*SnapshotRef) error + +func WithMaxRefAgeMs(maxRefAgeMs int64) setSnapshotRefOption { + return func(ref *SnapshotRef) error { + if maxRefAgeMs <= 0 { + return fmt.Errorf("%w: maxRefAgeMs %d, must be > 0", iceberg.ErrInvalidArgument, maxRefAgeMs) + } + ref.MaxRefAgeMs = &maxRefAgeMs + return nil + } +} + +func WithMaxSnapshotAgeMs(maxSnapshotAgeMs int64) setSnapshotRefOption { + return func(ref *SnapshotRef) error { + if maxSnapshotAgeMs <= 0 { + return fmt.Errorf("%w: maxSnapshotAgeMs %d, must be > 0", iceberg.ErrInvalidArgument, maxSnapshotAgeMs) + } + ref.MaxSnapshotAgeMs = &maxSnapshotAgeMs + return nil + } +} + +func WithMinSnapshotsToKeep(minSnapshotsToKeep int) setSnapshotRefOption { + return func(ref *SnapshotRef) error { + if minSnapshotsToKeep <= 0 { + return fmt.Errorf("%w: minSnapshotsToKeep %d, must be > 0", iceberg.ErrInvalidArgument, minSnapshotsToKeep) + } + ref.MinSnapshotsToKeep = &minSnapshotsToKeep + return nil + } +} + func (b *MetadataBuilder) SetSnapshotRef( name string, snapshotID int64, refType RefType, - maxRefAgeMs, maxSnapshotAgeMs *int64, - minSnapshotsToKeep *int, + options ...setSnapshotRefOption, ) (*MetadataBuilder, error) { ref := SnapshotRef{ - SnapshotID: snapshotID, - SnapshotRefType: refType, - MinSnapshotsToKeep: minSnapshotsToKeep, - MaxRefAgeMs: maxRefAgeMs, - MaxSnapshotAgeMs: maxSnapshotAgeMs, + SnapshotID: snapshotID, + SnapshotRefType: refType, + } + for _, opt := range options { + if err := opt(&ref); err != nil { + return nil, fmt.Errorf("invalid snapshot ref option: %w", err) + } + } + + var maxRefAgeMs, maxSnapshotAgeMs int64 + var minSnapshotsToKeep int + if ref.MaxRefAgeMs != nil { + maxRefAgeMs = *ref.MaxRefAgeMs + } + if ref.MaxSnapshotAgeMs != nil { + maxSnapshotAgeMs = *ref.MaxSnapshotAgeMs + } + if ref.MinSnapshotsToKeep != nil { + minSnapshotsToKeep = *ref.MinSnapshotsToKeep } if existingRef, ok := b.refs[name]; ok && existingRef.Equals(ref) { @@ -415,8 +460,8 @@ func (b *MetadataBuilder) SetSnapshotRef( return nil, fmt.Errorf("can't set snapshot ref %s to unknown snapshot %d: %w", name, snapshotID, err) } - b.updates = append(b.updates, NewSetSnapshotRefUpdate(name, snapshotID, refType, maxRefAgeMs, maxSnapshotAgeMs, minSnapshotsToKeep)) if refType == MainBranch { + b.updates = append(b.updates, NewSetSnapshotRefUpdate(name, snapshotID, refType, maxRefAgeMs, maxSnapshotAgeMs, minSnapshotsToKeep)) b.currentSnapshotID = &snapshotID b.snapshotLog = append(b.snapshotLog, SnapshotLogEntry{ SnapshotID: snapshotID, @@ -429,13 +474,7 @@ func (b *MetadataBuilder) SetSnapshotRef( b.lastUpdatedMS = snapshot.TimestampMs } - b.refs[name] = SnapshotRef{ - SnapshotID: snapshotID, - SnapshotRefType: refType, - MinSnapshotsToKeep: minSnapshotsToKeep, - MaxRefAgeMs: maxRefAgeMs, - MaxSnapshotAgeMs: maxSnapshotAgeMs, - } + b.refs[name] = ref return b, nil } diff --git a/table/updates.go b/table/updates.go index 88a752a..b6e9442 100644 --- a/table/updates.go +++ b/table/updates.go @@ -186,17 +186,17 @@ type SetSnapshotRefUpdate struct { RefName string `json:"ref-name"` RefType RefType `json:"type"` SnapshotID int64 `json:"snapshot-id"` - MaxRefAgeMs *int64 `json:"max-ref-age-ms,omitempty"` - MaxSnapshotAgeMs *int64 `json:"max-snapshot-age-ms,omitempty"` - MinSnapshotsToKeep *int `json:"min-snapshots-to-keep,omitempty"` + MaxRefAgeMs int64 `json:"max-ref-age-ms,omitempty"` + MaxSnapshotAgeMs int64 `json:"max-snapshot-age-ms,omitempty"` + MinSnapshotsToKeep int `json:"min-snapshots-to-keep,omitempty"` } func NewSetSnapshotRefUpdate( name string, snapshotID int64, refType RefType, - maxRefAgeMs, maxSnapshotAgeMs *int64, - minSnapshotsToKeep *int, + maxRefAgeMs, maxSnapshotAgeMs int64, + minSnapshotsToKeep int, ) *SetSnapshotRefUpdate { return &SetSnapshotRefUpdate{ baseUpdate: baseUpdate{ActionName: "set-snapshot-ref"}, @@ -210,13 +210,22 @@ func NewSetSnapshotRefUpdate( } func (u *SetSnapshotRefUpdate) Apply(builder *MetadataBuilder) error { + opts := []setSnapshotRefOption{} + if u.MaxRefAgeMs != 0 { + opts = append(opts, WithMaxRefAgeMs(u.MaxRefAgeMs)) + } + if u.MaxSnapshotAgeMs != 0 { + opts = append(opts, WithMaxSnapshotAgeMs(u.MaxSnapshotAgeMs)) + } + if u.MinSnapshotsToKeep != 0 { + opts = append(opts, WithMinSnapshotsToKeep(u.MinSnapshotsToKeep)) + } + _, err := builder.SetSnapshotRef( u.RefName, u.SnapshotID, u.RefType, - u.MaxRefAgeMs, - u.MaxSnapshotAgeMs, - u.MinSnapshotsToKeep, + opts..., ) return err } From adcef8faed6bb817b9efd86b711d738f301d1f80 Mon Sep 17 00:00:00 2001 From: Jonas Weile Date: Wed, 11 Sep 2024 12:07:39 +0000 Subject: [PATCH 11/28] reduce duplication --- table/metadata.go | 128 +++++++++++++++------------------------------- 1 file changed, 42 insertions(+), 86 deletions(-) diff --git a/table/metadata.go b/table/metadata.go index 2c81eea..258fcff 100644 --- a/table/metadata.go +++ b/table/metadata.go @@ -290,8 +290,12 @@ func (b *MetadataBuilder) RemoveProperties(keys []string) (*MetadataBuilder, err func (b *MetadataBuilder) SetCurrentSchemaID(currentSchemaID int) (*MetadataBuilder, error) { if currentSchemaID == -1 { - currentSchemaID = b.MaxSchemaID() - if !b.isAddedSchemaID(currentSchemaID) { + currentSchemaID = maxBy(b.schemaList, func(s *iceberg.Schema) int { + return s.ID + }) + if !containsBy(b.updates, func(u Update) bool { + return u.Action() == "add-schema" && u.(*AddSchemaUpdate).Schema.ID == currentSchemaID + }) { return nil, errors.New("can't set current schema to last added schema, no schema has been added") } } @@ -312,8 +316,12 @@ func (b *MetadataBuilder) SetCurrentSchemaID(currentSchemaID int) (*MetadataBuil func (b *MetadataBuilder) SetDefaultSortOrderID(defaultSortOrderID int) (*MetadataBuilder, error) { if defaultSortOrderID == -1 { - defaultSortOrderID = b.MaxSortOrderID() - if !b.isAddedSortOrder(defaultSortOrderID) { + defaultSortOrderID = maxBy(b.sortOrderList, func(s SortOrder) int { + return s.OrderID + }) + if !containsBy(b.updates, func(u Update) bool { + return u.Action() == "add-sort-order" && u.(*AddSortOrderUpdate).SortOrder.OrderID == defaultSortOrderID + }) { return nil, fmt.Errorf("can't set default sort order to last added with no added sort orders") } } @@ -333,8 +341,12 @@ func (b *MetadataBuilder) SetDefaultSortOrderID(defaultSortOrderID int) (*Metada func (b *MetadataBuilder) SetDefaultSpecID(defaultSpecID int) (*MetadataBuilder, error) { if defaultSpecID == -1 { - defaultSpecID = b.MaxSpecID() - if !b.isAddedSpecID(defaultSpecID) { + defaultSpecID = maxBy(b.specs, func(s iceberg.PartitionSpec) int { + return s.ID() + }) + if !containsBy(b.updates, func(u Update) bool { + return u.Action() == "add-partition-spec" && u.(*AddPartitionSpecUpdate).Spec.ID() == defaultSpecID + }) { return nil, fmt.Errorf("can't set default spec to last added with no added partition specs") } } @@ -470,7 +482,9 @@ func (b *MetadataBuilder) SetSnapshotRef( b.lastUpdatedMS = time.Now().Local().UnixMilli() } - if b.isAddedSnapshot(snapshotID) { + if containsBy(b.updates, func(u Update) bool { + return u.Action() == "add-snapshot" && u.(*AddSnapshotUpdate).Snapshot.SnapshotID == snapshotID + }) { b.lastUpdatedMS = snapshot.TimestampMs } @@ -518,7 +532,7 @@ func (b *MetadataBuilder) GetSchemaByID(id int) (*iceberg.Schema, error) { } } - return nil, fmt.Errorf("schema with id %d not found", id) + return nil, fmt.Errorf("%w: schema with id %d not found", iceberg.ErrInvalidArgument, id) } func (b *MetadataBuilder) GetSpecByID(id int) (*iceberg.PartitionSpec, error) { @@ -541,39 +555,6 @@ func (b *MetadataBuilder) GetSortOrderByID(id int) (*SortOrder, error) { return nil, fmt.Errorf("sort order with id %d not found", id) } -func (b *MetadataBuilder) MaxSchemaID() int { - max := 0 - for _, s := range b.schemaList { - if s.ID > max { - max = s.ID - } - } - - return max -} - -func (b *MetadataBuilder) MaxSpecID() int { - max := 0 - for _, s := range b.specs { - if s.ID() > max { - max = s.ID() - } - } - - return max -} - -func (b *MetadataBuilder) MaxSortOrderID() int { - max := 0 - for _, s := range b.sortOrderList { - if s.OrderID > max { - max = s.OrderID - } - } - - return max -} - func (b *MetadataBuilder) SnapshotByID(id int64) (*Snapshot, error) { for _, s := range b.snapshotList { if s.SnapshotID == id { @@ -584,51 +565,6 @@ func (b *MetadataBuilder) SnapshotByID(id int64) (*Snapshot, error) { return nil, fmt.Errorf("snapshot with id %d not found", id) } -func (b *MetadataBuilder) isAddedSchemaID(id int) bool { - for _, u := range b.updates { - if u.Action() == "add-schema" && - u.(*AddSchemaUpdate).Schema.ID == id { - return true - } - } - - return false -} - -func (b *MetadataBuilder) isAddedSnapshot(id int64) bool { - for _, u := range b.updates { - if u.Action() == "add-snapshot" && - u.(*AddSnapshotUpdate).Snapshot.SnapshotID == id { - return true - } - } - - return false -} - -func (b *MetadataBuilder) isAddedSpecID(id int) bool { - for _, u := range b.updates { - if u.Action() == "add-partition-spec" && - u.(*AddPartitionSpecUpdate).Spec.ID() == id { - return true - } - - } - - return false -} - -func (b *MetadataBuilder) isAddedSortOrder(id int) bool { - for _, u := range b.updates { - if u.Action() == "add-sort-order" && - u.(*AddSortOrderUpdate).SortOrder.OrderID == id { - return true - } - } - - return false -} - func (b *MetadataBuilder) Build() (Metadata, error) { common := b.buildCommonMetadata() switch b.formatVersion { @@ -656,6 +592,26 @@ func (b *MetadataBuilder) Build() (Metadata, error) { } } +// containsBy returns true if found(e) is true for any e in elems. +func containsBy[S []E, E any](elems S, found func(e E) bool) bool { + for _, e := range elems { + if found(e) { + return true + } + } + return false +} + +// maxBy returns the maximum value of extract(e) for all e in elems. +// If elems is empty, returns 0. +func maxBy[S []E, E any](elems S, extract func(e E) int) int { + m := 0 + for _, e := range elems { + m = max(m, extract(e)) + } + return m +} + var ( ErrInvalidMetadataFormatVersion = errors.New("invalid or missing format-version in table metadata") ErrInvalidMetadata = errors.New("invalid metadata") From ca88a69b71bf68473ef2fc7776412bfbbab42b17 Mon Sep 17 00:00:00 2001 From: Jonas Weile Date: Wed, 11 Sep 2024 12:20:16 +0000 Subject: [PATCH 12/28] maintain immutability --- table/metadata.go | 24 ++++++++++++------------ table/metadata_internal_test.go | 2 +- table/refs.go | 6 +++--- table/refs_test.go | 2 +- 4 files changed, 17 insertions(+), 17 deletions(-) diff --git a/table/metadata.go b/table/metadata.go index 258fcff..d7fc491 100644 --- a/table/metadata.go +++ b/table/metadata.go @@ -410,7 +410,7 @@ func WithMaxRefAgeMs(maxRefAgeMs int64) setSnapshotRefOption { if maxRefAgeMs <= 0 { return fmt.Errorf("%w: maxRefAgeMs %d, must be > 0", iceberg.ErrInvalidArgument, maxRefAgeMs) } - ref.MaxRefAgeMs = &maxRefAgeMs + ref.MaxRefAgeMs = maxRefAgeMs return nil } } @@ -420,7 +420,7 @@ func WithMaxSnapshotAgeMs(maxSnapshotAgeMs int64) setSnapshotRefOption { if maxSnapshotAgeMs <= 0 { return fmt.Errorf("%w: maxSnapshotAgeMs %d, must be > 0", iceberg.ErrInvalidArgument, maxSnapshotAgeMs) } - ref.MaxSnapshotAgeMs = &maxSnapshotAgeMs + ref.MaxSnapshotAgeMs = maxSnapshotAgeMs return nil } } @@ -430,7 +430,7 @@ func WithMinSnapshotsToKeep(minSnapshotsToKeep int) setSnapshotRefOption { if minSnapshotsToKeep <= 0 { return fmt.Errorf("%w: minSnapshotsToKeep %d, must be > 0", iceberg.ErrInvalidArgument, minSnapshotsToKeep) } - ref.MinSnapshotsToKeep = &minSnapshotsToKeep + ref.MinSnapshotsToKeep = minSnapshotsToKeep return nil } } @@ -453,14 +453,14 @@ func (b *MetadataBuilder) SetSnapshotRef( var maxRefAgeMs, maxSnapshotAgeMs int64 var minSnapshotsToKeep int - if ref.MaxRefAgeMs != nil { - maxRefAgeMs = *ref.MaxRefAgeMs + if ref.MaxRefAgeMs > 0 { + maxRefAgeMs = ref.MaxRefAgeMs } - if ref.MaxSnapshotAgeMs != nil { - maxSnapshotAgeMs = *ref.MaxSnapshotAgeMs + if ref.MaxSnapshotAgeMs > 0 { + maxSnapshotAgeMs = ref.MaxSnapshotAgeMs } - if ref.MinSnapshotsToKeep != nil { - minSnapshotsToKeep = *ref.MinSnapshotsToKeep + if ref.MinSnapshotsToKeep > 0 { + minSnapshotsToKeep = ref.MinSnapshotsToKeep } if existingRef, ok := b.refs[name]; ok && existingRef.Equals(ref) { @@ -685,9 +685,9 @@ type commonMetadata struct { } func (c *commonMetadata) Ref() SnapshotRef { return c.SnapshotRefs[MainBranch] } -func (c *commonMetadata) Refs() map[string]SnapshotRef { return c.SnapshotRefs } -func (c *commonMetadata) SnapshotLogs() []SnapshotLogEntry { return c.SnapshotLog } -func (c *commonMetadata) PreviousFiles() []MetadataLogEntry { return c.MetadataLog } +func (c *commonMetadata) Refs() map[string]SnapshotRef { return maps.Clone(c.SnapshotRefs) } +func (c *commonMetadata) SnapshotLogs() []SnapshotLogEntry { return slices.Clone(c.SnapshotLog) } +func (c *commonMetadata) PreviousFiles() []MetadataLogEntry { return slices.Clone(c.MetadataLog) } func (c *commonMetadata) Equals(other *commonMetadata) bool { switch { case c.LastPartitionID == nil && other.LastPartitionID != nil: diff --git a/table/metadata_internal_test.go b/table/metadata_internal_test.go index 95001b8..50daced 100644 --- a/table/metadata_internal_test.go +++ b/table/metadata_internal_test.go @@ -412,7 +412,7 @@ func TestV2RefCreation(t *testing.T) { "test": { SnapshotID: 3051729675574597004, SnapshotRefType: TagRef, - MaxRefAgeMs: &maxRefAge, + MaxRefAgeMs: maxRefAge, }, }, meta.SnapshotRefs) } diff --git a/table/refs.go b/table/refs.go index f0eb697..40e7d12 100644 --- a/table/refs.go +++ b/table/refs.go @@ -41,9 +41,9 @@ var ( 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"` + MinSnapshotsToKeep int `json:"min-snapshots-to-keep,omitempty"` + MaxSnapshotAgeMs int64 `json:"max-snapshot-age-ms,omitempty"` + MaxRefAgeMs int64 `json:"max-ref-age-ms,omitempty"` } func (s *SnapshotRef) Equals(rhs SnapshotRef) bool { diff --git a/table/refs_test.go b/table/refs_test.go index d8b54e4..fddde57 100644 --- a/table/refs_test.go +++ b/table/refs_test.go @@ -66,7 +66,7 @@ func TestSnapshotTagRef(t *testing.T) { assert.Equal(t, table.TagRef, snapRef.SnapshotRefType) assert.Equal(t, int64(3051729675574597004), snapRef.SnapshotID) - assert.Equal(t, 10, *snapRef.MinSnapshotsToKeep) + assert.Equal(t, 10, snapRef.MinSnapshotsToKeep) assert.Nil(t, snapRef.MaxRefAgeMs) assert.Nil(t, snapRef.MaxSnapshotAgeMs) } From bdef610d4d0c7982414b06300f472d7350729fe6 Mon Sep 17 00:00:00 2001 From: Jonas Weile Date: Wed, 11 Sep 2024 12:51:26 +0000 Subject: [PATCH 13/28] handle pointers --- schema.go | 8 ++++---- table/metadata.go | 26 +++++++++++++++++++++++++- 2 files changed, 29 insertions(+), 5 deletions(-) diff --git a/schema.go b/schema.go index a204b54..d6ad24d 100644 --- a/schema.go +++ b/schema.go @@ -297,14 +297,14 @@ func (s *Schema) accessorForField(id int) (accessor, bool) { // Equals compares the fields and identifierIDs, but does not compare // the schema ID itself. func (s *Schema) Equals(other *Schema) bool { - if other == nil { - return false - } - if s == other { return true } + if s == nil || other == nil { + return false + } + if len(s.fields) != len(other.fields) { return false } diff --git a/table/metadata.go b/table/metadata.go index d7fc491..6daa333 100644 --- a/table/metadata.go +++ b/table/metadata.go @@ -689,6 +689,14 @@ func (c *commonMetadata) Refs() map[string]SnapshotRef { return maps.Clone( func (c *commonMetadata) SnapshotLogs() []SnapshotLogEntry { return slices.Clone(c.SnapshotLog) } func (c *commonMetadata) PreviousFiles() []MetadataLogEntry { return slices.Clone(c.MetadataLog) } func (c *commonMetadata) Equals(other *commonMetadata) bool { + if c == nil || other == nil { + return c == other + } + + if c == other { + return true + } + switch { case c.LastPartitionID == nil && other.LastPartitionID != nil: fallthrough @@ -909,12 +917,20 @@ func (m *metadataV1) Equals(other Metadata) bool { return false } + if m == rhs { + return true + } + + if m == nil || rhs == nil { + return false + } + return m.Schema.Equals(rhs.Schema) && slices.Equal(m.Partition, rhs.Partition) && m.commonMetadata.Equals(&rhs.commonMetadata) } func (m *metadataV1) preValidate() { - if len(m.SchemaList) == 0 { + if len(m.SchemaList) == 0 && m.Schema != nil { m.SchemaList = []*iceberg.Schema{m.Schema} } @@ -976,6 +992,14 @@ func (m *metadataV2) Equals(other Metadata) bool { return false } + if m == rhs { + return true + } + + if m == nil || rhs == nil { + return false + } + return m.LastSequenceNumber == rhs.LastSequenceNumber && m.commonMetadata.Equals(&rhs.commonMetadata) } From fa6c5b58fd3b4eec193e46488fcb5c2f68bec8b1 Mon Sep 17 00:00:00 2001 From: Jonas Weile Date: Wed, 11 Sep 2024 12:52:49 +0000 Subject: [PATCH 14/28] add apache licenses --- table/requirements.go | 17 +++++++++++++++++ table/updates.go | 17 +++++++++++++++++ 2 files changed, 34 insertions(+) diff --git a/table/requirements.go b/table/requirements.go index 9b00636..77124c3 100644 --- a/table/requirements.go +++ b/table/requirements.go @@ -1,3 +1,20 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + package table import ( diff --git a/table/updates.go b/table/updates.go index b6e9442..1947add 100644 --- a/table/updates.go +++ b/table/updates.go @@ -1,3 +1,20 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + package table import ( From 795a6c9ef4773d8bb61ab9acb1cf9d28f9020999 Mon Sep 17 00:00:00 2001 From: Jonas Weile Date: Wed, 11 Sep 2024 14:19:57 +0000 Subject: [PATCH 15/28] naming --- catalog/rest.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/catalog/rest.go b/catalog/rest.go index 3e6c1cd..746b6fd 100644 --- a/catalog/rest.go +++ b/catalog/rest.go @@ -124,31 +124,31 @@ func (t *loadTableResponse) UnmarshalJSON(b []byte) (err error) { type createTableOption func(*createTableRequest) -func WithTableLocation(loc string) createTableOption { +func WithLocation(loc string) createTableOption { return func(req *createTableRequest) { req.Location = strings.TrimRight(loc, "/") } } -func WithTablePartitionSpec(spec iceberg.PartitionSpec) createTableOption { +func WithPartitionSpec(spec iceberg.PartitionSpec) createTableOption { return func(req *createTableRequest) { req.PartitionSpec = spec } } -func WithTableWriteOrder(order table.SortOrder) createTableOption { +func WithWriteOrder(order table.SortOrder) createTableOption { return func(req *createTableRequest) { req.WriteOrder = order } } -func WithTableStagingCreate() createTableOption { +func WithStageCreate() createTableOption { return func(req *createTableRequest) { req.StageCreate = true } } -func WithTableProperties(props iceberg.Properties) createTableOption { +func WithProperties(props iceberg.Properties) createTableOption { return func(req *createTableRequest) { req.Props = props } From a952f2bf1317160a9cf131a6342401ff4d8280a4 Mon Sep 17 00:00:00 2001 From: Jonas Weile Date: Wed, 11 Sep 2024 14:42:07 +0000 Subject: [PATCH 16/28] docstrings for updates --- table/updates.go | 54 ++++++++++++++++++++++++++++++++++++++++++++---- 1 file changed, 50 insertions(+), 4 deletions(-) diff --git a/table/updates.go b/table/updates.go index 1947add..00a73e5 100644 --- a/table/updates.go +++ b/table/updates.go @@ -24,24 +24,31 @@ import ( "github.com/google/uuid" ) +// Update represents a change to a table's metadata. 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 } +// baseUpdate contains the common fields for all updates. It is used to identify the type +// of the update. type baseUpdate struct { - ActionName string `json:"action"` + ActionName string `json:"Action"` } func (u *baseUpdate) Action() string { return u.ActionName } +// AssignUUIDUpdate assigns a UUID to the table metadata. type AssignUUIDUpdate struct { baseUpdate UUID uuid.UUID `json:"uuid"` } +// NewAssignUUIDUpdate creates a new AssignUUIDUpdate with the given UUID. func NewAssignUUIDUpdate(uuid uuid.UUID) *AssignUUIDUpdate { return &AssignUUIDUpdate{ baseUpdate: baseUpdate{ActionName: "assign-uuid"}, @@ -49,16 +56,19 @@ func NewAssignUUIDUpdate(uuid uuid.UUID) *AssignUUIDUpdate { } } +// Apply updates the UUID on the given metadata builder. func (u *AssignUUIDUpdate) Apply(builder *MetadataBuilder) error { _, err := builder.SetUUID(u.UUID) return err } +// UpgradeFormatVersionUpdate upgrades the format version of the table metadata. type UpgradeFormatVersionUpdate struct { baseUpdate FormatVersion int `json:"format-version"` } +// NewUpgradeFormatVersionUpdate creates a new UpgradeFormatVersionUpdate with the given format version. func NewUpgradeFormatVersionUpdate(formatVersion int) *UpgradeFormatVersionUpdate { return &UpgradeFormatVersionUpdate{ baseUpdate: baseUpdate{ActionName: "upgrade-format-version"}, @@ -66,11 +76,13 @@ func NewUpgradeFormatVersionUpdate(formatVersion int) *UpgradeFormatVersionUpdat } } +// Apply upgrades the format version on the given metadata builder. func (u *UpgradeFormatVersionUpdate) Apply(builder *MetadataBuilder) error { _, err := builder.SetFormatVersion(u.FormatVersion) return err } +// AddSchemaUpdate adds a schema to the table metadata. type AddSchemaUpdate struct { baseUpdate Schema *iceberg.Schema `json:"schema"` @@ -78,6 +90,9 @@ type AddSchemaUpdate struct { initial bool } +// NewAddSchemaUpdate creates a new AddSchemaUpdate with the given schema and last column ID. +// If the initial flag is set to true, the schema is considered the initial schema of the table, +// and all previously added schemas in the metadata builder are removed. func NewAddSchemaUpdate(schema *iceberg.Schema, lastColumnID int, initial bool) *AddSchemaUpdate { return &AddSchemaUpdate{ baseUpdate: baseUpdate{ActionName: "add-schema"}, @@ -92,11 +107,13 @@ func (u *AddSchemaUpdate) Apply(builder *MetadataBuilder) error { return err } +// SetCurrentSchemaUpdate sets the current schema of the table metadata. type SetCurrentSchemaUpdate struct { baseUpdate SchemaID int `json:"schema-id"` } +// NewSetCurrentSchemaUpdate creates a new SetCurrentSchemaUpdate with the given schema ID. func NewSetCurrentSchemaUpdate(id int) *SetCurrentSchemaUpdate { return &SetCurrentSchemaUpdate{ baseUpdate: baseUpdate{ActionName: "set-current-schema"}, @@ -109,12 +126,16 @@ func (u *SetCurrentSchemaUpdate) Apply(builder *MetadataBuilder) error { return err } +// AddPartitionSpecUpdate adds a partition spec to the table metadata. type AddPartitionSpecUpdate struct { baseUpdate Spec *iceberg.PartitionSpec `json:"spec"` initial bool } +// NewAddPartitionSpecUpdate creates a new AddPartitionSpecUpdate with the given partition spec. +// 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 remoed. func NewAddPartitionSpecUpdate(spec *iceberg.PartitionSpec, initial bool) *AddPartitionSpecUpdate { return &AddPartitionSpecUpdate{ baseUpdate: baseUpdate{ActionName: "add-spec"}, @@ -128,11 +149,13 @@ func (u *AddPartitionSpecUpdate) Apply(builder *MetadataBuilder) error { return err } +// SetDefaultSpecUpdate sets the default partition spec of the table metadata. type SetDefaultSpecUpdate struct { baseUpdate SpecID int `json:"spec-id"` } +// NewSetDefaultSpecUpdate creates a new SetDefaultSpecUpdate with the given spec ID. func NewSetDefaultSpecUpdate(id int) *SetDefaultSpecUpdate { return &SetDefaultSpecUpdate{ baseUpdate: baseUpdate{ActionName: "set-default-spec"}, @@ -145,12 +168,16 @@ func (u *SetDefaultSpecUpdate) Apply(builder *MetadataBuilder) error { return err } +// AddSortOrderUpdate adds a sort order to the table metadata. type AddSortOrderUpdate struct { baseUpdate SortOrder *SortOrder `json:"sort-order"` initial bool } +// NewAddSortOrderUpdate creates a new AddSortOrderUpdate with the given sort order. +// 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 NewAddSortOrderUpdate(sortOrder *SortOrder, initial bool) *AddSortOrderUpdate { return &AddSortOrderUpdate{ baseUpdate: baseUpdate{ActionName: "add-sort-order"}, @@ -164,11 +191,13 @@ func (u *AddSortOrderUpdate) Apply(builder *MetadataBuilder) error { return err } +// SetDefaultSortOrderUpdate sets the default sort order of the table metadata. type SetDefaultSortOrderUpdate struct { baseUpdate SortOrderID int `json:"sort-order-id"` } +// NewSetDefaultSortOrderUpdate creates a new SetDefaultSortOrderUpdate with the given sort order ID. func NewSetDefaultSortOrderUpdate(id int) *SetDefaultSortOrderUpdate { return &SetDefaultSortOrderUpdate{ baseUpdate: baseUpdate{ActionName: "set-default-sort-order"}, @@ -181,11 +210,13 @@ func (u *SetDefaultSortOrderUpdate) Apply(builder *MetadataBuilder) error { return err } +// AddSnapshotUpdate adds a snapshot to the table metadata. type AddSnapshotUpdate struct { baseUpdate Snapshot *Snapshot `json:"snapshot"` } +// NewAddSnapshotUpdate creates a new AddSnapshotUpdate with the given snapshot. func NewAddSnapshotUpdate(snapshot *Snapshot) *AddSnapshotUpdate { return &AddSnapshotUpdate{ baseUpdate: baseUpdate{ActionName: "add-snapshot"}, @@ -198,6 +229,7 @@ func (u *AddSnapshotUpdate) Apply(builder *MetadataBuilder) error { return err } +// SetCurrentSnapshotUpdate sets the current snapshot of the table metadata. type SetSnapshotRefUpdate struct { baseUpdate RefName string `json:"ref-name"` @@ -208,6 +240,8 @@ type SetSnapshotRefUpdate struct { MinSnapshotsToKeep int `json:"min-snapshots-to-keep,omitempty"` } +// NewSetSnapshotRefUpdate creates a new SetSnapshotRefUpdate with the given snapshot reference information. +// MaxRefAgeMs, MaxSnapshotAgeMs, and MinSnapshotsToKeep are optional, and any non-positive values are ignored. func NewSetSnapshotRefUpdate( name string, snapshotID int64, @@ -228,13 +262,13 @@ func NewSetSnapshotRefUpdate( func (u *SetSnapshotRefUpdate) Apply(builder *MetadataBuilder) error { opts := []setSnapshotRefOption{} - if u.MaxRefAgeMs != 0 { + if u.MaxRefAgeMs >= 0 { opts = append(opts, WithMaxRefAgeMs(u.MaxRefAgeMs)) } - if u.MaxSnapshotAgeMs != 0 { + if u.MaxSnapshotAgeMs >= 0 { opts = append(opts, WithMaxSnapshotAgeMs(u.MaxSnapshotAgeMs)) } - if u.MinSnapshotsToKeep != 0 { + if u.MinSnapshotsToKeep >= 0 { opts = append(opts, WithMinSnapshotsToKeep(u.MinSnapshotsToKeep)) } @@ -247,11 +281,13 @@ func (u *SetSnapshotRefUpdate) Apply(builder *MetadataBuilder) error { return err } +// SetLocationUpdate sets the location of the table metadata. type SetLocationUpdate struct { baseUpdate Location string `json:"location"` } +// NewSetLocationUpdate creates a new SetLocationUpdate with the given location. func NewSetLocationUpdate(loc string) *SetLocationUpdate { return &SetLocationUpdate{ baseUpdate: baseUpdate{ActionName: "set-location"}, @@ -264,11 +300,13 @@ func (u *SetLocationUpdate) Apply(builder *MetadataBuilder) error { return err } +// SetPropertyUpdate sets a number of properties in the table metadata. type SetPropertiesUpdate struct { baseUpdate Updates iceberg.Properties `json:"updates"` } +// NewSetPropertiesUpdate creates a new SetPropertiesUpdate with the given properties. func NewSetPropertiesUpdate(updates iceberg.Properties) *SetPropertiesUpdate { return &SetPropertiesUpdate{ baseUpdate: baseUpdate{ActionName: "set-properties"}, @@ -281,11 +319,15 @@ func (u *SetPropertiesUpdate) Apply(builder *MetadataBuilder) error { return err } +// RemovePropertiesUpdate removes a number of 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. type RemovePropertiesUpdate struct { baseUpdate Removals []string `json:"removals"` } +// NewRemovePropertiesUpdate creates a new RemovePropertiesUpdate with the given property names. func NewRemovePropertiesUpdate(removals []string) *RemovePropertiesUpdate { return &RemovePropertiesUpdate{ baseUpdate: baseUpdate{ActionName: "remove-properties"}, @@ -298,11 +340,13 @@ func (u *RemovePropertiesUpdate) Apply(builder *MetadataBuilder) error { return err } +// RemoveSnapshotsUpdate removes a number of snapshots from the table metadata. type RemoveSnapshotsUpdate struct { baseUpdate SnapshotIDs []int64 `json:"snapshot-ids"` } +// NewRemoveSnapshotsUpdate creates a new RemoveSnapshotsUpdate with the given snapshot IDs. func NewRemoveSnapshotsUpdate(ids []int64) *RemoveSnapshotsUpdate { return &RemoveSnapshotsUpdate{ baseUpdate: baseUpdate{ActionName: "remove-snapshots"}, @@ -314,11 +358,13 @@ func (u *RemoveSnapshotsUpdate) Apply(builder *MetadataBuilder) error { return fmt.Errorf("%w: remove-snapshots", iceberg.ErrNotImplemented) } +// RemoveSnapshotRefUpdate removes a snapshot reference from the table metadata. type RemoveSnapshotRefUpdate struct { baseUpdate RefName string `json:"ref-name"` } +// NewRemoveSnapshotRefUpdate creates a new RemoveSnapshotRefUpdate with the given reference name. func NewRemoveSnapshotRefUpdate(ref string) *RemoveSnapshotRefUpdate { return &RemoveSnapshotRefUpdate{ baseUpdate: baseUpdate{ActionName: "remove-snapshot-ref"}, From 48b3f2d0ff13b932308778320a601919bd333247 Mon Sep 17 00:00:00 2001 From: Jonas Weile Date: Wed, 11 Sep 2024 14:56:37 +0000 Subject: [PATCH 17/28] spelling --- table/requirements.go | 16 ++++++++++++++++ table/updates.go | 2 +- 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/table/requirements.go b/table/requirements.go index 77124c3..c2a71c8 100644 --- a/table/requirements.go +++ b/table/requirements.go @@ -23,14 +23,21 @@ import ( "github.com/google/uuid" ) +// 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. type Requirement interface { + // Validate checks that the current table metadata satisfies the requirement. Validate(Metadata) error } +// baseRequirement is a common struct that all requirements embed. It is used to +// identify the type of the requirement. type baseRequirement struct { Type string `json:"type"` } +// AssertCreate validates that the table does not already exist. type AssertCreate struct { baseRequirement } @@ -49,6 +56,7 @@ func (a *AssertCreate) Validate(meta Metadata) error { return nil } +// AssertTableUuid validates that the table UUID matches the requirement's `UUID`. type AssertTableUuid struct { baseRequirement UUID uuid.UUID `json:"uuid"` @@ -73,6 +81,9 @@ func (a *AssertTableUuid) Validate(meta Metadata) error { return nil } +// AssertRefSnapshotID validates that the table branch or tag identified by the +// requirement's `Ref` must reference the requirement's `SnapshotID`. +// if `SnapshotID` is `nil`, the ref must not already exist. type AssertRefSnapshotID struct { baseRequirement Ref string `json:"ref"` @@ -108,6 +119,7 @@ func (a *AssertRefSnapshotID) Validate(meta Metadata) error { return nil } +// AssertTableType validates that the table's last assigned column ID matches the requirement's `LastAssignedFieldID`. type AssertLastAssignedFieldId struct { baseRequirement LastAssignedFieldID int `json:"last-assigned-field-id"` @@ -132,6 +144,7 @@ func (a *AssertLastAssignedFieldId) Validate(meta Metadata) error { return nil } +// AssertCurrentSchemaId validates that the table's current schema ID matches the requirement's `CurrentSchemaID`. type AssertCurrentSchemaId struct { baseRequirement CurrentSchemaID int `json:"current-schema-id"` @@ -156,6 +169,7 @@ func (a *AssertCurrentSchemaId) Validate(meta Metadata) error { return nil } +// AssertLastAssignedPartitionId validates that the table's last assigned partition ID matches the requirement's `LastAssignedPartitionID`. type AssertLastAssignedPartitionId struct { baseRequirement LastAssignedPartitionID int `json:"last-assigned-partition-id"` @@ -180,6 +194,7 @@ func (a *AssertLastAssignedPartitionId) Validate(meta Metadata) error { return nil } +// AssertDefaultSpecId validates that the table's default partition spec ID matches the requirement's `DefaultSpecID`. type AssertDefaultSpecId struct { baseRequirement DefaultSpecID int `json:"default-spec-id"` @@ -204,6 +219,7 @@ func (a *AssertDefaultSpecId) Validate(meta Metadata) error { return nil } +// AssertDefaultSortOrderId validates that the table's default sort order ID matches the requirement's `DefaultSortOrderID`. type AssertDefaultSortOrderId struct { baseRequirement DefaultSortOrderID int `json:"default-sort-order-id"` diff --git a/table/updates.go b/table/updates.go index 00a73e5..0907708 100644 --- a/table/updates.go +++ b/table/updates.go @@ -135,7 +135,7 @@ type AddPartitionSpecUpdate struct { // NewAddPartitionSpecUpdate creates a new AddPartitionSpecUpdate with the given partition spec. // 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 remoed. +// and all other previously added specs in the metadata builder are removed. func NewAddPartitionSpecUpdate(spec *iceberg.PartitionSpec, initial bool) *AddPartitionSpecUpdate { return &AddPartitionSpecUpdate{ baseUpdate: baseUpdate{ActionName: "add-spec"}, From 5b70ef0f5723fb5db48e465a6f3a0ab8ff76f3ea Mon Sep 17 00:00:00 2001 From: Jonas Weile Date: Thu, 12 Sep 2024 06:08:10 +0000 Subject: [PATCH 18/28] fix tests --- table/metadata.go | 18 +++++++++--------- table/metadata_internal_test.go | 4 ++-- table/refs.go | 6 +++--- table/refs_test.go | 2 +- 4 files changed, 15 insertions(+), 15 deletions(-) diff --git a/table/metadata.go b/table/metadata.go index 6daa333..6c7af0d 100644 --- a/table/metadata.go +++ b/table/metadata.go @@ -410,7 +410,7 @@ func WithMaxRefAgeMs(maxRefAgeMs int64) setSnapshotRefOption { if maxRefAgeMs <= 0 { return fmt.Errorf("%w: maxRefAgeMs %d, must be > 0", iceberg.ErrInvalidArgument, maxRefAgeMs) } - ref.MaxRefAgeMs = maxRefAgeMs + ref.MaxRefAgeMs = &maxRefAgeMs return nil } } @@ -420,7 +420,7 @@ func WithMaxSnapshotAgeMs(maxSnapshotAgeMs int64) setSnapshotRefOption { if maxSnapshotAgeMs <= 0 { return fmt.Errorf("%w: maxSnapshotAgeMs %d, must be > 0", iceberg.ErrInvalidArgument, maxSnapshotAgeMs) } - ref.MaxSnapshotAgeMs = maxSnapshotAgeMs + ref.MaxSnapshotAgeMs = &maxSnapshotAgeMs return nil } } @@ -430,7 +430,7 @@ func WithMinSnapshotsToKeep(minSnapshotsToKeep int) setSnapshotRefOption { if minSnapshotsToKeep <= 0 { return fmt.Errorf("%w: minSnapshotsToKeep %d, must be > 0", iceberg.ErrInvalidArgument, minSnapshotsToKeep) } - ref.MinSnapshotsToKeep = minSnapshotsToKeep + ref.MinSnapshotsToKeep = &minSnapshotsToKeep return nil } } @@ -453,14 +453,14 @@ func (b *MetadataBuilder) SetSnapshotRef( var maxRefAgeMs, maxSnapshotAgeMs int64 var minSnapshotsToKeep int - if ref.MaxRefAgeMs > 0 { - maxRefAgeMs = ref.MaxRefAgeMs + if ref.MaxRefAgeMs != nil { + maxRefAgeMs = *ref.MaxRefAgeMs } - if ref.MaxSnapshotAgeMs > 0 { - maxSnapshotAgeMs = ref.MaxSnapshotAgeMs + if ref.MaxSnapshotAgeMs != nil { + maxSnapshotAgeMs = *ref.MaxSnapshotAgeMs } - if ref.MinSnapshotsToKeep > 0 { - minSnapshotsToKeep = ref.MinSnapshotsToKeep + if ref.MinSnapshotsToKeep != nil { + minSnapshotsToKeep = *ref.MinSnapshotsToKeep } if existingRef, ok := b.refs[name]; ok && existingRef.Equals(ref) { diff --git a/table/metadata_internal_test.go b/table/metadata_internal_test.go index 50daced..a02ac7f 100644 --- a/table/metadata_internal_test.go +++ b/table/metadata_internal_test.go @@ -169,7 +169,7 @@ func TestMetadataV2Parsing(t *testing.T) { data := meta.(*metadataV2) assert.Equal(t, uuid.MustParse("9c12d441-03fe-4693-9a96-a0705ddf69c1"), data.UUID) assert.Equal(t, "s3://bucket/test/location", data.Location()) - assert.Equal(t, 34, data.LastSequenceNumber) + assert.Equal(t, int64(34), data.LastSequenceNumber) assert.Equal(t, int64(1602638573590), data.LastUpdatedMS) assert.Equal(t, 3, data.LastColumnId) assert.Equal(t, 0, data.SchemaList[0].ID) @@ -412,7 +412,7 @@ func TestV2RefCreation(t *testing.T) { "test": { SnapshotID: 3051729675574597004, SnapshotRefType: TagRef, - MaxRefAgeMs: maxRefAge, + MaxRefAgeMs: &maxRefAge, }, }, meta.SnapshotRefs) } diff --git a/table/refs.go b/table/refs.go index 40e7d12..f0eb697 100644 --- a/table/refs.go +++ b/table/refs.go @@ -41,9 +41,9 @@ var ( 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"` + MinSnapshotsToKeep *int `json:"min-snapshots-to-keep,omitempty"` + MaxSnapshotAgeMs *int64 `json:"max-snapshot-age-ms,omitempty"` + MaxRefAgeMs *int64 `json:"max-ref-age-ms,omitempty"` } func (s *SnapshotRef) Equals(rhs SnapshotRef) bool { diff --git a/table/refs_test.go b/table/refs_test.go index fddde57..d8b54e4 100644 --- a/table/refs_test.go +++ b/table/refs_test.go @@ -66,7 +66,7 @@ func TestSnapshotTagRef(t *testing.T) { assert.Equal(t, table.TagRef, snapRef.SnapshotRefType) assert.Equal(t, int64(3051729675574597004), snapRef.SnapshotID) - assert.Equal(t, 10, snapRef.MinSnapshotsToKeep) + assert.Equal(t, 10, *snapRef.MinSnapshotsToKeep) assert.Nil(t, snapRef.MaxRefAgeMs) assert.Nil(t, snapRef.MaxSnapshotAgeMs) } From f2e1e943f17264c8cfce2fbb5b0eb3f987f287fe Mon Sep 17 00:00:00 2001 From: Jonas Weile Date: Tue, 8 Oct 2024 10:26:07 +0000 Subject: [PATCH 19/28] omit empty fields in CreateTableRequest --- catalog/rest.go | 18 +++++++++--------- go.mod | 21 +++++++++++---------- go.sum | 32 +++++++++++--------------------- 3 files changed, 31 insertions(+), 40 deletions(-) diff --git a/catalog/rest.go b/catalog/rest.go index 746b6fd..952d127 100644 --- a/catalog/rest.go +++ b/catalog/rest.go @@ -130,13 +130,13 @@ func WithLocation(loc string) createTableOption { } } -func WithPartitionSpec(spec iceberg.PartitionSpec) createTableOption { +func WithPartitionSpec(spec *iceberg.PartitionSpec) createTableOption { return func(req *createTableRequest) { req.PartitionSpec = spec } } -func WithWriteOrder(order table.SortOrder) createTableOption { +func WithWriteOrder(order *table.SortOrder) createTableOption { return func(req *createTableRequest) { req.WriteOrder = order } @@ -155,13 +155,13 @@ func WithProperties(props iceberg.Properties) createTableOption { } type createTableRequest struct { - Name string `json:"name"` - Location string `json:"location"` - Schema *iceberg.Schema `json:"schema"` - PartitionSpec iceberg.PartitionSpec `json:"partition-spec"` - WriteOrder table.SortOrder `json:"write-order"` - StageCreate bool `json:"stage-create"` - Props iceberg.Properties `json:"properties"` + Name string `json:"name"` + Schema *iceberg.Schema `json:"schema"` + Location string `json:"location,omitempty"` + PartitionSpec *iceberg.PartitionSpec `json:"partition-spec,omitempty"` + WriteOrder *table.SortOrder `json:"write-order,omitempty"` + StageCreate bool `json:"stage-create,omitempty"` + Props iceberg.Properties `json:"properties,omitempty"` } type oauthTokenResponse struct { diff --git a/go.mod b/go.mod index cee4426..ac5b59b 100644 --- a/go.mod +++ b/go.mod @@ -54,14 +54,15 @@ require ( github.com/aws/aws-sdk-go-v2/service/sso v1.22.5 // indirect github.com/aws/aws-sdk-go-v2/service/ssooidc v1.26.5 // indirect github.com/aws/aws-sdk-go-v2/service/sts v1.30.5 // indirect - github.com/containerd/console v1.0.3 // indirect + github.com/containerd/console v1.0.4 // indirect github.com/davecgh/go-spew v1.1.1 // indirect - github.com/goccy/go-json v0.10.2 // indirect + github.com/goccy/go-json v0.10.3 // indirect github.com/golang/snappy v0.0.4 // indirect github.com/google/flatbuffers v24.3.25+incompatible // indirect github.com/gookit/color v1.5.4 // indirect github.com/json-iterator/go v1.1.12 // indirect github.com/klauspost/compress v1.17.9 // indirect + github.com/klauspost/cpuid/v2 v2.2.8 // indirect github.com/kr/pretty v0.3.1 // indirect github.com/lithammer/fuzzysearch v1.1.8 // indirect github.com/mattn/go-runewidth v0.0.15 // indirect @@ -69,17 +70,17 @@ require ( github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect - github.com/rivo/uniseg v0.4.4 // indirect + github.com/rivo/uniseg v0.4.7 // indirect github.com/rogpeppe/go-internal v1.12.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/xo/terminfo v0.0.0-20220910002029-abceb7e1c41e // indirect - golang.org/x/mod v0.19.0 // indirect - golang.org/x/net v0.27.0 // indirect - golang.org/x/sync v0.7.0 // indirect - golang.org/x/sys v0.22.0 // indirect - golang.org/x/term v0.22.0 // indirect - golang.org/x/text v0.16.0 // indirect - golang.org/x/tools v0.23.0 // indirect + golang.org/x/mod v0.20.0 // indirect + golang.org/x/net v0.28.0 // indirect + golang.org/x/sync v0.8.0 // indirect + golang.org/x/sys v0.24.0 // indirect + golang.org/x/term v0.23.0 // indirect + golang.org/x/text v0.17.0 // indirect + golang.org/x/tools v0.24.0 // indirect golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 // indirect gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/go.sum b/go.sum index c205a48..56e13a1 100644 --- a/go.sum +++ b/go.sum @@ -56,16 +56,15 @@ github.com/aws/aws-sdk-go-v2/service/sts v1.30.5 h1:OMsEmCyz2i89XwRwPouAJvhj81wI github.com/aws/aws-sdk-go-v2/service/sts v1.30.5/go.mod h1:vmSqFK+BVIwVpDAGZB3CoCXHzurt4qBE8lf+I/kRTh0= github.com/aws/smithy-go v1.20.4 h1:2HK1zBdPgRbjFOHlfeQZfpC4r72MOb9bZkiFwggKO+4= github.com/aws/smithy-go v1.20.4/go.mod h1:irrKGvNn1InZwb2d7fkIRNucdfwR8R+Ts3wxYa/cJHg= -github.com/containerd/console v1.0.3 h1:lIr7SlA5PxZyMV30bDW0MGbiOPXwc63yRuCP0ARubLw= github.com/containerd/console v1.0.3/go.mod h1:7LqA/THxQ86k76b8c/EMSiaJ3h1eZkMkXar0TQ1gf3U= +github.com/containerd/console v1.0.4 h1:F2g4+oChYvBTsASRTz8NP6iIAi97J3TtSAsLbIFn4ro= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/docopt/docopt-go v0.0.0-20180111231733-ee0de3bc6815 h1:bWDMxwH3px2JBh6AyO7hdCn/PkvCZXii8TGj7sbtEbQ= github.com/docopt/docopt-go v0.0.0-20180111231733-ee0de3bc6815/go.mod h1:WwZ+bS3ebgob9U8Nd0kOddGdZWjyMGR8Wziv+TBNwSE= -github.com/goccy/go-json v0.10.2 h1:CrxCmQqYDkv1z7lO7Wbh2HN93uovUHgrECaO5ZrCXAU= -github.com/goccy/go-json v0.10.2/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= +github.com/goccy/go-json v0.10.3 h1:KZ5WoDbxAIgm2HNbYckL0se1fHD6rz5j4ywS6ebzDqA= github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/google/flatbuffers v24.3.25+incompatible h1:CX395cjN9Kke9mmalRoL3d81AtFUxJM+yDthflgJGkI= @@ -86,8 +85,7 @@ github.com/klauspost/compress v1.17.9/go.mod h1:Di0epgTjJY877eYKx5yC51cX2A2Vl2ib github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= github.com/klauspost/cpuid/v2 v2.0.10/go.mod h1:g2LTdtYhdyuGPqyWyv7qRAmj1WBqxuObKfj5c0PQa7c= github.com/klauspost/cpuid/v2 v2.0.12/go.mod h1:g2LTdtYhdyuGPqyWyv7qRAmj1WBqxuObKfj5c0PQa7c= -github.com/klauspost/cpuid/v2 v2.2.7 h1:ZWSB3igEs+d0qvnxR/ZBzXVmxkgt8DdzP6m9pfuVLDM= -github.com/klauspost/cpuid/v2 v2.2.7/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws= +github.com/klauspost/cpuid/v2 v2.2.8 h1:+StwCXwm9PdpiEkPyzBXIy+M9KUb4ODm0Zarf1kS5BM= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= @@ -121,8 +119,7 @@ github.com/pterm/pterm v0.12.40/go.mod h1:ffwPLwlbXxP+rxT0GsgDTzS3y3rmpAO1NMjUkG github.com/pterm/pterm v0.12.79 h1:lH3yrYMhdpeqX9y5Ep1u7DejyHy7NSQg9qrBjF9dFT4= github.com/pterm/pterm v0.12.79/go.mod h1:1v/gzOF1N0FsjbgTHZ1wVycRkKiatFvJSJC4IGaQAAo= github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= -github.com/rivo/uniseg v0.4.4 h1:8TfxU8dW6PdqD27gjM8MVNuicgxIjxpm4K7x4jp8sis= -github.com/rivo/uniseg v0.4.4/go.mod h1:FN3SvrM+Zdj16jyLfmOkMNblXMcoc8DfTHruCPUcx88= +github.com/rivo/uniseg v0.4.7 h1:WUdvkW8uEhrYfLC4ZzdpI2ztxP1I582+49Oc5Mq64VQ= github.com/rogpeppe/go-internal v1.9.0/go.mod h1:WtVeX8xhTBvf0smdhujwtBcq4Qrzq/fJaraNFVN+nFs= github.com/rogpeppe/go-internal v1.12.0 h1:exVL4IDcn6na9z1rAb56Vxr+CgyK3nn3O+epU5NdKM8= github.com/rogpeppe/go-internal v1.12.0/go.mod h1:E+RYuTGaKKdloAfM02xzb0FW3Paa99yedzYV+kq4uf4= @@ -153,19 +150,16 @@ golang.org/x/exp v0.0.0-20240222234643-814bf88cf225 h1:LfspQV/FYTatPTr/3HzIcmiUF golang.org/x/exp v0.0.0-20240222234643-814bf88cf225/go.mod h1:CxmFvTBINI24O/j8iY7H1xHzx2i4OsyguNBmN/uPtqc= golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= -golang.org/x/mod v0.19.0 h1:fEdghXQSo20giMthA7cd28ZC+jts4amQ3YMXiP5oMQ8= -golang.org/x/mod v0.19.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= +golang.org/x/mod v0.20.0 h1:utOm6MM3R3dnawAiJgn0y+xvuYRsm1RKM/4giyfDgV0= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= -golang.org/x/net v0.27.0 h1:5K3Njcw06/l2y9vpGCSdcxWOYHOUk3dVNGDXN+FvAys= -golang.org/x/net v0.27.0/go.mod h1:dDi0PyhWNoiUOrAS8uXv/vnScO4wnHQO4mj9fn/RytE= +golang.org/x/net v0.28.0 h1:a9JDOJc5GMUJ0+UDqmLT86WiEy7iWyIhz8gz8E4e5hE= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.7.0 h1:YsImfSBoP9QPYL0xyKJPq0gcaJdG3rInoqxTWbfQu9M= -golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -176,28 +170,24 @@ golang.org/x/sys v0.0.0-20220319134239-a9b59b0215f8/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.22.0 h1:RI27ohtqKCnwULzJLqkv897zojh5/DwS/ENaMzUOaWI= -golang.org/x/sys v0.22.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210220032956-6a3ed077a48d/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210615171337-6886f2dfbf5b/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= -golang.org/x/term v0.22.0 h1:BbsgPEJULsl2fV/AT3v15Mjva5yXKQDyKf+TbDz7QJk= -golang.org/x/term v0.22.0/go.mod h1:F3qCibpT5AMpCRfhfT53vVJwhLtIVHhB9XDjfFvnMI4= +golang.org/x/term v0.23.0 h1:F6D4vR+EHoL9/sWAWgAR1H2DcHr4PareCbAaCo1RpuU= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= -golang.org/x/text v0.16.0 h1:a94ExnEXNtEwYLGJSIUxnWoxoRz/ZcCsV63ROupILh4= -golang.org/x/text v0.16.0/go.mod h1:GhwF1Be+LQoKShO3cGOHzqOgRrGaYc9AvblQOmPVHnI= +golang.org/x/text v0.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= -golang.org/x/tools v0.23.0 h1:SGsXPZ+2l4JsgaCKkx+FQ9YZ5XEtA1GZYuoDjenLjvg= -golang.org/x/tools v0.23.0/go.mod h1:pnu6ufv6vQkll6szChhK3C3L/ruaIv5eBeztNG8wtsI= +golang.org/x/tools v0.24.0 h1:J1shsA93PJUEVaUSaay7UXAyE8aimq3GW0pjlolpa24= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 h1:+cNy6SZtPcJQH3LJVLOSmiC7MMxXNOb3PU/VUEz+EhU= golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028/go.mod h1:NDW/Ps6MPRej6fsCIbMTohpP40sJ/P/vI1MoTEGwX90= From 21719337dd1218ff05f2145d2b308aca8e4c41d2 Mon Sep 17 00:00:00 2001 From: Jonas Weile Date: Sat, 12 Oct 2024 19:31:41 +0000 Subject: [PATCH 20/28] lowercase --- table/updates.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/table/updates.go b/table/updates.go index 0907708..640935a 100644 --- a/table/updates.go +++ b/table/updates.go @@ -35,7 +35,7 @@ type Update interface { // baseUpdate contains the common fields for all updates. It is used to identify the type // of the update. type baseUpdate struct { - ActionName string `json:"Action"` + ActionName string `json:"action"` } func (u *baseUpdate) Action() string { From 182cc9d2475c57e5398aee44f0af8e1bf2af48e1 Mon Sep 17 00:00:00 2001 From: Jonas Weile Date: Sun, 27 Oct 2024 21:23:05 +0000 Subject: [PATCH 21/28] implement feedback --- catalog/rest.go | 35 ++++---- go.mod | 2 +- go.sum | 12 +++ partitions.go | 8 +- schema.go | 8 +- table/metadata.go | 112 +++++++++++++++---------- table/requirements.go | 105 +++++++++++++----------- table/updates.go | 186 ++++++++++++++++++++---------------------- 8 files changed, 252 insertions(+), 216 deletions(-) diff --git a/catalog/rest.go b/catalog/rest.go index 952d127..87d50a7 100644 --- a/catalog/rest.go +++ b/catalog/rest.go @@ -84,7 +84,7 @@ func (e errorResponse) Error() string { return e.Type + ": " + e.Message } -type Identifier struct { +type identifier struct { Namespace []string `json:"namespace"` Name string `json:"name"` } @@ -640,7 +640,7 @@ func (r *RestCatalog) ListTables(ctx context.Context, namespace table.Identifier path := []string{"namespaces", ns, "tables"} type resp struct { - Identifiers []Identifier `json:"identifiers"` + Identifiers []identifier `json:"identifiers"` } rsp, err := doGet[resp](ctx, r.baseURI, path, r.cl, map[int]error{http.StatusNotFound: ErrNoSuchNamespace}) if err != nil { @@ -685,9 +685,7 @@ func (r *RestCatalog) CreateTable(ctx context.Context, identifier table.Identifi config := maps.Clone(r.props) maps.Copy(config, ret.Metadata.Properties()) - for k, v := range ret.Config { - config[k] = v - } + maps.Copy(config, ret.Config) return r.tableFromResponse(identifier, ret.Metadata, ret.MetadataLoc, config) } @@ -711,10 +709,7 @@ func (r *RestCatalog) RegisterTable(ctx context.Context, identifier table.Identi config := maps.Clone(r.props) maps.Copy(config, ret.Metadata.Properties()) - for k, v := range ret.Config { - config[k] = v - } - + maps.Copy(config, ret.Config) return r.tableFromResponse(identifier, ret.Metadata, ret.MetadataLoc, config) } @@ -740,23 +735,23 @@ func (r *RestCatalog) LoadTable(ctx context.Context, identifier table.Identifier return r.tableFromResponse(identifier, ret.Metadata, ret.MetadataLoc, config) } -func (r *RestCatalog) UpdateTable(ctx context.Context, identifier table.Identifier, requirements []table.Requirement, updates []table.Update) (*table.Table, error) { - ns, tbl, err := splitIdentForPath(identifier) +func (r *RestCatalog) UpdateTable(ctx context.Context, ident table.Identifier, requirements []table.Requirement, updates []table.Update) (*table.Table, error) { + ns, tbl, err := splitIdentForPath(ident) if err != nil { return nil, err } - ident := Identifier{ - Namespace: NamespaceFromIdent(identifier), + restIdentifier := identifier{ + Namespace: NamespaceFromIdent(ident), Name: tbl, } type payload struct { - Identifier Identifier `json:"identifier"` + Identifier identifier `json:"identifier"` Requirements []table.Requirement `json:"requirements"` Updates []table.Update `json:"updates"` } ret, err := doPost[payload, commitTableResponse](ctx, r.baseURI, []string{"namespaces", ns, "tables", tbl}, - payload{Identifier: ident, Requirements: requirements, Updates: updates}, r.cl, + payload{Identifier: restIdentifier, Requirements: requirements, Updates: updates}, r.cl, map[int]error{http.StatusNotFound: ErrNoSuchTable, http.StatusConflict: ErrCommitFailed}) if err != nil { return nil, err @@ -765,7 +760,7 @@ func (r *RestCatalog) UpdateTable(ctx context.Context, identifier table.Identifi config := maps.Clone(r.props) maps.Copy(config, ret.Metadata.Properties()) - return r.tableFromResponse(identifier, ret.Metadata, ret.MetadataLoc, config) + return r.tableFromResponse(ident, ret.Metadata, ret.MetadataLoc, config) } func (r *RestCatalog) DropTable(ctx context.Context, identifier table.Identifier, purge bool) error { @@ -789,14 +784,14 @@ func (r *RestCatalog) DropTable(ctx context.Context, identifier table.Identifier func (r *RestCatalog) RenameTable(ctx context.Context, from, to table.Identifier) (*table.Table, error) { type payload struct { - From Identifier `json:"from"` - To Identifier `json:"to"` + From identifier `json:"from"` + To identifier `json:"to"` } - f := Identifier{ + f := identifier{ Namespace: NamespaceFromIdent(from), Name: TableNameFromIdent(from), } - t := Identifier{ + t := identifier{ Namespace: NamespaceFromIdent(to), Name: TableNameFromIdent(to), } diff --git a/go.mod b/go.mod index ac5b59b..3704529 100644 --- a/go.mod +++ b/go.mod @@ -17,7 +17,7 @@ module github.com/apache/iceberg-go -go 1.21 +go 1.23 require ( github.com/apache/arrow/go/v16 v16.1.0 diff --git a/go.sum b/go.sum index 56e13a1..82d745d 100644 --- a/go.sum +++ b/go.sum @@ -58,6 +58,7 @@ github.com/aws/smithy-go v1.20.4 h1:2HK1zBdPgRbjFOHlfeQZfpC4r72MOb9bZkiFwggKO+4= github.com/aws/smithy-go v1.20.4/go.mod h1:irrKGvNn1InZwb2d7fkIRNucdfwR8R+Ts3wxYa/cJHg= github.com/containerd/console v1.0.3/go.mod h1:7LqA/THxQ86k76b8c/EMSiaJ3h1eZkMkXar0TQ1gf3U= github.com/containerd/console v1.0.4 h1:F2g4+oChYvBTsASRTz8NP6iIAi97J3TtSAsLbIFn4ro= +github.com/containerd/console v1.0.4/go.mod h1:YynlIjWYF8myEu6sdkwKIvGQq+cOckRm6So2avqoYAk= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= @@ -65,6 +66,7 @@ github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSs github.com/docopt/docopt-go v0.0.0-20180111231733-ee0de3bc6815 h1:bWDMxwH3px2JBh6AyO7hdCn/PkvCZXii8TGj7sbtEbQ= github.com/docopt/docopt-go v0.0.0-20180111231733-ee0de3bc6815/go.mod h1:WwZ+bS3ebgob9U8Nd0kOddGdZWjyMGR8Wziv+TBNwSE= github.com/goccy/go-json v0.10.3 h1:KZ5WoDbxAIgm2HNbYckL0se1fHD6rz5j4ywS6ebzDqA= +github.com/goccy/go-json v0.10.3/go.mod h1:oq7eo15ShAhp70Anwd5lgX2pLfOS3QCiwU/PULtXL6M= github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/google/flatbuffers v24.3.25+incompatible h1:CX395cjN9Kke9mmalRoL3d81AtFUxJM+yDthflgJGkI= @@ -86,6 +88,7 @@ github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa02 github.com/klauspost/cpuid/v2 v2.0.10/go.mod h1:g2LTdtYhdyuGPqyWyv7qRAmj1WBqxuObKfj5c0PQa7c= github.com/klauspost/cpuid/v2 v2.0.12/go.mod h1:g2LTdtYhdyuGPqyWyv7qRAmj1WBqxuObKfj5c0PQa7c= github.com/klauspost/cpuid/v2 v2.2.8 h1:+StwCXwm9PdpiEkPyzBXIy+M9KUb4ODm0Zarf1kS5BM= +github.com/klauspost/cpuid/v2 v2.2.8/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= @@ -120,6 +123,7 @@ github.com/pterm/pterm v0.12.79 h1:lH3yrYMhdpeqX9y5Ep1u7DejyHy7NSQg9qrBjF9dFT4= github.com/pterm/pterm v0.12.79/go.mod h1:1v/gzOF1N0FsjbgTHZ1wVycRkKiatFvJSJC4IGaQAAo= github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= github.com/rivo/uniseg v0.4.7 h1:WUdvkW8uEhrYfLC4ZzdpI2ztxP1I582+49Oc5Mq64VQ= +github.com/rivo/uniseg v0.4.7/go.mod h1:FN3SvrM+Zdj16jyLfmOkMNblXMcoc8DfTHruCPUcx88= github.com/rogpeppe/go-internal v1.9.0/go.mod h1:WtVeX8xhTBvf0smdhujwtBcq4Qrzq/fJaraNFVN+nFs= github.com/rogpeppe/go-internal v1.12.0 h1:exVL4IDcn6na9z1rAb56Vxr+CgyK3nn3O+epU5NdKM8= github.com/rogpeppe/go-internal v1.12.0/go.mod h1:E+RYuTGaKKdloAfM02xzb0FW3Paa99yedzYV+kq4uf4= @@ -151,15 +155,18 @@ golang.org/x/exp v0.0.0-20240222234643-814bf88cf225/go.mod h1:CxmFvTBINI24O/j8iY golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.20.0 h1:utOm6MM3R3dnawAiJgn0y+xvuYRsm1RKM/4giyfDgV0= +golang.org/x/mod v0.20.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.28.0 h1:a9JDOJc5GMUJ0+UDqmLT86WiEy7iWyIhz8gz8E4e5hE= +golang.org/x/net v0.28.0/go.mod h1:yqtgsTWOOnlGLG9GFRrK3++bGOUEkNBoHZc8MEDWPNg= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= +golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -169,25 +176,30 @@ golang.org/x/sys v0.0.0-20211013075003-97ac67df715c/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220319134239-a9b59b0215f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.1.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= +golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210220032956-6a3ed077a48d/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210615171337-6886f2dfbf5b/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= golang.org/x/term v0.23.0 h1:F6D4vR+EHoL9/sWAWgAR1H2DcHr4PareCbAaCo1RpuU= +golang.org/x/term v0.23.0/go.mod h1:DgV24QBUrK6jhZXl+20l6UWznPlwAHm1Q1mGHtydmSk= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= golang.org/x/text v0.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= +golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= golang.org/x/tools v0.24.0 h1:J1shsA93PJUEVaUSaay7UXAyE8aimq3GW0pjlolpa24= +golang.org/x/tools v0.24.0/go.mod h1:YhNqVBIfWHdzvTLs0d8LCuMhkKUgSUKldakyV7W/WDQ= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 h1:+cNy6SZtPcJQH3LJVLOSmiC7MMxXNOb3PU/VUEz+EhU= golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028/go.mod h1:NDW/Ps6MPRej6fsCIbMTohpP40sJ/P/vI1MoTEGwX90= diff --git a/partitions.go b/partitions.go index c5c4b5a..9416d70 100644 --- a/partitions.go +++ b/partitions.go @@ -20,9 +20,9 @@ package iceberg import ( "encoding/json" "fmt" + "iter" + "slices" "strings" - - "golang.org/x/exp/slices" ) const ( @@ -118,8 +118,8 @@ func (ps PartitionSpec) Equals(other PartitionSpec) bool { } // Fields returns a clone of the partition fields in this spec. -func (ps *PartitionSpec) Fields() []PartitionField { - return slices.Clone(ps.fields) +func (ps *PartitionSpec) Fields() iter.Seq[PartitionField] { + return slices.Values(ps.fields) } func (ps PartitionSpec) MarshalJSON() ([]byte, error) { diff --git a/schema.go b/schema.go index d6ad24d..a204b54 100644 --- a/schema.go +++ b/schema.go @@ -297,12 +297,12 @@ func (s *Schema) accessorForField(id int) (accessor, bool) { // Equals compares the fields and identifierIDs, but does not compare // the schema ID itself. func (s *Schema) Equals(other *Schema) bool { - if s == other { - return true + if other == nil { + return false } - if s == nil || other == nil { - return false + if s == other { + return true } if len(s.fields) != len(other.fields) { diff --git a/table/metadata.go b/table/metadata.go index 6c7af0d..b506575 100644 --- a/table/metadata.go +++ b/table/metadata.go @@ -22,6 +22,7 @@ import ( "errors" "fmt" "io" + "iter" "maps" "slices" "time" @@ -32,8 +33,13 @@ import ( ) const ( - PARTITION_FIELD_ID_START = 1000 - SUPPORTED_TABLE_FORMAT_VERSION = 2 + partitionFieldStartID = 1000 + supportedTableFormatVersion = 2 + + addPartionSpecAction = "add-partition-spec" + addSchemaAction = "add-schema" + addSnapshotAction = "add-snapshot" + addSortOrderAction = "add-sort-order" ) // Metadata for an iceberg table as specified in the Iceberg spec @@ -89,9 +95,9 @@ type Metadata interface { // Ref returns the snapshot ref for the main branch. Ref() SnapshotRef // Refs returns a map of snapshot refs by name. - Refs() map[string]SnapshotRef + Refs() iter.Seq2[string, SnapshotRef] // SnapshotLogs returns the list of snapshot logs for the table. - SnapshotLogs() []SnapshotLogEntry + 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 @@ -106,7 +112,7 @@ type Metadata interface { // is used to control the number of commit retries. Properties() iceberg.Properties // PreviousFiles returns the list of metadata log entries for the table. - PreviousFiles() []MetadataLogEntry + PreviousFiles() iter.Seq[MetadataLogEntry] Equals(Metadata) bool } @@ -172,9 +178,21 @@ func MetadataBuilderFromBase(metadata Metadata) (*MetadataBuilder, error) { b.currentSnapshotID = &metadata.CurrentSnapshot().SnapshotID b.sortOrderList = metadata.SortOrders() b.defaultSortOrderID = metadata.DefaultSortOrder() - b.refs = metadata.Refs() - b.snapshotLog = metadata.SnapshotLogs() - b.metadataLog = metadata.PreviousFiles() + + b.refs = make(map[string]SnapshotRef) + for name, ref := range metadata.Refs() { + b.refs[name] = ref + } + + b.snapshotLog = make([]SnapshotLogEntry, 0) + for log := range metadata.SnapshotLogs() { + b.snapshotLog = append(b.snapshotLog, log) + } + + b.metadataLog = make([]MetadataLogEntry, 0) + for entry := range metadata.PreviousFiles() { + b.metadataLog = append(b.metadataLog, entry) + } return b, nil } @@ -205,15 +223,12 @@ func (b *MetadataBuilder) AddPartitionSpec(spec *iceberg.PartitionSpec, initial } } - var maxFieldID int - if len(spec.Fields()) > 0 { - maxField := slices.MaxFunc(spec.Fields(), func(a, b iceberg.PartitionField) int { - return a.FieldID - b.FieldID - }) - maxFieldID = maxField.FieldID + maxFieldID := 0 + for f := range spec.Fields() { + maxFieldID = max(maxFieldID, f.FieldID) } - prev := PARTITION_FIELD_ID_START - 1 + prev := partitionFieldStartID - 1 if b.lastPartitionID != nil { prev = *b.lastPartitionID } @@ -242,8 +257,6 @@ func (b *MetadataBuilder) AddSnapshot(snapshot *Snapshot) (*MetadataBuilder, err return nil, errors.New("can't add snapshot with no added schemas") } else if len(b.specs) == 0 { return nil, errors.New("can't add snapshot with no added partition specs") - } else if len(b.sortOrderList) == 0 { - return nil, errors.New("can't add snapshot with no added sort orders") } else if s, _ := b.SnapshotByID(snapshot.SnapshotID); s != nil { return nil, fmt.Errorf("can't add snapshot with id %d, already exists", snapshot.SnapshotID) } else if b.formatVersion == 2 && @@ -262,6 +275,12 @@ func (b *MetadataBuilder) AddSnapshot(snapshot *Snapshot) (*MetadataBuilder, err } func (b *MetadataBuilder) AddSortOrder(sortOrder *SortOrder, initial bool) (*MetadataBuilder, error) { + for _, s := range b.sortOrderList { + if s.OrderID == sortOrder.OrderID && !initial { + return nil, fmt.Errorf("sort order with id %d already exists", sortOrder.OrderID) + } + } + var sortOrders []SortOrder if initial { sortOrders = []SortOrder{*sortOrder} @@ -293,8 +312,8 @@ func (b *MetadataBuilder) SetCurrentSchemaID(currentSchemaID int) (*MetadataBuil currentSchemaID = maxBy(b.schemaList, func(s *iceberg.Schema) int { return s.ID }) - if !containsBy(b.updates, func(u Update) bool { - return u.Action() == "add-schema" && u.(*AddSchemaUpdate).Schema.ID == currentSchemaID + if !slices.ContainsFunc(b.updates, func(u Update) bool { + return u.Action() == addSchemaAction && u.(*addSchemaUpdate).Schema.ID == currentSchemaID }) { return nil, errors.New("can't set current schema to last added schema, no schema has been added") } @@ -319,8 +338,8 @@ func (b *MetadataBuilder) SetDefaultSortOrderID(defaultSortOrderID int) (*Metada defaultSortOrderID = maxBy(b.sortOrderList, func(s SortOrder) int { return s.OrderID }) - if !containsBy(b.updates, func(u Update) bool { - return u.Action() == "add-sort-order" && u.(*AddSortOrderUpdate).SortOrder.OrderID == defaultSortOrderID + if !slices.ContainsFunc(b.updates, func(u Update) bool { + return u.Action() == addSortOrderAction && u.(*addSortOrderUpdate).SortOrder.OrderID == defaultSortOrderID }) { return nil, fmt.Errorf("can't set default sort order to last added with no added sort orders") } @@ -344,8 +363,8 @@ func (b *MetadataBuilder) SetDefaultSpecID(defaultSpecID int) (*MetadataBuilder, defaultSpecID = maxBy(b.specs, func(s iceberg.PartitionSpec) int { return s.ID() }) - if !containsBy(b.updates, func(u Update) bool { - return u.Action() == "add-partition-spec" && u.(*AddPartitionSpecUpdate).Spec.ID() == defaultSpecID + if !slices.ContainsFunc(b.updates, func(u Update) bool { + return u.Action() == addPartionSpecAction && u.(*addPartitionSpecUpdate).Spec.ID() == defaultSpecID }) { return nil, fmt.Errorf("can't set default spec to last added with no added partition specs") } @@ -370,7 +389,7 @@ func (b *MetadataBuilder) SetFormatVersion(formatVersion int) (*MetadataBuilder, b.formatVersion, formatVersion) } - if formatVersion > SUPPORTED_TABLE_FORMAT_VERSION { + if formatVersion > supportedTableFormatVersion { return nil, fmt.Errorf("unsupported format version %d", formatVersion) } @@ -482,8 +501,8 @@ func (b *MetadataBuilder) SetSnapshotRef( b.lastUpdatedMS = time.Now().Local().UnixMilli() } - if containsBy(b.updates, func(u Update) bool { - return u.Action() == "add-snapshot" && u.(*AddSnapshotUpdate).Snapshot.SnapshotID == snapshotID + if slices.ContainsFunc(b.updates, func(u Update) bool { + return u.Action() == addSnapshotAction && u.(*addSnapshotUpdate).Snapshot.SnapshotID == snapshotID }) { b.lastUpdatedMS = snapshot.TimestampMs } @@ -573,38 +592,37 @@ func (b *MetadataBuilder) Build() (Metadata, error) { if err != nil { return nil, fmt.Errorf("can't build metadata, missing schema for schema ID %d: %w", b.currentSchemaID, err) } + partition, err := b.GetSpecByID(b.defaultSpecID) if err != nil { return nil, fmt.Errorf("can't build metadata, missing partition spec for spec ID %d: %w", b.defaultSpecID, err) } + + partitionFields := make([]iceberg.PartitionField, 0) + for f := range partition.Fields() { + partitionFields = append(partitionFields, f) + } + return &metadataV1{ Schema: schema, - Partition: partition.Fields(), + Partition: partitionFields, commonMetadata: *common, }, nil + case 2: return &metadataV2{ LastSequenceNumber: *b.lastSequenceNumber, commonMetadata: *common, }, nil + default: panic("unreachable: invalid format version") } } -// containsBy returns true if found(e) is true for any e in elems. -func containsBy[S []E, E any](elems S, found func(e E) bool) bool { - for _, e := range elems { - if found(e) { - return true - } - } - return false -} - // maxBy returns the maximum value of extract(e) for all e in elems. // If elems is empty, returns 0. -func maxBy[S []E, E any](elems S, extract func(e E) int) int { +func maxBy[S ~[]E, E any](elems S, extract func(e E) int) int { m := 0 for _, e := range elems { m = max(m, extract(e)) @@ -684,13 +702,19 @@ type commonMetadata struct { SnapshotRefs map[string]SnapshotRef `json:"refs"` } -func (c *commonMetadata) Ref() SnapshotRef { return c.SnapshotRefs[MainBranch] } -func (c *commonMetadata) Refs() map[string]SnapshotRef { return maps.Clone(c.SnapshotRefs) } -func (c *commonMetadata) SnapshotLogs() []SnapshotLogEntry { return slices.Clone(c.SnapshotLog) } -func (c *commonMetadata) PreviousFiles() []MetadataLogEntry { return slices.Clone(c.MetadataLog) } +func (c *commonMetadata) Ref() SnapshotRef { return c.SnapshotRefs[MainBranch] } +func (c *commonMetadata) Refs() iter.Seq2[string, SnapshotRef] { return maps.All(c.SnapshotRefs) } +func (c *commonMetadata) SnapshotLogs() iter.Seq[SnapshotLogEntry] { + return slices.Values(c.SnapshotLog) +} + +func (c *commonMetadata) PreviousFiles() iter.Seq[MetadataLogEntry] { + return slices.Values(c.MetadataLog) +} + func (c *commonMetadata) Equals(other *commonMetadata) bool { - if c == nil || other == nil { - return c == other + if other == nil { + return false } if c == other { diff --git a/table/requirements.go b/table/requirements.go index c2a71c8..0f3110d 100644 --- a/table/requirements.go +++ b/table/requirements.go @@ -37,18 +37,18 @@ type baseRequirement struct { Type string `json:"type"` } -// AssertCreate validates that the table does not already exist. -type AssertCreate struct { +type assertCreate struct { baseRequirement } -func NewAssertCreate() *AssertCreate { - return &AssertCreate{ +// AssertCreate creates a requirement that the table does not already exist. +func AssertCreate() Requirement { + return &assertCreate{ baseRequirement: baseRequirement{Type: "assert-create"}, } } -func (a *AssertCreate) Validate(meta Metadata) error { +func (a *assertCreate) Validate(meta Metadata) error { if meta != nil { return fmt.Errorf("Table already exists") } @@ -56,20 +56,20 @@ func (a *AssertCreate) Validate(meta Metadata) error { return nil } -// AssertTableUuid validates that the table UUID matches the requirement's `UUID`. -type AssertTableUuid struct { +type assertTableUuid struct { baseRequirement UUID uuid.UUID `json:"uuid"` } -func NewAssertTableUUID(uuid uuid.UUID) *AssertTableUuid { - return &AssertTableUuid{ +// AssertTableUUID creates a requirement that the table UUID matches the given UUID. +func AssertTableUUID(uuid uuid.UUID) Requirement { + return &assertTableUuid{ baseRequirement: baseRequirement{Type: "assert-table-uuid"}, UUID: uuid, } } -func (a *AssertTableUuid) Validate(meta Metadata) error { +func (a *assertTableUuid) Validate(meta Metadata) error { if meta == nil { return fmt.Errorf("requirement failed: current table metadata does not exist") } @@ -81,58 +81,65 @@ func (a *AssertTableUuid) Validate(meta Metadata) error { return nil } -// AssertRefSnapshotID validates that the table branch or tag identified by the -// requirement's `Ref` must reference the requirement's `SnapshotID`. -// if `SnapshotID` is `nil`, the ref must not already exist. -type AssertRefSnapshotID struct { +type assertRefSnapshotID struct { baseRequirement Ref string `json:"ref"` SnapshotID *int64 `json:"snapshot-id"` } -func NewAssertRefSnapshotID(ref string, id *int64) *AssertRefSnapshotID { - return &AssertRefSnapshotID{ +// 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 AssertRefSnapshotID(ref string, id *int64) Requirement { + return &assertRefSnapshotID{ baseRequirement: baseRequirement{Type: "assert-ref-snapshot-id"}, Ref: ref, SnapshotID: id, } } -func (a *AssertRefSnapshotID) Validate(meta Metadata) error { +func (a *assertRefSnapshotID) Validate(meta Metadata) error { if meta == nil { return fmt.Errorf("requirement failed: current table metadata does not exist") } - ref, ok := meta.Refs()[a.Ref] - if !ok { + var r *SnapshotRef + for name, ref := range meta.Refs() { + if name == a.Ref { + r = &ref + break + } + } + if r == nil { return fmt.Errorf("requirement failed: branch or tag %s is missing, expected %d", a.Ref, a.SnapshotID) } if a.SnapshotID == nil { - return fmt.Errorf("requirement failed: %s %s was created concurrently", ref.SnapshotRefType, a.Ref) + return fmt.Errorf("requirement failed: %s %s was created concurrently", r.SnapshotRefType, a.Ref) } - if ref.SnapshotID != *a.SnapshotID { - return fmt.Errorf("requirement failed: %s %s has changed: expected id %d, found %d", ref.SnapshotRefType, a.Ref, a.SnapshotID, ref.SnapshotID) + if r.SnapshotID != *a.SnapshotID { + return fmt.Errorf("requirement failed: %s %s has changed: expected id %d, found %d", r.SnapshotRefType, a.Ref, a.SnapshotID, r.SnapshotID) } return nil } -// AssertTableType validates that the table's last assigned column ID matches the requirement's `LastAssignedFieldID`. -type AssertLastAssignedFieldId struct { +type assertLastAssignedFieldId struct { baseRequirement LastAssignedFieldID int `json:"last-assigned-field-id"` } -func NewAssertLastAssignedFieldID(id int) *AssertLastAssignedFieldId { - return &AssertLastAssignedFieldId{ +// AssertLastAssignedFieldID validates that the table's last assigned column ID +// matches the given id. +func AssertLastAssignedFieldID(id int) Requirement { + return &assertLastAssignedFieldId{ baseRequirement: baseRequirement{Type: "assert-last-assigned-field-id"}, LastAssignedFieldID: id, } } -func (a *AssertLastAssignedFieldId) Validate(meta Metadata) error { +func (a *assertLastAssignedFieldId) Validate(meta Metadata) error { if meta == nil { return fmt.Errorf("requirement failed: current table metadata does not exist") } @@ -144,20 +151,21 @@ func (a *AssertLastAssignedFieldId) Validate(meta Metadata) error { return nil } -// AssertCurrentSchemaId validates that the table's current schema ID matches the requirement's `CurrentSchemaID`. -type AssertCurrentSchemaId struct { +type assertCurrentSchemaId struct { baseRequirement CurrentSchemaID int `json:"current-schema-id"` } -func NewAssertCurrentSchemaID(id int) *AssertCurrentSchemaId { - return &AssertCurrentSchemaId{ +// AssertCurrentSchemaId creates a requirement that the table's current schema ID +// matches the given id. +func AssertCurrentSchemaID(id int) Requirement { + return &assertCurrentSchemaId{ baseRequirement: baseRequirement{Type: "assert-current-schema-id"}, CurrentSchemaID: id, } } -func (a *AssertCurrentSchemaId) Validate(meta Metadata) error { +func (a *assertCurrentSchemaId) Validate(meta Metadata) error { if meta == nil { return fmt.Errorf("requirement failed: current table metadata does not exist") } @@ -169,20 +177,21 @@ func (a *AssertCurrentSchemaId) Validate(meta Metadata) error { return nil } -// AssertLastAssignedPartitionId validates that the table's last assigned partition ID matches the requirement's `LastAssignedPartitionID`. -type AssertLastAssignedPartitionId struct { +type assertLastAssignedPartitionId struct { baseRequirement LastAssignedPartitionID int `json:"last-assigned-partition-id"` } -func NewAssertLastAssignedPartitionID(id int) *AssertLastAssignedPartitionId { - return &AssertLastAssignedPartitionId{ +// AssertLastAssignedPartitionID creates a requriement that the table's last assigned partition ID +// matches the given id. +func AssertLastAssignedPartitionID(id int) Requirement { + return &assertLastAssignedPartitionId{ baseRequirement: baseRequirement{Type: "assert-last-assigned-partition-id"}, LastAssignedPartitionID: id, } } -func (a *AssertLastAssignedPartitionId) Validate(meta Metadata) error { +func (a *assertLastAssignedPartitionId) Validate(meta Metadata) error { if meta == nil { return fmt.Errorf("requirement failed: current table metadata does not exist") } @@ -194,20 +203,21 @@ func (a *AssertLastAssignedPartitionId) Validate(meta Metadata) error { return nil } -// AssertDefaultSpecId validates that the table's default partition spec ID matches the requirement's `DefaultSpecID`. -type AssertDefaultSpecId struct { +type assertDefaultSpecId struct { baseRequirement DefaultSpecID int `json:"default-spec-id"` } -func NewAssertDefaultSpecID(id int) *AssertDefaultSpecId { - return &AssertDefaultSpecId{ +// AssertDefaultSpecID creates a requirement that the table's default partition spec ID +// matches the given id. +func AssertDefaultSpecID(id int) Requirement { + return &assertDefaultSpecId{ baseRequirement: baseRequirement{Type: "assert-default-spec-id"}, DefaultSpecID: id, } } -func (a *AssertDefaultSpecId) Validate(meta Metadata) error { +func (a *assertDefaultSpecId) Validate(meta Metadata) error { if meta == nil { return fmt.Errorf("requirement failed: current table metadata does not exist") } @@ -219,20 +229,21 @@ func (a *AssertDefaultSpecId) Validate(meta Metadata) error { return nil } -// AssertDefaultSortOrderId validates that the table's default sort order ID matches the requirement's `DefaultSortOrderID`. -type AssertDefaultSortOrderId struct { +type assertDefaultSortOrderId struct { baseRequirement DefaultSortOrderID int `json:"default-sort-order-id"` } -func NewAssertDefaultSortOrderID(id int) *AssertDefaultSortOrderId { - return &AssertDefaultSortOrderId{ +// AssertDefaultSortOrderID creates a requirement that the table's default sort order ID +// matches the given id. +func AssertDefaultSortOrderID(id int) Requirement { + return &assertDefaultSortOrderId{ baseRequirement: baseRequirement{Type: "assert-default-sort-order-id"}, DefaultSortOrderID: id, } } -func (a *AssertDefaultSortOrderId) Validate(meta Metadata) error { +func (a *assertDefaultSortOrderId) Validate(meta Metadata) error { if meta == nil { return fmt.Errorf("requirement failed: current table metadata does not exist") } diff --git a/table/updates.go b/table/updates.go index 640935a..9cfaf49 100644 --- a/table/updates.go +++ b/table/updates.go @@ -42,59 +42,58 @@ func (u *baseUpdate) Action() string { return u.ActionName } -// AssignUUIDUpdate assigns a UUID to the table metadata. -type AssignUUIDUpdate struct { +type assignUUIDUpdate struct { baseUpdate UUID uuid.UUID `json:"uuid"` } -// NewAssignUUIDUpdate creates a new AssignUUIDUpdate with the given UUID. -func NewAssignUUIDUpdate(uuid uuid.UUID) *AssignUUIDUpdate { - return &AssignUUIDUpdate{ +// NewAssignUUIDUpdate creates a new update to assign a UUID to the table metadata. +func NewAssignUUIDUpdate(uuid uuid.UUID) Update { + return &assignUUIDUpdate{ baseUpdate: baseUpdate{ActionName: "assign-uuid"}, UUID: uuid, } } // Apply updates the UUID on the given metadata builder. -func (u *AssignUUIDUpdate) Apply(builder *MetadataBuilder) error { +func (u *assignUUIDUpdate) Apply(builder *MetadataBuilder) error { _, err := builder.SetUUID(u.UUID) return err } -// UpgradeFormatVersionUpdate upgrades the format version of the table metadata. -type UpgradeFormatVersionUpdate struct { +type upgradeFormatVersionUpdate struct { baseUpdate FormatVersion int `json:"format-version"` } -// NewUpgradeFormatVersionUpdate creates a new UpgradeFormatVersionUpdate with the given format version. -func NewUpgradeFormatVersionUpdate(formatVersion int) *UpgradeFormatVersionUpdate { - return &UpgradeFormatVersionUpdate{ +// NewUpgradeFormatVersionUpdate creates a new update that upgrades the format version +// of the table metadata to the given formatVersion. +func NewUpgradeFormatVersionUpdate(formatVersion int) Update { + return &upgradeFormatVersionUpdate{ baseUpdate: baseUpdate{ActionName: "upgrade-format-version"}, FormatVersion: formatVersion, } } // Apply upgrades the format version on the given metadata builder. -func (u *UpgradeFormatVersionUpdate) Apply(builder *MetadataBuilder) error { +func (u *upgradeFormatVersionUpdate) Apply(builder *MetadataBuilder) error { _, err := builder.SetFormatVersion(u.FormatVersion) return err } -// AddSchemaUpdate adds a schema to the table metadata. -type AddSchemaUpdate struct { +// addSchemaUpdate adds a schema to the table metadata. +type addSchemaUpdate struct { baseUpdate Schema *iceberg.Schema `json:"schema"` LastColumnID int `json:"last-column-id"` initial bool } -// NewAddSchemaUpdate creates a new AddSchemaUpdate with the given schema and last column ID. -// If the initial flag is set to true, the schema is considered the initial schema of the table, -// and all previously added schemas in the metadata builder are removed. -func NewAddSchemaUpdate(schema *iceberg.Schema, lastColumnID int, initial bool) *AddSchemaUpdate { - return &AddSchemaUpdate{ +// NewAddSchemaUpdate creates a new update that adds the given schema and last column ID to +// the table metadata. If the initial flag is set to true, the schema is considered the initial +// schema of the table, and all previously added schemas in the metadata builder are removed. +func NewAddSchemaUpdate(schema *iceberg.Schema, lastColumnID int, initial bool) Update { + return &addSchemaUpdate{ baseUpdate: baseUpdate{ActionName: "add-schema"}, Schema: schema, LastColumnID: lastColumnID, @@ -102,135 +101,131 @@ func NewAddSchemaUpdate(schema *iceberg.Schema, lastColumnID int, initial bool) } } -func (u *AddSchemaUpdate) Apply(builder *MetadataBuilder) error { +func (u *addSchemaUpdate) Apply(builder *MetadataBuilder) error { _, err := builder.AddSchema(u.Schema, u.LastColumnID, u.initial) return err } -// SetCurrentSchemaUpdate sets the current schema of the table metadata. -type SetCurrentSchemaUpdate struct { +type setCurrentSchemaUpdate struct { baseUpdate SchemaID int `json:"schema-id"` } -// NewSetCurrentSchemaUpdate creates a new SetCurrentSchemaUpdate with the given schema ID. -func NewSetCurrentSchemaUpdate(id int) *SetCurrentSchemaUpdate { - return &SetCurrentSchemaUpdate{ +// NewSetCurrentSchemaUpdate creates a new update that sets the current schema of the table +// metadata to the given schema ID. +func NewSetCurrentSchemaUpdate(id int) Update { + return &setCurrentSchemaUpdate{ baseUpdate: baseUpdate{ActionName: "set-current-schema"}, SchemaID: id, } } -func (u *SetCurrentSchemaUpdate) Apply(builder *MetadataBuilder) error { +func (u *setCurrentSchemaUpdate) Apply(builder *MetadataBuilder) error { _, err := builder.SetCurrentSchemaID(u.SchemaID) return err } -// AddPartitionSpecUpdate adds a partition spec to the table metadata. -type AddPartitionSpecUpdate struct { +type addPartitionSpecUpdate struct { baseUpdate Spec *iceberg.PartitionSpec `json:"spec"` initial bool } -// NewAddPartitionSpecUpdate creates a new AddPartitionSpecUpdate with the given partition spec. -// If the initial flag is set to true, the spec is considered the initial spec of the table, +// 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 NewAddPartitionSpecUpdate(spec *iceberg.PartitionSpec, initial bool) *AddPartitionSpecUpdate { - return &AddPartitionSpecUpdate{ +func NewAddPartitionSpecUpdate(spec *iceberg.PartitionSpec, initial bool) Update { + return &addPartitionSpecUpdate{ baseUpdate: baseUpdate{ActionName: "add-spec"}, Spec: spec, initial: initial, } } -func (u *AddPartitionSpecUpdate) Apply(builder *MetadataBuilder) error { +func (u *addPartitionSpecUpdate) Apply(builder *MetadataBuilder) error { _, err := builder.AddPartitionSpec(u.Spec, u.initial) return err } -// SetDefaultSpecUpdate sets the default partition spec of the table metadata. -type SetDefaultSpecUpdate struct { +type setDefaultSpecUpdate struct { baseUpdate SpecID int `json:"spec-id"` } -// NewSetDefaultSpecUpdate creates a new SetDefaultSpecUpdate with the given spec ID. -func NewSetDefaultSpecUpdate(id int) *SetDefaultSpecUpdate { - return &SetDefaultSpecUpdate{ +// NewSetDefaultSpecUpdate creates a new update that sets the default partition spec of the +// table metadata to the given spec ID. +func NewSetDefaultSpecUpdate(id int) Update { + return &setDefaultSpecUpdate{ baseUpdate: baseUpdate{ActionName: "set-default-spec"}, SpecID: id, } } -func (u *SetDefaultSpecUpdate) Apply(builder *MetadataBuilder) error { +func (u *setDefaultSpecUpdate) Apply(builder *MetadataBuilder) error { _, err := builder.SetDefaultSpecID(u.SpecID) return err } -// AddSortOrderUpdate adds a sort order to the table metadata. -type AddSortOrderUpdate struct { +type addSortOrderUpdate struct { baseUpdate SortOrder *SortOrder `json:"sort-order"` initial bool } -// NewAddSortOrderUpdate creates a new AddSortOrderUpdate with the given sort order. +// 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 NewAddSortOrderUpdate(sortOrder *SortOrder, initial bool) *AddSortOrderUpdate { - return &AddSortOrderUpdate{ +func NewAddSortOrderUpdate(sortOrder *SortOrder, initial bool) Update { + return &addSortOrderUpdate{ baseUpdate: baseUpdate{ActionName: "add-sort-order"}, SortOrder: sortOrder, initial: initial, } } -func (u *AddSortOrderUpdate) Apply(builder *MetadataBuilder) error { +func (u *addSortOrderUpdate) Apply(builder *MetadataBuilder) error { _, err := builder.AddSortOrder(u.SortOrder, u.initial) return err } -// SetDefaultSortOrderUpdate sets the default sort order of the table metadata. -type SetDefaultSortOrderUpdate struct { +type setDefaultSortOrderUpdate struct { baseUpdate SortOrderID int `json:"sort-order-id"` } -// NewSetDefaultSortOrderUpdate creates a new SetDefaultSortOrderUpdate with the given sort order ID. -func NewSetDefaultSortOrderUpdate(id int) *SetDefaultSortOrderUpdate { - return &SetDefaultSortOrderUpdate{ +// NewSetDefaultSortOrderUpdate creates a new update that sets the default sort order of the table metadata +// to the given sort order ID. +func NewSetDefaultSortOrderUpdate(id int) Update { + return &setDefaultSortOrderUpdate{ baseUpdate: baseUpdate{ActionName: "set-default-sort-order"}, SortOrderID: id, } } -func (u *SetDefaultSortOrderUpdate) Apply(builder *MetadataBuilder) error { +func (u *setDefaultSortOrderUpdate) Apply(builder *MetadataBuilder) error { _, err := builder.SetDefaultSortOrderID(u.SortOrderID) return err } -// AddSnapshotUpdate adds a snapshot to the table metadata. -type AddSnapshotUpdate struct { +type addSnapshotUpdate struct { baseUpdate Snapshot *Snapshot `json:"snapshot"` } -// NewAddSnapshotUpdate creates a new AddSnapshotUpdate with the given snapshot. -func NewAddSnapshotUpdate(snapshot *Snapshot) *AddSnapshotUpdate { - return &AddSnapshotUpdate{ +// NewAddSnapshotUpdate creates a new update that adds the given snapshot to the table metadata. +func NewAddSnapshotUpdate(snapshot *Snapshot) Update { + return &addSnapshotUpdate{ baseUpdate: baseUpdate{ActionName: "add-snapshot"}, Snapshot: snapshot, } } -func (u *AddSnapshotUpdate) Apply(builder *MetadataBuilder) error { +func (u *addSnapshotUpdate) Apply(builder *MetadataBuilder) error { _, err := builder.AddSnapshot(u.Snapshot) return err } -// SetCurrentSnapshotUpdate sets the current snapshot of the table metadata. -type SetSnapshotRefUpdate struct { +type setSnapshotRefUpdate struct { baseUpdate RefName string `json:"ref-name"` RefType RefType `json:"type"` @@ -240,16 +235,17 @@ type SetSnapshotRefUpdate struct { MinSnapshotsToKeep int `json:"min-snapshots-to-keep,omitempty"` } -// NewSetSnapshotRefUpdate creates a new SetSnapshotRefUpdate with the given snapshot reference information. -// MaxRefAgeMs, MaxSnapshotAgeMs, and MinSnapshotsToKeep are optional, and any non-positive values are ignored. +// 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 NewSetSnapshotRefUpdate( name string, snapshotID int64, refType RefType, maxRefAgeMs, maxSnapshotAgeMs int64, minSnapshotsToKeep int, -) *SetSnapshotRefUpdate { - return &SetSnapshotRefUpdate{ +) Update { + return &setSnapshotRefUpdate{ baseUpdate: baseUpdate{ActionName: "set-snapshot-ref"}, RefName: name, RefType: refType, @@ -260,7 +256,7 @@ func NewSetSnapshotRefUpdate( } } -func (u *SetSnapshotRefUpdate) Apply(builder *MetadataBuilder) error { +func (u *setSnapshotRefUpdate) Apply(builder *MetadataBuilder) error { opts := []setSnapshotRefOption{} if u.MaxRefAgeMs >= 0 { opts = append(opts, WithMaxRefAgeMs(u.MaxRefAgeMs)) @@ -281,97 +277,95 @@ func (u *SetSnapshotRefUpdate) Apply(builder *MetadataBuilder) error { return err } -// SetLocationUpdate sets the location of the table metadata. -type SetLocationUpdate struct { +type setLocationUpdate struct { baseUpdate Location string `json:"location"` } -// NewSetLocationUpdate creates a new SetLocationUpdate with the given location. -func NewSetLocationUpdate(loc string) *SetLocationUpdate { - return &SetLocationUpdate{ +// NewSetLocationUpdate creates a new update that sets the location of the table metadata. +func NewSetLocationUpdate(loc string) Update { + return &setLocationUpdate{ baseUpdate: baseUpdate{ActionName: "set-location"}, Location: loc, } } -func (u *SetLocationUpdate) Apply(builder *MetadataBuilder) error { +func (u *setLocationUpdate) Apply(builder *MetadataBuilder) error { _, err := builder.SetLoc(u.Location) return err } -// SetPropertyUpdate sets a number of properties in the table metadata. -type SetPropertiesUpdate struct { +type setPropertiesUpdate struct { baseUpdate Updates iceberg.Properties `json:"updates"` } -// NewSetPropertiesUpdate creates a new SetPropertiesUpdate with the given properties. -func NewSetPropertiesUpdate(updates iceberg.Properties) *SetPropertiesUpdate { - return &SetPropertiesUpdate{ +// NewSetPropertiesUpdate creates a new update that sets the given properties in the +// table metadata. +func NewSetPropertiesUpdate(updates iceberg.Properties) *setPropertiesUpdate { + return &setPropertiesUpdate{ baseUpdate: baseUpdate{ActionName: "set-properties"}, Updates: updates, } } -func (u *SetPropertiesUpdate) Apply(builder *MetadataBuilder) error { +func (u *setPropertiesUpdate) Apply(builder *MetadataBuilder) error { _, err := builder.SetProperties(u.Updates) return err } -// RemovePropertiesUpdate removes a number of 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. -type RemovePropertiesUpdate struct { +type removePropertiesUpdate struct { baseUpdate Removals []string `json:"removals"` } -// NewRemovePropertiesUpdate creates a new RemovePropertiesUpdate with the given property names. -func NewRemovePropertiesUpdate(removals []string) *RemovePropertiesUpdate { - return &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 NewRemovePropertiesUpdate(removals []string) Update { + return &removePropertiesUpdate{ baseUpdate: baseUpdate{ActionName: "remove-properties"}, Removals: removals, } } -func (u *RemovePropertiesUpdate) Apply(builder *MetadataBuilder) error { +func (u *removePropertiesUpdate) Apply(builder *MetadataBuilder) error { _, err := builder.RemoveProperties(u.Removals) return err } -// RemoveSnapshotsUpdate removes a number of snapshots from the table metadata. -type RemoveSnapshotsUpdate struct { +type removeSnapshotsUpdate struct { baseUpdate SnapshotIDs []int64 `json:"snapshot-ids"` } -// NewRemoveSnapshotsUpdate creates a new RemoveSnapshotsUpdate with the given snapshot IDs. -func NewRemoveSnapshotsUpdate(ids []int64) *RemoveSnapshotsUpdate { - return &RemoveSnapshotsUpdate{ +// NewRemoveSnapshotsUpdate creates a new update that removes all snapshots from +// the table metadata with the given snapshot IDs. +func NewRemoveSnapshotsUpdate(ids []int64) Update { + return &removeSnapshotsUpdate{ baseUpdate: baseUpdate{ActionName: "remove-snapshots"}, SnapshotIDs: ids, } } -func (u *RemoveSnapshotsUpdate) Apply(builder *MetadataBuilder) error { +func (u *removeSnapshotsUpdate) Apply(builder *MetadataBuilder) error { return fmt.Errorf("%w: remove-snapshots", iceberg.ErrNotImplemented) } -// RemoveSnapshotRefUpdate removes a snapshot reference from the table metadata. -type RemoveSnapshotRefUpdate struct { +type removeSnapshotRefUpdate struct { baseUpdate RefName string `json:"ref-name"` } -// NewRemoveSnapshotRefUpdate creates a new RemoveSnapshotRefUpdate with the given reference name. -func NewRemoveSnapshotRefUpdate(ref string) *RemoveSnapshotRefUpdate { - return &RemoveSnapshotRefUpdate{ +// NewRemoveSnapshotRefUpdate creates a new update that removes a snapshot reference +// from the table metadata. +func NewRemoveSnapshotRefUpdate(ref string) *removeSnapshotRefUpdate { + return &removeSnapshotRefUpdate{ baseUpdate: baseUpdate{ActionName: "remove-snapshot-ref"}, RefName: ref, } } -func (u *RemoveSnapshotRefUpdate) Apply(builder *MetadataBuilder) error { +func (u *removeSnapshotRefUpdate) Apply(builder *MetadataBuilder) error { return fmt.Errorf("%w: remove-snapshot-ref", iceberg.ErrNotImplemented) } From 1b155a65d0a2ac3c844aa347e9e573916910454a Mon Sep 17 00:00:00 2001 From: Jonas Weile Date: Wed, 18 Dec 2024 19:26:49 +0000 Subject: [PATCH 22/28] extract consts --- table/requirements.go | 27 +++++++++++++++++++-------- table/updates.go | 27 +++++++++++++++++++-------- 2 files changed, 38 insertions(+), 16 deletions(-) diff --git a/table/requirements.go b/table/requirements.go index 0f3110d..5a4b883 100644 --- a/table/requirements.go +++ b/table/requirements.go @@ -23,6 +23,17 @@ import ( "github.com/google/uuid" ) +const ( + reqAssertCreate = "assert-create" + reqAssertTableUUID = "assert-table-uuid" + reqAssertRefSnapshotID = "assert-ref-snapshot-id" + reqAssertDefaultSpecID = "assert-default-spec-id" + reqAssertCurrentSchemaID = "assert-current-schema-id" + reqAssertDefaultSortOrderID = "assert-default-sort-order-id" + reqAssertLastAssignedFieldID = "assert-last-assigned-field-id" + reqAssertLastAssignedPartitionID = "assert-last-assigned-partition-id" +) + // 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. @@ -44,7 +55,7 @@ type assertCreate struct { // AssertCreate creates a requirement that the table does not already exist. func AssertCreate() Requirement { return &assertCreate{ - baseRequirement: baseRequirement{Type: "assert-create"}, + baseRequirement: baseRequirement{Type: reqAssertCreate}, } } @@ -64,7 +75,7 @@ type assertTableUuid struct { // AssertTableUUID creates a requirement that the table UUID matches the given UUID. func AssertTableUUID(uuid uuid.UUID) Requirement { return &assertTableUuid{ - baseRequirement: baseRequirement{Type: "assert-table-uuid"}, + baseRequirement: baseRequirement{Type: reqAssertTableUUID}, UUID: uuid, } } @@ -92,7 +103,7 @@ type assertRefSnapshotID struct { // If the id is nil, the ref must not already exist. func AssertRefSnapshotID(ref string, id *int64) Requirement { return &assertRefSnapshotID{ - baseRequirement: baseRequirement{Type: "assert-ref-snapshot-id"}, + baseRequirement: baseRequirement{Type: reqAssertRefSnapshotID}, Ref: ref, SnapshotID: id, } @@ -134,7 +145,7 @@ type assertLastAssignedFieldId struct { // matches the given id. func AssertLastAssignedFieldID(id int) Requirement { return &assertLastAssignedFieldId{ - baseRequirement: baseRequirement{Type: "assert-last-assigned-field-id"}, + baseRequirement: baseRequirement{Type: reqAssertLastAssignedFieldID}, LastAssignedFieldID: id, } } @@ -160,7 +171,7 @@ type assertCurrentSchemaId struct { // matches the given id. func AssertCurrentSchemaID(id int) Requirement { return &assertCurrentSchemaId{ - baseRequirement: baseRequirement{Type: "assert-current-schema-id"}, + baseRequirement: baseRequirement{Type: reqAssertCurrentSchemaID}, CurrentSchemaID: id, } } @@ -186,7 +197,7 @@ type assertLastAssignedPartitionId struct { // matches the given id. func AssertLastAssignedPartitionID(id int) Requirement { return &assertLastAssignedPartitionId{ - baseRequirement: baseRequirement{Type: "assert-last-assigned-partition-id"}, + baseRequirement: baseRequirement{Type: reqAssertLastAssignedPartitionID}, LastAssignedPartitionID: id, } } @@ -212,7 +223,7 @@ type assertDefaultSpecId struct { // matches the given id. func AssertDefaultSpecID(id int) Requirement { return &assertDefaultSpecId{ - baseRequirement: baseRequirement{Type: "assert-default-spec-id"}, + baseRequirement: baseRequirement{Type: reqAssertDefaultSpecID}, DefaultSpecID: id, } } @@ -238,7 +249,7 @@ type assertDefaultSortOrderId struct { // matches the given id. func AssertDefaultSortOrderID(id int) Requirement { return &assertDefaultSortOrderId{ - baseRequirement: baseRequirement{Type: "assert-default-sort-order-id"}, + baseRequirement: baseRequirement{Type: reqAssertDefaultSortOrderID}, DefaultSortOrderID: id, } } diff --git a/table/updates.go b/table/updates.go index 9cfaf49..089467c 100644 --- a/table/updates.go +++ b/table/updates.go @@ -24,6 +24,17 @@ import ( "github.com/google/uuid" ) +const ( + updateSpec = "add-spec" + updateAddSchema = "add-schema" + updateSortOrder = "add-sort-order" + updateAssignUUID = "assign-uuid" + updateDefaultSpec = "set-default-spec" + updateCurrentSchema = "set-current-schema" + updateDefaultSortOrder = "set-default-sort-order" + updateUpgradeFormat = "upgrade-format-version" +) + // Update represents a change to a table's metadata. type Update interface { // Action returns the name of the action that the update represents. @@ -50,7 +61,7 @@ type assignUUIDUpdate struct { // NewAssignUUIDUpdate creates a new update to assign a UUID to the table metadata. func NewAssignUUIDUpdate(uuid uuid.UUID) Update { return &assignUUIDUpdate{ - baseUpdate: baseUpdate{ActionName: "assign-uuid"}, + baseUpdate: baseUpdate{ActionName: updateAssignUUID}, UUID: uuid, } } @@ -70,7 +81,7 @@ type upgradeFormatVersionUpdate struct { // of the table metadata to the given formatVersion. func NewUpgradeFormatVersionUpdate(formatVersion int) Update { return &upgradeFormatVersionUpdate{ - baseUpdate: baseUpdate{ActionName: "upgrade-format-version"}, + baseUpdate: baseUpdate{ActionName: updateUpgradeFormat}, FormatVersion: formatVersion, } } @@ -94,7 +105,7 @@ type addSchemaUpdate struct { // schema of the table, and all previously added schemas in the metadata builder are removed. func NewAddSchemaUpdate(schema *iceberg.Schema, lastColumnID int, initial bool) Update { return &addSchemaUpdate{ - baseUpdate: baseUpdate{ActionName: "add-schema"}, + baseUpdate: baseUpdate{ActionName: updateAddSchema}, Schema: schema, LastColumnID: lastColumnID, initial: initial, @@ -115,7 +126,7 @@ type setCurrentSchemaUpdate struct { // metadata to the given schema ID. func NewSetCurrentSchemaUpdate(id int) Update { return &setCurrentSchemaUpdate{ - baseUpdate: baseUpdate{ActionName: "set-current-schema"}, + baseUpdate: baseUpdate{ActionName: updateCurrentSchema}, SchemaID: id, } } @@ -136,7 +147,7 @@ type addPartitionSpecUpdate struct { // and all other previously added specs in the metadata builder are removed. func NewAddPartitionSpecUpdate(spec *iceberg.PartitionSpec, initial bool) Update { return &addPartitionSpecUpdate{ - baseUpdate: baseUpdate{ActionName: "add-spec"}, + baseUpdate: baseUpdate{ActionName: updateSpec}, Spec: spec, initial: initial, } @@ -156,7 +167,7 @@ type setDefaultSpecUpdate struct { // table metadata to the given spec ID. func NewSetDefaultSpecUpdate(id int) Update { return &setDefaultSpecUpdate{ - baseUpdate: baseUpdate{ActionName: "set-default-spec"}, + baseUpdate: baseUpdate{ActionName: updateDefaultSpec}, SpecID: id, } } @@ -177,7 +188,7 @@ type addSortOrderUpdate struct { // and all previously added sort orders in the metadata builder are removed. func NewAddSortOrderUpdate(sortOrder *SortOrder, initial bool) Update { return &addSortOrderUpdate{ - baseUpdate: baseUpdate{ActionName: "add-sort-order"}, + baseUpdate: baseUpdate{ActionName: updateSortOrder}, SortOrder: sortOrder, initial: initial, } @@ -197,7 +208,7 @@ type setDefaultSortOrderUpdate struct { // to the given sort order ID. func NewSetDefaultSortOrderUpdate(id int) Update { return &setDefaultSortOrderUpdate{ - baseUpdate: baseUpdate{ActionName: "set-default-sort-order"}, + baseUpdate: baseUpdate{ActionName: updateDefaultSortOrder}, SortOrderID: id, } } From 1291ddd7d20287cf645589d6297e07afe5b2cb5b Mon Sep 17 00:00:00 2001 From: Jonas Weile Date: Wed, 18 Dec 2024 19:29:03 +0000 Subject: [PATCH 23/28] make comments consistent --- table/updates.go | 3 --- 1 file changed, 3 deletions(-) diff --git a/table/updates.go b/table/updates.go index 089467c..7b09a0f 100644 --- a/table/updates.go +++ b/table/updates.go @@ -66,7 +66,6 @@ func NewAssignUUIDUpdate(uuid uuid.UUID) Update { } } -// Apply updates the UUID on the given metadata builder. func (u *assignUUIDUpdate) Apply(builder *MetadataBuilder) error { _, err := builder.SetUUID(u.UUID) return err @@ -86,13 +85,11 @@ func NewUpgradeFormatVersionUpdate(formatVersion int) Update { } } -// Apply upgrades the format version on the given metadata builder. func (u *upgradeFormatVersionUpdate) Apply(builder *MetadataBuilder) error { _, err := builder.SetFormatVersion(u.FormatVersion) return err } -// addSchemaUpdate adds a schema to the table metadata. type addSchemaUpdate struct { baseUpdate Schema *iceberg.Schema `json:"schema"` From 29d1ce29334244707b506bf19a3ab79828328704 Mon Sep 17 00:00:00 2001 From: Jonas Weile Date: Wed, 18 Dec 2024 20:06:02 +0000 Subject: [PATCH 24/28] run go tidy --- go.sum | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/go.sum b/go.sum index 074d534..e490952 100644 --- a/go.sum +++ b/go.sum @@ -238,10 +238,10 @@ github.com/pterm/pterm v0.12.40/go.mod h1:ffwPLwlbXxP+rxT0GsgDTzS3y3rmpAO1NMjUkG github.com/pterm/pterm v0.12.80 h1:mM55B+GnKUnLMUSqhdINe4s6tOuVQIetQ3my8JGyAIg= github.com/pterm/pterm v0.12.80/go.mod h1:c6DeF9bSnOSeFPZlfs4ZRAFcf5SCoTwvwQ5xaKGQlHo= github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= -github.com/rivo/uniseg v0.4.4 h1:8TfxU8dW6PdqD27gjM8MVNuicgxIjxpm4K7x4jp8sis= -github.com/rivo/uniseg v0.4.4/go.mod h1:FN3SvrM+Zdj16jyLfmOkMNblXMcoc8DfTHruCPUcx88= -github.com/rogpeppe/go-internal v1.9.0 h1:73kH8U+JUqXU8lRuOHeVHaa/SZPifC7BkcraZVejAe8= -github.com/rogpeppe/go-internal v1.9.0/go.mod h1:WtVeX8xhTBvf0smdhujwtBcq4Qrzq/fJaraNFVN+nFs= +github.com/rivo/uniseg v0.4.7 h1:WUdvkW8uEhrYfLC4ZzdpI2ztxP1I582+49Oc5Mq64VQ= +github.com/rivo/uniseg v0.4.7/go.mod h1:FN3SvrM+Zdj16jyLfmOkMNblXMcoc8DfTHruCPUcx88= +github.com/rogpeppe/go-internal v1.12.0 h1:exVL4IDcn6na9z1rAb56Vxr+CgyK3nn3O+epU5NdKM8= +github.com/rogpeppe/go-internal v1.12.0/go.mod h1:E+RYuTGaKKdloAfM02xzb0FW3Paa99yedzYV+kq4uf4= github.com/sergi/go-diff v1.2.0 h1:XU+rvMAioB0UC3q1MFrIQy4Vo5/4VsRDQQXHsEya6xQ= github.com/sergi/go-diff v1.2.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= From be00619801743d1dbccbe8c5128a4f1e2928c312 Mon Sep 17 00:00:00 2001 From: Jonas Weile Date: Wed, 18 Dec 2024 20:06:27 +0000 Subject: [PATCH 25/28] add seperate PurgeTable func --- catalog/catalog.go | 3 +-- catalog/rest.go | 22 ++++++++++++++++------ 2 files changed, 17 insertions(+), 8 deletions(-) diff --git a/catalog/catalog.go b/catalog/catalog.go index f048a60..9b5776d 100644 --- a/catalog/catalog.go +++ b/catalog/catalog.go @@ -158,8 +158,7 @@ type Catalog interface { // LoadTable loads a table from the catalog and returns a Table with the metadata. LoadTable(ctx context.Context, identifier table.Identifier, props iceberg.Properties) (*table.Table, error) // DropTable tells the catalog to drop the table entirely. - // If the purge flag is set, it is requested to purge the underlying table's data and metadata. - DropTable(ctx context.Context, identifier table.Identifier, purge bool) error + DropTable(ctx context.Context, identifier table.Identifier) error // RenameTable tells the catalog to rename a given table by the identifiers // provided, and then loads and returns the destination table RenameTable(ctx context.Context, from, to table.Identifier) (*table.Table, error) diff --git a/catalog/rest.go b/catalog/rest.go index 87d50a7..47be1eb 100644 --- a/catalog/rest.go +++ b/catalog/rest.go @@ -763,19 +763,29 @@ func (r *RestCatalog) UpdateTable(ctx context.Context, ident table.Identifier, r return r.tableFromResponse(ident, ret.Metadata, ret.MetadataLoc, config) } -func (r *RestCatalog) DropTable(ctx context.Context, identifier table.Identifier, purge bool) error { +func (r *RestCatalog) DropTable(ctx context.Context, identifier table.Identifier) error { ns, tbl, err := splitIdentForPath(identifier) if err != nil { return err } - uri := r.baseURI.JoinPath("namespaces", ns, "tables", tbl) - if purge { - v := url.Values{} - v.Set("purgeRequested", "true") - uri.RawQuery = v.Encode() + _, err = doDelete[struct{}](ctx, r.baseURI, []string{"namespaces", ns, "tables", tbl}, r.cl, + map[int]error{http.StatusNotFound: ErrNoSuchTable}) + + return err +} + +func (r *RestCatalog) PurgeTable(ctx context.Context, identifier table.Identifier) error { + ns, tbl, err := splitIdentForPath(identifier) + if err != nil { + return err } + uri := r.baseURI.JoinPath("namespaces", ns, "tables", tbl) + v := url.Values{} + v.Set("purgeRequested", "true") + uri.RawQuery = v.Encode() + _, err = doDelete[struct{}](ctx, uri, []string{}, r.cl, map[int]error{http.StatusNotFound: ErrNoSuchTable}) From 8f51473382cd2f93235fe3f80cff8993d07ccc22 Mon Sep 17 00:00:00 2001 From: Jonas Weile Date: Wed, 18 Dec 2024 20:23:55 +0000 Subject: [PATCH 26/28] impossible nil checks --- table/metadata.go | 28 +++++++++------------------- 1 file changed, 9 insertions(+), 19 deletions(-) diff --git a/table/metadata.go b/table/metadata.go index b506575..bd03464 100644 --- a/table/metadata.go +++ b/table/metadata.go @@ -94,7 +94,7 @@ type Metadata interface { CurrentSnapshot() *Snapshot // Ref returns the snapshot ref for the main branch. Ref() SnapshotRef - // Refs returns a map of snapshot refs by name. + // 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] @@ -275,20 +275,18 @@ func (b *MetadataBuilder) AddSnapshot(snapshot *Snapshot) (*MetadataBuilder, err } func (b *MetadataBuilder) AddSortOrder(sortOrder *SortOrder, initial bool) (*MetadataBuilder, error) { - for _, s := range b.sortOrderList { - if s.OrderID == sortOrder.OrderID && !initial { - return nil, fmt.Errorf("sort order with id %d already exists", sortOrder.OrderID) - } - } - var sortOrders []SortOrder - if initial { - sortOrders = []SortOrder{*sortOrder} - } else { + if !initial { sortOrders = append(b.sortOrderList, *sortOrder) } - b.sortOrderList = sortOrders + for _, s := range sortOrders { + if s.OrderID == sortOrder.OrderID { + return nil, fmt.Errorf("sort order with id %d already exists", sortOrder.OrderID) + } + } + + b.sortOrderList = append(sortOrders, *sortOrder) b.updates = append(b.updates, NewAddSortOrderUpdate(sortOrder, initial)) return b, nil @@ -945,10 +943,6 @@ func (m *metadataV1) Equals(other Metadata) bool { return true } - if m == nil || rhs == nil { - return false - } - return m.Schema.Equals(rhs.Schema) && slices.Equal(m.Partition, rhs.Partition) && m.commonMetadata.Equals(&rhs.commonMetadata) } @@ -1020,10 +1014,6 @@ func (m *metadataV2) Equals(other Metadata) bool { return true } - if m == nil || rhs == nil { - return false - } - return m.LastSequenceNumber == rhs.LastSequenceNumber && m.commonMetadata.Equals(&rhs.commonMetadata) } From b578caded2fd3b7e269839df2b96113fac97bfc0 Mon Sep 17 00:00:00 2001 From: Jonas Weile Date: Wed, 18 Dec 2024 20:27:54 +0000 Subject: [PATCH 27/28] remove extra arg --- cmd/iceberg/main.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmd/iceberg/main.go b/cmd/iceberg/main.go index 1bf687a..8db7cc7 100644 --- a/cmd/iceberg/main.go +++ b/cmd/iceberg/main.go @@ -212,7 +212,7 @@ func main() { os.Exit(1) } case cfg.Table: - err := cat.DropTable(context.Background(), catalog.ToRestIdentifier(cfg.Ident), false) + err := cat.DropTable(context.Background(), catalog.ToRestIdentifier(cfg.Ident)) if err != nil { output.Error(err) os.Exit(1) From da889ecf456c4fdf0b2bb14be3f807c24162242e Mon Sep 17 00:00:00 2001 From: Jonas Weile Date: Wed, 18 Dec 2024 20:38:27 +0000 Subject: [PATCH 28/28] fix --- table/metadata.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/table/metadata.go b/table/metadata.go index bd03464..73021c5 100644 --- a/table/metadata.go +++ b/table/metadata.go @@ -277,7 +277,7 @@ func (b *MetadataBuilder) AddSnapshot(snapshot *Snapshot) (*MetadataBuilder, err func (b *MetadataBuilder) AddSortOrder(sortOrder *SortOrder, initial bool) (*MetadataBuilder, error) { var sortOrders []SortOrder if !initial { - sortOrders = append(b.sortOrderList, *sortOrder) + sortOrders = append(sortOrders, b.sortOrderList...) } for _, s := range sortOrders {