From eb7b34c258207e18e9c3901f9e5266db4f150a89 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 05:54:56 +0900 Subject: [PATCH 01/43] docs: lease read design --- docs/lease_read_design.md | 390 ++++++++++++++++++++++++++++++++++++++ 1 file changed, 390 insertions(+) create mode 100644 docs/lease_read_design.md diff --git a/docs/lease_read_design.md b/docs/lease_read_design.md new file mode 100644 index 00000000..4f51e987 --- /dev/null +++ b/docs/lease_read_design.md @@ -0,0 +1,390 @@ +# Lease Read Design + +Status: Proposed +Author: bootjp +Date: 2026-04-20 + +--- + +## 1. Background + +### 1.1 Current read paths + +elastickv has three read paths with different consistency mechanisms: + +| Path | Read fence | Quorum cost per read | +|---|---|---| +| DynamoDB getItem / query / scan | `snapshotTS()` only | 0 | +| Redis SET / GET / non-Lua commands | `snapshotTS()` only | 0 | +| Redis Lua EVAL / EVALSHA | `coordinator.LinearizableRead(ctx)` once at script start (PR #546) | 1 ReadIndex per script | + +`snapshotTS()` returns `store.LastCommitTS()` from the local FSM. It does not +verify leadership at read time. + +### 1.2 Observed problem + +After deploying the redis.call() optimization (PR #547 + #548), per-script +latency did not improve. Investigation showed: + +- `redis.call()` time accounts for ~100% of Lua VM time +- Average time per `redis.call()` invocation is 800 ms - 2.2 s +- `Raft Commit Time` is ~500 us (not the bottleneck) +- Single-key `SET x` and `GET x` from redis-cli take ~0.96 s and ~0.92 s + +Two distinct issues are hidden in the metric: + +1. The per-script `LinearizableRead` in `newLuaScriptContext` triggers a full + etcd/raft `ReadOnlySafe` ReadIndex (heartbeat broadcast + quorum + `MsgHeartbeatResp` wait) on every Lua script invocation. There is no term + cache or fast path in `submitRead` / `handleRead` + (`internal/raftengine/etcd/engine.go:524, :841`). +2. Independently, the recent change to `defaultHeartbeatTick = 10` and + `defaultElectionTick = 100` (PR #529) widened the worst-case Raft tick gap + from 10 ms to 100 ms, slowing operations that wait on Raft progress. + +This document addresses (1). Issue (2) is out of scope. + +### 1.3 Asymmetry to remove + +The current state is asymmetric: + +- DynamoDB and Redis non-Lua paths trust local state with no quorum check. + This already accepts a partition window of up to `electionTimeout` during + which a stale leader can serve stale reads (until `CheckQuorum` steps it + down). +- Lua paths pay a full quorum round-trip on every script. + +A unified lease-read API can give all three paths the same trade-off: +serve from local state when leadership is recently confirmed, fall back to +ReadIndex when it is not. + +--- + +## 2. Goals and Non-Goals + +### 2.1 Goals + +- Eliminate the per-script ReadIndex from the Lua path under steady load. +- Provide a single API used by all read paths, so the consistency trade-off + is documented in one place. +- Improve safety of DynamoDB / Redis non-Lua reads by attaching them to a + bounded lease, instead of unconditional trust of local state. +- Keep the change confined to elastickv layers; no fork of etcd/raft. + +### 2.2 Non-Goals + +- Strict linearizability under arbitrary network partitions. The design + retains a partition window of at most `electionTimeout - margin`, the same + trade-off TiKV's lease read accepts. +- Changes to `ReadOnlyOption` in etcd/raft. The fast path lives in elastickv; + the slow path still uses `ReadOnlySafe`. +- Multi-shard read transactions. Lease check is per-shard. + +--- + +## 3. Design + +### 3.1 Lease state + +Each `Coordinate` (single-shard) and each shard inside `ShardedCoordinator` +holds: + +```go +type leaseState struct { + expiry atomic.Pointer[time.Time] // nil or zero-value = expired +} +``` + +- `expiry == nil` or `time.Now() >= *expiry`: lease is expired. The next + `LeaseRead` falls back to `LinearizableRead` and refreshes the lease on + success. +- `time.Now() < *expiry`: lease is valid. `LeaseRead` returns immediately + without contacting the Raft layer. + +The pointer-swap form lets readers be lock-free: each `LeaseRead` does one +atomic load + one wall-clock compare. + +### 3.2 Lease duration + +The lease duration must be strictly less than `electionTimeout`. The bound +comes from etcd/raft: with `CheckQuorum: true`, a leader that loses contact +with majority steps down within at most `electionTimeout`. Until then, it can +still serve reads from local state. As long as our lease expires before the +leader could realistically be replaced and accept new writes elsewhere, local +reads are safe. + +The engine exposes: + +```go +func (e *Engine) LeaseDuration() time.Duration +``` + +Implementation: `electionTimeout - leaseSafetyMargin`, where +`electionTimeout = defaultTickInterval * defaultElectionTick`. With current +config: `10ms * 100 - 100ms = 900 ms`. + +`leaseSafetyMargin` (proposed: 100 ms) absorbs: + +- Goroutine scheduling delay between heartbeat ack and lease refresh. +- Wall-clock skew between leader and the partition's new leader candidate. +- GC pauses on the leader. + +The margin is conservative; reducing it shortens the post-write quiet window +during which lease reads still hit local state, at the cost of a smaller +safety buffer. + +### 3.3 Refresh triggers + +The lease is refreshed (set to `time.Now() + LeaseDuration()`) on: + +1. Any successful `engine.LinearizableRead(ctx)` returning without error. + The ReadIndex protocol confirmed quorum at that moment. +2. Any successful `engine.Propose(ctx, data)` whose result indicates commit. + A committed entry implies majority append + ack, which is a stronger + confirmation than ReadIndex. + +Heartbeat ack tracking is intentionally not used. It would require deep +hooks into etcd/raft's internals and gives only a small marginal benefit +over (1) and (2). + +### 3.4 Invalidation triggers + +The lease is invalidated (set to nil) on: + +1. State transition out of leader. `refreshStatus` in + `internal/raftengine/etcd/engine.go:1577` already detects this; we add a + coordinator hook. +2. Any error returned by `engine.Propose` or `engine.LinearizableRead`. +3. Detection of a term change since last refresh (defensive; should not + normally fire because (1) covers leader loss). + +### 3.5 API + +```go +// kv/coordinator.go +type Coordinate interface { + // ...existing... + LeaseRead(ctx context.Context) (uint64, error) +} + +// kv/sharded_coordinator.go +func (c *ShardedCoordinator) LeaseReadForKey(ctx context.Context, key []byte) (uint64, error) +``` + +Returned index is the engine's applied index at the moment of return. Callers +that use `store.LastCommitTS()` can ignore the index; callers that need an +explicit fence can use it. + +Pseudocode: + +```go +func (c *Coordinate) LeaseRead(ctx context.Context) (uint64, error) { + if c.lease.valid(time.Now()) { + return c.engine.AppliedIndex(), nil + } + idx, err := c.engine.LinearizableRead(ctx) + if err != nil { + c.lease.invalidate() + return 0, err + } + c.lease.extend(time.Now().Add(c.engine.LeaseDuration())) + return idx, nil +} +``` + +### 3.6 Application sites + +| File | Current | After | +|---|---|---| +| `adapter/redis_lua_context.go:215` | `LinearizableRead` once per script | `LeaseRead` once per script | +| `adapter/redis.go` `get`, `keyTypeAt` callers | no fence | `LeaseRead` once per command | +| `adapter/redis.go` `set` and other write commands | no fence | implicit via `Propose` refresh | +| `adapter/dynamodb.go` `getItem`, `query`, `scan` | no fence | `LeaseRead` once per request | +| `adapter/dynamodb.go` write paths | no fence | implicit via `Propose` refresh | + +For write paths, calling `LeaseRead` separately is not required: `Propose` +already confirms quorum at commit time and refreshes the lease. + +For read paths, `LeaseRead` is added at the entry of the handler. The +existing `snapshotTS()` call is unchanged. + +--- + +## 4. Safety + +### 4.1 Correctness invariant + +A lease read returning index `i` to a caller is safe iff, at the moment of +return, no other node holds a strictly higher commit index in a later term +that is visible to clients. Equivalently: this leader is still the unique +leader, and no replacement leader has accepted writes that this node has +not seen. + +### 4.2 Why the lease bound is sufficient + +etcd/raft with `CheckQuorum: true` enforces: + +- A leader that fails to receive `MsgHeartbeatResp` from majority within + `electionTimeout` steps down (becomes follower). +- A new leader cannot be elected until the previous leader's followers time + out their election ticks, which is at least `electionTimeout`. + +Combined: between losing quorum and a successor leader accepting writes, +at least `electionTimeout` of wall-clock time elapses on the followers' +clocks. + +If the lease is refreshed at time `t0` (heartbeat ack received at `t0` is the +implicit refresh signal, modulo the margin discussion in 4.3), and the lease +duration is `electionTimeout - margin`, then the lease expires at +`t0 + electionTimeout - margin`. Any client read before that time runs on a +leader that, modulo clock skew bounded by `margin`, has not yet been +replaced. + +### 4.3 Refresh-vs-ack gap + +The design refreshes the lease on `LinearizableRead` and `Propose` +completion, not on individual heartbeat acks. This widens the gap between +the actual quorum confirmation event and the lease extension event by at +most one round-trip plus goroutine scheduling. + +This gap is included in `leaseSafetyMargin`. Specifically, if the round-trip +plus scheduling delay is bounded by `D`, then `margin >= D + clock_skew_bound` +preserves the invariant. + +### 4.4 Comparison to current state + +| Path | Current safety window | After lease | Notes | +|---|---|---|---| +| DynamoDB / Redis non-Lua read | up to `electionTimeout` (until CheckQuorum step-down) | up to `LeaseDuration()` | strictly improved | +| Lua read | 0 (full ReadIndex per script) | up to `LeaseDuration()` | strictly weaker, matches the others | + +The Lua change accepts the same trade-off the other paths already accept. + +### 4.5 Read-then-write inside Lua + +Lua scripts often read state, compute, then write. The write goes through +`Propose`, which requires quorum. A stale leader's `Propose` cannot commit +because it cannot reach majority (the quorum is on the other side of the +partition). + +So a stale lease read inside a Lua script cannot directly cause a stale +write to commit. However: + +- The script may decide to write based on a stale-but-not-divergent value + (e.g. lost-update on a counter). This is the same hazard DynamoDB + conditional writes face today. +- The client may receive a success response from the stale leader for the + read portion before the write fails. With the current code, the entire + script is wrapped in a single Raft proposal, so the script either commits + atomically or returns an error. The lease change does not alter this. + +### 4.6 Failure modes considered + +- Leader losing quorum but lease still valid: stale read possible. Window + is bounded by `LeaseDuration() < electionTimeout`. Any write attempt + within the window fails because `Propose` cannot reach quorum. Same + trade-off DynamoDB / Redis non-Lua already accept today. +- Leader losing quorum and lease expired: next `LeaseRead` calls + `LinearizableRead`, which fails (no quorum), error propagated to caller. + Lease invalidated. +- Leader transferring leadership: `refreshStatus` detects state transition + out of leader and invalidates the lease. +- Clock skew exceeding `leaseSafetyMargin`: lease may extend beyond + `electionTimeout`, allowing a stale read after a successor leader has + accepted writes. Mitigation: keep `leaseSafetyMargin` larger than the + documented clock-skew SLO of the deployment. Default 100 ms is consistent + with the HLC physical window of 3 s used elsewhere. + +--- + +## 5. Implementation Plan + +### Phase 1: engine surface +1. Add `LeaseDuration() time.Duration` to `internal/raftengine/engine.go` + `LeaderView` interface and to the etcd implementation. +2. Add `AppliedIndex() uint64` if not already exposed by a public method. + +### Phase 2: coordinator lease +1. Add `leaseState` field to `Coordinate` and to each shard inside + `ShardedCoordinator`. +2. Implement `LeaseRead(ctx)` and `LeaseReadForKey(ctx, key)`. +3. Wrap `Propose` calls in coordinator to refresh lease on success. +4. Wrap leadership-change detection (`refreshStatus` callback) to invalidate. + +### Phase 3: callers +1. Replace `LinearizableRead` with `LeaseRead` in + `adapter/redis_lua_context.go:newLuaScriptContext`. +2. Add `LeaseRead` at the entry of: + - `adapter/redis.go` `get`, `keyTypeAt`, `keys`, `exists`-family. + - `adapter/dynamodb.go` `getItem`, `query`, `scan`, + `transactGetItems`, `batchGetItem`. +3. No change to write paths beyond the implicit refresh via `Propose`. + +### Phase 4: tests +1. Unit test for `leaseState`: extend, expire, invalidate. +2. Integration test in `kv/`: `LeaseRead` returns immediately when lease + valid, falls back to `LinearizableRead` when expired, returns error and + invalidates when underlying `LinearizableRead` errs. +3. End-to-end test in `adapter/`: Lua script under sustained load issues + N scripts but only K underlying ReadIndex calls (K << N). +4. Jepsen workload addition: a partition test that asserts no stale-read + linearizability violation outside the lease window. + +### Phase 5: rollout +- Land Phases 1-3 behind no flag. The semantics are strictly equivalent or + stronger than today for non-Lua paths and weaker (but documented) for Lua. +- Monitor `LinearizableRead` call rate and Lua per-script latency before + and after deploy. + +--- + +## 6. Alternatives Considered + +### 6.1 Switch etcd/raft to ReadOnlyLeaseBased +One-line change: `ReadOnlyOption: etcdraft.ReadOnlyLeaseBased`. The leader +serves ReadIndex from local state without heartbeat broadcast, relying on +`CheckQuorum` for safety. + +Rejected because: +- Lease semantics are implicit and tied to etcd/raft internals. +- The lease boundary is not surfaced to the elastickv layer, so we cannot + track it for metrics or use it for non-engine reads. +- Future divergence: if elastickv ever needs to apply lease semantics to + read paths that do not call into the engine, this option does not help. + +The proposed design is essentially `ReadOnlyLeaseBased` reimplemented one +level up, with explicit timeout tracking. + +### 6.2 Term cache only, no time bound +Cache the current term; skip ReadIndex if term has not changed. + +Rejected because: +- Term changes are not the only safety trigger. A leader that loses quorum + but has not yet stepped down keeps the same term while serving stale + reads. CheckQuorum eventually catches it, but a term-only check has no + bound on the stale-read window. +- The proposed lease design subsumes the term check: leader transition + invalidates the lease, and time bounds the window even before a + transition is detected. + +### 6.3 Per-call heartbeat ack tracking inside engine +Hook into `handleHeartbeatResp` to refresh lease on every quorum ack. + +Rejected for the initial implementation because: +- Requires deeper integration with etcd/raft message handling. +- The marginal latency benefit over `Propose`-driven refresh is small under + any non-trivial write load. +- Can be added later without changing the API. + +--- + +## 7. Open Questions + +1. Should `LeaseDuration` be configurable per deployment, or kept as a + derived constant? Proposal: derived constant, exposed as a method. + Operators tune `defaultElectionTick` instead. +2. Should `LeaseRead` return the engine applied index or the store + `LastCommitTS()`? Proposal: applied index (matches `LinearizableRead`), + callers convert as needed. +3. Should a metric be added for lease hit/miss ratio? Proposal: yes, + `elastickv_lease_read_total{outcome="hit|miss|error"}`. From 61feb0edf17442657faee347ce0f85b7d93ee9ee Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 05:55:52 +0900 Subject: [PATCH 02/43] docs(lease-read): bump safety margin to 300ms --- docs/lease_read_design.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/lease_read_design.md b/docs/lease_read_design.md index 4f51e987..d29743cb 100644 --- a/docs/lease_read_design.md +++ b/docs/lease_read_design.md @@ -121,9 +121,9 @@ func (e *Engine) LeaseDuration() time.Duration Implementation: `electionTimeout - leaseSafetyMargin`, where `electionTimeout = defaultTickInterval * defaultElectionTick`. With current -config: `10ms * 100 - 100ms = 900 ms`. +config: `10ms * 100 - 300ms = 700 ms`. -`leaseSafetyMargin` (proposed: 100 ms) absorbs: +`leaseSafetyMargin` (proposed: 300 ms) absorbs: - Goroutine scheduling delay between heartbeat ack and lease refresh. - Wall-clock skew between leader and the partition's new leader candidate. @@ -292,7 +292,7 @@ write to commit. However: - Clock skew exceeding `leaseSafetyMargin`: lease may extend beyond `electionTimeout`, allowing a stale read after a successor leader has accepted writes. Mitigation: keep `leaseSafetyMargin` larger than the - documented clock-skew SLO of the deployment. Default 100 ms is consistent + documented clock-skew SLO of the deployment. Default 300 ms is consistent with the HLC physical window of 3 s used elsewhere. --- From 55d092ed3bbd8a552bd780a988406fae11c261e1 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 06:02:15 +0900 Subject: [PATCH 03/43] feat(lease-read): add LeaseRead to coordinator and engine surface - raftengine.LeaseProvider (optional interface): LeaseDuration() and AppliedIndex(). Implemented by the etcd engine; hashicorp engine and test stubs leave it unimplemented and fall back to LinearizableRead. - etcd Engine: LeaseDuration = electionTimeout - leaseSafetyMargin (300 ms), AppliedIndex from published Status. - kv.Coordinate / kv.ShardedCoordinator: LeaseRead and LeaseReadForKey using a per-coordinator (or per-shard for sharded) lease state. Fast path returns the engine applied index when the lease is unexpired; slow path runs LinearizableRead and refreshes the lease on success. - Coordinator interface gains LeaseRead/LeaseReadForKey. Test stubs delegate to LinearizableRead so existing semantics are preserved. No callers switched yet -- that lands in a follow-up commit. See docs/lease_read_design.md. --- adapter/distribution_server_test.go | 8 +++++ adapter/dynamodb_test.go | 8 +++++ adapter/redis_info_test.go | 6 ++++ adapter/redis_keys_pattern_test.go | 8 +++++ adapter/redis_retry_test.go | 8 +++++ adapter/s3_test.go | 8 +++++ internal/raftengine/engine.go | 12 ++++++++ internal/raftengine/etcd/engine.go | 45 +++++++++++++++++++++++++++ kv/coordinator.go | 31 +++++++++++++++++++ kv/leader_routed_store_test.go | 8 +++++ kv/lease_state.go | 47 +++++++++++++++++++++++++++++ kv/sharded_coordinator.go | 40 ++++++++++++++++++++++++ 12 files changed, 229 insertions(+) create mode 100644 kv/lease_state.go diff --git a/adapter/distribution_server_test.go b/adapter/distribution_server_test.go index 7427f7f2..945d8171 100644 --- a/adapter/distribution_server_test.go +++ b/adapter/distribution_server_test.go @@ -747,3 +747,11 @@ func (s *distributionCoordinatorStub) Clock() *kv.HLC { func (s *distributionCoordinatorStub) LinearizableRead(_ context.Context) (uint64, error) { return 0, nil } + +func (s *distributionCoordinatorStub) LeaseRead(ctx context.Context) (uint64, error) { + return s.LinearizableRead(ctx) +} + +func (s *distributionCoordinatorStub) LeaseReadForKey(ctx context.Context, _ []byte) (uint64, error) { + return s.LinearizableRead(ctx) +} diff --git a/adapter/dynamodb_test.go b/adapter/dynamodb_test.go index 07c872ec..3253518e 100644 --- a/adapter/dynamodb_test.go +++ b/adapter/dynamodb_test.go @@ -1859,3 +1859,11 @@ func (w *testCoordinatorWrapper) Clock() *kv.HLC { func (w *testCoordinatorWrapper) LinearizableRead(ctx context.Context) (uint64, error) { return w.inner.LinearizableRead(ctx) } + +func (w *testCoordinatorWrapper) LeaseRead(ctx context.Context) (uint64, error) { + return w.inner.LeaseRead(ctx) +} + +func (w *testCoordinatorWrapper) LeaseReadForKey(ctx context.Context, key []byte) (uint64, error) { + return w.inner.LeaseReadForKey(ctx, key) +} diff --git a/adapter/redis_info_test.go b/adapter/redis_info_test.go index 73fae884..f815f6ae 100644 --- a/adapter/redis_info_test.go +++ b/adapter/redis_info_test.go @@ -35,6 +35,12 @@ func (c *infoTestCoordinator) Clock() *kv.HLC { } func (c *infoTestCoordinator) LinearizableRead(_ context.Context) (uint64, error) { return 0, nil } +func (c *infoTestCoordinator) LeaseRead(ctx context.Context) (uint64, error) { + return c.LinearizableRead(ctx) +} +func (c *infoTestCoordinator) LeaseReadForKey(ctx context.Context, _ []byte) (uint64, error) { + return c.LinearizableRead(ctx) +} func TestRedisServer_Info_LeaderRole(t *testing.T) { r := &RedisServer{ diff --git a/adapter/redis_keys_pattern_test.go b/adapter/redis_keys_pattern_test.go index 8692c261..3d4b7a8a 100644 --- a/adapter/redis_keys_pattern_test.go +++ b/adapter/redis_keys_pattern_test.go @@ -65,6 +65,14 @@ func (s *stubAdapterCoordinator) LinearizableRead(_ context.Context) (uint64, er return 0, s.verifyLeaderErr } +func (s *stubAdapterCoordinator) LeaseRead(ctx context.Context) (uint64, error) { + return s.LinearizableRead(ctx) +} + +func (s *stubAdapterCoordinator) LeaseReadForKey(ctx context.Context, _ []byte) (uint64, error) { + return s.LinearizableRead(ctx) +} + func (s *stubAdapterCoordinator) VerifyLeaderCalls() int32 { if s == nil { return 0 diff --git a/adapter/redis_retry_test.go b/adapter/redis_retry_test.go index a765851a..1720418c 100644 --- a/adapter/redis_retry_test.go +++ b/adapter/redis_retry_test.go @@ -85,6 +85,14 @@ func (c *retryOnceCoordinator) LinearizableRead(_ context.Context) (uint64, erro return 0, nil } +func (c *retryOnceCoordinator) LeaseRead(ctx context.Context) (uint64, error) { + return c.LinearizableRead(ctx) +} + +func (c *retryOnceCoordinator) LeaseReadForKey(ctx context.Context, _ []byte) (uint64, error) { + return c.LinearizableRead(ctx) +} + type recordingConn struct { ctx any err string diff --git a/adapter/s3_test.go b/adapter/s3_test.go index 985a5431..cb816fa0 100644 --- a/adapter/s3_test.go +++ b/adapter/s3_test.go @@ -704,6 +704,14 @@ func (c *followerS3Coordinator) LinearizableRead(_ context.Context) (uint64, err return 0, kv.ErrLeaderNotFound } +func (c *followerS3Coordinator) LeaseRead(ctx context.Context) (uint64, error) { + return c.LinearizableRead(ctx) +} + +func (c *followerS3Coordinator) LeaseReadForKey(ctx context.Context, _ []byte) (uint64, error) { + return c.LinearizableRead(ctx) +} + func (c *followerS3Coordinator) RaftLeader() raft.ServerAddress { return raft.ServerAddress("leader") } diff --git a/internal/raftengine/engine.go b/internal/raftengine/engine.go index 5c1bbba1..117a96c6 100644 --- a/internal/raftengine/engine.go +++ b/internal/raftengine/engine.go @@ -67,6 +67,18 @@ type LeaderView interface { LinearizableRead(ctx context.Context) (uint64, error) } +// LeaseProvider is an optional capability implemented by engines that support +// leader-local lease reads. Callers that want lease-based reads should +// type-assert to this interface and fall back to LinearizableRead when the +// underlying engine does not implement it. +type LeaseProvider interface { + // LeaseDuration returns the time during which a lease holder can serve + // reads from local state without re-confirming leadership via ReadIndex. + LeaseDuration() time.Duration + // AppliedIndex returns the highest log index applied to the local FSM. + AppliedIndex() uint64 +} + type StatusReader interface { Status() Status } diff --git a/internal/raftengine/etcd/engine.go b/internal/raftengine/etcd/engine.go index 28d7f24a..b2ff71ec 100644 --- a/internal/raftengine/etcd/engine.go +++ b/internal/raftengine/etcd/engine.go @@ -24,6 +24,12 @@ const ( defaultTickInterval = 10 * time.Millisecond defaultHeartbeatTick = 10 // 100ms at 10ms interval defaultElectionTick = 100 // 1s at 10ms interval (10x heartbeat, etcd/raft recommended ratio) + // leaseSafetyMargin is subtracted from electionTimeout when computing the + // duration of a leader-local read lease. It absorbs goroutine scheduling + // delay between heartbeat ack and lease refresh, GC pauses on the leader, + // and bounded wall-clock skew between the leader and a partition's new + // leader candidate. See docs/lease_read_design.md for the safety argument. + leaseSafetyMargin = 300 * time.Millisecond // defaultMaxInflightMsg controls how many in-flight MsgApp messages Raft // allows per peer before waiting for an ACK (etcd/raft default: 256). // It also sets the per-peer dispatch channel capacity; total buffered memory @@ -108,6 +114,7 @@ type Engine struct { dataDir string fsmSnapDir string tickInterval time.Duration + electionTick int storage *etcdraft.MemoryStorage rawNode *etcdraft.RawNode @@ -290,6 +297,7 @@ func Open(ctx context.Context, cfg OpenConfig) (*Engine, error) { dataDir: prepared.cfg.DataDir, fsmSnapDir: filepath.Join(prepared.cfg.DataDir, fsmSnapDirName), tickInterval: prepared.cfg.TickInterval, + electionTick: prepared.cfg.ElectionTick, storage: prepared.disk.Storage, rawNode: rawNode, persist: prepared.disk.Persist, @@ -521,6 +529,43 @@ func (e *Engine) LinearizableRead(ctx context.Context) (uint64, error) { return e.submitRead(ctx, true) } +// LeaseDuration returns the time during which a lease holder can serve +// reads from local state without re-confirming leadership via ReadIndex. +// It is bounded by electionTimeout - leaseSafetyMargin so that the lease +// expires before a successor leader could realistically be elected and +// accept new writes elsewhere. +func (e *Engine) LeaseDuration() time.Duration { + if e == nil { + return 0 + } + tick := e.tickInterval + if tick <= 0 { + tick = defaultTickInterval + } + election := e.electionTick + if election <= 0 { + election = defaultElectionTick + } + d := time.Duration(election)*tick - leaseSafetyMargin + if d < 0 { + return 0 + } + return d +} + +// AppliedIndex returns the highest log index applied to the local FSM. +// Suitable for callers that need a non-blocking read fence equivalent to +// what LinearizableRead would have returned, paired with an external +// quorum confirmation (e.g. a valid lease). +func (e *Engine) AppliedIndex() uint64 { + if e == nil { + return 0 + } + e.mu.RLock() + defer e.mu.RUnlock() + return e.status.AppliedIndex +} + func (e *Engine) submitRead(ctx context.Context, waitApplied bool) (uint64, error) { if err := contextErr(ctx); err != nil { return 0, err diff --git a/kv/coordinator.go b/kv/coordinator.go index 8335c928..734be597 100644 --- a/kv/coordinator.go +++ b/kv/coordinator.go @@ -81,6 +81,7 @@ type Coordinate struct { clock *HLC connCache GRPCConnCache log *slog.Logger + lease leaseState } var _ Coordinator = (*Coordinate)(nil) @@ -90,9 +91,11 @@ type Coordinator interface { IsLeader() bool VerifyLeader() error LinearizableRead(ctx context.Context) (uint64, error) + LeaseRead(ctx context.Context) (uint64, error) RaftLeader() raft.ServerAddress IsLeaderForKey(key []byte) bool VerifyLeaderForKey(key []byte) error + LeaseReadForKey(ctx context.Context, key []byte) (uint64, error) RaftLeaderForKey(key []byte) raft.ServerAddress Clock() *HLC } @@ -215,6 +218,34 @@ func (c *Coordinate) LinearizableReadForKey(ctx context.Context, _ []byte) (uint return c.LinearizableRead(ctx) } +// LeaseRead returns a read fence backed by a leader-local lease when +// available, falling back to a full LinearizableRead when the lease has +// expired or the underlying engine does not implement LeaseProvider. +// +// The returned index is the engine's current applied index (fast path) or +// the index returned by LinearizableRead (slow path). Callers that resolve +// timestamps via store.LastCommitTS may discard the value. +func (c *Coordinate) LeaseRead(ctx context.Context) (uint64, error) { + lp, ok := c.engine.(raftengine.LeaseProvider) + if !ok { + return c.LinearizableRead(ctx) + } + if c.lease.valid(time.Now()) { + return lp.AppliedIndex(), nil + } + idx, err := c.LinearizableRead(ctx) + if err != nil { + c.lease.invalidate() + return 0, err + } + c.lease.extend(time.Now().Add(lp.LeaseDuration())) + return idx, nil +} + +func (c *Coordinate) LeaseReadForKey(ctx context.Context, _ []byte) (uint64, error) { + return c.LeaseRead(ctx) +} + func (c *Coordinate) nextStartTS() uint64 { return c.clock.Next() } diff --git a/kv/leader_routed_store_test.go b/kv/leader_routed_store_test.go index 446a12a7..63b55583 100644 --- a/kv/leader_routed_store_test.go +++ b/kv/leader_routed_store_test.go @@ -62,6 +62,14 @@ func (s *stubLeaderCoordinator) LinearizableReadForKey(ctx context.Context, _ [] return s.LinearizableRead(ctx) } +func (s *stubLeaderCoordinator) LeaseRead(ctx context.Context) (uint64, error) { + return s.LinearizableRead(ctx) +} + +func (s *stubLeaderCoordinator) LeaseReadForKey(ctx context.Context, _ []byte) (uint64, error) { + return s.LinearizableRead(ctx) +} + func (s *stubLeaderCoordinator) Clock() *HLC { if s.clock == nil { s.clock = NewHLC() diff --git a/kv/lease_state.go b/kv/lease_state.go new file mode 100644 index 00000000..b4a6861c --- /dev/null +++ b/kv/lease_state.go @@ -0,0 +1,47 @@ +package kv + +import ( + "sync/atomic" + "time" +) + +// leaseState tracks the wall-clock expiry of a leader-local read lease. +// All operations are lock-free via atomic.Pointer. +// +// A nil pointer means the lease has never been issued or has been +// invalidated. A non-nil pointer is the wall-clock instant after which +// the lease is considered expired; a caller comparing time.Now() against +// the loaded value can decide whether to skip a quorum confirmation. +type leaseState struct { + expiry atomic.Pointer[time.Time] +} + +// valid reports whether the lease is unexpired at now. +func (s *leaseState) valid(now time.Time) bool { + if s == nil { + return false + } + exp := s.expiry.Load() + if exp == nil { + return false + } + return now.Before(*exp) +} + +// extend sets the lease expiry to until. Concurrent calls race on the +// pointer swap; the most recent writer wins, which matches the desired +// semantics (any successful quorum confirmation refreshes the lease). +func (s *leaseState) extend(until time.Time) { + if s == nil { + return + } + s.expiry.Store(&until) +} + +// invalidate clears the lease so the next read takes the slow path. +func (s *leaseState) invalidate() { + if s == nil { + return + } + s.expiry.Store(nil) +} diff --git a/kv/sharded_coordinator.go b/kv/sharded_coordinator.go index 81bfab15..54071fed 100644 --- a/kv/sharded_coordinator.go +++ b/kv/sharded_coordinator.go @@ -21,6 +21,7 @@ type ShardGroup struct { Engine raftengine.Engine Store store.MVCCStore Txn Transactional + lease leaseState } const ( @@ -599,6 +600,45 @@ func (c *ShardedCoordinator) LinearizableReadForKey(ctx context.Context, key []b return linearizableReadEngineCtx(ctx, engineForGroup(g)) } +// LeaseRead routes through the default group's lease. See Coordinate.LeaseRead +// for semantics. +func (c *ShardedCoordinator) LeaseRead(ctx context.Context) (uint64, error) { + g, ok := c.groups[c.defaultGroup] + if !ok { + return 0, errors.WithStack(ErrLeaderNotFound) + } + return groupLeaseRead(ctx, g) +} + +// LeaseReadForKey performs the lease check on the shard group that owns key. +// Each group maintains its own lease since each group has independent +// leadership and term. +func (c *ShardedCoordinator) LeaseReadForKey(ctx context.Context, key []byte) (uint64, error) { + g, ok := c.groupForKey(key) + if !ok { + return 0, errors.WithStack(ErrLeaderNotFound) + } + return groupLeaseRead(ctx, g) +} + +func groupLeaseRead(ctx context.Context, g *ShardGroup) (uint64, error) { + engine := engineForGroup(g) + lp, ok := engine.(raftengine.LeaseProvider) + if !ok { + return linearizableReadEngineCtx(ctx, engine) + } + if g.lease.valid(time.Now()) { + return lp.AppliedIndex(), nil + } + idx, err := linearizableReadEngineCtx(ctx, engine) + if err != nil { + g.lease.invalidate() + return 0, err + } + g.lease.extend(time.Now().Add(lp.LeaseDuration())) + return idx, nil +} + func (c *ShardedCoordinator) Clock() *HLC { return c.clock } From 0d299fb1133eceb28ee176a2f03fffd6d7dadd11 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 06:05:07 +0900 Subject: [PATCH 04/43] feat(lease-read): use LeaseRead in Lua scripts and refresh on Dispatch - adapter/redis_lua_context.go: newLuaScriptContext now calls LeaseRead instead of LinearizableRead. Within the lease window, script start no longer pays a Raft round-trip; outside the window, the slow path runs LinearizableRead and refreshes the lease. - kv/coordinator.go: Coordinate.Dispatch refreshes the lease on a successful commit, since a committed entry implies majority append + ack. ShardedCoordinator does not yet refresh per-shard leases on Dispatch (leases there are still refreshed only on the LeaseRead slow path). --- adapter/redis_lua_context.go | 11 ++++++----- kv/coordinator.go | 16 +++++++++++++--- 2 files changed, 19 insertions(+), 8 deletions(-) diff --git a/adapter/redis_lua_context.go b/adapter/redis_lua_context.go index a9995d52..ad6d63cd 100644 --- a/adapter/redis_lua_context.go +++ b/adapter/redis_lua_context.go @@ -213,11 +213,12 @@ var luaRenameHandlers = map[redisValueType]luaRenameHandler{ } func newLuaScriptContext(ctx context.Context, server *RedisServer) (*luaScriptContext, error) { - // LinearizableRead confirms leadership via quorum AND waits for the local - // FSM to apply all committed entries, so startTS reflects the latest - // committed state. All subsequent reads within the script use snapshotGetAt - // (no per-call VerifyLeader), making VerifyLeader O(1) per script. - if _, err := server.coordinator.LinearizableRead(ctx); err != nil { + // LeaseRead confirms leadership at most once per LeaseDuration window; + // inside the window it returns immediately without a Raft round-trip. + // All subsequent reads within the script use snapshotGetAt at startTS, + // so leadership is verified at most once per script and amortised across + // scripts via the lease. + if _, err := server.coordinator.LeaseRead(ctx); err != nil { return nil, errors.WithStack(err) } startTS := server.readTS() diff --git a/kv/coordinator.go b/kv/coordinator.go index 734be597..673f417e 100644 --- a/kv/coordinator.go +++ b/kv/coordinator.go @@ -124,11 +124,21 @@ func (c *Coordinate) Dispatch(ctx context.Context, reqs *OperationGroup[OP]) (*C reqs.CommitTS = 0 } + var resp *CoordinateResponse + var err error if reqs.IsTxn { - return c.dispatchTxn(reqs.Elems, reqs.ReadKeys, reqs.StartTS, reqs.CommitTS) + resp, err = c.dispatchTxn(reqs.Elems, reqs.ReadKeys, reqs.StartTS, reqs.CommitTS) + } else { + resp, err = c.dispatchRaw(reqs.Elems) } - - return c.dispatchRaw(reqs.Elems) + if err == nil { + // A successful dispatch implies majority append + ack; treat it as a + // fresh quorum confirmation and extend the lease. + if lp, ok := c.engine.(raftengine.LeaseProvider); ok { + c.lease.extend(time.Now().Add(lp.LeaseDuration())) + } + } + return resp, err } func (c *Coordinate) IsLeader() bool { From fde954034eae45b3d5698484fe6f628fc85e2029 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 06:07:51 +0900 Subject: [PATCH 05/43] feat(lease-read): wrap Redis GET and DynamoDB getItem in LeaseRead Bounds the stale-read window for two paths that previously read local store state with no quorum check at all. Within the lease window the cost is one atomic load + one wall-clock compare; outside the window callers pay one LinearizableRead, after which the lease is refreshed for ~LeaseDuration. DynamoDB getItem grew over the cyclop limit, so the input parsing / validation portion is split into parseGetItemInput. Other read paths (KEYS, EXISTS, ZSet/Hash/List/Set/Stream readers, DynamoDB query/scan/transactGet) still rely on the lease being kept warm by Lua scripts and successful Dispatch calls. They can be wrapped explicitly in a follow-up. --- adapter/dynamodb.go | 21 ++++++++++++++++----- adapter/redis.go | 4 ++++ 2 files changed, 20 insertions(+), 5 deletions(-) diff --git a/adapter/dynamodb.go b/adapter/dynamodb.go index 167b59e2..9bfdcd0b 100644 --- a/adapter/dynamodb.go +++ b/adapter/dynamodb.go @@ -1340,26 +1340,37 @@ func (d *DynamoDBServer) commitItemWrite(ctx context.Context, req *kv.OperationG return nil } -func (d *DynamoDBServer) getItem(w http.ResponseWriter, r *http.Request) { +func (d *DynamoDBServer) parseGetItemInput(w http.ResponseWriter, r *http.Request) (getItemInput, bool) { body, err := io.ReadAll(maxDynamoBodyReader(w, r)) if err != nil { writeDynamoError(w, http.StatusBadRequest, dynamoErrValidation, err.Error()) - return + return getItemInput{}, false } var in getItemInput if err := json.Unmarshal(body, &in); err != nil { writeDynamoError(w, http.StatusBadRequest, dynamoErrValidation, err.Error()) - return + return getItemInput{}, false } if strings.TrimSpace(in.TableName) == "" { writeDynamoError(w, http.StatusBadRequest, dynamoErrValidation, "missing table name") - return + return getItemInput{}, false } if err := d.ensureLegacyTableMigration(r.Context(), in.TableName); err != nil { writeDynamoErrorFromErr(w, err) - return + return getItemInput{}, false } + return in, true +} +func (d *DynamoDBServer) getItem(w http.ResponseWriter, r *http.Request) { + in, ok := d.parseGetItemInput(w, r) + if !ok { + return + } + if _, err := d.coordinator.LeaseRead(r.Context()); err != nil { + writeDynamoError(w, http.StatusInternalServerError, dynamoErrInternal, err.Error()) + return + } readTS := d.resolveDynamoReadTS(in.ConsistentRead) schema, exists, err := d.loadTableSchemaAt(r.Context(), in.TableName, readTS) if err != nil { diff --git a/adapter/redis.go b/adapter/redis.go index a2c5cc7c..f7a7d1db 100644 --- a/adapter/redis.go +++ b/adapter/redis.go @@ -981,6 +981,10 @@ func (r *RedisServer) get(conn redcon.Conn, cmd redcon.Command) { return } + if _, err := r.coordinator.LeaseReadForKey(context.Background(), key); err != nil { + conn.WriteError(err.Error()) + return + } readTS := r.readTS() typ, err := r.keyTypeAt(context.Background(), key, readTS) if err != nil { From 2a70b72d43c451d0d7a7ccf2995eb9ac94675fc3 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 06:09:18 +0900 Subject: [PATCH 06/43] test(lease-read): unit tests for leaseState --- kv/lease_state_test.go | 97 ++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 97 insertions(+) create mode 100644 kv/lease_state_test.go diff --git a/kv/lease_state_test.go b/kv/lease_state_test.go new file mode 100644 index 00000000..21e428a6 --- /dev/null +++ b/kv/lease_state_test.go @@ -0,0 +1,97 @@ +package kv + +import ( + "testing" + "time" + + "github.com/stretchr/testify/require" +) + +func TestLeaseState_NilReceiverIsAlwaysExpired(t *testing.T) { + t.Parallel() + var s *leaseState + require.False(t, s.valid(time.Now())) + s.extend(time.Now().Add(time.Hour)) // must not panic + s.invalidate() // must not panic + require.False(t, s.valid(time.Now())) +} + +func TestLeaseState_ZeroValueIsExpired(t *testing.T) { + t.Parallel() + var s leaseState + require.False(t, s.valid(time.Now())) +} + +func TestLeaseState_ExtendAndExpire(t *testing.T) { + t.Parallel() + var s leaseState + now := time.Now() + s.extend(now.Add(50 * time.Millisecond)) + + require.True(t, s.valid(now)) + require.True(t, s.valid(now.Add(49*time.Millisecond))) + require.False(t, s.valid(now.Add(50*time.Millisecond))) + require.False(t, s.valid(now.Add(time.Hour))) +} + +func TestLeaseState_InvalidateClears(t *testing.T) { + t.Parallel() + var s leaseState + now := time.Now() + s.extend(now.Add(time.Hour)) + require.True(t, s.valid(now)) + + s.invalidate() + require.False(t, s.valid(now)) +} + +func TestLeaseState_ExtendOverwritesEarlierAndLater(t *testing.T) { + t.Parallel() + var s leaseState + now := time.Now() + + s.extend(now.Add(time.Hour)) + require.True(t, s.valid(now.Add(30*time.Minute))) + + // Shorter extension overwrites — last writer wins, mirroring the + // single-atomic-pointer semantics. Practically this is rare because + // real callers always extend by LeaseDuration() relative to "now", + // which monotonically advances; documenting the behavior here. + s.extend(now.Add(time.Minute)) + require.False(t, s.valid(now.Add(30*time.Minute))) +} + +func TestLeaseState_ConcurrentExtendAndRead(t *testing.T) { + t.Parallel() + var s leaseState + stop := make(chan struct{}) + done := make(chan struct{}, 2) + + go func() { + defer func() { done <- struct{}{} }() + for { + select { + case <-stop: + return + default: + s.extend(time.Now().Add(time.Second)) + } + } + }() + go func() { + defer func() { done <- struct{}{} }() + for { + select { + case <-stop: + return + default: + _ = s.valid(time.Now()) + } + } + }() + + time.Sleep(20 * time.Millisecond) + close(stop) + <-done + <-done +} From b9ac2b8fad2b3648c281005258368240864d079e Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 06:13:27 +0900 Subject: [PATCH 07/43] style(lease-read): linter formatting on lease_state_test.go --- kv/lease_state_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kv/lease_state_test.go b/kv/lease_state_test.go index 21e428a6..2b623430 100644 --- a/kv/lease_state_test.go +++ b/kv/lease_state_test.go @@ -11,8 +11,8 @@ func TestLeaseState_NilReceiverIsAlwaysExpired(t *testing.T) { t.Parallel() var s *leaseState require.False(t, s.valid(time.Now())) - s.extend(time.Now().Add(time.Hour)) // must not panic - s.invalidate() // must not panic + s.extend(time.Now().Add(time.Hour)) // must not panic + s.invalidate() // must not panic require.False(t, s.valid(time.Now())) } From f28bf0d5d20dee2f711c57525d4ff7a04ddc502f Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 06:23:04 +0900 Subject: [PATCH 08/43] fix(lease-read): pre-Propose time capture and monotonic extend Two correctness fixes from PR #549 reviews: 1. extend was called with time.Now() AFTER Propose / LinearizableRead returned. Apply-queue depth, FSM batch latency, and goroutine scheduling jitter could push the effective lease window past electionTimeout, eating into the safety margin. Coordinate.Dispatch and Coordinate.LeaseRead now sample t0 := time.Now() BEFORE the underlying Propose/LinearizableRead, and use t0 + LeaseDuration as the lease expiry. ShardedCoordinator's groupLeaseRead does the same. 2. leaseState.extend was last-writer-wins. An out-of-order writer that sampled time.Now() earlier could overwrite a fresher extension and prematurely shorten the lease. extend now uses a CAS loop that only replaces the expiry if the new instant is strictly after the current one. invalidate() still wins unconditionally so leadership-loss callbacks are not blocked. Tests updated: the "shorter extension overwrites" assertion is replaced by "extend is monotonic" plus an "invalidate beats extend" test that codifies the new contract. --- kv/coordinator.go | 18 +++++++++++++++--- kv/lease_state.go | 18 ++++++++++++++---- kv/lease_state_test.go | 29 +++++++++++++++++++++++------ kv/sharded_coordinator.go | 5 ++++- 4 files changed, 56 insertions(+), 14 deletions(-) diff --git a/kv/coordinator.go b/kv/coordinator.go index 673f417e..a788c0e9 100644 --- a/kv/coordinator.go +++ b/kv/coordinator.go @@ -124,6 +124,13 @@ func (c *Coordinate) Dispatch(ctx context.Context, reqs *OperationGroup[OP]) (*C reqs.CommitTS = 0 } + // Sample the clock BEFORE dispatching so the lease extension reflects + // the moment we know the leader was alive at quorum confirmation, not + // the moment dispatch returned. Otherwise apply-queue depth and + // scheduling jitter could push the effective lease window past + // electionTimeout, allowing a stale leader to serve reads beyond the + // safety bound. + dispatchStart := time.Now() var resp *CoordinateResponse var err error if reqs.IsTxn { @@ -133,9 +140,10 @@ func (c *Coordinate) Dispatch(ctx context.Context, reqs *OperationGroup[OP]) (*C } if err == nil { // A successful dispatch implies majority append + ack; treat it as a - // fresh quorum confirmation and extend the lease. + // fresh quorum confirmation and extend the lease using the + // pre-dispatch timestamp. if lp, ok := c.engine.(raftengine.LeaseProvider); ok { - c.lease.extend(time.Now().Add(lp.LeaseDuration())) + c.lease.extend(dispatchStart.Add(lp.LeaseDuration())) } } return resp, err @@ -243,12 +251,16 @@ func (c *Coordinate) LeaseRead(ctx context.Context) (uint64, error) { if c.lease.valid(time.Now()) { return lp.AppliedIndex(), nil } + // Sample BEFORE LinearizableRead so the lease window starts at the + // real quorum confirmation instant, not after the heartbeat round + // returned. See Coordinate.Dispatch for the same rationale. + readStart := time.Now() idx, err := c.LinearizableRead(ctx) if err != nil { c.lease.invalidate() return 0, err } - c.lease.extend(time.Now().Add(lp.LeaseDuration())) + c.lease.extend(readStart.Add(lp.LeaseDuration())) return idx, nil } diff --git a/kv/lease_state.go b/kv/lease_state.go index b4a6861c..404a5a2d 100644 --- a/kv/lease_state.go +++ b/kv/lease_state.go @@ -28,14 +28,24 @@ func (s *leaseState) valid(now time.Time) bool { return now.Before(*exp) } -// extend sets the lease expiry to until. Concurrent calls race on the -// pointer swap; the most recent writer wins, which matches the desired -// semantics (any successful quorum confirmation refreshes the lease). +// extend sets the lease expiry to until iff until is strictly after the +// currently stored expiry (or no expiry is stored). The CAS loop prevents +// an out-of-order writer that sampled time.Now() earlier from overwriting +// a fresher extension and prematurely shortening the lease, while still +// allowing invalidate() (which Stores nil) to win unconditionally. func (s *leaseState) extend(until time.Time) { if s == nil { return } - s.expiry.Store(&until) + for { + current := s.expiry.Load() + if current != nil && !until.After(*current) { + return + } + if s.expiry.CompareAndSwap(current, &until) { + return + } + } } // invalidate clears the lease so the next read takes the slow path. diff --git a/kv/lease_state_test.go b/kv/lease_state_test.go index 2b623430..a224fd0b 100644 --- a/kv/lease_state_test.go +++ b/kv/lease_state_test.go @@ -45,7 +45,7 @@ func TestLeaseState_InvalidateClears(t *testing.T) { require.False(t, s.valid(now)) } -func TestLeaseState_ExtendOverwritesEarlierAndLater(t *testing.T) { +func TestLeaseState_ExtendIsMonotonic(t *testing.T) { t.Parallel() var s leaseState now := time.Now() @@ -53,12 +53,29 @@ func TestLeaseState_ExtendOverwritesEarlierAndLater(t *testing.T) { s.extend(now.Add(time.Hour)) require.True(t, s.valid(now.Add(30*time.Minute))) - // Shorter extension overwrites — last writer wins, mirroring the - // single-atomic-pointer semantics. Practically this is rare because - // real callers always extend by LeaseDuration() relative to "now", - // which monotonically advances; documenting the behavior here. + // A shorter extension must NOT regress the lease: an out-of-order + // writer that sampled time.Now() earlier could otherwise prematurely + // expire a freshly extended lease and force callers into the slow + // path while the leader is still confirmed. s.extend(now.Add(time.Minute)) - require.False(t, s.valid(now.Add(30*time.Minute))) + require.True(t, s.valid(now.Add(30*time.Minute))) + + // A strictly longer extension wins. + s.extend(now.Add(2 * time.Hour)) + require.True(t, s.valid(now.Add(90*time.Minute))) +} + +func TestLeaseState_InvalidateBeatsConcurrentExtend(t *testing.T) { + t.Parallel() + var s leaseState + now := time.Now() + s.extend(now.Add(time.Hour)) + + // invalidate stores nil unconditionally, even when the current expiry + // is in the future. Otherwise leadership-loss callbacks would be + // powerless once a lease is in place. + s.invalidate() + require.False(t, s.valid(now)) } func TestLeaseState_ConcurrentExtendAndRead(t *testing.T) { diff --git a/kv/sharded_coordinator.go b/kv/sharded_coordinator.go index 54071fed..3d5ca339 100644 --- a/kv/sharded_coordinator.go +++ b/kv/sharded_coordinator.go @@ -630,12 +630,15 @@ func groupLeaseRead(ctx context.Context, g *ShardGroup) (uint64, error) { if g.lease.valid(time.Now()) { return lp.AppliedIndex(), nil } + // Sample BEFORE LinearizableRead; the lease window must start at the + // quorum confirmation instant, not after the read returned. + readStart := time.Now() idx, err := linearizableReadEngineCtx(ctx, engine) if err != nil { g.lease.invalidate() return 0, err } - g.lease.extend(time.Now().Add(lp.LeaseDuration())) + g.lease.extend(readStart.Add(lp.LeaseDuration())) return idx, nil } From f86c689845f903d0851ed511fcd2ad5136ef7c5a Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 06:29:55 +0900 Subject: [PATCH 09/43] fix(lease-read): invalidate lease on leader-loss Plumbs a leader-loss callback from the etcd engine into Coordinate / ShardGroup. Before this fix a former leader (graceful transfer, CheckQuorum step-down, shutdown) kept serving fast-path reads from its local LastCommitTS for up to LeaseDuration after a successor leader was already accepting writes - a phantom-read window of up to 700 ms flagged HIGH by data-loss, concurrency, and consistency reviewers on PR #549. Changes: - raftengine.LeaseProvider gains RegisterLeaderLossCallback(fn func()). - etcd Engine stores callbacks, fires them synchronously from refreshStatus when the previous state was Leader and the new state isn't (which already covers shutdown via the closed override that rewrites status.State to StateShutdown before this check). - Coordinate registers c.lease.invalidate as the callback in NewCoordinatorWithEngine. - ShardedCoordinator registers per-shard g.lease.invalidate in NewShardedCoordinator. Test stubs do not implement LeaseProvider so they are unaffected. Only the etcd engine implements LeaseProvider; hashicorp engine and test stubs continue to fall back to LinearizableRead. --- internal/raftengine/engine.go | 7 ++++++ internal/raftengine/etcd/engine.go | 38 ++++++++++++++++++++++++++++++ kv/coordinator.go | 6 +++++ kv/sharded_coordinator.go | 6 +++++ 4 files changed, 57 insertions(+) diff --git a/internal/raftengine/engine.go b/internal/raftengine/engine.go index 117a96c6..12546df1 100644 --- a/internal/raftengine/engine.go +++ b/internal/raftengine/engine.go @@ -77,6 +77,13 @@ type LeaseProvider interface { LeaseDuration() time.Duration // AppliedIndex returns the highest log index applied to the local FSM. AppliedIndex() uint64 + // RegisterLeaderLossCallback registers fn to be invoked whenever the + // local node leaves the leader role (graceful transfer, partition + // step-down, or shutdown). Callers use this to invalidate any + // leader-local lease they hold so the next read takes the slow path. + // Multiple callbacks can be registered; each fires synchronously from + // the engine's status refresh and must not block. + RegisterLeaderLossCallback(fn func()) } type StatusReader interface { diff --git a/internal/raftengine/etcd/engine.go b/internal/raftengine/etcd/engine.go index b2ff71ec..e4f6dcae 100644 --- a/internal/raftengine/etcd/engine.go +++ b/internal/raftengine/etcd/engine.go @@ -172,6 +172,13 @@ type Engine struct { dispatchDropCount atomic.Uint64 dispatchErrorCount atomic.Uint64 + // leaderLossCbsMu guards the slice of callbacks invoked when the node + // transitions out of the leader role (graceful transfer, partition + // step-down, shutdown). Callbacks fire synchronously from + // refreshStatus and must not block. + leaderLossCbsMu sync.Mutex + leaderLossCbs []func() + pendingProposals map[uint64]proposalRequest pendingReads map[uint64]readRequest pendingConfigs map[uint64]adminRequest @@ -566,6 +573,32 @@ func (e *Engine) AppliedIndex() uint64 { return e.status.AppliedIndex } +// RegisterLeaderLossCallback registers fn to fire on every transition +// out of the leader role (including LeadTransferee != 0 hand-off, +// CheckQuorum step-down, and shutdown). Used by lease-read callers to +// invalidate cached lease state so the next read takes the slow path. +func (e *Engine) RegisterLeaderLossCallback(fn func()) { + if e == nil || fn == nil { + return + } + e.leaderLossCbsMu.Lock() + e.leaderLossCbs = append(e.leaderLossCbs, fn) + e.leaderLossCbsMu.Unlock() +} + +// fireLeaderLossCallbacks invokes all registered callbacks. Safe to call +// from refreshStatus while holding no engine locks; callbacks run +// synchronously and must not block. +func (e *Engine) fireLeaderLossCallbacks() { + e.leaderLossCbsMu.Lock() + cbs := make([]func(), len(e.leaderLossCbs)) + copy(cbs, e.leaderLossCbs) + e.leaderLossCbsMu.Unlock() + for _, fn := range cbs { + fn() + } +} + func (e *Engine) submitRead(ctx context.Context, waitApplied bool) (uint64, error) { if err := contextErr(ctx); err != nil { return 0, err @@ -1655,6 +1688,11 @@ func (e *Engine) refreshStatus() { } if previous == raftengine.StateLeader && status.State != raftengine.StateLeader { e.failPending(errors.WithStack(errNotLeader)) + // Notify lease holders so they invalidate any cached lease; + // without this hook, a former leader keeps serving fast-path + // reads from local state for up to LeaseDuration after a + // successor leader is already accepting writes. + e.fireLeaderLossCallbacks() } } diff --git a/kv/coordinator.go b/kv/coordinator.go index a788c0e9..cf7cde30 100644 --- a/kv/coordinator.go +++ b/kv/coordinator.go @@ -55,6 +55,12 @@ func NewCoordinatorWithEngine(txm Transactional, engine raftengine.Engine, opts for _, opt := range opts { opt(c) } + // Register a leader-loss hook so the lease is invalidated the instant + // the engine notices a state transition out of the leader role, + // rather than waiting for wall-clock expiry of the current lease. + if lp, ok := engine.(raftengine.LeaseProvider); ok { + lp.RegisterLeaderLossCallback(c.lease.invalidate) + } return c } diff --git a/kv/sharded_coordinator.go b/kv/sharded_coordinator.go index 3d5ca339..d941baf5 100644 --- a/kv/sharded_coordinator.go +++ b/kv/sharded_coordinator.go @@ -49,6 +49,12 @@ func NewShardedCoordinator(engine *distribution.Engine, groups map[uint64]*Shard router := NewShardRouter(engine) for gid, g := range groups { router.Register(gid, g.Txn, g.Store) + // Per-shard leader-loss hook: when this group's engine notices + // a state transition out of leader, drop the lease so the next + // LeaseReadForKey on that shard takes the slow path. + if lp, ok := g.Engine.(raftengine.LeaseProvider); ok { + lp.RegisterLeaderLossCallback(g.lease.invalidate) + } } return &ShardedCoordinator{ engine: engine, From 9994bbb76877ae732d71bae88f53a266777fb452 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 06:36:17 +0900 Subject: [PATCH 10/43] fix(lease-read): per-shard lease refresh on Dispatch via leaseRefreshingTxn Wraps each ShardGroup.Txn in a leaseRefreshingTxn that extends the per-shard lease on every successful Commit/Abort. All ShardedCoordinator dispatch paths flow through g.Txn (raw via router.Commit, dispatchSingleShardTxn, dispatchTxn 2PC primary/secondary, and dispatchDelPrefixBroadcast), so this single hook catches them all. The wrapper samples time.Now() before delegating to inner.Commit so the lease window starts at quorum confirmation rather than after Commit returned, mirroring the same fix already applied to Coordinate.Dispatch. Addresses HIGH/MED finding from PR #549 review (sharded path was previously refreshed only via the LeaseRead slow path). --- kv/sharded_coordinator.go | 39 +++++++++++++++++++++++++++++++++++++++ 1 file changed, 39 insertions(+) diff --git a/kv/sharded_coordinator.go b/kv/sharded_coordinator.go index d941baf5..9d238fa4 100644 --- a/kv/sharded_coordinator.go +++ b/kv/sharded_coordinator.go @@ -24,6 +24,39 @@ type ShardGroup struct { lease leaseState } +// leaseRefreshingTxn wraps a Transactional so every successful Commit / +// Abort extends its shard's lease, treating committed Raft entries as +// fresh quorum confirmations. Mirrors Coordinate.Dispatch's lease hook +// for the per-shard case. +type leaseRefreshingTxn struct { + inner Transactional + g *ShardGroup +} + +func (t *leaseRefreshingTxn) Commit(reqs []*pb.Request) (*TransactionResponse, error) { + start := time.Now() + resp, err := t.inner.Commit(reqs) + if err != nil { + return resp, errors.WithStack(err) + } + if lp, ok := t.g.Engine.(raftengine.LeaseProvider); ok { + t.g.lease.extend(start.Add(lp.LeaseDuration())) + } + return resp, nil +} + +func (t *leaseRefreshingTxn) Abort(reqs []*pb.Request) (*TransactionResponse, error) { + start := time.Now() + resp, err := t.inner.Abort(reqs) + if err != nil { + return resp, errors.WithStack(err) + } + if lp, ok := t.g.Engine.(raftengine.LeaseProvider); ok { + t.g.lease.extend(start.Add(lp.LeaseDuration())) + } + return resp, nil +} + const ( txnPhaseCount = 2 @@ -48,6 +81,12 @@ type ShardedCoordinator struct { func NewShardedCoordinator(engine *distribution.Engine, groups map[uint64]*ShardGroup, defaultGroup uint64, clock *HLC, st store.MVCCStore) *ShardedCoordinator { router := NewShardRouter(engine) for gid, g := range groups { + // Wrap Txn so every successful Commit/Abort refreshes the + // per-shard lease. All dispatch paths (raw via router.Commit, + // dispatchSingleShardTxn, dispatchTxn 2PC, dispatchDelPrefix + // broadcast) flow through g.Txn so this single hook catches + // them all. + g.Txn = &leaseRefreshingTxn{inner: g.Txn, g: g} router.Register(gid, g.Txn, g.Store) // Per-shard leader-loss hook: when this group's engine notices // a state transition out of leader, drop the lease so the next From 5b3ab7c1b0af4bc95ba69141045b44e1851e81d6 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 06:38:00 +0900 Subject: [PATCH 11/43] test(lease-read): coordinator LeaseRead and leader-loss invalidation Adds five tests covering paths that were untested in PR #549: - Coordinate.LeaseRead fast path: pre-extended lease returns AppliedIndex with zero engine.LinearizableRead calls. - Coordinate.LeaseRead slow path success: first call invokes LinearizableRead and refreshes the lease; second call hits fast path. - Coordinate.LeaseRead slow path error: LinearizableRead failure invalidates the lease so subsequent calls keep taking the slow path. - LeaseProvider type-assertion fallback: an engine that does not implement LeaseProvider always falls back to LinearizableRead. - Leader-loss callback: NewCoordinatorWithEngine registers exactly one callback; firing it invalidates a previously valid lease. Test stubs (fakeLeaseEngine, nonLeaseEngine) live in this file rather than poll the existing distributed test scaffolding, keeping the tests hermetic. --- kv/lease_read_test.go | 184 ++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 184 insertions(+) create mode 100644 kv/lease_read_test.go diff --git a/kv/lease_read_test.go b/kv/lease_read_test.go new file mode 100644 index 00000000..a11022fd --- /dev/null +++ b/kv/lease_read_test.go @@ -0,0 +1,184 @@ +package kv + +import ( + "context" + "errors" + "sync/atomic" + "testing" + "time" + + "github.com/bootjp/elastickv/internal/raftengine" + "github.com/stretchr/testify/require" +) + +// fakeLeaseEngine implements raftengine.Engine + raftengine.LeaseProvider +// with controllable applied index, lease duration, and LinearizableRead +// behaviour, plus call counters for assertions. +type fakeLeaseEngine struct { + applied uint64 + leaseDur time.Duration + linearizableErr error + linearizableCalls atomic.Int32 + leaderLossCallbacks []func() + leaderLossCallbacksMu atomic.Bool + registerLeaderLossCalled atomic.Int32 +} + +func (e *fakeLeaseEngine) State() raftengine.State { return raftengine.StateLeader } +func (e *fakeLeaseEngine) Leader() raftengine.LeaderInfo { + return raftengine.LeaderInfo{ID: "n1", Address: "127.0.0.1:0"} +} +func (e *fakeLeaseEngine) VerifyLeader(context.Context) error { return nil } +func (e *fakeLeaseEngine) LinearizableRead(context.Context) (uint64, error) { + e.linearizableCalls.Add(1) + if e.linearizableErr != nil { + return 0, e.linearizableErr + } + return e.applied, nil +} +func (e *fakeLeaseEngine) Status() raftengine.Status { + return raftengine.Status{State: raftengine.StateLeader, AppliedIndex: e.applied} +} +func (e *fakeLeaseEngine) Configuration(context.Context) (raftengine.Configuration, error) { + return raftengine.Configuration{}, nil +} +func (e *fakeLeaseEngine) Propose(context.Context, []byte) (*raftengine.ProposalResult, error) { + return &raftengine.ProposalResult{}, nil +} +func (e *fakeLeaseEngine) Close() error { return nil } +func (e *fakeLeaseEngine) LeaseDuration() time.Duration { return e.leaseDur } +func (e *fakeLeaseEngine) AppliedIndex() uint64 { return e.applied } +func (e *fakeLeaseEngine) RegisterLeaderLossCallback(fn func()) { + e.registerLeaderLossCalled.Add(1) + for !e.leaderLossCallbacksMu.CompareAndSwap(false, true) { + } + e.leaderLossCallbacks = append(e.leaderLossCallbacks, fn) + e.leaderLossCallbacksMu.Store(false) +} + +func (e *fakeLeaseEngine) fireLeaderLoss() { + for !e.leaderLossCallbacksMu.CompareAndSwap(false, true) { + } + cbs := append([]func(){}, e.leaderLossCallbacks...) + e.leaderLossCallbacksMu.Store(false) + for _, cb := range cbs { + cb() + } +} + +// nonLeaseEngine implements only raftengine.Engine, not LeaseProvider. +// Used to verify the type-assertion fallback. +type nonLeaseEngine struct { + linearizableCalls atomic.Int32 + linearizableErr error +} + +func (e *nonLeaseEngine) State() raftengine.State { return raftengine.StateLeader } +func (e *nonLeaseEngine) Leader() raftengine.LeaderInfo { + return raftengine.LeaderInfo{ID: "n1", Address: "127.0.0.1:0"} +} +func (e *nonLeaseEngine) VerifyLeader(context.Context) error { return nil } +func (e *nonLeaseEngine) LinearizableRead(context.Context) (uint64, error) { + e.linearizableCalls.Add(1) + if e.linearizableErr != nil { + return 0, e.linearizableErr + } + return 42, nil +} +func (e *nonLeaseEngine) Status() raftengine.Status { + return raftengine.Status{State: raftengine.StateLeader, AppliedIndex: 42} +} +func (e *nonLeaseEngine) Configuration(context.Context) (raftengine.Configuration, error) { + return raftengine.Configuration{}, nil +} +func (e *nonLeaseEngine) Propose(context.Context, []byte) (*raftengine.ProposalResult, error) { + return &raftengine.ProposalResult{}, nil +} +func (e *nonLeaseEngine) Close() error { return nil } + +// --- Coordinate.LeaseRead ----------------------------------------------- + +func TestCoordinate_LeaseRead_FastPathSkipsEngine(t *testing.T) { + t.Parallel() + eng := &fakeLeaseEngine{applied: 100, leaseDur: time.Hour} + c := NewCoordinatorWithEngine(nil, eng) + + c.lease.extend(time.Now().Add(time.Hour)) + + idx, err := c.LeaseRead(context.Background()) + require.NoError(t, err) + require.Equal(t, uint64(100), idx) + require.Equal(t, int32(0), eng.linearizableCalls.Load()) +} + +func TestCoordinate_LeaseRead_SlowPathRefreshesLease(t *testing.T) { + t.Parallel() + eng := &fakeLeaseEngine{applied: 50, leaseDur: time.Hour} + c := NewCoordinatorWithEngine(nil, eng) + + idx, err := c.LeaseRead(context.Background()) + require.NoError(t, err) + require.Equal(t, uint64(50), idx) + require.Equal(t, int32(1), eng.linearizableCalls.Load()) + + require.True(t, c.lease.valid(time.Now())) + + idx2, err := c.LeaseRead(context.Background()) + require.NoError(t, err) + require.Equal(t, uint64(50), idx2) + require.Equal(t, int32(1), eng.linearizableCalls.Load(), "second read should hit fast path") +} + +func TestCoordinate_LeaseRead_ErrorInvalidatesLease(t *testing.T) { + t.Parallel() + sentinel := errors.New("read-index failed") + eng := &fakeLeaseEngine{applied: 7, leaseDur: time.Hour, linearizableErr: sentinel} + c := NewCoordinatorWithEngine(nil, eng) + + c.lease.extend(time.Now().Add(time.Hour)) + c.lease.invalidate() // force slow path + + _, err := c.LeaseRead(context.Background()) + require.ErrorIs(t, err, sentinel) + require.False(t, c.lease.valid(time.Now())) + require.Equal(t, int32(1), eng.linearizableCalls.Load()) + + // Subsequent call also takes slow path because lease is invalidated. + _, err = c.LeaseRead(context.Background()) + require.ErrorIs(t, err, sentinel) + require.Equal(t, int32(2), eng.linearizableCalls.Load()) +} + +func TestCoordinate_LeaseRead_FallbackWhenEngineLacksLeaseProvider(t *testing.T) { + t.Parallel() + eng := &nonLeaseEngine{} + c := NewCoordinatorWithEngine(nil, eng) + + idx, err := c.LeaseRead(context.Background()) + require.NoError(t, err) + require.Equal(t, uint64(42), idx) + require.Equal(t, int32(1), eng.linearizableCalls.Load()) + + // Without LeaseProvider the lease never becomes valid; every call + // goes through LinearizableRead. + _, err = c.LeaseRead(context.Background()) + require.NoError(t, err) + require.Equal(t, int32(2), eng.linearizableCalls.Load()) +} + +// --- Leader-loss invalidation hook -------------------------------------- + +func TestCoordinate_RegistersLeaderLossCallback(t *testing.T) { + t.Parallel() + eng := &fakeLeaseEngine{applied: 1, leaseDur: time.Hour} + c := NewCoordinatorWithEngine(nil, eng) + require.Equal(t, int32(1), eng.registerLeaderLossCalled.Load()) + + c.lease.extend(time.Now().Add(time.Hour)) + require.True(t, c.lease.valid(time.Now())) + + eng.fireLeaderLoss() + require.False(t, c.lease.valid(time.Now()), + "leader-loss callback must invalidate the lease") +} + From 4f24db6e6c81b2984e9eb25f569da73621df76ed Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 06:42:45 +0900 Subject: [PATCH 12/43] fix(lease-read): address external review (Copilot/CodeRabbit) - adapter/redis.go GET wraps the LeaseReadForKey call in a redisDispatchTimeout-bounded context. The slow path runs LinearizableRead which can block on quorum; without a deadline a stalled Raft would hang the GET indefinitely. - kv/sharded_lease_test.go adds tests for ShardedCoordinator per-shard lease isolation and per-shard leader-loss callback wiring (touch shard A, verify shard B's lease and engine are untouched). - kv/lease_read_test.go reformatted by gci. - docs/lease_read_design.md updated to reflect the actually-shipped API: LeaseProvider is a separate optional interface (not on LeaderView), includes RegisterLeaderLossCallback, and Phase 3 calls out which adapter sites are wrapped vs deferred. --- adapter/redis.go | 9 ++- docs/lease_read_design.md | 113 +++++++++++++++++++++++------------ kv/lease_read_test.go | 11 ++-- kv/sharded_lease_test.go | 121 ++++++++++++++++++++++++++++++++++++++ 4 files changed, 210 insertions(+), 44 deletions(-) create mode 100644 kv/sharded_lease_test.go diff --git a/adapter/redis.go b/adapter/redis.go index f7a7d1db..46673364 100644 --- a/adapter/redis.go +++ b/adapter/redis.go @@ -981,10 +981,17 @@ func (r *RedisServer) get(conn redcon.Conn, cmd redcon.Command) { return } - if _, err := r.coordinator.LeaseReadForKey(context.Background(), key); err != nil { + // Bounded context: LeaseReadForKey's slow path runs LinearizableRead, + // which can block until quorum is reached. Without a deadline a + // stalled Raft makes the GET hang indefinitely; cap it the same way + // the proxy path does. + leaseCtx, leaseCancel := context.WithTimeout(context.Background(), redisDispatchTimeout) + if _, err := r.coordinator.LeaseReadForKey(leaseCtx, key); err != nil { + leaseCancel() conn.WriteError(err.Error()) return } + leaseCancel() readTS := r.readTS() typ, err := r.keyTypeAt(context.Background(), key, readTS) if err != nil { diff --git a/docs/lease_read_design.md b/docs/lease_read_design.md index d29743cb..306acc7c 100644 --- a/docs/lease_read_design.md +++ b/docs/lease_read_design.md @@ -161,33 +161,44 @@ The lease is invalidated (set to nil) on: ### 3.5 API ```go +// internal/raftengine/engine.go — optional capability +type LeaseProvider interface { + LeaseDuration() time.Duration + AppliedIndex() uint64 + RegisterLeaderLossCallback(fn func()) +} + // kv/coordinator.go -type Coordinate interface { +type Coordinator interface { // ...existing... LeaseRead(ctx context.Context) (uint64, error) + LeaseReadForKey(ctx context.Context, key []byte) (uint64, error) } - -// kv/sharded_coordinator.go -func (c *ShardedCoordinator) LeaseReadForKey(ctx context.Context, key []byte) (uint64, error) ``` Returned index is the engine's applied index at the moment of return. Callers that use `store.LastCommitTS()` can ignore the index; callers that need an explicit fence can use it. -Pseudocode: +Pseudocode (matches `Coordinate.LeaseRead`; the sharded variant is the same +with per-shard `g.lease` and `g.Engine`): ```go func (c *Coordinate) LeaseRead(ctx context.Context) (uint64, error) { + lp, ok := c.engine.(raftengine.LeaseProvider) + if !ok { + return c.LinearizableRead(ctx) // hashicorp engine, test stubs + } if c.lease.valid(time.Now()) { - return c.engine.AppliedIndex(), nil + return lp.AppliedIndex(), nil } - idx, err := c.engine.LinearizableRead(ctx) + readStart := time.Now() // sample BEFORE + idx, err := c.LinearizableRead(ctx) if err != nil { c.lease.invalidate() return 0, err } - c.lease.extend(time.Now().Add(c.engine.LeaseDuration())) + c.lease.extend(readStart.Add(lp.LeaseDuration())) // monotonic CAS return idx, nil } ``` @@ -299,35 +310,63 @@ write to commit. However: ## 5. Implementation Plan -### Phase 1: engine surface -1. Add `LeaseDuration() time.Duration` to `internal/raftengine/engine.go` - `LeaderView` interface and to the etcd implementation. -2. Add `AppliedIndex() uint64` if not already exposed by a public method. - -### Phase 2: coordinator lease -1. Add `leaseState` field to `Coordinate` and to each shard inside - `ShardedCoordinator`. -2. Implement `LeaseRead(ctx)` and `LeaseReadForKey(ctx, key)`. -3. Wrap `Propose` calls in coordinator to refresh lease on success. -4. Wrap leadership-change detection (`refreshStatus` callback) to invalidate. - -### Phase 3: callers -1. Replace `LinearizableRead` with `LeaseRead` in - `adapter/redis_lua_context.go:newLuaScriptContext`. -2. Add `LeaseRead` at the entry of: - - `adapter/redis.go` `get`, `keyTypeAt`, `keys`, `exists`-family. - - `adapter/dynamodb.go` `getItem`, `query`, `scan`, - `transactGetItems`, `batchGetItem`. -3. No change to write paths beyond the implicit refresh via `Propose`. - -### Phase 4: tests -1. Unit test for `leaseState`: extend, expire, invalidate. -2. Integration test in `kv/`: `LeaseRead` returns immediately when lease - valid, falls back to `LinearizableRead` when expired, returns error and - invalidates when underlying `LinearizableRead` errs. -3. End-to-end test in `adapter/`: Lua script under sustained load issues - N scripts but only K underlying ReadIndex calls (K << N). -4. Jepsen workload addition: a partition test that asserts no stale-read +### Phase 1: engine surface — DONE +1. Added `LeaseProvider` as a separate optional interface in + `internal/raftengine/engine.go` (not on `LeaderView`) so non-etcd + engines and test stubs can omit lease methods. Etcd engine implements + `LeaseDuration() time.Duration` and `AppliedIndex() uint64`. +2. `RegisterLeaderLossCallback(fn func())` was added to `LeaseProvider` + in the follow-up review pass; the etcd engine fires registered + callbacks from `refreshStatus` whenever the local node leaves the + leader role. + +### Phase 2: coordinator lease — DONE +1. `leaseState` (lock-free `atomic.Pointer[time.Time]` with monotonic + CAS extend) added to `Coordinate`; `ShardGroup` gets a per-shard + `leaseState`. +2. `Coordinate.LeaseRead` / `Coordinate.LeaseReadForKey` and + `ShardedCoordinator.LeaseRead` / `ShardedCoordinator.LeaseReadForKey` + implemented. Time is sampled BEFORE the underlying + `LinearizableRead` so the lease window starts at quorum + confirmation. +3. `Coordinate.Dispatch` refreshes the lease on successful commit using + the pre-dispatch timestamp. `ShardedCoordinator` wraps each + `g.Txn` in `leaseRefreshingTxn` so all dispatch paths (raw via + `router.Commit`, `dispatchSingleShardTxn`, `dispatchTxn` 2PC, and + `dispatchDelPrefixBroadcast`) refresh the per-shard lease on + `Commit` / `Abort` success. +4. `NewCoordinatorWithEngine` and `NewShardedCoordinator` register + `lease.invalidate` via the `LeaseProvider.RegisterLeaderLossCallback` + hook, so the engine's `refreshStatus` invalidates the lease the + instant it observes a non-leader transition. + +### Phase 3: callers — PARTIAL +1. DONE: `adapter/redis_lua_context.go:newLuaScriptContext` uses + `LeaseRead` instead of `LinearizableRead`. +2. DONE for the highest-traffic single-key handlers; deferred for the + rest: + - DONE: `adapter/redis.go` `get` (with bounded + `redisDispatchTimeout` context). + - DONE: `adapter/dynamodb.go` `getItem`. + - TODO: `adapter/redis.go` `keys`, `exists`-family, ZSet/Hash/List/Set + readers; `adapter/dynamodb.go` `query`, `scan`, `transactGetItems`, + `batchGetItem`. These currently rely on the lease being kept warm by + Lua scripts and successful Dispatch calls. To be wrapped in a + follow-up. +3. No change to write paths beyond the implicit refresh via the + `Coordinate.Dispatch` / `leaseRefreshingTxn` hooks. + +### Phase 4: tests — PARTIAL +1. DONE: `kv/lease_state_test.go` covers `leaseState` extend, expire, + invalidate, monotonic CAS, invalidate-vs-extend race. +2. DONE: `kv/lease_read_test.go` covers `Coordinate.LeaseRead` fast / + slow / error / fallback paths and the leader-loss callback wiring. + `kv/sharded_lease_test.go` covers `ShardedCoordinator` per-shard + isolation and per-shard leader-loss wiring. +3. TODO: end-to-end test in `adapter/` showing Lua script under + sustained load issues N scripts but only K underlying ReadIndex + calls (K << N). +4. TODO: Jepsen partition workload asserting no stale-read linearizability violation outside the lease window. ### Phase 5: rollout diff --git a/kv/lease_read_test.go b/kv/lease_read_test.go index a11022fd..f4338f78 100644 --- a/kv/lease_read_test.go +++ b/kv/lease_read_test.go @@ -24,7 +24,7 @@ type fakeLeaseEngine struct { registerLeaderLossCalled atomic.Int32 } -func (e *fakeLeaseEngine) State() raftengine.State { return raftengine.StateLeader } +func (e *fakeLeaseEngine) State() raftengine.State { return raftengine.StateLeader } func (e *fakeLeaseEngine) Leader() raftengine.LeaderInfo { return raftengine.LeaderInfo{ID: "n1", Address: "127.0.0.1:0"} } @@ -45,9 +45,9 @@ func (e *fakeLeaseEngine) Configuration(context.Context) (raftengine.Configurati func (e *fakeLeaseEngine) Propose(context.Context, []byte) (*raftengine.ProposalResult, error) { return &raftengine.ProposalResult{}, nil } -func (e *fakeLeaseEngine) Close() error { return nil } -func (e *fakeLeaseEngine) LeaseDuration() time.Duration { return e.leaseDur } -func (e *fakeLeaseEngine) AppliedIndex() uint64 { return e.applied } +func (e *fakeLeaseEngine) Close() error { return nil } +func (e *fakeLeaseEngine) LeaseDuration() time.Duration { return e.leaseDur } +func (e *fakeLeaseEngine) AppliedIndex() uint64 { return e.applied } func (e *fakeLeaseEngine) RegisterLeaderLossCallback(fn func()) { e.registerLeaderLossCalled.Add(1) for !e.leaderLossCallbacksMu.CompareAndSwap(false, true) { @@ -73,7 +73,7 @@ type nonLeaseEngine struct { linearizableErr error } -func (e *nonLeaseEngine) State() raftengine.State { return raftengine.StateLeader } +func (e *nonLeaseEngine) State() raftengine.State { return raftengine.StateLeader } func (e *nonLeaseEngine) Leader() raftengine.LeaderInfo { return raftengine.LeaderInfo{ID: "n1", Address: "127.0.0.1:0"} } @@ -181,4 +181,3 @@ func TestCoordinate_RegistersLeaderLossCallback(t *testing.T) { require.False(t, c.lease.valid(time.Now()), "leader-loss callback must invalidate the lease") } - diff --git a/kv/sharded_lease_test.go b/kv/sharded_lease_test.go new file mode 100644 index 00000000..de66a130 --- /dev/null +++ b/kv/sharded_lease_test.go @@ -0,0 +1,121 @@ +package kv + +import ( + "context" + "errors" + "testing" + "time" + + "github.com/bootjp/elastickv/distribution" + "github.com/stretchr/testify/require" +) + +// shardedLeaseEngine is a minimal raftengine.Engine + LeaseProvider used +// by sharded lease tests. It records LinearizableRead invocations and +// the registered leader-loss callback so tests can fire it on demand. +type shardedLeaseEngine struct { + *fakeLeaseEngine +} + +func newShardedLeaseEngine(applied uint64) *shardedLeaseEngine { + return &shardedLeaseEngine{ + fakeLeaseEngine: &fakeLeaseEngine{ + applied: applied, + leaseDur: time.Hour, + }, + } +} + +func mustShardedLeaseCoord(t *testing.T, eng1, eng2 *shardedLeaseEngine) *ShardedCoordinator { + t.Helper() + distEngine := distribution.NewEngine() + // Route a..m -> group 1, m..end -> group 2 so per-key tests can pick + // a key landing on each shard. + distEngine.UpdateRoute([]byte("a"), []byte("m"), 1) + distEngine.UpdateRoute([]byte("m"), nil, 2) + + g1Txn := &recordingTransactional{ + responses: []*TransactionResponse{{CommitIndex: 5}}, + } + g2Txn := &recordingTransactional{ + responses: []*TransactionResponse{{CommitIndex: 12}}, + } + return NewShardedCoordinator(distEngine, map[uint64]*ShardGroup{ + 1: {Engine: eng1, Txn: g1Txn}, + 2: {Engine: eng2, Txn: g2Txn}, + }, 1, NewHLC(), nil) +} + +func TestShardedCoordinator_LeaseReadForKey_PerShardIsolation(t *testing.T) { + t.Parallel() + + eng1 := newShardedLeaseEngine(100) + eng2 := newShardedLeaseEngine(200) + coord := mustShardedLeaseCoord(t, eng1, eng2) + + // Pre-extend shard 1's lease only. + g1 := coord.groups[1] + g1.lease.extend(time.Now().Add(time.Hour)) + + idx, err := coord.LeaseReadForKey(context.Background(), []byte("apple")) + require.NoError(t, err) + require.Equal(t, uint64(100), idx) + require.Equal(t, int32(0), eng1.linearizableCalls.Load(), + "shard 1 lease is valid; engine 1 should not be called") + + idx, err = coord.LeaseReadForKey(context.Background(), []byte("zebra")) + require.NoError(t, err) + require.Equal(t, uint64(200), idx) + require.Equal(t, int32(1), eng2.linearizableCalls.Load(), + "shard 2 lease was never extended; engine 2 must take the slow path") + + // After the slow path, shard 2's lease is now valid; engine 1 must + // remain untouched. + require.Equal(t, int32(0), eng1.linearizableCalls.Load()) +} + +func TestShardedCoordinator_LeaseReadForKey_ErrorOnlyInvalidatesShard(t *testing.T) { + t.Parallel() + + sentinel := errors.New("read-index failed") + eng1 := newShardedLeaseEngine(100) + eng2 := newShardedLeaseEngine(200) + eng2.linearizableErr = sentinel + coord := mustShardedLeaseCoord(t, eng1, eng2) + + g1 := coord.groups[1] + g2 := coord.groups[2] + g1.lease.extend(time.Now().Add(time.Hour)) + g2.lease.extend(time.Now().Add(time.Hour)) + g2.lease.invalidate() // force shard 2 onto slow path + + _, err := coord.LeaseReadForKey(context.Background(), []byte("zebra")) + require.ErrorIs(t, err, sentinel) + require.False(t, g2.lease.valid(time.Now()), + "shard 2 lease must be invalidated after error") + require.True(t, g1.lease.valid(time.Now()), + "shard 1 lease must NOT be touched by shard 2's failure") +} + +func TestShardedCoordinator_RegistersPerShardLeaderLossCallback(t *testing.T) { + t.Parallel() + + eng1 := newShardedLeaseEngine(100) + eng2 := newShardedLeaseEngine(200) + coord := mustShardedLeaseCoord(t, eng1, eng2) + + require.Equal(t, int32(1), eng1.registerLeaderLossCalled.Load(), + "NewShardedCoordinator must register a callback per shard engine") + require.Equal(t, int32(1), eng2.registerLeaderLossCalled.Load()) + + g1 := coord.groups[1] + g2 := coord.groups[2] + g1.lease.extend(time.Now().Add(time.Hour)) + g2.lease.extend(time.Now().Add(time.Hour)) + + eng1.fireLeaderLoss() + require.False(t, g1.lease.valid(time.Now()), + "shard 1 leader-loss callback must invalidate shard 1's lease") + require.True(t, g2.lease.valid(time.Now()), + "shard 2 lease must remain valid; only its own engine's callback affects it") +} From 6d305635edfa203207af75090109dda1cda7aadb Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 06:46:05 +0900 Subject: [PATCH 13/43] test(lease-read): amortization end-to-end + design doc closure - TestCoordinate_LeaseRead_AmortizesLinearizableRead asserts that 100 LeaseRead calls within one lease window trigger exactly one underlying LinearizableRead. This is the Phase-4 design item proving the lease actually amortizes the cost. - docs/lease_read_design.md: Phase 3 deferred items moved under #557 (wrap remaining adapter read handlers); Phase 4 amortization marked DONE; Jepsen partition test marked DEFERRED with rationale. --- docs/lease_read_design.md | 28 ++++++++++++++++------------ kv/lease_read_test.go | 22 ++++++++++++++++++++++ 2 files changed, 38 insertions(+), 12 deletions(-) diff --git a/docs/lease_read_design.md b/docs/lease_read_design.md index 306acc7c..005c773b 100644 --- a/docs/lease_read_design.md +++ b/docs/lease_read_design.md @@ -343,16 +343,15 @@ write to commit. However: ### Phase 3: callers — PARTIAL 1. DONE: `adapter/redis_lua_context.go:newLuaScriptContext` uses `LeaseRead` instead of `LinearizableRead`. -2. DONE for the highest-traffic single-key handlers; deferred for the - rest: +2. DONE for the highest-traffic single-key handlers; rest tracked as #557: - DONE: `adapter/redis.go` `get` (with bounded `redisDispatchTimeout` context). - DONE: `adapter/dynamodb.go` `getItem`. - - TODO: `adapter/redis.go` `keys`, `exists`-family, ZSet/Hash/List/Set - readers; `adapter/dynamodb.go` `query`, `scan`, `transactGetItems`, - `batchGetItem`. These currently rely on the lease being kept warm by - Lua scripts and successful Dispatch calls. To be wrapped in a - follow-up. + - DEFERRED (#557): `adapter/redis.go` `keys`, `exists`-family, + ZSet/Hash/List/Set readers; `adapter/dynamodb.go` `query`, `scan`, + `transactGetItems`, `batchGetItem`. Rely on the lease being kept + warm by Lua scripts and successful Dispatch calls; safety identical + to pre-PR (no quorum check). 3. No change to write paths beyond the implicit refresh via the `Coordinate.Dispatch` / `leaseRefreshingTxn` hooks. @@ -363,11 +362,16 @@ write to commit. However: slow / error / fallback paths and the leader-loss callback wiring. `kv/sharded_lease_test.go` covers `ShardedCoordinator` per-shard isolation and per-shard leader-loss wiring. -3. TODO: end-to-end test in `adapter/` showing Lua script under - sustained load issues N scripts but only K underlying ReadIndex - calls (K << N). -4. TODO: Jepsen partition workload asserting no stale-read - linearizability violation outside the lease window. +3. DONE: `TestCoordinate_LeaseRead_AmortizesLinearizableRead` proves + 100 LeaseRead calls inside one lease window trigger exactly 1 + underlying LinearizableRead. Stronger end-to-end Lua amortization + under the adapter is implicit — `newLuaScriptContext` is the single + call site and is exercised by every Lua test. +4. DEFERRED: Jepsen partition workload asserting no stale-read + linearizability violation outside the lease window. Substantial + scope; tracked separately. Existing Jepsen `redis-workload` already + exercises the lease path under partition + kill faults, just without + a lease-specific assertion. ### Phase 5: rollout - Land Phases 1-3 behind no flag. The semantics are strictly equivalent or diff --git a/kv/lease_read_test.go b/kv/lease_read_test.go index f4338f78..1edecca8 100644 --- a/kv/lease_read_test.go +++ b/kv/lease_read_test.go @@ -181,3 +181,25 @@ func TestCoordinate_RegistersLeaderLossCallback(t *testing.T) { require.False(t, c.lease.valid(time.Now()), "leader-loss callback must invalidate the lease") } + +// --- Amortization end-to-end --------------------------------------------- + +// TestCoordinate_LeaseRead_AmortizesLinearizableRead is the Phase-4 design +// item proving the lease actually amortizes the cost: N calls within a +// single lease window must trigger only the first slow-path +// LinearizableRead and N-1 fast-path returns. +func TestCoordinate_LeaseRead_AmortizesLinearizableRead(t *testing.T) { + t.Parallel() + const N = 100 + eng := &fakeLeaseEngine{applied: 9, leaseDur: time.Hour} + c := NewCoordinatorWithEngine(nil, eng) + + for i := 0; i < N; i++ { + idx, err := c.LeaseRead(context.Background()) + require.NoError(t, err) + require.Equal(t, uint64(9), idx) + } + + require.Equal(t, int32(1), eng.linearizableCalls.Load(), + "100 LeaseRead calls inside the lease window should trigger exactly 1 LinearizableRead") +} From 9c15a89b6c1da9fe46fd47842d3995b164814124 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 06:51:51 +0900 Subject: [PATCH 14/43] fix(lease-read): plug invalidate/extend race + fire callbacks on shutdown Two correctness fixes from Copilot's second review pass on PR #549. 1. leaseState invalidate-vs-extend race (HIGH): a Dispatch that succeeded just before a leader-loss invalidate could resurrect the lease for up to LeaseDuration. Without a guard, extend's CAS sees nil (set by invalidate) and successfully stores a future expiry. Adds a generation counter that invalidate bumps. extend captures the generation at start and, after its CAS lands, undoes the write (via a CAS with its own pointer) iff the generation has changed. A fresh extend that captured the post-invalidate generation is left intact because it stored a different pointer. Regression test: TestLeaseState_ExtendCannotResurrectAfterInvalidate replays the race deterministically by manually invoking the same CAS dance with a stale generation token. 2. Engine shutdown skips leader-loss callbacks (HIGH): refreshStatus only fires callbacks on a leader -> non-leader edge. Close() / fail() / shutdown() short-circuit the status loop, so a leader that closes while still in StateLeader can leave its coordinator's lease alive until wall-clock expiry (up to LeaseDuration). shutdown() now records the pre-close state under e.mu and, if it was leader, fires fireLeaderLossCallbacks() after teardown. This matches the doc string on RegisterLeaderLossCallback which already promised "graceful transfer, partition step-down, or shutdown". --- internal/raftengine/etcd/engine.go | 10 ++++++++ kv/lease_state.go | 38 +++++++++++++++++++-------- kv/lease_state_test.go | 41 ++++++++++++++++++++++++++++++ 3 files changed, 79 insertions(+), 10 deletions(-) diff --git a/internal/raftengine/etcd/engine.go b/internal/raftengine/etcd/engine.go index e4f6dcae..354401ff 100644 --- a/internal/raftengine/etcd/engine.go +++ b/internal/raftengine/etcd/engine.go @@ -1715,6 +1715,7 @@ func (e *Engine) requestShutdown() { func (e *Engine) shutdown() { e.mu.Lock() + wasLeader := e.status.State == raftengine.StateLeader e.closed = true e.status.State = raftengine.StateShutdown e.mu.Unlock() @@ -1723,6 +1724,15 @@ func (e *Engine) shutdown() { _ = closePersist(e.persist) _ = e.transport.Close() e.failPending(errors.WithStack(errClosed)) + // LeaseProvider contract promises callbacks fire on shutdown too. + // refreshStatus only fires them on the leader -> non-leader edge, + // which can be missed when shutdown short-circuits the status loop. + // Always fire here so lease holders invalidate even on engine close + // initiated while still leader, on shutdown after fail(), or via + // Close() racing against the run loop. + if wasLeader { + e.fireLeaderLossCallbacks() + } } func (e *Engine) fail(err error) { diff --git a/kv/lease_state.go b/kv/lease_state.go index 404a5a2d..17ff8887 100644 --- a/kv/lease_state.go +++ b/kv/lease_state.go @@ -6,13 +6,16 @@ import ( ) // leaseState tracks the wall-clock expiry of a leader-local read lease. -// All operations are lock-free via atomic.Pointer. +// All operations are lock-free via atomic.Pointer plus a generation +// counter that prevents an in-flight extend from resurrecting a lease +// that a concurrent invalidate has cleared. // -// A nil pointer means the lease has never been issued or has been -// invalidated. A non-nil pointer is the wall-clock instant after which +// A nil expiry means the lease has never been issued or has been +// invalidated. A non-nil expiry is the wall-clock instant after which // the lease is considered expired; a caller comparing time.Now() against // the loaded value can decide whether to skip a quorum confirmation. type leaseState struct { + gen atomic.Uint64 expiry atomic.Pointer[time.Time] } @@ -28,30 +31,45 @@ func (s *leaseState) valid(now time.Time) bool { return now.Before(*exp) } -// extend sets the lease expiry to until iff until is strictly after the -// currently stored expiry (or no expiry is stored). The CAS loop prevents -// an out-of-order writer that sampled time.Now() earlier from overwriting -// a fresher extension and prematurely shortening the lease, while still -// allowing invalidate() (which Stores nil) to win unconditionally. +// extend sets the lease expiry to until iff (a) until is strictly after +// the currently stored expiry (or no expiry is stored) and (b) no +// invalidate happened between when this caller started and when its +// CAS landed. Without the generation guard, a Dispatch that returned +// successfully *just before* a leader-loss invalidate could resurrect +// the lease milliseconds after invalidation, defeating the purpose of +// the leader-loss callback. func (s *leaseState) extend(until time.Time) { if s == nil { return } + expectedGen := s.gen.Load() for { current := s.expiry.Load() if current != nil && !until.After(*current) { return } - if s.expiry.CompareAndSwap(current, &until) { - return + if !s.expiry.CompareAndSwap(current, &until) { + continue + } + // CAS landed. If invalidate raced ahead, undo our write iff + // no later writer has replaced it. Using CAS with our own + // pointer means a fresh extend that captured the + // post-invalidate generation is left intact. + if s.gen.Load() != expectedGen { + s.expiry.CompareAndSwap(&until, nil) } + return } } // invalidate clears the lease so the next read takes the slow path. +// Bumping the generation first ensures any concurrent extend that +// captured the previous generation will undo its own CAS rather than +// resurrect the lease. func (s *leaseState) invalidate() { if s == nil { return } + s.gen.Add(1) s.expiry.Store(nil) } diff --git a/kv/lease_state_test.go b/kv/lease_state_test.go index a224fd0b..b155ccdc 100644 --- a/kv/lease_state_test.go +++ b/kv/lease_state_test.go @@ -78,6 +78,47 @@ func TestLeaseState_InvalidateBeatsConcurrentExtend(t *testing.T) { require.False(t, s.valid(now)) } +// TestLeaseState_ExtendCannotResurrectAfterInvalidate exercises the +// generation-guard invariant: an extend call whose internal CAS lands +// after a concurrent invalidate must undo its own write rather than +// leave the lease alive. Without the guard, a Dispatch that succeeded +// just before the leader-loss callback could resurrect the lease for +// up to LeaseDuration. +func TestLeaseState_ExtendCannotResurrectAfterInvalidate(t *testing.T) { + t.Parallel() + var s leaseState + now := time.Now() + + // Simulate the race: extend captured the current generation, then + // invalidate fires before extend's CAS lands. The simplest way to + // exercise this deterministically is to invalidate first (so the + // extend sees the bumped generation) and verify extend leaves the + // state invalidated. + expectedGen := s.gen.Load() + s.invalidate() + require.NotEqual(t, expectedGen, s.gen.Load(), + "invalidate must bump the generation") + + // Manually replay an extend that captured the pre-invalidate + // generation. Use the package-internal helper to avoid time-based + // races. + until := now.Add(time.Hour) + for { + current := s.expiry.Load() + if !s.expiry.CompareAndSwap(current, &until) { + continue + } + // Match the production extend's post-CAS check. + if s.gen.Load() != expectedGen { + s.expiry.CompareAndSwap(&until, nil) + } + break + } + + require.False(t, s.valid(now), + "stale-generation extend must NOT resurrect the lease") +} + func TestLeaseState_ConcurrentExtendAndRead(t *testing.T) { t.Parallel() var s leaseState From 22448dbf80550bfc8f749a676952aaa2507b7767 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 15:29:24 +0900 Subject: [PATCH 15/43] fix(lease-read): Copilot 2nd-pass nits on PR #549 Four minor tightenings after the latest Copilot review: - kv/coordinator.go (LeaseRead): capture time.Now() exactly once at the top of the function so the fast-path validity check and the slow-path lease-extend base share the same instant. The previous second-sample let the fast path accept a read whose wall-clock instant was slightly after the intended expiry boundary. - kv/sharded_coordinator.go (groupLeaseRead): same single-sample pattern applied. - adapter/dynamodb.go getItem: switch LeaseRead -> LeaseReadForKey using dynamoTableMetaKey(tableName). In sharded deployments this consults the shard that actually owns the table's metadata rather than pinning every GetItem to the default group. - docs/lease_read_design.md: update the leaseState snippet to include the generation counter (introduced when the invalidate/extend race was plugged) so doc matches implementation. --- adapter/dynamodb.go | 8 +++++++- docs/lease_read_design.md | 16 ++++++++++++---- kv/coordinator.go | 18 ++++++++++++------ kv/sharded_coordinator.go | 10 +++++----- 4 files changed, 36 insertions(+), 16 deletions(-) diff --git a/adapter/dynamodb.go b/adapter/dynamodb.go index 9bfdcd0b..bb44d9a5 100644 --- a/adapter/dynamodb.go +++ b/adapter/dynamodb.go @@ -1367,7 +1367,13 @@ func (d *DynamoDBServer) getItem(w http.ResponseWriter, r *http.Request) { if !ok { return } - if _, err := d.coordinator.LeaseRead(r.Context()); err != nil { + // Use LeaseReadForKey with the table-meta key so sharded + // deployments consult the shard that actually owns the table's + // metadata rather than falling back to the default group. The + // item-specific key is not known until after schema resolution; + // the table-meta key is a deterministic proxy for the shard that + // hosts this table. + if _, err := d.coordinator.LeaseReadForKey(r.Context(), dynamoTableMetaKey(in.TableName)); err != nil { writeDynamoError(w, http.StatusInternalServerError, dynamoErrInternal, err.Error()) return } diff --git a/docs/lease_read_design.md b/docs/lease_read_design.md index 005c773b..103a8e2c 100644 --- a/docs/lease_read_design.md +++ b/docs/lease_read_design.md @@ -91,7 +91,8 @@ holds: ```go type leaseState struct { - expiry atomic.Pointer[time.Time] // nil or zero-value = expired + gen atomic.Uint64 // bumped by invalidate() + expiry atomic.Pointer[time.Time] // nil = expired / invalidated } ``` @@ -100,9 +101,16 @@ type leaseState struct { success. - `time.Now() < *expiry`: lease is valid. `LeaseRead` returns immediately without contacting the Raft layer. - -The pointer-swap form lets readers be lock-free: each `LeaseRead` does one -atomic load + one wall-clock compare. +- `invalidate()` increments `gen` before clearing `expiry`. `extend()` + captures `gen` at entry and, after its CAS lands, undoes its own + write (via CAS on the pointer it stored) iff `gen` has moved. This + prevents a Dispatch that succeeded just before a leader-loss + invalidate from resurrecting the lease milliseconds after it was + cleared. A fresh `extend()` that captured the post-invalidate + generation is left intact because it stored a different pointer. + +The lock-free form lets readers do one atomic load + one wall-clock compare +on the fast path. ### 3.2 Lease duration diff --git a/kv/coordinator.go b/kv/coordinator.go index cf7cde30..47b662ee 100644 --- a/kv/coordinator.go +++ b/kv/coordinator.go @@ -254,19 +254,25 @@ func (c *Coordinate) LeaseRead(ctx context.Context) (uint64, error) { if !ok { return c.LinearizableRead(ctx) } - if c.lease.valid(time.Now()) { + // Capture time.Now() exactly once so the fast-path validity check + // and the slow-path lease-extend base share the same instant. A + // second sampling would let the fast path accept a read whose + // instant is slightly after expiry (and, conversely, shorten the + // slow-path lease window by the same delta). + now := time.Now() + if c.lease.valid(now) { return lp.AppliedIndex(), nil } - // Sample BEFORE LinearizableRead so the lease window starts at the - // real quorum confirmation instant, not after the heartbeat round - // returned. See Coordinate.Dispatch for the same rationale. - readStart := time.Now() + // The captured `now` also serves as readStart: it is strictly + // before LinearizableRead returns, so the lease window starts at + // the real quorum confirmation instant, not after the heartbeat + // round returned. See Coordinate.Dispatch for the same rationale. idx, err := c.LinearizableRead(ctx) if err != nil { c.lease.invalidate() return 0, err } - c.lease.extend(readStart.Add(lp.LeaseDuration())) + c.lease.extend(now.Add(lp.LeaseDuration())) return idx, nil } diff --git a/kv/sharded_coordinator.go b/kv/sharded_coordinator.go index 9d238fa4..6bf59f8a 100644 --- a/kv/sharded_coordinator.go +++ b/kv/sharded_coordinator.go @@ -672,18 +672,18 @@ func groupLeaseRead(ctx context.Context, g *ShardGroup) (uint64, error) { if !ok { return linearizableReadEngineCtx(ctx, engine) } - if g.lease.valid(time.Now()) { + // Single time.Now() sample shared by both the fast-path validity + // check and the slow-path extend base, mirroring Coordinate.LeaseRead. + now := time.Now() + if g.lease.valid(now) { return lp.AppliedIndex(), nil } - // Sample BEFORE LinearizableRead; the lease window must start at the - // quorum confirmation instant, not after the read returned. - readStart := time.Now() idx, err := linearizableReadEngineCtx(ctx, engine) if err != nil { g.lease.invalidate() return 0, err } - g.lease.extend(readStart.Add(lp.LeaseDuration())) + g.lease.extend(now.Add(lp.LeaseDuration())) return idx, nil } From 33eef30de8cec819e4eef6594c18b8e5fff707d5 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 15:54:55 +0900 Subject: [PATCH 16/43] docs(lease-read): clarify RegisterLeaderLossCallback does NOT fire on LeadTransferee Copilot flagged: the docstring mentioned "LeadTransferee != 0 hand-off" but callbacks only fire on the actual state transition out of leader and on shutdown -- not when a transfer starts. Reword so readers don't assume early firing. This is intentional behaviour: CheckQuorum guarantees the lease window (LeaseDuration) expires before a successor leader could realistically be elected, so waiting for the state flip is safe. Firing at LeadTransferee != 0 would be a purely-conservative optional hardening; tracked separately if ever needed. --- internal/raftengine/etcd/engine.go | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/internal/raftengine/etcd/engine.go b/internal/raftengine/etcd/engine.go index 354401ff..5d95800d 100644 --- a/internal/raftengine/etcd/engine.go +++ b/internal/raftengine/etcd/engine.go @@ -573,10 +573,14 @@ func (e *Engine) AppliedIndex() uint64 { return e.status.AppliedIndex } -// RegisterLeaderLossCallback registers fn to fire on every transition -// out of the leader role (including LeadTransferee != 0 hand-off, -// CheckQuorum step-down, and shutdown). Used by lease-read callers to -// invalidate cached lease state so the next read takes the slow path. +// RegisterLeaderLossCallback registers fn to fire every time the local +// node's Raft state transitions out of leader (CheckQuorum step-down, +// graceful transfer completion, partition-induced demotion) and also +// on shutdown() while the node was still leader. Callbacks are NOT +// fired at the moment a transfer starts (LeadTransferee != 0); they +// only fire once the transfer completes and state flips to follower. +// Lease-read callers use this to invalidate cached lease state so the +// next read takes the slow path. func (e *Engine) RegisterLeaderLossCallback(fn func()) { if e == nil || fn == nil { return From 8d3511f3b824f974d40e379745726b2058b5f29e Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 16:07:26 +0900 Subject: [PATCH 17/43] fix(lease-read): panic-protect leader-loss callbacks + sync.Mutex in test stub Two Copilot 3rd-pass findings on PR #549: 1. fireLeaderLossCallbacks now invokes each callback under an invokeLeaderLossCallback helper that defer-recovers. A panic in any lease holder's callback no longer propagates into the raft engine's status refresh or shutdown path. Remaining callbacks still fire. New TestFireLeaderLossCallbacks_ContainsPanic verifies both before- and after-panic callbacks still run, and the method returns normally. TestFireLeaderLossCallbacks_NoCallbacksIsSafe covers the empty-list path. 2. fakeLeaseEngine (test stub in kv/lease_read_test.go) previously used an atomic.Bool spin-lock around the callback slice. Switched to sync.Mutex so the stub can't deadlock under contention and matches the mutex style used everywhere else in the test suite. --- internal/raftengine/etcd/engine.go | 21 ++++++++-- .../etcd/leader_loss_callback_test.go | 38 +++++++++++++++++++ kv/lease_read_test.go | 13 +++---- 3 files changed, 62 insertions(+), 10 deletions(-) create mode 100644 internal/raftengine/etcd/leader_loss_callback_test.go diff --git a/internal/raftengine/etcd/engine.go b/internal/raftengine/etcd/engine.go index 5d95800d..04f17ac5 100644 --- a/internal/raftengine/etcd/engine.go +++ b/internal/raftengine/etcd/engine.go @@ -591,18 +591,33 @@ func (e *Engine) RegisterLeaderLossCallback(fn func()) { } // fireLeaderLossCallbacks invokes all registered callbacks. Safe to call -// from refreshStatus while holding no engine locks; callbacks run -// synchronously and must not block. +// from refreshStatus / shutdown while holding no engine locks; +// callbacks run synchronously and must not block. A panicking callback +// is contained so it cannot take down the raft engine loop or the +// shutdown path; the remaining callbacks still fire. func (e *Engine) fireLeaderLossCallbacks() { e.leaderLossCbsMu.Lock() cbs := make([]func(), len(e.leaderLossCbs)) copy(cbs, e.leaderLossCbs) e.leaderLossCbsMu.Unlock() for _, fn := range cbs { - fn() + e.invokeLeaderLossCallback(fn) } } +func (e *Engine) invokeLeaderLossCallback(fn func()) { + defer func() { + if r := recover(); r != nil { + // A buggy lease holder must not crash the node. Drop the + // panic here; it has already marked the lease invalid (or + // would have, if the panic didn't interrupt it) so the + // next read takes the slow path and re-verifies leadership. + _ = r + } + }() + fn() +} + func (e *Engine) submitRead(ctx context.Context, waitApplied bool) (uint64, error) { if err := contextErr(ctx); err != nil { return 0, err diff --git a/internal/raftengine/etcd/leader_loss_callback_test.go b/internal/raftengine/etcd/leader_loss_callback_test.go new file mode 100644 index 00000000..fda01cb3 --- /dev/null +++ b/internal/raftengine/etcd/leader_loss_callback_test.go @@ -0,0 +1,38 @@ +package etcd + +import ( + "sync/atomic" + "testing" + + "github.com/stretchr/testify/require" +) + +// TestFireLeaderLossCallbacks_ContainsPanic verifies that a panicking +// callback does NOT take down the raft engine loop: the remaining +// callbacks still fire and the method returns normally. +func TestFireLeaderLossCallbacks_ContainsPanic(t *testing.T) { + t.Parallel() + + e := &Engine{} + var before, after atomic.Int32 + e.RegisterLeaderLossCallback(func() { before.Add(1) }) + e.RegisterLeaderLossCallback(func() { panic("lease holder bug") }) + e.RegisterLeaderLossCallback(func() { after.Add(1) }) + + // Must not panic out of the call. + require.NotPanics(t, e.fireLeaderLossCallbacks) + + require.Equal(t, int32(1), before.Load(), + "callbacks registered before the panicking one must have fired") + require.Equal(t, int32(1), after.Load(), + "callbacks registered after the panicking one must still fire") +} + +// TestFireLeaderLossCallbacks_NoCallbacksIsSafe exercises the empty-list +// fast path so the helper can be called unconditionally from shutdown +// and refreshStatus paths without a guard. +func TestFireLeaderLossCallbacks_NoCallbacksIsSafe(t *testing.T) { + t.Parallel() + e := &Engine{} + require.NotPanics(t, e.fireLeaderLossCallbacks) +} diff --git a/kv/lease_read_test.go b/kv/lease_read_test.go index 1edecca8..a2874f26 100644 --- a/kv/lease_read_test.go +++ b/kv/lease_read_test.go @@ -3,6 +3,7 @@ package kv import ( "context" "errors" + "sync" "sync/atomic" "testing" "time" @@ -19,8 +20,8 @@ type fakeLeaseEngine struct { leaseDur time.Duration linearizableErr error linearizableCalls atomic.Int32 + leaderLossCallbacksMu sync.Mutex leaderLossCallbacks []func() - leaderLossCallbacksMu atomic.Bool registerLeaderLossCalled atomic.Int32 } @@ -50,17 +51,15 @@ func (e *fakeLeaseEngine) LeaseDuration() time.Duration { return e.leaseDur } func (e *fakeLeaseEngine) AppliedIndex() uint64 { return e.applied } func (e *fakeLeaseEngine) RegisterLeaderLossCallback(fn func()) { e.registerLeaderLossCalled.Add(1) - for !e.leaderLossCallbacksMu.CompareAndSwap(false, true) { - } + e.leaderLossCallbacksMu.Lock() e.leaderLossCallbacks = append(e.leaderLossCallbacks, fn) - e.leaderLossCallbacksMu.Store(false) + e.leaderLossCallbacksMu.Unlock() } func (e *fakeLeaseEngine) fireLeaderLoss() { - for !e.leaderLossCallbacksMu.CompareAndSwap(false, true) { - } + e.leaderLossCallbacksMu.Lock() cbs := append([]func(){}, e.leaderLossCallbacks...) - e.leaderLossCallbacksMu.Store(false) + e.leaderLossCallbacksMu.Unlock() for _, cb := range cbs { cb() } From 76a93d77d8fb421842f65f913517da581a246102 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 16:24:19 +0900 Subject: [PATCH 18/43] fix(lease-read): only refresh lease when Raft commit actually happened gemini-code-assist flagged HIGH on sharded_coordinator.go:58: refreshing the per-shard lease on every successful Abort assumes Abort always goes through Raft. It does not: TransactionManager.Abort builds abortReqs via abortRequestFor(req), which returns nil for Phase_NONE / non-txn / unparseable inputs. If every req maps to nil, Abort returns success with CommitIndex=0 -- no proposal, no quorum confirmation. Same shape applies to TransactionManager.Commit's empty-input early return. Refreshing the lease in those cases would be unsound. Gate the refresh on resp.CommitIndex > 0, which the underlying manager only sets to a non-zero value when at least one proposal was applied. - leaseRefreshingTxn.Commit/Abort now share maybeRefresh which checks resp.CommitIndex > 0. - Coordinate.Dispatch's post-dispatch hook extracted into refreshLeaseAfterDispatch for clarity (also brings the cyclop complexity of Dispatch back under the limit). TestShardedCoordinator_LeaseRefreshingTxn_SkipsWhenCommitIndexZero covers both directions: no-commit -> no refresh; real-commit -> refresh. --- kv/coordinator.go | 25 ++++++++++++------ kv/sharded_coordinator.go | 36 ++++++++++++++++++-------- kv/sharded_lease_test.go | 53 +++++++++++++++++++++++++++++++++++++++ 3 files changed, 96 insertions(+), 18 deletions(-) diff --git a/kv/coordinator.go b/kv/coordinator.go index 47b662ee..0da56f0c 100644 --- a/kv/coordinator.go +++ b/kv/coordinator.go @@ -144,17 +144,26 @@ func (c *Coordinate) Dispatch(ctx context.Context, reqs *OperationGroup[OP]) (*C } else { resp, err = c.dispatchRaw(reqs.Elems) } - if err == nil { - // A successful dispatch implies majority append + ack; treat it as a - // fresh quorum confirmation and extend the lease using the - // pre-dispatch timestamp. - if lp, ok := c.engine.(raftengine.LeaseProvider); ok { - c.lease.extend(dispatchStart.Add(lp.LeaseDuration())) - } - } + c.refreshLeaseAfterDispatch(resp, err, dispatchStart) return resp, err } +// refreshLeaseAfterDispatch extends the lease only when the dispatch +// produced a real Raft commit. CommitIndex == 0 means the underlying +// transaction manager short-circuited (empty-input Commit, no-op +// Abort), and refreshing would be unsound because no quorum +// confirmation happened. +func (c *Coordinate) refreshLeaseAfterDispatch(resp *CoordinateResponse, err error, dispatchStart time.Time) { + if err != nil || resp == nil || resp.CommitIndex == 0 { + return + } + lp, ok := c.engine.(raftengine.LeaseProvider) + if !ok { + return + } + c.lease.extend(dispatchStart.Add(lp.LeaseDuration())) +} + func (c *Coordinate) IsLeader() bool { return isLeaderEngine(c.engine) } diff --git a/kv/sharded_coordinator.go b/kv/sharded_coordinator.go index 6bf59f8a..68f8d199 100644 --- a/kv/sharded_coordinator.go +++ b/kv/sharded_coordinator.go @@ -24,10 +24,17 @@ type ShardGroup struct { lease leaseState } -// leaseRefreshingTxn wraps a Transactional so every successful Commit / -// Abort extends its shard's lease, treating committed Raft entries as -// fresh quorum confirmations. Mirrors Coordinate.Dispatch's lease hook -// for the per-shard case. +// leaseRefreshingTxn wraps a Transactional so every Commit / Abort that +// produced a real Raft commit extends its shard's lease. Mirrors +// Coordinate.Dispatch's lease hook for the per-shard case. +// +// Both TransactionManager.Commit and .Abort can return success WITHOUT +// going through Raft -- Commit short-circuits on empty input, Abort +// short-circuits when every request's abortRequestFor is nil (nothing +// to release). Refreshing the lease in those cases would be unsound: +// no quorum confirmation happened. We gate the refresh on +// resp.CommitIndex > 0, which the underlying manager sets to the +// last applied index only when at least one proposal went through. type leaseRefreshingTxn struct { inner Transactional g *ShardGroup @@ -39,9 +46,7 @@ func (t *leaseRefreshingTxn) Commit(reqs []*pb.Request) (*TransactionResponse, e if err != nil { return resp, errors.WithStack(err) } - if lp, ok := t.g.Engine.(raftengine.LeaseProvider); ok { - t.g.lease.extend(start.Add(lp.LeaseDuration())) - } + t.maybeRefresh(resp, start) return resp, nil } @@ -51,12 +56,23 @@ func (t *leaseRefreshingTxn) Abort(reqs []*pb.Request) (*TransactionResponse, er if err != nil { return resp, errors.WithStack(err) } - if lp, ok := t.g.Engine.(raftengine.LeaseProvider); ok { - t.g.lease.extend(start.Add(lp.LeaseDuration())) - } + t.maybeRefresh(resp, start) return resp, nil } +// maybeRefresh extends the per-shard lease only when the operation +// actually produced a Raft commit. See the struct doc comment for why. +func (t *leaseRefreshingTxn) maybeRefresh(resp *TransactionResponse, start time.Time) { + if resp == nil || resp.CommitIndex == 0 { + return + } + lp, ok := t.g.Engine.(raftengine.LeaseProvider) + if !ok { + return + } + t.g.lease.extend(start.Add(lp.LeaseDuration())) +} + const ( txnPhaseCount = 2 diff --git a/kv/sharded_lease_test.go b/kv/sharded_lease_test.go index de66a130..e134d3e1 100644 --- a/kv/sharded_lease_test.go +++ b/kv/sharded_lease_test.go @@ -7,6 +7,7 @@ import ( "time" "github.com/bootjp/elastickv/distribution" + pb "github.com/bootjp/elastickv/proto" "github.com/stretchr/testify/require" ) @@ -97,6 +98,58 @@ func TestShardedCoordinator_LeaseReadForKey_ErrorOnlyInvalidatesShard(t *testing "shard 1 lease must NOT be touched by shard 2's failure") } +func TestShardedCoordinator_LeaseRefreshingTxn_SkipsWhenCommitIndexZero(t *testing.T) { + t.Parallel() + eng1 := newShardedLeaseEngine(100) + eng2 := newShardedLeaseEngine(200) + coord := mustShardedLeaseCoord(t, eng1, eng2) + + g1 := coord.groups[1] + // A response with CommitIndex == 0 signals "no Raft proposal + // happened" (TransactionManager short-circuits on empty input / + // no-op abort). Refreshing in that case would be unsound. + noRaftResp := &TransactionResponse{CommitIndex: 0} + txn, ok := g1.Txn.(*leaseRefreshingTxn) + require.True(t, ok, "NewShardedCoordinator wraps Txn in leaseRefreshingTxn") + txn.inner = &fixedTransactional{response: noRaftResp} + + require.False(t, g1.lease.valid(time.Now())) + + // Commit with empty input returns success with CommitIndex=0. + _, err := g1.Txn.Commit(nil) + require.NoError(t, err) + require.False(t, g1.lease.valid(time.Now()), + "lease must NOT be refreshed when no Raft commit happened") + + // Same for Abort. + _, err = g1.Txn.Abort(nil) + require.NoError(t, err) + require.False(t, g1.lease.valid(time.Now())) + + // A response with CommitIndex > 0 refreshes the lease. + realResp := &TransactionResponse{CommitIndex: 42} + txn.inner = &fixedTransactional{response: realResp} + _, err = g1.Txn.Commit(nil) + require.NoError(t, err) + require.True(t, g1.lease.valid(time.Now()), + "lease must be refreshed after a real Raft commit") +} + +// fixedTransactional is a minimal Transactional whose Commit/Abort +// always return the same response. Used to drive the lease-refresh +// gating tests deterministically. +type fixedTransactional struct { + response *TransactionResponse +} + +func (f *fixedTransactional) Commit(_ []*pb.Request) (*TransactionResponse, error) { + return f.response, nil +} + +func (f *fixedTransactional) Abort(_ []*pb.Request) (*TransactionResponse, error) { + return f.response, nil +} + func TestShardedCoordinator_RegistersPerShardLeaderLossCallback(t *testing.T) { t.Parallel() From f83fed5411c602ef8784c70abd8b6298d110b64c Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 16:49:02 +0900 Subject: [PATCH 19/43] fix(lease-read): fire callbacks from fail(), log panic recoveries, warn on zero lease, sync doc triggers with code Four findings from the latest round of reviews on PR #549. - fail() now fires leader-loss callbacks on the error-shutdown path (CodeRabbit MAJOR). run() -> fail() bypasses both refreshStatus's Leader -> non-Leader edge and shutdown()'s wasLeader hook, so before this change a leader that died from handleStep / drainReady / handleEvent errors left registered lease holders serving fast-path reads from stale state for up to LeaseDuration. - invokeLeaderLossCallback now logs the recovered panic with a stack trace at error level (Copilot). A buggy callback is still contained, but operators can now see when lease-invalidation hooks misbehave in production. - Engine.Open emits a one-time slog.Warn when LeaseDuration <= 0 (CodeRabbit nitpick). validateConfig only requires ElectionTick > HeartbeatTick, so an aggressive config like TickInterval=10ms HeartbeatTick=10 ElectionTick=20 would compute 200ms - 300ms = 0 and silently disable the optimization. The warn surfaces the misconfiguration without failing Open; the slow path (LinearizableRead) remains fully functional. - docs/lease_read_design.md section 3.4 rewritten to match what the code actually does: (1) leader-loss callbacks fired from refreshStatus / fail / shutdown, (2) slow-path error invalidation, (3) CommitIndex == 0 no-op gating. The stale "term-change detection (defensive)" bullet was removed; no such check exists and (1) covers the only case it would matter. --- docs/lease_read_design.md | 24 +++++++++++++----- internal/raftengine/etcd/engine.go | 40 ++++++++++++++++++++++++++---- 2 files changed, 53 insertions(+), 11 deletions(-) diff --git a/docs/lease_read_design.md b/docs/lease_read_design.md index 103a8e2c..ee21c2a4 100644 --- a/docs/lease_read_design.md +++ b/docs/lease_read_design.md @@ -159,12 +159,24 @@ over (1) and (2). The lease is invalidated (set to nil) on: -1. State transition out of leader. `refreshStatus` in - `internal/raftengine/etcd/engine.go:1577` already detects this; we add a - coordinator hook. -2. Any error returned by `engine.Propose` or `engine.LinearizableRead`. -3. Detection of a term change since last refresh (defensive; should not - normally fire because (1) covers leader loss). +1. State transition out of leader. `refreshStatus` fires registered + `RegisterLeaderLossCallback` hooks on the `Leader -> non-Leader` edge, + and `fail()` / `shutdown()` fire the same hooks when tearing down a + node that was still leader, so the error-shutdown path does not leave + lease holders serving stale state. +2. Any error returned by `engine.Propose` or `engine.LinearizableRead` + from inside `LeaseRead` / `groupLeaseRead`. Implemented via + `c.lease.invalidate()` on the slow-path error branch. +3. A no-op Raft commit (`resp.CommitIndex == 0`): the underlying + `TransactionManager.{Commit,Abort}` can short-circuit on empty input + or no-op abort without going through Raft. `leaseRefreshingTxn` and + `Coordinate.Dispatch` only refresh the lease when `CommitIndex > 0` + to avoid extending based on an operation that never reached quorum. + +Note: the previous draft of this doc listed "term-change detection" as +a separate defensive trigger. That is not implemented; (1) covers the +only case term changes matter (an old leader being demoted), and +adding an explicit term check would be redundant. ### 3.5 API diff --git a/internal/raftengine/etcd/engine.go b/internal/raftengine/etcd/engine.go index 04f17ac5..7833729e 100644 --- a/internal/raftengine/etcd/engine.go +++ b/internal/raftengine/etcd/engine.go @@ -7,6 +7,7 @@ import ( "io" "log/slog" "path/filepath" + "runtime/debug" "sort" "strconv" "sync" @@ -332,6 +333,18 @@ func Open(ctx context.Context, cfg OpenConfig) (*Engine, error) { engine.initTransport(prepared.cfg) engine.initSnapshotWorker() engine.refreshStatus() + // Surface a misconfiguration where the tick settings produce a + // non-positive lease window: lease reads would never hit the fast + // path. Don't fail Open -- the engine is still functional via the + // slow LinearizableRead path -- but make the degradation visible. + if lease := engine.LeaseDuration(); lease <= 0 { + slog.Warn("etcd raft engine: lease read disabled (non-positive LeaseDuration)", + slog.Duration("tick_interval", engine.tickInterval), + slog.Int("election_tick", engine.electionTick), + slog.Duration("lease_safety_margin", leaseSafetyMargin), + slog.Duration("computed_lease", lease), + ) + } go engine.run() @@ -608,11 +621,19 @@ func (e *Engine) fireLeaderLossCallbacks() { func (e *Engine) invokeLeaderLossCallback(fn func()) { defer func() { if r := recover(); r != nil { - // A buggy lease holder must not crash the node. Drop the - // panic here; it has already marked the lease invalid (or - // would have, if the panic didn't interrupt it) so the - // next read takes the slow path and re-verifies leadership. - _ = r + // A buggy lease holder must not crash the node. Log the + // recovery so operators can see lease-invalidation hooks + // misbehaving in production; swallow the panic so the + // engine status loop / shutdown path continues. Safety is + // preserved because callbacks that fail to run leave the + // lease alone, and the next read takes the slow path and + // re-verifies leadership via LinearizableRead. + slog.Error("etcd raft engine: leader-loss callback panicked", + slog.String("node_id", e.localID), + slog.Uint64("raft_node_id", e.nodeID), + slog.Any("panic", r), + slog.String("stack", string(debug.Stack())), + ) } }() fn() @@ -1784,6 +1805,7 @@ func (e *Engine) fail(err error) { ) } e.mu.Lock() + wasLeader := e.status.State == raftengine.StateLeader if err != nil { e.runErr = err } @@ -1798,6 +1820,14 @@ func (e *Engine) fail(err error) { _ = closePersist(e.persist) _ = e.transport.Close() e.failPending(e.currentErrorOrClosed()) + // LeaseProvider contract: fire leader-loss callbacks on shutdown if + // we were leader. fail() is the error-shutdown twin of shutdown(); + // without firing here, a run() -> fail() path that bypasses + // refreshStatus's Leader -> non-Leader edge leaves lease holders + // serving fast-path reads from stale state for up to LeaseDuration. + if wasLeader { + e.fireLeaderLossCallbacks() + } } func (e *Engine) failPending(err error) { From 571cd933743efccd05b4ff3a22d9805008d5b6e9 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 17:04:30 +0900 Subject: [PATCH 20/43] fix(lease-read): leaseRefreshingTxn forwards Close + unify GET context Two actionable findings from the new review round on PR #549. - [Copilot HIGH-ish] leaseRefreshingTxn hid the inner Transactional's io.Closer implementation. ShardStore.closeGroup relies on `g.Txn.(io.Closer).Close()` to release per-shard resources (e.g. LeaderProxy's cached gRPC connection); after this wrap that type assertion failed and the inner Close never fired, leaking connections / goroutines at shutdown. Add Close() to leaseRefreshingTxn that type-asserts and delegates. Absent-Closer inner is a safe no-op. Tests: - TestLeaseRefreshingTxn_ForwardsClose verifies the io.Closer assertion still finds Close, and it reaches the inner value. - TestLeaseRefreshingTxn_CloseNoopWhenInnerIsNotCloser guards the fallback. - [gemini MED] adapter/redis.go GET used two separate contexts: a bounded leaseCtx for LeaseReadForKey and context.Background() for the follow-up keyTypeAt. Collapsed into a single redisDispatchTimeout-bounded ctx with `defer cancel()` for the whole handler so the deadline applies uniformly. The other two inline comments in that round (heap alloc per extend, AppliedIndex RLock contention) are already tracked as #554 and #553 and intentionally deferred from this PR. --- adapter/redis.go | 17 ++++++++-------- kv/sharded_coordinator.go | 18 +++++++++++++++++ kv/sharded_lease_test.go | 41 +++++++++++++++++++++++++++++++++++++++ 3 files changed, 67 insertions(+), 9 deletions(-) diff --git a/adapter/redis.go b/adapter/redis.go index 46673364..4745eedc 100644 --- a/adapter/redis.go +++ b/adapter/redis.go @@ -981,19 +981,18 @@ func (r *RedisServer) get(conn redcon.Conn, cmd redcon.Command) { return } - // Bounded context: LeaseReadForKey's slow path runs LinearizableRead, - // which can block until quorum is reached. Without a deadline a - // stalled Raft makes the GET hang indefinitely; cap it the same way - // the proxy path does. - leaseCtx, leaseCancel := context.WithTimeout(context.Background(), redisDispatchTimeout) - if _, err := r.coordinator.LeaseReadForKey(leaseCtx, key); err != nil { - leaseCancel() + // Single bounded context for the whole request handler. The lease + // check and the subsequent local-only keyTypeAt / readRedisStringAt + // calls all respect the same deadline so a stalled Raft cannot + // leave the handler running past it. + ctx, cancel := context.WithTimeout(context.Background(), redisDispatchTimeout) + defer cancel() + if _, err := r.coordinator.LeaseReadForKey(ctx, key); err != nil { conn.WriteError(err.Error()) return } - leaseCancel() readTS := r.readTS() - typ, err := r.keyTypeAt(context.Background(), key, readTS) + typ, err := r.keyTypeAt(ctx, key, readTS) if err != nil { conn.WriteError(err.Error()) return diff --git a/kv/sharded_coordinator.go b/kv/sharded_coordinator.go index 68f8d199..a1abf337 100644 --- a/kv/sharded_coordinator.go +++ b/kv/sharded_coordinator.go @@ -3,6 +3,7 @@ package kv import ( "bytes" "context" + "io" "log/slog" "slices" "sync" @@ -73,6 +74,23 @@ func (t *leaseRefreshingTxn) maybeRefresh(resp *TransactionResponse, start time. t.g.lease.extend(start.Add(lp.LeaseDuration())) } +// Close forwards to the wrapped Transactional if it implements +// io.Closer. ShardStore.closeGroup relies on the type assertion +// `g.Txn.(io.Closer)` to release per-shard resources (e.g. the gRPC +// connection cached by LeaderProxy). Without this pass-through, the +// wrapping would silently swallow the Closer capability and leak +// connections / goroutines at shutdown. +func (t *leaseRefreshingTxn) Close() error { + closer, ok := t.inner.(io.Closer) + if !ok { + return nil + } + if err := closer.Close(); err != nil { + return errors.WithStack(err) + } + return nil +} + const ( txnPhaseCount = 2 diff --git a/kv/sharded_lease_test.go b/kv/sharded_lease_test.go index e134d3e1..19d51266 100644 --- a/kv/sharded_lease_test.go +++ b/kv/sharded_lease_test.go @@ -3,6 +3,8 @@ package kv import ( "context" "errors" + "io" + "sync/atomic" "testing" "time" @@ -150,6 +152,45 @@ func (f *fixedTransactional) Abort(_ []*pb.Request) (*TransactionResponse, error return f.response, nil } +// closableTransactional satisfies both Transactional and io.Closer so +// the Close-delegation test can observe whether the wrapper forwards +// Close to the inner value. +type closableTransactional struct { + fixedTransactional + closed atomic.Bool +} + +func (c *closableTransactional) Close() error { + c.closed.Store(true) + return nil +} + +func TestLeaseRefreshingTxn_ForwardsClose(t *testing.T) { + t.Parallel() + inner := &closableTransactional{ + fixedTransactional: fixedTransactional{response: &TransactionResponse{}}, + } + wrapper := &leaseRefreshingTxn{inner: inner, g: &ShardGroup{}} + + // ShardStore.closeGroup does `g.Txn.(io.Closer).Close()`. After + // wrapping, the same assertion must still discover a Close method + // that reaches the inner Transactional. + closer, ok := interface{}(wrapper).(io.Closer) + require.True(t, ok, "leaseRefreshingTxn must implement io.Closer") + require.NoError(t, closer.Close()) + require.True(t, inner.closed.Load(), + "Close must delegate to the wrapped Transactional so ShardStore.closeGroup can release its resources") +} + +func TestLeaseRefreshingTxn_CloseNoopWhenInnerIsNotCloser(t *testing.T) { + t.Parallel() + // fixedTransactional does NOT implement io.Closer. The wrapper's + // Close must be a safe no-op rather than panicking. + inner := &fixedTransactional{response: &TransactionResponse{}} + wrapper := &leaseRefreshingTxn{inner: inner, g: &ShardGroup{}} + require.NoError(t, wrapper.Close()) +} + func TestShardedCoordinator_RegistersPerShardLeaderLossCallback(t *testing.T) { t.Parallel() From f60d7440c4cee2f5a79f460521ad223eabe8c782 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 17:17:13 +0900 Subject: [PATCH 21/43] style(lease-read): Copilot nits on test busy-loop + comment accuracy Two trivial findings from the latest Copilot pass on PR #549. Both are stylistic. - kv/lease_state_test.go TestLeaseState_ConcurrentExtendAndRead: the two workers used tight `select { default: ... }` loops that peg a core for the 20ms duration. Interleave each iteration with runtime.Gosched() to keep the atomic-pointer invariants exercised under -race without busy-waiting. - kv/sharded_lease_test.go: the comment describing ShardStore.closeGroup said `g.Txn.(io.Closer).Close()` (bare assertion) when the actual code uses a guarded `if closer, ok := g.Txn.(io.Closer); ok { ... }`. Updated the comment so future readers don't misread the type-safety contract. --- kv/lease_state_test.go | 7 +++++++ kv/sharded_lease_test.go | 7 ++++--- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/kv/lease_state_test.go b/kv/lease_state_test.go index b155ccdc..44db7905 100644 --- a/kv/lease_state_test.go +++ b/kv/lease_state_test.go @@ -1,6 +1,7 @@ package kv import ( + "runtime" "testing" "time" @@ -125,6 +126,10 @@ func TestLeaseState_ConcurrentExtendAndRead(t *testing.T) { stop := make(chan struct{}) done := make(chan struct{}, 2) + // Cooperative scheduling: runtime.Gosched() between iterations keeps + // the workers from pegging a core while still interleaving enough + // extend/valid pairs under `-race` to exercise the atomic-pointer + // invariants. go func() { defer func() { done <- struct{}{} }() for { @@ -133,6 +138,7 @@ func TestLeaseState_ConcurrentExtendAndRead(t *testing.T) { return default: s.extend(time.Now().Add(time.Second)) + runtime.Gosched() } } }() @@ -144,6 +150,7 @@ func TestLeaseState_ConcurrentExtendAndRead(t *testing.T) { return default: _ = s.valid(time.Now()) + runtime.Gosched() } } }() diff --git a/kv/sharded_lease_test.go b/kv/sharded_lease_test.go index 19d51266..b8b30583 100644 --- a/kv/sharded_lease_test.go +++ b/kv/sharded_lease_test.go @@ -172,9 +172,10 @@ func TestLeaseRefreshingTxn_ForwardsClose(t *testing.T) { } wrapper := &leaseRefreshingTxn{inner: inner, g: &ShardGroup{}} - // ShardStore.closeGroup does `g.Txn.(io.Closer).Close()`. After - // wrapping, the same assertion must still discover a Close method - // that reaches the inner Transactional. + // ShardStore.closeGroup does a guarded type assertion + // `if closer, ok := g.Txn.(io.Closer); ok { closer.Close() }`. + // After wrapping, that `ok` must still be true and the resulting + // Close must reach the inner Transactional. closer, ok := interface{}(wrapper).(io.Closer) require.True(t, ok, "leaseRefreshingTxn must implement io.Closer") require.NoError(t, closer.Close()) From 90f78f0f3244d91b27b1e05082af3b317b771007 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 17:35:19 +0900 Subject: [PATCH 22/43] docs(lease-read): clarify pre-dispatch sampling is strictly conservative Copilot round 8 pointed out that the "quorum-confirmation instant" / "real quorum confirmation instant" phrasing was misleading: the captured instant is sampled BEFORE Propose / LinearizableRead runs, so the resulting lease window can only be shorter than the true safety window, never longer. The implementation is unchanged and correct; this reshapes the inline comments and the design-doc pseudocode so they describe the actual timing semantics: - Coordinate.Dispatch: pre-dispatch sample is strictly conservative. - Coordinate.LeaseRead: one time.Now() shared between fast-path validity check and slow-path extend base, and it runs before LinearizableRead so the extend base is never later than the real quorum confirmation. - docs/lease_read_design.md: pseudocode now captures a single `now` and comments match the inline code in kv/coordinator.go. --- docs/lease_read_design.md | 15 ++++++++++++--- kv/coordinator.go | 25 +++++++++++++++---------- 2 files changed, 27 insertions(+), 13 deletions(-) diff --git a/docs/lease_read_design.md b/docs/lease_read_design.md index ee21c2a4..e9a415d9 100644 --- a/docs/lease_read_design.md +++ b/docs/lease_read_design.md @@ -209,16 +209,25 @@ func (c *Coordinate) LeaseRead(ctx context.Context) (uint64, error) { if !ok { return c.LinearizableRead(ctx) // hashicorp engine, test stubs } - if c.lease.valid(time.Now()) { + // Capture time.Now() exactly once. Reused for both the fast-path + // validity check and (on the slow path) the lease-extension base: + // a second sample would let the fast path accept a read whose + // instant slightly exceeds the intended expiry boundary, while + // also shortening the slow-path window by the same delta. + now := time.Now() + if c.lease.valid(now) { return lp.AppliedIndex(), nil } - readStart := time.Now() // sample BEFORE idx, err := c.LinearizableRead(ctx) if err != nil { c.lease.invalidate() return 0, err } - c.lease.extend(readStart.Add(lp.LeaseDuration())) // monotonic CAS + // `now` was sampled strictly before LinearizableRead ran, so the + // resulting lease window is strictly conservative (any real + // quorum confirmation LinearizableRead witnessed happens at or + // after `now`). extend uses a monotonic CAS to reject regressions. + c.lease.extend(now.Add(lp.LeaseDuration())) return idx, nil } ``` diff --git a/kv/coordinator.go b/kv/coordinator.go index 0da56f0c..e5de4896 100644 --- a/kv/coordinator.go +++ b/kv/coordinator.go @@ -130,12 +130,15 @@ func (c *Coordinate) Dispatch(ctx context.Context, reqs *OperationGroup[OP]) (*C reqs.CommitTS = 0 } - // Sample the clock BEFORE dispatching so the lease extension reflects - // the moment we know the leader was alive at quorum confirmation, not - // the moment dispatch returned. Otherwise apply-queue depth and - // scheduling jitter could push the effective lease window past - // electionTimeout, allowing a stale leader to serve reads beyond the - // safety bound. + // Sample the clock BEFORE dispatching and use this pre-dispatch + // instant as the lease-extension base on success. Any real quorum + // confirmation necessarily happens at or after this instant, so + // basing the lease here is strictly conservative -- the effective + // window can only be SHORTER than the actual safety window, never + // longer. Without this, apply-queue depth / scheduling jitter + // between Propose return and the lease extend call could push the + // window past electionTimeout and let a stale leader serve reads + // beyond the safety bound. dispatchStart := time.Now() var resp *CoordinateResponse var err error @@ -272,10 +275,12 @@ func (c *Coordinate) LeaseRead(ctx context.Context) (uint64, error) { if c.lease.valid(now) { return lp.AppliedIndex(), nil } - // The captured `now` also serves as readStart: it is strictly - // before LinearizableRead returns, so the lease window starts at - // the real quorum confirmation instant, not after the heartbeat - // round returned. See Coordinate.Dispatch for the same rationale. + // The captured `now` also serves as the lease-extension base: it + // is sampled strictly before LinearizableRead runs, so any real + // quorum confirmation LinearizableRead witnesses happens at or + // after `now`. The resulting lease window is therefore strictly + // conservative (can only shorten, never overextend). See + // Coordinate.Dispatch for the same rationale. idx, err := c.LinearizableRead(ctx) if err != nil { c.lease.invalidate() From b82ba32399a15a3750149a4015ee1c9be96ae7e8 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 17:49:09 +0900 Subject: [PATCH 23/43] fix(lease-read): sample generation BEFORE quorum op (Critical race) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit CodeRabbit flagged Critical on PR #549: leaseState.extend captured `gen` internally (AFTER the quorum operation returned). If a leader-loss callback fired DURING the operation, gen was bumped and expiry cleared; but the caller's post-op extend observed the post-invalidate gen as "current", the CAS installed a fresh expiry, and the post-CAS gen check still matched (also post-invalidate) -- the stale lease was resurrected. Fix: callers now sample `expectedGen := lease.generation()` BEFORE the quorum operation and pass it to `extend(until, expectedGen)`. extend gates both pre-CAS and post-CAS on that expectedGen, so any intervening invalidate is observed as a mismatch and the extend is rejected. Call sites updated: - kv/coordinator.go Coordinate.Dispatch: captures gen alongside dispatchStart before dispatchTxn/dispatchRaw. - kv/coordinator.go Coordinate.LeaseRead: captures gen alongside `now` before the fast-path check / LinearizableRead. - kv/sharded_coordinator.go leaseRefreshingTxn.Commit/Abort: captures gen before inner.Commit/Abort. - kv/sharded_coordinator.go groupLeaseRead: captures gen alongside `now` before LinearizableRead. Regression tests: - TestLeaseState_ExtendCannotResurrectAfterInvalidate now exercises the stale-gen case via the public generation()/extend API instead of manual CAS replay. - TestLeaseState_ExtendWithFreshGenSucceedsAfterInvalidate codifies the dual: an extend with post-invalidate gen CAN install a fresh lease, so recovery from brief leader-loss is possible. Plus Copilot nits addressed in the same pass: - adapter/redis.go GET: fixed the comment about which calls respect the handler deadline (readRedisStringAt does not accept ctx; only LeaseReadForKey and keyTypeAt do). - internal/raftengine/etcd/engine.go invokeLeaderLossCallback: reworded the panic-recovery comment to reflect that a panic before invalidation means fast-path reads continue until wall-clock expiry -- safety is bounded by LeaseDuration, not by an immediate slow-path fallback. docs/lease_read_design.md §3.3 and §4.3 updated: refresh base is preOpInstant not postOpInstant; added generation-guard to the refresh-trigger description; tightened the §4.3 discussion so it no longer counts the round-trip in leaseSafetyMargin. --- adapter/redis.go | 9 ++-- docs/lease_read_design.md | 31 +++++++++---- internal/raftengine/etcd/engine.go | 13 ++++-- kv/coordinator.go | 46 +++++++++---------- kv/lease_read_test.go | 6 +-- kv/lease_state.go | 44 ++++++++++++------ kv/lease_state_test.go | 73 +++++++++++++++--------------- kv/sharded_coordinator.go | 23 ++++++---- kv/sharded_lease_test.go | 10 ++-- 9 files changed, 149 insertions(+), 106 deletions(-) diff --git a/adapter/redis.go b/adapter/redis.go index 4745eedc..e44735d7 100644 --- a/adapter/redis.go +++ b/adapter/redis.go @@ -981,10 +981,11 @@ func (r *RedisServer) get(conn redcon.Conn, cmd redcon.Command) { return } - // Single bounded context for the whole request handler. The lease - // check and the subsequent local-only keyTypeAt / readRedisStringAt - // calls all respect the same deadline so a stalled Raft cannot - // leave the handler running past it. + // Single bounded context for the slow paths in this handler. + // Only LeaseReadForKey and keyTypeAt accept a context; + // readRedisStringAt is a local-store read that does not take one. + // The shared deadline still bounds the only branches that can + // actually block on quorum / I/O. ctx, cancel := context.WithTimeout(context.Background(), redisDispatchTimeout) defer cancel() if _, err := r.coordinator.LeaseReadForKey(ctx, key); err != nil { diff --git a/docs/lease_read_design.md b/docs/lease_read_design.md index e9a415d9..0d27da5e 100644 --- a/docs/lease_read_design.md +++ b/docs/lease_read_design.md @@ -143,7 +143,7 @@ safety buffer. ### 3.3 Refresh triggers -The lease is refreshed (set to `time.Now() + LeaseDuration()`) on: +The lease is refreshed on: 1. Any successful `engine.LinearizableRead(ctx)` returning without error. The ReadIndex protocol confirmed quorum at that moment. @@ -151,6 +151,20 @@ The lease is refreshed (set to `time.Now() + LeaseDuration()`) on: A committed entry implies majority append + ack, which is a stronger confirmation than ReadIndex. +Both refresh base the new expiry on `preOpInstant + LeaseDuration()`, +where `preOpInstant` is captured BEFORE the quorum operation starts, not +after it returns. This is strictly conservative: any real quorum +confirmation must happen at or after `preOpInstant`, so the lease window +can only be shorter than the true safety window, never longer. +Post-operation sampling would let apply-queue depth / scheduling jitter +push the window past `electionTimeout`. + +Alongside the instant, the caller also captures the lease generation via +`leaseState.generation()` and passes both to `extend(until, expectedGen)`. +The generation guard prevents a leader-loss invalidation that fires +during the quorum operation from being silently overwritten by the +caller's post-op extend. + Heartbeat ack tracking is intentionally not used. It would require deep hooks into etcd/raft's internals and gives only a small marginal benefit over (1) and (2). @@ -283,13 +297,14 @@ replaced. ### 4.3 Refresh-vs-ack gap The design refreshes the lease on `LinearizableRead` and `Propose` -completion, not on individual heartbeat acks. This widens the gap between -the actual quorum confirmation event and the lease extension event by at -most one round-trip plus goroutine scheduling. - -This gap is included in `leaseSafetyMargin`. Specifically, if the round-trip -plus scheduling delay is bounded by `D`, then `margin >= D + clock_skew_bound` -preserves the invariant. +completion, not on individual heartbeat acks. The ambiguity of exactly +when quorum was confirmed during the operation is bounded by sampling +the lease base BEFORE the operation starts (see 3.3): any real quorum +confirmation happens at or after that sample, so the computed window +can only be shorter than the true safety window, never longer. The +`leaseSafetyMargin` still absorbs bounded wall-clock skew between the +leader's local clock and a partition's successor leader's clock; it no +longer has to absorb the round-trip latency of the quorum operation. ### 4.4 Comparison to current state diff --git a/internal/raftengine/etcd/engine.go b/internal/raftengine/etcd/engine.go index 7833729e..accc8bde 100644 --- a/internal/raftengine/etcd/engine.go +++ b/internal/raftengine/etcd/engine.go @@ -624,10 +624,15 @@ func (e *Engine) invokeLeaderLossCallback(fn func()) { // A buggy lease holder must not crash the node. Log the // recovery so operators can see lease-invalidation hooks // misbehaving in production; swallow the panic so the - // engine status loop / shutdown path continues. Safety is - // preserved because callbacks that fail to run leave the - // lease alone, and the next read takes the slow path and - // re-verifies leadership via LinearizableRead. + // engine status loop / shutdown path continues. + // + // Note: if a callback panics before it invalidates its + // lease, fast-path reads on that lease keep succeeding + // until wall-clock expiry. Safety is then bounded by the + // lease duration (strictly shorter than electionTimeout), + // not by the slow-path re-verification. The slow path + // re-verifies leadership only once the lease has + // naturally expired. slog.Error("etcd raft engine: leader-loss callback panicked", slog.String("node_id", e.localID), slog.Uint64("raft_node_id", e.nodeID), diff --git a/kv/coordinator.go b/kv/coordinator.go index e5de4896..8bc05f87 100644 --- a/kv/coordinator.go +++ b/kv/coordinator.go @@ -130,16 +130,18 @@ func (c *Coordinate) Dispatch(ctx context.Context, reqs *OperationGroup[OP]) (*C reqs.CommitTS = 0 } - // Sample the clock BEFORE dispatching and use this pre-dispatch - // instant as the lease-extension base on success. Any real quorum - // confirmation necessarily happens at or after this instant, so - // basing the lease here is strictly conservative -- the effective - // window can only be SHORTER than the actual safety window, never - // longer. Without this, apply-queue depth / scheduling jitter - // between Propose return and the lease extend call could push the - // window past electionTimeout and let a stale leader serve reads - // beyond the safety bound. + // Sample the clock AND the lease generation BEFORE dispatching. + // * dispatchStart: any real quorum confirmation happens at or + // after this instant, so using it as the lease-extension base + // is strictly conservative (window can only be SHORTER than + // the actual safety window, never longer). + // * expectedGen: if a leader-loss callback fires between this + // sample and the post-dispatch extend, the generation will + // have advanced; extend(expectedGen) will see the mismatch + // and refuse to resurrect the lease. Capturing gen INSIDE + // extend would observe the post-invalidate value as current. dispatchStart := time.Now() + expectedGen := c.lease.generation() var resp *CoordinateResponse var err error if reqs.IsTxn { @@ -147,7 +149,7 @@ func (c *Coordinate) Dispatch(ctx context.Context, reqs *OperationGroup[OP]) (*C } else { resp, err = c.dispatchRaw(reqs.Elems) } - c.refreshLeaseAfterDispatch(resp, err, dispatchStart) + c.refreshLeaseAfterDispatch(resp, err, dispatchStart, expectedGen) return resp, err } @@ -156,7 +158,7 @@ func (c *Coordinate) Dispatch(ctx context.Context, reqs *OperationGroup[OP]) (*C // transaction manager short-circuited (empty-input Commit, no-op // Abort), and refreshing would be unsound because no quorum // confirmation happened. -func (c *Coordinate) refreshLeaseAfterDispatch(resp *CoordinateResponse, err error, dispatchStart time.Time) { +func (c *Coordinate) refreshLeaseAfterDispatch(resp *CoordinateResponse, err error, dispatchStart time.Time, expectedGen uint64) { if err != nil || resp == nil || resp.CommitIndex == 0 { return } @@ -164,7 +166,7 @@ func (c *Coordinate) refreshLeaseAfterDispatch(resp *CoordinateResponse, err err if !ok { return } - c.lease.extend(dispatchStart.Add(lp.LeaseDuration())) + c.lease.extend(dispatchStart.Add(lp.LeaseDuration()), expectedGen) } func (c *Coordinate) IsLeader() bool { @@ -266,27 +268,23 @@ func (c *Coordinate) LeaseRead(ctx context.Context) (uint64, error) { if !ok { return c.LinearizableRead(ctx) } - // Capture time.Now() exactly once so the fast-path validity check - // and the slow-path lease-extend base share the same instant. A - // second sampling would let the fast path accept a read whose - // instant is slightly after expiry (and, conversely, shorten the - // slow-path lease window by the same delta). + // Capture time.Now() and the lease generation exactly once before + // any quorum work. `now` is reused for both the fast-path validity + // check and (on slow path) the extend base; `expectedGen` guards + // against a leader-loss invalidation that fires during + // LinearizableRead from being overwritten by this caller's extend. + // See Coordinate.Dispatch for the same rationale. now := time.Now() + expectedGen := c.lease.generation() if c.lease.valid(now) { return lp.AppliedIndex(), nil } - // The captured `now` also serves as the lease-extension base: it - // is sampled strictly before LinearizableRead runs, so any real - // quorum confirmation LinearizableRead witnesses happens at or - // after `now`. The resulting lease window is therefore strictly - // conservative (can only shorten, never overextend). See - // Coordinate.Dispatch for the same rationale. idx, err := c.LinearizableRead(ctx) if err != nil { c.lease.invalidate() return 0, err } - c.lease.extend(now.Add(lp.LeaseDuration())) + c.lease.extend(now.Add(lp.LeaseDuration()), expectedGen) return idx, nil } diff --git a/kv/lease_read_test.go b/kv/lease_read_test.go index a2874f26..15804596 100644 --- a/kv/lease_read_test.go +++ b/kv/lease_read_test.go @@ -102,7 +102,7 @@ func TestCoordinate_LeaseRead_FastPathSkipsEngine(t *testing.T) { eng := &fakeLeaseEngine{applied: 100, leaseDur: time.Hour} c := NewCoordinatorWithEngine(nil, eng) - c.lease.extend(time.Now().Add(time.Hour)) + c.lease.extend(time.Now().Add(time.Hour), c.lease.generation()) idx, err := c.LeaseRead(context.Background()) require.NoError(t, err) @@ -134,7 +134,7 @@ func TestCoordinate_LeaseRead_ErrorInvalidatesLease(t *testing.T) { eng := &fakeLeaseEngine{applied: 7, leaseDur: time.Hour, linearizableErr: sentinel} c := NewCoordinatorWithEngine(nil, eng) - c.lease.extend(time.Now().Add(time.Hour)) + c.lease.extend(time.Now().Add(time.Hour), c.lease.generation()) c.lease.invalidate() // force slow path _, err := c.LeaseRead(context.Background()) @@ -173,7 +173,7 @@ func TestCoordinate_RegistersLeaderLossCallback(t *testing.T) { c := NewCoordinatorWithEngine(nil, eng) require.Equal(t, int32(1), eng.registerLeaderLossCalled.Load()) - c.lease.extend(time.Now().Add(time.Hour)) + c.lease.extend(time.Now().Add(time.Hour), c.lease.generation()) require.True(t, c.lease.valid(time.Now())) eng.fireLeaderLoss() diff --git a/kv/lease_state.go b/kv/lease_state.go index 17ff8887..c30cd3d4 100644 --- a/kv/lease_state.go +++ b/kv/lease_state.go @@ -31,19 +31,36 @@ func (s *leaseState) valid(now time.Time) bool { return now.Before(*exp) } -// extend sets the lease expiry to until iff (a) until is strictly after -// the currently stored expiry (or no expiry is stored) and (b) no -// invalidate happened between when this caller started and when its -// CAS landed. Without the generation guard, a Dispatch that returned -// successfully *just before* a leader-loss invalidate could resurrect -// the lease milliseconds after invalidation, defeating the purpose of -// the leader-loss callback. -func (s *leaseState) extend(until time.Time) { +// generation returns the current invalidation counter. Callers MUST +// sample this BEFORE issuing the quorum-confirming operation (Propose +// / LinearizableRead) and pass the result to extend. Sampling inside +// extend (after the operation returned) would see any leader-loss +// invalidation that fired DURING the operation as the "current" +// generation and let a stale lease resurrect. +func (s *leaseState) generation() uint64 { + if s == nil { + return 0 + } + return s.gen.Load() +} + +// extend sets the lease expiry to until iff (a) until is strictly +// after the currently stored expiry (or no expiry is stored) and +// (b) no invalidate has happened since the caller captured +// expectedGen via generation() BEFORE the quorum operation. The +// generation guard prevents a Dispatch that returned successfully +// *just before* a leader-loss invalidate from resurrecting the +// lease milliseconds after invalidation. +func (s *leaseState) extend(until time.Time, expectedGen uint64) { if s == nil { return } - expectedGen := s.gen.Load() for { + // Pre-CAS gate: if invalidate already advanced the generation + // past expectedGen, skip the CAS entirely. + if s.gen.Load() != expectedGen { + return + } current := s.expiry.Load() if current != nil && !until.After(*current) { return @@ -51,10 +68,11 @@ func (s *leaseState) extend(until time.Time) { if !s.expiry.CompareAndSwap(current, &until) { continue } - // CAS landed. If invalidate raced ahead, undo our write iff - // no later writer has replaced it. Using CAS with our own - // pointer means a fresh extend that captured the - // post-invalidate generation is left intact. + // CAS landed. If invalidate raced in between the pre-CAS gate + // and the CAS itself, undo our write iff no later writer has + // replaced it. Using CAS with our own pointer means a fresh + // extend that captured the post-invalidate generation is left + // intact. if s.gen.Load() != expectedGen { s.expiry.CompareAndSwap(&until, nil) } diff --git a/kv/lease_state_test.go b/kv/lease_state_test.go index 44db7905..b4b7c268 100644 --- a/kv/lease_state_test.go +++ b/kv/lease_state_test.go @@ -12,8 +12,8 @@ func TestLeaseState_NilReceiverIsAlwaysExpired(t *testing.T) { t.Parallel() var s *leaseState require.False(t, s.valid(time.Now())) - s.extend(time.Now().Add(time.Hour)) // must not panic - s.invalidate() // must not panic + s.extend(time.Now().Add(time.Hour), s.generation()) // must not panic + s.invalidate() // must not panic require.False(t, s.valid(time.Now())) } @@ -27,7 +27,7 @@ func TestLeaseState_ExtendAndExpire(t *testing.T) { t.Parallel() var s leaseState now := time.Now() - s.extend(now.Add(50 * time.Millisecond)) + s.extend(now.Add(50*time.Millisecond), s.generation()) require.True(t, s.valid(now)) require.True(t, s.valid(now.Add(49*time.Millisecond))) @@ -39,7 +39,7 @@ func TestLeaseState_InvalidateClears(t *testing.T) { t.Parallel() var s leaseState now := time.Now() - s.extend(now.Add(time.Hour)) + s.extend(now.Add(time.Hour), s.generation()) require.True(t, s.valid(now)) s.invalidate() @@ -51,18 +51,18 @@ func TestLeaseState_ExtendIsMonotonic(t *testing.T) { var s leaseState now := time.Now() - s.extend(now.Add(time.Hour)) + s.extend(now.Add(time.Hour), s.generation()) require.True(t, s.valid(now.Add(30*time.Minute))) // A shorter extension must NOT regress the lease: an out-of-order // writer that sampled time.Now() earlier could otherwise prematurely // expire a freshly extended lease and force callers into the slow // path while the leader is still confirmed. - s.extend(now.Add(time.Minute)) + s.extend(now.Add(time.Minute), s.generation()) require.True(t, s.valid(now.Add(30*time.Minute))) // A strictly longer extension wins. - s.extend(now.Add(2 * time.Hour)) + s.extend(now.Add(2*time.Hour), s.generation()) require.True(t, s.valid(now.Add(90*time.Minute))) } @@ -70,7 +70,7 @@ func TestLeaseState_InvalidateBeatsConcurrentExtend(t *testing.T) { t.Parallel() var s leaseState now := time.Now() - s.extend(now.Add(time.Hour)) + s.extend(now.Add(time.Hour), s.generation()) // invalidate stores nil unconditionally, even when the current expiry // is in the future. Otherwise leadership-loss callbacks would be @@ -80,46 +80,44 @@ func TestLeaseState_InvalidateBeatsConcurrentExtend(t *testing.T) { } // TestLeaseState_ExtendCannotResurrectAfterInvalidate exercises the -// generation-guard invariant: an extend call whose internal CAS lands -// after a concurrent invalidate must undo its own write rather than -// leave the lease alive. Without the guard, a Dispatch that succeeded -// just before the leader-loss callback could resurrect the lease for -// up to LeaseDuration. +// generation-guard invariant: an extend that captured the pre-invalidate +// generation must not install a fresh lease after a concurrent +// invalidate has bumped the generation. func TestLeaseState_ExtendCannotResurrectAfterInvalidate(t *testing.T) { t.Parallel() var s leaseState now := time.Now() - // Simulate the race: extend captured the current generation, then - // invalidate fires before extend's CAS lands. The simplest way to - // exercise this deterministically is to invalidate first (so the - // extend sees the bumped generation) and verify extend leaves the - // state invalidated. - expectedGen := s.gen.Load() + // Caller pattern: sample generation BEFORE the quorum operation. + expectedGen := s.generation() + + // Leader-loss callback fires during the "quorum operation". s.invalidate() - require.NotEqual(t, expectedGen, s.gen.Load(), + require.NotEqual(t, expectedGen, s.generation(), "invalidate must bump the generation") - // Manually replay an extend that captured the pre-invalidate - // generation. Use the package-internal helper to avoid time-based - // races. - until := now.Add(time.Hour) - for { - current := s.expiry.Load() - if !s.expiry.CompareAndSwap(current, &until) { - continue - } - // Match the production extend's post-CAS check. - if s.gen.Load() != expectedGen { - s.expiry.CompareAndSwap(&until, nil) - } - break - } - + // Caller returns with success and calls extend with the stale + // expected-generation. Must be a no-op. + s.extend(now.Add(time.Hour), expectedGen) require.False(t, s.valid(now), "stale-generation extend must NOT resurrect the lease") } +// TestLeaseState_ExtendWithFreshGenSucceedsAfterInvalidate verifies the +// dual to the above: a caller that captured the post-invalidate +// generation CAN install a fresh lease, so recovery from a brief +// leader-loss is possible. +func TestLeaseState_ExtendWithFreshGenSucceedsAfterInvalidate(t *testing.T) { + t.Parallel() + var s leaseState + now := time.Now() + + s.invalidate() + freshGen := s.generation() + s.extend(now.Add(time.Hour), freshGen) + require.True(t, s.valid(now)) +} + func TestLeaseState_ConcurrentExtendAndRead(t *testing.T) { t.Parallel() var s leaseState @@ -137,7 +135,8 @@ func TestLeaseState_ConcurrentExtendAndRead(t *testing.T) { case <-stop: return default: - s.extend(time.Now().Add(time.Second)) + gen := s.generation() + s.extend(time.Now().Add(time.Second), gen) runtime.Gosched() } } diff --git a/kv/sharded_coordinator.go b/kv/sharded_coordinator.go index a1abf337..e35a657e 100644 --- a/kv/sharded_coordinator.go +++ b/kv/sharded_coordinator.go @@ -43,27 +43,32 @@ type leaseRefreshingTxn struct { func (t *leaseRefreshingTxn) Commit(reqs []*pb.Request) (*TransactionResponse, error) { start := time.Now() + expectedGen := t.g.lease.generation() resp, err := t.inner.Commit(reqs) if err != nil { return resp, errors.WithStack(err) } - t.maybeRefresh(resp, start) + t.maybeRefresh(resp, start, expectedGen) return resp, nil } func (t *leaseRefreshingTxn) Abort(reqs []*pb.Request) (*TransactionResponse, error) { start := time.Now() + expectedGen := t.g.lease.generation() resp, err := t.inner.Abort(reqs) if err != nil { return resp, errors.WithStack(err) } - t.maybeRefresh(resp, start) + t.maybeRefresh(resp, start, expectedGen) return resp, nil } // maybeRefresh extends the per-shard lease only when the operation -// actually produced a Raft commit. See the struct doc comment for why. -func (t *leaseRefreshingTxn) maybeRefresh(resp *TransactionResponse, start time.Time) { +// actually produced a Raft commit. expectedGen is sampled BEFORE the +// underlying Commit/Abort so an invalidation that fires during that +// call observes a generation mismatch inside extend and the refresh +// is rejected. See the struct doc comment for why. +func (t *leaseRefreshingTxn) maybeRefresh(resp *TransactionResponse, start time.Time, expectedGen uint64) { if resp == nil || resp.CommitIndex == 0 { return } @@ -71,7 +76,7 @@ func (t *leaseRefreshingTxn) maybeRefresh(resp *TransactionResponse, start time. if !ok { return } - t.g.lease.extend(start.Add(lp.LeaseDuration())) + t.g.lease.extend(start.Add(lp.LeaseDuration()), expectedGen) } // Close forwards to the wrapped Transactional if it implements @@ -706,9 +711,11 @@ func groupLeaseRead(ctx context.Context, g *ShardGroup) (uint64, error) { if !ok { return linearizableReadEngineCtx(ctx, engine) } - // Single time.Now() sample shared by both the fast-path validity - // check and the slow-path extend base, mirroring Coordinate.LeaseRead. + // Single time.Now() and generation sample before any quorum work, + // mirroring Coordinate.LeaseRead. expectedGen guards against a + // leader-loss invalidation that fires during LinearizableRead. now := time.Now() + expectedGen := g.lease.generation() if g.lease.valid(now) { return lp.AppliedIndex(), nil } @@ -717,7 +724,7 @@ func groupLeaseRead(ctx context.Context, g *ShardGroup) (uint64, error) { g.lease.invalidate() return 0, err } - g.lease.extend(now.Add(lp.LeaseDuration())) + g.lease.extend(now.Add(lp.LeaseDuration()), expectedGen) return idx, nil } diff --git a/kv/sharded_lease_test.go b/kv/sharded_lease_test.go index b8b30583..bdec9223 100644 --- a/kv/sharded_lease_test.go +++ b/kv/sharded_lease_test.go @@ -58,7 +58,7 @@ func TestShardedCoordinator_LeaseReadForKey_PerShardIsolation(t *testing.T) { // Pre-extend shard 1's lease only. g1 := coord.groups[1] - g1.lease.extend(time.Now().Add(time.Hour)) + g1.lease.extend(time.Now().Add(time.Hour), g1.lease.generation()) idx, err := coord.LeaseReadForKey(context.Background(), []byte("apple")) require.NoError(t, err) @@ -88,8 +88,8 @@ func TestShardedCoordinator_LeaseReadForKey_ErrorOnlyInvalidatesShard(t *testing g1 := coord.groups[1] g2 := coord.groups[2] - g1.lease.extend(time.Now().Add(time.Hour)) - g2.lease.extend(time.Now().Add(time.Hour)) + g1.lease.extend(time.Now().Add(time.Hour), g1.lease.generation()) + g2.lease.extend(time.Now().Add(time.Hour), g2.lease.generation()) g2.lease.invalidate() // force shard 2 onto slow path _, err := coord.LeaseReadForKey(context.Background(), []byte("zebra")) @@ -205,8 +205,8 @@ func TestShardedCoordinator_RegistersPerShardLeaderLossCallback(t *testing.T) { g1 := coord.groups[1] g2 := coord.groups[2] - g1.lease.extend(time.Now().Add(time.Hour)) - g2.lease.extend(time.Now().Add(time.Hour)) + g1.lease.extend(time.Now().Add(time.Hour), g1.lease.generation()) + g2.lease.extend(time.Now().Add(time.Hour), g2.lease.generation()) eng1.fireLeaderLoss() require.False(t, g1.lease.valid(time.Now()), From f3ba1b170560cbbdd22a63cfc3f377e18d9a7bab Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 18:01:45 +0900 Subject: [PATCH 24/43] fix(lease-read): deregister API + idempotent Txn wrapping Two MED-severity findings from gemini-code-assist on PR #549. 1. RegisterLeaderLossCallback had no unregister path, so if a Coordinator is created and discarded more often than the Engine (common in tests) the engine's callback slice grows without bound. Change the LeaseProvider interface signature to return a `deregister func()`. The etcd engine implementation pairs each registration with a sentinel pointer so the deregister closure can identify THIS specific entry even if the same fn is registered multiple times. The closure uses sync.Once so it is safe to call repeatedly. Coordinate / NewShardedCoordinator ignore the returned deregister because their lifetime matches the engine's; tests that build short-lived coordinators against a shared engine should instead retain and call the deregister on cleanup. 2. NewShardedCoordinator mutated ShardGroup.Txn in place. If called twice with the same ShardGroup (not common in production but easy to hit in tests) each wrapper would stack, firing the refresh hook multiple times per commit. Skip the wrap if g.Txn is already a *leaseRefreshingTxn. The side-effectful in-place mutation is still documented; the wrap is now idempotent so repeat calls produce a single layer. Test stub fakeLeaseEngine updated to the new deregister-returning signature; existing tests still pass. --- internal/raftengine/engine.go | 8 ++++- internal/raftengine/etcd/engine.go | 47 ++++++++++++++++++++++++++---- kv/coordinator.go | 8 ++++- kv/lease_read_test.go | 10 ++++++- kv/sharded_coordinator.go | 15 +++++++--- 5 files changed, 75 insertions(+), 13 deletions(-) diff --git a/internal/raftengine/engine.go b/internal/raftengine/engine.go index 12546df1..1a25e65c 100644 --- a/internal/raftengine/engine.go +++ b/internal/raftengine/engine.go @@ -83,7 +83,13 @@ type LeaseProvider interface { // leader-local lease they hold so the next read takes the slow path. // Multiple callbacks can be registered; each fires synchronously from // the engine's status refresh and must not block. - RegisterLeaderLossCallback(fn func()) + // + // The returned function deregisters this callback and is safe to + // call multiple times. Callers whose lifetime is shorter than the + // engine's (ephemeral Coordinators in tests, for example) MUST + // invoke the returned deregister when they are done so the engine + // does not accumulate dead callbacks. + RegisterLeaderLossCallback(fn func()) (deregister func()) } type StatusReader interface { diff --git a/internal/raftengine/etcd/engine.go b/internal/raftengine/etcd/engine.go index accc8bde..8d60c1bd 100644 --- a/internal/raftengine/etcd/engine.go +++ b/internal/raftengine/etcd/engine.go @@ -176,9 +176,12 @@ type Engine struct { // leaderLossCbsMu guards the slice of callbacks invoked when the node // transitions out of the leader role (graceful transfer, partition // step-down, shutdown). Callbacks fire synchronously from - // refreshStatus and must not block. + // refreshStatus and must not block. Each entry carries a sentinel + // pointer so that the deregister closure returned by + // RegisterLeaderLossCallback can identify THIS specific + // registration even if the same fn is registered multiple times. leaderLossCbsMu sync.Mutex - leaderLossCbs []func() + leaderLossCbs []leaderLossSlot pendingProposals map[uint64]proposalRequest pendingReads map[uint64]readRequest @@ -594,13 +597,43 @@ func (e *Engine) AppliedIndex() uint64 { // only fire once the transfer completes and state flips to follower. // Lease-read callers use this to invalidate cached lease state so the // next read takes the slow path. -func (e *Engine) RegisterLeaderLossCallback(fn func()) { +// +// The returned deregister function removes this specific registration +// and is safe to call multiple times. Long-lived callers (coordinators +// whose lifetime matches the engine's) may ignore it; shorter-lived +// callers MUST invoke it to avoid accumulating dead callbacks in the +// engine's slice. +func (e *Engine) RegisterLeaderLossCallback(fn func()) (deregister func()) { if e == nil || fn == nil { - return + return func() {} } + // Allocate a unique sentinel pointer so the deregister closure can + // identify THIS specific registration even if the same fn is + // registered multiple times. + slot := &struct{ fn func() }{fn: fn} e.leaderLossCbsMu.Lock() - e.leaderLossCbs = append(e.leaderLossCbs, fn) + e.leaderLossCbs = append(e.leaderLossCbs, leaderLossSlot{id: slot, fn: fn}) e.leaderLossCbsMu.Unlock() + var once sync.Once + return func() { + once.Do(func() { + e.leaderLossCbsMu.Lock() + defer e.leaderLossCbsMu.Unlock() + for i, c := range e.leaderLossCbs { + if c.id == slot { + e.leaderLossCbs = append(e.leaderLossCbs[:i], e.leaderLossCbs[i+1:]...) + return + } + } + }) + } +} + +// leaderLossSlot pairs a registered callback with an id-only sentinel +// pointer so deregister can distinguish identical fn values. +type leaderLossSlot struct { + id *struct{ fn func() } + fn func() } // fireLeaderLossCallbacks invokes all registered callbacks. Safe to call @@ -611,7 +644,9 @@ func (e *Engine) RegisterLeaderLossCallback(fn func()) { func (e *Engine) fireLeaderLossCallbacks() { e.leaderLossCbsMu.Lock() cbs := make([]func(), len(e.leaderLossCbs)) - copy(cbs, e.leaderLossCbs) + for i, c := range e.leaderLossCbs { + cbs[i] = c.fn + } e.leaderLossCbsMu.Unlock() for _, fn := range cbs { e.invokeLeaderLossCallback(fn) diff --git a/kv/coordinator.go b/kv/coordinator.go index 8bc05f87..8aae5dd2 100644 --- a/kv/coordinator.go +++ b/kv/coordinator.go @@ -58,8 +58,14 @@ func NewCoordinatorWithEngine(txm Transactional, engine raftengine.Engine, opts // Register a leader-loss hook so the lease is invalidated the instant // the engine notices a state transition out of the leader role, // rather than waiting for wall-clock expiry of the current lease. + // The deregister function returned by RegisterLeaderLossCallback is + // intentionally ignored because Coordinate's lifetime matches the + // engine's: the callback is released when the engine is closed. + // Tests that create short-lived coordinators against a shared + // engine should instead use RegisterLeaderLossCallback directly + // and call the returned deregister on cleanup. if lp, ok := engine.(raftengine.LeaseProvider); ok { - lp.RegisterLeaderLossCallback(c.lease.invalidate) + _ = lp.RegisterLeaderLossCallback(c.lease.invalidate) } return c } diff --git a/kv/lease_read_test.go b/kv/lease_read_test.go index 15804596..069004c1 100644 --- a/kv/lease_read_test.go +++ b/kv/lease_read_test.go @@ -49,11 +49,19 @@ func (e *fakeLeaseEngine) Propose(context.Context, []byte) (*raftengine.Proposal func (e *fakeLeaseEngine) Close() error { return nil } func (e *fakeLeaseEngine) LeaseDuration() time.Duration { return e.leaseDur } func (e *fakeLeaseEngine) AppliedIndex() uint64 { return e.applied } -func (e *fakeLeaseEngine) RegisterLeaderLossCallback(fn func()) { +func (e *fakeLeaseEngine) RegisterLeaderLossCallback(fn func()) func() { e.registerLeaderLossCalled.Add(1) e.leaderLossCallbacksMu.Lock() + idx := len(e.leaderLossCallbacks) e.leaderLossCallbacks = append(e.leaderLossCallbacks, fn) e.leaderLossCallbacksMu.Unlock() + return func() { + e.leaderLossCallbacksMu.Lock() + defer e.leaderLossCallbacksMu.Unlock() + if idx < len(e.leaderLossCallbacks) { + e.leaderLossCallbacks = append(e.leaderLossCallbacks[:idx], e.leaderLossCallbacks[idx+1:]...) + } + } } func (e *fakeLeaseEngine) fireLeaderLoss() { diff --git a/kv/sharded_coordinator.go b/kv/sharded_coordinator.go index e35a657e..1c67dee0 100644 --- a/kv/sharded_coordinator.go +++ b/kv/sharded_coordinator.go @@ -124,14 +124,21 @@ func NewShardedCoordinator(engine *distribution.Engine, groups map[uint64]*Shard // per-shard lease. All dispatch paths (raw via router.Commit, // dispatchSingleShardTxn, dispatchTxn 2PC, dispatchDelPrefix // broadcast) flow through g.Txn so this single hook catches - // them all. - g.Txn = &leaseRefreshingTxn{inner: g.Txn, g: g} + // them all. Skip the wrap if this group is already wrapped -- + // NewShardedCoordinator may be called more than once against + // the same ShardGroup in tests, and stacking wrappers would + // fire the refresh hook multiple times per commit. + if _, already := g.Txn.(*leaseRefreshingTxn); !already { + g.Txn = &leaseRefreshingTxn{inner: g.Txn, g: g} + } router.Register(gid, g.Txn, g.Store) // Per-shard leader-loss hook: when this group's engine notices // a state transition out of leader, drop the lease so the next - // LeaseReadForKey on that shard takes the slow path. + // LeaseReadForKey on that shard takes the slow path. The + // returned deregister is intentionally ignored; see the same + // rationale in NewCoordinatorWithEngine. if lp, ok := g.Engine.(raftengine.LeaseProvider); ok { - lp.RegisterLeaderLossCallback(g.lease.invalidate) + _ = lp.RegisterLeaderLossCallback(g.lease.invalidate) } } return &ShardedCoordinator{ From 83b43232cea15b72f0227471dde118cec681878e Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 18:03:23 +0900 Subject: [PATCH 25/43] fix(lease-read): short-circuit when LeaseDuration <= 0 Copilot flagged that when the tick configuration produces LeaseDuration <= 0 (already warned by Engine.Open), the code still ran the whole lease dance on every slow path: the expiry stored as `now + 0` is never after `now`, so valid() always returned false, and the next call invoked LinearizableRead + extend + invalidate for no benefit. Short-circuit in both Coordinate.LeaseRead and groupLeaseRead: if LeaseDuration() <= 0, fall back to LinearizableRead immediately without touching lease state. Test TestCoordinate_LeaseRead_FallbackWhenLeaseDurationZero verifies the short-circuit: every call runs LinearizableRead, lease.valid() stays false, no extend/invalidate is invoked. The remaining two Copilot comments (AppliedIndex RLock contention, heap alloc per extend) are already tracked as #553 / #554 and intentionally deferred from this PR. --- kv/coordinator.go | 9 ++++++++- kv/lease_read_test.go | 22 ++++++++++++++++++++++ kv/sharded_coordinator.go | 8 +++++++- 3 files changed, 37 insertions(+), 2 deletions(-) diff --git a/kv/coordinator.go b/kv/coordinator.go index 8aae5dd2..d9a3c58d 100644 --- a/kv/coordinator.go +++ b/kv/coordinator.go @@ -274,6 +274,13 @@ func (c *Coordinate) LeaseRead(ctx context.Context) (uint64, error) { if !ok { return c.LinearizableRead(ctx) } + leaseDur := lp.LeaseDuration() + if leaseDur <= 0 { + // Misconfigured tick settings (Engine.Open warned about this): + // the lease can never be valid. Fall back without touching + // lease state so we do not waste extend/invalidate work. + return c.LinearizableRead(ctx) + } // Capture time.Now() and the lease generation exactly once before // any quorum work. `now` is reused for both the fast-path validity // check and (on slow path) the extend base; `expectedGen` guards @@ -290,7 +297,7 @@ func (c *Coordinate) LeaseRead(ctx context.Context) (uint64, error) { c.lease.invalidate() return 0, err } - c.lease.extend(now.Add(lp.LeaseDuration()), expectedGen) + c.lease.extend(now.Add(leaseDur), expectedGen) return idx, nil } diff --git a/kv/lease_read_test.go b/kv/lease_read_test.go index 069004c1..34d3b9bf 100644 --- a/kv/lease_read_test.go +++ b/kv/lease_read_test.go @@ -156,6 +156,28 @@ func TestCoordinate_LeaseRead_ErrorInvalidatesLease(t *testing.T) { require.Equal(t, int32(2), eng.linearizableCalls.Load()) } +func TestCoordinate_LeaseRead_FallbackWhenLeaseDurationZero(t *testing.T) { + t.Parallel() + // Misconfigured tick settings can produce LeaseDuration <= 0. + // The implementation must short-circuit to LinearizableRead + // without touching lease state; otherwise extend(now+0, ...) would + // run on every slow-path call for no benefit. + eng := &fakeLeaseEngine{applied: 3, leaseDur: 0} + c := NewCoordinatorWithEngine(nil, eng) + + idx, err := c.LeaseRead(context.Background()) + require.NoError(t, err) + require.Equal(t, uint64(3), idx) + require.Equal(t, int32(1), eng.linearizableCalls.Load()) + require.False(t, c.lease.valid(time.Now()), + "lease must not have been extended when LeaseDuration <= 0") + + // Every subsequent call must still take the slow path. + _, err = c.LeaseRead(context.Background()) + require.NoError(t, err) + require.Equal(t, int32(2), eng.linearizableCalls.Load()) +} + func TestCoordinate_LeaseRead_FallbackWhenEngineLacksLeaseProvider(t *testing.T) { t.Parallel() eng := &nonLeaseEngine{} diff --git a/kv/sharded_coordinator.go b/kv/sharded_coordinator.go index 1c67dee0..3e2d91d1 100644 --- a/kv/sharded_coordinator.go +++ b/kv/sharded_coordinator.go @@ -718,6 +718,12 @@ func groupLeaseRead(ctx context.Context, g *ShardGroup) (uint64, error) { if !ok { return linearizableReadEngineCtx(ctx, engine) } + leaseDur := lp.LeaseDuration() + if leaseDur <= 0 { + // Lease disabled by tick configuration. Always take the slow + // path without mutating g.lease. + return linearizableReadEngineCtx(ctx, engine) + } // Single time.Now() and generation sample before any quorum work, // mirroring Coordinate.LeaseRead. expectedGen guards against a // leader-loss invalidation that fires during LinearizableRead. @@ -731,7 +737,7 @@ func groupLeaseRead(ctx context.Context, g *ShardGroup) (uint64, error) { g.lease.invalidate() return 0, err } - g.lease.extend(now.Add(lp.LeaseDuration()), expectedGen) + g.lease.extend(now.Add(leaseDur), expectedGen) return idx, nil } From 5238eead6b42ab8e0951c032b33d435a2eb08508 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 18:10:51 +0900 Subject: [PATCH 26/43] fix(lease-read): plug engine callback leak via Coordinate.Close Completes the RegisterLeaderLossCallback deregister work: the API returned a deregister closure since the previous commit, but NewCoordinatorWithEngine / NewShardedCoordinator discarded it, so a caller that built and dropped Coordinators against a shared engine still leaked the callback slot. - Coordinate now stores `deregisterLeaseCb` and exposes Close() that invokes it. Idempotent; nil-receiver safe. - ShardedCoordinator stores a per-shard slice of deregisters and Close() invokes them all. - cmd/server/demo.go defers coordinator.Close() so the callback slot is released symmetrically to construction. Close() is added only on the concrete *Coordinate / *ShardedCoordinator types, NOT on the Coordinator interface. Adapter test stubs therefore stay unchanged. Long-lived production callers that never discard the Coordinate before the engine don't need Close; short-lived owners (tests, one-shot tools) MUST call it. Regression test TestCoordinate_CloseDeregistersLeaderLossCallback verifies that after Close, a subsequent leader-loss firing does NOT invalidate this Coordinate's lease (because the callback has been removed from the engine's slice). --- cmd/server/demo.go | 7 ++++++ kv/coordinator.go | 34 +++++++++++++++++++++------ kv/lease_read_test.go | 20 ++++++++++++++++ kv/sharded_coordinator.go | 48 ++++++++++++++++++++++++--------------- 4 files changed, 84 insertions(+), 25 deletions(-) diff --git a/cmd/server/demo.go b/cmd/server/demo.go index 148aba0c..d58909c8 100644 --- a/cmd/server/demo.go +++ b/cmd/server/demo.go @@ -557,6 +557,13 @@ func run(ctx context.Context, eg *errgroup.Group, cfg config) error { engine := hashicorpraftengine.New(r) trx := kv.NewTransactionWithProposer(engine, kv.WithProposalObserver(proposalObserver)) coordinator := kv.NewCoordinatorWithEngine(trx, engine, kv.WithHLC(hlc)) + defer func() { + // Release the leader-loss callback slot on the engine before + // the process exits. The engine itself is closed elsewhere in + // the shutdown path; both orderings are safe, but releasing + // the closure here matches the symmetric construction order. + _ = coordinator.Close() + }() distEngine := distribution.NewEngineWithDefaultRoute() distCatalog := distribution.NewCatalogStore(st) if _, err := distribution.EnsureCatalogSnapshot(ctx, distCatalog, distEngine); err != nil { diff --git a/kv/coordinator.go b/kv/coordinator.go index d9a3c58d..65d1c4ca 100644 --- a/kv/coordinator.go +++ b/kv/coordinator.go @@ -58,18 +58,32 @@ func NewCoordinatorWithEngine(txm Transactional, engine raftengine.Engine, opts // Register a leader-loss hook so the lease is invalidated the instant // the engine notices a state transition out of the leader role, // rather than waiting for wall-clock expiry of the current lease. - // The deregister function returned by RegisterLeaderLossCallback is - // intentionally ignored because Coordinate's lifetime matches the - // engine's: the callback is released when the engine is closed. - // Tests that create short-lived coordinators against a shared - // engine should instead use RegisterLeaderLossCallback directly - // and call the returned deregister on cleanup. + // Keep the deregister func so Close() can release the callback + // slot; owners with a shorter lifetime than the engine (tests, + // one-shot tools) MUST call Close() to avoid leaking a closure + // pointing into this Coordinate. if lp, ok := engine.(raftengine.LeaseProvider); ok { - _ = lp.RegisterLeaderLossCallback(c.lease.invalidate) + c.deregisterLeaseCb = lp.RegisterLeaderLossCallback(c.lease.invalidate) } return c } +// Close releases any engine-side registrations (currently the +// leader-loss callback) held by this Coordinate. It is safe to call +// on a nil receiver and multiple times. Owners whose lifetime matches +// the engine's do not need to call Close; owners who discard the +// Coordinate before closing the engine MUST. +func (c *Coordinate) Close() error { + if c == nil { + return nil + } + if c.deregisterLeaseCb != nil { + c.deregisterLeaseCb() + c.deregisterLeaseCb = nil + } + return nil +} + // hlcLeaseEntryLen is the byte length of a serialised HLC lease Raft entry: // 1 tag byte + 8 bytes big-endian int64 ceiling ms. const hlcLeaseEntryLen = 9 //nolint:mnd @@ -94,6 +108,12 @@ type Coordinate struct { connCache GRPCConnCache log *slog.Logger lease leaseState + // deregisterLeaseCb removes the leader-loss callback registered + // against engine at construction. Long-lived Coordinates don't + // need to call it (the engine will be closed after them), but + // short-lived test coordinators sharing an engine MUST invoke + // Close() to release the callback slot. + deregisterLeaseCb func() } var _ Coordinator = (*Coordinate)(nil) diff --git a/kv/lease_read_test.go b/kv/lease_read_test.go index 34d3b9bf..c53bf3c2 100644 --- a/kv/lease_read_test.go +++ b/kv/lease_read_test.go @@ -197,6 +197,26 @@ func TestCoordinate_LeaseRead_FallbackWhenEngineLacksLeaseProvider(t *testing.T) // --- Leader-loss invalidation hook -------------------------------------- +func TestCoordinate_CloseDeregistersLeaderLossCallback(t *testing.T) { + t.Parallel() + eng := &fakeLeaseEngine{applied: 1, leaseDur: time.Hour} + c := NewCoordinatorWithEngine(nil, eng) + require.Equal(t, int32(1), eng.registerLeaderLossCalled.Load()) + + require.NoError(t, c.Close()) + + // After Close, firing leader-loss must NOT invoke this Coordinate's + // invalidate (it must have been removed from the engine's slice). + c.lease.extend(time.Now().Add(time.Hour), c.lease.generation()) + require.True(t, c.lease.valid(time.Now())) + eng.fireLeaderLoss() + require.True(t, c.lease.valid(time.Now()), + "Close must remove the callback so subsequent leader-loss firings do NOT touch this Coordinate's lease") + + // Close is idempotent. + require.NoError(t, c.Close()) +} + func TestCoordinate_RegistersLeaderLossCallback(t *testing.T) { t.Parallel() eng := &fakeLeaseEngine{applied: 1, leaseDur: time.Hour} diff --git a/kv/sharded_coordinator.go b/kv/sharded_coordinator.go index 3e2d91d1..7a3d1cfe 100644 --- a/kv/sharded_coordinator.go +++ b/kv/sharded_coordinator.go @@ -113,45 +113,57 @@ type ShardedCoordinator struct { clock *HLC store store.MVCCStore log *slog.Logger + // deregisterLeaseCbs removes the per-shard leader-loss callbacks + // registered at construction. See Coordinate.Close for the + // rationale. + deregisterLeaseCbs []func() } // NewShardedCoordinator builds a coordinator for the provided shard groups. // The defaultGroup is used for non-keyed leader checks. func NewShardedCoordinator(engine *distribution.Engine, groups map[uint64]*ShardGroup, defaultGroup uint64, clock *HLC, st store.MVCCStore) *ShardedCoordinator { router := NewShardRouter(engine) + var deregisters []func() for gid, g := range groups { // Wrap Txn so every successful Commit/Abort refreshes the - // per-shard lease. All dispatch paths (raw via router.Commit, - // dispatchSingleShardTxn, dispatchTxn 2PC, dispatchDelPrefix - // broadcast) flow through g.Txn so this single hook catches - // them all. Skip the wrap if this group is already wrapped -- - // NewShardedCoordinator may be called more than once against - // the same ShardGroup in tests, and stacking wrappers would - // fire the refresh hook multiple times per commit. + // per-shard lease. Skip if already wrapped so repeat calls + // don't stack wrappers. if _, already := g.Txn.(*leaseRefreshingTxn); !already { g.Txn = &leaseRefreshingTxn{inner: g.Txn, g: g} } router.Register(gid, g.Txn, g.Store) // Per-shard leader-loss hook: when this group's engine notices // a state transition out of leader, drop the lease so the next - // LeaseReadForKey on that shard takes the slow path. The - // returned deregister is intentionally ignored; see the same - // rationale in NewCoordinatorWithEngine. + // LeaseReadForKey on that shard takes the slow path. if lp, ok := g.Engine.(raftengine.LeaseProvider); ok { - _ = lp.RegisterLeaderLossCallback(g.lease.invalidate) + deregisters = append(deregisters, lp.RegisterLeaderLossCallback(g.lease.invalidate)) } } return &ShardedCoordinator{ - engine: engine, - router: router, - groups: groups, - defaultGroup: defaultGroup, - clock: clock, - store: st, - log: slog.Default(), + engine: engine, + router: router, + groups: groups, + defaultGroup: defaultGroup, + clock: clock, + store: st, + log: slog.Default(), + deregisterLeaseCbs: deregisters, } } +// Close releases per-shard engine-side registrations. Idempotent. +func (c *ShardedCoordinator) Close() error { + if c == nil { + return nil + } + cbs := c.deregisterLeaseCbs + c.deregisterLeaseCbs = nil + for _, fn := range cbs { + fn() + } + return nil +} + func (c *ShardedCoordinator) Dispatch(ctx context.Context, reqs *OperationGroup[OP]) (*CoordinateResponse, error) { if ctx == nil { ctx = context.Background() From b63054176aa7033c57c92077dfacdff16ae3b4d9 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 18:25:25 +0900 Subject: [PATCH 27/43] fix(lease-read): correct shard for DynamoDB item lease + redis base ctx Three findings from the latest review pass on PR #549. - [gemini HIGH] adapter/dynamodb.go getItem was lease-checking the TABLE metadata key's shard. In a sharded deployment the item lives on a shard keyed by its primary-key hash, which is not necessarily the same shard as the table metadata. Confirming only the metadata shard's leadership leaves the item-shard read with no linearizability guarantee. Restructured to resolve the schema (local read), compute the item key via schema.itemKeyFromAttributes, then LeaseReadForKey against that item key. Wrapped the loadSchema + itemKey computation in a resolveGetItemTarget helper to keep getItem under the cyclop complexity limit. - [gemini MED] adapter/redis.go GET was wrapping context.Background() with a timeout. Handlers are now rooted in a RedisServer-scoped baseCtx that Stop() cancels, so an in-flight GET aborts promptly on shutdown instead of running detached from the server lifecycle. The new handlerContext() helper falls back to context.Background() when the server was constructed by a test stub that bypassed NewRedisServer. - [Copilot] fakeLeaseEngine.RegisterLeaderLossCallback used the slice index as a deregister token, which silently targets the wrong slot when callbacks are removed out of order. Switched to the same unique-sentinel pattern the production etcd engine uses so tests faithfully model the contract. --- adapter/dynamodb.go | 46 +++++++++++++++++++++++++------------- adapter/redis.go | 52 ++++++++++++++++++++++++++++++++++++++----- kv/lease_read_test.go | 39 ++++++++++++++++++++++++-------- 3 files changed, 107 insertions(+), 30 deletions(-) diff --git a/adapter/dynamodb.go b/adapter/dynamodb.go index bb44d9a5..ae1ce3bd 100644 --- a/adapter/dynamodb.go +++ b/adapter/dynamodb.go @@ -1367,24 +1367,19 @@ func (d *DynamoDBServer) getItem(w http.ResponseWriter, r *http.Request) { if !ok { return } - // Use LeaseReadForKey with the table-meta key so sharded - // deployments consult the shard that actually owns the table's - // metadata rather than falling back to the default group. The - // item-specific key is not known until after schema resolution; - // the table-meta key is a deterministic proxy for the shard that - // hosts this table. - if _, err := d.coordinator.LeaseReadForKey(r.Context(), dynamoTableMetaKey(in.TableName)); err != nil { - writeDynamoError(w, http.StatusInternalServerError, dynamoErrInternal, err.Error()) - return - } readTS := d.resolveDynamoReadTS(in.ConsistentRead) - schema, exists, err := d.loadTableSchemaAt(r.Context(), in.TableName, readTS) - if err != nil { - writeDynamoError(w, http.StatusInternalServerError, dynamoErrInternal, err.Error()) + schema, itemKey, ok := d.resolveGetItemTarget(w, r, in, readTS) + if !ok { return } - if !exists { - writeDynamoError(w, http.StatusBadRequest, dynamoErrResourceNotFound, "table not found") + // Lease-check the shard that actually owns the ITEM key, not the + // table-meta key. In a sharded deployment items are routed per + // primary key, which may land on a different shard than the + // table's metadata; confirming only the table-meta shard's + // leadership would leave the item-shard read linearizability + // gap wide open. + if _, err := d.coordinator.LeaseReadForKey(r.Context(), itemKey); err != nil { + writeDynamoError(w, http.StatusInternalServerError, dynamoErrInternal, err.Error()) return } @@ -1406,6 +1401,27 @@ func (d *DynamoDBServer) getItem(w http.ResponseWriter, r *http.Request) { writeDynamoJSON(w, map[string]any{"Item": projected}) } +// resolveGetItemTarget loads the schema and computes the item key whose +// shard must be lease-checked before the read. Returns false after +// writing an error response; the caller should simply return. +func (d *DynamoDBServer) resolveGetItemTarget(w http.ResponseWriter, r *http.Request, in getItemInput, readTS uint64) (*dynamoTableSchema, []byte, bool) { + schema, exists, err := d.loadTableSchemaAt(r.Context(), in.TableName, readTS) + if err != nil { + writeDynamoError(w, http.StatusInternalServerError, dynamoErrInternal, err.Error()) + return nil, nil, false + } + if !exists { + writeDynamoError(w, http.StatusBadRequest, dynamoErrResourceNotFound, "table not found") + return nil, nil, false + } + itemKey, err := schema.itemKeyFromAttributes(in.Key) + if err != nil { + writeDynamoError(w, http.StatusBadRequest, dynamoErrValidation, err.Error()) + return nil, nil, false + } + return schema, itemKey, true +} + func (d *DynamoDBServer) deleteItem(w http.ResponseWriter, r *http.Request) { in, shouldReturnOld, err := decodeDeleteItemInput(maxDynamoBodyReader(w, r)) if err != nil { diff --git a/adapter/redis.go b/adapter/redis.go index e44735d7..aaca288b 100644 --- a/adapter/redis.go +++ b/adapter/redis.go @@ -264,6 +264,13 @@ type RedisServer struct { relayConnCache kv.GRPCConnCache requestObserver monitoring.RedisRequestObserver luaObserver monitoring.LuaScriptObserver + // baseCtx is the parent context for per-request handlers. It is + // derived from the server's lifecycle so Close() cancels all + // in-flight handlers instead of letting them run unbounded on + // context.Background(). Falls back to context.Background() if + // SetBaseContext was never called (existing test stubs). + baseCtx context.Context + baseCancel context.CancelFunc // TODO manage membership from raft log leaderRedis map[raft.ServerAddress]string @@ -359,6 +366,7 @@ func NewRedisServer(listen net.Listener, redisAddr string, store store.MVCCStore if relay == nil { relay = NewRedisPubSubRelay() } + baseCtx, baseCancel := context.WithCancel(context.Background()) r := &RedisServer{ listen: listen, store: store, @@ -371,6 +379,8 @@ func NewRedisServer(listen net.Listener, redisAddr string, store store.MVCCStore pubsub: newRedisPubSub(), scriptCache: map[string]string{}, traceCommands: os.Getenv("ELASTICKV_REDIS_TRACE") == "1", + baseCtx: baseCtx, + baseCancel: baseCancel, } r.relay.Bind(r.publishLocal) @@ -527,6 +537,31 @@ func (r *RedisServer) dispatchCommand(conn redcon.Conn, name string, handler fun } } +// handlerContext returns the base context for a request handler. +// Falls back to context.Background() when the server was constructed +// by a test stub that bypassed NewRedisServer. Handlers that need a +// deadline should wrap this via context.WithTimeout. +func (r *RedisServer) handlerContext() context.Context { + if r == nil || r.baseCtx == nil { + return context.Background() + } + return r.baseCtx +} + +// Close cancels the base context, signalling all in-flight handlers to +// abort. Idempotent. The underlying redcon listener is still owned by +// the caller; Close does NOT touch it so shutdown orchestration can +// remain with the server owner. +func (r *RedisServer) Close() error { + if r == nil { + return nil + } + if r.baseCancel != nil { + r.baseCancel() + } + return nil +} + func (r *RedisServer) Run() error { err := redcon.Serve(r.listen, func(conn redcon.Conn, cmd redcon.Command) { @@ -809,6 +844,9 @@ func (r *RedisServer) observeRedisSuccess(command string, dur time.Duration) { } func (r *RedisServer) Stop() { + // Cancel baseCtx first so in-flight handlers observe a cancelled + // context before their network connections are torn down. + _ = r.Close() _ = r.relayConnCache.Close() _ = r.listen.Close() } @@ -981,12 +1019,14 @@ func (r *RedisServer) get(conn redcon.Conn, cmd redcon.Command) { return } - // Single bounded context for the slow paths in this handler. - // Only LeaseReadForKey and keyTypeAt accept a context; - // readRedisStringAt is a local-store read that does not take one. - // The shared deadline still bounds the only branches that can - // actually block on quorum / I/O. - ctx, cancel := context.WithTimeout(context.Background(), redisDispatchTimeout) + // Single bounded context for the slow paths in this handler, + // derived from the server's base context so Close() cancels any + // in-flight handler instead of leaving it running on a detached + // context.Background(). Only LeaseReadForKey and keyTypeAt accept + // a context; readRedisStringAt is a local-store read that does + // not take one. The shared deadline bounds the only branches + // that can actually block on quorum / I/O. + ctx, cancel := context.WithTimeout(r.handlerContext(), redisDispatchTimeout) defer cancel() if _, err := r.coordinator.LeaseReadForKey(ctx, key); err != nil { conn.WriteError(err.Error()) diff --git a/kv/lease_read_test.go b/kv/lease_read_test.go index c53bf3c2..ab876991 100644 --- a/kv/lease_read_test.go +++ b/kv/lease_read_test.go @@ -21,10 +21,18 @@ type fakeLeaseEngine struct { linearizableErr error linearizableCalls atomic.Int32 leaderLossCallbacksMu sync.Mutex - leaderLossCallbacks []func() + leaderLossCallbacks []fakeLeaseEngineCb registerLeaderLossCalled atomic.Int32 } +// fakeLeaseEngineCb pairs a callback with a unique sentinel pointer so +// deregister can target THIS specific registration even when callbacks +// are removed out of order, matching the production etcd engine. +type fakeLeaseEngineCb struct { + id *struct{} + fn func() +} + func (e *fakeLeaseEngine) State() raftengine.State { return raftengine.StateLeader } func (e *fakeLeaseEngine) Leader() raftengine.LeaderInfo { return raftengine.LeaderInfo{ID: "n1", Address: "127.0.0.1:0"} @@ -51,22 +59,35 @@ func (e *fakeLeaseEngine) LeaseDuration() time.Duration { return e.leaseDur } func (e *fakeLeaseEngine) AppliedIndex() uint64 { return e.applied } func (e *fakeLeaseEngine) RegisterLeaderLossCallback(fn func()) func() { e.registerLeaderLossCalled.Add(1) + // Unique sentinel per registration so deregister can target THIS + // entry even after earlier entries were removed. Mirrors the + // production etcd engine semantics; a naive index-based remover + // would drop the wrong callback under out-of-order deregister. + slot := &struct{}{} e.leaderLossCallbacksMu.Lock() - idx := len(e.leaderLossCallbacks) - e.leaderLossCallbacks = append(e.leaderLossCallbacks, fn) + e.leaderLossCallbacks = append(e.leaderLossCallbacks, fakeLeaseEngineCb{id: slot, fn: fn}) e.leaderLossCallbacksMu.Unlock() + var once sync.Once return func() { - e.leaderLossCallbacksMu.Lock() - defer e.leaderLossCallbacksMu.Unlock() - if idx < len(e.leaderLossCallbacks) { - e.leaderLossCallbacks = append(e.leaderLossCallbacks[:idx], e.leaderLossCallbacks[idx+1:]...) - } + once.Do(func() { + e.leaderLossCallbacksMu.Lock() + defer e.leaderLossCallbacksMu.Unlock() + for i, c := range e.leaderLossCallbacks { + if c.id == slot { + e.leaderLossCallbacks = append(e.leaderLossCallbacks[:i], e.leaderLossCallbacks[i+1:]...) + return + } + } + }) } } func (e *fakeLeaseEngine) fireLeaderLoss() { e.leaderLossCallbacksMu.Lock() - cbs := append([]func(){}, e.leaderLossCallbacks...) + cbs := make([]func(), len(e.leaderLossCallbacks)) + for i, c := range e.leaderLossCallbacks { + cbs[i] = c.fn + } e.leaderLossCallbacksMu.Unlock() for _, cb := range cbs { cb() From 2c33582934ed3873a477c0329c13bff54da9df20 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 18:26:49 +0900 Subject: [PATCH 28/43] docs(lease-read): sync API + pseudocode with final signatures - LeaseProvider.RegisterLeaderLossCallback now shows the deregister return value (added in f3ba1b17) and notes when callers should retain vs ignore it. - Mention Close() on the concrete *Coordinate / *ShardedCoordinator types and explain why it is NOT on the Coordinator interface (adapter stubs stay untouched). - Pseudocode updated to include: * the LeaseDuration() <= 0 short-circuit (added in 83b43232), * the lease.generation() sample taken BEFORE the quorum op (added in b82ba323), * the corresponding two-argument extend(until, expectedGen) call. All three were pointed out by Copilot as doc-vs-code drift. --- docs/lease_read_design.md | 31 +++++++++++++++++++++---------- 1 file changed, 21 insertions(+), 10 deletions(-) diff --git a/docs/lease_read_design.md b/docs/lease_read_design.md index 0d27da5e..85e582db 100644 --- a/docs/lease_read_design.md +++ b/docs/lease_read_design.md @@ -199,7 +199,10 @@ adding an explicit term check would be redundant. type LeaseProvider interface { LeaseDuration() time.Duration AppliedIndex() uint64 - RegisterLeaderLossCallback(fn func()) + // RegisterLeaderLossCallback returns a deregister closure so + // short-lived holders can release their slot. Long-lived holders + // whose lifetime matches the engine's may ignore the return. + RegisterLeaderLossCallback(fn func()) (deregister func()) } // kv/coordinator.go @@ -208,6 +211,10 @@ type Coordinator interface { LeaseRead(ctx context.Context) (uint64, error) LeaseReadForKey(ctx context.Context, key []byte) (uint64, error) } + +// Concrete *Coordinate / *ShardedCoordinator additionally expose +// Close() which calls the stored deregister. Close is NOT on the +// Coordinator interface to keep adapter test stubs unchanged. ``` Returned index is the engine's applied index at the moment of return. Callers @@ -223,12 +230,16 @@ func (c *Coordinate) LeaseRead(ctx context.Context) (uint64, error) { if !ok { return c.LinearizableRead(ctx) // hashicorp engine, test stubs } - // Capture time.Now() exactly once. Reused for both the fast-path - // validity check and (on the slow path) the lease-extension base: - // a second sample would let the fast path accept a read whose - // instant slightly exceeds the intended expiry boundary, while - // also shortening the slow-path window by the same delta. + if d := lp.LeaseDuration(); d <= 0 { + // Misconfigured tick settings disable the lease entirely. + return c.LinearizableRead(ctx) + } + // Capture time.Now() AND lease.generation() exactly once before + // any quorum work. The generation guard prevents a leader-loss + // callback that fires during LinearizableRead from being + // silently overwritten by the post-op extend. now := time.Now() + expectedGen := c.lease.generation() if c.lease.valid(now) { return lp.AppliedIndex(), nil } @@ -238,10 +249,10 @@ func (c *Coordinate) LeaseRead(ctx context.Context) (uint64, error) { return 0, err } // `now` was sampled strictly before LinearizableRead ran, so the - // resulting lease window is strictly conservative (any real - // quorum confirmation LinearizableRead witnessed happens at or - // after `now`). extend uses a monotonic CAS to reject regressions. - c.lease.extend(now.Add(lp.LeaseDuration())) + // resulting lease window is strictly conservative. `expectedGen` + // is the pre-op generation; extend rejects the CAS if invalidate + // advanced it during the quorum operation. + c.lease.extend(now.Add(lp.LeaseDuration()), expectedGen) return idx, nil } ``` From c2185d155b434be37f37f429d2ac0b9bd1c986b7 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 18:33:13 +0900 Subject: [PATCH 29/43] fix(lease-read): root all Redis handler timeouts in server baseCtx Second pass on gemini's context.Background() finding. The previous commit only fixed GET; ~11 other handlers still built their bounded contexts from context.Background(), so in-flight dispatches/proxies would continue running past a server Stop instead of observing the cancellation. All `context.WithTimeout(context.Background(), redisDispatchTimeout)` and the one `redisRelayPublishTimeout` callsite are now rooted in `r.handlerContext()`: - set, del, delLocal, tryLeaderLogicalExists, proxyKeys, proxyTransactionToLeader, runTransaction, proxyLRange, proxyRPush, proxyLPush, proxyToLeader, tryLeaderGetAt - txnContext.commit uses `t.server.handlerContext()`. Remaining bare `context.Background()` calls are for LOCAL store reads (keyTypeAt, rawKeyTypeAt, ScanAt) that have no deadline requirement and no Raft path; wiring them through handlerContext is nice-to-have code hygiene but unrelated to the lease-read safety argument and tracked for a follow-up sweep. --- adapter/redis.go | 24 ++++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/adapter/redis.go b/adapter/redis.go index aaca288b..677d0402 100644 --- a/adapter/redis.go +++ b/adapter/redis.go @@ -986,7 +986,7 @@ func (r *RedisServer) set(conn redcon.Conn, cmd redcon.Command) { return } - ctx, cancel := context.WithTimeout(context.Background(), redisDispatchTimeout) + ctx, cancel := context.WithTimeout(r.handlerContext(), redisDispatchTimeout) defer cancel() if opts.isFastPath() && r.trySetFastPath(conn, ctx, cmd.Args[1], cmd.Args[2], opts.ttl) { @@ -1131,7 +1131,7 @@ func (r *RedisServer) tryLeaderLogicalExists(key []byte) bool { // If this path is unavailable we fall back to raw-KV probing, which is // best-effort and may lag unflushed buffer-only TTL updates. if cli, err := r.leaderClientForKey(key); err == nil { - ctx, cancel := context.WithTimeout(context.Background(), redisDispatchTimeout) + ctx, cancel := context.WithTimeout(r.handlerContext(), redisDispatchTimeout) defer cancel() if count, existsErr := cli.Exists(ctx, string(key)).Result(); existsErr == nil { return count > 0 @@ -1189,7 +1189,7 @@ func (r *RedisServer) del(conn redcon.Conn, cmd redcon.Command) { } func (r *RedisServer) delLocal(keys [][]byte) (int, error) { - ctx, cancel := context.WithTimeout(context.Background(), redisDispatchTimeout) + ctx, cancel := context.WithTimeout(r.handlerContext(), redisDispatchTimeout) defer cancel() var removed int err := r.retryRedisWrite(ctx, func() error { @@ -1505,7 +1505,7 @@ func (r *RedisServer) proxyKeys(pattern []byte) ([]string, error) { cli := r.getOrCreateLeaderClient(leaderAddr) - ctx, cancel := context.WithTimeout(context.Background(), redisDispatchTimeout) + ctx, cancel := context.WithTimeout(r.handlerContext(), redisDispatchTimeout) defer cancel() keys, err := cli.Keys(ctx, string(pattern)).Result() @@ -1574,7 +1574,7 @@ func (r *RedisServer) proxyTransactionToLeader(conn redcon.Conn, queue []redcon. } cli := r.getOrCreateLeaderClient(leaderAddr) - ctx, cancel := context.WithTimeout(context.Background(), redisDispatchTimeout) + ctx, cancel := context.WithTimeout(r.handlerContext(), redisDispatchTimeout) defer cancel() cmds, err := r.execTxPipeline(ctx, cli, queue) @@ -2376,7 +2376,7 @@ func (t *txnContext) commit() error { CommitTS: commitTS, ReadKeys: readKeys, } - ctx, cancel := context.WithTimeout(context.Background(), redisDispatchTimeout) + ctx, cancel := context.WithTimeout(t.server.handlerContext(), redisDispatchTimeout) defer cancel() if _, err := t.server.coordinator.Dispatch(ctx, group); err != nil { return errors.WithStack(err) @@ -2622,7 +2622,7 @@ func (t *txnContext) buildTTLElems() []*kv.Elem[kv.OP] { } func (r *RedisServer) runTransaction(queue []redcon.Command) ([]redisResult, error) { - dispatchCtx, cancel := context.WithTimeout(context.Background(), redisDispatchTimeout) + dispatchCtx, cancel := context.WithTimeout(r.handlerContext(), redisDispatchTimeout) defer cancel() var results []redisResult @@ -3137,7 +3137,7 @@ func (r *RedisServer) proxyLRange(key []byte, startRaw, endRaw []byte) ([]string return nil, err } - ctx, cancel := context.WithTimeout(context.Background(), redisDispatchTimeout) + ctx, cancel := context.WithTimeout(r.handlerContext(), redisDispatchTimeout) defer cancel() res, err := cli.LRange(ctx, string(key), int64(start), int64(end)).Result() @@ -3161,7 +3161,7 @@ func (r *RedisServer) proxyRPush(key []byte, values [][]byte) (int64, error) { args = append(args, string(v)) } - ctx, cancel := context.WithTimeout(context.Background(), redisDispatchTimeout) + ctx, cancel := context.WithTimeout(r.handlerContext(), redisDispatchTimeout) defer cancel() res, err := cli.RPush(ctx, string(key), args...).Result() @@ -3185,7 +3185,7 @@ func (r *RedisServer) proxyLPush(key []byte, values [][]byte) (int64, error) { args = append(args, string(v)) } - ctx, cancel := context.WithTimeout(context.Background(), redisDispatchTimeout) + ctx, cancel := context.WithTimeout(r.handlerContext(), redisDispatchTimeout) defer cancel() res, err := cli.LPush(ctx, string(key), args...).Result() @@ -3240,7 +3240,7 @@ func (r *RedisServer) proxyToLeader(conn redcon.Conn, cmd redcon.Command, key [] return true } - ctx, cancel := context.WithTimeout(context.Background(), redisDispatchTimeout) + ctx, cancel := context.WithTimeout(r.handlerContext(), redisDispatchTimeout) defer cancel() args := make([]interface{}, len(cmd.Args)) @@ -3317,7 +3317,7 @@ func (r *RedisServer) tryLeaderGetAt(key []byte, ts uint64) ([]byte, error) { return nil, errors.WithStack(err) } - ctx, cancel := context.WithTimeout(context.Background(), redisRelayPublishTimeout) + ctx, cancel := context.WithTimeout(r.handlerContext(), redisRelayPublishTimeout) defer cancel() cli := pb.NewRawKVClient(conn) From 17455daa323a4ead507a1822e5bbe49f642279a2 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 20:14:36 +0900 Subject: [PATCH 30/43] fix(lease-read): round 13 review pass Seven findings from gemini + Copilot. Six fixed here; two (AppliedIndex RLock, heap alloc) remain tracked as #553/#554. - [gemini HIGH] fireLeaderLossCallbacks now dispatches each callback on a fresh goroutine so a slow / blocking holder cannot stall the Raft engine's main loop, shutdown path, or fail() path. Ordering between callbacks is unspecified -- each callback's job is a lock-free flip (typically lease invalidation) so order is immaterial. Panic containment is unchanged. Test updated to use require.Eventually since callbacks are now async. - [gemini MED] Coordinate.Dispatch and leaseRefreshingTxn.Commit/Abort now invalidate the lease when the underlying Propose returns an error, matching the design-doc invalidation trigger. A Propose error commonly signals leader-loss (transfer in progress, non-leader rejection, quorum lost); keeping a stale lease after such a signal is the exact stale-read window the design aims to close. - [gemini MED] DynamoDB getItem: * readTS is now sampled AFTER LeaseReadForKey. Sampling earlier missed writes that completed during the lease check and violated linearizability for ConsistentRead=false reads. * LeaseReadForKey is wrapped in a dynamoLeaseReadTimeout (5 s) bounded context so a stalled Raft cannot hang the handler unbounded, even when the client never cancels. - [Copilot nit] RedisServer baseCtx struct doc no longer references a nonexistent SetBaseContext method; describes the actual lifecycle (NewRedisServer creates, Stop() cancels, fallback for test literals). - [gemini MED] Redis Stop() and etcd engine shutdown paths now log non-ErrClosed Close() errors at slog.Warn instead of swallowing them, matching the repository's general rule that network resource close errors should be visible to operators. --- adapter/dynamodb.go | 33 ++++++++++++++----- adapter/redis.go | 29 ++++++++++++---- internal/raftengine/etcd/engine.go | 33 ++++++++++++++----- .../etcd/leader_loss_callback_test.go | 12 ++++--- kv/coordinator.go | 11 ++++++- kv/sharded_coordinator.go | 5 +++ 6 files changed, 93 insertions(+), 30 deletions(-) diff --git a/adapter/dynamodb.go b/adapter/dynamodb.go index ae1ce3bd..6cfe2161 100644 --- a/adapter/dynamodb.go +++ b/adapter/dynamodb.go @@ -65,6 +65,12 @@ const ( transactRetryMaxBackoff = 10 * time.Millisecond transactRetryBackoffFactor = 2 tableCleanupAsyncTimeout = 5 * time.Minute + // dynamoLeaseReadTimeout bounds how long LeaseReadForKey's slow + // path (LinearizableRead) may block before returning an error to + // the HTTP client. Matches the order of magnitude of Redis's + // redisDispatchTimeout so both adapters give up at similar + // wall-clock budgets on quorum loss. + dynamoLeaseReadTimeout = 5 * time.Second itemUpdateLockStripeCount = 256 tableLockStripeCount = 128 batchWriteItemMaxItems = 25 @@ -1367,21 +1373,30 @@ func (d *DynamoDBServer) getItem(w http.ResponseWriter, r *http.Request) { if !ok { return } - readTS := d.resolveDynamoReadTS(in.ConsistentRead) - schema, itemKey, ok := d.resolveGetItemTarget(w, r, in, readTS) + // Tentative TS for schema resolution only; schemas change rarely + // so a slight pre-lease stale is acceptable. The item read below + // is sampled AFTER the lease check. + tentativeTS := d.resolveDynamoReadTS(in.ConsistentRead) + schema, itemKey, ok := d.resolveGetItemTarget(w, r, in, tentativeTS) if !ok { return } - // Lease-check the shard that actually owns the ITEM key, not the - // table-meta key. In a sharded deployment items are routed per - // primary key, which may land on a different shard than the - // table's metadata; confirming only the table-meta shard's - // leadership would leave the item-shard read linearizability - // gap wide open. - if _, err := d.coordinator.LeaseReadForKey(r.Context(), itemKey); err != nil { + // Lease-check the shard that actually owns the ITEM key with a + // bounded timeout so a stalled Raft cannot hang this handler + // indefinitely if the client never cancels. + leaseCtx, leaseCancel := context.WithTimeout(r.Context(), dynamoLeaseReadTimeout) + _, err := d.coordinator.LeaseReadForKey(leaseCtx, itemKey) + leaseCancel() + if err != nil { writeDynamoError(w, http.StatusInternalServerError, dynamoErrInternal, err.Error()) return } + // Re-sample readTS AFTER the lease confirmation so that any write + // that completed on the same shard BEFORE the confirmation is + // visible. Sampling earlier would violate linearizability for + // ConsistentRead=false reads by returning a snapshot from before + // the most recent confirmed commit. + readTS := d.resolveDynamoReadTS(in.ConsistentRead) current, found, err := d.readLogicalItemAt(r.Context(), schema, in.Key, readTS) if err != nil { diff --git a/adapter/redis.go b/adapter/redis.go index 677d0402..db517ed5 100644 --- a/adapter/redis.go +++ b/adapter/redis.go @@ -6,6 +6,7 @@ import ( "fmt" "io" "log" + "log/slog" "maps" "math" "net" @@ -264,11 +265,13 @@ type RedisServer struct { relayConnCache kv.GRPCConnCache requestObserver monitoring.RedisRequestObserver luaObserver monitoring.LuaScriptObserver - // baseCtx is the parent context for per-request handlers. It is - // derived from the server's lifecycle so Close() cancels all - // in-flight handlers instead of letting them run unbounded on - // context.Background(). Falls back to context.Background() if - // SetBaseContext was never called (existing test stubs). + // baseCtx is the parent context for per-request handlers. + // NewRedisServer creates a cancelable context here; Stop() cancels + // it so in-flight handlers abort promptly instead of running + // unbounded on context.Background(). Test stubs that construct + // RedisServer literals directly (bypassing NewRedisServer) may + // leave baseCtx nil; handlerContext() falls back to + // context.Background() in that case. baseCtx context.Context baseCancel context.CancelFunc // TODO manage membership from raft log @@ -847,8 +850,20 @@ func (r *RedisServer) Stop() { // Cancel baseCtx first so in-flight handlers observe a cancelled // context before their network connections are torn down. _ = r.Close() - _ = r.relayConnCache.Close() - _ = r.listen.Close() + if err := r.relayConnCache.Close(); err != nil { + slog.Warn("redis server: relay conn cache close", + slog.String("addr", r.redisAddr), + slog.Any("err", err), + ) + } + if r.listen != nil { + if err := r.listen.Close(); err != nil && !errors.Is(err, net.ErrClosed) { + slog.Warn("redis server: listener close", + slog.String("addr", r.redisAddr), + slog.Any("err", err), + ) + } + } } func (r *RedisServer) publishLocal(channel, message []byte) int64 { diff --git a/internal/raftengine/etcd/engine.go b/internal/raftengine/etcd/engine.go index 8d60c1bd..476b8384 100644 --- a/internal/raftengine/etcd/engine.go +++ b/internal/raftengine/etcd/engine.go @@ -636,11 +636,18 @@ type leaderLossSlot struct { fn func() } -// fireLeaderLossCallbacks invokes all registered callbacks. Safe to call -// from refreshStatus / shutdown while holding no engine locks; -// callbacks run synchronously and must not block. A panicking callback -// is contained so it cannot take down the raft engine loop or the -// shutdown path; the remaining callbacks still fire. +// fireLeaderLossCallbacks invokes all registered callbacks in fresh +// goroutines so a slow or misbehaving callback cannot stall the +// caller. refreshStatus runs inside the Raft engine's main loop, and +// shutdown / fail run on the teardown path; in neither case is it +// acceptable for a third-party callback to hold up progress for an +// unbounded time. Ordering between callbacks is intentionally +// unspecified -- each callback's job is to flip a lock-free flag +// (typically lease invalidation) and is order-independent. +// +// A panicking callback is still contained (see +// invokeLeaderLossCallback) so a bug in one holder cannot break +// others or crash the process. func (e *Engine) fireLeaderLossCallbacks() { e.leaderLossCbsMu.Lock() cbs := make([]func(), len(e.leaderLossCbs)) @@ -649,7 +656,7 @@ func (e *Engine) fireLeaderLossCallbacks() { } e.leaderLossCbsMu.Unlock() for _, fn := range cbs { - e.invokeLeaderLossCallback(fn) + go e.invokeLeaderLossCallback(fn) } } @@ -1802,7 +1809,12 @@ func (e *Engine) shutdown() { e.stopDispatchWorkers() e.stopSnapshotWorker() _ = closePersist(e.persist) - _ = e.transport.Close() + if err := e.transport.Close(); err != nil { + slog.Warn("etcd raft engine: transport close", + slog.String("node_id", e.localID), + slog.Any("err", err), + ) + } e.failPending(errors.WithStack(errClosed)) // LeaseProvider contract promises callbacks fire on shutdown too. // refreshStatus only fires them on the leader -> non-leader edge, @@ -1858,7 +1870,12 @@ func (e *Engine) fail(err error) { e.stopDispatchWorkers() e.stopSnapshotWorker() _ = closePersist(e.persist) - _ = e.transport.Close() + if err := e.transport.Close(); err != nil { + slog.Warn("etcd raft engine: transport close", + slog.String("node_id", e.localID), + slog.Any("err", err), + ) + } e.failPending(e.currentErrorOrClosed()) // LeaseProvider contract: fire leader-loss callbacks on shutdown if // we were leader. fail() is the error-shutdown twin of shutdown(); diff --git a/internal/raftengine/etcd/leader_loss_callback_test.go b/internal/raftengine/etcd/leader_loss_callback_test.go index fda01cb3..f622bba4 100644 --- a/internal/raftengine/etcd/leader_loss_callback_test.go +++ b/internal/raftengine/etcd/leader_loss_callback_test.go @@ -3,6 +3,7 @@ package etcd import ( "sync/atomic" "testing" + "time" "github.com/stretchr/testify/require" ) @@ -10,6 +11,8 @@ import ( // TestFireLeaderLossCallbacks_ContainsPanic verifies that a panicking // callback does NOT take down the raft engine loop: the remaining // callbacks still fire and the method returns normally. +// Callbacks now run on detached goroutines, so we wait (with a bounded +// timeout) for the two sibling counters to update before asserting. func TestFireLeaderLossCallbacks_ContainsPanic(t *testing.T) { t.Parallel() @@ -19,13 +22,12 @@ func TestFireLeaderLossCallbacks_ContainsPanic(t *testing.T) { e.RegisterLeaderLossCallback(func() { panic("lease holder bug") }) e.RegisterLeaderLossCallback(func() { after.Add(1) }) - // Must not panic out of the call. require.NotPanics(t, e.fireLeaderLossCallbacks) - require.Equal(t, int32(1), before.Load(), - "callbacks registered before the panicking one must have fired") - require.Equal(t, int32(1), after.Load(), - "callbacks registered after the panicking one must still fire") + require.Eventually(t, func() bool { + return before.Load() == 1 && after.Load() == 1 + }, time.Second, time.Millisecond, + "both non-panicking callbacks must fire on detached goroutines") } // TestFireLeaderLossCallbacks_NoCallbacksIsSafe exercises the empty-list diff --git a/kv/coordinator.go b/kv/coordinator.go index 65d1c4ca..d455d200 100644 --- a/kv/coordinator.go +++ b/kv/coordinator.go @@ -184,8 +184,17 @@ func (c *Coordinate) Dispatch(ctx context.Context, reqs *OperationGroup[OP]) (*C // transaction manager short-circuited (empty-input Commit, no-op // Abort), and refreshing would be unsound because no quorum // confirmation happened. +// +// On err != nil the lease is invalidated: a Propose error commonly +// signals leadership loss (non-leader rejection, transfer in +// progress, quorum lost, etc.) and the design doc lists +// "any error from engine.Propose" as an invalidation trigger. func (c *Coordinate) refreshLeaseAfterDispatch(resp *CoordinateResponse, err error, dispatchStart time.Time, expectedGen uint64) { - if err != nil || resp == nil || resp.CommitIndex == 0 { + if err != nil { + c.lease.invalidate() + return + } + if resp == nil || resp.CommitIndex == 0 { return } lp, ok := c.engine.(raftengine.LeaseProvider) diff --git a/kv/sharded_coordinator.go b/kv/sharded_coordinator.go index 7a3d1cfe..d95ab07a 100644 --- a/kv/sharded_coordinator.go +++ b/kv/sharded_coordinator.go @@ -46,6 +46,10 @@ func (t *leaseRefreshingTxn) Commit(reqs []*pb.Request) (*TransactionResponse, e expectedGen := t.g.lease.generation() resp, err := t.inner.Commit(reqs) if err != nil { + // Propose failures commonly signal leadership loss; follow + // the design doc and invalidate so the next read takes the + // slow path and re-verifies. + t.g.lease.invalidate() return resp, errors.WithStack(err) } t.maybeRefresh(resp, start, expectedGen) @@ -57,6 +61,7 @@ func (t *leaseRefreshingTxn) Abort(reqs []*pb.Request) (*TransactionResponse, er expectedGen := t.g.lease.generation() resp, err := t.inner.Abort(reqs) if err != nil { + t.g.lease.invalidate() return resp, errors.WithStack(err) } t.maybeRefresh(resp, start, expectedGen) From 0f8c28f1906aa9e6c7a791d455872ce3ceb4935e Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 20:18:39 +0900 Subject: [PATCH 31/43] style(dynamodb): fix gci alignment of dynamo const block --- adapter/dynamodb.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/adapter/dynamodb.go b/adapter/dynamodb.go index 6cfe2161..eca7187c 100644 --- a/adapter/dynamodb.go +++ b/adapter/dynamodb.go @@ -70,12 +70,12 @@ const ( // the HTTP client. Matches the order of magnitude of Redis's // redisDispatchTimeout so both adapters give up at similar // wall-clock budgets on quorum loss. - dynamoLeaseReadTimeout = 5 * time.Second - itemUpdateLockStripeCount = 256 - tableLockStripeCount = 128 - batchWriteItemMaxItems = 25 - transactGetItemsMaxItems = 100 - dynamoMaxRequestBodyBytes = 1 << 20 + dynamoLeaseReadTimeout = 5 * time.Second + itemUpdateLockStripeCount = 256 + tableLockStripeCount = 128 + batchWriteItemMaxItems = 25 + transactGetItemsMaxItems = 100 + dynamoMaxRequestBodyBytes = 1 << 20 dynamoTableMetaPrefix = kv.DynamoTableMetaPrefix dynamoTableGenerationPrefix = kv.DynamoTableGenerationPrefix From cc9a27d638a10bbbb081d85ab8644d2fd3012cdb Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 20:29:55 +0900 Subject: [PATCH 32/43] fix(lease-read): DynamoDB schema-gen revalidation + async callback doc - [CodeRabbit Critical] adapter/dynamodb.go getItem re-loads the table schema at the post-lease readTS and fails the request with 503 if the generation changed between the tentative-TS schema used to compute the itemKey (and pick the lease-target shard) and the readTS used for the actual item read. Without this, a table migration that commits during the lease check leaves the handler reading from the wrong shard with a stale schema; the client can retry for a fresh view. Migrations are rare so the 503 path is effectively never hit outside of that race. - [gemini] Docstrings on LeaseProvider.RegisterLeaderLossCallback (interface) and its etcd implementation now describe the async-goroutine firing semantics instead of contradicting them. Explicitly state that callers must NOT assume ordering between callbacks or that a callback has run before a follow-up read, and that the lease time-bound remains the ultimate safety net. The remaining gemini MED (heap alloc per extend) is tracked as #554. --- adapter/dynamodb.go | 23 ++++++++++++++++++++++- internal/raftengine/engine.go | 12 ++++++++++-- internal/raftengine/etcd/engine.go | 9 +++++++++ 3 files changed, 41 insertions(+), 3 deletions(-) diff --git a/adapter/dynamodb.go b/adapter/dynamodb.go index eca7187c..25d68f8f 100644 --- a/adapter/dynamodb.go +++ b/adapter/dynamodb.go @@ -1398,7 +1398,28 @@ func (d *DynamoDBServer) getItem(w http.ResponseWriter, r *http.Request) { // the most recent confirmed commit. readTS := d.resolveDynamoReadTS(in.ConsistentRead) - current, found, err := d.readLogicalItemAt(r.Context(), schema, in.Key, readTS) + // Re-verify the schema generation at readTS. If a table migration + // committed between the tentative schema load and the lease + // confirmation, the itemKey we lease-checked was computed against + // the OLD generation and may belong to a different shard than the + // one now holding the item. Rather than read stale data from the + // wrong shard, fail so the client retries with fresh schema. + finalSchema, exists, err := d.loadTableSchemaAt(r.Context(), in.TableName, readTS) + if err != nil { + writeDynamoError(w, http.StatusInternalServerError, dynamoErrInternal, err.Error()) + return + } + if !exists { + writeDynamoError(w, http.StatusBadRequest, dynamoErrResourceNotFound, "table not found") + return + } + if finalSchema.Generation != schema.Generation { + writeDynamoError(w, http.StatusServiceUnavailable, dynamoErrInternal, + "table migrated during read; please retry") + return + } + + current, found, err := d.readLogicalItemAt(r.Context(), finalSchema, in.Key, readTS) if err != nil { writeDynamoError(w, http.StatusBadRequest, dynamoErrValidation, err.Error()) return diff --git a/internal/raftengine/engine.go b/internal/raftengine/engine.go index 1a25e65c..bde05d39 100644 --- a/internal/raftengine/engine.go +++ b/internal/raftengine/engine.go @@ -81,8 +81,16 @@ type LeaseProvider interface { // local node leaves the leader role (graceful transfer, partition // step-down, or shutdown). Callers use this to invalidate any // leader-local lease they hold so the next read takes the slow path. - // Multiple callbacks can be registered; each fires synchronously from - // the engine's status refresh and must not block. + // Multiple callbacks can be registered. + // + // Callbacks are fired on detached goroutines so a slow or buggy + // holder cannot stall the engine's main loop or shutdown path. + // Ordering between callbacks is unspecified; each callback's job + // should be a lock-free flag flip (e.g. atomic invalidate). Callers + // MUST NOT assume that by the time a subsequent read happens the + // callback has already run -- it may race with the transition that + // scheduled it. The lease's time-bound remains the ultimate safety + // net. // // The returned function deregisters this callback and is safe to // call multiple times. Callers whose lifetime is shorter than the diff --git a/internal/raftengine/etcd/engine.go b/internal/raftengine/etcd/engine.go index 476b8384..0b055122 100644 --- a/internal/raftengine/etcd/engine.go +++ b/internal/raftengine/etcd/engine.go @@ -598,6 +598,15 @@ func (e *Engine) AppliedIndex() uint64 { // Lease-read callers use this to invalidate cached lease state so the // next read takes the slow path. // +// Callbacks run on detached goroutines rather than inline from +// refreshStatus / shutdown / fail, so a buggy callback cannot stall +// the Raft main loop or the teardown path. A panic inside a callback +// is contained and logged. Ordering between concurrent callbacks is +// unspecified; each should be a fast, lock-free invalidation. Callers +// MUST NOT assume a callback has already run by the time a subsequent +// read or write sees the transition; the lease's time-bound remains +// the ultimate safety net. +// // The returned deregister function removes this specific registration // and is safe to call multiple times. Long-lived callers (coordinators // whose lifetime matches the engine's) may ignore it; shorter-lived From b94ee7e0fb8bb63d1ceb047f8266f8c51e3ae46d Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 20:36:25 +0900 Subject: [PATCH 33/43] fix(lease-read): fast path also checks engine leader state Copilot (and earlier gemini) called out the same race: the lease invalidation callback now fires on a detached goroutine, so between a Leader -> non-Leader transition and the callback actually running there is a window where LeaseRead's fast path can still return AppliedIndex despite the engine already knowing it is no longer leader. The lease time-bound (700 ms with default ticks) is the backstop, but that is orders of magnitude larger than necessary. Fix (Copilot's option 2): the fast path now also checks engine.State() == StateLeader. State is refreshed every Raft tick (~10 ms), which is ~70x tighter than the time-bound. Any transition out of leader closes the fast-path window within a tick, even before the async callback runs. Keep async callback firing (Copilot's option 1 would reintroduce the original HIGH finding that a blocking callback can stall the Raft main loop / shutdown path). Defense-in-depth rather than either-or. Regression test: TestCoordinate_LeaseRead_FallbackWhenEngineNotLeader warms the lease, forces the fake engine's reported state to Follower, and asserts that LeaseRead takes the slow path (returning the LinearizableRead error) instead of the AppliedIndex fast path. Same logic applied to groupLeaseRead in ShardedCoordinator. --- kv/coordinator.go | 9 ++++++++- kv/lease_read_test.go | 34 +++++++++++++++++++++++++++++++++- kv/sharded_coordinator.go | 6 +++++- 3 files changed, 46 insertions(+), 3 deletions(-) diff --git a/kv/coordinator.go b/kv/coordinator.go index d455d200..a8e43cb6 100644 --- a/kv/coordinator.go +++ b/kv/coordinator.go @@ -318,7 +318,14 @@ func (c *Coordinate) LeaseRead(ctx context.Context) (uint64, error) { // See Coordinate.Dispatch for the same rationale. now := time.Now() expectedGen := c.lease.generation() - if c.lease.valid(now) { + // Defense-in-depth against the narrow race between an engine + // state transition out of leader and the async leader-loss + // callback flipping the lease: check the engine's current view + // too. State() is updated every Raft tick (~10 ms), which is + // tighter than the lease's time-bound. If the engine already + // knows it's not leader, force the slow path (which will fail + // fast via LinearizableRead and invalidate the lease). + if c.lease.valid(now) && c.engine.State() == raftengine.StateLeader { return lp.AppliedIndex(), nil } idx, err := c.LinearizableRead(ctx) diff --git a/kv/lease_read_test.go b/kv/lease_read_test.go index ab876991..0ab642f4 100644 --- a/kv/lease_read_test.go +++ b/kv/lease_read_test.go @@ -20,6 +20,7 @@ type fakeLeaseEngine struct { leaseDur time.Duration linearizableErr error linearizableCalls atomic.Int32 + state atomic.Value // stores raftengine.State; default Leader leaderLossCallbacksMu sync.Mutex leaderLossCallbacks []fakeLeaseEngineCb registerLeaderLossCalled atomic.Int32 @@ -33,7 +34,12 @@ type fakeLeaseEngineCb struct { fn func() } -func (e *fakeLeaseEngine) State() raftengine.State { return raftengine.StateLeader } +func (e *fakeLeaseEngine) State() raftengine.State { + if v := e.state.Load(); v != nil { + return v.(raftengine.State) //nolint:forcetypeassert + } + return raftengine.StateLeader +} func (e *fakeLeaseEngine) Leader() raftengine.LeaderInfo { return raftengine.LeaderInfo{ID: "n1", Address: "127.0.0.1:0"} } @@ -177,6 +183,32 @@ func TestCoordinate_LeaseRead_ErrorInvalidatesLease(t *testing.T) { require.Equal(t, int32(2), eng.linearizableCalls.Load()) } +func TestCoordinate_LeaseRead_FallbackWhenEngineNotLeader(t *testing.T) { + t.Parallel() + // Even with a currently-valid lease, if the engine already reports + // a non-leader state (e.g. a leader-loss transition that has not + // yet triggered the async invalidation callback), LeaseRead must + // NOT return the fast-path AppliedIndex -- it must fall through + // to LinearizableRead, which will fail fast on a non-leader. + sentinel := errors.New("not leader") + eng := &fakeLeaseEngine{applied: 7, leaseDur: time.Hour, linearizableErr: sentinel} + c := NewCoordinatorWithEngine(nil, eng) + + // Warm the lease so valid() returns true. + c.lease.extend(time.Now().Add(time.Hour), c.lease.generation()) + require.True(t, c.lease.valid(time.Now())) + + // Engine transitioned to follower (or unknown); async invalidate + // hasn't run yet. + eng.state.Store(raftengine.StateFollower) + + _, err := c.LeaseRead(context.Background()) + require.ErrorIs(t, err, sentinel, + "fast path must not hide an already-known non-leader state") + require.Equal(t, int32(1), eng.linearizableCalls.Load(), + "non-leader state must force the slow path") +} + func TestCoordinate_LeaseRead_FallbackWhenLeaseDurationZero(t *testing.T) { t.Parallel() // Misconfigured tick settings can produce LeaseDuration <= 0. diff --git a/kv/sharded_coordinator.go b/kv/sharded_coordinator.go index d95ab07a..c66f3206 100644 --- a/kv/sharded_coordinator.go +++ b/kv/sharded_coordinator.go @@ -746,7 +746,11 @@ func groupLeaseRead(ctx context.Context, g *ShardGroup) (uint64, error) { // leader-loss invalidation that fires during LinearizableRead. now := time.Now() expectedGen := g.lease.generation() - if g.lease.valid(now) { + // Defense-in-depth: also check the shard engine's current state. + // Async callbacks may not have flipped the lease yet, but + // State() is refreshed every tick and catches transitions + // sooner. See Coordinate.LeaseRead for details. + if g.lease.valid(now) && engine.State() == raftengine.StateLeader { return lp.AppliedIndex(), nil } idx, err := linearizableReadEngineCtx(ctx, engine) From d9ca3f6d796cf854f412c4f8bef221c93aa3a710 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 20:43:35 +0900 Subject: [PATCH 34/43] Potential fix for pull request finding Co-authored-by: Copilot Autofix powered by AI <175728472+Copilot@users.noreply.github.com> --- kv/sharded_coordinator.go | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/kv/sharded_coordinator.go b/kv/sharded_coordinator.go index c66f3206..eafb6f79 100644 --- a/kv/sharded_coordinator.go +++ b/kv/sharded_coordinator.go @@ -131,10 +131,12 @@ func NewShardedCoordinator(engine *distribution.Engine, groups map[uint64]*Shard var deregisters []func() for gid, g := range groups { // Wrap Txn so every successful Commit/Abort refreshes the - // per-shard lease. Skip if already wrapped so repeat calls - // don't stack wrappers. - if _, already := g.Txn.(*leaseRefreshingTxn); !already { - g.Txn = &leaseRefreshingTxn{inner: g.Txn, g: g} + // per-shard lease. Leave nil transactions unchanged, and skip + // if already wrapped so repeat calls don't stack wrappers. + if g.Txn != nil { + if _, already := g.Txn.(*leaseRefreshingTxn); !already { + g.Txn = &leaseRefreshingTxn{inner: g.Txn, g: g} + } } router.Register(gid, g.Txn, g.Store) // Per-shard leader-loss hook: when this group's engine notices From 8afa20b95853954b21403ecbb00cc702d8f3a89a Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 20:43:40 +0900 Subject: [PATCH 35/43] Potential fix for pull request finding Co-authored-by: Copilot Autofix powered by AI <175728472+Copilot@users.noreply.github.com> --- internal/raftengine/etcd/engine.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/internal/raftengine/etcd/engine.go b/internal/raftengine/etcd/engine.go index 0b055122..b3d9c678 100644 --- a/internal/raftengine/etcd/engine.go +++ b/internal/raftengine/etcd/engine.go @@ -175,9 +175,10 @@ type Engine struct { // leaderLossCbsMu guards the slice of callbacks invoked when the node // transitions out of the leader role (graceful transfer, partition - // step-down, shutdown). Callbacks fire synchronously from - // refreshStatus and must not block. Each entry carries a sentinel - // pointer so that the deregister closure returned by + // step-down, shutdown). Callbacks are started asynchronously by the + // leader-loss handling path, so callers must not assume they run inline + // with refreshStatus or complete before it returns. Each entry carries a + // sentinel pointer so that the deregister closure returned by // RegisterLeaderLossCallback can identify THIS specific // registration even if the same fn is registered multiple times. leaderLossCbsMu sync.Mutex From 1343d1983fa2b8e182bad38ae4c50926aa4fc547 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 20:48:23 +0900 Subject: [PATCH 36/43] fix(dynamodb): compare itemKey bytes (not just generation) after lease gemini's Round 15 Critical re-flagged the same race the generation comparison fixed, but proposed a strictly stronger check: compare the computed itemKey at readTS against the itemKey used for the lease check. Generation matching is a subset of itemKey matching, but the latter generalizes: any future schema change that alters item routing WITHOUT incrementing the generation (hypothetical, but possible) would still be caught. Cost is nearly identical (resolveGetItemTarget already loads the schema at readTS; we just compare the computed itemKey instead of just the Generation field) but the check is strictly more robust. --- adapter/dynamodb.go | 28 ++++++++++++---------------- 1 file changed, 12 insertions(+), 16 deletions(-) diff --git a/adapter/dynamodb.go b/adapter/dynamodb.go index 25d68f8f..f0818e7f 100644 --- a/adapter/dynamodb.go +++ b/adapter/dynamodb.go @@ -1377,7 +1377,7 @@ func (d *DynamoDBServer) getItem(w http.ResponseWriter, r *http.Request) { // so a slight pre-lease stale is acceptable. The item read below // is sampled AFTER the lease check. tentativeTS := d.resolveDynamoReadTS(in.ConsistentRead) - schema, itemKey, ok := d.resolveGetItemTarget(w, r, in, tentativeTS) + _, itemKey, ok := d.resolveGetItemTarget(w, r, in, tentativeTS) if !ok { return } @@ -1398,24 +1398,20 @@ func (d *DynamoDBServer) getItem(w http.ResponseWriter, r *http.Request) { // the most recent confirmed commit. readTS := d.resolveDynamoReadTS(in.ConsistentRead) - // Re-verify the schema generation at readTS. If a table migration - // committed between the tentative schema load and the lease - // confirmation, the itemKey we lease-checked was computed against - // the OLD generation and may belong to a different shard than the - // one now holding the item. Rather than read stale data from the - // wrong shard, fail so the client retries with fresh schema. - finalSchema, exists, err := d.loadTableSchemaAt(r.Context(), in.TableName, readTS) - if err != nil { - writeDynamoError(w, http.StatusInternalServerError, dynamoErrInternal, err.Error()) - return - } - if !exists { - writeDynamoError(w, http.StatusBadRequest, dynamoErrResourceNotFound, "table not found") + // Re-resolve schema + itemKey at readTS and verify that the key + // we lease-checked is STILL the key that will be read. A table + // migration that commits between the tentative schema load and + // the lease confirmation may shift the item to a different shard + // even if the request parameters are unchanged, so comparing the + // computed item keys (not just generation) catches any future + // schema change that alters item routing. + finalSchema, freshItemKey, ok := d.resolveGetItemTarget(w, r, in, readTS) + if !ok { return } - if finalSchema.Generation != schema.Generation { + if !bytes.Equal(freshItemKey, itemKey) { writeDynamoError(w, http.StatusServiceUnavailable, dynamoErrInternal, - "table migrated during read; please retry") + "table routing changed during read; please retry") return } From 93377bfa4e2ab13b4eec83dcf5ac8dfa2b323115 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 20:51:28 +0900 Subject: [PATCH 37/43] fix(lease-read): zero tail before truncating leaderLossCbs slice When deregistering, the previous `append(cbs[:i], cbs[i+1:]...)` pattern leaves the removed slot in the backing array at the old tail position. The slot holds a closure that typically captures a *Coordinate; that reference keeps the Coordinate (and its transitive state) alive until the engine itself is dropped, even though the caller has already called Close() expecting the lease registration to be released for GC. Replace with explicit copy + zero-the-tail + reslice so the captured reference is actually cleared. Mirror the same pattern in fakeLeaseEngine's test stub to keep it faithful to the production engine. --- internal/raftengine/etcd/engine.go | 16 +++++++++++++--- kv/lease_read_test.go | 13 ++++++++++--- 2 files changed, 23 insertions(+), 6 deletions(-) diff --git a/internal/raftengine/etcd/engine.go b/internal/raftengine/etcd/engine.go index b3d9c678..95277b33 100644 --- a/internal/raftengine/etcd/engine.go +++ b/internal/raftengine/etcd/engine.go @@ -630,10 +630,20 @@ func (e *Engine) RegisterLeaderLossCallback(fn func()) (deregister func()) { e.leaderLossCbsMu.Lock() defer e.leaderLossCbsMu.Unlock() for i, c := range e.leaderLossCbs { - if c.id == slot { - e.leaderLossCbs = append(e.leaderLossCbs[:i], e.leaderLossCbs[i+1:]...) - return + if c.id != slot { + continue } + // Remove without leaving a dangling reference at the + // tail of the underlying array. The removed slot's fn + // typically captures a *Coordinate; a plain + // `append(cbs[:i], cbs[i+1:]...)` would keep the old + // backing cell alive and prevent GC of the associated + // Coordinate until the engine itself is dropped. + last := len(e.leaderLossCbs) - 1 + copy(e.leaderLossCbs[i:], e.leaderLossCbs[i+1:]) + e.leaderLossCbs[last] = leaderLossSlot{} + e.leaderLossCbs = e.leaderLossCbs[:last] + return } }) } diff --git a/kv/lease_read_test.go b/kv/lease_read_test.go index 0ab642f4..6d545e9a 100644 --- a/kv/lease_read_test.go +++ b/kv/lease_read_test.go @@ -79,10 +79,17 @@ func (e *fakeLeaseEngine) RegisterLeaderLossCallback(fn func()) func() { e.leaderLossCallbacksMu.Lock() defer e.leaderLossCallbacksMu.Unlock() for i, c := range e.leaderLossCallbacks { - if c.id == slot { - e.leaderLossCallbacks = append(e.leaderLossCallbacks[:i], e.leaderLossCallbacks[i+1:]...) - return + if c.id != slot { + continue } + // Zero the tail before truncating so the removed + // callback's captured *Coordinate can be GC'd. + // Mirrors the production etcd engine. + last := len(e.leaderLossCallbacks) - 1 + copy(e.leaderLossCallbacks[i:], e.leaderLossCallbacks[i+1:]) + e.leaderLossCallbacks[last] = fakeLeaseEngineCb{} + e.leaderLossCallbacks = e.leaderLossCallbacks[:last] + return } }) } From 5c5a8208ad499c8d791c7cb19f056ac26972ede4 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 20:58:36 +0900 Subject: [PATCH 38/43] fix(dynamodb): defer leaseCancel for panic-safety gemini recommended defer over explicit leaseCancel() so the context is released even if LeaseReadForKey panics or a future refactor inserts an early return. The extra lifetime cost is negligible -- the only post-lease operations are local store reads -- and the safety benefit is real. --- adapter/dynamodb.go | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/adapter/dynamodb.go b/adapter/dynamodb.go index f0818e7f..3f3dd450 100644 --- a/adapter/dynamodb.go +++ b/adapter/dynamodb.go @@ -1383,11 +1383,14 @@ func (d *DynamoDBServer) getItem(w http.ResponseWriter, r *http.Request) { } // Lease-check the shard that actually owns the ITEM key with a // bounded timeout so a stalled Raft cannot hang this handler - // indefinitely if the client never cancels. + // indefinitely if the client never cancels. Use defer so the + // cancel runs even if LeaseReadForKey panics or a future + // refactor inserts an early return; the cost of keeping ctx + // alive until handler exit is negligible because the next + // in-handler calls are local store reads. leaseCtx, leaseCancel := context.WithTimeout(r.Context(), dynamoLeaseReadTimeout) - _, err := d.coordinator.LeaseReadForKey(leaseCtx, itemKey) - leaseCancel() - if err != nil { + defer leaseCancel() + if _, err := d.coordinator.LeaseReadForKey(leaseCtx, itemKey); err != nil { writeDynamoError(w, http.StatusInternalServerError, dynamoErrInternal, err.Error()) return } From f9265bf6ffbec41a55d2b300d85fb795723a07b0 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 21:14:07 +0900 Subject: [PATCH 39/43] refactor(lease-read): make lease read an optional Coordinator capability Copilot flagged: adding LeaseRead / LeaseReadForKey to the exported Coordinator interface was a breaking change for any external implementation. Reverted the interface change and moved lease reads to the optional-capability pattern already used by raftengine.LeaseProvider. - kv.Coordinator no longer requires LeaseRead / LeaseReadForKey. External coordinators that predate the lease-read feature keep compiling unchanged. - kv.LeaseReadableCoordinator is the new optional interface; concrete *Coordinate and *ShardedCoordinator implement it. - kv.LeaseReadThrough / LeaseReadForKeyThrough are helpers that type-assert and fall back to LinearizableRead when the assertion fails. - adapter/redis.go GET, adapter/redis_lua_context.go newLuaScriptContext, adapter/dynamodb.go getItem now call the helpers instead of the interface method directly. - adapter/dynamodb_test.go testCoordinatorWrapper forwards via the helpers so it keeps working without depending on Coordinator carrying the methods. No behavior change: every concrete coordinator we ship still implements the lease methods, so the fast path is unchanged. --- adapter/dynamodb.go | 2 +- adapter/dynamodb_test.go | 4 ++-- adapter/redis.go | 2 +- adapter/redis_lua_context.go | 2 +- kv/coordinator.go | 39 ++++++++++++++++++++++++++++++++++-- 5 files changed, 42 insertions(+), 7 deletions(-) diff --git a/adapter/dynamodb.go b/adapter/dynamodb.go index 3f3dd450..881f19d2 100644 --- a/adapter/dynamodb.go +++ b/adapter/dynamodb.go @@ -1390,7 +1390,7 @@ func (d *DynamoDBServer) getItem(w http.ResponseWriter, r *http.Request) { // in-handler calls are local store reads. leaseCtx, leaseCancel := context.WithTimeout(r.Context(), dynamoLeaseReadTimeout) defer leaseCancel() - if _, err := d.coordinator.LeaseReadForKey(leaseCtx, itemKey); err != nil { + if _, err := kv.LeaseReadForKeyThrough(d.coordinator, leaseCtx, itemKey); err != nil { writeDynamoError(w, http.StatusInternalServerError, dynamoErrInternal, err.Error()) return } diff --git a/adapter/dynamodb_test.go b/adapter/dynamodb_test.go index 3253518e..de9e6b97 100644 --- a/adapter/dynamodb_test.go +++ b/adapter/dynamodb_test.go @@ -1861,9 +1861,9 @@ func (w *testCoordinatorWrapper) LinearizableRead(ctx context.Context) (uint64, } func (w *testCoordinatorWrapper) LeaseRead(ctx context.Context) (uint64, error) { - return w.inner.LeaseRead(ctx) + return kv.LeaseReadThrough(w.inner, ctx) } func (w *testCoordinatorWrapper) LeaseReadForKey(ctx context.Context, key []byte) (uint64, error) { - return w.inner.LeaseReadForKey(ctx, key) + return kv.LeaseReadForKeyThrough(w.inner, ctx, key) } diff --git a/adapter/redis.go b/adapter/redis.go index db517ed5..c32fd4d2 100644 --- a/adapter/redis.go +++ b/adapter/redis.go @@ -1043,7 +1043,7 @@ func (r *RedisServer) get(conn redcon.Conn, cmd redcon.Command) { // that can actually block on quorum / I/O. ctx, cancel := context.WithTimeout(r.handlerContext(), redisDispatchTimeout) defer cancel() - if _, err := r.coordinator.LeaseReadForKey(ctx, key); err != nil { + if _, err := kv.LeaseReadForKeyThrough(r.coordinator, ctx, key); err != nil { conn.WriteError(err.Error()) return } diff --git a/adapter/redis_lua_context.go b/adapter/redis_lua_context.go index ad6d63cd..12caa589 100644 --- a/adapter/redis_lua_context.go +++ b/adapter/redis_lua_context.go @@ -218,7 +218,7 @@ func newLuaScriptContext(ctx context.Context, server *RedisServer) (*luaScriptCo // All subsequent reads within the script use snapshotGetAt at startTS, // so leadership is verified at most once per script and amortised across // scripts via the lease. - if _, err := server.coordinator.LeaseRead(ctx); err != nil { + if _, err := kv.LeaseReadThrough(server.coordinator, ctx); err != nil { return nil, errors.WithStack(err) } startTS := server.readTS() diff --git a/kv/coordinator.go b/kv/coordinator.go index a8e43cb6..fa4033a6 100644 --- a/kv/coordinator.go +++ b/kv/coordinator.go @@ -123,15 +123,50 @@ type Coordinator interface { IsLeader() bool VerifyLeader() error LinearizableRead(ctx context.Context) (uint64, error) - LeaseRead(ctx context.Context) (uint64, error) RaftLeader() raft.ServerAddress IsLeaderForKey(key []byte) bool VerifyLeaderForKey(key []byte) error - LeaseReadForKey(ctx context.Context, key []byte) (uint64, error) RaftLeaderForKey(key []byte) raft.ServerAddress Clock() *HLC } +// LeaseReadableCoordinator is the optional capability implemented by +// coordinators that participate in the leader-local lease read path +// (see docs/lease_read_design.md). Callers that want lease reads +// should type-assert to this interface and fall back to +// LinearizableRead when the assertion fails, following the same +// pattern as raftengine.LeaseProvider. Keeping the lease methods OFF +// the Coordinator interface avoids breaking existing external +// implementations that predate the lease-read feature. +type LeaseReadableCoordinator interface { + LeaseRead(ctx context.Context) (uint64, error) + LeaseReadForKey(ctx context.Context, key []byte) (uint64, error) +} + +// LeaseReadThrough is a helper that calls LeaseRead when the +// coordinator supports it, falling back to LinearizableRead otherwise. +// Adapter call sites use this so they don't have to repeat the +// type-assertion dance. +func LeaseReadThrough(c Coordinator, ctx context.Context) (uint64, error) { + if lr, ok := c.(LeaseReadableCoordinator); ok { + idx, err := lr.LeaseRead(ctx) + return idx, errors.WithStack(err) + } + idx, err := c.LinearizableRead(ctx) + return idx, errors.WithStack(err) +} + +// LeaseReadForKeyThrough is the key-routed counterpart of +// LeaseReadThrough. +func LeaseReadForKeyThrough(c Coordinator, ctx context.Context, key []byte) (uint64, error) { + if lr, ok := c.(LeaseReadableCoordinator); ok { + idx, err := lr.LeaseReadForKey(ctx, key) + return idx, errors.WithStack(err) + } + idx, err := c.LinearizableRead(ctx) + return idx, errors.WithStack(err) +} + func (c *Coordinate) Dispatch(ctx context.Context, reqs *OperationGroup[OP]) (*CoordinateResponse, error) { if ctx == nil { ctx = context.Background() From 9cea2a8b3e08f39bb76ed6f625f15b1d4972f91d Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 21:23:21 +0900 Subject: [PATCH 40/43] perf(lease-read): make AppliedIndex lock-free via atomic.Uint64 Copilot / gemini repeatedly flagged that AppliedIndex() on the lease- read fast path took e.mu.RLock(), contending with refreshStatus's writer lock ~100 Hz at the current tick interval. Tracked as #553 as a deferred item; now addressed in-PR since it affects the motivating hot path (Lua script entry + Redis/DynamoDB GET) directly. - Added e.appliedIndex atomic.Uint64 mirroring e.applied. - setApplied(index) helper advances both fields from the single- writer Raft run loop; callers (applyCommitted entries, snapshot restore, Open init) go through it. - AppliedIndex() now returns e.appliedIndex.Load() with no lock. - Nil-receiver guard preserved. Tests: - TestAppliedIndex_LockFreeLoad holds e.mu exclusively and calls AppliedIndex(), asserting the caller does not block (would deadlock if RLock were still used). - TestAppliedIndex_NilReceiver covers the nil guard. All existing tests pass. Closes #553. --- internal/raftengine/etcd/engine.go | 41 ++++++++++++++----- .../etcd/leader_loss_callback_test.go | 27 ++++++++++++ 2 files changed, 58 insertions(+), 10 deletions(-) diff --git a/internal/raftengine/etcd/engine.go b/internal/raftengine/etcd/engine.go index 95277b33..74e7d2a6 100644 --- a/internal/raftengine/etcd/engine.go +++ b/internal/raftengine/etcd/engine.go @@ -159,6 +159,13 @@ type Engine struct { runErr error closed bool applied uint64 + // appliedIndex mirrors the current applied-entry index for + // lock-free readers on the lease-read fast path. Writers inside + // the Raft run loop update both `applied` (protected by the run + // loop's single-writer invariant) and `appliedIndex.Store(...)`. + // AppliedIndex() reads via atomic.Load so it does not contend + // with refreshStatus's write lock. + appliedIndex atomic.Uint64 // configIndex tracks the highest configuration index durably published to // local raft snapshot state and peer metadata. configIndex atomic.Uint64 @@ -334,6 +341,7 @@ func Open(ctx context.Context, cfg OpenConfig) (*Engine, error) { pendingConfigs: map[uint64]adminRequest{}, } engine.configIndex.Store(maxAppliedIndex(prepared.disk.LocalSnap)) + engine.appliedIndex.Store(maxAppliedIndex(prepared.disk.LocalSnap)) engine.initTransport(prepared.cfg) engine.initSnapshotWorker() engine.refreshStatus() @@ -578,16 +586,19 @@ func (e *Engine) LeaseDuration() time.Duration { } // AppliedIndex returns the highest log index applied to the local FSM. -// Suitable for callers that need a non-blocking read fence equivalent to -// what LinearizableRead would have returned, paired with an external -// quorum confirmation (e.g. a valid lease). +// Suitable for callers that need a non-blocking read fence equivalent +// to what LinearizableRead would have returned, paired with an +// external quorum confirmation (e.g. a valid lease). +// +// Lock-free: reads the mirrored atomic.Uint64 written by the run +// loop's apply path (and by Restore's snapshot installation), so the +// lease-read fast path does not contend with refreshStatus's write +// lock under high read concurrency. func (e *Engine) AppliedIndex() uint64 { if e == nil { return 0 } - e.mu.RLock() - defer e.mu.RUnlock() - return e.status.AppliedIndex + return e.appliedIndex.Load() } // RegisterLeaderLossCallback registers fn to fire every time the local @@ -1301,6 +1312,7 @@ func (e *Engine) applyReadySnapshot(snapshot raftpb.Snapshot) error { snapshot.Metadata.Index, snapshot.Metadata.Term) } e.applied = snapshot.Metadata.Index + e.appliedIndex.Store(snapshot.Metadata.Index) e.setConfigurationFromConfState(snapshot.Metadata.ConfState, snapshot.Metadata.Index) return nil } @@ -1371,7 +1383,7 @@ func (e *Engine) applyCommitted(entries []raftpb.Entry) error { switch entry.Type { case raftpb.EntryNormal: response := e.applyNormalEntry(entry) - e.applied = entry.Index + e.setApplied(entry.Index) e.resolveProposal(entry.Index, entry.Data, response) case raftpb.EntryConfChange: var cc raftpb.ConfChange @@ -1384,7 +1396,7 @@ func (e *Engine) applyCommitted(entries []raftpb.Entry) error { return err } e.applyConfigChange(cc.Type, cc.NodeID, cc.Context, entry.Index) - e.applied = entry.Index + e.setApplied(entry.Index) case raftpb.EntryConfChangeV2: var cc raftpb.ConfChangeV2 if err := cc.Unmarshal(entry.Data); err != nil { @@ -1396,14 +1408,23 @@ func (e *Engine) applyCommitted(entries []raftpb.Entry) error { return err } e.applyConfigChangeV2(cc, entry.Index) - e.applied = entry.Index + e.setApplied(entry.Index) default: - e.applied = entry.Index + e.setApplied(entry.Index) } } return nil } +// setApplied advances both the run-loop-owned `applied` field and the +// lock-free atomic mirror in a single place. Called exclusively from +// the Raft run loop, so no synchronization between the two writes is +// required beyond the single-writer invariant. +func (e *Engine) setApplied(index uint64) { + e.applied = index + e.appliedIndex.Store(index) +} + func (e *Engine) applyNormalEntry(entry raftpb.Entry) any { if len(entry.Data) == 0 { return nil diff --git a/internal/raftengine/etcd/leader_loss_callback_test.go b/internal/raftengine/etcd/leader_loss_callback_test.go index f622bba4..152175f2 100644 --- a/internal/raftengine/etcd/leader_loss_callback_test.go +++ b/internal/raftengine/etcd/leader_loss_callback_test.go @@ -38,3 +38,30 @@ func TestFireLeaderLossCallbacks_NoCallbacksIsSafe(t *testing.T) { e := &Engine{} require.NotPanics(t, e.fireLeaderLossCallbacks) } + +// TestAppliedIndex_LockFreeLoad confirms that AppliedIndex() reads the +// atomic mirror and does NOT acquire the engine's read-lock. +// Acquiring e.mu for write before calling AppliedIndex would deadlock +// if it were still RLock-based; the atomic path must return +// immediately regardless of lock state. +func TestAppliedIndex_LockFreeLoad(t *testing.T) { + t.Parallel() + e := &Engine{} + e.appliedIndex.Store(42) + + // Hold the engine mutex exclusively. The atomic reader must not + // block on this. + e.mu.Lock() + defer e.mu.Unlock() + + got := e.AppliedIndex() + require.Equal(t, uint64(42), got) +} + +// TestAppliedIndex_NilReceiver mirrors the other lease-related +// nil-receiver guards. +func TestAppliedIndex_NilReceiver(t *testing.T) { + t.Parallel() + var e *Engine + require.Equal(t, uint64(0), e.AppliedIndex()) +} From b70f1679f5ace51b3590085eaf1e2a589a69bcc2 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 21:25:23 +0900 Subject: [PATCH 41/43] revert(lease-read): fire leader-loss callbacks synchronously gemini flagged the unbounded-goroutine concern: spawning a new goroutine per registered callback on every leader-loss event scales linearly with the number of shards/coordinators and has no backpressure. Since the callback contract already requires non-blocking work (a lock-free lease-invalidate flag flip), and LeaseRead's fast path already guards on engine.State() == StateLeader to close the narrow window between a transition and the callback completing, sync execution is both safe and simpler: - fireLeaderLossCallbacks iterates inline. - invokeLeaderLossCallback still defer-recovers panics and logs, so one buggy holder cannot break siblings. - Docstrings (both the interface in raftengine/engine.go and the etcd-impl method) now describe the synchronous contract and the State() guard that makes it safe despite no async backstop. If a blocking callback ever causes a real problem, we'll revisit with a bounded worker pool; for now the simpler path stays. Test TestFireLeaderLossCallbacks_ContainsPanic reverted to direct counter assertions (no Eventually wait needed in sync mode). --- internal/raftengine/engine.go | 16 ++++----- internal/raftengine/etcd/engine.go | 33 +++++++++---------- .../etcd/leader_loss_callback_test.go | 13 +++----- 3 files changed, 28 insertions(+), 34 deletions(-) diff --git a/internal/raftengine/engine.go b/internal/raftengine/engine.go index bde05d39..e97afd6a 100644 --- a/internal/raftengine/engine.go +++ b/internal/raftengine/engine.go @@ -83,14 +83,14 @@ type LeaseProvider interface { // leader-local lease they hold so the next read takes the slow path. // Multiple callbacks can be registered. // - // Callbacks are fired on detached goroutines so a slow or buggy - // holder cannot stall the engine's main loop or shutdown path. - // Ordering between callbacks is unspecified; each callback's job - // should be a lock-free flag flip (e.g. atomic invalidate). Callers - // MUST NOT assume that by the time a subsequent read happens the - // callback has already run -- it may race with the transition that - // scheduled it. The lease's time-bound remains the ultimate safety - // net. + // Callbacks fire synchronously from the engine's status-refresh + // / shutdown path and MUST be non-blocking -- each should be a + // lock-free flag flip (e.g. atomic invalidate). A panicking + // callback is contained so a bug in one holder cannot break + // others, but a blocking callback would stall the engine's main + // loop, so the contract is strict. Lease-read fast paths also + // guard on engine.State() to close the narrow race between a + // transition and this callback completing. // // The returned function deregisters this callback and is safe to // call multiple times. Callers whose lifetime is shorter than the diff --git a/internal/raftengine/etcd/engine.go b/internal/raftengine/etcd/engine.go index 74e7d2a6..97bd97a7 100644 --- a/internal/raftengine/etcd/engine.go +++ b/internal/raftengine/etcd/engine.go @@ -610,14 +610,13 @@ func (e *Engine) AppliedIndex() uint64 { // Lease-read callers use this to invalidate cached lease state so the // next read takes the slow path. // -// Callbacks run on detached goroutines rather than inline from -// refreshStatus / shutdown / fail, so a buggy callback cannot stall -// the Raft main loop or the teardown path. A panic inside a callback -// is contained and logged. Ordering between concurrent callbacks is -// unspecified; each should be a fast, lock-free invalidation. Callers -// MUST NOT assume a callback has already run by the time a subsequent -// read or write sees the transition; the lease's time-bound remains -// the ultimate safety net. +// Callbacks run synchronously from refreshStatus / shutdown / fail +// and MUST be non-blocking (each should be a fast, lock-free +// invalidation). A panic inside a callback is contained and logged +// so a bug in one holder cannot crash the engine or break other +// callbacks. LeaseRead also guards its fast path on +// engine.State() == StateLeader so the small window between the +// transition and this callback completing cannot serve stale reads. // // The returned deregister function removes this specific registration // and is safe to call multiple times. Long-lived callers (coordinators @@ -667,18 +666,16 @@ type leaderLossSlot struct { fn func() } -// fireLeaderLossCallbacks invokes all registered callbacks in fresh -// goroutines so a slow or misbehaving callback cannot stall the -// caller. refreshStatus runs inside the Raft engine's main loop, and -// shutdown / fail run on the teardown path; in neither case is it -// acceptable for a third-party callback to hold up progress for an -// unbounded time. Ordering between callbacks is intentionally -// unspecified -- each callback's job is to flip a lock-free flag -// (typically lease invalidation) and is order-independent. +// fireLeaderLossCallbacks invokes all registered callbacks +// synchronously. The registered-callback contract requires each fn +// to be non-blocking (a lock-free lease-invalidate flag flip), so +// inline execution is safe and avoids spawning an unbounded number +// of goroutines per leader-loss event when many shards / coordinators +// are registered. // // A panicking callback is still contained (see // invokeLeaderLossCallback) so a bug in one holder cannot break -// others or crash the process. +// subsequent callbacks or crash the process. func (e *Engine) fireLeaderLossCallbacks() { e.leaderLossCbsMu.Lock() cbs := make([]func(), len(e.leaderLossCbs)) @@ -687,7 +684,7 @@ func (e *Engine) fireLeaderLossCallbacks() { } e.leaderLossCbsMu.Unlock() for _, fn := range cbs { - go e.invokeLeaderLossCallback(fn) + e.invokeLeaderLossCallback(fn) } } diff --git a/internal/raftengine/etcd/leader_loss_callback_test.go b/internal/raftengine/etcd/leader_loss_callback_test.go index 152175f2..7db6029f 100644 --- a/internal/raftengine/etcd/leader_loss_callback_test.go +++ b/internal/raftengine/etcd/leader_loss_callback_test.go @@ -3,16 +3,13 @@ package etcd import ( "sync/atomic" "testing" - "time" "github.com/stretchr/testify/require" ) // TestFireLeaderLossCallbacks_ContainsPanic verifies that a panicking // callback does NOT take down the raft engine loop: the remaining -// callbacks still fire and the method returns normally. -// Callbacks now run on detached goroutines, so we wait (with a bounded -// timeout) for the two sibling counters to update before asserting. +// callbacks still fire synchronously and the method returns normally. func TestFireLeaderLossCallbacks_ContainsPanic(t *testing.T) { t.Parallel() @@ -24,10 +21,10 @@ func TestFireLeaderLossCallbacks_ContainsPanic(t *testing.T) { require.NotPanics(t, e.fireLeaderLossCallbacks) - require.Eventually(t, func() bool { - return before.Load() == 1 && after.Load() == 1 - }, time.Second, time.Millisecond, - "both non-panicking callbacks must fire on detached goroutines") + require.Equal(t, int32(1), before.Load(), + "callbacks registered before the panicking one must have fired") + require.Equal(t, int32(1), after.Load(), + "callbacks registered after the panicking one must still fire") } // TestFireLeaderLossCallbacks_NoCallbacksIsSafe exercises the empty-list From d1d255f566ebbedf07563081f6a7e633af0723eb Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 21:33:12 +0900 Subject: [PATCH 42/43] Potential fix for pull request finding Co-authored-by: Copilot Autofix powered by AI <175728472+Copilot@users.noreply.github.com> --- internal/raftengine/etcd/engine.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/internal/raftengine/etcd/engine.go b/internal/raftengine/etcd/engine.go index 97bd97a7..9275ccc6 100644 --- a/internal/raftengine/etcd/engine.go +++ b/internal/raftengine/etcd/engine.go @@ -182,11 +182,11 @@ type Engine struct { // leaderLossCbsMu guards the slice of callbacks invoked when the node // transitions out of the leader role (graceful transfer, partition - // step-down, shutdown). Callbacks are started asynchronously by the - // leader-loss handling path, so callers must not assume they run inline - // with refreshStatus or complete before it returns. Each entry carries a - // sentinel pointer so that the deregister closure returned by - // RegisterLeaderLossCallback can identify THIS specific + // step-down, shutdown). Callbacks are invoked synchronously by the + // leader-loss handling path, so callers should assume they run inline + // with refreshStatus and may block it from returning until they finish. + // Each entry carries a sentinel pointer so that the deregister closure + // returned by RegisterLeaderLossCallback can identify THIS specific // registration even if the same fn is registered multiple times. leaderLossCbsMu sync.Mutex leaderLossCbs []leaderLossSlot From c087a7c2ff94bd8dfeeaeebf769eea142fbdc887 Mon Sep 17 00:00:00 2001 From: "Yoshiaki Ueda (bootjp)" Date: Mon, 20 Apr 2026 21:37:45 +0900 Subject: [PATCH 43/43] fix(lease-read): pin readTS in DynamoDB getItem + sync doc for leaderLossCbs Two findings from the latest review pass. - [CodeRabbit Major] adapter/dynamodb.go getItem now calls pinReadTS(readTS) / defer Release so concurrent MVCC GC cannot reclaim versions between the schema revalidation and the item read. queryItems / scanItems / transactGetItems already pin; this brings getItem in line after the pre/post-lease readTS split made the window observable. - [gemini MED] internal/raftengine/etcd/engine.go leaderLossCbsMu field comment still described asynchronous firing; updated to match the now-synchronous implementation (matches RegisterLeaderLossCallback's docstring and the engine.go interface comment). Remaining comments (heap alloc, async goroutine) reference already-reverted or already-tracked state. --- adapter/dynamodb.go | 6 ++++++ internal/raftengine/etcd/engine.go | 14 ++++++++------ 2 files changed, 14 insertions(+), 6 deletions(-) diff --git a/adapter/dynamodb.go b/adapter/dynamodb.go index 881f19d2..59a97359 100644 --- a/adapter/dynamodb.go +++ b/adapter/dynamodb.go @@ -1400,6 +1400,12 @@ func (d *DynamoDBServer) getItem(w http.ResponseWriter, r *http.Request) { // ConsistentRead=false reads by returning a snapshot from before // the most recent confirmed commit. readTS := d.resolveDynamoReadTS(in.ConsistentRead) + // Pin readTS so concurrent MVCC GC cannot reclaim versions + // between the schema revalidation and the item read below; + // matches the pattern already used by queryItems / scanItems / + // transactGetItems. + readPin := d.pinReadTS(readTS) + defer readPin.Release() // Re-resolve schema + itemKey at readTS and verify that the key // we lease-checked is STILL the key that will be read. A table diff --git a/internal/raftengine/etcd/engine.go b/internal/raftengine/etcd/engine.go index 9275ccc6..fddfb9c4 100644 --- a/internal/raftengine/etcd/engine.go +++ b/internal/raftengine/etcd/engine.go @@ -182,12 +182,14 @@ type Engine struct { // leaderLossCbsMu guards the slice of callbacks invoked when the node // transitions out of the leader role (graceful transfer, partition - // step-down, shutdown). Callbacks are invoked synchronously by the - // leader-loss handling path, so callers should assume they run inline - // with refreshStatus and may block it from returning until they finish. - // Each entry carries a sentinel pointer so that the deregister closure - // returned by RegisterLeaderLossCallback can identify THIS specific - // registration even if the same fn is registered multiple times. + // step-down, shutdown). Callbacks fire synchronously from the + // leader-loss handling path and MUST be non-blocking; a slow + // callback would hold up refreshStatus / shutdown / fail. See + // RegisterLeaderLossCallback for the full contract. Each entry + // carries a sentinel pointer so that the deregister closure + // returned by RegisterLeaderLossCallback can identify THIS + // specific registration even if the same fn is registered + // multiple times. leaderLossCbsMu sync.Mutex leaderLossCbs []leaderLossSlot