Commit Graph

150 Commits

Author SHA1 Message Date
Chris Lu
75a6a34528 dlm: resilient distributed locks via consistent hashing + backup replication (#8860)
* dlm: replace modulo hashing with consistent hash ring

Introduce HashRing with virtual nodes (CRC32-based consistent hashing)
to replace the modulo-based hashKeyToServer. When a filer node is
removed, only keys that hashed to that node are remapped to the next
server on the ring, leaving all other mappings stable. This is the
foundation for backup replication — the successor on the ring is
always the natural takeover node.

* dlm: add Generation and IsBackup fields to Lock

Lock now carries IsBackup (whether this node holds the lock as a backup
replica) and Generation (a monotonic fencing token that increments on
each fresh acquisition, stays the same on renewal). Add helper methods:
AllLocks, PromoteLock, DemoteLock, InsertBackupLock, RemoveLock, GetLock.

* dlm: add ReplicateLock RPC and generation/is_backup proto fields

Add generation field to LockResponse for fencing tokens.
Add generation and is_backup fields to Lock message.
Add ReplicateLock RPC for primary-to-backup lock replication.
Add ReplicateLockRequest/ReplicateLockResponse messages.

* dlm: add async backup replication to DistributedLockManager

Route lock/unlock via consistent hash ring's GetPrimaryAndBackup().
After a successful lock or unlock on the primary, asynchronously
replicate the operation to the backup server via ReplicateFunc
callback. Single-server deployments skip replication.

* dlm: add ReplicateLock handler and backup-aware topology changes

Add ReplicateLock gRPC handler for primary-to-backup replication.
Revise OnDlmChangeSnapshot to handle three cases on topology change:
- Promote backup locks when this node becomes primary
- Demote primary locks when this node becomes backup
- Transfer locks when this node is neither primary nor backup
Wire up SetupDlmReplication during filer server initialization.

* dlm: expose generation fencing token in lock client

LiveLock now captures the generation from LockResponse and exposes it
via Generation() method. Consumers can use this as a fencing token to
detect stale lock holders.

* dlm: update empty folder cleaner to use consistent hash ring

Replace local modulo-based hashKeyToServer with LockRing.GetPrimary()
which uses the shared consistent hash ring for folder ownership.

* dlm: add unit tests for consistent hash ring

Test basic operations, consistency on server removal (only keys from
removed server move), backup-is-successor property (backup becomes
new primary when primary is removed), and key distribution balance.

* dlm: add integration tests for lock replication failure scenarios

Test cases:
- Primary crash with backup promotion (backup has valid token)
- Backup crash with primary continuing
- Both primary and backup crash (lock lost, re-acquirable)
- Rolling restart across all nodes
- Generation fencing token increments on new acquisition
- Replication failure (primary still works independently)
- Unlock replicates deletion to backup
- Lock survives server addition (topology change)
- Consistent hashing minimal disruption (only removed server's keys move)

* dlm: address PR review findings

1. Causal replication ordering: Add per-lock sequence number (Seq) that
   increments on every mutation. Backup rejects incoming mutations with
   seq <= current seq, preventing stale async replications from
   overwriting newer state. Unlock replication also carries seq and is
   rejected if stale.

2. Demote-after-handoff: OnDlmChangeSnapshot now transfers the lock to
   the new primary first and only demotes to backup after a successful
   TransferLocks RPC. If the transfer fails, the lock stays as primary
   on this node.

3. SetSnapshot candidateServers leak: Replace the candidateServers map
   entirely instead of appending, so removed servers don't linger.

4. TransferLocks preserves Generation and Seq: InsertLock now accepts
   generation and seq parameters. After accepting a transferred lock,
   the receiving node re-replicates to its backup.

5. Rolling restart test: Add re-replication step after promotion and
   assert survivedCount > 0. Add TestDLM_StaleReplicationRejected.

6. Mixed-version upgrade note: Add comment on HashRing documenting that
   all filer nodes must be upgraded together.

* dlm: serve renewals locally during transfer window on node join

When a new node joins and steals hash ranges from surviving nodes,
there's a window between ring update and lock transfer where the
client gets redirected to a node that doesn't have the lock yet.

Fix: if the ring says primary != self but we still hold the lock
locally (non-backup, matching token), serve the renewal/unlock here
rather than redirecting. The lock will be transferred by
OnDlmChangeSnapshot, and subsequent requests will go to the new
primary once the transfer completes.

Add tests:
- TestDLM_NodeDropAndJoin_OwnershipDisruption: measures disruption
  when a node drops and a new one joins (14/100 surviving-node locks
  disrupted, all handled by transfer logic)
- TestDLM_RenewalDuringTransferWindow: verifies renewal succeeds on
  old primary during the transfer window

* dlm: master-managed lock ring with stabilization batching

The master now owns the lock ring membership. Instead of filers
independently reacting to individual ClusterNodeUpdate add/remove
events, the master:

1. Tracks filer membership in LockRingManager
2. Batches rapid changes with a 1-second stabilization timer
   (e.g., a node drop + join within 1 second → single ring update)
3. Broadcasts the complete ring snapshot atomically via the new
   LockRingUpdate message in KeepConnectedResponse

Filers receive the ring as a complete snapshot and apply it via
SetSnapshot, ensuring all filers converge to the same ring state
without intermediate churn.

This eliminates the double-churn problem where a rapid drop+join
would fire two separate ring mutations, each triggering lock
transfers and disrupting ownership on surviving nodes.

* dlm: track ring version, reject stale updates, remove dead code

SetSnapshot now takes a version parameter from the master. Stale
updates (version < current) are rejected, preventing reordered
messages from overwriting a newer ring state. Version 0 is always
accepted for bootstrap.

Remove AddServer/RemoveServer from LockRing — the ring is now
exclusively managed by the master via SetSnapshot. Remove the
candidateServers map that was only used by those methods.

* dlm: fix SelectLocks data race, advance generation on backup insert

- SelectLocks: change RLock to Lock since the function deletes map
  entries, which is a write operation and causes a data race under RLock.
- InsertBackupLock: advance nextGeneration to at least the incoming
  generation so that after failover promotion, new lock acquisitions
  get a generation strictly greater than any replicated lock.
- Bump replication failure log from V(1) to Warningf for production
  visibility.

* dlm: fix SetSnapshot race, test reliability, timer edge cases

- SetSnapshot: hold LockRing lock through both version update and
  Ring.SetServers() so they're atomic. Prevents a concurrent caller
  from seeing the new version but applying stale servers.
- Transfer window test: search for a key that actually moves primary
  when filer4 joins, instead of relying on a fixed key that may not.
- renewLock redirect: pass the existing token to the new primary
  instead of empty string, so redirected renewals work correctly.
- scheduleBroadcast: check timer.Stop() return value. If the timer
  already fired, the callback picks up latest state.
- FlushPending: only broadcast if timer.Stop() returns true (timer
  was still pending). If false, the callback is already running.
- Fix test comment: "idempotent" → "accepted, state-changing".

* dlm: use wall-clock nanoseconds for lock ring version

The lock ring version was an in-memory counter that reset to 0 on
master restart. A filer that had seen version 5 would reject version 1
from the restarted master.

Fix: use time.Now().UnixNano() as the version. This survives master
restarts without persistence — the restarted master produces a
version greater than any pre-restart value.

* dlm: treat expired lock owners as missing

Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com>

* dlm: reject stale lock transfers

Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com>

* dlm: order replication by generation

Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com>

* dlm: bootstrap lock ring on reconnect

Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com>

---------

Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com>
2026-03-30 23:29:56 -07:00
Chris Lu
15f4a97029 fix: improve raft leader election reliability and failover speed (#8692)
* fix: clear raft vote state file on non-resume startup

The seaweedfs/raft library v1.1.7 added a persistent `state` file for
currentTerm and votedFor. When RaftResumeState=false (the default), the
log, conf, and snapshot directories are cleared but this state file was
not. On repeated restarts, different masters accumulate divergent terms,
causing AppendEntries rejections and preventing leader election.

Fixes #8690

* fix: recover TopologyId from snapshot before clearing raft state

When RaftResumeState=false clears log/conf/snapshot, the TopologyId
(used for license validation) was lost. Now extract it from the latest
snapshot before cleanup and restore it on the topology.

Both seaweedfs/raft and hashicorp/raft paths are handled, with a shared
recoverTopologyIdFromState helper in raft_common.go.

* fix: stagger multi-master bootstrap delay by peer index

Previously all masters used a fixed 1500ms delay before the bootstrap
check. Now the delay is proportional to the peer's sorted index with
randomization (matching the hashicorp raft path), giving the designated
bootstrap node (peer 0) a head start while later peers wait for gRPC
servers to be ready.

Also adds diagnostic logging showing why DoJoinCommand was or wasn't
called, making leader election issues easier to diagnose from logs.

* fix: skip unreachable masters during leader reconnection

When a master leader goes down, non-leader masters still redirect
clients to the stale leader address. The masterClient would follow
these redirects, fail, and retry — wasting round-trips each cycle.

Now tryAllMasters tracks which masters failed within a cycle and skips
redirects pointing to them, reducing log spam and connection overhead
during leader failover.

* fix: take snapshot after TopologyId generation for recovery

After generating a new TopologyId on the leader, immediately take a raft
snapshot so the ID can be recovered from the snapshot on future restarts
with RaftResumeState=false. Without this, short-lived clusters would
lose the TopologyId on restart since no automatic snapshot had been
taken yet.

* test: add multi-master raft failover integration tests

Integration test framework and 5 test scenarios for 3-node master
clusters:

- TestLeaderConsistencyAcrossNodes: all nodes agree on leader and
  TopologyId
- TestLeaderDownAndRecoverQuickly: leader stops, new leader elected,
  old leader rejoins as follower
- TestLeaderDownSlowRecover: leader gone for extended period, cluster
  continues with 2/3 quorum
- TestTwoMastersDownAndRestart: quorum lost (2/3 down), recovered
  when both restart
- TestAllMastersDownAndRestart: full cluster restart, leader elected,
  all nodes agree on TopologyId

* fix: address PR review comments

- peerIndex: return -1 (not 0) when self not found, add warning log
- recoverTopologyIdFromSnapshot: defer dir.Close()
- tests: check GetTopologyId errors instead of discarding them

* fix: address review comments on failover tests

- Assert no leader after quorum loss (was only logging)
- Verify follower cs.Leader matches expected leader via
  ServerAddress.ToHttpAddress() comparison
- Check GetTopologyId error in TestTwoMastersDownAndRestart
2026-03-18 23:28:07 -07:00
Chris Lu
baae672b6f feat: auto-disable master vacuum when plugin worker is active (#8624)
* feat: auto-disable master vacuum when plugin vacuum worker is active

When a vacuum-capable plugin worker connects to the admin server, the
admin server calls DisableVacuum on the master to prevent the automatic
scheduled vacuum from conflicting with the plugin worker's vacuum. When
the worker disconnects, EnableVacuum is called to restore the default
behavior. A safety net in the topology refresh loop re-enables vacuum
if the admin server disconnects without cleanup.

* rename isAdminServerConnected to isAdminServerConnectedFunc

* add 5s timeout to DisableVacuum/EnableVacuum gRPC calls

Prevents the monitor goroutine from blocking indefinitely if the
master is unresponsive.

* track plugin ownership of vacuum disable to avoid overriding operator

- Add vacuumDisabledByPlugin flag to Topology, set when DisableVacuum
  is called while admin server is connected (i.e., by plugin monitor)
- Safety net only re-enables vacuum when it was disabled by plugin,
  not when an operator intentionally disabled it via shell command
- EnableVacuum clears the plugin flag

* extract syncVacuumState for testability, add fake toggler tests

Extract the single sync step into syncVacuumState() with a
vacuumToggler interface. Add TestSyncVacuumState with a fake
toggler that verifies disable/enable calls on state transitions.

* use atomic.Bool for isDisableVacuum and vacuumDisabledByPlugin

Both fields are written by gRPC handlers and read by the vacuum
goroutine, causing a data race. Use atomic.Bool with Store/Load
for thread-safe access.

* use explicit by_plugin field instead of connection heuristic

Add by_plugin bool to DisableVacuumRequest proto so the caller
declares intent explicitly. The admin server monitor sets it to
true; shell commands leave it false. This prevents an operator's
intentional disable from being auto-reversed by the safety net.

* use setter for admin server callback instead of function parameter

Move isAdminServerConnected from StartRefreshWritableVolumes
parameter to Topology.SetAdminServerConnectedFunc() setter.
Keeps the function signature stable and decouples the topology
layer from the admin server concept.

* suppress repeated log messages on persistent sync failures

Add retrying parameter to syncVacuumState so the initial
state transition is logged at V(0) but subsequent retries
of the same transition are silent until the call succeeds.

* clear plugin ownership flag on manual DisableVacuum

Prevents stale plugin flag from causing incorrect auto-enable
when an operator manually disables vacuum after a plugin had
previously disabled it.

* add by_plugin to EnableVacuumRequest for symmetric ownership tracking

Plugin-driven EnableVacuum now only re-enables if the plugin was
the one that disabled it. If an operator manually disabled vacuum
after the plugin, the plugin's EnableVacuum is a no-op. This
prevents the plugin monitor from overriding operator intent on
worker disconnect.

* use cancellable context for monitorVacuumWorker goroutine

Replace context.Background() with a cancellable context stored
as bgCancel on AdminServer. Shutdown() calls bgCancel() so
monitorVacuumWorker exits cleanly via ctx.Done().

* track operator and plugin vacuum disables independently

Replace single isDisableVacuum flag with two independent flags:
vacuumDisabledByOperator and vacuumDisabledByPlugin. Each caller
only flips its own flag. The effective disabled state is the OR
of both. This prevents a plugin connect/disconnect cycle from
overriding an operator's manual disable, and vice versa.

* fix safety net to clear plugin flag, not operator flag

The safety net should call EnableVacuumByPlugin() to clear only
the plugin disable flag when the admin server disconnects. The
previous call to EnableVacuum() incorrectly cleared the operator
flag instead.
2026-03-13 22:49:12 -07:00
Chris Lu
2ec0a67ee3 master: return 503/Unavailable during topology warmup after leader change (#8529)
* master: return 503/Unavailable during topology warmup after leader change

After a master restart or leader change, the topology is empty until
volume servers reconnect and send heartbeats. During this warmup window
(3 heartbeat intervals = 15 seconds), volume lookups that fail now
return 503 Service Unavailable (HTTP) or gRPC Unavailable instead of
404 Not Found, signaling clients to retry with other masters.

* master: skip warmup 503 on fresh start and single-master setups

- Check MaxVolumeId > 0 to distinguish restart from fresh start
  (MaxVolumeId is Raft-persisted, so 0 means no prior data)
- Check peer count > 1 so single-master deployments aren't affected
  (no point suggesting "retry with other masters" if there are none)

* master: address review feedback and block assigns during warmup

- Protect LastLeaderChangeTime with dedicated mutex (fix data race)
- Extract warmup multiplier as WarmupPulseMultiplier constant
- Derive Retry-After header from pulse config instead of hardcoding
- Only trigger warmup 503 for "not found" errors, not parse errors
- Return nil response (not partial) on gRPC Unavailable
- Add doc comments to IsWarmingUp, getter/setter, WarmupDuration
- Block volume assign requests (HTTP and gRPC) during warmup,
  since the topology is incomplete and assignments would be unreliable
- Skip warmup behavior for single-master setups (no peers to retry)

* master: apply warmup to all setups, skip only on fresh start

Single-master restarts still have an empty topology until heartbeats
arrive, so warmup protection should apply there too. The only case
to skip is a fresh cluster start (MaxVolumeId == 0), which already
has no volumes to look up.

- Remove GetMasterCount() > 1 guard from all warmup checks
- Remove now-unused GetMasterCount helper
- Update error messages to "topology is still loading" (not
  "retry with other masters" which doesn't apply to single-master)

* master: add client-side retry on Unavailable for lookup and assign

The server-side 503/Unavailable during warmup needs client cooperation.
Previously, LookupVolumeIds and Assign would immediately propagate the
error without retry.

Now both paths retry with exponential backoff (1s -> 1.5s -> ... up to
6s) when receiving Unavailable, respecting context cancellation. This
covers the warmup window where the master's topology is still loading
after a restart or leader change.

* master: seed warmup timestamp in legacy raft path at setup

The legacy raft path only set lastLeaderChangeTime inside the event
listener callback, which could fire after IsLeader() was already
observed as true in SetRaftServer. Seed the timestamp at setup time
(matching the hashicorp path) so IsWarmingUp() is active immediately.

* master: fix assign retry loop to cover full warmup window

The retry loop used waitTime <= maxWaitTime as a stop condition,
causing it to give up after ~13s while warmup lasts 15s. Now cap
each individual sleep at maxWaitTime but keep retrying until the
context is cancelled.

* master: preserve gRPC status in lookup retry and fix retry window

Return the raw gRPC error instead of wrapping with fmt.Errorf so
status.FromError() can extract the status code. Use proper gRPC
status check (codes.Unavailable) instead of string matching. Also
cap individual sleep at maxWaitTime while retrying until ctx is done.

* master: use gRPC status code instead of string matching in assign retry

Use status.FromError/codes.Unavailable instead of brittle
strings.Contains for detecting retriable gRPC errors in the
assign retry loop.

* master: use remaining warmup duration for Retry-After header

Set Retry-After to the remaining warmup time instead of the full
warmup duration, so clients don't wait longer than necessary.

* master: reset ret.Replicas before populating from assign response

Clear Replicas slice before appending to prevent duplicate entries
when the assign response is retried or when alternative requests
are attempted.

* master: add unit tests for warmup retry behavior

Test that Assign() and LookupVolumeIds() retry on codes.Unavailable
and stop promptly when the context is cancelled.

* master: record leader change time before initialization work

Move SetLastLeaderChangeTime() to fire immediately when the leader
change event is received, before DoBarrier(), EnsureTopologyId(),
and updatePeers(), so the warmup clock starts at the true moment
of leadership transition.

* master: use topology warmup duration in volume growth wait loop

Replace hardcoded constants.VolumePulsePeriod * 2 with
topo.IsWarmingUp() and topo.WarmupDuration() so the growth wait
stays in sync with the configured warmup window. Remove unused
constants import.

* master: resolve master before creating RPC timeout context

Move GetMaster() call before context.WithTimeout() so master
resolution blocking doesn't consume the gRPC call timeout.

* master: use NotFound flag instead of string matching for volume lookup

Add a NotFound field to LookupResult and set it in findVolumeLocation
when a volume is genuinely missing. Update HTTP and gRPC warmup
checks to use this flag instead of strings.Contains on the error
message.

* master: bound assign retry loop to 30s for deadline-free contexts

Without a context deadline, the Unavailable retry loop could spin
forever. Add a maxRetryDuration of 30s so the loop gives up even
when no context deadline is set.

* master: strengthen assign retry cancellation test

Verify the retry loop actually retried (callCount > 1) and that
the returned error is context.DeadlineExceeded, not just any error.

* master: extract shared retry-with-backoff utility

Add util.RetryWithBackoff for context-aware, bounded retry with
exponential backoff. Refactor both Assign() and LookupVolumeIds()
to use it instead of duplicating the retry/sleep/backoff logic.

* master: cap waitTime in RetryWithBackoff to prevent unbounded growth

Cap the backoff waitTime at maxWaitTime so it doesn't grow
indefinitely in long-running retry scenarios.

* master: only return Unavailable during warmup when all lookups failed

For batched LookupVolume requests, return partial results when some
volumes are found. Only return codes.Unavailable when no volumes
were successfully resolved, so clients benefit from partial results
instead of retrying unnecessarily.

* master: set retriable error message in 503 response body

When returning 503 during warmup, replace the "not found" error
in the JSON body with "service warming up, please retry" so
clients don't treat it as a permanent error.

* master: guard empty master address in LookupVolumeIds

If GetMaster() returns empty (no master found or ctx cancelled),
return an appropriate error instead of dialing an empty address.
Returns ctx.Err() if context is done, otherwise codes.Unavailable
to trigger retry.

* master: add comprehensive tests for RetryWithBackoff

Test success after retries, non-retryable error handling, context
cancellation, and maxDuration cap with context.Background().

* master: enforce hard maxDuration bound in RetryWithBackoff

Use a deadline instead of elapsed-time check so the last sleep is
capped to remaining time. This prevents the total retry duration
from overshooting maxDuration by up to one full backoff interval.

* master: respect fresh-start bypass in RemainingWarmupDuration

Check IsWarmingUp() first (which returns false when MaxVolumeId==0)
so RemainingWarmupDuration returns 0 on fresh clusters.

* master: round up Retry-After seconds to avoid underestimating

Use math.Ceil so fractional remaining seconds (e.g. 1.9s) round
up to the next integer (2) instead of flooring down (1).

* master: tighten batch lookup warmup to all-NotFound only

Only return codes.Unavailable when every requested volume ID was
a transient not-found. Mixed cases with non-NotFound errors now
return the response with per-volume error details preserved.

* master: reduce retry log noise and fix timer leak

Lower per-attempt retry log from V(0) to V(1) to reduce noise
during warmup. Replace time.After with time.NewTimer to avoid
lingering timers when context is cancelled.

* master: add per-attempt timeout for assign RPC

Use a 10s per-attempt timeout so a single slow RPC can't consume
the entire 30s retry budget when ctx has no deadline.

* master: share single 30s retry deadline across assign request entries

The Assign() function iterates over primary and fallback requests,
previously giving each its own 30s RetryWithBackoff budget. With a
primary + fallback, the total could reach 60s. Compute one deadline
up front and pass the remaining budget to each RetryWithBackoff call
so the entire Assign() call stays within a single 30s cap.

* master: strengthen context-cancel test with DeadlineExceeded and retry assertions

Assert errors.Is(err, context.DeadlineExceeded) to verify the error
is specifically from the context deadline, and check callCount > 1
to prove retries actually occurred before cancellation. Mirrors the
pattern used in TestAssignStopsOnContextCancel.

* master: bound GetMaster with per-attempt timeout in LookupVolumeIds

GetMaster() calls WaitUntilConnected() which can block indefinitely
if no master is available. Previously it used the outer ctx, so a
slow master resolution could consume the entire RetryWithBackoff
budget in a single attempt. Move the per-attempt timeoutCtx creation
before the GetMaster call so both master resolution and the gRPC
LookupVolume RPC share one grpcTimeout-bounded attempt.

* master: use deadline-aware context for assign retry budget

The shared 30s deadline only limited RetryWithBackoff's internal
wall-clock tracking, but per-attempt contexts were still derived
from the original ctx and could run for up to 10s even when the
budget was nearly exhausted. Create a deadlineCtx from the computed
deadline and derive both RetryWithBackoff and per-attempt timeouts
from it so all operations honor the shared 30s cap.

* master: skip warmup gate for empty lookup requests

When VolumeOrFileIds is empty, notFoundCount == len(req.VolumeOrFileIds)
is 0 == 0 which is true, causing empty lookup batches during warmup to
return codes.Unavailable and be retried endlessly. Add a
len(req.VolumeOrFileIds) > 0 guard so empty requests pass through.

* master: validate request fields before warmup gate in Assign

Move Replication and Ttl parsing before the IsWarmingUp() check so
invalid inputs get a proper validation error instead of being masked
by codes.Unavailable during warmup. Pure syntactic validation does
not depend on topology state and should run first.

* master: check deadline and context before starting retry attempt

RetryWithBackoff only checked the deadline and context after an
attempt completed or during the sleep select. If the deadline
expired or context was canceled during sleep, the next iteration
would still call operation() before detecting it. Add pre-operation
checks so no new attempt starts after the budget is exhausted.

* master: always return ctx.Err() on context cancellation in RetryWithBackoff

When ctx.Err() is non-nil, the pre-operation check was returning
lastErr instead of ctx.Err(). This broke callers checking
errors.Is(err, context.DeadlineExceeded) and contradicted the
documented contract. Always return ctx.Err() so the cancellation
reason is properly surfaced.

* master: handle warmup errors in StreamAssign without killing the stream

StreamAssign was returning codes.Unavailable errors from Assign
directly, which terminates the gRPC stream and breaks pooled
connections. Instead, return transient errors as in-band error
responses so the stream survives warmup periods.

Also reset assignClient in doAssign on Send/Recv failures so a
broken stream doesn't leave the proxy permanently dead.

* master: wait for warmup before slot search in findAndGrow

findEmptySlotsForOneVolume was called before the warmup wait loop,
selecting slots from an incomplete topology. Move the warmup wait
before slot search so volume placement uses the fully warmed-up
topology with all servers registered.

* master: add Retry-After header to /dir/assign warmup response

The /dir/lookup handler already sets Retry-After during warmup but
/dir/assign did not, leaving HTTP clients without guidance on when
to retry. Add the same header using RemainingWarmupDuration().

* master: only seed warmup timestamp on leader at startup

SetLastLeaderChangeTime was called unconditionally for both leader
and follower nodes. Followers don't need warmup state, and the
leader change event listener handles real elections. Move the seed
into the IsLeader() block so only the startup leader gets warmup
initialized.

* master: preserve codes.Unavailable for StreamAssign warmup errors in doAssign

StreamAssign returns transient warmup errors as in-band
AssignResponse.Error messages. doAssign was converting these to plain
fmt.Errorf, losing the codes.Unavailable classification needed for
the caller's retry logic. Detect warmup error messages and wrap them
as status.Error(codes.Unavailable) so RetryWithBackoff can retry.
2026-03-08 16:05:45 -07:00
Chris Lu
230ae9c24e no need to set default scripts now 2026-03-04 22:27:02 -08:00
Chris Lu
b3620c7e14 admin: auto migrating master maintenance scripts to admin_script plugin config (#8509)
* admin: seed admin_script plugin config from master maintenance scripts

When the admin server starts, fetch the maintenance scripts configuration
from the master via GetMasterConfiguration. If the admin_script plugin
worker does not already have a saved config, use the master's scripts as
the default value. This enables seamless migration from master.toml
[master.maintenance] to the admin script plugin worker.

Changes:
- Add maintenance_scripts and maintenance_sleep_minutes fields to
  GetMasterConfigurationResponse in master.proto
- Populate the new fields from viper config in master_grpc_server.go
- On admin server startup, fetch the master config and seed the
  admin_script plugin config if no config exists yet
- Strip lock/unlock commands from the master scripts since the admin
  script worker handles locking automatically

Co-Authored-By: Claude Opus 4.6 <noreply@anthropic.com>

* fix: address review comments on admin_script seeding

- Replace TOCTOU race (separate Load+Save) with atomic
  SaveJobTypeConfigIfNotExists on ConfigStore and Plugin
- Replace ineffective polling loop with single GetMaster call using
  30s context timeout, since GetMaster respects context cancellation
- Add unit tests for SaveJobTypeConfigIfNotExists (in-memory + on-disk)

Co-Authored-By: Claude Opus 4.6 <noreply@anthropic.com>

* fix: apply maintenance script defaults in gRPC handler

The gRPC handler for GetMasterConfiguration read maintenance scripts
from viper without calling SetDefault, relying on startAdminScripts
having run first. If the admin server calls GetMasterConfiguration
before startAdminScripts sets the defaults, viper returns empty
strings and the seeding is silently skipped.

Apply SetDefault in the gRPC handler itself so it is self-contained.

Co-Authored-By: Claude Opus 4.6 <noreply@anthropic.com>

* Revert "fix: apply maintenance script defaults in gRPC handler"

This reverts commit 068a5063303f6bc34825a07bb681adfa67e6f9de.

* fix: use atomic save in ensureJobTypeConfigFromDescriptor

ensureJobTypeConfigFromDescriptor used a separate Load + Save, racing
with seedAdminScriptFromMaster. If the descriptor defaults (empty
script) were saved first, SaveJobTypeConfigIfNotExists in the seeding
goroutine would see an existing config and skip, losing the master's
maintenance scripts.

Switch to SaveJobTypeConfigIfNotExists so both paths are atomic. Whichever
wins, the other is a safe no-op.

Co-Authored-By: Claude Opus 4.6 <noreply@anthropic.com>

* fix: fetch master scripts inline during config bootstrap, not in goroutine

Replace the seedAdminScriptFromMaster goroutine with a
ConfigDefaultsProvider callback. When the plugin bootstraps
admin_script defaults from the worker descriptor, it calls the
provider which fetches maintenance scripts from the master
synchronously. This eliminates the race between the seeding
goroutine and the descriptor-based config bootstrap.

Co-Authored-By: Claude Opus 4.6 <noreply@anthropic.com>

* skip commented lock unlock

Co-Authored-By: Copilot <223556219+Copilot@users.noreply.github.com>

* reduce grpc calls

---------

Co-authored-by: Claude Opus 4.6 <noreply@anthropic.com>
Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com>
2026-03-04 22:11:07 -08:00
Chris Lu
45ce18266a Disable master maintenance scripts when admin server runs (#8499)
* Disable master maintenance scripts when admin server runs

* Stop defaulting master maintenance scripts

* Apply suggestion from @gemini-code-assist[bot]

Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com>

* Apply suggestion from @gemini-code-assist[bot]

Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com>

* Clarify master scripts are disabled by default

* Skip master maintenance scripts when admin server is connected

* Restore default master maintenance scripts

* Document admin server skip for master maintenance scripts

---------

Co-authored-by: Copilot <copilot@github.com>
Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com>
2026-03-04 00:40:40 -08:00
Chris Lu
57ab99d13e fix: generate topology uuid uniformly in single-master mode (#8405)
* fix: ensure topology uuid is generated in single master setups

* ensureTopologyId adds a Hashicorp-aware implementation

* simplify
2026-02-22 23:45:48 -08:00
Chris Lu
3300874cb5 filer: add default log purging to master maintenance scripts (#8359)
* filer: add default log purging to master maintenance scripts

* filer: fix default maintenance scripts to include full set of tasks

* filer: refactor maintenance scripts to avoid duplication
2026-02-16 16:58:15 -08:00
Chris Lu
cb9e21cdc5 Normalize hashicorp raft peer ids (#8253)
* Normalize raft voter ids

* 4.11

* Update raft_hashicorp.go
2026-02-09 07:46:34 -08:00
Chris Lu
753e1db096 Prevent split-brain: Persistent ClusterID and Join Validation (#8022)
* Prevent split-brain: Persistent ClusterID and Join Validation

- Persist ClusterId in Raft store to survive restarts.
- Validate ClusterId on Raft command application (piggybacked on MaxVolumeId).
- Prevent masters with conflicting ClusterIds from joining/operating together.
- Update Telemetry to report the persistent ClusterId.

* Refine ClusterID validation based on feedback

- Improved error message in cluster_commands.go.
- Added ClusterId mismatch check in RaftServer.Recovery.

* Handle Raft errors and support Hashicorp Raft for ClusterId

- Check for errors when persisting ClusterId in legacy Raft.
- Implement ClusterId generation and persistence for Hashicorp Raft leader changes.
- Ensure consistent error logging.

* Refactor ClusterId validation

- Centralize ClusterId mismatch check in Topology.SetClusterId.
- Simplify MaxVolumeIdCommand.Apply and RaftServer.Recovery to rely on SetClusterId.

* Fix goroutine leak and add timeout

- Handle channel closure in Hashicorp Raft leader listener.
- Add timeout to Raft Apply call to prevent blocking.

* Fix deadlock in legacy Raft listener

- Wrap ClusterId generation/persistence in a goroutine to avoid blocking the Raft event loop (deadlock).

* Rename ClusterId to SystemId

- Renamed ClusterId to SystemId across the codebase (protobuf, topology, server, telemetry).
- Regenerated telemetry.pb.go with new field.

* Rename SystemId to TopologyId

- Rename to SystemId was intermediate step.
- Final name is TopologyId for the persistent cluster identifier.
- Updated protobuf, topology, raft server, master server, and telemetry.

* Optimize Hashicorp Raft listener

- Integrated TopologyId generation into existing monitorLeaderLoop.
- Removed extra goroutine in master_server.go.

* Fix optimistic TopologyId update

- Removed premature local state update of TopologyId in master_server.go and raft_hashicorp.go.
- State is now solely updated via the Raft state machine Apply/Restore methods after consensus.

* Add explicit log for recovered TopologyId

- Added glog.V(0) info log in RaftServer.Recovery to print the recovered TopologyId on startup.

* Add Raft barrier to prevent TopologyId race condition

- Implement ensureTopologyId helper method
- Send no-op MaxVolumeIdCommand to sync Raft log before checking TopologyId
- Ensures persisted TopologyId is recovered before generating new one
- Prevents race where generation happens during log replay

* Serialize TopologyId generation with mutex

- Add topologyIdGenLock mutex to MasterServer struct
- Wrap ensureTopologyId method with lock to prevent concurrent generation
- Fixes race where event listener and manual leadership check both generate IDs
- Second caller waits for first to complete and sees the generated ID

* Add TopologyId recovery logging to Apply method

- Change log level from V(1) to V(0) for visibility
- Log 'Recovered TopologyId' when applying from Raft log
- Ensures recovery is visible whether from snapshot or log replay
- Matches Recovery() method logging for consistency

* Fix Raft barrier timing issue

- Add 100ms delay after barrier command to ensure log application completes
- Add debug logging to track barrier execution and TopologyId state
- Return early if barrier command fails
- Prevents TopologyId generation before old logs are fully applied

* ensure leader

* address comments

* address comments

* redundant

* clean up

* double check

* refactoring

* comment
2026-01-18 14:02:34 -08:00
Chris Lu
07dc552e1c master: Fix raft url (#7255)
* fix signature

* fix url scheme
2025-09-18 14:46:53 -07:00
Dmitriy Pavlov
cd78e653e1 add disable volume_growth flag (#7196) 2025-09-04 05:39:56 -07:00
Chris Lu
e446234e9c remove spoof-able request header (#7103)
* remove spoof-able request header

https://github.com/seaweedfs/seaweedfs/issues/7094#issuecomment-3158320497

* Update weed/security/guard.go

Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>

---------

Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
2025-08-06 10:08:30 -07:00
Chris Lu
0703308270 remote address parsing should handle special cases (#7101)
* remote address parsing should handle special cases

* handling ipv6

* simplify

* Update weed/security/guard.go

Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>

* Update weed/security/guard.go

Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com>

* x-real-ip

* Update guard.go

* fixes

 Hostname Whitelisting: Fully restored - supports localhost, example.com, etc.
 IP Whitelisting: Still works - supports exact IPs and CIDR ranges
 Header Support: Consistent handling of X-Forwarded-For, X-Real-IP

* simplify

* Update weed/security/guard.go

Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com>

* Update weed/security/guard.go

Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com>

* Update guard.go

* adjust function signature

* Update weed/security/guard.go

Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com>

* indention

* skip empty host

---------

Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com>
2025-08-06 01:03:00 -07:00
chrislu
798f797158 use float for sleep seconds
fix https://github.com/seaweedfs/seaweedfs/pull/6795
2025-07-06 14:16:41 -07:00
chrislu
1733d0ce68 remove features and deployments fields 2025-06-28 20:03:06 -07:00
Chris Lu
a1aab8a083 add telemetry (#6926)
* add telemetry

* fix go mod

* add default telemetry server url

* Update README.md

* replace with broker count instead of s3 count

* Update telemetry.pb.go

* github action to deploy
2025-06-28 14:11:55 -07:00
Aleksey Kosov
5182d46e22 Added middleware for processing request_id grpc and http requests (#6805) 2025-05-21 07:57:39 -07:00
Lisandro Pin
fc4df944a0 Remove rate limit semaphore on master's leader selection logic. (#6494)
This was introduced by 054374c7 (2024-03-12) and serves no practical purpose,
yet it caps the maximum QPS master servers can handle.
2025-01-30 13:08:36 -08:00
Konstantin Lebedev
b65eb2ec45 [security] reload whiteList on http seerver (#6302)
* reload whiteList

* white_list add to scaffold
2024-12-02 10:38:10 -08:00
Konstantin Lebedev
fec88e64eb [master] update LastLeaderChangeTime for hashicorp raft (#6292) 2024-11-26 08:02:45 -08:00
chrislu
ccf1795e6f wait a bit before getting the next volume id if the leader is recently elected 2024-11-23 19:58:45 -08:00
chrislu
6564ceda91 skip resource heavy commands from running on master nodes 2024-09-29 10:51:17 -07:00
chrislu
4463296811 add parallel vacuuming 2024-08-21 22:53:54 -07:00
Riccardo Bertossa
6fe8639504 add http endpoint to get the size of a collection (#5910) 2024-08-19 07:44:45 -07:00
wyang
4b1f539ab8 fix allocate reduplicated volumeId to different volume (#5811)
* fix allocate reduplicated volumeId to different volume

* only check barrier when read

---------

Co-authored-by: Yang Wang <yangwang@weride.ai>
2024-07-26 21:48:36 -07:00
vadimartynov
86d92a42b4 Added tls for http clients (#5766)
* Added global http client

* Added Do func for global http client

* Changed the code to use the global http client

* Fix http client in volume uploader

* Fixed pkg name

* Fixed http util funcs

* Fixed http client for bench_filer_upload

* Fixed http client for stress_filer_upload

* Fixed http client for filer_server_handlers_proxy

* Fixed http client for command_fs_merge_volumes

* Fixed http client for command_fs_merge_volumes and command_volume_fsck

* Fixed http client for s3api_server

* Added init global client for main funcs

* Rename global_client to client

* Changed:
- fixed NewHttpClient;
- added CheckIsHttpsClientEnabled func
- updated security.toml in scaffold

* Reduce the visibility of some functions in the util/http/client pkg

* Added the loadSecurityConfig function

* Use util.LoadSecurityConfiguration() in NewHttpClient func
2024-07-16 23:14:09 -07:00
Konstantin Lebedev
67edf1d014 [master] Do Automatic Volume Grow in background (#5781)
* Do Automatic Volume Grow in backgound

* pass lastGrowCount to master

* fix build

* fix count to uint64
2024-07-16 08:03:40 -07:00
vadimartynov
8aae82dd71 Added context for the MasterClient's methods to avoid endless loops (#5628)
* Added context for the MasterClient's methods to avoid endless loops

* Returned WithClient function. Added WithClientCustomGetMaster function

* Hid unused ctx arguments

* Using a common context for the KeepConnectedToMaster and WaitUntilConnected functions

* Changed the context termination check in the tryConnectToMaster function

* Added a child context to the tryConnectToMaster function

* Added a common context for KeepConnectedToMaster and WaitUntilConnected functions in benchmark
2024-06-14 11:40:34 -07:00
shenxingwuying
ee25ada732 reduce ambiguity about use memory_sequencer (#5555) 2024-04-29 21:51:00 -07:00
chrislu
55976ae04a avoid repeated calls to heavy-weighted viper 2024-04-18 09:09:45 -07:00
chrislu
d9490c5e1f rename 2024-04-18 08:47:45 -07:00
Nico D'Cotta
796b7508f3 Implement SRV lookups for filer (#4767) 2023-08-24 07:08:56 -07:00
chrislu
a315490f7d proxy to master uses http address
fix https://github.com/seaweedfs/seaweedfs/issues/4607
2023-07-04 11:45:21 -07:00
chrislu
adb90bd252 avoid lower casing the command
fix https://github.com/seaweedfs/seaweedfs/pull/4321
2023-03-19 21:20:46 -07:00
Konstantin Lebedev
b9933d5589 master server graceful stop (#3797) 2022-10-06 09:30:30 -07:00
Konstantin Lebedev
e90ab4ac60 avoid race conditions for OnPeerUpdate (#3525)
https://github.com/seaweedfs/seaweedfs/issues/3524
2022-08-26 10:18:49 -07:00
Patrick Schmidt
7b424a54dc Add raft server access mutex to avoid races (#3503) 2022-08-24 09:49:05 -07:00
chrislu
10414fd81c ping timeout at 15 seconds
this 72 minute timeout setting seems unreasonably long

15 seconds is around the time when a new raft leader should be elected.
2022-08-23 23:28:16 -07:00
askeipx
2e78a522ab remove old raft servers if they don't answer to pings for too long (#3398)
* remove old raft servers if they don't answer to pings for too long

add ping durations as options

rename ping fields

fix some todos

get masters through masterclient

raft remove server from leader

use raft servers to ping them

CheckMastersAlive for hashicorp raft only

* prepare blocking ping

* pass waitForReady as param

* pass waitForReady through all functions

* waitForReady works

* refactor

* remove unneeded params

* rollback unneeded changes

* fix
2022-08-23 23:18:21 -07:00
Konstantin Lebedev
4d4cd0948d avoid infinite loop WaitUntilConnected() (#3431)
https://github.com/seaweedfs/seaweedfs/issues/3421
2022-08-11 15:03:26 -07:00
Chris Lu
b59bc607bf Merge pull request #3338 from kmlebedev/issues/3083
rollback over onPeerUpdate implementation of automatic clean-up of failed servers in favor of synchronous ping
2022-08-01 08:23:10 -07:00
Konstantin Lebedev
a98f6d66a3 rollback over onPeerupdate implementation of automatic clean-up of failed servers in favor of synchronous ping 2022-08-01 12:51:41 +05:00
chrislu
26dbc6c905 move to https://github.com/seaweedfs/seaweedfs 2022-07-29 00:17:28 -07:00
chrislu
bb01b68fa0 refactor 2022-07-28 23:24:38 -07:00
chrislu
68065128b8 add dc and rack 2022-07-28 23:22:51 -07:00
chrislu
3828b8ce87 "github.com/chrislusf/raft" => "github.com/seaweedfs/raft" 2022-07-27 12:12:40 -07:00
Konstantin Lebedev
c88ea31f62 fix RUnlock of unlocked RWMutex 2022-07-26 12:57:07 +05:00
Konstantin Lebedev
3c42814b58 avoid deadlock 2022-07-21 17:15:10 +05:00