Files
seaweedFS/weed/plugin/worker/iceberg/handler.go
Chris Lu 8cde3d4486 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>
2026-03-15 11:27:42 -07:00

512 lines
20 KiB
Go

package iceberg
import (
"context"
"fmt"
"path"
"strings"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/plugin_pb"
pluginworker "github.com/seaweedfs/seaweedfs/weed/plugin/worker"
"google.golang.org/grpc"
"google.golang.org/protobuf/types/known/timestamppb"
)
func init() {
pluginworker.RegisterHandler(pluginworker.HandlerFactory{
JobType: jobType,
Category: pluginworker.CategoryHeavy,
Aliases: []string{"iceberg-maintenance", "iceberg.maintenance", "iceberg"},
Build: func(opts pluginworker.HandlerBuildOptions) (pluginworker.JobHandler, error) {
return NewHandler(opts.GrpcDialOption), nil
},
})
}
// Handler implements the JobHandler interface for Iceberg table maintenance:
// snapshot expiration, orphan file removal, and manifest rewriting.
type Handler struct {
grpcDialOption grpc.DialOption
}
// NewHandler creates a new handler for iceberg table maintenance.
func NewHandler(grpcDialOption grpc.DialOption) *Handler {
return &Handler{grpcDialOption: grpcDialOption}
}
func (h *Handler) Capability() *plugin_pb.JobTypeCapability {
return &plugin_pb.JobTypeCapability{
JobType: jobType,
CanDetect: true,
CanExecute: true,
MaxDetectionConcurrency: 1,
MaxExecutionConcurrency: 4,
DisplayName: "Iceberg Maintenance",
Description: "Compacts, expires snapshots, removes orphans, and rewrites manifests for Iceberg tables in S3 table buckets",
Weight: 50,
}
}
func (h *Handler) Descriptor() *plugin_pb.JobTypeDescriptor {
return &plugin_pb.JobTypeDescriptor{
JobType: jobType,
DisplayName: "Iceberg Maintenance",
Description: "Automated maintenance for Iceberg tables: snapshot expiration, orphan removal, manifest rewriting",
Icon: "fas fa-snowflake",
DescriptorVersion: 1,
AdminConfigForm: &plugin_pb.ConfigForm{
FormId: "iceberg-maintenance-admin",
Title: "Iceberg Maintenance Admin Config",
Description: "Admin-side controls for Iceberg table maintenance scope.",
Sections: []*plugin_pb.ConfigSection{
{
SectionId: "scope",
Title: "Scope",
Description: "Filters to restrict which tables are scanned for maintenance.",
Fields: []*plugin_pb.ConfigField{
{
Name: "bucket_filter",
Label: "Bucket Filter",
Description: "Comma-separated wildcard patterns for table buckets (* and ? supported). Blank = all.",
Placeholder: "prod-*, staging-*",
FieldType: plugin_pb.ConfigFieldType_CONFIG_FIELD_TYPE_STRING,
Widget: plugin_pb.ConfigWidget_CONFIG_WIDGET_TEXT,
},
{
Name: "namespace_filter",
Label: "Namespace Filter",
Description: "Comma-separated wildcard patterns for namespaces (* and ? supported). Blank = all.",
Placeholder: "analytics, events-*",
FieldType: plugin_pb.ConfigFieldType_CONFIG_FIELD_TYPE_STRING,
Widget: plugin_pb.ConfigWidget_CONFIG_WIDGET_TEXT,
},
{
Name: "table_filter",
Label: "Table Filter",
Description: "Comma-separated wildcard patterns for table names (* and ? supported). Blank = all.",
Placeholder: "clicks, orders-*",
FieldType: plugin_pb.ConfigFieldType_CONFIG_FIELD_TYPE_STRING,
Widget: plugin_pb.ConfigWidget_CONFIG_WIDGET_TEXT,
},
},
},
},
DefaultValues: map[string]*plugin_pb.ConfigValue{
"bucket_filter": {Kind: &plugin_pb.ConfigValue_StringValue{StringValue: ""}},
"namespace_filter": {Kind: &plugin_pb.ConfigValue_StringValue{StringValue: ""}},
"table_filter": {Kind: &plugin_pb.ConfigValue_StringValue{StringValue: ""}},
},
},
WorkerConfigForm: &plugin_pb.ConfigForm{
FormId: "iceberg-maintenance-worker",
Title: "Iceberg Maintenance Worker Config",
Description: "Worker-side thresholds for maintenance operations.",
Sections: []*plugin_pb.ConfigSection{
{
SectionId: "snapshots",
Title: "Snapshot Expiration",
Description: "Controls for automatic snapshot cleanup.",
Fields: []*plugin_pb.ConfigField{
{
Name: "snapshot_retention_hours",
Label: "Retention (hours)",
Description: "Expire snapshots older than this many hours.",
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: 1}},
},
{
Name: "max_snapshots_to_keep",
Label: "Max Snapshots",
Description: "Always keep at least this many most recent snapshots.",
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: 1}},
},
},
},
{
SectionId: "compaction",
Title: "Data Compaction",
Description: "Controls for bin-packing small Parquet data files.",
Fields: []*plugin_pb.ConfigField{
{
Name: "target_file_size_bytes",
Label: "Target File Size (bytes)",
Description: "Files smaller than this are candidates for compaction.",
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: 1024 * 1024}},
},
{
Name: "min_input_files",
Label: "Min Input Files",
Description: "Minimum number of small files in a partition to trigger compaction.",
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: "orphans",
Title: "Orphan Removal",
Description: "Controls for orphan file cleanup.",
Fields: []*plugin_pb.ConfigField{
{
Name: "orphan_older_than_hours",
Label: "Safety Window (hours)",
Description: "Only remove orphan files older than this many hours.",
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: 1}},
},
},
},
{
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",
Description: "General maintenance settings.",
Fields: []*plugin_pb.ConfigField{
{
Name: "max_commit_retries",
Label: "Max Commit Retries",
Description: "Maximum number of commit retries on version conflict.",
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: 1}},
MaxValue: &plugin_pb.ConfigValue{Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: 20}},
},
{
Name: "operations",
Label: "Operations",
Description: "Comma-separated list of operations to run: compact, expire_snapshots, remove_orphans, rewrite_manifests, or 'all'.",
FieldType: plugin_pb.ConfigFieldType_CONFIG_FIELD_TYPE_STRING,
Widget: plugin_pb.ConfigWidget_CONFIG_WIDGET_TEXT,
},
},
},
},
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}},
"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}},
"max_commit_retries": {Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: defaultMaxCommitRetries}},
"operations": {Kind: &plugin_pb.ConfigValue_StringValue{StringValue: defaultOperations}},
},
},
AdminRuntimeDefaults: &plugin_pb.AdminRuntimeDefaults{
Enabled: false, // disabled by default
DetectionIntervalSeconds: 3600, // 1 hour
DetectionTimeoutSeconds: 300,
MaxJobsPerDetection: 100,
GlobalExecutionConcurrency: 4,
PerWorkerExecutionConcurrency: 2,
RetryLimit: 1,
RetryBackoffSeconds: 60,
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}},
"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}},
"max_commit_retries": {Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: defaultMaxCommitRetries}},
"operations": {Kind: &plugin_pb.ConfigValue_StringValue{StringValue: defaultOperations}},
},
}
}
func (h *Handler) Detect(ctx context.Context, request *plugin_pb.RunDetectionRequest, sender pluginworker.DetectionSender) error {
if request == nil {
return fmt.Errorf("run detection request is nil")
}
if sender == nil {
return fmt.Errorf("detection sender is nil")
}
if request.JobType != "" && request.JobType != jobType {
return fmt.Errorf("job type %q is not handled by iceberg maintenance handler", request.JobType)
}
workerConfig := ParseConfig(request.GetWorkerConfigValues())
if _, err := parseOperations(workerConfig.Operations); err != nil {
return fmt.Errorf("invalid operations config: %w", err)
}
// Detection interval is managed by the scheduler via AdminRuntimeDefaults.DetectionIntervalSeconds.
// Get filer addresses from cluster context
filerAddresses := make([]string, 0)
if request.ClusterContext != nil {
filerAddresses = append(filerAddresses, request.ClusterContext.FilerGrpcAddresses...)
}
if len(filerAddresses) == 0 {
_ = sender.SendActivity(pluginworker.BuildDetectorActivity("skipped", "no filer addresses in cluster context", nil))
return h.sendEmptyDetection(sender)
}
// Read scope filters
bucketFilter := strings.TrimSpace(readStringConfig(request.GetAdminConfigValues(), "bucket_filter", ""))
namespaceFilter := strings.TrimSpace(readStringConfig(request.GetAdminConfigValues(), "namespace_filter", ""))
tableFilter := strings.TrimSpace(readStringConfig(request.GetAdminConfigValues(), "table_filter", ""))
// Connect to filer — try each address until one succeeds.
filerAddress, conn, err := h.connectToFiler(filerAddresses)
if err != nil {
return fmt.Errorf("connect to filer: %w", err)
}
defer conn.Close()
filerClient := filer_pb.NewSeaweedFilerClient(conn)
maxResults := int(request.MaxResults)
tables, err := h.scanTablesForMaintenance(ctx, filerClient, workerConfig, bucketFilter, namespaceFilter, tableFilter, maxResults)
if err != nil {
_ = sender.SendActivity(pluginworker.BuildDetectorActivity("scan_error", fmt.Sprintf("error scanning tables: %v", err), nil))
return fmt.Errorf("scan tables: %w", err)
}
_ = sender.SendActivity(pluginworker.BuildDetectorActivity("scan_complete",
fmt.Sprintf("found %d table(s) needing maintenance", len(tables)),
map[string]*plugin_pb.ConfigValue{
"tables_found": {Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: int64(len(tables))}},
}))
hasMore := false
if maxResults > 0 && len(tables) > maxResults {
hasMore = true
tables = tables[:maxResults]
}
proposals := make([]*plugin_pb.JobProposal, 0, len(tables))
for _, t := range tables {
proposal := h.buildMaintenanceProposal(t, filerAddress)
proposals = append(proposals, proposal)
}
if err := sender.SendProposals(&plugin_pb.DetectionProposals{
JobType: jobType,
Proposals: proposals,
HasMore: hasMore,
}); err != nil {
return err
}
return sender.SendComplete(&plugin_pb.DetectionComplete{
JobType: jobType,
Success: true,
TotalProposals: int32(len(proposals)),
})
}
func (h *Handler) Execute(ctx context.Context, request *plugin_pb.ExecuteJobRequest, sender pluginworker.ExecutionSender) error {
if request == nil || request.Job == nil {
return fmt.Errorf("execute request/job is nil")
}
if sender == nil {
return fmt.Errorf("execution sender is nil")
}
if request.Job.JobType != "" && request.Job.JobType != jobType {
return fmt.Errorf("job type %q is not handled by iceberg maintenance handler", request.Job.JobType)
}
canonicalJobType := request.Job.JobType
if canonicalJobType == "" {
canonicalJobType = jobType
}
params := request.Job.Parameters
bucketName := readStringConfig(params, "bucket_name", "")
namespace := readStringConfig(params, "namespace", "")
tableName := readStringConfig(params, "table_name", "")
tablePath := readStringConfig(params, "table_path", "")
filerAddress := readStringConfig(params, "filer_address", "")
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)
}
// Sanitize tablePath to prevent directory traversal.
tablePath = path.Clean(tablePath)
expected := path.Join(namespace, tableName)
if tablePath != expected && !strings.HasPrefix(tablePath, expected+"/") {
return fmt.Errorf("invalid table_path %q: must be %q or a subpath", tablePath, expected)
}
workerConfig := ParseConfig(request.GetWorkerConfigValues())
ops, opsErr := parseOperations(workerConfig.Operations)
if opsErr != nil {
return fmt.Errorf("invalid operations config: %w", opsErr)
}
// Send initial progress
if err := sender.SendProgress(&plugin_pb.JobProgressUpdate{
JobId: request.Job.JobId,
JobType: canonicalJobType,
State: plugin_pb.JobState_JOB_STATE_ASSIGNED,
ProgressPercent: 0,
Stage: "assigned",
Message: fmt.Sprintf("maintenance job accepted for %s/%s/%s", bucketName, namespace, tableName),
Activities: []*plugin_pb.ActivityEvent{
pluginworker.BuildExecutorActivity("assigned", fmt.Sprintf("maintenance job accepted for %s/%s/%s", bucketName, namespace, tableName)),
},
}); err != nil {
return err
}
// Connect to filer
conn, err := grpc.NewClient(filerAddress, h.grpcDialOption)
if err != nil {
return fmt.Errorf("connect to filer %s: %w", filerAddress, err)
}
defer conn.Close()
filerClient := filer_pb.NewSeaweedFilerClient(conn)
var results []string
var lastErr error
totalOps := len(ops)
completedOps := 0
// Execute operations in correct Iceberg maintenance order:
// expire_snapshots → remove_orphans → rewrite_manifests
for _, op := range ops {
select {
case <-ctx.Done():
return ctx.Err()
default:
}
progress := float64(completedOps) / float64(totalOps) * 100
if err := sender.SendProgress(&plugin_pb.JobProgressUpdate{
JobId: request.Job.JobId,
JobType: canonicalJobType,
State: plugin_pb.JobState_JOB_STATE_RUNNING,
ProgressPercent: progress,
Stage: op,
Message: fmt.Sprintf("running %s", op),
Activities: []*plugin_pb.ActivityEvent{
pluginworker.BuildExecutorActivity(op, fmt.Sprintf("starting %s for %s/%s/%s", op, bucketName, namespace, tableName)),
},
}); err != nil {
return err
}
var opResult string
var opErr error
switch op {
case "compact":
opResult, opErr = h.compactDataFiles(ctx, filerClient, bucketName, tablePath, workerConfig)
case "expire_snapshots":
opResult, opErr = h.expireSnapshots(ctx, filerClient, bucketName, tablePath, workerConfig)
case "remove_orphans":
opResult, opErr = h.removeOrphans(ctx, filerClient, bucketName, tablePath, workerConfig)
case "rewrite_manifests":
opResult, opErr = h.rewriteManifests(ctx, filerClient, bucketName, tablePath, workerConfig)
default:
glog.Warningf("unknown maintenance operation: %s", op)
continue
}
completedOps++
if opErr != nil {
glog.Warningf("iceberg maintenance %s failed for %s/%s/%s: %v", op, bucketName, namespace, tableName, opErr)
results = append(results, fmt.Sprintf("%s: error: %v", op, opErr))
lastErr = opErr
} else {
results = append(results, fmt.Sprintf("%s: %s", op, opResult))
}
}
resultSummary := strings.Join(results, "; ")
success := lastErr == nil
return sender.SendCompleted(&plugin_pb.JobCompleted{
JobId: request.Job.JobId,
JobType: canonicalJobType,
Success: success,
ErrorMessage: func() string {
if lastErr != nil {
return lastErr.Error()
}
return ""
}(),
Result: &plugin_pb.JobResult{
Summary: resultSummary,
OutputValues: map[string]*plugin_pb.ConfigValue{
"bucket": {Kind: &plugin_pb.ConfigValue_StringValue{StringValue: bucketName}},
"namespace": {Kind: &plugin_pb.ConfigValue_StringValue{StringValue: namespace}},
"table": {Kind: &plugin_pb.ConfigValue_StringValue{StringValue: tableName}},
},
},
Activities: []*plugin_pb.ActivityEvent{
pluginworker.BuildExecutorActivity("completed", resultSummary),
},
CompletedAt: timestamppb.Now(),
})
}
func (h *Handler) sendEmptyDetection(sender pluginworker.DetectionSender) error {
if err := sender.SendProposals(&plugin_pb.DetectionProposals{
JobType: jobType,
Proposals: []*plugin_pb.JobProposal{},
HasMore: false,
}); err != nil {
return err
}
return sender.SendComplete(&plugin_pb.DetectionComplete{
JobType: jobType,
Success: true,
TotalProposals: 0,
})
}
// 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)