Closed awoods187 closed 5 years ago
I extended this so it'd be up tomorrow andy-nomerge
Isn't this the same as the other one you filed?
Not the same, note the different receiver on MarshalTo
.
panic(0x2d24ec0, 0x5486fa0)
/usr/local/go/src/runtime/panic.go:513 +0x1b9
github.com/cockroachdb/cockroach/pkg/roachpb.(*ScanRequest).MarshalTo(0xc0219b1f40, 0xc032607cc2, 0x15, 0x15, 0x17, 0x2, 0x13)
panic(0x2d24ec0, 0x5486fa0)
/usr/local/go/src/runtime/panic.go:513 +0x1b9
github.com/cockroachdb/cockroach/pkg/roachpb.(*ScanRequest).MarshalTo(0xc08bb7ba80, 0xc014e5a284, 0x15, 0x15, 0x17, 0x2, 0x13)
The operation that's carried out here is
which in particular means that dAtA
is freshly allocated memory.
The panic then happens on this line
which writes into what should be the same dAtA
slice.
That this would be out of bounds seems to say that the m.Size()
in the first snippet is either wrong orgoes stale as we actually write the data. Something like this would be expected if we put stuff in the ScanRequest that comes from some buffer pool, and we give the request to grpc while mutating the scan request.
There could also be some bug with this new XXX_sizecache
thing:
I don't know how that works exactly, but it pattern matches and is also newly introduced since we bumped the dependencies a few weeks back. @RaduBerinde do you know more about this?
@jordanlewis do we pool ScanRequest (or any part of BatchRequest really) before passing it to KV?
As far as I found in the proto code, when marshaling it first goes through the proto (recursively) to figure out the length of the marshaled data, and then goes through it again to do the actual marshaling. The second step also needs to know the size of each structure in the tree. If the structure has a XXX_sizecache
, the size is stored in there in the first step and retrieved in the second step instead of being recalculated.
I believe the value is only relevant during marshaling. I don't see it causing a problem unless a proto is changed while it is being marshaled (which sounds bad regardless). If the same proto is being marshaled twice in parallel, both threads could write to it perhaps but they'd be writing the same value.
That being said, you can disable XXX_sizecache
on a per-proto basis (we could also do it for all our protos if we choose to). Supposedly, removing it does come with a small regression during marshaling.
This is the relevant code, I don't think there's anything else using XXX_sizecache
: https://github.com/cockroachdb/vendored/blob/master/github.com/gogo/protobuf/proto/table_marshal.go#L223
There's no object pooling for ScanRequest (or anything with protos, I don't think). I did add a bunch of pooling recently but it was all objects higher up in the stack.
Issue #34256 seems to indicate it isn't a pooling issue - the whole stack is in a single goroutine, including initialization from fresh memory of the proto in question.
So one thing I realized this weekend is that I wasn't pointing the workload at all of the nodes. I ran:
roachprod run $CLUSTER:7 "./workload run tpcc --ramp=5m --warehouses=3400 --duration=15m --split --scatter {pgurl:1-3}"
instead of
roachprod run $CLUSTER:7 "./workload run tpcc --ramp=5m --warehouses=3400 --duration=15m --split --scatter {pgurl:1-6}"
So the only thing that's marshalled before the out of bounds is the start and end key of the RequestHeader. This means that any kind of potential mutation between the size calculation and the marshalling would have to occur on the span keys. Let's see if that's possible.
We set the span of a RequestHeader in just one place:
So, we're always setting it to the values in txnKVFetcher.spans
. Where do those come from? Two places, either from the resumeSpan
of a ScanResponse
, which seems like it would never get mutated since that's itself returned memory from gRPC, or from the spans passed in by the table reader. Let's examine that latter case.
The spans are written at the initialization of a txnKVFetcher
, by copying them into a fresh container:
So at this point we should be confident that a mutation to the container that got passed into the txnKVFetcher
shouldn't matter, right? Since everything's passed by value. So, the only case that could potentially get us is if somebody mutates the underlying span's Key/EndKey bytes.
I looked around for instances of that and couldn't find anything.
But, it's worth noting that there's something suspicious going on with these spans - I did add some object pooling for TableReaders in September (#30676) which attempts to reuse the memory for the roachpb.Spans
slice. That being said, I don't see where the problem could be even still, as the sole consumer of this slice makes a (shallow) copy before using it further.
All this being said, I still don't get what's going on.
Ugh. Unfortunately I also can't see where the problem with that would be. Can you (find someone who has time to) wrap this repro in a script and in particular run TPCC itself with a race build (after verifying that it sometimes repro without the race build)? Maybe we'll get lucky.
We could also catch the panic from that particular proto and print the actual size of dAtA
vs what the proto's new Size()
is, but the result would necessarily have to be that the new size is larger than dAtA
.
My assumption is that the size cache stuff exposes some preexisting condition.
PS I did look through DistSender and in particular its truncate
method, but that takes an appropriate shallow copy of all requests it truncates.
We've now seen this in the wild. #34241
@jordanlewis that's linking back to this issue, what's the real issue number?
Oops! #34674
Yikes. What do we do with this? Seems that at this point we want some more eyes on the problem and a way to get a core dump if it happens.
A core dump would definitely help. I (or someone else) should get on https://github.com/cockroachdb/cockroach/issues/34680. I'll try and find time to reproduce on Friday.
I'm very confused -- I was looking at this again and can't find XXX_sizecache
being used at all on current master? I was so sure I was missing something that I removed the XXX_sizecache
field from `BatchRequest, but everything just works, so definitely there isn't any generated code hooked up to it. I did the same in db4a8a1c64 and guess what, it wasn't ever hooked up from the looks of it.
Not that this would matter for the kind of bug at hand (if the protos change out from under you during a marshal, things are going to hurt), but it seems that we never actually turned on the sizecache, contrary to what the commit message suggested. cc @RaduBerinde
So the only thing that's marshalled before the out of bounds is the start and end key of the RequestHeader.
Taking a second look, this isn't actually correct. We initialize the memory right at the beginning of BatchRequest.Marshal
and unmarshal everything into it. It crashes eventually on some ScanRequest, but it doesn't say at all that it's the first one. Just that we eventually ran out of memory. So all we know (well, suspect) is that some memory used by the BatchRequest changes while it's being marshaled (and that it changes in a way that would really require a larger buffer than was provisioned).
The marshaling code uses reflection, and if the field exists it is used to avoid calculating structure size twice during marshaling. If it does not exist, marshaling still works, just might be slower in principle. Also see my comment above https://github.com/cockroachdb/cockroach/issues/34241#issuecomment-457551667
I looked at this again with @RaduBerinde and it really seems that we're not using this field at all except in some places. We blame this on the fact that gogoproto never really whole-heartedly picked up these new changes. Gogoproto appears to be abandonware since the main maintainer (Walter Schulze) has moved on.
Anyway, this is just a tangent that doesn't change the fact that we're probably looking at a data race here.
So the theory currently is that somebody's mutating... what exactly? A bytes field somewhere within the request, like the spans that I was searching for earlier? Or are there other possibilities? Just want to know where to swing my attention to try to help here.
Unclear unfortunately. It has to be some pointer dangling off of a BatchRequest
. So it's either in Header
(for example Txn
, ScanOptions
) or it's something in Requests
(for example, ScanRequest.RequestHeader.{End,}Key
). I personally hope that the assertion I put in today fires and tells us something because there aren't any obvious leads here.
I cleaned up the repro script in Andy's initial post. You can basically copy-paste all of it and it'll just work. Running it already.
No repro so far, 18h in.
Make that 28 hours total. :(
I believe this crash here might be a repro: https://github.com/cockroachdb/cockroach/issues/34695#issuecomment-469027213
panic: batch size 196 -> 194 bytes
re-marshaled protobuf:
00000000 0a 9d 01 0a 00 12 06 08 04 10 04 18 03 18 47 2a |..............G*|
00000010 87 01 0a 22 0a 10 d0 96 0f dd 5a a8 41 65 b0 b2 |..."......Z.Ae..|
00000020 01 80 af fa 3c 66 2a 0a 08 8a e7 e3 dc 8c 8e 9e |....<f*.........|
00000030 c4 15 30 87 b0 05 12 07 73 71 6c 20 74 78 6e 2a |..0.....sql txn*|
00000040 0a 08 8a e7 e3 dc 8c 8e 9e c4 15 32 0a 08 8a e7 |...........2....|
00000050 e3 dc 8c 8e 9e c4 15 3a 0a 08 8a b1 99 cb 8e 8e |.......:........|
00000060 9e c4 15 42 0e 08 01 12 0a 08 8a e7 e3 dc 8c 8e |...B............|
00000070 9e c4 15 42 0e 08 04 12 0a 08 e5 ed d3 ad 8d 8e |...B............|
00000080 9e c4 15 42 10 08 05 12 0c 08 ab ee f5 ab 8d 8e |...B............|
00000090 9e c4 15 10 0a 72 00 7a 00 40 8f 4e 50 01 58 01 |.....r.z.@.NP.X.|
000000a0 12 20 3a 1e 0a 1a 1a 0b bd 89 fd 05 fb 76 e8 54 |. :..........v.T|
000000b0 ce 80 06 22 0b bd 89 fd 05 fb 76 e8 55 63 00 01 |..."......v.Uc..|
000000c0 20 01 | .|
original panic: <nil>
[recovered]
panic: batch size 196 -> 194 bytes
re-marshaled protobuf:
00000000 0a 9d 01 0a 00 12 06 08 04 10 04 18 03 18 47 2a |..............G*|
00000010 87 01 0a 22 0a 10 d0 96 0f dd 5a a8 41 65 b0 b2 |..."......Z.Ae..|
00000020 01 80 af fa 3c 66 2a 0a 08 8a e7 e3 dc 8c 8e 9e |....<f*.........|
00000030 c4 15 30 87 b0 05 12 07 73 71 6c 20 74 78 6e 2a |..0.....sql txn*|
00000040 0a 08 8a e7 e3 dc 8c 8e 9e c4 15 32 0a 08 8a e7 |...........2....|
00000050 e3 dc 8c 8e 9e c4 15 3a 0a 08 8a b1 99 cb 8e 8e |.......:........|
00000060 9e c4 15 42 0e 08 01 12 0a 08 8a e7 e3 dc 8c 8e |...B............|
00000070 9e c4 15 42 0e 08 04 12 0a 08 e5 ed d3 ad 8d 8e |...B............|
00000080 9e c4 15 42 10 08 05 12 0c 08 ab ee f5 ab 8d 8e |...B............|
00000090 9e c4 15 10 0a 72 00 7a 00 40 8f 4e 50 01 58 01 |.....r.z.@.NP.X.|
000000a0 12 20 3a 1e 0a 1a 1a 0b bd 89 fd 05 fb 76 e8 54 |. :..........v.T|
000000b0 ce 80 06 22 0b bd 89 fd 05 fb 76 e8 55 63 00 01 |..."......v.Uc..|
000000c0 20 01 | .|
original panic: <nil>
goroutine 66204 [running]:
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).divideAndSendBatchToRanges.func1(0xc009678318, 0xc009678608, 0xc009678548, 0xc009678600, 0xc00967827f, 0xc009678300, 0xc009678284)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:825 +0x50e
panic(0x2d635a0, 0xc00915b8c0)
/usr/local/go/src/runtime/panic.go:513 +0x1b9
github.com/cockroachdb/cockroach/pkg/kv.withMarshalingDebugging.func1(0xc009502f80, 0xc4)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/transport.go:181 +0x264
github.com/cockroachdb/cockroach/pkg/kv.withMarshalingDebugging(0x3a16420, 0xc007b8b1d0, 0x0, 0x0, 0x400000004, 0x3, 0x47, 0x0, 0xc0088e4a00, 0x0, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/transport.go:185 +0xac
github.com/cockroachdb/cockroach/pkg/kv.(*grpcTransport).SendNext(0xc007b8b1a0, 0x3a16420, 0xc00885bef0, 0x0, 0x0, 0x400000004, 0x3, 0x47, 0x0, 0xc0088e4a00, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/transport.go:202 +0x211
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).sendToReplicas(0xc00020d7a0, 0x3a16420, 0xc00885bef0, 0xc00020d7f0, 0x47, 0xc00750a6e0, 0x3, 0x3, 0x0, 0x0, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:1363 +0x2d3
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).sendRPC(0xc00020d7a0, 0x3a16420, 0xc00885bef0, 0x47, 0xc00750a6e0, 0x3, 0x3, 0x0, 0x0, 0x0, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:416 +0x244
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).sendSingleRange(0xc00020d7a0, 0x3a16420, 0xc00885bef0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xc0088e4a00, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:496 +0x221
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).sendPartialBatch(0xc00020d7a0, 0x3a16420, 0xc00885bef0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xc0088e4a00, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:1139 +0x322
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).divideAndSendBatchToRanges(0xc00020d7a0, 0x3a16420, 0xc00885bef0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xc0088e4a00, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:962 +0x8b3
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).Send(0xc00020d7a0, 0x3a16420, 0xc00885bef0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xc00819da00, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:710 +0x48b
github.com/cockroachdb/cockroach/pkg/kv.(*txnLockGatekeeper).SendLocked(0xc009939f10, 0x3a16420, 0xc00885bef0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xc00819da00, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/txn_coord_sender.go:233 +0xe8
github.com/cockroachdb/cockroach/pkg/kv.(*txnSpanRefresher).sendLockedWithRefreshAttempts(0xc009939e48, 0x3a16420, 0xc00885bef0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xc00819da00, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/txn_interceptor_span_refresher.go:160 +0x83
github.com/cockroachdb/cockroach/pkg/kv.(*txnSpanRefresher).SendLocked(0xc009939e48, 0x3a16420, 0xc00885bef0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xc00819da00, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/txn_interceptor_span_refresher.go:101 +0xf9
github.com/cockroachdb/cockroach/pkg/kv.(*txnPipeliner).SendLocked(0xc009939dc0, 0x3a16420, 0xc00885bef0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xc00819da00, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/txn_interceptor_pipeliner.go:182 +0xf9
github.com/cockroachdb/cockroach/pkg/kv.(*txnSeqNumAllocator).SendLocked(0xc009939d68, 0x3a16420, 0xc00885bef0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xc00819da00, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/txn_interceptor_seq_num_allocator.go:92 +0x23b
github.com/cockroachdb/cockroach/pkg/kv.(*TxnCoordSender).Send(0xc009939b00, 0x3a16420, 0xc00885bef0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/txn_coord_sender.go:698 +0x591
github.com/cockroachdb/cockroach/pkg/internal/client.(*DB).sendUsingSender(0xc000443200, 0x3a16420, 0xc00885bec0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/internal/client/db.go:622 +0x119
github.com/cockroachdb/cockroach/pkg/internal/client.(*Txn).Send(0xc0079034d0, 0x3a16420, 0xc00885bec0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/internal/client/txn.go:790 +0x13c
github.com/cockroachdb/cockroach/pkg/sql/row.(*txnKVFetcher).fetch(0xc007ef6d00, 0x3a16420, 0xc00885bec0, 0x0, 0x0)
/go/src/github.com/cockroachdb/cockroach/pkg/sql/row/kv_batch_fetcher.go:242 +0x626
github.com/cockroachdb/cockroach/pkg/sql/row.(*txnKVFetcher).nextBatch(0xc007ef6d00, 0x3a16420, 0xc00885bec0, 0xc007ca6e80, 0x30, 0xc007ca4ba0, 0x0, 0x0, 0x0, 0x0, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/sql/row/kv_batch_fetcher.go:326 +0x1dd
Another one found by @awoods187 here: https://github.com/cockroachdb/cockroach/issues/35393#issuecomment-469590471
interesting about that one is that it happened during an insert and there's a tablewriter on the stack, not a kvfetcher.
Also this one: https://github.com/cockroachdb/cockroach/issues/35357#issue-416959519
Though that's with a kvfetcher again.
Time to do some decoding.
Ok, first results are in. We have two repros from InitPut
requests where we win two bytes, and we have two from scans that lose two bytes (it seems to always be two bytes). The batch request represented by the hexdump confirms the final bytes from the assertion's error message. The code to do all the mangling is in https://github.com/cockroachdb/cockroach/pull/35414.
Full decoded batches here, but here's the rub:
--- PASS: TestMarshalDebugOutput (0.00s)
--- PASS: TestMarshalDebugOutput/txnKVfetcher_196->194_https://github.com/cockroachdb/cockroach/issues/34241#issuecomment-469317193 (0.00s)
transport_test.go:304: int(194)
transport_test.go:306: [/Table/53/1/431068929571323910, /Table/53/1/431068929581056001)
--- PASS: TestMarshalDebugOutput/tableWriterBase_239->241_https://github.com/cockroachdb/cockroach/issues/35393#issuecomment-469590471 (0.00s)
transport_test.go:304: int(241)
transport_test.go:306: [/Table/59/3/1050/6/2102/"{~jD\xc1!H\xaf\x9b`'\x95\xaeR\xa2M"/0, /Min)
--- PASS: TestMarshalDebugOutput/txnKVFetcher_213->211_https://github.com/cockroachdb/cockroach/issues/35357#issue-416959519_#1 (0.00s)
transport_test.go:304: int(211)
transport_test.go:306: [/Table/55/1/8466, /Table/55/1/8466/#)
--- PASS: TestMarshalDebugOutput/tableWriter_219_->_221_https://github.com/cockroachdb/cockroach/issues/35357#issue-416959519_#2 (0.00s)
transport_test.go:304: int(221)
transport_test.go:306: [/Table/58/3/1385/7/2758/"\r\n<pn(F\x9e\x8e\xae\x9c'_`\xcb\x0e"/0, /Min)
Looking at the pointers inside of a BatchRequest
, and taking into accounts fields that always contain nil pointers in these repros, the candidates I see are ba.Txn
, ba.Requests
, and ba.Requests[i].GetInner().X
. But the InitPut
s and Scan
s happen only on the code paths on which you expect them, and there's not a whole lot in them that you'd think would change (and if this were a pooling bug, why always exactly two bytes of difference? That doesn't make sense).
This seems to happen frequently enough to just get the pre-marshal hexdump as well and repro in a targeted setting, just to settle the question of what changed without thinking too hard.
Running six instances of https://github.com/cockroachdb/cockroach/issues/34695 now, hopefully that'll shake out something soon.
Got a repro using the binary from https://github.com/cockroachdb/cockroach/pull/35414 on andy-36
I190305 18:25:37.636027 6508958 storage/replicate_queue.go:264 [n4,replicate,s4,r14642/3:/Table/58/1/59{09/63â¦-10/67â¦}] snapshot failed: (n6,s6):?: remote declined preemptive snapshot 9f3211cf at applied index 156: store busy applying snapshots
panic: batch size 677 -> 675 bytes
re-marshaled protobuf:
00000000 0a e6 01 0a 00 12 06 08 04 10 04 18 02 18 fa 5d |...............]|
00000010 2a d2 01 0a 2f 0a 10 96 c4 13 c6 4e ef 48 aa a2 |*.../......N.H..|
00000020 0c 3c 77 5c 37 a6 f9 1a 07 bd 89 f7 0c b0 92 88 |.<w\7...........|
00000030 2a 0c 08 dd e9 f0 87 fe e8 c8 c4 15 10 1b 30 b5 |*.............0.|
00000040 b6 03 38 01 12 07 73 71 6c 20 74 78 6e 2a 0c 08 |..8...sql txn*..|
00000050 dd e9 f0 87 fe e8 c8 c4 15 10 1b 32 0c 08 dd e9 |...........2....|
00000060 f0 87 fe e8 c8 c4 15 10 1b 3a 0c 08 dd b3 a6 f6 |.........:......|
00000070 ff e8 c8 c4 15 10 1b 42 10 08 01 12 0c 08 dd e9 |.......B........|
00000080 f0 87 fe e8 c8 c4 15 10 1b 42 0e 08 02 12 0a 08 |.........B......|
00000090 87 f0 db 89 fe e8 c8 c4 15 42 10 08 03 12 0c 08 |.........B......|
000000a0 d6 ce c2 88 fe e8 c8 c4 15 10 11 42 0e 08 04 12 |...........B....|
000000b0 0a 08 d4 f4 c5 8e fe e8 c8 c4 15 42 10 08 05 12 |...........B....|
000000c0 0c 08 85 c2 c9 89 fe e8 c8 c4 15 10 45 42 10 08 |............EB..|
000000d0 06 12 0c 08 db a8 c2 8e fe e8 c8 c4 15 10 44 48 |..............DH|
000000e0 01 72 00 7a 00 50 01 58 01 12 1d 3a 1b 0a 17 1a |.r.z.P.X...:....|
000000f0 08 c2 89 f7 0c b0 f7 30 97 22 09 c2 89 f7 0c b0 |.......0."......|
00000100 f7 30 97 fe 28 01 20 01 12 1d 3a 1b 0a 17 1a 08 |.0..(. ...:.....|
00000110 c2 89 f7 0c b0 f7 34 9d 22 09 c2 89 f7 0c b0 f7 |......4.".......|
00000120 34 9d fe 28 01 20 01 12 1d 3a 1b 0a 17 1a 08 c2 |4..(. ...:......|
00000130 89 f7 0c b0 f7 40 fe 22 09 c2 89 f7 0c b0 f7 40 |.....@.".......@|
00000140 fe fe 28 01 20 01 12 1d 3a 1b 0a 17 1a 08 c2 89 |..(. ...:.......|
00000150 f7 0c b0 f7 49 22 22 09 c2 89 f7 0c b0 f7 49 22 |....I"".......I"|
00000160 fe 28 01 20 01 12 1d 3a 1b 0a 17 1a 08 c2 89 f7 |.(. ...:........|
00000170 0c b0 f7 7f fa 22 09 c2 89 f7 0c b0 f7 7f fa fe |....."..........|
00000180 28 01 20 01 12 1d 3a 1b 0a 17 1a 08 c2 89 f7 0c |(. ...:.........|
00000190 b0 f7 80 39 22 09 c2 89 f7 0c b0 f7 80 39 fe 28 |...9"........9.(|
000001a0 01 20 01 12 1d 3a 1b 0a 17 1a 08 c2 89 f7 0c b0 |. ...:..........|
000001b0 f7 80 e2 22 09 c2 89 f7 0c b0 f7 80 e2 fe 28 01 |..."..........(.|
000001c0 20 01 12 1d 3a 1b 0a 17 1a 08 c2 89 f7 0c b0 f7 | ...:...........|
000001d0 86 ae 22 09 c2 89 f7 0c b0 f7 86 ae fe 28 01 20 |.."..........(. |
000001e0 01 12 1d 3a 1b 0a 17 1a 08 c2 89 f7 0c b0 f7 c1 |...:............|
000001f0 1e 22 09 c2 89 f7 0c b0 f7 c1 1e fe 28 01 20 01 |."..........(. .|
00000200 12 1d 3a 1b 0a 17 1a 08 c2 89 f7 0c b0 f7 d8 6f |..:............o|
00000210 22 09 c2 89 f7 0c b0 f7 d8 6f fe 28 01 20 01 12 |"........o.(. ..|
00000220 1f 3a 1d 0a 19 1a 09 c2 89 f7 0c b0 f8 01 36 30 |.:............60|
00000230 22 0a c2 89 f7 0c b0 f8 01 36 30 fe 28 01 20 01 |"........60.(. .|
00000240 12 1f 3a 1d 0a 19 1a 09 c2 89 f7 0c b0 f8 01 39 |..:............9|
00000250 1b 22 0a c2 89 f7 0c b0 f8 01 39 1b fe 28 01 20 |."........9..(. |
00000260 01 12 1f 3a 1d 0a 19 1a 09 c2 89 f7 0c b0 f8 01 |...:............|
00000270 4e 20 22 0a c2 89 f7 0c b0 f8 01 4e 20 fe 28 01 |N "........N .(.|
00000280 20 01 12 1f 3a 1d 0a 19 1a 09 c2 89 f7 0c b0 f8 | ...:...........|
00000290 01 70 c2 22 0a c2 89 f7 0c b0 f8 01 70 c2 fe 28 |.p."........p..(|
000002a0 01 20 01 |. .|
original protobuf:
00000000 0a e8 01 0a 00 12 06 08 04 10 04 18 02 18 fa 5d |...............]|
00000010 2a d4 01 0a 2f 0a 10 96 c4 13 c6 4e ef 48 aa a2 |*.../......N.H..|
00000020 0c 3c 77 5c 37 a6 f9 1a 07 bd 89 f7 0c b0 92 88 |.<w\7...........|
00000030 2a 0c 08 dd e9 f0 87 fe e8 c8 c4 15 10 1b 30 b5 |*.............0.|
00000040 b6 03 38 01 12 07 73 71 6c 20 74 78 6e 2a 0c 08 |..8...sql txn*..|
00000050 dd e9 f0 87 fe e8 c8 c4 15 10 1b 32 0c 08 dd e9 |...........2....|
00000060 f0 87 fe e8 c8 c4 15 10 1b 3a 0c 08 dd b3 a6 f6 |.........:......|
00000070 ff e8 c8 c4 15 10 1b 42 10 08 01 12 0c 08 dd e9 |.......B........|
00000080 f0 87 fe e8 c8 c4 15 10 1b 42 0e 08 02 12 0a 08 |.........B......|
00000090 87 f0 db 89 fe e8 c8 c4 15 42 10 08 03 12 0c 08 |.........B......|
000000a0 d6 ce c2 88 fe e8 c8 c4 15 10 11 42 10 08 05 12 |...........B....|
000000b0 0c 08 85 c2 c9 89 fe e8 c8 c4 15 10 45 42 10 08 |............EB..|
000000c0 06 12 0c 08 db a8 c2 8e fe e8 c8 c4 15 10 44 42 |..............DB|
000000d0 10 08 07 12 0c 08 9d a3 f5 8a fe e8 c8 c4 15 10 |................|
000000e0 47 48 01 72 00 7a 00 50 01 58 01 12 1d 3a 1b 0a |GH.r.z.P.X...:..|
000000f0 17 1a 08 c2 89 f7 0c b0 f7 30 97 22 09 c2 89 f7 |.........0."....|
00000100 0c b0 f7 30 97 fe 28 01 20 01 12 1d 3a 1b 0a 17 |...0..(. ...:...|
00000110 1a 08 c2 89 f7 0c b0 f7 34 9d 22 09 c2 89 f7 0c |........4.".....|
00000120 b0 f7 34 9d fe 28 01 20 01 12 1d 3a 1b 0a 17 1a |..4..(. ...:....|
00000130 08 c2 89 f7 0c b0 f7 40 fe 22 09 c2 89 f7 0c b0 |.......@."......|
00000140 f7 40 fe fe 28 01 20 01 12 1d 3a 1b 0a 17 1a 08 |.@..(. ...:.....|
00000150 c2 89 f7 0c b0 f7 49 22 22 09 c2 89 f7 0c b0 f7 |......I"".......|
00000160 49 22 fe 28 01 20 01 12 1d 3a 1b 0a 17 1a 08 c2 |I".(. ...:......|
00000170 89 f7 0c b0 f7 7f fa 22 09 c2 89 f7 0c b0 f7 7f |......."........|
00000180 fa fe 28 01 20 01 12 1d 3a 1b 0a 17 1a 08 c2 89 |..(. ...:.......|
00000190 f7 0c b0 f7 80 39 22 09 c2 89 f7 0c b0 f7 80 39 |.....9"........9|
000001a0 fe 28 01 20 01 12 1d 3a 1b 0a 17 1a 08 c2 89 f7 |.(. ...:........|
000001b0 0c b0 f7 80 e2 22 09 c2 89 f7 0c b0 f7 80 e2 fe |....."..........|
000001c0 28 01 20 01 12 1d 3a 1b 0a 17 1a 08 c2 89 f7 0c |(. ...:.........|
000001d0 b0 f7 86 ae 22 09 c2 89 f7 0c b0 f7 86 ae fe 28 |...."..........(|
000001e0 01 20 01 12 1d 3a 1b 0a 17 1a 08 c2 89 f7 0c b0 |. ...:..........|
000001f0 f7 c1 1e 22 09 c2 89 f7 0c b0 f7 c1 1e fe 28 01 |..."..........(.|
00000200 20 01 12 1d 3a 1b 0a 17 1a 08 c2 89 f7 0c b0 f7 | ...:...........|
00000210 d8 6f 22 09 c2 89 f7 0c b0 f7 d8 6f fe 28 01 20 |.o"........o.(. |
00000220 01 12 1f 3a 1d 0a 19 1a 09 c2 89 f7 0c b0 f8 01 |...:............|
00000230 36 30 22 0a c2 89 f7 0c b0 f8 01 36 30 fe 28 01 |60"........60.(.|
00000240 20 01 12 1f 3a 1d 0a 19 1a 09 c2 89 f7 0c b0 f8 | ...:...........|
00000250 01 39 1b 22 0a c2 89 f7 0c b0 f8 01 39 1b fe 28 |.9."........9..(|
00000260 01 20 01 12 1f 3a 1d 0a 19 1a 09 c2 89 f7 0c b0 |. ...:..........|
00000270 f8 01 4e 20 22 0a c2 89 f7 0c b0 f8 01 4e 20 fe |..N "........N .|
00000280 28 01 20 01 12 1f 3a 1d 0a 19 1a 09 c2 89 f7 0c |(. ...:.........|
00000290 b0 f8 01 70 c2 22 0a c2 89 f7 0c b0 f8 01 70 c2 |...p."........p.|
000002a0 fe 28 01 20 01 |.(. .|
original panic: <nil>
[recovered]
panic: batch size 677 -> 675 bytes
re-marshaled protobuf:
00000000 0a e6 01 0a 00 12 06 08 04 10 04 18 02 18 fa 5d |...............]|
00000010 2a d2 01 0a 2f 0a 10 96 c4 13 c6 4e ef 48 aa a2 |*.../......N.H..|
00000020 0c 3c 77 5c 37 a6 f9 1a 07 bd 89 f7 0c b0 92 88 |.<w\7...........|
00000030 2a 0c 08 dd e9 f0 87 fe e8 c8 c4 15 10 1b 30 b5 |*.............0.|
00000040 b6 03 38 01 12 07 73 71 6c 20 74 78 6e 2a 0c 08 |..8...sql txn*..|
00000050 dd e9 f0 87 fe e8 c8 c4 15 10 1b 32 0c 08 dd e9 |...........2....|
00000060 f0 87 fe e8 c8 c4 15 10 1b 3a 0c 08 dd b3 a6 f6 |.........:......|
00000070 ff e8 c8 c4 15 10 1b 42 10 08 01 12 0c 08 dd e9 |.......B........|
00000080 f0 87 fe e8 c8 c4 15 10 1b 42 0e 08 02 12 0a 08 |.........B......|
00000090 87 f0 db 89 fe e8 c8 c4 15 42 10 08 03 12 0c 08 |.........B......|
000000a0 d6 ce c2 88 fe e8 c8 c4 15 10 11 42 0e 08 04 12 |...........B....|
000000b0 0a 08 d4 f4 c5 8e fe e8 c8 c4 15 42 10 08 05 12 |...........B....|
000000c0 0c 08 85 c2 c9 89 fe e8 c8 c4 15 10 45 42 10 08 |............EB..|
000000d0 06 12 0c 08 db a8 c2 8e fe e8 c8 c4 15 10 44 48 |..............DH|
000000e0 01 72 00 7a 00 50 01 58 01 12 1d 3a 1b 0a 17 1a |.r.z.P.X...:....|
000000f0 08 c2 89 f7 0c b0 f7 30 97 22 09 c2 89 f7 0c b0 |.......0."......|
00000100 f7 30 97 fe 28 01 20 01 12 1d 3a 1b 0a 17 1a 08 |.0..(. ...:.....|
00000110 c2 89 f7 0c b0 f7 34 9d 22 09 c2 89 f7 0c b0 f7 |......4.".......|
00000120 34 9d fe 28 01 20 01 12 1d 3a 1b 0a 17 1a 08 c2 |4..(. ...:......|
00000130 89 f7 0c b0 f7 40 fe 22 09 c2 89 f7 0c b0 f7 40 |.....@.".......@|
00000140 fe fe 28 01 20 01 12 1d 3a 1b 0a 17 1a 08 c2 89 |..(. ...:.......|
00000150 f7 0c b0 f7 49 22 22 09 c2 89 f7 0c b0 f7 49 22 |....I"".......I"|
00000160 fe 28 01 20 01 12 1d 3a 1b 0a 17 1a 08 c2 89 f7 |.(. ...:........|
00000170 0c b0 f7 7f fa 22 09 c2 89 f7 0c b0 f7 7f fa fe |....."..........|
00000180 28 01 20 01 12 1d 3a 1b 0a 17 1a 08 c2 89 f7 0c |(. ...:.........|
00000190 b0 f7 80 39 22 09 c2 89 f7 0c b0 f7 80 39 fe 28 |...9"........9.(|
000001a0 01 20 01 12 1d 3a 1b 0a 17 1a 08 c2 89 f7 0c b0 |. ...:..........|
000001b0 f7 80 e2 22 09 c2 89 f7 0c b0 f7 80 e2 fe 28 01 |..."..........(.|
000001c0 20 01 12 1d 3a 1b 0a 17 1a 08 c2 89 f7 0c b0 f7 | ...:...........|
000001d0 86 ae 22 09 c2 89 f7 0c b0 f7 86 ae fe 28 01 20 |.."..........(. |
000001e0 01 12 1d 3a 1b 0a 17 1a 08 c2 89 f7 0c b0 f7 c1 |...:............|
000001f0 1e 22 09 c2 89 f7 0c b0 f7 c1 1e fe 28 01 20 01 |."..........(. .|
00000200 12 1d 3a 1b 0a 17 1a 08 c2 89 f7 0c b0 f7 d8 6f |..:............o|
00000210 22 09 c2 89 f7 0c b0 f7 d8 6f fe 28 01 20 01 12 |"........o.(. ..|
00000220 1f 3a 1d 0a 19 1a 09 c2 89 f7 0c b0 f8 01 36 30 |.:............60|
00000230 22 0a c2 89 f7 0c b0 f8 01 36 30 fe 28 01 20 01 |"........60.(. .|
00000240 12 1f 3a 1d 0a 19 1a 09 c2 89 f7 0c b0 f8 01 39 |..:............9|
00000250 1b 22 0a c2 89 f7 0c b0 f8 01 39 1b fe 28 01 20 |."........9..(. |
00000260 01 12 1f 3a 1d 0a 19 1a 09 c2 89 f7 0c b0 f8 01 |...:............|
00000270 4e 20 22 0a c2 89 f7 0c b0 f8 01 4e 20 fe 28 01 |N "........N .(.|
00000280 20 01 12 1f 3a 1d 0a 19 1a 09 c2 89 f7 0c b0 f8 | ...:...........|
00000290 01 70 c2 22 0a c2 89 f7 0c b0 f8 01 70 c2 fe 28 |.p."........p..(|
000002a0 01 20 01 |. .|
original protobuf:
00000000 0a e8 01 0a 00 12 06 08 04 10 04 18 02 18 fa 5d |...............]|
00000010 2a d4 01 0a 2f 0a 10 96 c4 13 c6 4e ef 48 aa a2 |*.../......N.H..|
00000020 0c 3c 77 5c 37 a6 f9 1a 07 bd 89 f7 0c b0 92 88 |.<w\7...........|
00000030 2a 0c 08 dd e9 f0 87 fe e8 c8 c4 15 10 1b 30 b5 |*.............0.|
00000040 b6 03 38 01 12 07 73 71 6c 20 74 78 6e 2a 0c 08 |..8...sql txn*..|
00000050 dd e9 f0 87 fe e8 c8 c4 15 10 1b 32 0c 08 dd e9 |...........2....|
00000060 f0 87 fe e8 c8 c4 15 10 1b 3a 0c 08 dd b3 a6 f6 |.........:......|
00000070 ff e8 c8 c4 15 10 1b 42 10 08 01 12 0c 08 dd e9 |.......B........|
00000080 f0 87 fe e8 c8 c4 15 10 1b 42 0e 08 02 12 0a 08 |.........B......|
00000090 87 f0 db 89 fe e8 c8 c4 15 42 10 08 03 12 0c 08 |.........B......|
000000a0 d6 ce c2 88 fe e8 c8 c4 15 10 11 42 10 08 05 12 |...........B....|
000000b0 0c 08 85 c2 c9 89 fe e8 c8 c4 15 10 45 42 10 08 |............EB..|
000000c0 06 12 0c 08 db a8 c2 8e fe e8 c8 c4 15 10 44 42 |..............DB|
000000d0 10 08 07 12 0c 08 9d a3 f5 8a fe e8 c8 c4 15 10 |................|
000000e0 47 48 01 72 00 7a 00 50 01 58 01 12 1d 3a 1b 0a |GH.r.z.P.X...:..|
000000f0 17 1a 08 c2 89 f7 0c b0 f7 30 97 22 09 c2 89 f7 |.........0."....|
00000100 0c b0 f7 30 97 fe 28 01 20 01 12 1d 3a 1b 0a 17 |...0..(. ...:...|
00000110 1a 08 c2 89 f7 0c b0 f7 34 9d 22 09 c2 89 f7 0c |........4.".....|
00000120 b0 f7 34 9d fe 28 01 20 01 12 1d 3a 1b 0a 17 1a |..4..(. ...:....|
00000130 08 c2 89 f7 0c b0 f7 40 fe 22 09 c2 89 f7 0c b0 |.......@."......|
00000140 f7 40 fe fe 28 01 20 01 12 1d 3a 1b 0a 17 1a 08 |.@..(. ...:.....|
00000150 c2 89 f7 0c b0 f7 49 22 22 09 c2 89 f7 0c b0 f7 |......I"".......|
00000160 49 22 fe 28 01 20 01 12 1d 3a 1b 0a 17 1a 08 c2 |I".(. ...:......|
00000170 89 f7 0c b0 f7 7f fa 22 09 c2 89 f7 0c b0 f7 7f |......."........|
00000180 fa fe 28 01 20 01 12 1d 3a 1b 0a 17 1a 08 c2 89 |..(. ...:.......|
00000190 f7 0c b0 f7 80 39 22 09 c2 89 f7 0c b0 f7 80 39 |.....9"........9|
000001a0 fe 28 01 20 01 12 1d 3a 1b 0a 17 1a 08 c2 89 f7 |.(. ...:........|
000001b0 0c b0 f7 80 e2 22 09 c2 89 f7 0c b0 f7 80 e2 fe |....."..........|
000001c0 28 01 20 01 12 1d 3a 1b 0a 17 1a 08 c2 89 f7 0c |(. ...:.........|
000001d0 b0 f7 86 ae 22 09 c2 89 f7 0c b0 f7 86 ae fe 28 |...."..........(|
000001e0 01 20 01 12 1d 3a 1b 0a 17 1a 08 c2 89 f7 0c b0 |. ...:..........|
000001f0 f7 c1 1e 22 09 c2 89 f7 0c b0 f7 c1 1e fe 28 01 |..."..........(.|
00000200 20 01 12 1d 3a 1b 0a 17 1a 08 c2 89 f7 0c b0 f7 | ...:...........|
00000210 d8 6f 22 09 c2 89 f7 0c b0 f7 d8 6f fe 28 01 20 |.o"........o.(. |
00000220 01 12 1f 3a 1d 0a 19 1a 09 c2 89 f7 0c b0 f8 01 |...:............|
00000230 36 30 22 0a c2 89 f7 0c b0 f8 01 36 30 fe 28 01 |60"........60.(.|
00000240 20 01 12 1f 3a 1d 0a 19 1a 09 c2 89 f7 0c b0 f8 | ...:...........|
00000250 01 39 1b 22 0a c2 89 f7 0c b0 f8 01 39 1b fe 28 |.9."........9..(|
00000260 01 20 01 12 1f 3a 1d 0a 19 1a 09 c2 89 f7 0c b0 |. ...:..........|
00000270 f8 01 4e 20 22 0a c2 89 f7 0c b0 f8 01 4e 20 fe |..N "........N .|
00000280 28 01 20 01 12 1f 3a 1d 0a 19 1a 09 c2 89 f7 0c |(. ...:.........|
00000290 b0 f8 01 70 c2 22 0a c2 89 f7 0c b0 f8 01 70 c2 |...p."........p.|
000002a0 fe 28 01 20 01 |.(. .|
original panic: <nil>
goroutine 6519665 [running]:
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).divideAndSendBatchToRanges.func1(0xc0457e62d0, 0xc0457e65c0, 0xc0457e6500, 0xc0457e65b8, 0xc0457e6237, 0xc0457e62b8, 0xc0457e623c)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:825 +0x50e
panic(0x2d6a100, 0xc05c73ab20)
/usr/local/go/src/runtime/panic.go:513 +0x1b9
github.com/cockroachdb/cockroach/pkg/kv.withMarshalingDebugging.func1(0xc05b51f580, 0x2a5, 0xc0823242c0, 0x2a5, 0x2a5)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/transport.go:190 +0x386
github.com/cockroachdb/cockroach/pkg/kv.withMarshalingDebugging(0x3a1f260, 0xc02b426030, 0x0, 0x0, 0x400000004, 0x2, 0x2efa, 0x0, 0xc02faa3c00, 0x0, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/transport.go:194 +0x129
github.com/cockroachdb/cockroach/pkg/kv.(*grpcTransport).SendNext(0xc00f1a1d70, 0x3a1f260, 0xc060e3fb30, 0x0, 0x0, 0x400000004, 0x2, 0x2efa, 0x0, 0xc02faa3c00, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/transport.go:211 +0x211
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).sendToReplicas(0xc00005a120, 0x3a1f260, 0xc060e3fb30, 0xc00005a170, 0x2efa, 0xc07970c190, 0x3, 0x3, 0x0, 0x0, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:1363 +0x2d3
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).sendRPC(0xc00005a120, 0x3a1f260, 0xc060e3fb30, 0x2efa, 0xc07970c190, 0x3, 0x3, 0x0, 0x0, 0x0, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:416 +0x244
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).sendSingleRange(0xc00005a120, 0x3a1f260, 0xc060e3fb30, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xc02faa3c00, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:496 +0x221
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).sendPartialBatch(0xc00005a120, 0x3a1f260, 0xc060e3fb30, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xc02faa3c00, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:1139 +0x322
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).divideAndSendBatchToRanges(0xc00005a120, 0x3a1f260, 0xc060e3fb30, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xc02faa3c00, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:962 +0x8b3
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).Send(0xc00005a120, 0x3a1f260, 0xc060e3fb30, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xc02a9dd600, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:710 +0x48b
github.com/cockroachdb/cockroach/pkg/kv.(*txnLockGatekeeper).SendLocked(0xc059cf2890, 0x3a1f260, 0xc060e3fb30, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xc02a9dd600, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/txn_coord_sender.go:233 +0xe8
github.com/cockroachdb/cockroach/pkg/kv.(*txnSpanRefresher).sendLockedWithRefreshAttempts(0xc059cf27c8, 0x3a1f260, 0xc060e3fb30, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xc02a9dd600, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/txn_interceptor_span_refresher.go:160 +0x83
github.com/cockroachdb/cockroach/pkg/kv.(*txnSpanRefresher).SendLocked(0xc059cf27c8, 0x3a1f260, 0xc060e3fb30, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xc02a9dd600, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/txn_interceptor_span_refresher.go:101 +0xf9
github.com/cockroachdb/cockroach/pkg/kv.(*txnPipeliner).SendLocked(0xc059cf2740, 0x3a1f260, 0xc060e3fb30, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xc02a9dd600, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/txn_interceptor_pipeliner.go:182 +0xf9
github.com/cockroachdb/cockroach/pkg/kv.(*txnSeqNumAllocator).SendLocked(0xc059cf26e8, 0x3a1f260, 0xc060e3fb30, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xc02a9dd600, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/txn_interceptor_seq_num_allocator.go:92 +0x23b
github.com/cockroachdb/cockroach/pkg/kv.(*TxnCoordSender).Send(0xc059cf2480, 0x3a1f260, 0xc060e3fb30, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/kv/txn_coord_sender.go:698 +0x591
github.com/cockroachdb/cockroach/pkg/internal/client.(*DB).sendUsingSender(0xc000819700, 0x3a1f260, 0xc060e3fad0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/internal/client/db.go:622 +0x119
github.com/cockroachdb/cockroach/pkg/internal/client.(*Txn).Send(0xc031e3b9e0, 0x3a1f260, 0xc060e3fad0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/internal/client/txn.go:790 +0x13c
github.com/cockroachdb/cockroach/pkg/sql/row.(*txnKVFetcher).fetch(0xc0636685b0, 0x3a1f260, 0xc060e3fad0, 0x302c5a0, 0xcf3798)
/go/src/github.com/cockroachdb/cockroach/pkg/sql/row/kv_batch_fetcher.go:242 +0x626
github.com/cockroachdb/cockroach/pkg/sql/row.(*txnKVFetcher).nextBatch(0xc0636685b0, 0x3a1f260, 0xc060e3fad0, 0x203003, 0xc071480988, 0xcfa284, 0xc01e6ba000, 0x362, 0x362, 0x3226b40, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/sql/row/kv_batch_fetcher.go:326 +0x1dd
github.com/cockroachdb/cockroach/pkg/sql/row.(*kvFetcher).nextKV(0xc04f807cd8, 0x3a1f260, 0xc060e3fad0, 0x3a1f260, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/sql/row/kv_fetcher.go:71 +0x2ef
github.com/cockroachdb/cockroach/pkg/sql/row.(*Fetcher).NextKey(0xc04f807ca0, 0x3a1f260, 0xc060e3fad0, 0x0, 0x0, 0x0)
/go/src/github.com/cockroachdb/cockroach/pkg/sql/row/fetcher.go:490 +0x82
github.com/cockroachdb/cockroach/pkg/sql/row.(*Fetcher).StartScanFrom(0xc04f807ca0, 0x3a1f260, 0xc060e3fad0, 0x39f9920, 0xc0636685b0, 0x0, 0xe37001)
/go/src/github.com/cockroachdb/cockroach/pkg/sql/row/fetcher.go:480 +0x97
github.com/cockroachdb/cockroach/pkg/sql/row.(*Fetcher).StartScan(0xc04f807ca0, 0x3a1f260, 0xc060e3fad0, 0xc031e3b9e0, 0xc035b01500, 0xf, 0xf, 0x0, 0x0, 0xc78300, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/sql/row/fetcher.go:471 +0x1e7
github.com/cockroachdb/cockroach/pkg/sql/distsqlrun.(*tableReader).Start(0xc04f807800, 0x3a1f1a0, 0xc04060e3c0, 0xc0470e2c60, 0xc)
/go/src/github.com/cockroachdb/cockroach/pkg/sql/distsqlrun/tablereader.go:253 +0x271
github.com/cockroachdb/cockroach/pkg/sql/distsqlrun.(*sortAllProcessor).Start(0xc03e5aef00, 0x3a1f1a0, 0xc04060e3c0, 0xd19c2f, 0xc00086c000)
/go/src/github.com/cockroachdb/cockroach/pkg/sql/distsqlrun/sorter.go:283 +0x52
github.com/cockroachdb/cockroach/pkg/sql/distsqlrun.(*ProcessorBase).Run(0xc03e5aef00, 0x3a1f1a0, 0xc04060e3c0)
/go/src/github.com/cockroachdb/cockroach/pkg/sql/distsqlrun/processors.go:800 +0x52
github.com/cockroachdb/cockroach/pkg/sql/distsqlrun.(*Flow).startInternal.func1(0xc02056fc00, 0xc0267fd830, 0x0)
/go/src/github.com/cockroachdb/cockroach/pkg/sql/distsqlrun/flow.go:564 +0x67
created by github.com/cockroachdb/cockroach/pkg/sql/distsqlrun.(*Flow).startInternal
/go/src/github.com/cockroachdb/cockroach/pkg/sql/distsqlrun/flow.go:563 +0x342
This one's a winner! Here's the unmarshaled version.
The diff between the pretty printed versions is the following:
$ diff pre post
1c1
< transport_test.go:406: int(677)
---
> transport_test.go:406: int(675)
87c87
< NodeID: 5,
---
> NodeID: 4,
89,90c89,90
< WallTime: 1551810338233671941,
< Logical: 69,
---
> WallTime: 1551810338244098644,
> Logical: 0,
98c98
< NodeID: 6,
---
> NodeID: 5,
100,101c100,101
< WallTime: 1551810338244039771,
< Logical: 68,
---
> WallTime: 1551810338233671941,
> Logical: 69,
109c109
< NodeID: 7,
---
> NodeID: 6,
111,112c111,112
< WallTime: 1551810338236486045,
< Logical: 71,
---
> WallTime: 1551810338244039771,
> Logical: 68,
That is, everything is equal except for the ObservedTimestamps
slices which hang off the *Txn
.
Concretely, before sending, the batch has ObservedTimestamps entries for n1-n3, n5-n7. After sending, it has n1-n6. The result is that the size in this case shrunk by two bytes.
All repros so far have at least three entries in ObservedTimestamps
. We don't update that in many places. This can't be too hard.
(Note that there's a real change that this aggressive assertion is just catching another bug)
@awoods187 what kind of cluster is andy-36 and which TPCC dataset had you imported and with how many warehouses were you running?
7 node 36 cpu cluster running tpc-c 10k with 10k imported
I don't see anywhere beneath DistSender.divideAndSendBatchToRanges
that unconditionally clones a BatchRequest
of BatchResponse
's transaction proto. I think that means that if the first partial batch to respond in divideAndSendBatchToRanges
went to a local range (which skips the RPC) then the BatchResponse
, which is constructed here, could end up with the same Transaction
object in its header as the pre-split BatchRequest
. This seems like it could cause issues.
Here's one possible timeline that I can imagine:
BatchRequest
is divided 5 ways.DistSender
, and sets the transaction object on the BatchResponse
. It's observed timestamps happen to be {1,2,3,5,6}
.DistSender
, and is about to update the BatchResponse
object with a new observed timestamp. It creates a new slice with the updated timestamp by calling observedTimestampSlice.update
and is about to assign it to txn.ObservedTimestamps
here.UpdateObservedTimestamp
here. Again, observedTimestampSlice.update
defines a new slice to assign to txn.ObservedTimestamps
, this time without the updated timestamp from partialReq2.txn.ObservedTimestamps
, which now looks like {1,2,3,4,5,6}
BatchRequest
to marshall while observing this newly observed timestamp{1,2,3,5,6}
.DistSender
, and updates the txn object with its observed timestamp. They now look like {1,2,3,5,6,7}
.This probably isn't exactly what happened, but letting the BatchResponse
update the same Transaction
proto that is attached to concurrent BatchRequest
s seems like it could create entire classes of these issues.
Other random observations:
ObservedTimestamps
directly in three places: upon receiving a batch in Store.Send
, upon returning an error from Store.Send
, and upon returning no error from Store.Send
. All other modifications take place through Transaction.Update
.The following workload run against a 4-node cluster of c5d.18xlarge nodes reproduces this panic reliably within 10 seconds.
./workload run kv --init --read-percent 50 --batch 64 --concurrency 512 ${PGURLS}
Fantastic, thank you Andrew!
On Thu, Mar 7, 2019, 06:05 ajwerner notifications@github.com wrote:
The following workload run against a 4-node cluster of c5d.18xlarge nodes reproduces this panic reliably within 10 seconds.
./workload run kv --init --read-percent 50 --batch 64 --concurrency 512 ${PGURLS}
— You are receiving this because you were assigned. Reply to this email directly, view it on GitHub https://github.com/cockroachdb/cockroach/issues/34241#issuecomment-470386121, or mute the thread https://github.com/notifications/unsubscribe-auth/AE135FYQUCFqPnzOELCcGWXcX6hZ07FEks5vUJ38gaJpZM4aSCK7 .
(Repro works for me)
Exciting to see this bug getting cornered.
I spent today trying to figure out why our race tests don't catch this. I'd rather fix this when there's a fast local repro. No dice yet, but I realized that we basically never run the logic tests with race=on, which may lower our chances. Playing with that a bit but also considering writing a targeted test.
I can catch this with the race detector on my laptop via
make GOFLAGS=-race build
diff --git a/pkg/kv/transport_race.go b/pkg/kv/transport_race.go
index 8c2d00c523..642cf3f26f 100644
--- a/pkg/kv/transport_race.go
+++ b/pkg/kv/transport_race.go
@@ -101,7 +101,7 @@ func (tr raceTransport) SendNext(
func GRPCTransportFactory(
opts SendOptions, nodeDialer *nodedialer.Dialer, replicas ReplicaSlice,
) (Transport, error) {
- if atomic.AddInt32(&running, 1) <= 1 {
+ if false && atomic.AddInt32(&running, 1) <= 1 {
// NB: We can't use Stopper.RunWorker because doing so would race with
// calling Stopper.Stop.
if err := nodeDialer.Stopper().RunAsyncTask(
@@ -161,5 +161,6 @@ func GRPCTransportFactory(
if err != nil {
return nil, err
}
- return &raceTransport{Transport: t}, nil
+ return t, nil
+ // return &raceTransport{Transport: t}, nil
}
diff --git a/pkg/util/protoutil/fuzz_enabled.go b/pkg/util/protoutil/fuzz_enabled.go
index 110f61c01c..908b348003 100644
--- a/pkg/util/protoutil/fuzz_enabled.go
+++ b/pkg/util/protoutil/fuzz_enabled.go
@@ -16,4 +16,4 @@
package protoutil
-const fuzzEnabled = true
+const fuzzEnabled = false // hack
(the diff just serves to make this cluster less unbearably slow)
./bin/workload run kv --init --splits 100 --read-percent 50 --batch 64 --concurrency 4 postgresql://root@localhost:26257?sslmode=disable postgresql://root@localhost:26259?sslmode=disable postgresql://root@localhost:26261?sslmode=disable postgresql://root@localhost:26263?sslmode=disable
WARNING: DATA RACE
Read at 0x00c0009d1ea0 by goroutine 392:
github.com/cockroachdb/cockroach/pkg/roachpb.(*Transaction).Size()
/Users/tschottdorf/go/src/github.com/cockroachdb/cockroach/pkg/roachpb/data.pb.go:2770 +0x572
github.com/cockroachdb/cockroach/pkg/roachpb.(*Header).Size()
/Users/tschottdorf/go/src/github.com/cockroachdb/cockroach/pkg/roachpb/api.pb.go:17513 +0x40d
github.com/cockroachdb/cockroach/pkg/roachpb.(*BatchRequest).Size()
/Users/tschottdorf/go/src/github.com/cockroachdb/cockroach/pkg/roachpb/api.pb.go:17547 +0x52
github.com/cockroachdb/cockroach/pkg/kv.withMarshalingDebugging.func1()
/Users/tschottdorf/go/src/github.com/cockroachdb/cockroach/pkg/kv/transport.go:162 +0x50
github.com/cockroachdb/cockroach/pkg/kv.withMarshalingDebugging()
/Users/tschottdorf/go/src/github.com/cockroachdb/cockroach/pkg/kv/transport.go:185 +0xd0
github.com/cockroachdb/cockroach/pkg/kv.(*grpcTransport).SendNext()
/Users/tschottdorf/go/src/github.com/cockroachdb/cockroach/pkg/kv/transport.go:202 +0x2d2
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).sendToReplicas()
/Users/tschottdorf/go/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:1363 +0x3bd
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).sendRPC()
/Users/tschottdorf/go/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:416 +0x308
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).sendSingleRange()
/Users/tschottdorf/go/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:496 +0x20c
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).sendPartialBatch()
/Users/tschottdorf/go/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:1139 +0x422
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).sendPartialBatchAsync.func1()
/Users/tschottdorf/go/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:1061 +0x1f3
github.com/cockroachdb/cockroach/pkg/util/stop.(*Stopper).RunLimitedAsyncTask.func1()
/Users/tschottdorf/go/src/github.com/cockroachdb/cockroach/pkg/util/stop/stopper.go:385 +0x11d
Previous write at 0x00c0009d1ea0 by goroutine 388:
github.com/cockroachdb/cockroach/pkg/roachpb.observedTimestampSlice.update()
/Users/tschottdorf/go/src/github.com/cockroachdb/cockroach/pkg/roachpb/data.go:1769 +0x1ff
github.com/cockroachdb/cockroach/pkg/roachpb.(*Transaction).UpdateObservedTimestamp()
/Users/tschottdorf/go/src/github.com/cockroachdb/cockroach/pkg/roachpb/data.go:1115 +0xe1
github.com/cockroachdb/cockroach/pkg/roachpb.(*Transaction).Update()
/Users/tschottdorf/go/src/github.com/cockroachdb/cockroach/pkg/roachpb/data.go:1014 +0x770
github.com/cockroachdb/cockroach/pkg/roachpb.(*BatchResponse_Header).combine()
/Users/tschottdorf/go/src/github.com/cockroachdb/cockroach/pkg/roachpb/api.go:389 +0x212
github.com/cockroachdb/cockroach/pkg/roachpb.(*BatchResponse).Combine()
/Users/tschottdorf/go/src/github.com/cockroachdb/cockroach/pkg/roachpb/batch.go:405 +0xbc
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).divideAndSendBatchToRanges.func1()
/Users/tschottdorf/go/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:853 +0x208
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).divideAndSendBatchToRanges()
/Users/tschottdorf/go/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:1030 +0x9c0
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).Send()
/Users/tschottdorf/go/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:710 +0x6db
github.com/cockroachdb/cockroach/pkg/kv.(*txnLockGatekeeper).SendLocked()
/Users/tschottdorf/go/src/github.com/cockroachdb/cockroach/pkg/kv/txn_coord_sender.go:233 +0x15f
github.com/cockroachdb/cockroach/pkg/kv.(*txnMetricRecorder).SendLocked()
/Users/tschottdorf/go/src/github.com/cockroachdb/cockroach/pkg/kv/txn_interce
Unfortunately most of the time I'm getting these, which I've also seen in Go's CI here and here; they're probably darwin only. I was on Go 1.11, now on 1.12, will see if this persists.
fatal error: unexpected signal during runtime execution
[signal SIGSEGV: segmentation violation code=0x1 addr=0x4ec00 pc=0x6f3016a]
goroutine 32083 [running]:
runtime.throw(0x7000104eccf0, 0x700000000003)
/usr/local/Cellar/go/1.11.1/libexec/src/runtime/panic.go:608 +0x72 fp=0x7000104ecc90 sp=0x7000104ecc60 pc=0x402fc32
runtime: unexpected return pc for runtime.write_trampoline called from 0x7000104eccf0
stack: frame={sp:0x7000104ecc90, fp:0x7000104ecc98} stack=[0xc00967a000,0xc00967e000)
runtime.write_trampoline(0x700000000003, 0xc0099d7000, 0x7000104ecce0, 0x4050761, 0x40638f0, 0x7000104eccf0, 0x3, 0x78d2200, 0x7f61c70, 0x7000104ecd10, ...)
/usr/local/Cellar/go/1.11.1/libexec/src/runtime/sys_darwin_amd64.s:52 fp=0x7000104ecc98 sp=0x7000104ecc90 pc=0x40638f0
created by github.com/cockroachdb/cockroach/pkg/util/stop.(*Stopper).RunLimitedAsyncTask
/Users/tschottdorf/go/src/github.com/cockroachdb/cockroach/pkg/util/stop/stopper.go:379 +0x2ae
Ugh, on Go 1.12 even getting a running cluster (on darwin) seems to be hard. I don't know what's going on there, you start the first node and everything is fine, the moment the second one joins there are lots of timeout exceeded errors and the second node freezes up (though sometimes the first one too). Don't have time for that unfortunately, let's try linux
All of this just works on linux (go 1.11, but doesn't crash) and I was able to repro the data race without the withMarshalingDebug
wrapper (which triggered it in my previous post).
So these race-enabled repros "work", but they're still super slow. @ajwerner's is the fastest I've seen.
Anyway, the following diff "fixes" it in the sense that it's been a few minutes without the panic (where before it'd last seconds). Of course this isn't really a holistic solution. When the local server optimization is active, we still end up shoving a (mostly shallow) copy of the input txn back to DistSender. Unfortunately assertions currently force us to always return a transaction response if there's one in the input (don't want to change that at this point in the cycle), and it's similarly too late for something like https://github.com/cockroachdb/cockroach/pull/21458. We could insert a txn.Clone()
in the appropriate places and eat the perf penalty if we want to feel "safe". For 19.2 I hope that we can enact a better ownership model, in which we never mutate ba.Txn
(on top of never reusing its mutable memory for response txns). However, in the short term, I think (and think I checked) that ObservedTimestamps is the only deep field that we mutate, and that everything else is handled by shallow copies. @nvanbenschoten what do you think?
commit c9a0b448579318c6302a7fceca1722b630e64f98 (HEAD -> fix/txn-race, tbg/fix/txn-race)
Author: Tobias Schottdorf <tobias.schottdorf@gmail.com>
Date: Mon Mar 11 14:11:15 2019 +0100
storage: hacky fix
Release note: None
diff --git a/pkg/storage/replica_evaluate.go b/pkg/storage/replica_evaluate.go
index 77c1302fb6..677ec6a584 100644
--- a/pkg/storage/replica_evaluate.go
+++ b/pkg/storage/replica_evaluate.go
@@ -170,6 +170,7 @@ func evaluateBatch(
if ba.Txn != nil {
txnShallow := *ba.Txn
ba.Txn = &txnShallow
+ ba.Txn.ObservedTimestamps = nil
// Check whether this transaction has been aborted, if applicable.
// This applies to writes that leave intents (the use of the
diff --git a/pkg/storage/store.go b/pkg/storage/store.go
index 3e8db1454d..7016865374 100644
--- a/pkg/storage/store.go
+++ b/pkg/storage/store.go
@@ -2825,6 +2825,7 @@ func (s *Store) Send(
// this node, in which case the following is a no-op).
if _, ok := ba.Txn.GetObservedTimestamp(ba.Replica.NodeID); !ok {
shallowTxn := *ba.Txn
+ shallowTxn.ObservedTimestamps = nil
shallowTxn.UpdateObservedTimestamp(ba.Replica.NodeID, now)
ba.Txn = &shallowTxn
}
Anyway, the following diff "fixes" it in the sense that it's been a few minutes without the panic (where before it'd last seconds).
Do you have any theory as to why this has started happening more frequently recently? Could it be due to the switch to go1.11? Something else?
Unclear. From the code I've looked at, I don't recall anything having changed between 2.1 and now. OTOH it's weird that we wouldn't have seen this before, though we never quite pushed large machines and that seems to be the ingredient you need to make it happen frequently.
I'm running this against release-2.1
now to see what will happen. If it doesn't repro there, I could bisect but it's a little unclear what we'd do with the result and whether it'd be worth the effort.
Yeah, here you go. release-2.1 @ v2.1.5-20-gf55596e:
anic: runtime error: index out of range [recovered]
panic: runtime error: index out of range
goroutine 2451889 [running]:
github.com/cockroachdb/cockroach/pkg/util/stop.(*Stopper).Recover(0xc420275d40, 0x305b0e0, 0xc42d3b9a40)
/go/src/github.com/cockroachdb/cockroach/pkg/util/stop/stopper.go:184 +0x11f
panic(0x2775c60, 0x425dd80)
/usr/local/go/src/runtime/panic.go:502 +0x229
github.com/cockroachdb/cockroach/pkg/roachpb.(*Value).MarshalTo(0xc4378656f8, 0xc43f23bb13, 0xa, 0xa, 0xc, 0x14, 0x0)
/go/src/github.com/cockroachdb/cockroach/pkg/roachpb/data.pb.go:1069 +0x21d
github.com/cockroachdb/cockroach/pkg/roachpb.(*PutRequest).MarshalTo(0xc4378656c0, 0xc43f23baff, 0x1e, 0x1e, 0x20, 0x2, 0x1)
/go/src/github.com/cockroachdb/cockroach/pkg/roachpb/api.pb.go:6905 +0x1ba
github.com/cockroachdb/cockroach/pkg/roachpb.(*RequestUnion_Put).MarshalTo(0xc43297dee0, 0xc43f23bafd, 0x20, 0x20, 0xdf1409, 0xc43297dee0, 0x22)
@RaduBerinde
Exciting to see this bug getting cornered.
just curious -- are you just 🍿 or did you run into this too?
Describe the problem
Node died while running tpc-c on a six node cluster with range merging turned off. In the shell I saw
To Reproduce
Expected behavior Completing tpc-c
Additional data / screenshots
Environment: v2.2.0-alpha.20181217-820-g645c0c9
From log to stderr https://gist.github.com/awoods187/a44b3505f7921aa152d8c0b488afdccc