Commit Graph

42 Commits

Author SHA1 Message Date
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
Dmitriy Pavlov
cd78e653e1 add disable volume_growth flag (#7196) 2025-09-04 05:39:56 -07:00
chrislu
215c5de579 minor 2025-07-16 09:22:25 -07:00
chrislu
ff3d46637d better logging for volume growth 2024-09-07 12:38:34 -07:00
Konstantin Lebedev
b2ffcdaab2 [master] do sync grow request only if absolutely necessary (#5821)
* do sync grow request only if absolutely necessary
https://github.com/seaweedfs/seaweedfs/pull/5819

* remove check VolumeGrowStrategy Threshold on PickForWrite

* fix fmt.Errorf
2024-07-30 13:21:35 -07:00
chrislu
e2a07d11d5 Revert "Check ShouldGrowVolumes before returning error in assign. (#5819)"
This reverts commit 98d66338d0333cd955f7840c64ef95e3c4807a17.
2024-07-26 11:21:50 -07:00
wyang
0581ce6096 fix delete chunk failed if volumeSever specified grpc.port (#5820)
Co-authored-by: Yang Wang <yangwang@weride.ai>
2024-07-26 11:14:19 -07:00
chrislu
ec9455dab7 same change as #5819
follow up with https://github.com/seaweedfs/seaweedfs/pull/5819
2024-07-26 11:06:47 -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
Konstantin Lebedev
04f4b10884 fix: avoid timeout if datacenter does not exist in topology (#5772)
* fix: avoid timeout if datacenter does not exist in topology

* fix: error msg

* fix: rm dublicate check

* fix: compare

* revert minor change
2024-07-12 11:19:08 -07:00
chrislu
d9490c5e1f rename 2024-04-18 08:47:45 -07:00
Konstantin Lebedev
df40908e57 fix panic 5435 (#5436) 2024-03-28 16:17:59 -07:00
chrislu
49fcb48e04 same logic as grpc assign
fix https://github.com/seaweedfs/seaweedfs/issues/5213
2024-01-23 19:39:59 -08:00
chrislu
3002087541 http assign logic should be the same as grpc assign
similar logic as bebbc9fe44
2024-01-18 08:58:53 -08:00
chrislu
bebbc9fe44 create volume grow request if the selected volume is close to full 2023-12-27 11:45:44 -08:00
Konstantin Lebedev
4d08393b7c filer prefer volume server in same data center (#3405)
* initial prefer same data center
https://github.com/seaweedfs/seaweedfs/issues/3404

* GetDataCenter

* prefer same data center for ReplicationSource

* GetDataCenterId

* remove glog
2022-08-04 17:35:00 -07:00
chrislu
26dbc6c905 move to https://github.com/seaweedfs/seaweedfs 2022-07-29 00:17:28 -07:00
Sebastian Kurfuerst
d156d410ef rename security.GenJwt to security.GenJwtForVolumeServer 2021-12-29 12:39:41 +01:00
Chris Lu
332d49432d reduce concurrent volume grow requests 2021-10-05 01:58:30 -07:00
Chris Lu
96119eab00 refactor 2021-10-05 00:40:04 -07:00
Chris Lu
7a13816e94 refactor 2021-09-05 23:17:15 -07:00
Chris Lu
5a0f92423e use grpc and jwt 2021-08-12 21:40:33 -07:00
Chris Lu
1154e23e2d add logs for volume creation 2021-08-10 13:04:25 -07:00
Chris Lu
b624090398 go fmt 2021-07-01 01:21:14 -07:00
qieqieplus
c4d32f6937 ahead of time volume assignment 2021-05-06 18:55:44 +08:00
Chris Lu
73958e357d add descriptive error if no free volumes 2021-02-18 19:10:20 -08:00
Chris Lu
cd866664a8 skip JWT if fileId is empty
related to https://github.com/chrislusf/seaweedfs/issues/1808
2021-02-16 15:39:12 -08:00
Chris Lu
a9db24cd05 master allocate volumes if ssd type runs out 2020-12-13 19:44:57 -08:00
Chris Lu
3a5e4769cf prevent empty locations returned
fix https://github.com/chrislusf/seaweedfs/issues/1313
2020-05-13 03:46:38 -07:00
zhangsong
61fa485700 add volume number param in assign operation 2019-11-10 20:11:03 +08:00
Lei Liu
f2f90436ef fix leader master /dir/lookup api
Signed-off-by: Lei Liu <lei01.liu@horizon.ai>
2019-10-30 16:38:40 +08:00
Lei Liu
c2884cace2 misc updated
Signed-off-by: Lei Liu <lei01.liu@horizon.ai>
2019-10-29 21:28:28 +08:00
Lei Liu
3cc084269c master api: return http 404 when volumeId not exist
Signed-off-by: Lei Liu <lei01.liu@horizon.ai>
2019-09-29 14:23:39 +08:00
Chris Lu
8afd8d35b3 master: followers can also lookup and redirect
improve scalability
2019-07-28 03:58:13 -07:00
Chris Lu
50aa769554 jwt for read access control 2019-06-06 00:29:02 -07:00
Chris Lu
25941e0500 master: add jwt expires_after_seconds 2019-05-04 08:42:25 -07:00
Chris Lu
e5506152c0 refactoring 2019-04-18 21:43:36 -07:00
Chris Lu
77b9af531d adding grpc mutual tls 2019-02-18 12:11:52 -08:00
Chris Lu
215cd27b37 add authorizing fileId write access
need to secure upload/update/delete for benchmark/filer/mount
need to add secure grpc
2019-02-14 00:08:20 -08:00
Chris Lu
ff66269b62 use grpc to replace http APIs for batch volume id lookup and batch delete
1. remove batch volume id lookup http API /vol/lookup
2. remove batch delete http API /delete
2018-10-14 00:12:28 -07:00
Chris Lu
f5bed84340 add volume id to error message 2017-06-03 01:58:28 -07:00
Chris Lu
5ce6bbf076 directory structure change to work with glide
glide has its own requirements. My previous workaround caused me some
code checkin errors. Need to fix this.
2016-06-02 18:09:14 -07:00