Skip to content
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
5 changes: 5 additions & 0 deletions table/properties.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,11 @@ const (
ObjectStoreEnabledKey = "write.object-storage.enabled"
ObjectStoreEnabledDefault = false

DeleteMode = "write.delete.mode"
DeleteModeCopyOnWrite = "copy-on-write"
DeleteModeMergeOnRead = "merge-on-read"
DeleteModeDefault = DeleteModeCopyOnWrite

DefaultNameMappingKey = "schema.name-mapping.default"

MetricsModeColumnConfPrefix = "write.metadata.metrics.column"
Expand Down
14 changes: 14 additions & 0 deletions table/scanner.go
Original file line number Diff line number Diff line change
Expand Up @@ -501,3 +501,17 @@ func (scan *Scan) ToArrowTable(ctx context.Context) (arrow.Table, error) {

return array.NewTableFromRecords(schema, records), nil
}

func toArrowTableFromRecords(schema *arrow.Schema, recordItr iter.Seq2[arrow.Record, error]) (arrow.Table, error) {
records := make([]arrow.Record, 0)
for rec, err := range recordItr {
if err != nil {
return nil, err
}

defer rec.Release()
records = append(records, rec)
}

return array.NewTableFromRecords(schema, records), nil
}
179 changes: 179 additions & 0 deletions table/snapshot_producers.go
Original file line number Diff line number Diff line change
Expand Up @@ -410,6 +410,185 @@
return append(result, unmergedDeleteManifests...), nil
}

type deleteFiles struct {
base *snapshotProducer

predicate iceberg.BooleanExpression
caseSensitive bool

keepManifests []iceberg.ManifestFile
removedEntries []iceberg.ManifestEntry
partialRewrites bool
}

func newDeleteFilesProducer(
op Operation,
txn *Transaction,
predicate iceberg.BooleanExpression,
caseSensitive bool,
fs iceio.WriteFileIO,
commitUUID *uuid.UUID,
snapshotProps iceberg.Properties,
) *snapshotProducer {
prod := createSnapshotProducer(op, txn, fs, commitUUID, snapshotProps)
prod.producerImpl = &deleteFiles{
base: prod,
predicate: predicate,
caseSensitive: caseSensitive,
keepManifests: make([]iceberg.ManifestFile, 0),
removedEntries: make([]iceberg.ManifestEntry, 0),
partialRewrites: false,
}

return prod
}

func (df *deleteFiles) buildPartitionProjection(specID int32) (iceberg.BooleanExpression, error) {
schema := df.base.txn.tbl.Schema()
spec := df.base.txn.tbl.Metadata().PartitionSpecs()[specID]
project := newInclusiveProjection(schema, spec, df.caseSensitive)
partitionFilter, err := project(df.predicate)
if err != nil {
return nil, err
}
return partitionFilter, nil

Check failure on line 454 in table/snapshot_producers.go

View workflow job for this annotation

GitHub Actions / windows-latest go1.23.6

return with no blank line before (nlreturn)

Check failure on line 454 in table/snapshot_producers.go

View workflow job for this annotation

GitHub Actions / ubuntu-latest go1.23.6

return with no blank line before (nlreturn)

Check failure on line 454 in table/snapshot_producers.go

View workflow job for this annotation

GitHub Actions / macos-latest go1.23.6

return with no blank line before (nlreturn)
}

func (df *deleteFiles) partitionFilters() *keyDefaultMap[int32, iceberg.BooleanExpression] {
return newKeyDefaultMapWrapErr(df.buildPartitionProjection)
}

func (df *deleteFiles) buildManifestEvaluator(specID int32) (func(iceberg.ManifestFile) (bool, error), error) {
spec := df.base.txn.tbl.metadata.PartitionSpecs()[specID]
schema := df.base.txn.tbl.metadata.CurrentSchema()
return newManifestEvaluator(

Check failure on line 464 in table/snapshot_producers.go

View workflow job for this annotation

GitHub Actions / windows-latest go1.23.6

return with no blank line before (nlreturn)

Check failure on line 464 in table/snapshot_producers.go

View workflow job for this annotation

GitHub Actions / ubuntu-latest go1.23.6

return with no blank line before (nlreturn)

Check failure on line 464 in table/snapshot_producers.go

View workflow job for this annotation

GitHub Actions / macos-latest go1.23.6

return with no blank line before (nlreturn)
spec,
schema,
df.partitionFilters().Get(specID),
df.caseSensitive)
}

func (df *deleteFiles) computeDeletes() error {
schema := df.base.txn.tbl.Schema()

manifestEvaluators := newKeyDefaultMapWrapErr(df.buildManifestEvaluator)
strictMetricsEvaluator, err := newStrictMetricsEvaluator(schema, df.predicate, df.caseSensitive, false)
if err != nil {
return nil
}
inclusiveMetricsEvaluator, err := newInclusiveMetricsEvaluator(schema, df.predicate, df.caseSensitive, false)
if err != nil {
return nil
}

snapshot := df.base.txn.meta.currentSnapshot()
manifestFiles, err := snapshot.Manifests(df.base.io)
if err != nil {
return err
}
for _, manifestFile := range manifestFiles {
if manifestFile.ManifestContent() == iceberg.ManifestContentData {
containMatch, err := manifestEvaluators.Get(manifestFile.PartitionSpecID())(manifestFile)
if err != nil {
return err
}
if !containMatch {
df.keepManifests = append(df.keepManifests, manifestFile)
} else {
deletedEntries := make([]iceberg.ManifestEntry, 0)
keepEntries := make([]iceberg.ManifestEntry, 0)
entries, err := manifestFile.FetchEntries(df.base.io, true)
if err != nil {
return err
}
for _, entry := range entries {
ok, err := strictMetricsEvaluator(entry.DataFile())
if err != nil {
return err
}
if ok == rowsMustMatch {
deletedEntries = append(deletedEntries, df.copyWithNewStatus(entry, iceberg.EntryStatusDELETED))
} else {
keepEntries = append(keepEntries, df.copyWithNewStatus(entry, iceberg.EntryStatusEXISTING))
ok, err := inclusiveMetricsEvaluator(entry.DataFile())
if err != nil {
return err
}
if ok == rowsMightNotMatch {
df.partialRewrites = true
}
}
}

if len(deletedEntries) > 0 {
df.removedEntries = append(df.removedEntries, deletedEntries...)
if len(keepEntries) > 0 {
spec := df.base.txn.tbl.Metadata().PartitionSpecs()[manifestFile.PartitionSpecID()]
wr, path, counter, err := df.base.newManifestWriter(spec)
if err != nil {
return err
}
for _, entry := range keepEntries {
err = wr.Add(entry)
if err != nil {
return err
}
}
if err = wr.Close(); err != nil {
return err
}
mf, err := wr.ToManifestFile(path, counter.Count)
if err != nil {
return err
}
df.keepManifests = append(df.keepManifests, mf)
}
} else {
df.keepManifests = append(df.keepManifests, manifestFile)
}
}
} else {
df.keepManifests = append(df.keepManifests, manifestFile)
}
}

return nil
}

func (df *deleteFiles) processManifests(manifests []iceberg.ManifestFile) ([]iceberg.ManifestFile, error) {
// no post processing
return manifests, nil
}

func (df *deleteFiles) existingManifests() ([]iceberg.ManifestFile, error) {
return df.keepManifests, nil
}

func (df *deleteFiles) deletedEntries() ([]iceberg.ManifestEntry, error) {
return df.removedEntries, nil
}

func (df *deleteFiles) rewriteNeeded() bool {
return df.partialRewrites
}

func (df *deleteFiles) copyWithNewStatus(entry iceberg.ManifestEntry, status iceberg.ManifestEntryStatus) iceberg.ManifestEntry {
snapId := df.base.snapshotID
if status == iceberg.EntryStatusEXISTING {
snapId = entry.SnapshotID()
}

seqNum := entry.SequenceNum()

return iceberg.NewManifestEntry(
status,
&snapId,
&seqNum,
entry.FileSequenceNum(),
entry.DataFile(),
)
}

type snapshotProducer struct {
producerImpl

Expand Down
10 changes: 10 additions & 0 deletions table/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -125,6 +125,16 @@ func (t Table) Append(ctx context.Context, rdr array.RecordReader, snapshotProps
return txn.Commit(ctx)
}

// Delete is a shortcut for NewTransaction().Delete() and then committing the transaction
func (t Table) Delete(ctx context.Context, deleteFilter iceberg.BooleanExpression, snapshotProps iceberg.Properties, caseSensitive bool) (*Table, error) {
txn := t.NewTransaction()
if err := txn.Delete(ctx, deleteFilter, snapshotProps, caseSensitive); err != nil {
return nil, err
}

return txn.Commit(ctx)
}

func (t Table) AllManifests(ctx context.Context) iter.Seq2[iceberg.ManifestFile, error] {
fs, err := t.fsF(ctx)
if err != nil {
Expand Down
65 changes: 65 additions & 0 deletions table/table_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -778,6 +778,71 @@ func (t *TableWritingTestSuite) TestAddFilesReferencedCurrentSnapshotIgnoreDupli
t.Equal([]int32{0, 0, 0}, deleted)
}

func (t *TableWritingTestSuite) TestDelete() {
ident := table.Identifier{"default", "unpartitioned_table_v" + strconv.Itoa(t.formatVersion)}
tbl := t.createTable(ident, t.formatVersion,
*iceberg.UnpartitionedSpec, t.tableSchema)

t.NotNil(tbl)

files := make([]string, 0)
for i := range 5 {
filePath := fmt.Sprintf("%s/unpartitioned/test-%d.parquet", t.location, i)
t.writeParquet(mustFS(t.T(), tbl).(iceio.WriteFileIO), filePath, t.arrTbl)
files = append(files, filePath)
}

tx := tbl.NewTransaction()
t.Require().NoError(tx.AddFiles(t.ctx, files, nil, false))

stagedTbl, err := tx.StagedTable()
t.Require().NoError(err)
t.NotNil(stagedTbl.NameMapping())

t.Equal(stagedTbl.CurrentSnapshot().Summary,
&table.Summary{
Operation: table.OpAppend,
Properties: iceberg.Properties{
"added-data-files": "5",
"added-files-size": "3600",
"added-records": "5",
"total-data-files": "5",
"total-delete-files": "0",
"total-equality-deletes": "0",
"total-files-size": "3600",
"total-position-deletes": "0",
"total-records": "5",
},
})

scan, err := tx.Scan()
t.Require().NoError(err)

contents, err := scan.ToArrowTable(context.Background())
t.Require().NoError(err)
defer contents.Release()

t.EqualValues(5, contents.NumRows())
fmt.Printf("%s", contents.String())

deleteFilter := iceberg.EqualTo(iceberg.Reference("baz"), int32(123))
err = tx.Delete(t.ctx, deleteFilter, nil, true)
t.Require().NoError(err)

stagedTbl, err = tx.StagedTable()
t.Require().NoError(err)

scan, err = stagedTbl.NewTransaction().Scan()
t.Require().NoError(err)

contents, err = scan.ToArrowTable(context.Background())
t.Require().NoError(err)
defer contents.Release()

t.EqualValues(0, contents.NumRows())
fmt.Printf("%s", contents.String())
}

type mockedCatalog struct{}

func (m *mockedCatalog) LoadTable(ctx context.Context, ident table.Identifier, props iceberg.Properties) (*table.Table, error) {
Expand Down
Loading
Loading