Browse Source
feat: separate scheduler lanes for iceberg, lifecycle, and volume management (#8787)
feat: separate scheduler lanes for iceberg, lifecycle, and volume management (#8787)
* feat: introduce scheduler lanes for independent per-workload scheduling
Split the single plugin scheduler loop into independent per-lane
goroutines so that volume management, iceberg compaction, and lifecycle
operations never block each other.
Each lane has its own:
- Goroutine (laneSchedulerLoop)
- Wake channel for immediate scheduling
- Admin lock scope (e.g. "plugin scheduler:default")
- Configurable idle sleep duration
- Loop state tracking
Three lanes are defined:
- default: vacuum, volume_balance, ec_balance, erasure_coding, admin_script
- iceberg: iceberg_maintenance
- lifecycle: s3_lifecycle (new, handler coming in a later commit)
Job types are mapped to lanes via a hardcoded map with LaneDefault as
the fallback. The SchedulerJobTypeState and SchedulerStatus types now
include a Lane field for API consumers.
* feat: per-lane execution reservation pools for resource isolation
Each scheduler lane now maintains its own execution reservation map
so that a busy volume lane cannot consume execution slots needed by
iceberg or lifecycle lanes. The per-lane pool is used by default when
dispatching jobs through the lane scheduler; the global pool remains
as a fallback for the public DispatchProposals API.
* feat: add per-lane scheduler status API and lane worker UI pages
- GET /api/plugin/lanes returns all lanes with status and job types
- GET /api/plugin/workers?lane=X filters workers by lane
- GET /api/plugin/scheduler-states?lane=X filters job types by lane
- GET /api/plugin/scheduler-status?lane=X returns lane-scoped status
- GET /plugin/lanes/{lane}/workers renders per-lane worker page
- SchedulerJobTypeState now includes a "lane" field
The lane worker pages show scheduler status, job type configuration,
and connected workers scoped to a single lane, with links back to
the main plugin overview.
* feat: add s3_lifecycle worker handler for object store lifecycle management
Implements a full plugin worker handler for S3 lifecycle management,
assigned to the new "lifecycle" scheduler lane.
Detection phase:
- Reads filer.conf to find buckets with TTL lifecycle rules
- Creates one job proposal per bucket with active lifecycle rules
- Supports bucket_filter wildcard pattern from admin config
Execution phase:
- Walks the bucket directory tree breadth-first
- Identifies expired objects by checking TtlSec + Crtime < now
- Deletes expired objects in configurable batches
- Reports progress with scanned/expired/error counts
- Supports dry_run mode for safe testing
Configurable via admin UI:
- batch_size: entries per filer listing page (default 1000)
- max_deletes_per_bucket: safety cap per run (default 10000)
- dry_run: detect without deleting
- delete_marker_cleanup: clean expired delete markers
- abort_mpu_days: abort stale multipart uploads
The handler integrates with the existing PutBucketLifecycle flow which
sets TtlSec on entries via filer.conf path rules.
* feat: add per-lane submenu items under Workers sidebar menu
Replace the single "Workers" sidebar link with a collapsible submenu
containing three lane entries:
- Default (volume management + admin scripts) -> /plugin
- Iceberg (table compaction) -> /plugin/lanes/iceberg/workers
- Lifecycle (S3 object expiration) -> /plugin/lanes/lifecycle/workers
The submenu auto-expands when on any /plugin page and highlights the
active lane. Icons match each lane's job type descriptor (server,
snowflake, hourglass).
* feat: scope plugin pages to their scheduler lane
The plugin overview, configuration, detection, queue, and execution
pages now filter workers, job types, scheduler states, and scheduler
status to only show data for their lane.
- Plugin() templ function accepts a lane parameter (default: "default")
- JavaScript appends ?lane= to /api/plugin/workers, /job-types,
/scheduler-states, and /scheduler-status API calls
- GET /api/plugin/job-types now supports ?lane= filtering
- When ?job= is provided (e.g. ?job=iceberg_maintenance), the lane is
auto-derived from the job type so the page scopes correctly
This ensures /plugin shows only default-lane workers and
/plugin/configuration?job=iceberg_maintenance scopes to the iceberg lane.
* fix: remove "Lane" from lane worker page titles and capitalize properly
"lifecycle Lane Workers" -> "Lifecycle Workers"
"iceberg Lane Workers" -> "Iceberg Workers"
* refactor: promote lane items to top-level sidebar menu entries
Move Default, Iceberg, and Lifecycle from a collapsible submenu to
direct top-level items under the WORKERS heading. Removes the
intermediate "Workers" parent link and collapse toggle.
* admin: unify plugin lane routes and handlers
* admin: filter plugin jobs and activities by lane
* admin: reuse plugin UI for worker lane pages
* fix: use ServerAddress.ToGrpcAddress() for filer connections in lifecycle handler
ClusterContext addresses use ServerAddress format (host:port.grpcPort).
Convert to the actual gRPC address via ToGrpcAddress() before dialing,
and add a Ping verification after connecting.
Fixes: "dial tcp: lookup tcp/8888.18888: unknown port"
* fix: resolve ServerAddress gRPC port in iceberg and lifecycle filer connections
ClusterContext addresses use ServerAddress format (host:httpPort.grpcPort).
Both the iceberg and lifecycle handlers now detect the compound format
and extract the gRPC port via ToGrpcAddress() before dialing. Plain
host:port addresses (e.g. from tests) are passed through unchanged.
Fixes: "dial tcp: lookup tcp/8888.18888: unknown port"
* align url
* Potential fix for code scanning alert no. 335: Incorrect conversion between integer types
Co-authored-by: Copilot Autofix powered by AI <62310815+github-advanced-security[bot]@users.noreply.github.com>
* fix: address PR review findings across scheduler lanes and lifecycle handler
- Fix variable shadowing: rename loop var `w` to `worker` in
GetPluginWorkersAPI to avoid shadowing the http.ResponseWriter param
- Fix stale GetSchedulerStatus: aggregate loop states across all lanes
instead of reading never-updated legacy schedulerLoopState
- Scope InProcessJobs to lane in GetLaneSchedulerStatus
- Fix AbortMPUDays=0 treated as unset: change <= 0 to < 0 so 0 disables
- Propagate listing errors in lifecycle bucket walk instead of swallowing
- Implement DeleteMarkerCleanup: scan for S3 delete marker entries and
remove them
- Implement AbortMPUDays: scan .uploads directory and remove stale
multipart uploads older than the configured threshold
- Fix success determination: mark job failed when result.errors > 0
even if no fatal error occurred
- Add regression test for jobTypeLaneMap to catch drift from handler
registrations
* fix: guard against nil result in lifecycle completion and trim filer addresses
- Guard result dereference in completion summary: use local vars
defaulting to 0 when result is nil to prevent panic
- Append trimmed filer addresses instead of originals so whitespace
is not passed to the gRPC dialer
* fix: propagate ctx cancellation from deleteExpiredObjects and add config logging
- deleteExpiredObjects now returns a third error value when the context
is canceled mid-batch; the caller stops processing further batches
and returns the cancellation error to the job completion handler
- readBoolConfig and readInt64Config now log unexpected ConfigValue
types at V(1) for debugging, consistent with readStringConfig
* fix: propagate errors in lifecycle cleanup helpers and use correct delete marker key
- cleanupDeleteMarkers: return error on ctx cancellation and SeaweedList
failures instead of silently continuing
- abortIncompleteMPUs: log SeaweedList errors instead of discarding
- isDeleteMarker: use ExtDeleteMarkerKey ("Seaweed-X-Amz-Delete-Marker")
instead of ExtLatestVersionIsDeleteMarker which is for the parent entry
- batchSize cap: use math.MaxInt instead of math.MaxInt32
* fix: propagate ctx cancellation from abortIncompleteMPUs and log unrecognized bool strings
- abortIncompleteMPUs now returns (aborted, errors, ctxErr) matching
cleanupDeleteMarkers; caller stops on cancellation or listing failure
- readBoolConfig logs unrecognized string values before falling back
* fix: shared per-bucket budget across lifecycle phases and allow cleanup without expired objects
- Thread a shared remaining counter through TTL deletion, delete marker
cleanup, and MPU abort so the total operations per bucket never exceed
MaxDeletesPerBucket
- Remove early return when no TTL-expired objects found so delete marker
cleanup and MPU abort still run
- Add NOTE on cleanupDeleteMarkers about version-safety limitation
---------
Co-authored-by: Copilot Autofix powered by AI <62310815+github-advanced-security[bot]@users.noreply.github.com>
pull/8436/merge
committed by
GitHub
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
22 changed files with 2230 additions and 196 deletions
-
125weed/admin/dash/plugin_api.go
-
36weed/admin/dash/plugin_api_test.go
-
8weed/admin/handlers/admin_handlers.go
-
17weed/admin/handlers/admin_handlers_routes_test.go
-
68weed/admin/handlers/plugin_handlers.go
-
14weed/admin/plugin/plugin.go
-
221weed/admin/plugin/plugin_scheduler.go
-
109weed/admin/plugin/scheduler_lane.go
-
47weed/admin/plugin/scheduler_lane_test.go
-
182weed/admin/plugin/scheduler_status.go
-
1weed/admin/plugin/types.go
-
41weed/admin/view/app/plugin.templ
-
53weed/admin/view/app/plugin_templ.go
-
37weed/admin/view/app/template_helpers.go
-
46weed/admin/view/layout/layout.templ
-
364weed/admin/view/layout/layout_templ.go
-
12weed/plugin/worker/iceberg/exec_test.go
-
2weed/plugin/worker/iceberg/handler.go
-
131weed/plugin/worker/lifecycle/config.go
-
204weed/plugin/worker/lifecycle/detection.go
-
328weed/plugin/worker/lifecycle/execution.go
-
380weed/plugin/worker/lifecycle/handler.go
@ -0,0 +1,109 @@ |
|||
package plugin |
|||
|
|||
import ( |
|||
"sync" |
|||
"time" |
|||
) |
|||
|
|||
// SchedulerLane identifies an independent scheduling track. Each lane runs
|
|||
// its own goroutine, maintains its own detection timing, and acquires its
|
|||
// own admin lock so that workloads in different lanes never block each other.
|
|||
type SchedulerLane string |
|||
|
|||
const ( |
|||
// LaneDefault handles volume management operations (vacuum, balance,
|
|||
// erasure coding) and admin scripts. It is the fallback lane for any
|
|||
// job type that is not explicitly mapped elsewhere.
|
|||
LaneDefault SchedulerLane = "default" |
|||
|
|||
// LaneIceberg handles table-bucket Iceberg compaction and maintenance.
|
|||
LaneIceberg SchedulerLane = "iceberg" |
|||
|
|||
// LaneLifecycle handles S3 object store lifecycle management
|
|||
// (expiration, transition, abort incomplete multipart uploads).
|
|||
LaneLifecycle SchedulerLane = "lifecycle" |
|||
) |
|||
|
|||
// AllLanes returns every defined scheduler lane in a stable order.
|
|||
func AllLanes() []SchedulerLane { |
|||
return []SchedulerLane{LaneDefault, LaneIceberg, LaneLifecycle} |
|||
} |
|||
|
|||
// laneIdleSleep maps each lane to its default idle sleep duration.
|
|||
// Each lane can sleep for a different amount when no work is detected,
|
|||
// independent of the per-job-type DetectionInterval.
|
|||
var laneIdleSleep = map[SchedulerLane]time.Duration{ |
|||
LaneDefault: 61 * time.Second, |
|||
LaneIceberg: 61 * time.Second, |
|||
LaneLifecycle: 5 * time.Minute, |
|||
} |
|||
|
|||
// LaneIdleSleep returns the idle sleep duration for the given lane,
|
|||
// falling back to defaultSchedulerIdleSleep if the lane is unknown.
|
|||
func LaneIdleSleep(lane SchedulerLane) time.Duration { |
|||
if d, ok := laneIdleSleep[lane]; ok { |
|||
return d |
|||
} |
|||
return defaultSchedulerIdleSleep |
|||
} |
|||
|
|||
// jobTypeLaneMap is the hardcoded mapping from job type to scheduler lane.
|
|||
// Job types not present here are assigned to LaneDefault.
|
|||
var jobTypeLaneMap = map[string]SchedulerLane{ |
|||
// Volume management (default lane)
|
|||
"vacuum": LaneDefault, |
|||
"volume_balance": LaneDefault, |
|||
"ec_balance": LaneDefault, |
|||
"erasure_coding": LaneDefault, |
|||
"admin_script": LaneDefault, |
|||
|
|||
// Iceberg table maintenance
|
|||
"iceberg_maintenance": LaneIceberg, |
|||
|
|||
// S3 lifecycle management
|
|||
"s3_lifecycle": LaneLifecycle, |
|||
} |
|||
|
|||
// JobTypeLane returns the scheduler lane for the given job type.
|
|||
// Unknown job types are assigned to LaneDefault.
|
|||
func JobTypeLane(jobType string) SchedulerLane { |
|||
if lane, ok := jobTypeLaneMap[jobType]; ok { |
|||
return lane |
|||
} |
|||
return LaneDefault |
|||
} |
|||
|
|||
// LaneJobTypes returns the set of known job types assigned to the given lane.
|
|||
func LaneJobTypes(lane SchedulerLane) []string { |
|||
var result []string |
|||
for jobType, l := range jobTypeLaneMap { |
|||
if l == lane { |
|||
result = append(result, jobType) |
|||
} |
|||
} |
|||
return result |
|||
} |
|||
|
|||
// schedulerLaneState holds the per-lane runtime state used by the scheduler.
|
|||
type schedulerLaneState struct { |
|||
lane SchedulerLane |
|||
wakeCh chan struct{} |
|||
|
|||
loopMu sync.Mutex |
|||
loop schedulerLoopState |
|||
|
|||
// Per-lane execution reservation pool. Each lane tracks how many
|
|||
// execution slots it has reserved on each worker independently,
|
|||
// so lanes cannot starve each other.
|
|||
execMu sync.Mutex |
|||
execRes map[string]int |
|||
} |
|||
|
|||
// newLaneState creates a schedulerLaneState for the given lane.
|
|||
func newLaneState(lane SchedulerLane) *schedulerLaneState { |
|||
return &schedulerLaneState{ |
|||
lane: lane, |
|||
wakeCh: make(chan struct{}, 1), |
|||
execRes: make(map[string]int), |
|||
} |
|||
} |
|||
@ -0,0 +1,47 @@ |
|||
package plugin |
|||
|
|||
import ( |
|||
"testing" |
|||
) |
|||
|
|||
func TestJobTypeLaneMapCoversKnownTypes(t *testing.T) { |
|||
// Every job type in the map must resolve to a valid lane.
|
|||
for jobType, lane := range jobTypeLaneMap { |
|||
if lane != LaneDefault && lane != LaneIceberg && lane != LaneLifecycle { |
|||
t.Errorf("jobTypeLaneMap[%q] = %q, want a known lane", jobType, lane) |
|||
} |
|||
} |
|||
} |
|||
|
|||
func TestJobTypeLaneFallsBackToDefault(t *testing.T) { |
|||
if got := JobTypeLane("unknown_job_type"); got != LaneDefault { |
|||
t.Errorf("JobTypeLane(unknown) = %q, want %q", got, LaneDefault) |
|||
} |
|||
} |
|||
|
|||
func TestAllLanesHaveIdleSleep(t *testing.T) { |
|||
for _, lane := range AllLanes() { |
|||
if d := LaneIdleSleep(lane); d <= 0 { |
|||
t.Errorf("LaneIdleSleep(%q) = %v, want > 0", lane, d) |
|||
} |
|||
} |
|||
} |
|||
|
|||
func TestKnownJobTypesInMap(t *testing.T) { |
|||
// Ensure the well-known job types are mapped. This catches drift
|
|||
// if a handler's job type string changes without updating the map.
|
|||
expected := map[string]SchedulerLane{ |
|||
"vacuum": LaneDefault, |
|||
"volume_balance": LaneDefault, |
|||
"ec_balance": LaneDefault, |
|||
"erasure_coding": LaneDefault, |
|||
"admin_script": LaneDefault, |
|||
"iceberg_maintenance": LaneIceberg, |
|||
"s3_lifecycle": LaneLifecycle, |
|||
} |
|||
for jobType, wantLane := range expected { |
|||
if got := JobTypeLane(jobType); got != wantLane { |
|||
t.Errorf("JobTypeLane(%q) = %q, want %q", jobType, got, wantLane) |
|||
} |
|||
} |
|||
} |
|||
53
weed/admin/view/app/plugin_templ.go
File diff suppressed because it is too large
View File
File diff suppressed because it is too large
View File
@ -0,0 +1,131 @@ |
|||
package lifecycle |
|||
|
|||
import ( |
|||
"strconv" |
|||
"strings" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/glog" |
|||
"github.com/seaweedfs/seaweedfs/weed/pb/plugin_pb" |
|||
) |
|||
|
|||
const ( |
|||
jobType = "s3_lifecycle" |
|||
|
|||
defaultBatchSize = 1000 |
|||
defaultMaxDeletesPerBucket = 10000 |
|||
defaultDryRun = false |
|||
defaultDeleteMarkerCleanup = true |
|||
defaultAbortMPUDaysDefault = 7 |
|||
|
|||
MetricObjectsExpired = "objects_expired" |
|||
MetricObjectsScanned = "objects_scanned" |
|||
MetricBucketsScanned = "buckets_scanned" |
|||
MetricBucketsWithRules = "buckets_with_rules" |
|||
MetricDeleteMarkersClean = "delete_markers_cleaned" |
|||
MetricMPUAborted = "mpu_aborted" |
|||
MetricErrors = "errors" |
|||
MetricDurationMs = "duration_ms" |
|||
) |
|||
|
|||
// Config holds parsed worker config values for lifecycle management.
|
|||
type Config struct { |
|||
BatchSize int64 |
|||
MaxDeletesPerBucket int64 |
|||
DryRun bool |
|||
DeleteMarkerCleanup bool |
|||
AbortMPUDays int64 |
|||
} |
|||
|
|||
// ParseConfig extracts a lifecycle Config from plugin config values.
|
|||
func ParseConfig(values map[string]*plugin_pb.ConfigValue) Config { |
|||
cfg := Config{ |
|||
BatchSize: readInt64Config(values, "batch_size", defaultBatchSize), |
|||
MaxDeletesPerBucket: readInt64Config(values, "max_deletes_per_bucket", defaultMaxDeletesPerBucket), |
|||
DryRun: readBoolConfig(values, "dry_run", defaultDryRun), |
|||
DeleteMarkerCleanup: readBoolConfig(values, "delete_marker_cleanup", defaultDeleteMarkerCleanup), |
|||
AbortMPUDays: readInt64Config(values, "abort_mpu_days", defaultAbortMPUDaysDefault), |
|||
} |
|||
|
|||
if cfg.BatchSize <= 0 { |
|||
cfg.BatchSize = defaultBatchSize |
|||
} |
|||
if cfg.MaxDeletesPerBucket <= 0 { |
|||
cfg.MaxDeletesPerBucket = defaultMaxDeletesPerBucket |
|||
} |
|||
if cfg.AbortMPUDays < 0 { |
|||
cfg.AbortMPUDays = defaultAbortMPUDaysDefault |
|||
} |
|||
|
|||
return cfg |
|||
} |
|||
|
|||
func readStringConfig(values map[string]*plugin_pb.ConfigValue, field string, fallback string) string { |
|||
if values == nil { |
|||
return fallback |
|||
} |
|||
value := values[field] |
|||
if value == nil { |
|||
return fallback |
|||
} |
|||
switch kind := value.Kind.(type) { |
|||
case *plugin_pb.ConfigValue_StringValue: |
|||
return kind.StringValue |
|||
case *plugin_pb.ConfigValue_Int64Value: |
|||
return strconv.FormatInt(kind.Int64Value, 10) |
|||
default: |
|||
glog.V(1).Infof("readStringConfig: unexpected type %T for field %q", value.Kind, field) |
|||
} |
|||
return fallback |
|||
} |
|||
|
|||
func readBoolConfig(values map[string]*plugin_pb.ConfigValue, field string, fallback bool) bool { |
|||
if values == nil { |
|||
return fallback |
|||
} |
|||
value := values[field] |
|||
if value == nil { |
|||
return fallback |
|||
} |
|||
switch kind := value.Kind.(type) { |
|||
case *plugin_pb.ConfigValue_BoolValue: |
|||
return kind.BoolValue |
|||
case *plugin_pb.ConfigValue_StringValue: |
|||
s := strings.TrimSpace(strings.ToLower(kind.StringValue)) |
|||
if s == "true" || s == "1" || s == "yes" { |
|||
return true |
|||
} |
|||
if s == "false" || s == "0" || s == "no" { |
|||
return false |
|||
} |
|||
glog.V(1).Infof("readBoolConfig: unrecognized string value %q for field %q, using fallback %v", kind.StringValue, field, fallback) |
|||
case *plugin_pb.ConfigValue_Int64Value: |
|||
return kind.Int64Value != 0 |
|||
default: |
|||
glog.V(1).Infof("readBoolConfig: unexpected config value type %T for field %q, using fallback %v", value.Kind, field, fallback) |
|||
} |
|||
return fallback |
|||
} |
|||
|
|||
func readInt64Config(values map[string]*plugin_pb.ConfigValue, field string, fallback int64) int64 { |
|||
if values == nil { |
|||
return fallback |
|||
} |
|||
value := values[field] |
|||
if value == nil { |
|||
return fallback |
|||
} |
|||
switch kind := value.Kind.(type) { |
|||
case *plugin_pb.ConfigValue_Int64Value: |
|||
return kind.Int64Value |
|||
case *plugin_pb.ConfigValue_DoubleValue: |
|||
return int64(kind.DoubleValue) |
|||
case *plugin_pb.ConfigValue_StringValue: |
|||
parsed, err := strconv.ParseInt(strings.TrimSpace(kind.StringValue), 10, 64) |
|||
if err == nil { |
|||
return parsed |
|||
} |
|||
default: |
|||
glog.V(1).Infof("readInt64Config: unexpected config value type %T for field %q, using fallback %d", value.Kind, field, fallback) |
|||
} |
|||
return fallback |
|||
} |
|||
@ -0,0 +1,204 @@ |
|||
package lifecycle |
|||
|
|||
import ( |
|||
"context" |
|||
"fmt" |
|||
"path" |
|||
"strings" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/filer" |
|||
"github.com/seaweedfs/seaweedfs/weed/glog" |
|||
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb" |
|||
"github.com/seaweedfs/seaweedfs/weed/pb/plugin_pb" |
|||
"github.com/seaweedfs/seaweedfs/weed/util/wildcard" |
|||
) |
|||
|
|||
// detectBucketsWithLifecycleRules scans all S3 buckets to find those
|
|||
// with lifecycle (TTL) rules configured in filer.conf.
|
|||
func (h *Handler) detectBucketsWithLifecycleRules( |
|||
ctx context.Context, |
|||
filerClient filer_pb.SeaweedFilerClient, |
|||
config Config, |
|||
bucketFilter string, |
|||
maxResults int, |
|||
) ([]*plugin_pb.JobProposal, error) { |
|||
// Load filer configuration to find TTL rules.
|
|||
fc, err := loadFilerConf(ctx, filerClient) |
|||
if err != nil { |
|||
return nil, fmt.Errorf("load filer conf: %w", err) |
|||
} |
|||
|
|||
bucketsPath := defaultBucketsPath |
|||
bucketMatchers := wildcard.CompileWildcardMatchers(bucketFilter) |
|||
|
|||
// List all buckets.
|
|||
bucketEntries, err := listFilerEntries(ctx, filerClient, bucketsPath, "") |
|||
if err != nil { |
|||
return nil, fmt.Errorf("list buckets at %s: %w", bucketsPath, err) |
|||
} |
|||
|
|||
var proposals []*plugin_pb.JobProposal |
|||
for _, entry := range bucketEntries { |
|||
select { |
|||
case <-ctx.Done(): |
|||
return proposals, ctx.Err() |
|||
default: |
|||
} |
|||
|
|||
if !entry.IsDirectory { |
|||
continue |
|||
} |
|||
bucketName := entry.Name |
|||
if !wildcard.MatchesAnyWildcard(bucketMatchers, bucketName) { |
|||
continue |
|||
} |
|||
|
|||
// Derive the collection name for this bucket.
|
|||
collection := bucketName |
|||
ttls := fc.GetCollectionTtls(collection) |
|||
if len(ttls) == 0 { |
|||
continue |
|||
} |
|||
|
|||
glog.V(2).Infof("s3_lifecycle: bucket %s has %d lifecycle rule(s)", bucketName, len(ttls)) |
|||
|
|||
proposal := &plugin_pb.JobProposal{ |
|||
ProposalId: fmt.Sprintf("s3_lifecycle:%s", bucketName), |
|||
JobType: jobType, |
|||
Summary: fmt.Sprintf("Lifecycle management for bucket %s (%d rules)", bucketName, len(ttls)), |
|||
DedupeKey: fmt.Sprintf("s3_lifecycle:%s", bucketName), |
|||
Parameters: map[string]*plugin_pb.ConfigValue{ |
|||
"bucket": {Kind: &plugin_pb.ConfigValue_StringValue{StringValue: bucketName}}, |
|||
"buckets_path": {Kind: &plugin_pb.ConfigValue_StringValue{StringValue: bucketsPath}}, |
|||
"collection": {Kind: &plugin_pb.ConfigValue_StringValue{StringValue: collection}}, |
|||
"rule_count": {Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: int64(len(ttls))}}, |
|||
}, |
|||
Labels: map[string]string{ |
|||
"bucket": bucketName, |
|||
}, |
|||
} |
|||
|
|||
proposals = append(proposals, proposal) |
|||
if maxResults > 0 && len(proposals) >= maxResults { |
|||
break |
|||
} |
|||
} |
|||
|
|||
return proposals, nil |
|||
} |
|||
|
|||
const defaultBucketsPath = "/buckets" |
|||
|
|||
// loadFilerConf reads the filer configuration from the filer.
|
|||
func loadFilerConf(ctx context.Context, client filer_pb.SeaweedFilerClient) (*filer.FilerConf, error) { |
|||
fc := filer.NewFilerConf() |
|||
|
|||
content, err := filer.ReadInsideFiler(ctx, client, filer.DirectoryEtcSeaweedFS, filer.FilerConfName) |
|||
if err != nil { |
|||
// filer.conf may not exist yet - return empty config.
|
|||
glog.V(1).Infof("s3_lifecycle: filer.conf not found or unreadable: %v (using empty config)", err) |
|||
return fc, nil |
|||
} |
|||
if err := fc.LoadFromBytes(content); err != nil { |
|||
return nil, fmt.Errorf("parse filer.conf: %w", err) |
|||
} |
|||
|
|||
return fc, nil |
|||
} |
|||
|
|||
// listFilerEntries lists directory entries from the filer.
|
|||
func listFilerEntries(ctx context.Context, client filer_pb.SeaweedFilerClient, dir, startFrom string) ([]*filer_pb.Entry, error) { |
|||
var entries []*filer_pb.Entry |
|||
err := filer_pb.SeaweedList(ctx, client, dir, "", func(entry *filer_pb.Entry, isLast bool) error { |
|||
entries = append(entries, entry) |
|||
return nil |
|||
}, startFrom, false, 10000) |
|||
return entries, err |
|||
} |
|||
|
|||
type expiredObject struct { |
|||
dir string |
|||
name string |
|||
} |
|||
|
|||
// listExpiredObjects scans a bucket directory tree for objects whose TTL
|
|||
// has expired based on their TtlSec attribute set by PutBucketLifecycle.
|
|||
func listExpiredObjects( |
|||
ctx context.Context, |
|||
client filer_pb.SeaweedFilerClient, |
|||
bucketsPath, bucket string, |
|||
limit int64, |
|||
) ([]expiredObject, int64, error) { |
|||
var expired []expiredObject |
|||
var scanned int64 |
|||
|
|||
bucketPath := path.Join(bucketsPath, bucket) |
|||
|
|||
// Walk the bucket directory tree using breadth-first traversal.
|
|||
dirsToProcess := []string{bucketPath} |
|||
for len(dirsToProcess) > 0 { |
|||
select { |
|||
case <-ctx.Done(): |
|||
return expired, scanned, ctx.Err() |
|||
default: |
|||
} |
|||
|
|||
dir := dirsToProcess[0] |
|||
dirsToProcess = dirsToProcess[1:] |
|||
|
|||
limitReached := false |
|||
err := filer_pb.SeaweedList(ctx, client, dir, "", func(entry *filer_pb.Entry, isLast bool) error { |
|||
if entry.IsDirectory { |
|||
dirsToProcess = append(dirsToProcess, path.Join(dir, entry.Name)) |
|||
return nil |
|||
} |
|||
scanned++ |
|||
|
|||
if isExpiredByTTL(entry) { |
|||
expired = append(expired, expiredObject{ |
|||
dir: dir, |
|||
name: entry.Name, |
|||
}) |
|||
} |
|||
|
|||
if limit > 0 && int64(len(expired)) >= limit { |
|||
limitReached = true |
|||
return fmt.Errorf("limit reached") |
|||
} |
|||
return nil |
|||
}, "", false, 10000) |
|||
|
|||
if err != nil && !strings.Contains(err.Error(), "limit reached") { |
|||
return expired, scanned, fmt.Errorf("list %s: %w", dir, err) |
|||
} |
|||
|
|||
if limitReached || (limit > 0 && int64(len(expired)) >= limit) { |
|||
break |
|||
} |
|||
} |
|||
|
|||
return expired, scanned, nil |
|||
} |
|||
|
|||
// isExpiredByTTL checks if an entry is expired based on its TTL attribute.
|
|||
// SeaweedFS sets TtlSec on entries when lifecycle rules are applied via
|
|||
// PutBucketLifecycleConfiguration. An entry is expired when
|
|||
// creation_time + TTL < now.
|
|||
func isExpiredByTTL(entry *filer_pb.Entry) bool { |
|||
if entry == nil || entry.Attributes == nil { |
|||
return false |
|||
} |
|||
|
|||
ttlSec := entry.Attributes.TtlSec |
|||
if ttlSec <= 0 { |
|||
return false |
|||
} |
|||
|
|||
crTime := entry.Attributes.Crtime |
|||
if crTime <= 0 { |
|||
return false |
|||
} |
|||
|
|||
expirationUnix := crTime + int64(ttlSec) |
|||
return expirationUnix < nowUnix() |
|||
} |
|||
@ -0,0 +1,328 @@ |
|||
package lifecycle |
|||
|
|||
import ( |
|||
"context" |
|||
"fmt" |
|||
"math" |
|||
"path" |
|||
"strings" |
|||
"time" |
|||
|
|||
"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" |
|||
"github.com/seaweedfs/seaweedfs/weed/s3api/s3_constants" |
|||
) |
|||
|
|||
type executionResult struct { |
|||
objectsExpired int64 |
|||
objectsScanned int64 |
|||
deleteMarkersClean int64 |
|||
mpuAborted int64 |
|||
errors int64 |
|||
} |
|||
|
|||
// executeLifecycleForBucket processes lifecycle rules for a single bucket:
|
|||
// 1. Reads filer.conf to get TTL rules for the bucket's collection
|
|||
// 2. Walks the bucket directory tree to find expired objects
|
|||
// 3. Deletes expired objects (unless dry run)
|
|||
func (h *Handler) executeLifecycleForBucket( |
|||
ctx context.Context, |
|||
filerClient filer_pb.SeaweedFilerClient, |
|||
config Config, |
|||
bucket, bucketsPath string, |
|||
sender pluginworker.ExecutionSender, |
|||
jobID string, |
|||
) (*executionResult, error) { |
|||
result := &executionResult{} |
|||
|
|||
// Load filer.conf to verify TTL rules still exist.
|
|||
fc, err := loadFilerConf(ctx, filerClient) |
|||
if err != nil { |
|||
return result, fmt.Errorf("load filer conf: %w", err) |
|||
} |
|||
|
|||
collection := bucket |
|||
ttlRules := fc.GetCollectionTtls(collection) |
|||
if len(ttlRules) == 0 { |
|||
glog.V(1).Infof("s3_lifecycle: bucket %s has no lifecycle rules, skipping", bucket) |
|||
return result, nil |
|||
} |
|||
|
|||
_ = sender.SendProgress(&plugin_pb.JobProgressUpdate{ |
|||
JobId: jobID, |
|||
JobType: jobType, |
|||
State: plugin_pb.JobState_JOB_STATE_RUNNING, |
|||
ProgressPercent: 10, |
|||
Stage: "scanning", |
|||
Message: fmt.Sprintf("scanning bucket %s for expired objects (%d rules)", bucket, len(ttlRules)), |
|||
}) |
|||
|
|||
// Shared budget across all phases so we don't exceed MaxDeletesPerBucket.
|
|||
remaining := config.MaxDeletesPerBucket |
|||
|
|||
// Find expired objects.
|
|||
expired, scanned, err := listExpiredObjects(ctx, filerClient, bucketsPath, bucket, remaining) |
|||
result.objectsScanned = scanned |
|||
if err != nil { |
|||
return result, fmt.Errorf("list expired objects: %w", err) |
|||
} |
|||
|
|||
if len(expired) > 0 { |
|||
glog.V(1).Infof("s3_lifecycle: bucket %s: found %d expired objects out of %d scanned", bucket, len(expired), scanned) |
|||
} else { |
|||
glog.V(1).Infof("s3_lifecycle: bucket %s: scanned %d objects, none expired", bucket, scanned) |
|||
} |
|||
|
|||
if config.DryRun && len(expired) > 0 { |
|||
result.objectsExpired = int64(len(expired)) |
|||
_ = sender.SendProgress(&plugin_pb.JobProgressUpdate{ |
|||
JobId: jobID, |
|||
JobType: jobType, |
|||
State: plugin_pb.JobState_JOB_STATE_RUNNING, |
|||
ProgressPercent: 100, |
|||
Stage: "dry_run", |
|||
Message: fmt.Sprintf("dry run: would delete %d expired objects", len(expired)), |
|||
}) |
|||
return result, nil |
|||
} |
|||
|
|||
// Delete expired objects in batches.
|
|||
if len(expired) > 0 { |
|||
_ = sender.SendProgress(&plugin_pb.JobProgressUpdate{ |
|||
JobId: jobID, |
|||
JobType: jobType, |
|||
State: plugin_pb.JobState_JOB_STATE_RUNNING, |
|||
ProgressPercent: 50, |
|||
Stage: "deleting", |
|||
Message: fmt.Sprintf("deleting %d expired objects", len(expired)), |
|||
}) |
|||
|
|||
var batchSize int |
|||
if config.BatchSize <= 0 { |
|||
batchSize = defaultBatchSize |
|||
} else if config.BatchSize > math.MaxInt { |
|||
batchSize = math.MaxInt |
|||
} else { |
|||
batchSize = int(config.BatchSize) |
|||
} |
|||
|
|||
for i := 0; i < len(expired); i += batchSize { |
|||
select { |
|||
case <-ctx.Done(): |
|||
return result, ctx.Err() |
|||
default: |
|||
} |
|||
|
|||
end := i + batchSize |
|||
if end > len(expired) { |
|||
end = len(expired) |
|||
} |
|||
batch := expired[i:end] |
|||
|
|||
deleted, errs, batchErr := deleteExpiredObjects(ctx, filerClient, batch) |
|||
result.objectsExpired += int64(deleted) |
|||
result.errors += int64(errs) |
|||
|
|||
if batchErr != nil { |
|||
return result, batchErr |
|||
} |
|||
|
|||
progress := float64(end)/float64(len(expired))*50 + 50 // 50-100%
|
|||
_ = sender.SendProgress(&plugin_pb.JobProgressUpdate{ |
|||
JobId: jobID, |
|||
JobType: jobType, |
|||
State: plugin_pb.JobState_JOB_STATE_RUNNING, |
|||
ProgressPercent: progress, |
|||
Stage: "deleting", |
|||
Message: fmt.Sprintf("deleted %d/%d expired objects", result.objectsExpired, len(expired)), |
|||
}) |
|||
} |
|||
|
|||
remaining -= result.objectsExpired + result.errors |
|||
if remaining < 0 { |
|||
remaining = 0 |
|||
} |
|||
} |
|||
|
|||
// Delete marker cleanup.
|
|||
if config.DeleteMarkerCleanup && remaining > 0 { |
|||
_ = sender.SendProgress(&plugin_pb.JobProgressUpdate{ |
|||
JobId: jobID, JobType: jobType, |
|||
State: plugin_pb.JobState_JOB_STATE_RUNNING, |
|||
Stage: "cleaning_delete_markers", Message: "cleaning expired delete markers", |
|||
}) |
|||
cleaned, cleanErrs, cleanCtxErr := cleanupDeleteMarkers(ctx, filerClient, bucketsPath, bucket, remaining) |
|||
result.deleteMarkersClean = int64(cleaned) |
|||
result.errors += int64(cleanErrs) |
|||
if cleanCtxErr != nil { |
|||
return result, cleanCtxErr |
|||
} |
|||
remaining -= int64(cleaned + cleanErrs) |
|||
if remaining < 0 { |
|||
remaining = 0 |
|||
} |
|||
} |
|||
|
|||
// Abort incomplete multipart uploads.
|
|||
if config.AbortMPUDays > 0 && remaining > 0 { |
|||
_ = sender.SendProgress(&plugin_pb.JobProgressUpdate{ |
|||
JobId: jobID, JobType: jobType, |
|||
State: plugin_pb.JobState_JOB_STATE_RUNNING, |
|||
Stage: "aborting_mpus", Message: fmt.Sprintf("aborting multipart uploads older than %d days", config.AbortMPUDays), |
|||
}) |
|||
aborted, abortErrs, abortCtxErr := abortIncompleteMPUs(ctx, filerClient, bucketsPath, bucket, config.AbortMPUDays, remaining) |
|||
result.mpuAborted = int64(aborted) |
|||
result.errors += int64(abortErrs) |
|||
if abortCtxErr != nil { |
|||
return result, abortCtxErr |
|||
} |
|||
} |
|||
|
|||
return result, nil |
|||
} |
|||
|
|||
// cleanupDeleteMarkers scans the bucket for entries marked as delete markers
|
|||
// (via the S3 versioning extended attribute) and removes them.
|
|||
//
|
|||
// NOTE: This currently removes delete markers unconditionally without checking
|
|||
// whether prior non-expired versions exist. In versioned buckets, removing a
|
|||
// delete marker can resurface an older version. A future enhancement should
|
|||
// query version metadata before removal to match AWS ExpiredObjectDeleteMarker
|
|||
// semantics (only remove when no non-current versions remain).
|
|||
func cleanupDeleteMarkers( |
|||
ctx context.Context, |
|||
client filer_pb.SeaweedFilerClient, |
|||
bucketsPath, bucket string, |
|||
limit int64, |
|||
) (cleaned, errors int, ctxErr error) { |
|||
bucketPath := path.Join(bucketsPath, bucket) |
|||
|
|||
dirsToProcess := []string{bucketPath} |
|||
for len(dirsToProcess) > 0 { |
|||
if ctx.Err() != nil { |
|||
return cleaned, errors, ctx.Err() |
|||
} |
|||
|
|||
dir := dirsToProcess[0] |
|||
dirsToProcess = dirsToProcess[1:] |
|||
|
|||
listErr := filer_pb.SeaweedList(ctx, client, dir, "", func(entry *filer_pb.Entry, isLast bool) error { |
|||
if entry.IsDirectory { |
|||
// Skip .uploads directories.
|
|||
if entry.Name != ".uploads" { |
|||
dirsToProcess = append(dirsToProcess, path.Join(dir, entry.Name)) |
|||
} |
|||
return nil |
|||
} |
|||
|
|||
if isDeleteMarker(entry) { |
|||
if err := filer_pb.DoRemove(ctx, client, dir, entry.Name, true, false, false, false, nil); err != nil { |
|||
glog.V(1).Infof("s3_lifecycle: failed to remove delete marker %s/%s: %v", dir, entry.Name, err) |
|||
errors++ |
|||
} else { |
|||
cleaned++ |
|||
} |
|||
} |
|||
|
|||
if limit > 0 && int64(cleaned+errors) >= limit { |
|||
return fmt.Errorf("limit reached") |
|||
} |
|||
return nil |
|||
}, "", false, 10000) |
|||
|
|||
if listErr != nil && !strings.Contains(listErr.Error(), "limit reached") { |
|||
return cleaned, errors, fmt.Errorf("list %s: %w", dir, listErr) |
|||
} |
|||
|
|||
if limit > 0 && int64(cleaned+errors) >= limit { |
|||
break |
|||
} |
|||
} |
|||
return cleaned, errors, nil |
|||
} |
|||
|
|||
// isDeleteMarker checks if an entry is an S3 delete marker.
|
|||
func isDeleteMarker(entry *filer_pb.Entry) bool { |
|||
if entry == nil || entry.Extended == nil { |
|||
return false |
|||
} |
|||
return string(entry.Extended[s3_constants.ExtDeleteMarkerKey]) == "true" |
|||
} |
|||
|
|||
// abortIncompleteMPUs scans the .uploads directory under a bucket and
|
|||
// removes multipart upload entries older than the specified number of days.
|
|||
func abortIncompleteMPUs( |
|||
ctx context.Context, |
|||
client filer_pb.SeaweedFilerClient, |
|||
bucketsPath, bucket string, |
|||
olderThanDays, limit int64, |
|||
) (aborted, errors int, ctxErr error) { |
|||
uploadsDir := path.Join(bucketsPath, bucket, ".uploads") |
|||
cutoff := time.Now().Add(-time.Duration(olderThanDays) * 24 * time.Hour) |
|||
|
|||
listErr := filer_pb.SeaweedList(ctx, client, uploadsDir, "", func(entry *filer_pb.Entry, isLast bool) error { |
|||
if ctx.Err() != nil { |
|||
return ctx.Err() |
|||
} |
|||
|
|||
if !entry.IsDirectory { |
|||
return nil |
|||
} |
|||
|
|||
// Each subdirectory under .uploads is one multipart upload.
|
|||
// Check the directory creation time.
|
|||
if entry.Attributes != nil && entry.Attributes.Crtime > 0 { |
|||
created := time.Unix(entry.Attributes.Crtime, 0) |
|||
if created.Before(cutoff) { |
|||
uploadPath := path.Join(uploadsDir, entry.Name) |
|||
if err := filer_pb.DoRemove(ctx, client, uploadsDir, entry.Name, true, true, true, false, nil); err != nil { |
|||
glog.V(1).Infof("s3_lifecycle: failed to abort MPU %s: %v", uploadPath, err) |
|||
errors++ |
|||
} else { |
|||
aborted++ |
|||
} |
|||
} |
|||
} |
|||
|
|||
if limit > 0 && int64(aborted+errors) >= limit { |
|||
return fmt.Errorf("limit reached") |
|||
} |
|||
return nil |
|||
}, "", false, 10000) |
|||
|
|||
if listErr != nil && !strings.Contains(listErr.Error(), "limit reached") { |
|||
return aborted, errors, fmt.Errorf("list uploads in %s: %w", uploadsDir, listErr) |
|||
} |
|||
|
|||
return aborted, errors, nil |
|||
} |
|||
|
|||
// deleteExpiredObjects deletes a batch of expired objects from the filer.
|
|||
// Returns a non-nil error when the context is canceled mid-batch.
|
|||
func deleteExpiredObjects( |
|||
ctx context.Context, |
|||
client filer_pb.SeaweedFilerClient, |
|||
objects []expiredObject, |
|||
) (deleted, errors int, ctxErr error) { |
|||
for _, obj := range objects { |
|||
if ctx.Err() != nil { |
|||
return deleted, errors, ctx.Err() |
|||
} |
|||
|
|||
err := filer_pb.DoRemove(ctx, client, obj.dir, obj.name, true, false, false, false, nil) |
|||
if err != nil { |
|||
glog.V(1).Infof("s3_lifecycle: failed to delete %s/%s: %v", obj.dir, obj.name, err) |
|||
errors++ |
|||
continue |
|||
} |
|||
deleted++ |
|||
} |
|||
return deleted, errors, nil |
|||
} |
|||
|
|||
// nowUnix returns the current time as a Unix timestamp.
|
|||
func nowUnix() int64 { |
|||
return time.Now().Unix() |
|||
} |
|||
@ -0,0 +1,380 @@ |
|||
package lifecycle |
|||
|
|||
import ( |
|||
"context" |
|||
"fmt" |
|||
"strings" |
|||
"time" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/glog" |
|||
"github.com/seaweedfs/seaweedfs/weed/pb" |
|||
"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{"lifecycle", "s3-lifecycle", "s3.lifecycle"}, |
|||
Build: func(opts pluginworker.HandlerBuildOptions) (pluginworker.JobHandler, error) { |
|||
return NewHandler(opts.GrpcDialOption), nil |
|||
}, |
|||
}) |
|||
} |
|||
|
|||
// Handler implements the JobHandler interface for S3 lifecycle management:
|
|||
// object expiration, delete marker cleanup, and abort incomplete multipart uploads.
|
|||
type Handler struct { |
|||
grpcDialOption grpc.DialOption |
|||
} |
|||
|
|||
const filerConnectTimeout = 5 * time.Second |
|||
|
|||
// NewHandler creates a new handler for S3 lifecycle management.
|
|||
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: "S3 Lifecycle", |
|||
Description: "Manages S3 object lifecycle: expiration of objects based on TTL rules, delete marker cleanup, and abort of incomplete multipart uploads", |
|||
Weight: 40, |
|||
} |
|||
} |
|||
|
|||
func (h *Handler) Descriptor() *plugin_pb.JobTypeDescriptor { |
|||
return &plugin_pb.JobTypeDescriptor{ |
|||
JobType: jobType, |
|||
DisplayName: "S3 Lifecycle Management", |
|||
Description: "Automated S3 object lifecycle management: expire objects by TTL rules, clean up expired delete markers, and abort stale multipart uploads", |
|||
Icon: "fas fa-hourglass-half", |
|||
DescriptorVersion: 1, |
|||
AdminConfigForm: &plugin_pb.ConfigForm{ |
|||
FormId: "s3-lifecycle-admin", |
|||
Title: "S3 Lifecycle Admin Config", |
|||
Description: "Admin-side controls for S3 lifecycle management scope.", |
|||
Sections: []*plugin_pb.ConfigSection{ |
|||
{ |
|||
SectionId: "scope", |
|||
Title: "Scope", |
|||
Description: "Which buckets to include in lifecycle management.", |
|||
Fields: []*plugin_pb.ConfigField{ |
|||
{ |
|||
Name: "bucket_filter", |
|||
Label: "Bucket Filter", |
|||
Description: "Wildcard pattern for bucket names to include (e.g. \"prod-*\"). Empty means all buckets.", |
|||
FieldType: plugin_pb.ConfigFieldType_CONFIG_FIELD_TYPE_STRING, |
|||
Widget: plugin_pb.ConfigWidget_CONFIG_WIDGET_TEXT, |
|||
}, |
|||
}, |
|||
}, |
|||
}, |
|||
}, |
|||
WorkerConfigForm: &plugin_pb.ConfigForm{ |
|||
FormId: "s3-lifecycle-worker", |
|||
Title: "S3 Lifecycle Worker Config", |
|||
Description: "Worker-side controls for lifecycle execution behavior.", |
|||
Sections: []*plugin_pb.ConfigSection{ |
|||
{ |
|||
SectionId: "execution", |
|||
Title: "Execution", |
|||
Description: "Controls for lifecycle rule execution.", |
|||
Fields: []*plugin_pb.ConfigField{ |
|||
{ |
|||
Name: "batch_size", |
|||
Label: "Batch Size", |
|||
Description: "Number of entries to process per filer listing page.", |
|||
FieldType: plugin_pb.ConfigFieldType_CONFIG_FIELD_TYPE_INT64, |
|||
Widget: plugin_pb.ConfigWidget_CONFIG_WIDGET_NUMBER, |
|||
MinValue: configInt64(100), |
|||
MaxValue: configInt64(10000), |
|||
}, |
|||
{ |
|||
Name: "max_deletes_per_bucket", |
|||
Label: "Max Deletes Per Bucket", |
|||
Description: "Maximum number of expired objects to delete per bucket in one execution run.", |
|||
FieldType: plugin_pb.ConfigFieldType_CONFIG_FIELD_TYPE_INT64, |
|||
Widget: plugin_pb.ConfigWidget_CONFIG_WIDGET_NUMBER, |
|||
MinValue: configInt64(100), |
|||
MaxValue: configInt64(1000000), |
|||
}, |
|||
{ |
|||
Name: "dry_run", |
|||
Label: "Dry Run", |
|||
Description: "When enabled, detect expired objects but do not delete them.", |
|||
FieldType: plugin_pb.ConfigFieldType_CONFIG_FIELD_TYPE_BOOL, |
|||
Widget: plugin_pb.ConfigWidget_CONFIG_WIDGET_TOGGLE, |
|||
}, |
|||
{ |
|||
Name: "delete_marker_cleanup", |
|||
Label: "Delete Marker Cleanup", |
|||
Description: "Remove expired delete markers that have no non-current versions.", |
|||
FieldType: plugin_pb.ConfigFieldType_CONFIG_FIELD_TYPE_BOOL, |
|||
Widget: plugin_pb.ConfigWidget_CONFIG_WIDGET_TOGGLE, |
|||
}, |
|||
{ |
|||
Name: "abort_mpu_days", |
|||
Label: "Abort Incomplete MPU (days)", |
|||
Description: "Abort incomplete multipart uploads older than this many days. 0 disables.", |
|||
FieldType: plugin_pb.ConfigFieldType_CONFIG_FIELD_TYPE_INT64, |
|||
Widget: plugin_pb.ConfigWidget_CONFIG_WIDGET_NUMBER, |
|||
MinValue: configInt64(0), |
|||
MaxValue: configInt64(365), |
|||
}, |
|||
}, |
|||
}, |
|||
}, |
|||
}, |
|||
AdminRuntimeDefaults: &plugin_pb.AdminRuntimeDefaults{ |
|||
Enabled: true, |
|||
DetectionIntervalSeconds: 300, // 5 minutes
|
|||
DetectionTimeoutSeconds: 60, |
|||
MaxJobsPerDetection: 100, |
|||
GlobalExecutionConcurrency: 2, |
|||
PerWorkerExecutionConcurrency: 2, |
|||
RetryLimit: 1, |
|||
RetryBackoffSeconds: 10, |
|||
}, |
|||
WorkerDefaultValues: map[string]*plugin_pb.ConfigValue{ |
|||
"batch_size": {Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: defaultBatchSize}}, |
|||
"max_deletes_per_bucket": {Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: defaultMaxDeletesPerBucket}}, |
|||
"dry_run": {Kind: &plugin_pb.ConfigValue_BoolValue{BoolValue: defaultDryRun}}, |
|||
"delete_marker_cleanup": {Kind: &plugin_pb.ConfigValue_BoolValue{BoolValue: defaultDeleteMarkerCleanup}}, |
|||
"abort_mpu_days": {Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: defaultAbortMPUDaysDefault}}, |
|||
}, |
|||
} |
|||
} |
|||
|
|||
func (h *Handler) Detect(ctx context.Context, req *plugin_pb.RunDetectionRequest, sender pluginworker.DetectionSender) error { |
|||
if req == nil { |
|||
return fmt.Errorf("nil detection request") |
|||
} |
|||
|
|||
config := ParseConfig(req.WorkerConfigValues) |
|||
|
|||
bucketFilter := readStringConfig(req.AdminConfigValues, "bucket_filter", "") |
|||
|
|||
filerAddresses := filerAddressesFromCluster(req.ClusterContext) |
|||
if len(filerAddresses) == 0 { |
|||
_ = sender.SendActivity(pluginworker.BuildDetectorActivity("skipped", "no filer addresses in cluster context", nil)) |
|||
return sendEmptyDetection(sender) |
|||
} |
|||
|
|||
_ = sender.SendActivity(pluginworker.BuildDetectorActivity("connecting", "connecting to filer", nil)) |
|||
|
|||
filerClient, filerConn, err := connectToFiler(ctx, filerAddresses, h.grpcDialOption) |
|||
if err != nil { |
|||
return fmt.Errorf("failed to connect to any filer: %v", err) |
|||
} |
|||
defer filerConn.Close() |
|||
|
|||
maxResults := int(req.MaxResults) |
|||
if maxResults <= 0 { |
|||
maxResults = 100 |
|||
} |
|||
|
|||
_ = sender.SendActivity(pluginworker.BuildDetectorActivity("scanning", "scanning buckets for lifecycle rules", nil)) |
|||
proposals, err := h.detectBucketsWithLifecycleRules(ctx, filerClient, config, bucketFilter, maxResults) |
|||
if err != nil { |
|||
_ = sender.SendActivity(pluginworker.BuildDetectorActivity("scan_error", fmt.Sprintf("error scanning buckets: %v", err), nil)) |
|||
return fmt.Errorf("detect lifecycle rules: %w", err) |
|||
} |
|||
|
|||
_ = sender.SendActivity(pluginworker.BuildDetectorActivity("scan_complete", |
|||
fmt.Sprintf("found %d bucket(s) with lifecycle rules", len(proposals)), |
|||
map[string]*plugin_pb.ConfigValue{ |
|||
"buckets_found": {Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: int64(len(proposals))}}, |
|||
})) |
|||
|
|||
if err := sender.SendProposals(&plugin_pb.DetectionProposals{ |
|||
JobType: jobType, |
|||
Proposals: proposals, |
|||
HasMore: len(proposals) >= maxResults, |
|||
}); 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, req *plugin_pb.ExecuteJobRequest, sender pluginworker.ExecutionSender) error { |
|||
if req == nil || req.Job == nil { |
|||
return fmt.Errorf("nil execution request") |
|||
} |
|||
|
|||
job := req.Job |
|||
config := ParseConfig(req.WorkerConfigValues) |
|||
|
|||
bucket := readParamString(job.Parameters, "bucket") |
|||
bucketsPath := readParamString(job.Parameters, "buckets_path") |
|||
if bucket == "" || bucketsPath == "" { |
|||
return fmt.Errorf("missing bucket or buckets_path parameter") |
|||
} |
|||
|
|||
filerAddresses := filerAddressesFromCluster(req.ClusterContext) |
|||
if len(filerAddresses) == 0 { |
|||
return fmt.Errorf("no filer addresses in cluster context") |
|||
} |
|||
|
|||
filerClient, filerConn, err := connectToFiler(ctx, filerAddresses, h.grpcDialOption) |
|||
if err != nil { |
|||
return fmt.Errorf("failed to connect to any filer: %v", err) |
|||
} |
|||
defer filerConn.Close() |
|||
|
|||
_ = sender.SendProgress(&plugin_pb.JobProgressUpdate{ |
|||
JobId: job.JobId, |
|||
JobType: jobType, |
|||
State: plugin_pb.JobState_JOB_STATE_ASSIGNED, |
|||
ProgressPercent: 0, |
|||
Stage: "starting", |
|||
Message: fmt.Sprintf("executing lifecycle rules for bucket %s", bucket), |
|||
}) |
|||
|
|||
start := time.Now() |
|||
result, execErr := h.executeLifecycleForBucket(ctx, filerClient, config, bucket, bucketsPath, sender, job.JobId) |
|||
elapsed := time.Since(start) |
|||
|
|||
metrics := map[string]*plugin_pb.ConfigValue{ |
|||
MetricDurationMs: {Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: elapsed.Milliseconds()}}, |
|||
} |
|||
if result != nil { |
|||
metrics[MetricObjectsExpired] = &plugin_pb.ConfigValue{Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: result.objectsExpired}} |
|||
metrics[MetricObjectsScanned] = &plugin_pb.ConfigValue{Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: result.objectsScanned}} |
|||
metrics[MetricDeleteMarkersClean] = &plugin_pb.ConfigValue{Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: result.deleteMarkersClean}} |
|||
metrics[MetricMPUAborted] = &plugin_pb.ConfigValue{Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: result.mpuAborted}} |
|||
metrics[MetricErrors] = &plugin_pb.ConfigValue{Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: result.errors}} |
|||
} |
|||
|
|||
var scanned, expired int64 |
|||
if result != nil { |
|||
scanned = result.objectsScanned |
|||
expired = result.objectsExpired |
|||
} |
|||
|
|||
success := execErr == nil && (result == nil || result.errors == 0) |
|||
message := fmt.Sprintf("bucket %s: scanned %d objects, expired %d", bucket, scanned, expired) |
|||
if result != nil && result.deleteMarkersClean > 0 { |
|||
message += fmt.Sprintf(", delete markers cleaned %d", result.deleteMarkersClean) |
|||
} |
|||
if result != nil && result.mpuAborted > 0 { |
|||
message += fmt.Sprintf(", MPUs aborted %d", result.mpuAborted) |
|||
} |
|||
if config.DryRun { |
|||
message += " (dry run)" |
|||
} |
|||
if result != nil && result.errors > 0 { |
|||
message += fmt.Sprintf(" (%d errors)", result.errors) |
|||
} |
|||
if execErr != nil { |
|||
message = fmt.Sprintf("lifecycle execution failed for bucket %s: %v", bucket, execErr) |
|||
} |
|||
|
|||
errMsg := "" |
|||
if execErr != nil { |
|||
errMsg = execErr.Error() |
|||
} else if result != nil && result.errors > 0 { |
|||
errMsg = fmt.Sprintf("%d objects failed to process", result.errors) |
|||
} |
|||
|
|||
return sender.SendCompleted(&plugin_pb.JobCompleted{ |
|||
JobId: job.JobId, |
|||
JobType: jobType, |
|||
Success: success, |
|||
ErrorMessage: errMsg, |
|||
Result: &plugin_pb.JobResult{ |
|||
Summary: message, |
|||
OutputValues: metrics, |
|||
}, |
|||
CompletedAt: timestamppb.Now(), |
|||
}) |
|||
} |
|||
|
|||
func connectToFiler(ctx context.Context, addresses []string, dialOption grpc.DialOption) (filer_pb.SeaweedFilerClient, *grpc.ClientConn, error) { |
|||
var lastErr error |
|||
for _, addr := range addresses { |
|||
grpcAddr := pb.ServerAddress(addr).ToGrpcAddress() |
|||
connCtx, cancel := context.WithTimeout(ctx, filerConnectTimeout) |
|||
conn, err := pb.GrpcDial(connCtx, grpcAddr, false, dialOption) |
|||
cancel() |
|||
if err != nil { |
|||
lastErr = err |
|||
glog.V(1).Infof("s3_lifecycle: failed to connect to filer %s (grpc %s): %v", addr, grpcAddr, err) |
|||
continue |
|||
} |
|||
// Verify the connection with a ping.
|
|||
client := filer_pb.NewSeaweedFilerClient(conn) |
|||
pingCtx, pingCancel := context.WithTimeout(ctx, filerConnectTimeout) |
|||
_, pingErr := client.Ping(pingCtx, &filer_pb.PingRequest{}) |
|||
pingCancel() |
|||
if pingErr != nil { |
|||
_ = conn.Close() |
|||
lastErr = pingErr |
|||
glog.V(1).Infof("s3_lifecycle: filer %s ping failed: %v", grpcAddr, pingErr) |
|||
continue |
|||
} |
|||
return client, conn, nil |
|||
} |
|||
return nil, nil, lastErr |
|||
} |
|||
|
|||
func 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, |
|||
}) |
|||
} |
|||
|
|||
func filerAddressesFromCluster(cc *plugin_pb.ClusterContext) []string { |
|||
if cc == nil { |
|||
return nil |
|||
} |
|||
var addrs []string |
|||
for _, addr := range cc.FilerGrpcAddresses { |
|||
trimmed := strings.TrimSpace(addr) |
|||
if trimmed != "" { |
|||
addrs = append(addrs, trimmed) |
|||
} |
|||
} |
|||
return addrs |
|||
} |
|||
|
|||
func readParamString(params map[string]*plugin_pb.ConfigValue, key string) string { |
|||
if params == nil { |
|||
return "" |
|||
} |
|||
v := params[key] |
|||
if v == nil { |
|||
return "" |
|||
} |
|||
if sv, ok := v.Kind.(*plugin_pb.ConfigValue_StringValue); ok { |
|||
return sv.StringValue |
|||
} |
|||
return "" |
|||
} |
|||
|
|||
func configInt64(v int64) *plugin_pb.ConfigValue { |
|||
return &plugin_pb.ConfigValue{Kind: &plugin_pb.ConfigValue_Int64Value{Int64Value: v}} |
|||
} |
|||
Write
Preview
Loading…
Cancel
Save
Reference in new issue