Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

storage: sideloaded Raft entries don't play well with Raft memory limits #31618

Closed
nvanbenschoten opened this issue Oct 18, 2018 · 20 comments
Closed
Assignees
Labels
A-kv-replication Relating to Raft, consensus, and coordination. C-bug Code not up to spec/doc, specs & docs deemed correct. Solution expected to change code/behavior.

Comments

@nvanbenschoten
Copy link
Member

etcd/raft has a number of memory limits that it uses to avoid pulling too many entries into memory at once, sending messages that are too large, and applying batches of entries that are too large. These limits work by looking at the size reported by raftpb.Entry.Size. This doesn't play well with sideloaded entries, which at any given time may or may not be inlined. Entries may flow into etcd/raft (proposals, MsgApps, etc.) while inlined and exit it non-inlined (MsgApps, CommittedEntries, etc.). This breaks a few safeguards:

  1. the size limit placed on MsgApps will be inaccurate and we may exceed it
  2. the size limit placed on CommittedEntries will be inaccurate and we may exceed it
  3. internal tracking of uncommitted entry size is most likely broken, which can lead to leaked uncommitted allowance and stalled proposals

I think the fix here is to ensure that etcd/raft always sees inlined entries. I think that's already the case in most code-paths, but not in all. I'm sure I'm also missing a lot of context about the history of this and the decisions that have been made.

cc. @tschottdorf

@nvanbenschoten nvanbenschoten added C-bug Code not up to spec/doc, specs & docs deemed correct. Solution expected to change code/behavior. A-kv-replication Relating to Raft, consensus, and coordination. labels Oct 18, 2018
@tbg
Copy link
Member

tbg commented Oct 19, 2018

Raft should never see "thin" entries and so should correctly account for the inlined payload. We call maybeSideloadEntriesRaftMuLocked in the two locations that append to the persisted raft log (regular and snapshot path), but Raft should never see the thinned versions.

On the other hand, the method that loads persisted entries back into memory (Entries) always returns "fat" proposals. It doesn't do that if you pass a nil sideloaded storage, but that should only ever happen when called as part of looking up the term (where you don't ever look at the proposal).

Unless there's a bug that results in a nil storage here:

r.raftMu.sideloaded, lo, hi, maxBytes)

but that seems unlikely.

I just caught up on the discussion on Slack that you had on this in which you posted this message:

appending new entries to log would exceed uncommitted entry size limit; dropping proposal

from Raft. Can't this happen naturally without any sideloading accounting problems because it does all-or-nothing across multiple entries?

https://github.com/cockroachdb/vendored/blob/ca4276c48c28ef08550bc0014dd021a0081e9278/go.etcd.io/etcd/raft/raft.go#L973-L976

Assume you can have 100 bytes of uncommitted log. The replica proposes two proposals, 70 and 40 bytes. But both get dropped because the local Raft instance doesn't know the leader. After a few ticks, the replica proposes both again. Is it possible that both proposals reliably end up in the same m.Entries which then gets discarded every time it shows up?

Hmm, looks like the answer is "no, at least not in our code". We put each proposal in individually. But this still looks good to fix (is it possible to propose multiple entries at once?)

Back to the bug at hand, what are our safeguards against putting in a proposal that's simply getting dropped due to size? Wouldn't that just be reproposed and dropped, over and over again?

https://github.com/cockroachdb/cockroach/blob/master/pkg/storage/replica.go#L3805-L3812

@tbg tbg self-assigned this Oct 19, 2018
@tbg
Copy link
Member

tbg commented Oct 19, 2018

If this is the case, there are two choices: a) we make sure we detect this condition or prevent it (i.e. make sure these proposals are never that big) or b) we tweak semantics so that the limiting doesn't apply to the first uncommitted entry (after all, we're doing this to make sure there isn't a long log, not to make sure there isn't a single thing in it).

But this seems like such an obvious bug that I'm afraid I might be missing something.

@tbg
Copy link
Member

tbg commented Oct 19, 2018

Hmm, yeah, I'm definitely missing something. On a local 3 node cluster, I would otherwise expect to not be able to insert a 12mb blob, but that just works.

Ah, I'm seeing that it already does b) from above:

https://github.com/cockroachdb/vendored/blob/ca4276c48c28ef08550bc0014dd021a0081e9278/go.etcd.io/etcd/raft/raft.go#L1496-L1503

Ok, you must be right about there being something about sideloading. I've got to page this back in but this snippet strongly suggests that the committed entries are "thin", which would indeed confuse Raft's internal tracking:

for _, e := range rd.CommittedEntries {
switch e.Type {
case raftpb.EntryNormal:
// Committed entries come straight from the Raft log. Consequently,
// sideloaded SSTables are not usually inlined.
if newEnt, err := maybeInlineSideloadedRaftCommand(
ctx, r.RangeID, e, r.raftMu.sideloaded, r.store.raftEntryCache,
); err != nil {
const expl = "maybeInlineSideloadedRaftCommand"
return stats, expl, errors.Wrap(err, expl)
} else if newEnt != nil {
e = *newEnt
}

@tbg
Copy link
Member

tbg commented Oct 19, 2018

Wait a minute... I don't see why this needs to sideload. The proposals should already have been inlined. Is this just an inefficiency that has gone overlooked? I've commented this out and all the sideloading tests still pass.

tbg added a commit to tbg/cockroach that referenced this issue Oct 19, 2018
Entries are "thinned" only when passed to `r.append()` (i.e. written
to disk) and they are always returned "fat" from `Entries()` (i.e.
Raft's way to get entries from disk). Consequently Raft never sees
thin entries and won't ask us to commit them.

Touches cockroachdb#31618.

Release note: None
tbg added a commit to tbg/cockroach that referenced this issue Oct 19, 2018
If this were passed as nil, we'd be returning "thin" (i.e. with
sideloading payloads not inlined) Entries. This isn't supposed
to happen, but check it.

See:
cockroachdb#31618 (comment).

Release note: None
@benesch
Copy link
Contributor

benesch commented Oct 19, 2018

Hah, yeah, I did that too last night. I can’t see any reason why you’d need to fatten entries anywhere other than entries and the snapshot send path.

@benesch
Copy link
Contributor

benesch commented Oct 19, 2018

Finally caught a repro. Here's /debug/requests:

13:53:04.549669 | 2.991626 | ... [n9,s9,r163/9:/Table/53/1/{5/1/-1…-9/1/-2…}] re-submitting command 33416f1483cfa380 to Raft: reasonTicks
-- | -- | --
13:53:04.558199 | .  8530 | ... [n9,s9,r163/9:/Table/53/1/{5/1/-1…-9/1/-2…}] proposal is large: 24 MiB
13:53:04.558205 | .     6 | ... [n9,s9,r163/9:/Table/53/1/{5/1/-1…-9/1/-2…}] sideloadable proposal detected
13:53:07.549624 | 2.991419 | ... [n9,s9,r163/9:/Table/53/1/{5/1/-1…-9/1/-2…}] re-submitting command 33416f1483cfa380 to Raft: reasonTicks
13:53:07.557889 | .  8265 | ... [n9,s9,r163/9:/Table/53/1/{5/1/-1…-9/1/-2…}] proposal is large: 24 MiB
13:53:07.557901 | .    12 | ... [n9,s9,r163/9:/Table/53/1/{5/1/-1…-9/1/-2…}] sideloadable proposal detected
13:53:10.549587 | 2.991686 | ... [n9,s9,r163/9:/Table/53/1/{5/1/-1…-9/1/-2…}] re-submitting command 33416f1483cfa380 to Raft: reasonTicks
13:53:10.558032 | .  8445 | ... [n9,s9,r163/9:/Table/53/1/{5/1/-1…-9/1/-2…}] proposal is large: 24 MiB
13:53:10.558038 | .     5 | ... [n9,s9,r163/9:/Table/53/1/{5/1/-1…-9/1/-2…}] sideloadable proposal detected
13:53:13.549524 | 2.991486 | ... [n9,s9,r163/9:/Table/53/1/{5/1/-1…-9/1/-2…}] re-submitting command 33416f1483cfa380 to Raft: reasonTicks
13:53:13.557812 | .  8288 | ... [n9,s9,r163/9:/Table/53/1/{5/1/-1…-9/1/-2…}] proposal is large: 24 MiB
13:53:13.557819 | .     7 | ... [n9,s9,r163/9:/Table/53/1/{5/1/-1…-9/1/-2…}] sideloadable proposal detected
13:53:16.549555 | 2.991735 | ... [n9,s9,r163/9:/Table/53/1/{5/1/-1…-9/1/-2…}] re-submitting command 33416f1483cfa380 to Raft: reasonTicks
13:53:16.557944 | .  8390 | ... [n9,s9,r163/9:/Table/53/1/{5/1/-1…-9/1/-2…}] proposal is large: 24 MiB
13:53:16.557950 | .     6 | ... [n9,s9,r163/9:/Table/53/1/{5/1/-1…-9/1/-2…}] sideloadable proposal detected
13:53:19.549557 | 2.991607 | ... [n9,s9,r163/9:/Table/53/1/{5/1/-1…-9/1/-2…}] re-submitting command 33416f1483cfa380 to Raft: reasonTicks
13:53:19.557282 | .  7725 | ... [n9,s9,r163/9:/Table/53/1/{5/1/-1…-9/1/-2…}] proposal is large: 24 MiB
13:53:19.557289 | .     6 | ... [n9,s9,r163/9:/Table/53/1/{5/1/-1…-9/1/-2…}] sideloadable proposal detected
13:53:22.549495 | 2.992206 | ... [n9,s9,r163/9:/Table/53/1/{5/1/-1…-9/1/-2…}] re-submitting command 33416f1483cfa380 to Raft: reasonTicks
13:53:22.558263 | .  8768 | ... [n9,s9,r163/9:/Table/53/1/{5/1/-1…-9/1/-2…}] proposal is large: 24 MiB
13:53:22.558269 | .     6 | ... [n9,s9,r163/9:/Table/53/1/{5/1/-1…-9/1/-2…}] sideloadable proposal detected
13:53:25.549563 | 2.991294 | ... [n9,s9,r163/9:/Table/53/1/{5/1/-1…-9/1/-2…}] re-submitting command 33416f1483cfa380 to Raft: reasonTicks
13:53:25.557826 | .  8263 | ... [n9,s9,r163/9:/Table/53/1/{5/1/-1…-9/1/-2…}] proposal is large: 24 MiB
13:53:25.557832 | .     5 | ... [n9,s9,r163/9:/Table/53/1/{5/1/-1…-9/1/-2…}] sideloadable proposal detected
13:53:28.549544 | 2.991712 | ... [n9,s9,r163/9:/Table/53/1/{5/1/-1…-9/1/-2…}] re-submitting command 33416f1483cfa380 to Raft: reasonTicks
13:53:28.558044 | .  8500 | ... [n9,s9,r163/9:/Table/53/1/{5/1/-1…-9/1/-2…}] proposal is large: 24 MiB
13:53:28.558050 | .     6 | ... [n9,s9,r163/9:/Table/53/1/{5/1/-1…-9/1/-2…}] sideloadable proposal detected
13:53:31.549575 | 2.991526 | ... [n9,s9,r163/9:/Table/53/1/{5/1/-1…-9/1/-2…}] re-submitting command 33416f1483cfa380 to Raft: reasonTicks
13:53:31.557999 | .  8424 | ... [n9,s9,r163/9:/Table/53/1/{5/1/-1…-9/1/-2…}] proposal is large: 24 MiB
13:53:31.558007 | .     8 | ... [n9,s9,r163/9:/Table/53/1/{5/1/-1…-9/1/-2…}] sideloadable proposal detected


@tbg
Copy link
Member

tbg commented Oct 19, 2018

Probably not the root cause here, but looks like a buglet:

https://github.com/cockroachdb/vendored/blob/ca4276c48c28ef08550bc0014dd021a0081e9278/go.etcd.io/etcd/raft/raft.go#L973-L983

If we hit the config change path then we'll have added the original size but we'll commit a no-op.

@tbg
Copy link
Member

tbg commented Oct 19, 2018

Also another buglet:

https://github.com/cockroachdb/vendored/blob/ca4276c48c28ef08550bc0014dd021a0081e9278/go.etcd.io/etcd/raft/raft.go#L989

If you go into this method you see that it updates the term and index. Those are included in ent.Size() and so what we added at the beginning may not be what you subtract in the end. (You'd expect the mutated ones to be larger though, so this is hopefully just another inconsequential bug - the first one is "worse").

@tbg
Copy link
Member

tbg commented Oct 19, 2018

Going to try with this diff:

@@ -198,7 +199,19 @@ func (rn *RawNode) Step(m pb.Message) error {
 func (rn *RawNode) Ready() Ready {
        rd := rn.newReady()
        rn.raft.msgs = nil
+       before := rn.raft.uncommittedSize
        rn.raft.reduceUncommittedSize(rd.CommittedEntries)
+       after := rn.raft.uncommittedSize
+       if rn.raft.uncommittedSize != 0 && len(rd.CommittedEntries) != 0 {
+               if rn.raft.raftLog.lastIndex() == rd.CommittedEntries[len(rd.CommittedEntries)-1].Index {
+                       rn.raft.logger.Fatalf(
+                               "committed all entries, but size dropped from %d to %d only:\n%+v",
+                               before, after,
+                               DescribeEntries(
+                                       rd.CommittedEntries, func(b []byte) string { return fmt.Sprintf("%d bytes of data", len(b)) }),
+                       )
+               }
+       }
        return rd
 }

and @benesch's one-liner 20% chance repro:

roachprod put $cluster ./cockroach-linux* cockroach
roachprod wipe $cluster && roachprod start $cluster && roachprod sql $cluster -- -e 'set cluster setting trace.debug.enable = true' && time bin/workload fixtures load tpcc --checks=false $(eval echo $(roachprod pgurl --external $cluster:1)) --warehouses=100

@tbg
Copy link
Member

tbg commented Oct 19, 2018

Two nodes immediately folded, way before AddSSTable came around. Perhaps due to the buglets above, perhaps because my assertion is somehow flawed:

I181019 15:13:58.142341 1869 storage/replica_command.go:300  [n9,s9,r184/5:/{Table/61/3/3…-Max}] initiating a split of this range at key /Table/61/3/68/3/2521/"\xbe\x87\x85\x11\xd4\x02MF\x86\x8d\xf7^A\x16cz"/PrefixEnd [r189]
W181019 15:13:58.202726 187 vendor/go.etcd.io/etcd/raft/raft.go:1503  [n9,s9,r184/5:/{Table/61/3/3…-Max}] now at 585 (increment of 585)
W181019 15:13:58.202752 187 vendor/go.etcd.io/etcd/raft/raft.go:1503  [n9,s9,r184/5:/{Table/61/3/3…-Max}] now at 922 (increment of 337)
W181019 15:13:58.202795 187 vendor/go.etcd.io/etcd/raft/raft.go:1527  [n9,s9,r184/5:/{Table/61/3/3…-Max}] now at 922 (reduction of 0)
W181019 15:13:58.216733 187 vendor/go.etcd.io/etcd/raft/raft.go:1527  [n9,s9,r184/5:/{Table/61/3/3…-Max}] now at 922 (reduction of 0)
W181019 15:13:58.220014 196 vendor/go.etcd.io/etcd/raft/raft.go:1527  [n9,s9,r184/5:/{Table/61/3/3…-Max}] now at 922 (reduction of 0)
I181019 15:13:58.221376 1936 storage/store_snapshot.go:621  [n9,raftsnapshot,s9,r184/5:/{Table/61/3/3…-Max}] sending Raft snapshot c8d138d1 at applied index 10
I181019 15:13:58.221555 1936 storage/store_snapshot.go:664  [n9,raftsnapshot,s9,r184/5:/{Table/61/3/3…-Max}] streamed snapshot to (n7,s7):4: kv pairs: 6, log entries: 0, rate-limit: 8.0 MiB/sec, 1ms
W181019 15:13:58.222091 174 vendor/go.etcd.io/etcd/raft/raft.go:1527  [n9,s9,r184/5:/{Table/61/3/3…-Max}] now at 922 (reduction of 0)
W181019 15:13:58.223056 178 vendor/go.etcd.io/etcd/raft/raft.go:1527  [n9,s9,r184/5:/{Table/61/3/3…-Max}] now at 916 (reduction of 6)
W181019 15:13:58.223183 178 vendor/go.etcd.io/etcd/raft/raft.go:1503  [n9,s9,r184/5:/{Table/61/3/3…-Max}] now at 1501 (increment of 585)
W181019 15:13:58.223210 178 vendor/go.etcd.io/etcd/raft/raft.go:1503  [n9,s9,r184/5:/{Table/61/3/3…-Max}] now at 1838 (increment of 337)
W181019 15:13:58.223234 178 vendor/go.etcd.io/etcd/raft/raft.go:1527  [n9,s9,r184/5:/{Table/61/3/3…-Max}] now at 1838 (reduction of 0)
W181019 15:13:58.239488 175 vendor/go.etcd.io/etcd/raft/raft.go:1527  [n9,s9,r184/5:/{Table/61/3/3…-Max}] now at 916 (reduction of 922)
W181019 15:13:58.240789 1869 vendor/go.etcd.io/etcd/raft/raft.go:1503  [n9,s9,r184/5:/{Table/61/3/3…-Max}] now at 2097 (increment of 1181)
W181019 15:13:58.240853 180 vendor/go.etcd.io/etcd/raft/raft.go:1527  [n9,s9,r184/5:/{Table/61/3/3…-Max}] now at 2097 (reduction of 0)
W181019 15:13:58.248695 180 vendor/go.etcd.io/etcd/raft/raft.go:1527  [n9,s9,r184/5:/{Table/61/3/3…-Max}] now at 1175 (reduction of 922)
W181019 15:13:58.275165 177 vendor/go.etcd.io/etcd/raft/raft.go:1527  [n9,s9,r184/5:/{Table/61/3/3…-Max}] now at 0 (reduction of 1181)
I181019 15:13:58.297700 1965 storage/store_snapshot.go:621  [n9,raftsnapshot,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] sending Raft snapshot a655cf4e at applied index 16
I181019 15:13:58.298692 1965 storage/store_snapshot.go:664  [n9,raftsnapshot,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] streamed snapshot to (n4,s4):3: kv pairs: 8, log entries: 6, rate-limit: 8.0 MiB/sec, 6ms
W181019 15:13:58.402908 1960 vendor/go.etcd.io/etcd/raft/raft.go:1503  [n9,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] now at 132 (increment of 132)
W181019 15:13:58.402933 171 vendor/go.etcd.io/etcd/raft/raft.go:1527  [n9,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] now at 132 (reduction of 0)
W181019 15:13:58.425684 171 vendor/go.etcd.io/etcd/raft/raft.go:1527  [n9,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] now at 0 (reduction of 132)
W181019 15:13:59.207211 2558 vendor/go.etcd.io/etcd/raft/raft.go:1503  [n9,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] now at 186 (increment of 186)
W181019 15:13:59.207280 190 vendor/go.etcd.io/etcd/raft/raft.go:1527  [n9,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] now at 186 (reduction of 0)
W181019 15:13:59.224307 190 vendor/go.etcd.io/etcd/raft/raft.go:1527  [n9,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] now at 0 (reduction of 186)
I181019 15:14:00.652518 175 storage/replica_proposal.go:212  [n9,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] new range lease repl=(n9,s9):5 seq=5 start=1539962040.637289635,0 epo=1 pro=1539962040.637293646,0 following repl=(n7,s7):4 seq=4 start=1539962039.206928793,0 epo=1 pro=1539962039.206931242,0
I181019 15:14:00.653803 3178 storage/replica_command.go:816  [n9,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] change replicas (REMOVE_REPLICA (n10,s10):2): read existing descriptor r184:/Table/61/3/{33/3/1771/"\xab\x84(W\xe0\x9eC\xe4\x9cu\x99ܛ\u0086\xb8"/0-68/3/2521/"\xbe\x87\x85\x11\xd4\x02MF\x86\x8d\xf7^A\x16cz"/PrefixEnd} [(n5,s5):1, (n10,s10):2, (n4,s4):3, (n7,s7):4, (n9,s9):5, next=6, gen=1]
I181019 15:14:00.700349 3178 storage/replica.go:3884  [n9,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] proposing REMOVE_REPLICA((n10,s10):2): updated=[(n5,s5):1 (n9,s9):5 (n4,s4):3 (n7,s7):4] next=6
I181019 15:14:00.720248 183 storage/replica.go:3884  [n9,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] proposing REMOVE_REPLICA((n10,s10):2): updated=[(n5,s5):1 (n9,s9):5 (n4,s4):3 (n7,s7):4] next=6
W181019 15:14:00.720258 183 vendor/go.etcd.io/etcd/raft/raft.go:1503  [n9,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] now at 711 (increment of 711)
W181019 15:14:00.720273 183 vendor/go.etcd.io/etcd/raft/raft.go:1527  [n9,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] now at 711 (reduction of 0)
W181019 15:14:00.734554 183 vendor/go.etcd.io/etcd/raft/raft.go:1527  [n9,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] now at 705 (reduction of 6)
I181019 15:14:00.734770 183 storage/replica.go:3884  [n9,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] proposing REMOVE_REPLICA((n10,s10):2): updated=[(n5,s5):1 (n9,s9):5 (n4,s4):3 (n7,s7):4] next=6
W181019 15:14:00.734782 183 vendor/go.etcd.io/etcd/raft/raft.go:1503  [n9,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] now at 1416 (increment of 711)
W181019 15:14:00.735005 183 vendor/go.etcd.io/etcd/raft/raft.go:1527  [n9,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] now at 1416 (reduction of 0)
W181019 15:14:00.752966 183 vendor/go.etcd.io/etcd/raft/raft.go:1527  [n9,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] now at 705 (reduction of 711)
I181019 15:14:00.758545 3169 storage/replica_command.go:816  [n9,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] change replicas (REMOVE_REPLICA (n7,s7):4): read existing descriptor r184:/Table/61/3/{33/3/1771/"\xab\x84(W\xe0\x9eC\xe4\x9cu\x99ܛ\u0086\xb8"/0-68/3/2521/"\xbe\x87\x85\x11\xd4\x02MF\x86\x8d\xf7^A\x16cz"/PrefixEnd} [(n5,s5):1, (n9,s9):5, (n4,s4):3, (n7,s7):4, next=6, gen=1]
W181019 15:14:00.758874 3169 vendor/go.etcd.io/etcd/raft/raft.go:1503  [n9,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] now at 1283 (increment of 578)
W181019 15:14:00.759076 178 vendor/go.etcd.io/etcd/raft/raft.go:1527  [n9,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] now at 1283 (reduction of 0)
W181019 15:14:00.770496 174 vendor/go.etcd.io/etcd/raft/raft.go:1527  [n9,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] now at 1277 (reduction of 6)
W181019 15:14:00.770745 174 vendor/go.etcd.io/etcd/raft/raft.go:1503  [n9,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] now at 1855 (increment of 578)
W181019 15:14:00.770784 174 vendor/go.etcd.io/etcd/raft/raft.go:1527  [n9,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] now at 1855 (reduction of 0)
W181019 15:14:00.797029 182 vendor/go.etcd.io/etcd/raft/raft.go:1527  [n9,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] now at 1277 (reduction of 578)
W181019 15:14:00.809126 3235 vendor/go.etcd.io/etcd/raft/raft.go:1503  [n9,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] now at 1409 (increment of 132)
W181019 15:14:00.809208 196 vendor/go.etcd.io/etcd/raft/raft.go:1527  [n9,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] now at 1409 (reduction of 0)
W181019 15:14:00.823695 196 vendor/go.etcd.io/etcd/raft/raft.go:1527  [n9,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] now at 831 (reduction of 578)
W181019 15:14:00.842006 172 vendor/go.etcd.io/etcd/raft/raft.go:1527  [n9,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] now at 699 (reduction of 132)
E181019 15:14:00.842218 172 util/log/crash_reporting.go:477  [n9,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] Reported as error aa0290ffc341439fab25925776a82a9c
F181019 15:14:00.842234 172 vendor/go.etcd.io/etcd/raft/rawnode.go:207  [n9,s9,r184/5:/Table/61/3/{33/3/1…-68/3/2…}] committed all entries, but size dropped from 831 to 699 only

@tbg
Copy link
Member

tbg commented Oct 19, 2018

Ok, @benesch points out the obvious: if we leak even a single byte, the AddSSTables are going to bounce off the Raft log forever. So these buglets above are actually the real bugs.

@tbg
Copy link
Member

tbg commented Oct 19, 2018

I'm going to switch this code over to using only len(ent.Data) to cover up the two buglets. We'll see what happens.

@benesch
Copy link
Contributor

benesch commented Oct 19, 2018

I'm going to switch this code over to using only len(ent.Data) to cover up the two buglets. We'll see what happens.

Is that sufficient? Won't dropped conf changes still cause misaccounting? It seems like you could just move the call to increaseUncommittedSize right above the call to r.appendEntry(m.Entries...)

@tbg
Copy link
Member

tbg commented Oct 19, 2018

Right, I have to do that and count only the data size (to make sure changing term and index is irrelevant).

@benesch
Copy link
Contributor

benesch commented Oct 19, 2018

Ah, gotcha. SGTM.

@tbg
Copy link
Member

tbg commented Oct 19, 2018

Gotta run this a little longer, but I've imported the first table without hitting the assertion (or getting stuck). Decent chance that this is the fix, I'd say. Gonna give it a couple runs.

@tbg
Copy link
Member

tbg commented Oct 19, 2018

Passed perhaps a dozen times without flinching 🤷‍♂️ I think I can get started polishing this up for upstream.

tbg added a commit to tbg/etcd that referenced this issue Oct 19, 2018
The previous code was using the proto-generated `Size()` method to
track the size of an incoming proposal at the leader. This includes
the Index and Term, which were mutated after the call to `Size()`
when appending to the log. Additionally, it was not taking into
account that an ignored configuration change would ignore the
original proposal and append an empty entry instead.

As a result, a fully committed Raft group could end up with a non-
zero tracked uncommitted Raft log counter that would eventually hit
the ceiling and drop all future proposals indiscriminately. It would
also immediately imply that proposals exceeding the threshold alone
would get refused (as the "first uncommitted proposal" gets special
treatment and is always allowed in).

Track only the size of the payload actually appended to the Raft log
instead.

For context, see:
cockroachdb/cockroach#31618 (comment)
tbg added a commit to tbg/etcd that referenced this issue Oct 19, 2018
The previous code was using the proto-generated `Size()` method to
track the size of an incoming proposal at the leader. This includes
the Index and Term, which were mutated after the call to `Size()`
when appending to the log. Additionally, it was not taking into
account that an ignored configuration change would ignore the
original proposal and append an empty entry instead.

As a result, a fully committed Raft group could end up with a non-
zero tracked uncommitted Raft log counter that would eventually hit
the ceiling and drop all future proposals indiscriminately. It would
also immediately imply that proposals exceeding the threshold alone
would get refused (as the "first uncommitted proposal" gets special
treatment and is always allowed in).

Track only the size of the payload actually appended to the Raft log
instead.

For context, see:
cockroachdb/cockroach#31618 (comment)
tbg added a commit to tbg/etcd that referenced this issue Oct 19, 2018
The previous code was using the proto-generated `Size()` method to
track the size of an incoming proposal at the leader. This includes
the Index and Term, which were mutated after the call to `Size()`
when appending to the log. Additionally, it was not taking into
account that an ignored configuration change would ignore the
original proposal and append an empty entry instead.

As a result, a fully committed Raft group could end up with a non-
zero tracked uncommitted Raft log counter that would eventually hit
the ceiling and drop all future proposals indiscriminately. It would
also immediately imply that proposals exceeding the threshold alone
would get refused (as the "first uncommitted proposal" gets special
treatment and is always allowed in).

Track only the size of the payload actually appended to the Raft log
instead.

For context, see:
cockroachdb/cockroach#31618 (comment)
tbg added a commit to tbg/etcd that referenced this issue Oct 19, 2018
The previous code was using the proto-generated `Size()` method to
track the size of an incoming proposal at the leader. This includes
the Index and Term, which were mutated after the call to `Size()`
when appending to the log. Additionally, it was not taking into
account that an ignored configuration change would ignore the
original proposal and append an empty entry instead.

As a result, a fully committed Raft group could end up with a non-
zero tracked uncommitted Raft log counter that would eventually hit
the ceiling and drop all future proposals indiscriminately. It would
also immediately imply that proposals exceeding the threshold alone
would get refused (as the "first uncommitted proposal" gets special
treatment and is always allowed in).

Track only the size of the payload actually appended to the Raft log
instead.

For context, see:
cockroachdb/cockroach#31618 (comment)
@nvanbenschoten
Copy link
Member Author

Thanks for both of you looking into this. I think you're spot on about both issues, and correct that the first one was the actual cause of the stall while the second one would harmlessly underestimate the uncommitted log size. Looks like etcd-io/etcd#10199 fixes both of the issues.

I'm also happy to see #31627.

@tbg
Copy link
Member

tbg commented Oct 19, 2018

It passed many more times. I'm going to test with the newly opened upstream PR now:

etcd-io/etcd#10199

Hopefully I didn't un-fix anything there.

tbg added a commit to tbg/etcd that referenced this issue Oct 22, 2018
The previous code was using the proto-generated `Size()` method to
track the size of an incoming proposal at the leader. This includes
the Index and Term, which were mutated after the call to `Size()`
when appending to the log. Additionally, it was not taking into
account that an ignored configuration change would ignore the
original proposal and append an empty entry instead.

As a result, a fully committed Raft group could end up with a non-
zero tracked uncommitted Raft log counter that would eventually hit
the ceiling and drop all future proposals indiscriminately. It would
also immediately imply that proposals exceeding the threshold alone
would get refused (as the "first uncommitted proposal" gets special
treatment and is always allowed in).

Track only the size of the payload actually appended to the Raft log
instead.

For context, see:
cockroachdb/cockroach#31618 (comment)
tbg added a commit to tbg/etcd that referenced this issue Oct 22, 2018
The previous code was using the proto-generated `Size()` method to
track the size of an incoming proposal at the leader. This includes
the Index and Term, which were mutated after the call to `Size()`
when appending to the log. Additionally, it was not taking into
account that an ignored configuration change would ignore the
original proposal and append an empty entry instead.

As a result, a fully committed Raft group could end up with a non-
zero tracked uncommitted Raft log counter that would eventually hit
the ceiling and drop all future proposals indiscriminately. It would
also immediately imply that proposals exceeding the threshold alone
would get refused (as the "first uncommitted proposal" gets special
treatment and is always allowed in).

Track only the size of the payload actually appended to the Raft log
instead.

For context, see:
cockroachdb/cockroach#31618 (comment)
tbg added a commit to tbg/cockroach that referenced this issue Oct 22, 2018
Entries are "thinned" only when passed to `r.append()` (i.e. written
to disk) and they are always returned "fat" from `Entries()` (i.e.
Raft's way to get entries from disk). Consequently Raft never sees
thin entries and won't ask us to commit them.

Touches cockroachdb#31618.

Release note: None
tbg added a commit to tbg/cockroach that referenced this issue Oct 22, 2018
If this were passed as nil, we'd be returning "thin" (i.e. with
sideloading payloads not inlined) Entries. This isn't supposed
to happen, but check it.

See:
cockroachdb#31618 (comment).

Release note: None
craig bot pushed a commit that referenced this issue Oct 22, 2018
31556: importccl: re-enable job control tests r=mjibson a=mjibson

I tracked down the problem. It was that after the CANCEL JOB was issued,
sometimes the 2nd stage of the IMPORT (the shuffle) would have started,
and sometimes it wouldn't have. If it did not start then RunJob would
block forever trying to send on the allowResponse chan. Fix this by
making a draining go routine after the first block.

Closes #24623
Closes #24658

Release note: None

31627: storage: remove spurious call to maybeInlineSideloadedRaftCommand r=nvanbenschoten,benesch a=tschottdorf

Entries are "thinned" only when passed to `r.append()` (i.e. written to
disk) and they are always returned "fat" from `Entries()` (i.e. Raft's way
to get entries from disk). Consequently Raft never sees thin entries and
won't ask us to commit them.

Touches #31618.

Release note: None

31695: bitarray: don't allow FromEncodingParts to return invalid bit array r=knz a=benesch

It is invalid for a bit array's lastBitsUsed field to be greater than
64. The FromEncodingParts function, however, would happily construct
an invalid bitarray if given a too-large lastBitsUsed value. Teach the
FromEncodingParts to return an error instead.

This presented as a panic when attempting to pretty-print a key with a
bitarray whose lastBitsUsed encoded value was 65. Such a key can be
created when calling PrefixEnd on a key with a bitarray whose
lastBitsUsed value is 64. By returning an error instead, the
pretty-printing code will try again after calling UndoPrefixEnd and be
able to print the key.

Fix #31115.

Release note: None

31697: partitionccl: deflake TestDropIndexWithZoneConfigCCL r=danhhz,eriktrinh a=benesch

A particularly adversarial goroutine schedule can cause this test to
observe the moment in time where the data is dropped but the zone config
is not. Deflake by retrying the check for the dropped zone config until
it succeeds (or times out).

Fix #31678.

Release note: None

Co-authored-by: Matt Jibson <[email protected]>
Co-authored-by: Tobias Schottdorf <[email protected]>
Co-authored-by: Nikhil Benesch <[email protected]>
@tbg
Copy link
Member

tbg commented Oct 23, 2018

Fixed in #31689.

@tbg tbg closed this as completed Oct 23, 2018
tbg added a commit to tbg/cockroach that referenced this issue Nov 16, 2018
Entries are "thinned" only when passed to `r.append()` (i.e. written
to disk) and they are always returned "fat" from `Entries()` (i.e.
Raft's way to get entries from disk). Consequently Raft never sees
thin entries and won't ask us to commit them.

Touches cockroachdb#31618.

Release note: None
tbg added a commit to tbg/cockroach that referenced this issue Nov 16, 2018
If this were passed as nil, we'd be returning "thin" (i.e. with
sideloading payloads not inlined) Entries. This isn't supposed
to happen, but check it.

See:
cockroachdb#31618 (comment).

Release note: None
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
A-kv-replication Relating to Raft, consensus, and coordination. C-bug Code not up to spec/doc, specs & docs deemed correct. Solution expected to change code/behavior.
Projects
None yet
Development

No branches or pull requests

3 participants