filer: add conditional update preconditions (#8647)

* filer: add conditional update preconditions

* iceberg: tighten metadata CAS preconditions
This commit is contained in:
Chris Lu
2026-03-16 12:33:32 -07:00
committed by GitHub
parent 61d6f2608e
commit acea36a181
7 changed files with 254 additions and 100 deletions

View File

@@ -8,6 +8,7 @@ import (
"fmt"
"io"
"path"
"strconv"
"strings"
"sync"
"time"
@@ -280,9 +281,10 @@ func deleteFilerFile(ctx context.Context, client filer_pb.SeaweedFilerClient, di
}
// updateTableMetadataXattr updates the table entry's metadata xattr with
// the new Iceberg metadata. It performs a compare-and-swap: if the stored
// metadataVersion does not match expectedVersion, it returns
// errMetadataVersionConflict so the caller can retry.
// the new Iceberg metadata. It verifies the stored metadataVersion before
// writing and passes the previous metadata xattr back to the filer as a
// server-side precondition so concurrent writers fail with a retryable
// metadata version conflict.
// newMetadataLocation is the table-relative path to the new metadata file
// (e.g. "metadata/v3.metadata.json").
func updateTableMetadataXattr(ctx context.Context, client filer_pb.SeaweedFilerClient, tableDir string, expectedVersion int, newFullMetadata []byte, newMetadataLocation string) error {
@@ -311,13 +313,8 @@ func updateTableMetadataXattr(ctx context.Context, client filer_pb.SeaweedFilerC
return fmt.Errorf("unmarshal existing xattr: %w", err)
}
// Compare-and-swap: verify the stored metadataVersion matches what we expect.
// NOTE: This is a client-side CAS — two workers could both read the same
// version, pass this check, and race at UpdateEntry (last-write-wins).
// The proper fix is server-side precondition support on UpdateEntryRequest
// (e.g. expect-version or If-Match semantics). Until then, commitWithRetry
// with exponential backoff mitigates but does not eliminate the race.
// Avoid scheduling concurrent maintenance on the same table.
// Verify the stored metadataVersion matches what we expect before issuing
// the conditional UpdateEntry request below.
versionRaw, ok := internalMeta["metadataVersion"]
if !ok {
return fmt.Errorf("%w: metadataVersion field missing from xattr", errMetadataVersionConflict)
@@ -368,17 +365,29 @@ func updateTableMetadataXattr(ctx context.Context, client filer_pb.SeaweedFilerC
return fmt.Errorf("marshal updated xattr: %w", err)
}
expectedVersionXattr := resp.Entry.Extended[s3tables.ExtendedKeyMetadataVersion]
resp.Entry.Extended[s3tables.ExtendedKeyMetadata] = updatedXattr
resp.Entry.Extended[s3tables.ExtendedKeyMetadataVersion] = metadataVersionXattr(newVersion)
_, err = client.UpdateEntry(ctx, &filer_pb.UpdateEntryRequest{
Directory: parentDir,
Entry: resp.Entry,
ExpectedExtended: map[string][]byte{
s3tables.ExtendedKeyMetadataVersion: expectedVersionXattr,
},
})
if err != nil {
if status.Code(err) == codes.FailedPrecondition {
return fmt.Errorf("%w: table metadata changed during update", errMetadataVersionConflict)
}
return fmt.Errorf("update table entry: %w", err)
}
return nil
}
func metadataVersionXattr(version int) []byte {
return []byte(strconv.Itoa(version))
}
// generateIcebergVersionToken produces a random hex token, mirroring the
// logic in s3tables.generateVersionToken (which is unexported).
func generateIcebergVersionToken() string {