Add data file compaction to iceberg maintenance (Phase 2) (#8503)

* 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>
This commit is contained in:
Chris Lu
2026-03-15 11:27:42 -07:00
committed by GitHub
parent 47799a5b4f
commit 8cde3d4486
30 changed files with 970 additions and 162 deletions

View File

@@ -162,10 +162,10 @@ func (h *ECBalanceHandler) Descriptor() *plugin_pb.JobTypeDescriptor {
},
},
DefaultValues: map[string]*plugin_pb.ConfigValue{
"imbalance_threshold": {Kind: &plugin_pb.ConfigValue_DoubleValue{DoubleValue: 0.2}},
"min_server_count": {Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: 3}},
"imbalance_threshold": {Kind: &plugin_pb.ConfigValue_DoubleValue{DoubleValue: 0.2}},
"min_server_count": {Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: 3}},
"min_interval_seconds": {Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: 60 * 60}},
"preferred_tags": {Kind: &plugin_pb.ConfigValue_StringValue{StringValue: ""}},
"preferred_tags": {Kind: &plugin_pb.ConfigValue_StringValue{StringValue: ""}},
},
},
AdminRuntimeDefaults: &plugin_pb.AdminRuntimeDefaults{
@@ -180,10 +180,10 @@ func (h *ECBalanceHandler) Descriptor() *plugin_pb.JobTypeDescriptor {
JobTypeMaxRuntimeSeconds: 1800,
},
WorkerDefaultValues: map[string]*plugin_pb.ConfigValue{
"imbalance_threshold": {Kind: &plugin_pb.ConfigValue_DoubleValue{DoubleValue: 0.2}},
"min_server_count": {Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: 3}},
"imbalance_threshold": {Kind: &plugin_pb.ConfigValue_DoubleValue{DoubleValue: 0.2}},
"min_server_count": {Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: 3}},
"min_interval_seconds": {Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: 60 * 60}},
"preferred_tags": {Kind: &plugin_pb.ConfigValue_StringValue{StringValue: ""}},
"preferred_tags": {Kind: &plugin_pb.ConfigValue_StringValue{StringValue: ""}},
},
}
}
@@ -527,11 +527,11 @@ func buildECBalanceProposal(result *workertypes.TaskDetectionResult) (*plugin_pb
},
},
Labels: map[string]string{
"task_type": "ec_balance",
"volume_id": fmt.Sprintf("%d", result.VolumeID),
"collection": result.Collection,
"source_node": sourceNode,
"target_node": targetNode,
"task_type": "ec_balance",
"volume_id": fmt.Sprintf("%d", result.VolumeID),
"collection": result.Collection,
"source_node": sourceNode,
"target_node": targetNode,
},
}, nil
}

View File

@@ -12,9 +12,9 @@ import (
func TestDeriveECBalanceWorkerConfig(t *testing.T) {
tests := []struct {
name string
values map[string]*plugin_pb.ConfigValue
check func(t *testing.T, config *ecBalanceWorkerConfig)
name string
values map[string]*plugin_pb.ConfigValue
check func(t *testing.T, config *ecBalanceWorkerConfig)
}{
{
name: "nil values uses defaults",

View File

@@ -11,8 +11,8 @@ import (
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb/plugin_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/worker_pb"
"github.com/seaweedfs/seaweedfs/weed/util"
ecstorage "github.com/seaweedfs/seaweedfs/weed/storage/erasure_coding"
"github.com/seaweedfs/seaweedfs/weed/util"
"github.com/seaweedfs/seaweedfs/weed/util/wildcard"
erasurecodingtask "github.com/seaweedfs/seaweedfs/weed/worker/tasks/erasure_coding"
workertypes "github.com/seaweedfs/seaweedfs/weed/worker/types"

View File

@@ -364,14 +364,23 @@ func buildCompactionBins(entries []iceberg.ManifestEntry, targetSize int64, minF
// splitOversizedBin splits a bin whose total size exceeds targetSize into
// sub-bins that each stay under targetSize while meeting minFiles.
// Entries are sorted by size descending before splitting so that large
// files are placed first, improving bin packing efficiency.
func splitOversizedBin(bin compactionBin, targetSize int64, minFiles int) []compactionBin {
// Sort largest-first for better packing.
sorted := make([]iceberg.ManifestEntry, len(bin.Entries))
copy(sorted, bin.Entries)
sort.Slice(sorted, func(i, j int) bool {
return sorted[i].DataFile().FileSizeBytes() > sorted[j].DataFile().FileSizeBytes()
})
var bins []compactionBin
current := compactionBin{
PartitionKey: bin.PartitionKey,
Partition: bin.Partition,
}
for _, entry := range bin.Entries {
if current.TotalSize > 0 && current.TotalSize+entry.DataFile().FileSizeBytes() > targetSize && len(current.Entries) >= minFiles {
for _, entry := range sorted {
if current.TotalSize > 0 && current.TotalSize+entry.DataFile().FileSizeBytes() > targetSize {
bins = append(bins, current)
current = compactionBin{
PartitionKey: bin.PartitionKey,
@@ -381,9 +390,44 @@ func splitOversizedBin(bin compactionBin, targetSize int64, minFiles int) []comp
current.Entries = append(current.Entries, entry)
current.TotalSize += entry.DataFile().FileSizeBytes()
}
if len(current.Entries) >= minFiles {
if len(current.Entries) > 0 {
bins = append(bins, current)
}
// Merge any bin with fewer than minFiles entries into its neighbor.
// Repeat until no more merges are needed. This handles runts at the
// end (from the split) and small leading bins (from largest-first
// sorting placing single large files into their own bins).
for changed := true; changed && len(bins) > 1; {
changed = false
for i := 0; i < len(bins); i++ {
if len(bins[i].Entries) >= minFiles {
continue
}
// Pick the better neighbor to merge into.
var target int
if i == 0 {
target = 1
} else if i == len(bins)-1 {
target = i - 1
} else if bins[i-1].TotalSize <= bins[i+1].TotalSize {
target = i - 1
} else {
target = i + 1
}
bins[target].Entries = append(bins[target].Entries, bins[i].Entries...)
bins[target].TotalSize += bins[i].TotalSize
bins = append(bins[:i], bins[i+1:]...)
changed = true
break // restart scan after structural change
}
}
// Final guard: if a single remaining bin has fewer than minFiles
// (entire input too small), return nothing.
if len(bins) == 1 && len(bins[0].Entries) < minFiles {
return nil
}
return bins
}
@@ -413,10 +457,11 @@ func partitionKey(partition map[int]any) string {
}
// mergeParquetFiles reads multiple small Parquet files and merges them into
// a single Parquet file. It reads rows from each source and writes them to
// the output using the schema from the first file.
//
// Files are loaded into memory (appropriate for compacting small files).
// a single Parquet file. Files are processed one at a time: each source file
// is loaded, its rows are streamed into the output writer, and then its data
// is released before the next file is loaded. This keeps peak memory
// proportional to the size of a single input file plus the output buffer,
// rather than the sum of all inputs.
func mergeParquetFiles(
ctx context.Context,
filerClient filer_pb.SeaweedFilerClient,
@@ -427,72 +472,84 @@ func mergeParquetFiles(
return nil, 0, fmt.Errorf("no entries to merge")
}
// Read all source files and create parquet readers
type sourceFile struct {
reader *parquet.Reader
data []byte
// Load the first file to obtain the schema for the writer.
firstData, err := loadFileByIcebergPath(ctx, filerClient, bucketName, tablePath, entries[0].DataFile().FilePath())
if err != nil {
return nil, 0, fmt.Errorf("read parquet file %s: %w", entries[0].DataFile().FilePath(), err)
}
var sources []sourceFile
defer func() {
for _, src := range sources {
if src.reader != nil {
src.reader.Close()
firstReader := parquet.NewReader(bytes.NewReader(firstData))
parquetSchema := firstReader.Schema()
if parquetSchema == nil {
firstReader.Close()
return nil, 0, fmt.Errorf("no parquet schema found in %s", entries[0].DataFile().FilePath())
}
var outputBuf bytes.Buffer
writer := parquet.NewWriter(&outputBuf, parquetSchema)
// drainReader streams all rows from reader into writer, then closes reader.
// source identifies the input file for error messages.
var totalRows int64
rows := make([]parquet.Row, 256)
drainReader := func(reader *parquet.Reader, source string) error {
defer reader.Close()
for {
select {
case <-ctx.Done():
return ctx.Err()
default:
}
n, readErr := reader.ReadRows(rows)
if n > 0 {
if _, writeErr := writer.WriteRows(rows[:n]); writeErr != nil {
return fmt.Errorf("write rows from %s: %w", source, writeErr)
}
totalRows += int64(n)
}
if readErr != nil {
if readErr == io.EOF {
return nil
}
return fmt.Errorf("read rows from %s: %w", source, readErr)
}
}
}()
}
var parquetSchema *parquet.Schema
for _, entry := range entries {
// Drain the first file.
firstSource := entries[0].DataFile().FilePath()
if err := drainReader(firstReader, firstSource); err != nil {
writer.Close()
return nil, 0, err
}
firstData = nil // allow GC
// Process remaining files one at a time.
for _, entry := range entries[1:] {
select {
case <-ctx.Done():
writer.Close()
return nil, 0, ctx.Err()
default:
}
data, err := loadFileByIcebergPath(ctx, filerClient, bucketName, tablePath, entry.DataFile().FilePath())
if err != nil {
writer.Close()
return nil, 0, fmt.Errorf("read parquet file %s: %w", entry.DataFile().FilePath(), err)
}
reader := parquet.NewReader(bytes.NewReader(data))
readerSchema := reader.Schema()
if parquetSchema == nil {
parquetSchema = readerSchema
} else if !schemasEqual(parquetSchema, readerSchema) {
if !schemasEqual(parquetSchema, reader.Schema()) {
reader.Close()
writer.Close()
return nil, 0, fmt.Errorf("schema mismatch in %s: cannot merge files with different schemas", entry.DataFile().FilePath())
}
sources = append(sources, sourceFile{reader: reader, data: data})
}
if parquetSchema == nil {
return nil, 0, fmt.Errorf("no parquet schema found")
}
// Write merged output
var outputBuf bytes.Buffer
writer := parquet.NewWriter(&outputBuf, parquetSchema)
var totalRows int64
rows := make([]parquet.Row, 256)
for _, src := range sources {
for {
n, err := src.reader.ReadRows(rows)
if n > 0 {
if _, writeErr := writer.WriteRows(rows[:n]); writeErr != nil {
writer.Close()
return nil, 0, fmt.Errorf("write rows: %w", writeErr)
}
totalRows += int64(n)
}
if err != nil {
if err == io.EOF {
break
}
writer.Close()
return nil, 0, fmt.Errorf("read rows: %w", err)
}
if err := drainReader(reader, entry.DataFile().FilePath()); err != nil {
writer.Close()
return nil, 0, err
}
// data goes out of scope here, eligible for GC before next iteration.
}
if err := writer.Close(); err != nil {

View File

@@ -18,6 +18,7 @@ const (
defaultMaxCommitRetries = 5
defaultTargetFileSizeBytes = 256 * 1024 * 1024
defaultMinInputFiles = 5
defaultMinManifestsToRewrite = 5
defaultOperations = "all"
)
@@ -29,6 +30,7 @@ type Config struct {
MaxCommitRetries int64
TargetFileSizeBytes int64
MinInputFiles int64
MinManifestsToRewrite int64
Operations string
}
@@ -42,6 +44,7 @@ func ParseConfig(values map[string]*plugin_pb.ConfigValue) Config {
MaxCommitRetries: readInt64Config(values, "max_commit_retries", defaultMaxCommitRetries),
TargetFileSizeBytes: readInt64Config(values, "target_file_size_bytes", defaultTargetFileSizeBytes),
MinInputFiles: readInt64Config(values, "min_input_files", defaultMinInputFiles),
MinManifestsToRewrite: readInt64Config(values, "min_manifests_to_rewrite", defaultMinManifestsToRewrite),
Operations: readStringConfig(values, "operations", defaultOperations),
}
@@ -64,6 +67,9 @@ func ParseConfig(values map[string]*plugin_pb.ConfigValue) Config {
if cfg.MinInputFiles < 2 {
cfg.MinInputFiles = defaultMinInputFiles
}
if cfg.MinManifestsToRewrite < 2 {
cfg.MinManifestsToRewrite = 2
}
return cfg
}

View File

@@ -649,8 +649,9 @@ func TestRewriteManifestsBelowThreshold(t *testing.T) {
handler := NewHandler(nil)
config := Config{
MinInputFiles: 10, // threshold higher than actual count (1)
MaxCommitRetries: 3,
MinInputFiles: 10,
MinManifestsToRewrite: 10, // threshold higher than actual manifest count (1)
MaxCommitRetries: 3,
}
result, err := handler.rewriteManifests(context.Background(), client, setup.BucketName, setup.tablePath(), config)
@@ -775,8 +776,8 @@ func TestDetectWithFakeFiler(t *testing.T) {
handler := NewHandler(nil)
config := Config{
SnapshotRetentionHours: 0, // everything is expired
MaxSnapshotsToKeep: 2, // 3 > 2, needs maintenance
SnapshotRetentionHours: 0, // everything is expired
MaxSnapshotsToKeep: 2, // 3 > 2, needs maintenance
MaxCommitRetries: 3,
}
@@ -785,7 +786,7 @@ func TestDetectWithFakeFiler(t *testing.T) {
client,
config,
"", "", "", // no filters
0, // no limit
0, // no limit
)
if err != nil {
t.Fatalf("scanTablesForMaintenance failed: %v", err)

View File

@@ -165,6 +165,21 @@ func (h *Handler) Descriptor() *plugin_pb.JobTypeDescriptor {
},
},
},
{
SectionId: "manifests",
Title: "Manifest Rewriting",
Description: "Controls for merging small manifests.",
Fields: []*plugin_pb.ConfigField{
{
Name: "min_manifests_to_rewrite",
Label: "Min Manifests",
Description: "Minimum number of manifests before rewriting is triggered.",
FieldType: plugin_pb.ConfigFieldType_CONFIG_FIELD_TYPE_INT64,
Widget: plugin_pb.ConfigWidget_CONFIG_WIDGET_NUMBER,
MinValue: &plugin_pb.ConfigValue{Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: 2}},
},
},
},
{
SectionId: "general",
Title: "General",
@@ -190,8 +205,9 @@ func (h *Handler) Descriptor() *plugin_pb.JobTypeDescriptor {
},
},
DefaultValues: map[string]*plugin_pb.ConfigValue{
"target_file_size_bytes": {Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: defaultTargetFileSizeBytes}},
"min_input_files": {Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: defaultMinInputFiles}},
"target_file_size_bytes": {Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: defaultTargetFileSizeBytes}},
"min_input_files": {Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: defaultMinInputFiles}},
"min_manifests_to_rewrite": {Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: defaultMinManifestsToRewrite}},
"snapshot_retention_hours": {Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: defaultSnapshotRetentionHours}},
"max_snapshots_to_keep": {Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: defaultMaxSnapshotsToKeep}},
"orphan_older_than_hours": {Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: defaultOrphanOlderThanHours}},
@@ -211,8 +227,8 @@ func (h *Handler) Descriptor() *plugin_pb.JobTypeDescriptor {
JobTypeMaxRuntimeSeconds: 3600, // 1 hour max
},
WorkerDefaultValues: map[string]*plugin_pb.ConfigValue{
"target_file_size_bytes": {Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: defaultTargetFileSizeBytes}},
"min_input_files": {Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: defaultMinInputFiles}},
"target_file_size_bytes": {Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: defaultTargetFileSizeBytes}},
"min_input_files": {Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: defaultMinInputFiles}},
"snapshot_retention_hours": {Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: defaultSnapshotRetentionHours}},
"max_snapshots_to_keep": {Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: defaultMaxSnapshotsToKeep}},
"orphan_older_than_hours": {Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: defaultOrphanOlderThanHours}},
@@ -255,11 +271,10 @@ func (h *Handler) Detect(ctx context.Context, request *plugin_pb.RunDetectionReq
namespaceFilter := strings.TrimSpace(readStringConfig(request.GetAdminConfigValues(), "namespace_filter", ""))
tableFilter := strings.TrimSpace(readStringConfig(request.GetAdminConfigValues(), "table_filter", ""))
// Connect to filer to scan table buckets
filerAddress := filerAddresses[0]
conn, err := grpc.NewClient(filerAddress, h.grpcDialOption)
// Connect to filer — try each address until one succeeds.
filerAddress, conn, err := h.connectToFiler(filerAddresses)
if err != nil {
return fmt.Errorf("connect to filer %s: %w", filerAddress, err)
return fmt.Errorf("connect to filer: %w", err)
}
defer conn.Close()
filerClient := filer_pb.NewSeaweedFilerClient(conn)
@@ -329,6 +344,12 @@ func (h *Handler) Execute(ctx context.Context, request *plugin_pb.ExecuteJobRequ
if bucketName == "" || namespace == "" || tableName == "" || filerAddress == "" {
return fmt.Errorf("missing required parameters: bucket_name=%q, namespace=%q, table_name=%q, filer_address=%q", bucketName, namespace, tableName, filerAddress)
}
// Reject path traversal in bucket/namespace/table names.
for _, name := range []string{bucketName, namespace, tableName} {
if strings.Contains(name, "..") || strings.ContainsAny(name, "/\\") {
return fmt.Errorf("invalid name %q: must not contain path separators or '..'", name)
}
}
if tablePath == "" {
tablePath = path.Join(namespace, tableName)
}
@@ -467,5 +488,24 @@ func (h *Handler) sendEmptyDetection(sender pluginworker.DetectionSender) error
})
}
// connectToFiler tries each filer address in order and returns the first
// successful gRPC connection. If all addresses fail, it returns a
// consolidated error.
func (h *Handler) connectToFiler(addresses []string) (string, *grpc.ClientConn, error) {
var lastErr error
for _, addr := range addresses {
conn, err := grpc.NewClient(addr, h.grpcDialOption)
if err != nil {
lastErr = fmt.Errorf("filer %s: %w", addr, err)
continue
}
return addr, conn, nil
}
if lastErr == nil {
lastErr = fmt.Errorf("no filer addresses provided")
}
return "", nil, lastErr
}
// Ensure Handler implements JobHandler.
var _ pluginworker.JobHandler = (*Handler)(nil)

View File

@@ -363,11 +363,11 @@ func TestManifestRewriteNestedPathConsistency(t *testing.T) {
func TestNormalizeIcebergPath(t *testing.T) {
tests := []struct {
name string
name string
icebergPath string
bucket string
tablePath string
expected string
bucket string
tablePath string
expected string
}{
{
"relative metadata path",

View File

@@ -322,8 +322,8 @@ func (h *Handler) rewriteManifests(
return "", fmt.Errorf("parse manifest list: %w", err)
}
if int64(len(manifests)) < config.MinInputFiles {
return fmt.Sprintf("only %d manifests, below threshold of %d", len(manifests), config.MinInputFiles), nil
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

View File

@@ -0,0 +1,26 @@
package iceberg
import (
"context"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
)
// The following methods export the internal maintenance operations for use
// by integration tests. They are intentionally thin wrappers.
func (h *Handler) ExpireSnapshots(ctx context.Context, client filer_pb.SeaweedFilerClient, bucketName, tablePath string, config Config) (string, error) {
return h.expireSnapshots(ctx, client, bucketName, tablePath, config)
}
func (h *Handler) RemoveOrphans(ctx context.Context, client filer_pb.SeaweedFilerClient, bucketName, tablePath string, config Config) (string, error) {
return h.removeOrphans(ctx, client, bucketName, tablePath, config)
}
func (h *Handler) RewriteManifests(ctx context.Context, client filer_pb.SeaweedFilerClient, bucketName, tablePath string, config Config) (string, error) {
return h.rewriteManifests(ctx, client, bucketName, tablePath, config)
}
func (h *Handler) CompactDataFiles(ctx context.Context, client filer_pb.SeaweedFilerClient, bucketName, tablePath string, config Config) (string, error) {
return h.compactDataFiles(ctx, client, bucketName, tablePath, config)
}

View File

@@ -14,8 +14,8 @@ import (
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb/plugin_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/worker_pb"
balancetask "github.com/seaweedfs/seaweedfs/weed/worker/tasks/balance"
"github.com/seaweedfs/seaweedfs/weed/util/wildcard"
balancetask "github.com/seaweedfs/seaweedfs/weed/worker/tasks/balance"
workertypes "github.com/seaweedfs/seaweedfs/weed/worker/types"
"google.golang.org/grpc"
"google.golang.org/protobuf/proto"
@@ -1391,7 +1391,7 @@ func buildBatchVolumeBalanceProposals(
},
Labels: map[string]string{
"task_type": "balance",
"batch": "true",
"batch": "true",
"batch_size": fmt.Sprintf("%d", len(moves)),
},
})