* Add iceberg_maintenance plugin worker handler (Phase 1) Implement automated Iceberg table maintenance as a new plugin worker job type. The handler scans S3 table buckets for tables needing maintenance and executes operations in the correct Iceberg order: expire snapshots, remove orphan files, and rewrite manifests. Co-Authored-By: Claude Opus 4.6 <noreply@anthropic.com> * Add data file compaction to iceberg maintenance handler (Phase 2) Implement bin-packing compaction for small Parquet data files: - Enumerate data files from manifests, group by partition - Merge small files using parquet-go (read rows, write merged output) - Create new manifest with ADDED/DELETED/EXISTING entries - Commit new snapshot with compaction metadata Add 'compact' operation to maintenance order (runs before expire_snapshots), configurable via target_file_size_bytes and min_input_files thresholds. Co-Authored-By: Claude Opus 4.6 <noreply@anthropic.com> * Fix memory exhaustion in mergeParquetFiles by processing files sequentially Previously all source Parquet files were loaded into memory simultaneously, risking OOM when a compaction bin contained many small files. Now each file is loaded, its rows are streamed into the output writer, and its data is released before the next file is loaded — keeping peak memory proportional to one input file plus the output buffer. * Validate bucket/namespace/table names against path traversal Reject names containing '..', '/', or '\' in Execute to prevent directory traversal via crafted job parameters. * Add filer address failover in iceberg maintenance handler Try each filer address from cluster context in order instead of only using the first one. This improves resilience when the primary filer is temporarily unreachable. * Add separate MinManifestsToRewrite config for manifest rewrite threshold The rewrite_manifests operation was reusing MinInputFiles (meant for compaction bin file counts) as its manifest count threshold. Add a dedicated MinManifestsToRewrite field with its own config UI section and default value (5) so the two thresholds can be tuned independently. * Fix risky mtime fallback in orphan removal that could delete new files When entry.Attributes is nil, mtime defaulted to Unix epoch (1970), which would always be older than the safety threshold, causing the file to be treated as eligible for deletion. Skip entries with nil Attributes instead, matching the safer logic in operations.go. * Fix undefined function references in iceberg_maintenance_handler.go Use the exported function names (ShouldSkipDetectionByInterval, BuildDetectorActivity, BuildExecutorActivity) matching their definitions in vacuum_handler.go. * Remove duplicated iceberg maintenance handler in favor of iceberg/ subpackage The IcebergMaintenanceHandler and its compaction code in the parent pluginworker package duplicated the logic already present in the iceberg/ subpackage (which self-registers via init()). The old code lacked stale-plan guards, proper path normalization, CAS-based xattr updates, and error-returning parseOperations. Since the registry pattern (default "all") makes the old handler unreachable, remove it entirely. All functionality is provided by iceberg.Handler with the reviewed improvements. * Fix MinManifestsToRewrite clamping to match UI minimum of 2 The clamp reset values below 2 to the default of 5, contradicting the UI's advertised MinValue of 2. Clamp to 2 instead. * Sort entries by size descending in splitOversizedBin for better packing Entries were processed in insertion order which is non-deterministic from map iteration. Sorting largest-first before the splitting loop improves bin packing efficiency by filling bins more evenly. * Add context cancellation check to drainReader loop The row-streaming loop in drainReader did not check ctx between iterations, making long compaction merges uncancellable. Check ctx.Done() at the top of each iteration. * Fix splitOversizedBin to always respect targetSize limit The minFiles check in the split condition allowed bins to grow past targetSize when they had fewer than minFiles entries, defeating the OOM protection. Now bins always split at targetSize, and a trailing runt with fewer than minFiles entries is merged into the previous bin. * Add integration tests for iceberg table maintenance plugin worker Tests start a real weed mini cluster, create S3 buckets and Iceberg table metadata via filer gRPC, then exercise the iceberg.Handler operations (ExpireSnapshots, RemoveOrphans, RewriteManifests) against the live filer. A full maintenance cycle test runs all operations in sequence and verifies metadata consistency. Also adds exported method wrappers (testing_api.go) so the integration test package can call the unexported handler methods. * Fix splitOversizedBin dropping files and add source path to drainReader errors The runt-merge step could leave leading bins with fewer than minFiles entries (e.g. [80,80,10,10] with targetSize=100, minFiles=2 would drop the first 80-byte file). Replace the filter-based approach with an iterative merge that folds any sub-minFiles bin into its smallest neighbor, preserving all eligible files. Also add the source file path to drainReader error messages so callers can identify which Parquet file caused a read/write failure. * Harden integration test error handling - s3put: fail immediately on HTTP 4xx/5xx instead of logging and continuing - lookupEntry: distinguish NotFound (return nil) from unexpected RPC errors (fail the test) - writeOrphan and orphan creation in FullMaintenanceCycle: check CreateEntryResponse.Error in addition to the RPC error * go fmt --------- Co-authored-by: Copilot <copilot@github.com> Co-authored-by: Claude Opus 4.6 <noreply@anthropic.com>
605 lines
20 KiB
Go
605 lines
20 KiB
Go
package iceberg
|
||
|
||
import (
|
||
"bytes"
|
||
"context"
|
||
"encoding/json"
|
||
"errors"
|
||
"fmt"
|
||
"math/rand/v2"
|
||
"path"
|
||
"sort"
|
||
"strings"
|
||
"time"
|
||
|
||
"github.com/apache/iceberg-go"
|
||
"github.com/apache/iceberg-go/table"
|
||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
|
||
"github.com/seaweedfs/seaweedfs/weed/s3api/s3tables"
|
||
)
|
||
|
||
// errStalePlan is returned by a commit mutation when the table head has
|
||
// advanced since planning. The caller should not retry the same plan.
|
||
var errStalePlan = errors.New("stale plan: table head changed since planning")
|
||
|
||
// errMetadataVersionConflict is returned when the xattr update detects a
|
||
// concurrent metadata version change (compare-and-swap failure).
|
||
var errMetadataVersionConflict = errors.New("metadata version conflict")
|
||
|
||
// ---------------------------------------------------------------------------
|
||
// Operation: Expire Snapshots
|
||
// ---------------------------------------------------------------------------
|
||
|
||
// expireSnapshots removes old snapshots from the table metadata and cleans up
|
||
// their manifest list files.
|
||
func (h *Handler) expireSnapshots(
|
||
ctx context.Context,
|
||
filerClient filer_pb.SeaweedFilerClient,
|
||
bucketName, tablePath string,
|
||
config Config,
|
||
) (string, error) {
|
||
// Load current metadata
|
||
meta, metadataFileName, err := loadCurrentMetadata(ctx, filerClient, bucketName, tablePath)
|
||
if err != nil {
|
||
return "", fmt.Errorf("load metadata: %w", err)
|
||
}
|
||
|
||
snapshots := meta.Snapshots()
|
||
if len(snapshots) == 0 {
|
||
return "no snapshots", nil
|
||
}
|
||
|
||
// Determine which snapshots to expire
|
||
currentSnap := meta.CurrentSnapshot()
|
||
var currentSnapID int64
|
||
if currentSnap != nil {
|
||
currentSnapID = currentSnap.SnapshotID
|
||
}
|
||
|
||
retentionMs := config.SnapshotRetentionHours * 3600 * 1000
|
||
nowMs := time.Now().UnixMilli()
|
||
|
||
// Sort snapshots by timestamp descending (most recent first) so that
|
||
// the keep-count logic always preserves the newest snapshots.
|
||
sorted := make([]table.Snapshot, len(snapshots))
|
||
copy(sorted, snapshots)
|
||
sort.Slice(sorted, func(i, j int) bool {
|
||
return sorted[i].TimestampMs > sorted[j].TimestampMs
|
||
})
|
||
|
||
// Walk from newest to oldest. The current snapshot is always kept.
|
||
// Among the remaining, keep up to MaxSnapshotsToKeep-1 (since current
|
||
// counts toward the quota). Expire the rest only if they exceed the
|
||
// retention window; snapshots within the window are kept regardless.
|
||
var toExpire []int64
|
||
var kept int64
|
||
for _, snap := range sorted {
|
||
if snap.SnapshotID == currentSnapID {
|
||
kept++
|
||
continue
|
||
}
|
||
age := nowMs - snap.TimestampMs
|
||
if kept < config.MaxSnapshotsToKeep {
|
||
kept++
|
||
continue
|
||
}
|
||
if age > retentionMs {
|
||
toExpire = append(toExpire, snap.SnapshotID)
|
||
} else {
|
||
kept++
|
||
}
|
||
}
|
||
|
||
if len(toExpire) == 0 {
|
||
return "no snapshots expired", nil
|
||
}
|
||
|
||
// Split snapshots into expired and kept sets
|
||
expireSet := make(map[int64]struct{}, len(toExpire))
|
||
for _, id := range toExpire {
|
||
expireSet[id] = struct{}{}
|
||
}
|
||
var expiredSnaps, keptSnaps []table.Snapshot
|
||
for _, snap := range sorted {
|
||
if _, ok := expireSet[snap.SnapshotID]; ok {
|
||
expiredSnaps = append(expiredSnaps, snap)
|
||
} else {
|
||
keptSnaps = append(keptSnaps, snap)
|
||
}
|
||
}
|
||
|
||
// Collect all files referenced by each set before modifying metadata.
|
||
// This lets us determine which files become unreferenced.
|
||
expiredFiles, err := collectSnapshotFiles(ctx, filerClient, bucketName, tablePath, expiredSnaps)
|
||
if err != nil {
|
||
return "", fmt.Errorf("collect expired snapshot files: %w", err)
|
||
}
|
||
keptFiles, err := collectSnapshotFiles(ctx, filerClient, bucketName, tablePath, keptSnaps)
|
||
if err != nil {
|
||
return "", fmt.Errorf("collect kept snapshot files: %w", err)
|
||
}
|
||
|
||
// Normalize kept file paths for consistent comparison
|
||
normalizedKept := make(map[string]struct{}, len(keptFiles))
|
||
for f := range keptFiles {
|
||
normalizedKept[normalizeIcebergPath(f, bucketName, tablePath)] = struct{}{}
|
||
}
|
||
|
||
// Use MetadataBuilder to remove snapshots and create new metadata
|
||
err = h.commitWithRetry(ctx, filerClient, bucketName, tablePath, metadataFileName, config, func(currentMeta table.Metadata, builder *table.MetadataBuilder) error {
|
||
// Guard: verify table head hasn't changed since we planned
|
||
cs := currentMeta.CurrentSnapshot()
|
||
if (cs == nil) != (currentSnapID == 0) || (cs != nil && cs.SnapshotID != currentSnapID) {
|
||
return errStalePlan
|
||
}
|
||
return builder.RemoveSnapshots(toExpire)
|
||
})
|
||
if err != nil {
|
||
return "", fmt.Errorf("commit snapshot expiration: %w", err)
|
||
}
|
||
|
||
// Delete files exclusively referenced by expired snapshots (best-effort)
|
||
tableBasePath := path.Join(s3tables.TablesPath, bucketName, tablePath)
|
||
deletedCount := 0
|
||
for filePath := range expiredFiles {
|
||
normalized := normalizeIcebergPath(filePath, bucketName, tablePath)
|
||
if _, stillReferenced := normalizedKept[normalized]; stillReferenced {
|
||
continue
|
||
}
|
||
dir := path.Join(tableBasePath, path.Dir(normalized))
|
||
fileName := path.Base(normalized)
|
||
if delErr := deleteFilerFile(ctx, filerClient, dir, fileName); delErr != nil {
|
||
glog.Warningf("iceberg maintenance: failed to delete unreferenced file %s: %v", filePath, delErr)
|
||
} else {
|
||
deletedCount++
|
||
}
|
||
}
|
||
|
||
return fmt.Sprintf("expired %d snapshot(s), deleted %d unreferenced file(s)", len(toExpire), deletedCount), nil
|
||
}
|
||
|
||
// collectSnapshotFiles returns all file paths (manifest lists, manifest files,
|
||
// data files) referenced by the given snapshots. It returns an error if any
|
||
// manifest list or manifest cannot be read/parsed, to prevent delete decisions
|
||
// based on incomplete reference data.
|
||
func collectSnapshotFiles(
|
||
ctx context.Context,
|
||
filerClient filer_pb.SeaweedFilerClient,
|
||
bucketName, tablePath string,
|
||
snapshots []table.Snapshot,
|
||
) (map[string]struct{}, error) {
|
||
files := make(map[string]struct{})
|
||
for _, snap := range snapshots {
|
||
if snap.ManifestList == "" {
|
||
continue
|
||
}
|
||
files[snap.ManifestList] = struct{}{}
|
||
|
||
manifestListData, err := loadFileByIcebergPath(ctx, filerClient, bucketName, tablePath, snap.ManifestList)
|
||
if err != nil {
|
||
return nil, fmt.Errorf("read manifest list %s: %w", snap.ManifestList, err)
|
||
}
|
||
manifests, err := iceberg.ReadManifestList(bytes.NewReader(manifestListData))
|
||
if err != nil {
|
||
return nil, fmt.Errorf("parse manifest list %s: %w", snap.ManifestList, err)
|
||
}
|
||
|
||
for _, mf := range manifests {
|
||
files[mf.FilePath()] = struct{}{}
|
||
|
||
manifestData, err := loadFileByIcebergPath(ctx, filerClient, bucketName, tablePath, mf.FilePath())
|
||
if err != nil {
|
||
return nil, fmt.Errorf("read manifest %s: %w", mf.FilePath(), err)
|
||
}
|
||
entries, err := iceberg.ReadManifest(mf, bytes.NewReader(manifestData), false)
|
||
if err != nil {
|
||
return nil, fmt.Errorf("parse manifest %s: %w", mf.FilePath(), err)
|
||
}
|
||
for _, entry := range entries {
|
||
files[entry.DataFile().FilePath()] = struct{}{}
|
||
}
|
||
}
|
||
}
|
||
return files, nil
|
||
}
|
||
|
||
// ---------------------------------------------------------------------------
|
||
// Operation: Remove Orphans
|
||
// ---------------------------------------------------------------------------
|
||
|
||
// removeOrphans finds and deletes unreferenced files from the table's
|
||
// metadata/ and data/ directories.
|
||
func (h *Handler) removeOrphans(
|
||
ctx context.Context,
|
||
filerClient filer_pb.SeaweedFilerClient,
|
||
bucketName, tablePath string,
|
||
config Config,
|
||
) (string, error) {
|
||
// Load current metadata
|
||
meta, metadataFileName, err := loadCurrentMetadata(ctx, filerClient, bucketName, tablePath)
|
||
if err != nil {
|
||
return "", fmt.Errorf("load metadata: %w", err)
|
||
}
|
||
|
||
// Collect all referenced files from all snapshots
|
||
referencedFiles, err := collectSnapshotFiles(ctx, filerClient, bucketName, tablePath, meta.Snapshots())
|
||
if err != nil {
|
||
return "", fmt.Errorf("collect referenced files: %w", err)
|
||
}
|
||
|
||
// Reference the active metadata file so it is not treated as orphan
|
||
referencedFiles[path.Join("metadata", metadataFileName)] = struct{}{}
|
||
|
||
// Also reference the current metadata files
|
||
for mle := range meta.PreviousFiles() {
|
||
referencedFiles[mle.MetadataFile] = struct{}{}
|
||
}
|
||
|
||
// Precompute a normalized lookup set so orphan checks are O(1) per file.
|
||
normalizedRefs := make(map[string]struct{}, len(referencedFiles))
|
||
for ref := range referencedFiles {
|
||
normalizedRefs[ref] = struct{}{}
|
||
normalizedRefs[normalizeIcebergPath(ref, bucketName, tablePath)] = struct{}{}
|
||
}
|
||
|
||
// List actual files on filer in metadata/ and data/ directories
|
||
tableBasePath := path.Join(s3tables.TablesPath, bucketName, tablePath)
|
||
safetyThreshold := time.Now().Add(-time.Duration(config.OrphanOlderThanHours) * time.Hour)
|
||
orphanCount := 0
|
||
|
||
for _, subdir := range []string{"metadata", "data"} {
|
||
dirPath := path.Join(tableBasePath, subdir)
|
||
fileEntries, err := walkFilerEntries(ctx, filerClient, dirPath)
|
||
if err != nil {
|
||
glog.V(2).Infof("iceberg maintenance: cannot walk %s: %v", dirPath, err)
|
||
continue
|
||
}
|
||
|
||
for _, fe := range fileEntries {
|
||
entry := fe.Entry
|
||
// Build relative path from the table base (e.g. "data/region=us/file.parquet")
|
||
fullPath := path.Join(fe.Dir, entry.Name)
|
||
relPath := strings.TrimPrefix(fullPath, tableBasePath+"/")
|
||
|
||
_, isReferenced := normalizedRefs[relPath]
|
||
|
||
if isReferenced {
|
||
continue
|
||
}
|
||
|
||
// Check safety window — skip entries with unknown age
|
||
if entry.Attributes == nil {
|
||
continue
|
||
}
|
||
mtime := time.Unix(entry.Attributes.Mtime, 0)
|
||
if mtime.After(safetyThreshold) {
|
||
continue
|
||
}
|
||
|
||
// Delete orphan
|
||
if delErr := deleteFilerFile(ctx, filerClient, fe.Dir, entry.Name); delErr != nil {
|
||
glog.Warningf("iceberg maintenance: failed to delete orphan %s/%s: %v", fe.Dir, entry.Name, delErr)
|
||
} else {
|
||
orphanCount++
|
||
}
|
||
}
|
||
}
|
||
|
||
return fmt.Sprintf("removed %d orphan file(s)", orphanCount), nil
|
||
}
|
||
|
||
// ---------------------------------------------------------------------------
|
||
// Operation: Rewrite Manifests
|
||
// ---------------------------------------------------------------------------
|
||
|
||
// rewriteManifests merges small manifests into fewer, larger ones.
|
||
func (h *Handler) rewriteManifests(
|
||
ctx context.Context,
|
||
filerClient filer_pb.SeaweedFilerClient,
|
||
bucketName, tablePath string,
|
||
config Config,
|
||
) (string, error) {
|
||
// Load current metadata
|
||
meta, metadataFileName, err := loadCurrentMetadata(ctx, filerClient, bucketName, tablePath)
|
||
if err != nil {
|
||
return "", fmt.Errorf("load metadata: %w", err)
|
||
}
|
||
|
||
currentSnap := meta.CurrentSnapshot()
|
||
if currentSnap == nil || currentSnap.ManifestList == "" {
|
||
return "no current snapshot", nil
|
||
}
|
||
|
||
// Read manifest list
|
||
manifestListData, err := loadFileByIcebergPath(ctx, filerClient, bucketName, tablePath, currentSnap.ManifestList)
|
||
if err != nil {
|
||
return "", fmt.Errorf("read manifest list: %w", err)
|
||
}
|
||
|
||
manifests, err := iceberg.ReadManifestList(bytes.NewReader(manifestListData))
|
||
if err != nil {
|
||
return "", fmt.Errorf("parse manifest list: %w", err)
|
||
}
|
||
|
||
if int64(len(manifests)) < config.MinManifestsToRewrite {
|
||
return fmt.Sprintf("only %d manifests, below threshold of %d", len(manifests), config.MinManifestsToRewrite), nil
|
||
}
|
||
|
||
// Collect all entries from data manifests, grouped by partition spec ID
|
||
// so we write one merged manifest per spec (required for spec-evolved tables).
|
||
type specEntries struct {
|
||
specID int32
|
||
spec iceberg.PartitionSpec
|
||
entries []iceberg.ManifestEntry
|
||
}
|
||
specMap := make(map[int32]*specEntries)
|
||
|
||
// Build a lookup from spec ID to PartitionSpec
|
||
specByID := make(map[int]iceberg.PartitionSpec)
|
||
for _, ps := range meta.PartitionSpecs() {
|
||
specByID[ps.ID()] = ps
|
||
}
|
||
|
||
for _, mf := range manifests {
|
||
if mf.ManifestContent() != iceberg.ManifestContentData {
|
||
continue
|
||
}
|
||
manifestData, err := loadFileByIcebergPath(ctx, filerClient, bucketName, tablePath, mf.FilePath())
|
||
if err != nil {
|
||
return "", fmt.Errorf("read manifest %s: %w", mf.FilePath(), err)
|
||
}
|
||
entries, err := iceberg.ReadManifest(mf, bytes.NewReader(manifestData), true)
|
||
if err != nil {
|
||
return "", fmt.Errorf("parse manifest %s: %w", mf.FilePath(), err)
|
||
}
|
||
|
||
sid := mf.PartitionSpecID()
|
||
se, ok := specMap[sid]
|
||
if !ok {
|
||
ps, found := specByID[int(sid)]
|
||
if !found {
|
||
return "", fmt.Errorf("partition spec %d not found in table metadata", sid)
|
||
}
|
||
se = &specEntries{specID: sid, spec: ps}
|
||
specMap[sid] = se
|
||
}
|
||
se.entries = append(se.entries, entries...)
|
||
}
|
||
|
||
if len(specMap) == 0 {
|
||
return "no data entries to rewrite", nil
|
||
}
|
||
|
||
schema := meta.CurrentSchema()
|
||
version := meta.Version()
|
||
snapshotID := currentSnap.SnapshotID
|
||
newSnapshotID := time.Now().UnixMilli()
|
||
newSeqNum := currentSnap.SequenceNumber + 1
|
||
metaDir := path.Join(s3tables.TablesPath, bucketName, tablePath, "metadata")
|
||
|
||
// Track written artifacts so we can clean them up if the commit fails.
|
||
type artifact struct {
|
||
dir, fileName string
|
||
}
|
||
var writtenArtifacts []artifact
|
||
committed := false
|
||
|
||
defer func() {
|
||
if committed || len(writtenArtifacts) == 0 {
|
||
return
|
||
}
|
||
cleanupCtx, cancel := context.WithTimeout(context.Background(), 30*time.Second)
|
||
defer cancel()
|
||
for _, a := range writtenArtifacts {
|
||
if err := deleteFilerFile(cleanupCtx, filerClient, a.dir, a.fileName); err != nil {
|
||
glog.Warningf("iceberg rewrite-manifests: failed to clean up artifact %s/%s: %v", a.dir, a.fileName, err)
|
||
}
|
||
}
|
||
}()
|
||
|
||
// Write one merged manifest per partition spec
|
||
var newManifests []iceberg.ManifestFile
|
||
totalEntries := 0
|
||
for _, se := range specMap {
|
||
totalEntries += len(se.entries)
|
||
manifestFileName := fmt.Sprintf("merged-%d-spec%d-%d.avro", newSnapshotID, se.specID, time.Now().UnixMilli())
|
||
manifestPath := path.Join("metadata", manifestFileName)
|
||
|
||
var manifestBuf bytes.Buffer
|
||
mergedManifest, err := iceberg.WriteManifest(
|
||
manifestPath,
|
||
&manifestBuf,
|
||
version,
|
||
se.spec,
|
||
schema,
|
||
newSnapshotID,
|
||
se.entries,
|
||
)
|
||
if err != nil {
|
||
return "", fmt.Errorf("write merged manifest for spec %d: %w", se.specID, err)
|
||
}
|
||
|
||
if err := saveFilerFile(ctx, filerClient, metaDir, manifestFileName, manifestBuf.Bytes()); err != nil {
|
||
return "", fmt.Errorf("save merged manifest for spec %d: %w", se.specID, err)
|
||
}
|
||
writtenArtifacts = append(writtenArtifacts, artifact{dir: metaDir, fileName: manifestFileName})
|
||
newManifests = append(newManifests, mergedManifest)
|
||
}
|
||
|
||
// Include any delete manifests that were not rewritten
|
||
for _, mf := range manifests {
|
||
if mf.ManifestContent() != iceberg.ManifestContentData {
|
||
newManifests = append(newManifests, mf)
|
||
}
|
||
}
|
||
|
||
var manifestListBuf bytes.Buffer
|
||
err = iceberg.WriteManifestList(version, &manifestListBuf, newSnapshotID, &snapshotID, &newSeqNum, 0, newManifests)
|
||
if err != nil {
|
||
return "", fmt.Errorf("write manifest list: %w", err)
|
||
}
|
||
|
||
// Save new manifest list
|
||
manifestListFileName := fmt.Sprintf("snap-%d-%d.avro", newSnapshotID, time.Now().UnixMilli())
|
||
if err := saveFilerFile(ctx, filerClient, metaDir, manifestListFileName, manifestListBuf.Bytes()); err != nil {
|
||
return "", fmt.Errorf("save manifest list: %w", err)
|
||
}
|
||
writtenArtifacts = append(writtenArtifacts, artifact{dir: metaDir, fileName: manifestListFileName})
|
||
|
||
// Create new snapshot with the rewritten manifest list
|
||
manifestListLocation := path.Join("metadata", manifestListFileName)
|
||
|
||
err = h.commitWithRetry(ctx, filerClient, bucketName, tablePath, metadataFileName, config, func(currentMeta table.Metadata, builder *table.MetadataBuilder) error {
|
||
// Guard: verify table head hasn't advanced since we planned.
|
||
// The merged manifest and manifest list were built against snapshotID;
|
||
// if the head moved, they reference stale state.
|
||
cs := currentMeta.CurrentSnapshot()
|
||
if cs == nil || cs.SnapshotID != snapshotID {
|
||
return errStalePlan
|
||
}
|
||
|
||
newSnapshot := &table.Snapshot{
|
||
SnapshotID: newSnapshotID,
|
||
ParentSnapshotID: &snapshotID,
|
||
SequenceNumber: cs.SequenceNumber + 1,
|
||
TimestampMs: time.Now().UnixMilli(),
|
||
ManifestList: manifestListLocation,
|
||
Summary: &table.Summary{
|
||
Operation: table.OpReplace,
|
||
Properties: map[string]string{"maintenance": "rewrite_manifests"},
|
||
},
|
||
SchemaID: func() *int {
|
||
id := schema.ID
|
||
return &id
|
||
}(),
|
||
}
|
||
if err := builder.AddSnapshot(newSnapshot); err != nil {
|
||
return err
|
||
}
|
||
return builder.SetSnapshotRef(
|
||
table.MainBranch,
|
||
newSnapshotID,
|
||
table.BranchRef,
|
||
)
|
||
})
|
||
if err != nil {
|
||
return "", fmt.Errorf("commit manifest rewrite: %w", err)
|
||
}
|
||
|
||
committed = true
|
||
return fmt.Sprintf("rewrote %d manifests into %d (%d entries)", len(manifests), len(specMap), totalEntries), nil
|
||
}
|
||
|
||
// ---------------------------------------------------------------------------
|
||
// Commit Protocol with Retry
|
||
// ---------------------------------------------------------------------------
|
||
|
||
// commitWithRetry implements optimistic concurrency for metadata updates.
|
||
// It reads the current metadata, applies the mutation, writes a new metadata
|
||
// file, and updates the table entry. On version conflict, it retries.
|
||
func (h *Handler) commitWithRetry(
|
||
ctx context.Context,
|
||
filerClient filer_pb.SeaweedFilerClient,
|
||
bucketName, tablePath, currentMetadataFileName string,
|
||
config Config,
|
||
mutate func(currentMeta table.Metadata, builder *table.MetadataBuilder) error,
|
||
) error {
|
||
maxRetries := config.MaxCommitRetries
|
||
if maxRetries <= 0 || maxRetries > 20 {
|
||
maxRetries = defaultMaxCommitRetries
|
||
}
|
||
|
||
for attempt := int64(0); attempt < maxRetries; attempt++ {
|
||
if attempt > 0 {
|
||
backoff := time.Duration(50*(1<<(attempt-1))) * time.Millisecond // exponential: 50ms, 100ms, 200ms, ...
|
||
const maxBackoff = 5 * time.Second
|
||
if backoff > maxBackoff {
|
||
backoff = maxBackoff
|
||
}
|
||
jitter := time.Duration(rand.Int64N(int64(backoff) / 5)) // 0–20% of backoff
|
||
timer := time.NewTimer(backoff + jitter)
|
||
select {
|
||
case <-timer.C:
|
||
case <-ctx.Done():
|
||
timer.Stop()
|
||
return ctx.Err()
|
||
}
|
||
}
|
||
|
||
// Load current metadata
|
||
meta, metaFileName, err := loadCurrentMetadata(ctx, filerClient, bucketName, tablePath)
|
||
if err != nil {
|
||
return fmt.Errorf("load metadata (attempt %d): %w", attempt, err)
|
||
}
|
||
|
||
// Build new metadata — pass the current metadata file path so the
|
||
// metadata log correctly records where the previous version lives.
|
||
currentMetaFilePath := path.Join("metadata", metaFileName)
|
||
builder, err := table.MetadataBuilderFromBase(meta, currentMetaFilePath)
|
||
if err != nil {
|
||
return fmt.Errorf("create metadata builder (attempt %d): %w", attempt, err)
|
||
}
|
||
|
||
// Apply the mutation
|
||
if err := mutate(meta, builder); err != nil {
|
||
return fmt.Errorf("apply mutation (attempt %d): %w", attempt, err)
|
||
}
|
||
|
||
if !builder.HasChanges() {
|
||
return nil // nothing to commit
|
||
}
|
||
|
||
newMeta, err := builder.Build()
|
||
if err != nil {
|
||
return fmt.Errorf("build metadata (attempt %d): %w", attempt, err)
|
||
}
|
||
|
||
// Serialize
|
||
metadataBytes, err := json.Marshal(newMeta)
|
||
if err != nil {
|
||
return fmt.Errorf("marshal metadata (attempt %d): %w", attempt, err)
|
||
}
|
||
|
||
// Determine new metadata file name. Include a timestamp suffix so
|
||
// concurrent writers stage to distinct files instead of clobbering.
|
||
currentVersion := extractMetadataVersion(metaFileName)
|
||
newVersion := currentVersion + 1
|
||
newMetadataFileName := fmt.Sprintf("v%d-%d.metadata.json", newVersion, time.Now().UnixNano())
|
||
|
||
// Save new metadata file
|
||
metaDir := path.Join(s3tables.TablesPath, bucketName, tablePath, "metadata")
|
||
if err := saveFilerFile(ctx, filerClient, metaDir, newMetadataFileName, metadataBytes); err != nil {
|
||
return fmt.Errorf("save metadata file (attempt %d): %w", attempt, err)
|
||
}
|
||
|
||
// Update the table entry's xattr with new metadata (CAS on version)
|
||
tableDir := path.Join(s3tables.TablesPath, bucketName, tablePath)
|
||
newMetadataLocation := path.Join("metadata", newMetadataFileName)
|
||
err = updateTableMetadataXattr(ctx, filerClient, tableDir, currentVersion, metadataBytes, newMetadataLocation)
|
||
if err != nil {
|
||
// Use a detached context for cleanup so staged files are removed
|
||
// even if the original context was canceled.
|
||
cleanupCtx, cleanupCancel := context.WithTimeout(context.Background(), 10*time.Second)
|
||
if !errors.Is(err, errMetadataVersionConflict) {
|
||
// Non-conflict error (permissions, transport, etc.): fail immediately.
|
||
_ = deleteFilerFile(cleanupCtx, filerClient, metaDir, newMetadataFileName)
|
||
cleanupCancel()
|
||
return fmt.Errorf("update table xattr (attempt %d): %w", attempt, err)
|
||
}
|
||
// Version conflict: clean up the new metadata file and retry
|
||
_ = deleteFilerFile(cleanupCtx, filerClient, metaDir, newMetadataFileName)
|
||
cleanupCancel()
|
||
if attempt < maxRetries-1 {
|
||
glog.V(1).Infof("iceberg maintenance: version conflict on %s/%s, retrying (attempt %d)", bucketName, tablePath, attempt)
|
||
continue
|
||
}
|
||
return fmt.Errorf("update table xattr (attempt %d): %w", attempt, err)
|
||
}
|
||
|
||
return nil
|
||
}
|
||
|
||
return fmt.Errorf("exceeded max commit retries (%d)", maxRetries)
|
||
}
|