Iceberg table maintenance Phase 3: multi-spec compaction, delete handling, and metrics (#8643)
* Add multi-partition-spec compaction and delete-aware compaction (Phase 3) Multi-partition-spec compaction: - Add SpecID to compactionBin struct and group by spec+partition key - Remove the len(specIDs) > 1 skip that blocked spec-evolved tables - Write per-spec manifests in compaction commit using specByID map - Use per-bin PartitionSpec when calling NewDataFileBuilder Delete-aware compaction: - Add ApplyDeletes config (default: true) with readBoolConfig helper - Implement position delete collection (file_path + pos Parquet columns) - Implement equality delete collection (field ID to column mapping) - Update mergeParquetFiles to filter rows via position deletes (binary search) and equality deletes (hash set lookup) - Smart delete manifest carry-forward: drop when all data files compacted - Fix EXISTING/DELETED entries to include sequence numbers Tests for multi-spec bins, delete collection, merge filtering, and end-to-end compaction with position/equality/mixed deletes. * Add structured metrics and per-bin progress to iceberg maintenance - Change return type of all four operations from (string, error) to (string, map[string]int64, error) with structured metric counts (files_merged, snapshots_expired, orphans_removed, duration_ms, etc.) - Add onProgress callback to compactDataFiles for per-bin progress - In Execute, pass progress callback that sends JobProgressUpdate with per-bin stage messages - Accumulate per-operation metrics with dot-prefixed keys (e.g. compact.files_merged) into OutputValues on completion - Update testing_api.go wrappers and integration test call sites - Add tests: TestCompactDataFilesMetrics, TestExpireSnapshotsMetrics, TestExecuteCompletionOutputValues * Address review feedback: group equality deletes by field IDs, use metric constants - Group equality deletes by distinct equality_ids sets so different delete files with different equality columns are handled correctly - Use length-prefixed type-aware encoding in buildEqualityKey to avoid ambiguity between types and collisions from null bytes - Extract metric key strings into package-level constants * Fix buildEqualityKey to use length-prefixed type-aware encoding The previous implementation used plain String() concatenation with null byte separators, which caused type ambiguity (int 123 vs string "123") and separator collisions when values contain null bytes. Now each value is serialized as "kind:length:value" for unambiguous composite keys. This fix was missed in the prior cherry-pick due to a merge conflict. * Address nitpick review comments - Document patchManifestContentToDeletes workaround: explain that iceberg-go WriteManifest cannot create delete manifests, and note the fail-fast validation on pattern match - Document makeTestEntries: note that specID field is ignored and callers should use makeTestEntriesWithSpec for multi-spec testing * fmt * Fix path normalization, manifest threshold, and artifact filename collisions - Normalize file paths in position delete collection and lookup so that absolute S3 URLs and relative paths match correctly - Fix rewriteManifests threshold check to count only data manifests (was including delete manifests in the count and metric) - Add random suffix to artifact filenames in compactDataFiles and rewriteManifests to prevent collisions between concurrent runs - Sort compaction bins by SpecID then PartitionKey for deterministic ordering across specs * Fix pos delete read, deduplicate column resolution, minor cleanups - Remove broken Column() guard in position delete reading that silently defaulted pos to 0; unconditionally extract Int64() instead - Deduplicate column resolution in readEqualityDeleteFile by calling resolveEqualityColIndices instead of inlining the same logic - Add warning log in readBoolConfig for unrecognized string values - Fix CompactDataFiles call site in integration test to capture 3 return values * Advance progress on all bins, deterministic manifest order, assert metrics - Call onProgress for every bin iteration including skipped/failed bins so progress reporting never appears stalled - Sort spec IDs before iterating specEntriesMap to produce deterministic manifest list ordering across runs - Assert expected metric keys in CompactDataFiles integration test --------- Co-authored-by: Copilot <copilot@github.com>
This commit is contained in:
@@ -5,6 +5,7 @@ import (
|
||||
"context"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"io"
|
||||
"net"
|
||||
"path"
|
||||
"sort"
|
||||
@@ -15,6 +16,7 @@ import (
|
||||
|
||||
"github.com/apache/iceberg-go"
|
||||
"github.com/apache/iceberg-go/table"
|
||||
"github.com/parquet-go/parquet-go"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/plugin_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/s3api/s3tables"
|
||||
@@ -420,7 +422,7 @@ func TestExpireSnapshotsExecution(t *testing.T) {
|
||||
Operations: "expire_snapshots",
|
||||
}
|
||||
|
||||
result, err := handler.expireSnapshots(context.Background(), client, setup.BucketName, setup.tablePath(), config)
|
||||
result, _, err := handler.expireSnapshots(context.Background(), client, setup.BucketName, setup.tablePath(), config)
|
||||
if err != nil {
|
||||
t.Fatalf("expireSnapshots failed: %v", err)
|
||||
}
|
||||
@@ -460,7 +462,7 @@ func TestExpireSnapshotsNothingToExpire(t *testing.T) {
|
||||
MaxCommitRetries: 3,
|
||||
}
|
||||
|
||||
result, err := handler.expireSnapshots(context.Background(), client, setup.BucketName, setup.tablePath(), config)
|
||||
result, _, err := handler.expireSnapshots(context.Background(), client, setup.BucketName, setup.tablePath(), config)
|
||||
if err != nil {
|
||||
t.Fatalf("expireSnapshots failed: %v", err)
|
||||
}
|
||||
@@ -508,7 +510,7 @@ func TestRemoveOrphansExecution(t *testing.T) {
|
||||
MaxCommitRetries: 3,
|
||||
}
|
||||
|
||||
result, err := handler.removeOrphans(context.Background(), client, setup.BucketName, setup.tablePath(), config)
|
||||
result, _, err := handler.removeOrphans(context.Background(), client, setup.BucketName, setup.tablePath(), config)
|
||||
if err != nil {
|
||||
t.Fatalf("removeOrphans failed: %v", err)
|
||||
}
|
||||
@@ -550,7 +552,7 @@ func TestRemoveOrphansPreservesReferencedFiles(t *testing.T) {
|
||||
MaxCommitRetries: 3,
|
||||
}
|
||||
|
||||
result, err := handler.removeOrphans(context.Background(), client, setup.BucketName, setup.tablePath(), config)
|
||||
result, _, err := handler.removeOrphans(context.Background(), client, setup.BucketName, setup.tablePath(), config)
|
||||
if err != nil {
|
||||
t.Fatalf("removeOrphans failed: %v", err)
|
||||
}
|
||||
@@ -642,7 +644,7 @@ func TestRewriteManifestsExecution(t *testing.T) {
|
||||
MaxCommitRetries: 3,
|
||||
}
|
||||
|
||||
result, err := handler.rewriteManifests(context.Background(), client, setup.BucketName, setup.tablePath(), config)
|
||||
result, _, err := handler.rewriteManifests(context.Background(), client, setup.BucketName, setup.tablePath(), config)
|
||||
if err != nil {
|
||||
t.Fatalf("rewriteManifests failed: %v", err)
|
||||
}
|
||||
@@ -688,7 +690,7 @@ func TestRewriteManifestsBelowThreshold(t *testing.T) {
|
||||
MaxCommitRetries: 3,
|
||||
}
|
||||
|
||||
result, err := handler.rewriteManifests(context.Background(), client, setup.BucketName, setup.tablePath(), config)
|
||||
result, _, err := handler.rewriteManifests(context.Background(), client, setup.BucketName, setup.tablePath(), config)
|
||||
if err != nil {
|
||||
t.Fatalf("rewriteManifests failed: %v", err)
|
||||
}
|
||||
@@ -762,11 +764,11 @@ func TestFullExecuteFlow(t *testing.T) {
|
||||
var opErr error
|
||||
switch op {
|
||||
case "expire_snapshots":
|
||||
opResult, opErr = handler.expireSnapshots(context.Background(), client, setup.BucketName, setup.tablePath(), workerConfig)
|
||||
opResult, _, opErr = handler.expireSnapshots(context.Background(), client, setup.BucketName, setup.tablePath(), workerConfig)
|
||||
case "remove_orphans":
|
||||
opResult, opErr = handler.removeOrphans(context.Background(), client, setup.BucketName, setup.tablePath(), workerConfig)
|
||||
opResult, _, opErr = handler.removeOrphans(context.Background(), client, setup.BucketName, setup.tablePath(), workerConfig)
|
||||
case "rewrite_manifests":
|
||||
opResult, opErr = handler.rewriteManifests(context.Background(), client, setup.BucketName, setup.tablePath(), workerConfig)
|
||||
opResult, _, opErr = handler.rewriteManifests(context.Background(), client, setup.BucketName, setup.tablePath(), workerConfig)
|
||||
}
|
||||
if opErr != nil {
|
||||
t.Fatalf("operation %s failed: %v", op, opErr)
|
||||
@@ -1015,3 +1017,791 @@ func TestMetadataVersionCAS(t *testing.T) {
|
||||
t.Errorf("expected version 2 after update, got %d", version)
|
||||
}
|
||||
}
|
||||
|
||||
// ---------------------------------------------------------------------------
|
||||
// Avro manifest content patching for tests
|
||||
// ---------------------------------------------------------------------------
|
||||
|
||||
// patchManifestContentToDeletes performs a binary patch on an Avro manifest
|
||||
// file to change the "content" metadata value from "data" to "deletes".
|
||||
// This workaround is needed because iceberg-go's WriteManifest API always
|
||||
// sets content="data" and provides no way to create delete manifests.
|
||||
// The function validates the pattern was found (bytes.Equal check) and fails
|
||||
// fast if not, so breakage from encoding changes is caught immediately.
|
||||
//
|
||||
// In Avro OCF encoding, strings are stored as zigzag-encoded length + bytes.
|
||||
// "content" (7 chars) = \x0e + "content", "data" (4 chars) = \x08 + "data",
|
||||
// "deletes" (7 chars) = \x0e + "deletes".
|
||||
func patchManifestContentToDeletes(t *testing.T, manifestBytes []byte) []byte {
|
||||
t.Helper()
|
||||
|
||||
// Pattern: zigzag(7)="content" zigzag(4)="data"
|
||||
old := append([]byte{0x0e}, []byte("content")...)
|
||||
old = append(old, 0x08)
|
||||
old = append(old, []byte("data")...)
|
||||
|
||||
// Replacement: zigzag(7)="content" zigzag(7)="deletes"
|
||||
new := append([]byte{0x0e}, []byte("content")...)
|
||||
new = append(new, 0x0e)
|
||||
new = append(new, []byte("deletes")...)
|
||||
|
||||
result := bytes.Replace(manifestBytes, old, new, 1)
|
||||
if bytes.Equal(result, manifestBytes) {
|
||||
t.Fatal("patchManifestContentToDeletes: pattern not found in manifest bytes")
|
||||
}
|
||||
return result
|
||||
}
|
||||
|
||||
// ---------------------------------------------------------------------------
|
||||
// End-to-end compaction tests with deletes
|
||||
// ---------------------------------------------------------------------------
|
||||
|
||||
// writeTestParquetFile creates a Parquet file with id/name columns in the fake filer.
|
||||
func writeTestParquetFile(t *testing.T, fs *fakeFilerServer, dir, name string, rows []struct {
|
||||
ID int64
|
||||
Name string
|
||||
}) []byte {
|
||||
t.Helper()
|
||||
type dataRow struct {
|
||||
ID int64 `parquet:"id"`
|
||||
Name string `parquet:"name"`
|
||||
}
|
||||
var buf bytes.Buffer
|
||||
w := parquet.NewWriter(&buf, parquet.SchemaOf(new(dataRow)))
|
||||
for _, r := range rows {
|
||||
if err := w.Write(&dataRow{r.ID, r.Name}); err != nil {
|
||||
t.Fatalf("write parquet row: %v", err)
|
||||
}
|
||||
}
|
||||
if err := w.Close(); err != nil {
|
||||
t.Fatalf("close parquet writer: %v", err)
|
||||
}
|
||||
data := buf.Bytes()
|
||||
fs.putEntry(dir, name, &filer_pb.Entry{
|
||||
Name: name,
|
||||
Content: data,
|
||||
Attributes: &filer_pb.FuseAttributes{Mtime: time.Now().Unix(), FileSize: uint64(len(data))},
|
||||
})
|
||||
return data
|
||||
}
|
||||
|
||||
// populateTableWithDeleteFiles sets up a table with data files and delete manifest(s)
|
||||
// for compaction testing. Returns the table metadata.
|
||||
func populateTableWithDeleteFiles(
|
||||
t *testing.T,
|
||||
fs *fakeFilerServer,
|
||||
setup tableSetup,
|
||||
dataFiles []struct {
|
||||
Name string
|
||||
Rows []struct {
|
||||
ID int64
|
||||
Name string
|
||||
}
|
||||
},
|
||||
posDeleteFiles []struct {
|
||||
Name string
|
||||
Rows []struct {
|
||||
FilePath string
|
||||
Pos int64
|
||||
}
|
||||
},
|
||||
eqDeleteFiles []struct {
|
||||
Name string
|
||||
FieldIDs []int
|
||||
Rows []struct {
|
||||
ID int64
|
||||
Name string
|
||||
}
|
||||
},
|
||||
) table.Metadata {
|
||||
t.Helper()
|
||||
|
||||
schema := newTestSchema()
|
||||
spec := *iceberg.UnpartitionedSpec
|
||||
|
||||
meta, err := table.NewMetadata(schema, &spec, table.UnsortedSortOrder, "s3://"+setup.BucketName+"/"+setup.tablePath(), nil)
|
||||
if err != nil {
|
||||
t.Fatalf("create metadata: %v", err)
|
||||
}
|
||||
|
||||
bucketsPath := s3tables.TablesPath
|
||||
bucketPath := path.Join(bucketsPath, setup.BucketName)
|
||||
nsPath := path.Join(bucketPath, setup.Namespace)
|
||||
tableFilerPath := path.Join(nsPath, setup.TableName)
|
||||
metaDir := path.Join(tableFilerPath, "metadata")
|
||||
dataDir := path.Join(tableFilerPath, "data")
|
||||
|
||||
version := meta.Version()
|
||||
|
||||
// Write data files
|
||||
var dataManifestEntries []iceberg.ManifestEntry
|
||||
for _, df := range dataFiles {
|
||||
data := writeTestParquetFile(t, fs, dataDir, df.Name, df.Rows)
|
||||
dfb, err := iceberg.NewDataFileBuilder(spec, iceberg.EntryContentData, "data/"+df.Name, iceberg.ParquetFile, map[int]any{}, nil, nil, int64(len(df.Rows)), int64(len(data)))
|
||||
if err != nil {
|
||||
t.Fatalf("build data file %s: %v", df.Name, err)
|
||||
}
|
||||
snapID := int64(1)
|
||||
dataManifestEntries = append(dataManifestEntries, iceberg.NewManifestEntry(iceberg.EntryStatusADDED, &snapID, nil, nil, dfb.Build()))
|
||||
}
|
||||
|
||||
// Write data manifest
|
||||
var dataManifestBuf bytes.Buffer
|
||||
dataManifestName := "data-manifest-1.avro"
|
||||
dataMf, err := iceberg.WriteManifest(path.Join("metadata", dataManifestName), &dataManifestBuf, version, spec, schema, 1, dataManifestEntries)
|
||||
if err != nil {
|
||||
t.Fatalf("write data manifest: %v", err)
|
||||
}
|
||||
fs.putEntry(metaDir, dataManifestName, &filer_pb.Entry{
|
||||
Name: dataManifestName, Content: dataManifestBuf.Bytes(),
|
||||
Attributes: &filer_pb.FuseAttributes{Mtime: time.Now().Unix(), FileSize: uint64(dataManifestBuf.Len())},
|
||||
})
|
||||
|
||||
allManifests := []iceberg.ManifestFile{dataMf}
|
||||
|
||||
// Write position delete files and manifests
|
||||
if len(posDeleteFiles) > 0 {
|
||||
var posDeleteEntries []iceberg.ManifestEntry
|
||||
for _, pdf := range posDeleteFiles {
|
||||
type posRow struct {
|
||||
FilePath string `parquet:"file_path"`
|
||||
Pos int64 `parquet:"pos"`
|
||||
}
|
||||
var buf bytes.Buffer
|
||||
w := parquet.NewWriter(&buf, parquet.SchemaOf(new(posRow)))
|
||||
for _, r := range pdf.Rows {
|
||||
if err := w.Write(&posRow{r.FilePath, r.Pos}); err != nil {
|
||||
t.Fatalf("write pos delete: %v", err)
|
||||
}
|
||||
}
|
||||
if err := w.Close(); err != nil {
|
||||
t.Fatalf("close pos delete: %v", err)
|
||||
}
|
||||
fs.putEntry(dataDir, pdf.Name, &filer_pb.Entry{
|
||||
Name: pdf.Name, Content: buf.Bytes(),
|
||||
Attributes: &filer_pb.FuseAttributes{Mtime: time.Now().Unix(), FileSize: uint64(buf.Len())},
|
||||
})
|
||||
dfb, err := iceberg.NewDataFileBuilder(spec, iceberg.EntryContentPosDeletes, "data/"+pdf.Name, iceberg.ParquetFile, map[int]any{}, nil, nil, int64(len(pdf.Rows)), int64(buf.Len()))
|
||||
if err != nil {
|
||||
t.Fatalf("build pos delete file: %v", err)
|
||||
}
|
||||
snapID := int64(1)
|
||||
posDeleteEntries = append(posDeleteEntries, iceberg.NewManifestEntry(iceberg.EntryStatusADDED, &snapID, nil, nil, dfb.Build()))
|
||||
}
|
||||
|
||||
// WriteManifest always sets content="data", so we patch the Avro
|
||||
// metadata to "deletes" and build a ManifestFile with the right content type.
|
||||
var posManifestBuf bytes.Buffer
|
||||
posManifestName := "pos-delete-manifest-1.avro"
|
||||
posManifestPath := path.Join("metadata", posManifestName)
|
||||
_, err := iceberg.WriteManifest(posManifestPath, &posManifestBuf, version, spec, schema, 1, posDeleteEntries)
|
||||
if err != nil {
|
||||
t.Fatalf("write pos delete manifest: %v", err)
|
||||
}
|
||||
patchedBytes := patchManifestContentToDeletes(t, posManifestBuf.Bytes())
|
||||
fs.putEntry(metaDir, posManifestName, &filer_pb.Entry{
|
||||
Name: posManifestName, Content: patchedBytes,
|
||||
Attributes: &filer_pb.FuseAttributes{Mtime: time.Now().Unix(), FileSize: uint64(len(patchedBytes))},
|
||||
})
|
||||
posMf := iceberg.NewManifestFile(version, posManifestPath, int64(len(patchedBytes)), int32(spec.ID()), 1).
|
||||
Content(iceberg.ManifestContentDeletes).
|
||||
AddedFiles(int32(len(posDeleteEntries))).
|
||||
AddedRows(int64(len(posDeleteFiles[0].Rows))).
|
||||
Build()
|
||||
allManifests = append(allManifests, posMf)
|
||||
}
|
||||
|
||||
// Write equality delete files and manifests
|
||||
if len(eqDeleteFiles) > 0 {
|
||||
var eqDeleteEntries []iceberg.ManifestEntry
|
||||
for _, edf := range eqDeleteFiles {
|
||||
type eqRow struct {
|
||||
ID int64 `parquet:"id"`
|
||||
Name string `parquet:"name"`
|
||||
}
|
||||
var buf bytes.Buffer
|
||||
w := parquet.NewWriter(&buf, parquet.SchemaOf(new(eqRow)))
|
||||
for _, r := range edf.Rows {
|
||||
if err := w.Write(&eqRow{r.ID, r.Name}); err != nil {
|
||||
t.Fatalf("write eq delete: %v", err)
|
||||
}
|
||||
}
|
||||
if err := w.Close(); err != nil {
|
||||
t.Fatalf("close eq delete: %v", err)
|
||||
}
|
||||
fs.putEntry(dataDir, edf.Name, &filer_pb.Entry{
|
||||
Name: edf.Name, Content: buf.Bytes(),
|
||||
Attributes: &filer_pb.FuseAttributes{Mtime: time.Now().Unix(), FileSize: uint64(buf.Len())},
|
||||
})
|
||||
dfb, err := iceberg.NewDataFileBuilder(spec, iceberg.EntryContentEqDeletes, "data/"+edf.Name, iceberg.ParquetFile, map[int]any{}, nil, nil, int64(len(edf.Rows)), int64(buf.Len()))
|
||||
if err != nil {
|
||||
t.Fatalf("build eq delete file: %v", err)
|
||||
}
|
||||
dfb.EqualityFieldIDs(edf.FieldIDs)
|
||||
snapID := int64(1)
|
||||
eqDeleteEntries = append(eqDeleteEntries, iceberg.NewManifestEntry(iceberg.EntryStatusADDED, &snapID, nil, nil, dfb.Build()))
|
||||
}
|
||||
|
||||
var eqManifestBuf bytes.Buffer
|
||||
eqManifestName := "eq-delete-manifest-1.avro"
|
||||
eqManifestPath := path.Join("metadata", eqManifestName)
|
||||
_, err := iceberg.WriteManifest(eqManifestPath, &eqManifestBuf, version, spec, schema, 1, eqDeleteEntries)
|
||||
if err != nil {
|
||||
t.Fatalf("write eq delete manifest: %v", err)
|
||||
}
|
||||
patchedBytes := patchManifestContentToDeletes(t, eqManifestBuf.Bytes())
|
||||
fs.putEntry(metaDir, eqManifestName, &filer_pb.Entry{
|
||||
Name: eqManifestName, Content: patchedBytes,
|
||||
Attributes: &filer_pb.FuseAttributes{Mtime: time.Now().Unix(), FileSize: uint64(len(patchedBytes))},
|
||||
})
|
||||
eqMf := iceberg.NewManifestFile(version, eqManifestPath, int64(len(patchedBytes)), int32(spec.ID()), 1).
|
||||
Content(iceberg.ManifestContentDeletes).
|
||||
AddedFiles(int32(len(eqDeleteEntries))).
|
||||
AddedRows(int64(len(eqDeleteFiles[0].Rows))).
|
||||
Build()
|
||||
allManifests = append(allManifests, eqMf)
|
||||
}
|
||||
|
||||
// Write manifest list
|
||||
var mlBuf bytes.Buffer
|
||||
seqNum := int64(1)
|
||||
if err := iceberg.WriteManifestList(version, &mlBuf, 1, nil, &seqNum, 0, allManifests); err != nil {
|
||||
t.Fatalf("write manifest list: %v", err)
|
||||
}
|
||||
fs.putEntry(metaDir, "snap-1.avro", &filer_pb.Entry{
|
||||
Name: "snap-1.avro", Content: mlBuf.Bytes(),
|
||||
Attributes: &filer_pb.FuseAttributes{Mtime: time.Now().Unix(), FileSize: uint64(mlBuf.Len())},
|
||||
})
|
||||
|
||||
// Build final metadata with snapshot
|
||||
now := time.Now().UnixMilli()
|
||||
snap := table.Snapshot{SnapshotID: 1, TimestampMs: now, ManifestList: "metadata/snap-1.avro"}
|
||||
meta = buildTestMetadata(t, []table.Snapshot{snap})
|
||||
|
||||
// Register table structure
|
||||
fullMetadataJSON, _ := json.Marshal(meta)
|
||||
internalMeta := map[string]interface{}{
|
||||
"metadataVersion": 1,
|
||||
"metadata": map[string]interface{}{"fullMetadata": json.RawMessage(fullMetadataJSON)},
|
||||
}
|
||||
xattr, _ := json.Marshal(internalMeta)
|
||||
|
||||
fs.putEntry(bucketsPath, setup.BucketName, &filer_pb.Entry{
|
||||
Name: setup.BucketName, IsDirectory: true,
|
||||
Extended: map[string][]byte{s3tables.ExtendedKeyTableBucket: []byte("true")},
|
||||
})
|
||||
fs.putEntry(bucketPath, setup.Namespace, &filer_pb.Entry{Name: setup.Namespace, IsDirectory: true})
|
||||
fs.putEntry(nsPath, setup.TableName, &filer_pb.Entry{
|
||||
Name: setup.TableName, IsDirectory: true,
|
||||
Extended: map[string][]byte{s3tables.ExtendedKeyMetadata: xattr},
|
||||
})
|
||||
|
||||
return meta
|
||||
}
|
||||
|
||||
func TestCompactDataFilesMetrics(t *testing.T) {
|
||||
fs, client := startFakeFiler(t)
|
||||
|
||||
setup := tableSetup{BucketName: "tb", Namespace: "ns", TableName: "tbl"}
|
||||
populateTableWithDeleteFiles(t, fs, setup,
|
||||
[]struct {
|
||||
Name string
|
||||
Rows []struct {
|
||||
ID int64
|
||||
Name string
|
||||
}
|
||||
}{
|
||||
{"d1.parquet", []struct {
|
||||
ID int64
|
||||
Name string
|
||||
}{{1, "a"}, {2, "b"}}},
|
||||
{"d2.parquet", []struct {
|
||||
ID int64
|
||||
Name string
|
||||
}{{3, "c"}}},
|
||||
},
|
||||
nil, nil,
|
||||
)
|
||||
|
||||
handler := NewHandler(nil)
|
||||
config := Config{
|
||||
TargetFileSizeBytes: 256 * 1024 * 1024,
|
||||
MinInputFiles: 2,
|
||||
MaxCommitRetries: 3,
|
||||
ApplyDeletes: true,
|
||||
}
|
||||
|
||||
// Track progress callbacks
|
||||
var progressCalls []int
|
||||
result, metrics, err := handler.compactDataFiles(context.Background(), client, setup.BucketName, setup.tablePath(), config, func(binIdx, totalBins int) {
|
||||
progressCalls = append(progressCalls, binIdx)
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("compactDataFiles: %v", err)
|
||||
}
|
||||
|
||||
if !strings.Contains(result, "compacted") {
|
||||
t.Errorf("expected compaction result, got %q", result)
|
||||
}
|
||||
|
||||
// Verify metrics
|
||||
if metrics == nil {
|
||||
t.Fatal("expected non-nil metrics")
|
||||
}
|
||||
if metrics[MetricFilesMerged] != 2 {
|
||||
t.Errorf("expected files_merged=2, got %d", metrics[MetricFilesMerged])
|
||||
}
|
||||
if metrics[MetricFilesWritten] != 1 {
|
||||
t.Errorf("expected files_written=1, got %d", metrics[MetricFilesWritten])
|
||||
}
|
||||
if metrics[MetricBins] != 1 {
|
||||
t.Errorf("expected bins=1, got %d", metrics[MetricBins])
|
||||
}
|
||||
if metrics[MetricDurationMs] < 0 {
|
||||
t.Errorf("expected non-negative duration_ms, got %d", metrics[MetricDurationMs])
|
||||
}
|
||||
|
||||
// Verify progress callback was invoked
|
||||
if len(progressCalls) != 1 {
|
||||
t.Errorf("expected 1 progress call, got %d", len(progressCalls))
|
||||
}
|
||||
}
|
||||
|
||||
func TestExpireSnapshotsMetrics(t *testing.T) {
|
||||
fs, client := startFakeFiler(t)
|
||||
|
||||
now := time.Now().UnixMilli()
|
||||
setup := tableSetup{
|
||||
BucketName: "test-bucket",
|
||||
Namespace: "ns",
|
||||
TableName: "tbl",
|
||||
Snapshots: []table.Snapshot{
|
||||
{SnapshotID: 1, TimestampMs: now, ManifestList: "metadata/snap-1.avro"},
|
||||
{SnapshotID: 2, TimestampMs: now + 1, ManifestList: "metadata/snap-2.avro"},
|
||||
{SnapshotID: 3, TimestampMs: now + 2, ManifestList: "metadata/snap-3.avro"},
|
||||
},
|
||||
}
|
||||
populateTable(t, fs, setup)
|
||||
|
||||
handler := NewHandler(nil)
|
||||
config := Config{
|
||||
SnapshotRetentionHours: 0,
|
||||
MaxSnapshotsToKeep: 1,
|
||||
MaxCommitRetries: 3,
|
||||
}
|
||||
|
||||
_, metrics, err := handler.expireSnapshots(context.Background(), client, setup.BucketName, setup.tablePath(), config)
|
||||
if err != nil {
|
||||
t.Fatalf("expireSnapshots: %v", err)
|
||||
}
|
||||
|
||||
if metrics == nil {
|
||||
t.Fatal("expected non-nil metrics")
|
||||
}
|
||||
if metrics[MetricSnapshotsExpired] == 0 {
|
||||
t.Error("expected snapshots_expired > 0")
|
||||
}
|
||||
if metrics[MetricDurationMs] < 0 {
|
||||
t.Errorf("expected non-negative duration_ms, got %d", metrics[MetricDurationMs])
|
||||
}
|
||||
}
|
||||
|
||||
func TestExecuteCompletionOutputValues(t *testing.T) {
|
||||
fs, client := startFakeFiler(t)
|
||||
|
||||
now := time.Now().UnixMilli()
|
||||
setup := tableSetup{
|
||||
BucketName: "test-bucket",
|
||||
Namespace: "ns",
|
||||
TableName: "tbl",
|
||||
Snapshots: []table.Snapshot{
|
||||
{SnapshotID: 1, TimestampMs: now, ManifestList: "metadata/snap-1.avro"},
|
||||
{SnapshotID: 2, TimestampMs: now + 1, ManifestList: "metadata/snap-2.avro"},
|
||||
{SnapshotID: 3, TimestampMs: now + 2, ManifestList: "metadata/snap-3.avro"},
|
||||
},
|
||||
}
|
||||
populateTable(t, fs, setup)
|
||||
|
||||
handler := NewHandler(nil)
|
||||
config := Config{
|
||||
SnapshotRetentionHours: 0,
|
||||
MaxSnapshotsToKeep: 1,
|
||||
MaxCommitRetries: 3,
|
||||
}
|
||||
|
||||
_, metrics, err := handler.expireSnapshots(context.Background(), client, setup.BucketName, setup.tablePath(), config)
|
||||
if err != nil {
|
||||
t.Fatalf("expireSnapshots: %v", err)
|
||||
}
|
||||
|
||||
// Verify metrics have the expected keys
|
||||
if _, ok := metrics[MetricSnapshotsExpired]; !ok {
|
||||
t.Error("expected 'snapshots_expired' key in metrics")
|
||||
}
|
||||
if _, ok := metrics[MetricFilesDeleted]; !ok {
|
||||
t.Error("expected 'files_deleted' key in metrics")
|
||||
}
|
||||
if _, ok := metrics[MetricDurationMs]; !ok {
|
||||
t.Error("expected 'duration_ms' key in metrics")
|
||||
}
|
||||
}
|
||||
|
||||
func TestCompactDataFilesWithPositionDeletes(t *testing.T) {
|
||||
fs, client := startFakeFiler(t)
|
||||
|
||||
setup := tableSetup{BucketName: "tb", Namespace: "ns", TableName: "tbl"}
|
||||
populateTableWithDeleteFiles(t, fs, setup,
|
||||
// 3 small data files (to meet min_input_files=2)
|
||||
[]struct {
|
||||
Name string
|
||||
Rows []struct {
|
||||
ID int64
|
||||
Name string
|
||||
}
|
||||
}{
|
||||
{"d1.parquet", []struct {
|
||||
ID int64
|
||||
Name string
|
||||
}{{1, "alice"}, {2, "bob"}, {3, "charlie"}}},
|
||||
{"d2.parquet", []struct {
|
||||
ID int64
|
||||
Name string
|
||||
}{{4, "dave"}, {5, "eve"}}},
|
||||
},
|
||||
// Position deletes: delete row 1 (bob) from d1
|
||||
[]struct {
|
||||
Name string
|
||||
Rows []struct {
|
||||
FilePath string
|
||||
Pos int64
|
||||
}
|
||||
}{
|
||||
{"pd1.parquet", []struct {
|
||||
FilePath string
|
||||
Pos int64
|
||||
}{{"data/d1.parquet", 1}}},
|
||||
},
|
||||
nil, // no equality deletes
|
||||
)
|
||||
|
||||
handler := NewHandler(nil)
|
||||
config := Config{
|
||||
TargetFileSizeBytes: 256 * 1024 * 1024,
|
||||
MinInputFiles: 2,
|
||||
MaxCommitRetries: 3,
|
||||
ApplyDeletes: true,
|
||||
}
|
||||
|
||||
result, _, err := handler.compactDataFiles(context.Background(), client, setup.BucketName, setup.tablePath(), config, nil)
|
||||
if err != nil {
|
||||
t.Fatalf("compactDataFiles: %v", err)
|
||||
}
|
||||
|
||||
if !strings.Contains(result, "compacted") {
|
||||
t.Errorf("expected compaction result, got %q", result)
|
||||
}
|
||||
t.Logf("result: %s", result)
|
||||
|
||||
// Verify: read the merged output and count rows
|
||||
// The merged file should have 4 rows (5 total - 1 position delete)
|
||||
dataDir := path.Join(s3tables.TablesPath, setup.BucketName, setup.tablePath(), "data")
|
||||
entries := fs.listDir(dataDir)
|
||||
var mergedContent []byte
|
||||
for _, e := range entries {
|
||||
if strings.HasPrefix(e.Name, "compact-") {
|
||||
mergedContent = e.Content
|
||||
break
|
||||
}
|
||||
}
|
||||
if mergedContent == nil {
|
||||
t.Fatal("no merged file found")
|
||||
}
|
||||
|
||||
reader := parquet.NewReader(bytes.NewReader(mergedContent))
|
||||
defer reader.Close()
|
||||
type row struct {
|
||||
ID int64 `parquet:"id"`
|
||||
Name string `parquet:"name"`
|
||||
}
|
||||
var outputRows []row
|
||||
for {
|
||||
var r row
|
||||
if err := reader.Read(&r); err != nil {
|
||||
if err == io.EOF {
|
||||
break
|
||||
}
|
||||
t.Fatalf("read: %v", err)
|
||||
}
|
||||
outputRows = append(outputRows, r)
|
||||
}
|
||||
|
||||
if len(outputRows) != 4 {
|
||||
t.Errorf("expected 4 rows (5 - 1 pos delete), got %d", len(outputRows))
|
||||
}
|
||||
for _, r := range outputRows {
|
||||
if r.Name == "bob" {
|
||||
t.Error("bob should have been deleted by position delete")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestCompactDataFilesWithEqualityDeletes(t *testing.T) {
|
||||
fs, client := startFakeFiler(t)
|
||||
|
||||
setup := tableSetup{BucketName: "tb", Namespace: "ns", TableName: "tbl"}
|
||||
populateTableWithDeleteFiles(t, fs, setup,
|
||||
[]struct {
|
||||
Name string
|
||||
Rows []struct {
|
||||
ID int64
|
||||
Name string
|
||||
}
|
||||
}{
|
||||
{"d1.parquet", []struct {
|
||||
ID int64
|
||||
Name string
|
||||
}{{1, "alice"}, {2, "bob"}}},
|
||||
{"d2.parquet", []struct {
|
||||
ID int64
|
||||
Name string
|
||||
}{{3, "charlie"}, {4, "dave"}}},
|
||||
},
|
||||
nil, // no position deletes
|
||||
// Equality deletes: delete rows where name="bob" or name="dave"
|
||||
[]struct {
|
||||
Name string
|
||||
FieldIDs []int
|
||||
Rows []struct {
|
||||
ID int64
|
||||
Name string
|
||||
}
|
||||
}{
|
||||
{"ed1.parquet", []int{2}, []struct {
|
||||
ID int64
|
||||
Name string
|
||||
}{{0, "bob"}, {0, "dave"}}},
|
||||
},
|
||||
)
|
||||
|
||||
handler := NewHandler(nil)
|
||||
config := Config{
|
||||
TargetFileSizeBytes: 256 * 1024 * 1024,
|
||||
MinInputFiles: 2,
|
||||
MaxCommitRetries: 3,
|
||||
ApplyDeletes: true,
|
||||
}
|
||||
|
||||
result, _, err := handler.compactDataFiles(context.Background(), client, setup.BucketName, setup.tablePath(), config, nil)
|
||||
if err != nil {
|
||||
t.Fatalf("compactDataFiles: %v", err)
|
||||
}
|
||||
|
||||
if !strings.Contains(result, "compacted") {
|
||||
t.Errorf("expected compaction result, got %q", result)
|
||||
}
|
||||
t.Logf("result: %s", result)
|
||||
|
||||
// Verify merged output
|
||||
dataDir := path.Join(s3tables.TablesPath, setup.BucketName, setup.tablePath(), "data")
|
||||
entries := fs.listDir(dataDir)
|
||||
var mergedContent []byte
|
||||
for _, e := range entries {
|
||||
if strings.HasPrefix(e.Name, "compact-") {
|
||||
mergedContent = e.Content
|
||||
break
|
||||
}
|
||||
}
|
||||
if mergedContent == nil {
|
||||
t.Fatal("no merged file found")
|
||||
}
|
||||
|
||||
reader := parquet.NewReader(bytes.NewReader(mergedContent))
|
||||
defer reader.Close()
|
||||
type row struct {
|
||||
ID int64 `parquet:"id"`
|
||||
Name string `parquet:"name"`
|
||||
}
|
||||
var outputRows []row
|
||||
for {
|
||||
var r row
|
||||
if err := reader.Read(&r); err != nil {
|
||||
if err == io.EOF {
|
||||
break
|
||||
}
|
||||
t.Fatalf("read: %v", err)
|
||||
}
|
||||
outputRows = append(outputRows, r)
|
||||
}
|
||||
|
||||
if len(outputRows) != 2 {
|
||||
t.Errorf("expected 2 rows (4 - 2 eq deletes), got %d", len(outputRows))
|
||||
}
|
||||
for _, r := range outputRows {
|
||||
if r.Name == "bob" || r.Name == "dave" {
|
||||
t.Errorf("row %q should have been deleted by equality delete", r.Name)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestCompactDataFilesApplyDeletesDisabled(t *testing.T) {
|
||||
fs, client := startFakeFiler(t)
|
||||
|
||||
setup := tableSetup{BucketName: "tb", Namespace: "ns", TableName: "tbl"}
|
||||
populateTableWithDeleteFiles(t, fs, setup,
|
||||
[]struct {
|
||||
Name string
|
||||
Rows []struct {
|
||||
ID int64
|
||||
Name string
|
||||
}
|
||||
}{
|
||||
{"d1.parquet", []struct {
|
||||
ID int64
|
||||
Name string
|
||||
}{{1, "a"}, {2, "b"}}},
|
||||
{"d2.parquet", []struct {
|
||||
ID int64
|
||||
Name string
|
||||
}{{3, "c"}}},
|
||||
},
|
||||
[]struct {
|
||||
Name string
|
||||
Rows []struct {
|
||||
FilePath string
|
||||
Pos int64
|
||||
}
|
||||
}{
|
||||
{"pd1.parquet", []struct {
|
||||
FilePath string
|
||||
Pos int64
|
||||
}{{"data/d1.parquet", 0}}},
|
||||
},
|
||||
nil,
|
||||
)
|
||||
|
||||
handler := NewHandler(nil)
|
||||
config := Config{
|
||||
TargetFileSizeBytes: 256 * 1024 * 1024,
|
||||
MinInputFiles: 2,
|
||||
MaxCommitRetries: 3,
|
||||
ApplyDeletes: false, // disabled
|
||||
}
|
||||
|
||||
result, _, err := handler.compactDataFiles(context.Background(), client, setup.BucketName, setup.tablePath(), config, nil)
|
||||
if err != nil {
|
||||
t.Fatalf("compactDataFiles: %v", err)
|
||||
}
|
||||
|
||||
if !strings.Contains(result, "skipped") {
|
||||
t.Errorf("expected skip when apply_deletes=false, got %q", result)
|
||||
}
|
||||
}
|
||||
|
||||
func TestCompactDataFilesWithMixedDeletes(t *testing.T) {
|
||||
fs, client := startFakeFiler(t)
|
||||
|
||||
setup := tableSetup{BucketName: "tb", Namespace: "ns", TableName: "tbl"}
|
||||
populateTableWithDeleteFiles(t, fs, setup,
|
||||
[]struct {
|
||||
Name string
|
||||
Rows []struct {
|
||||
ID int64
|
||||
Name string
|
||||
}
|
||||
}{
|
||||
{"d1.parquet", []struct {
|
||||
ID int64
|
||||
Name string
|
||||
}{{1, "alice"}, {2, "bob"}, {3, "charlie"}}},
|
||||
{"d2.parquet", []struct {
|
||||
ID int64
|
||||
Name string
|
||||
}{{4, "dave"}, {5, "eve"}}},
|
||||
},
|
||||
// Position delete: row 0 (alice) from d1
|
||||
[]struct {
|
||||
Name string
|
||||
Rows []struct {
|
||||
FilePath string
|
||||
Pos int64
|
||||
}
|
||||
}{
|
||||
{"pd1.parquet", []struct {
|
||||
FilePath string
|
||||
Pos int64
|
||||
}{{"data/d1.parquet", 0}}},
|
||||
},
|
||||
// Equality delete: name="eve"
|
||||
[]struct {
|
||||
Name string
|
||||
FieldIDs []int
|
||||
Rows []struct {
|
||||
ID int64
|
||||
Name string
|
||||
}
|
||||
}{
|
||||
{"ed1.parquet", []int{2}, []struct {
|
||||
ID int64
|
||||
Name string
|
||||
}{{0, "eve"}}},
|
||||
},
|
||||
)
|
||||
|
||||
handler := NewHandler(nil)
|
||||
config := Config{
|
||||
TargetFileSizeBytes: 256 * 1024 * 1024,
|
||||
MinInputFiles: 2,
|
||||
MaxCommitRetries: 3,
|
||||
ApplyDeletes: true,
|
||||
}
|
||||
|
||||
result, _, err := handler.compactDataFiles(context.Background(), client, setup.BucketName, setup.tablePath(), config, nil)
|
||||
if err != nil {
|
||||
t.Fatalf("compactDataFiles: %v", err)
|
||||
}
|
||||
|
||||
if !strings.Contains(result, "compacted") {
|
||||
t.Errorf("expected compaction, got %q", result)
|
||||
}
|
||||
|
||||
// Verify: 5 total - 1 pos delete (alice) - 1 eq delete (eve) = 3 rows
|
||||
dataDir := path.Join(s3tables.TablesPath, setup.BucketName, setup.tablePath(), "data")
|
||||
entries := fs.listDir(dataDir)
|
||||
var mergedContent []byte
|
||||
for _, e := range entries {
|
||||
if strings.HasPrefix(e.Name, "compact-") {
|
||||
mergedContent = e.Content
|
||||
break
|
||||
}
|
||||
}
|
||||
if mergedContent == nil {
|
||||
t.Fatal("no merged file found")
|
||||
}
|
||||
|
||||
reader := parquet.NewReader(bytes.NewReader(mergedContent))
|
||||
defer reader.Close()
|
||||
type row struct {
|
||||
ID int64 `parquet:"id"`
|
||||
Name string `parquet:"name"`
|
||||
}
|
||||
var outputRows []row
|
||||
for {
|
||||
var r row
|
||||
if err := reader.Read(&r); err != nil {
|
||||
if err == io.EOF {
|
||||
break
|
||||
}
|
||||
t.Fatalf("read: %v", err)
|
||||
}
|
||||
outputRows = append(outputRows, r)
|
||||
}
|
||||
|
||||
if len(outputRows) != 3 {
|
||||
t.Errorf("expected 3 rows (5 - 1 pos - 1 eq), got %d", len(outputRows))
|
||||
}
|
||||
for _, r := range outputRows {
|
||||
if r.Name == "alice" || r.Name == "eve" {
|
||||
t.Errorf("%q should have been deleted", r.Name)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user