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

kvnemeses: begin scaffolding for a jepsen-style kv test #43565

Merged
merged 3 commits into from Jan 22, 2020

Conversation

danhhz
Copy link
Contributor

@danhhz danhhz commented Dec 26, 2019

Package kvnemeses exercises the KV api with random traffic and then
validates that the observed behaviors are consistent with our
guarantees.

A set of Operations are generated which represent usage of the public KV
api. These include both "workload" operations like Gets and Puts as well
as "admin" operations like rebalances. These Operations can be handed to
an Applier, which runs them against the KV api and records the results.

Operations do allow for concurrency (this testing is much less
interesting otherwise), which means that the state of the KV map is not
recoverable from only the input. TODO(dan): We can use RangeFeed to
recover the exact KV history. This plus some Kyle magic can be used to
check our transactional guarantees.

TODO (in later commits)

  • Validate the log
  • CPut/InitPut/Increment/Delete
  • DeleteRange/ClearRange/RevertRange/Scan/ReverseScan
  • ChangeReplicas/TransferLease
  • ExportRequest
  • AddSSTable
  • Root and leaf transactions
  • GCRequest
  • Protected timestamps

Release note: None

@cockroach-teamcity
Copy link
Member

This change is Reviewable

Copy link
Contributor

@ajwerner ajwerner left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Are you trying to merge this in roughly this state or do you envision adding more functionality before trying to get this in?

Reviewed 1 of 9 files at r1.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @ajwerner and @nvanbenschoten)

@danhhz
Copy link
Contributor Author

danhhz commented Jan 2, 2020

I have some more locally that sets up a changefeed and uses it to do validation. My plan for today is to get that cleaned up and out for review. Feel free to look at the high level structure now if you like. If there's any big changes to make, it'll be easier while I'm in the code today

Copy link
Contributor

@ajwerner ajwerner left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewed 1 of 6 files at r2.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @ajwerner, @danhhz, and @nvanbenschoten)


pkg/kv/kvnemeses/operations.proto, line 70 at r2 (raw file):

  option (gogoproto.onlyone) = true;

  BatchOperation batch = 1;

How come you're just putting all of the different operation types directly into Operation as opposed to using a oneOf?

Copy link
Contributor Author

@danhhz danhhz left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @ajwerner and @nvanbenschoten)


pkg/kv/kvnemeses/operations.proto, line 70 at r2 (raw file):

Previously, ajwerner wrote…

How come you're just putting all of the different operation types directly into Operation as opposed to using a oneOf?

It's using option (gogoproto.onlyone) = true which is similar in spirit but I find the generated code much easier to use. YMMV so if you like oneof better I'm all ears

@danhhz
Copy link
Contributor Author

danhhz commented Jan 6, 2020

Okay, this is still missing a bit of polish, but it's ready for an initial review.

The one remaining large piece is something that checks our transaction guarantees given the set of operations and a complete kv history. I was originally thinking that we need to do something complicated, but if we have the actual kv history with timestamps (we do) and all writes are of a unique value (they are) then we can easily construct the actual ordering of all read-write transactions. I assume it's then not too hard to fit in read-only transactions.

@danhhz
Copy link
Contributor Author

danhhz commented Jan 6, 2020

The test seems to keep hanging under stress. The interesting stacktrace seems to always be in txnwait.Queue.MaybeWaitForPush. Any ideas where to start debugging it? What's strange is the select doesn't have a time next to it (so it hasn't been waiting long, right?) but I also don't see the "pusher: ..." log messages in the logs.

goroutine 1099 [select]:
github.com/cockroachdb/cockroach/pkg/storage/txnwait.(*Queue).MaybeWaitForPush(0xc000b121e0, 0x7cbe7c0, 0xc000794930, 0x7c53fc0, 0xc00099b000, 0xc00094e1c0, 0x0, 0x0)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/storage/txnwait/queue.go:509 +0xf6b
github.com/cockroachdb/cockroach/pkg/storage.(*Replica).maybeWaitForPushee(0xc00099b000, 0x7cbe7c0, 0xc000794930, 0xc001a04280, 0xc0016db860, 0xc0015e3440)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/storage/store_send.go:246 +0xe9
github.com/cockroachdb/cockroach/pkg/storage.(*Replica).executeBatchWithConcurrencyRetries(0xc00099b000, 0x7cbe7c0, 0xc000794930, 0xc001a04280, 0x7540d28, 0x0, 0x0)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/storage/replica_send.go:185 +0x184
github.com/cockroachdb/cockroach/pkg/storage.(*Replica).sendWithRangeID(0xc00099b000, 0x7cbe7c0, 0xc000794900, 0x1c, 0xc001a04280, 0x0, 0x0)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/storage/replica_send.go:95 +0x679
github.com/cockroachdb/cockroach/pkg/storage.(*Replica).Send(0xc00099b000, 0x7cbe7c0, 0xc000794900, 0x15e76ffc45511a68, 0x0, 0x100000001, 0x1, 0x0, 0x1c, 0x0, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/storage/replica_send.go:37 +0x91
github.com/cockroachdb/cockroach/pkg/storage.(*Store).Send(0xc0005ae000, 0x7cbe7c0, 0xc0007948a0, 0x15e76ffc45511a68, 0x0, 0x100000001, 0x1, 0x0, 0x1c, 0x0, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/storage/store_send.go:197 +0x63b
github.com/cockroachdb/cockroach/pkg/storage.(*Stores).Send(0xc000326000, 0x7cbe7c0, 0xc0007948a0, 0x15e76ffc45511a68, 0x0, 0x100000001, 0x1, 0x0, 0x1c, 0x0, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/storage/stores.go:181 +0xce
github.com/cockroachdb/cockroach/pkg/server.(*Node).batchInternal.func1(0x7cbe7c0, 0xc0007948a0, 0x0, 0x0)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/server/node.go:934 +0x1e8
github.com/cockroachdb/cockroach/pkg/util/stop.(*Stopper).RunTaskWithErr(0xc0003e7b90, 0x7cbe7c0, 0xc0007948a0, 0x73f3f46, 0x10, 0xc0019458f0, 0x0, 0x0)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/util/stop/stopper.go:301 +0xeb
github.com/cockroachdb/cockroach/pkg/server.(*Node).batchInternal(0xc000223900, 0x7cbe7c0, 0xc0007948a0, 0xc001a04200, 0xc0007948a0, 0xc00095eb70, 0x0)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/server/node.go:922 +0x195
github.com/cockroachdb/cockroach/pkg/server.(*Node).Batch(0xc000223900, 0x7cbe7c0, 0xc000794870, 0xc001a04200, 0xc00095f3b0, 0x0, 0x0)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/server/node.go:960 +0x9f
github.com/cockroachdb/cockroach/pkg/rpc.internalClientAdapter.Batch(...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/rpc/context.go:536
github.com/cockroachdb/cockroach/pkg/kv.(*grpcTransport).sendBatch(0xc00159e6c0, 0x7cbe7c0, 0xc000794870, 0x1, 0x7c84840, 0xc000209470, 0x15e76ffc45511a68, 0x0, 0x100000001, 0x1, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/kv/transport.go:199 +0x128
github.com/cockroachdb/cockroach/pkg/kv.(*grpcTransport).SendNext(0xc00159e6c0, 0x7cbe7c0, 0xc00095f3b0, 0x15e76ffc45511a68, 0x0, 0x100000001, 0x1, 0x0, 0x1c, 0x0, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/kv/transport.go:168 +0x15d
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).sendToReplicas(0xc00020efc0, 0x7cbe7c0, 0xc00095f3b0, 0x15e76ffc45511a68, 0x0, 0x0, 0x0, 0x0, 0x1c, 0x0, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:1653 +0x2a3
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).sendRPC(0xc00020efc0, 0x7cbe7c0, 0xc00095f3b0, 0x15e76ffc45511a68, 0x0, 0x0, 0x0, 0x0, 0x1c, 0x0, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:440 +0x196
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).sendSingleRange(0xc00020efc0, 0x7cbe7c0, 0xc00095f3b0, 0x15e76ffc45511a68, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:522 +0x2f2
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).sendPartialBatch(0xc00020efc0, 0x7cbe7c0, 0xc00095f3b0, 0x15e76ffc45511a68, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:1424 +0x3ce
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).divideAndSendBatchToRanges(0xc00020efc0, 0x7cbe7c0, 0xc00095f3b0, 0x15e76ffc45511a68, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:1090 +0x15d1
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).Send(0xc00020efc0, 0x7cbe7c0, 0xc00095f3b0, 0x15e76ffc45511a68, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:736 +0xd39
github.com/cockroachdb/cockroach/pkg/internal/client.(*CrossRangeTxnWrapperSender).Send(0xc0000281e0, 0x7cbe7c0, 0xc00095f3b0, 0x15e76ffc45511a68, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/internal/client/db.go:223 +0xaf
github.com/cockroachdb/cockroach/pkg/internal/client.(*DB).sendUsingSender(0xc000028180, 0x7cbe7c0, 0xc00095f3b0, 0x15e76ffc45511a68, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/internal/client/db.go:754 +0x122
github.com/cockroachdb/cockroach/pkg/internal/client.(*DB).send(...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/internal/client/db.go:736
github.com/cockroachdb/cockroach/pkg/internal/client.sendAndFill(0x7cbe7c0, 0xc00095f3b0, 0xc000f8f4f0, 0xc0011c2000, 0x0, 0xc000f8fb90)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/internal/client/db.go:672 +0xf7
github.com/cockroachdb/cockroach/pkg/internal/client.(*DB).Run(0xc000028180, 0x7cbe7c0, 0xc00095f3b0, 0xc0011c2000, 0x19, 0xc000f8f718)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/internal/client/db.go:695 +0x9c
github.com/cockroachdb/cockroach/pkg/storage/intentresolver.(*IntentResolver).MaybePushTransactions(0xc000962ea0, 0x7cbe7c0, 0xc00095f3b0, 0xc000f8fb90, 0x15e76ffc454df1d0, 0x0, 0x100000001, 0x1, 0x0, 0x1d, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/storage/intentresolver/intent_resolver.go:435 +0x778
github.com/cockroachdb/cockroach/pkg/storage/intentresolver.(*IntentResolver).maybePushIntents(0xc000962ea0, 0x7cbe7c0, 0xc00095f3b0, 0xc000f3c900, 0x1, 0x1, 0x15e76ffc454df1d0, 0x0, 0x100000001, 0x1, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/storage/intentresolver/intent_resolver.go:337 +0x25d
github.com/cockroachdb/cockroach/pkg/storage/intentresolver.(*IntentResolver).ProcessWriteIntentError(0xc000962ea0, 0x7cbe7c0, 0xc00095f3b0, 0xc0011a0f50, 0x15e76ffc454df1d0, 0x0, 0x100000001, 0x1, 0x0, 0x1d, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/storage/intentresolver/intent_resolver.go:254 +0x148
github.com/cockroachdb/cockroach/pkg/storage.(*Replica).handleWriteIntentError(0xc00059e800, 0x7cbe7c0, 0xc00095f3b0, 0xc000b0c700, 0xc0011a0f50, 0xc000426be0, 0x0, 0xc0011a0f50, 0x7cbe7c0)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/storage/replica_send.go:281 +0x149
github.com/cockroachdb/cockroach/pkg/storage.(*Replica).executeBatchWithConcurrencyRetries(0xc00059e800, 0x7cbe7c0, 0xc00095f3b0, 0xc000b0c700, 0x7540d28, 0x0, 0xc0011a0f50)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/storage/replica_send.go:205 +0x3f0
github.com/cockroachdb/cockroach/pkg/storage.(*Replica).sendWithRangeID(0xc00059e800, 0x7cbe7c0, 0xc00095f380, 0x1d, 0xc000b0c700, 0x0, 0x0)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/storage/replica_send.go:95 +0x679
github.com/cockroachdb/cockroach/pkg/storage.(*Replica).Send(0xc00059e800, 0x7cbe7c0, 0xc00095f380, 0x15e76ffc454df1d0, 0x0, 0x100000001, 0x1, 0x0, 0x1d, 0x0, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/storage/replica_send.go:37 +0x91
github.com/cockroachdb/cockroach/pkg/storage.(*Store).Send(0xc0005ae000, 0x7cbe7c0, 0xc00095f2f0, 0x15e76ffc454df1d0, 0x0, 0x100000001, 0x1, 0x0, 0x1d, 0x0, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/storage/store_send.go:197 +0x63b
github.com/cockroachdb/cockroach/pkg/storage.(*Stores).Send(0xc000326000, 0x7cbe7c0, 0xc00095f2f0, 0x0, 0x0, 0x100000001, 0x1, 0x0, 0x1d, 0x0, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/storage/stores.go:181 +0xce
github.com/cockroachdb/cockroach/pkg/server.(*Node).batchInternal.func1(0x7cbe7c0, 0xc00095f2f0, 0x0, 0x0)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/server/node.go:934 +0x1e8
github.com/cockroachdb/cockroach/pkg/util/stop.(*Stopper).RunTaskWithErr(0xc0003e7b90, 0x7cbe7c0, 0xc00095f2f0, 0x73f3f46, 0x10, 0xc000f90e40, 0x0, 0x0)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/util/stop/stopper.go:301 +0xeb
github.com/cockroachdb/cockroach/pkg/server.(*Node).batchInternal(0xc000223900, 0x7cbe7c0, 0xc00095f2f0, 0xc000b0c680, 0xc00095f2f0, 0x80, 0x80)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/server/node.go:922 +0x195
github.com/cockroachdb/cockroach/pkg/server.(*Node).Batch(0xc000223900, 0x7cbe7c0, 0xc00095f2c0, 0xc000b0c680, 0xc00095eb70, 0x0, 0x0)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/server/node.go:960 +0x9f
github.com/cockroachdb/cockroach/pkg/rpc.internalClientAdapter.Batch(...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/rpc/context.go:536
github.com/cockroachdb/cockroach/pkg/kv.(*grpcTransport).sendBatch(0xc0012e6340, 0x7cbe7c0, 0xc00095f2c0, 0x1, 0x7c84840, 0xc000209470, 0x0, 0x0, 0x100000001, 0x1, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/kv/transport.go:199 +0x128
github.com/cockroachdb/cockroach/pkg/kv.(*grpcTransport).SendNext(0xc0012e6340, 0x7cbe7c0, 0xc00095eb70, 0x0, 0x0, 0x100000001, 0x1, 0x0, 0x1d, 0x0, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/kv/transport.go:168 +0x15d
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).sendToReplicas(0xc00020efc0, 0x7cbe7c0, 0xc00095eb70, 0x0, 0x0, 0x0, 0x0, 0x0, 0x1d, 0x0, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:1653 +0x2a3
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).sendRPC(0xc00020efc0, 0x7cbe7c0, 0xc00095eb70, 0x0, 0x0, 0x0, 0x0, 0x0, 0x1d, 0x0, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:440 +0x196
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).sendSingleRange(0xc00020efc0, 0x7cbe7c0, 0xc00095eb70, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:522 +0x2f2
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).sendPartialBatch(0xc00020efc0, 0x7cbe7c0, 0xc00095eb70, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:1424 +0x3ce
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).divideAndSendBatchToRanges(0xc00020efc0, 0x7cbe7c0, 0xc00095eb70, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:1090 +0x15d1
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).sendPartialBatch(0xc00020efc0, 0x7cbe7c0, 0xc00095eb70, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:1490 +0x9c9
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).divideAndSendBatchToRanges(0xc00020efc0, 0x7cbe7c0, 0xc00095eb70, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:1090 +0x15d1
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).sendPartialBatch(0xc00020efc0, 0x7cbe7c0, 0xc00095eb70, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:1490 +0x9c9
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).divideAndSendBatchToRanges(0xc00020efc0, 0x7cbe7c0, 0xc00095eb70, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:1090 +0x15d1
github.com/cockroachdb/cockroach/pkg/kv.(*DistSender).Send(0xc00020efc0, 0x7cbe700, 0xc001026600, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/kv/dist_sender.go:736 +0xd39
github.com/cockroachdb/cockroach/pkg/internal/client.(*CrossRangeTxnWrapperSender).Send(0xc0000281e0, 0x7cbe700, 0xc001026600, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/internal/client/db.go:223 +0xaf
github.com/cockroachdb/cockroach/pkg/internal/client.(*DB).sendUsingSender(0xc000028180, 0x7cbe700, 0xc001026600, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, ...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/internal/client/db.go:754 +0x122
github.com/cockroachdb/cockroach/pkg/internal/client.(*DB).send(...)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/internal/client/db.go:736
github.com/cockroachdb/cockroach/pkg/internal/client.sendAndFill(0x7cbe700, 0xc001026600, 0xc0014bbb90, 0xc0015a4580, 0x8, 0x8)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/internal/client/db.go:672 +0xf7
github.com/cockroachdb/cockroach/pkg/internal/client.(*DB).Run(0xc000028180, 0x7cbe700, 0xc001026600, 0xc0015a4580, 0xc000314280, 0x0)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/internal/client/db.go:695 +0x9c
github.com/cockroachdb/cockroach/pkg/internal/client.(*DB).Put(0xc000028180, 0x7cbe700, 0xc001026600, 0x6e5f520, 0xc000314260, 0x6e5f520, 0xc000314280, 0xc0018e2f50, 0xc00035d7a0)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/internal/client/db.go:358 +0x9e
github.com/cockroachdb/cockroach/pkg/kv/kvnemeses.applyBatchOp(0x7cbe700, 0xc001026600, 0x7c93a40, 0xc000028180, 0xc0017025a0)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/kv/kvnemeses/applier.go:131 +0xa5f
github.com/cockroachdb/cockroach/pkg/kv/kvnemeses.(*Applier).applyOp(0xc0003aba00, 0x7cbe700, 0xc001026600, 0xc0017025a0)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/kv/kvnemeses/applier.go:67 +0x24e
github.com/cockroachdb/cockroach/pkg/kv/kvnemeses.(*Applier).Apply.func1(0x7cbe700, 0xc001026600, 0x0, 0x0)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/kv/kvnemeses/applier.go:53 +0xb6
github.com/cockroachdb/cockroach/pkg/util/ctxgroup.Group.GoCtx.func1(0x8, 0x7544740)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/pkg/util/ctxgroup/ctxgroup.go:166 +0x3a
github.com/cockroachdb/cockroach/vendor/golang.org/x/sync/errgroup.(*Group).Go.func1(0xc001098450, 0xc001098480)
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/vendor/golang.org/x/sync/errgroup/errgroup.go:57 +0x57
created by github.com/cockroachdb/cockroach/vendor/golang.org/x/sync/errgroup.(*Group).Go
	/Users/dan/crdb/src/github.com/cockroachdb/cockroach/vendor/golang.org/x/sync/errgroup/errgroup.go:54 +0x66

@danhhz
Copy link
Contributor Author

danhhz commented Jan 7, 2020

Oh, of course. We actually have an outstanding transaction. The way a Step is constructed assumes that txns will get pushed as necessary, I think because I still have an outdated mental model. I wonder what my options are here

@danhhz
Copy link
Contributor Author

danhhz commented Jan 7, 2020

Discussed with nathan offline what the options were here. One idea was reacting to the OnPusherBlocked testing knob by waiting for a bit and then manually pushing. I have this working but it seems both gross and brittle.

Nathan was generally skeptical of the current step+operation model and seemed generally in favor of a more traditional worker threads running concurrently model. I'm coming around to this opinion...

Copy link
Contributor

@ajwerner ajwerner left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nathan was generally skeptical of the current step+operation model and seemed generally in favor of a more traditional worker threads running concurrently model. I'm coming around to this opinion...

I'd love to hear more about the thought process here.

Reviewed 1 of 1 files at r5, 6 of 7 files at r7, 2 of 14 files at r8.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @ajwerner and @nvanbenschoten)


pkg/kv/kvnemeses/applier.go, line 108 at r8 (raw file):

}

type batchI interface {

batchI sort of confuses me as a name. How about clientI or dbI?


pkg/kv/kvnemeses/stepper.go, line 113 at r8 (raw file):

	ops := make([]Operation, numOps)
	for idx := range ops {
		switch rng.Intn(5) {

totally bikeshedding nit but something feels wrong to me about this switch and Intn pattern you've adopted. How do you feel about the following construct:

type opGenFunc func(*Stepper, *rand.Rand) Operation

func genOp(s *Stepper, rng *rand.Rand, choices []opGenFunc) Operation {
    return choices[rng.Intn(len(choices)](s, rng)
}

Then this switch can be:

var stepChoices = []opGenFunc{
      (*Stepper).randBatch,
      (*Stepper).randTxnOp,
      (*Stepper).randOp,
      (*Stepper).randSplit,
      (*Stepper).randMerge,
}

...

for idx := range ops {
    ops[idx] = genOp(s, rng, stepChoices)
}

and randOp() can be:

var randOpChoices = []opGenFunc{
    (*Stepper).randGet,
    (*Stepper).randPut,
}

...
return genOp(s, rng, randOpChoices)

pkg/kv/kvnemeses/watcher.go, line 72 at r8 (raw file):

// Finish tears down the Watcher and returns all the kvs it has ingested.
func (w *Watcher) Finish() *Engine {

Want to add a note that this is safe to call multiple times but cannot be called concurrently?

@danhhz danhhz force-pushed the kvnemeses branch 2 times, most recently from f6f3879 to 5c374a7 Compare January 10, 2020 20:06
Copy link
Contributor Author

@danhhz danhhz left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

RFAL. I want to take another pass at the package godoc, but I think this is ready to go

I'd love to hear more about the thought process here.

I was hoping that the "all concurrency is within a step" model would make it easier to reason about histories, but I just kept hitting deadlocks due to contention causing blocking (trying to put/get a key that had been modified by a txn which wasn't committing in this step). The "many actors concurrently grabbing steps" model that's here now also more closely matches our actual usage. I still find something unsatisfying with where I've ended up, especially in this revision where the separate Begin/Use/Commit/Rollback stuff got pulled out in favor of only using the db.Txn closure. What we had before seems to me to be much more likely to generate interesting histories.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @ajwerner and @nvanbenschoten)


pkg/kv/kvnemeses/applier.go, line 108 at r8 (raw file):

Previously, ajwerner wrote…

batchI sort of confuses me as a name. How about clientI or dbI?

I like clientI. Done


pkg/kv/kvnemeses/stepper.go, line 113 at r8 (raw file):

Previously, ajwerner wrote…

totally bikeshedding nit but something feels wrong to me about this switch and Intn pattern you've adopted. How do you feel about the following construct:

type opGenFunc func(*Stepper, *rand.Rand) Operation

func genOp(s *Stepper, rng *rand.Rand, choices []opGenFunc) Operation {
    return choices[rng.Intn(len(choices)](s, rng)
}

Then this switch can be:

var stepChoices = []opGenFunc{
      (*Stepper).randBatch,
      (*Stepper).randTxnOp,
      (*Stepper).randOp,
      (*Stepper).randSplit,
      (*Stepper).randMerge,
}

...

for idx := range ops {
    ops[idx] = genOp(s, rng, stepChoices)
}

and randOp() can be:

var randOpChoices = []opGenFunc{
    (*Stepper).randGet,
    (*Stepper).randPut,
}

...
return genOp(s, rng, randOpChoices)

Yeah, this was always meant to be temporary. I've been imagining something more like how the changefeed nemeses works where the overall relative proportions are passed in and the test selects between which whichever ones are contextually appropriate. Hows this?

I've come to the reluctant conclusion that Stepper needs to avoid including two writes for the same key in a single batch/txn. (Because it breaks our assumptions that every Put shows up in kv, plus it errors when done in a batch.) When I pushed this revision, I was thinking this extra bookkeeping prevented me from doing the opGenFunc factorization, but I think that it's actually still possible and even makes the usedKey stuff more clear. I'll include this in the next revision.


pkg/kv/kvnemeses/watcher.go, line 72 at r8 (raw file):

Previously, ajwerner wrote…

Want to add a note that this is safe to call multiple times but cannot be called concurrently?

Done

Copy link
Collaborator

@petermattis petermattis left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @ajwerner, @danhhz, and @nvanbenschoten)


pkg/kv/kvnemeses/applier_test.go, line 60 at r11 (raw file):

{
  b := &Batch{}
  b.Put(ctx, "b", 2) -> nil

(Drive-by comments).

Are you planning to parse this textual representation? If yes, the -> nil will make parsing a little more difficult. You can borrow the technique used in the Pebble metamorphic test and put the result in a comment at the end of the line: // <nil> (or something like that).

If you're not planning to parse this textual representation, then what is the purpose of the ctx argument?

Do you have any thoughts on allowing test runs to be re-executed? I realize there is non-determinism due to concurrency, but I bet there will still be value in being able to re-run exactly the set of operations with exactly the set of workers that was used in a failed run.

Copy link
Contributor Author

@danhhz danhhz left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @ajwerner, @nvanbenschoten, and @petermattis)


pkg/kv/kvnemeses/applier_test.go, line 60 at r11 (raw file):

Previously, petermattis (Peter Mattis) wrote…

(Drive-by comments).

Are you planning to parse this textual representation? If yes, the -> nil will make parsing a little more difficult. You can borrow the technique used in the Pebble metamorphic test and put the result in a comment at the end of the line: // <nil> (or something like that).

If you're not planning to parse this textual representation, then what is the purpose of the ctx argument?

Do you have any thoughts on allowing test runs to be re-executed? I realize there is non-determinism due to concurrency, but I bet there will still be value in being able to re-run exactly the set of operations with exactly the set of workers that was used in a failed run.

Yeah, this string representation is part of my plan to make it easy to re-execute it when something happens. What does pebble metamorphic do with the errors that are returned by the various calls? I've been struggling with the db.Txn closure, where it's illegal to continue to use the txn after some operation returns an error, so I don't want to just drop them, but printing them is really noisy

Copy link
Collaborator

@petermattis petermattis left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @ajwerner, @danhhz, and @nvanbenschoten)


pkg/kv/kvnemeses/applier_test.go, line 60 at r11 (raw file):

Previously, danhhz (Daniel Harrison) wrote…

Yeah, this string representation is part of my plan to make it easy to re-execute it when something happens. What does pebble metamorphic do with the errors that are returned by the various calls? I've been struggling with the db.Txn closure, where it's illegal to continue to use the txn after some operation returns an error, so I don't want to just drop them, but printing them is really noisy

The Pebble test has two similar formats. The first is the ops, without results:

Init(45 /* batches */, 43 /* iters */, 42 /* snapshots */)
db.Delete("orqbhr")
db.Get("orqbhr")
db.Delete("orqbhr")
db.Set("aihwir", "zmsegmk")
db.Set("wrqhtxrb", "cbjjefsgvchypbdwlwp")
db.Get("aihwir")
db.Merge("wrqhtxrb", "qwgolijxtdywtk")

After execution, this becomes something like:

// INFO: [JOB 1] MANIFEST created 000001
// INFO: [JOB 1] WAL created 000002
db.Open() // <nil> #1
Init(45 /* batches */, 43 /* iters */, 42 /* snapshots */) #2
db.Delete("orqbhr") // <nil> #3
db.Get("orqbhr") // [""] pebble: not found #4
db.Delete("orqbhr") // <nil> #5
db.Set("aihwir", "zmsegmk") // <nil> #6
db.Set("wrqhtxrb", "cbjjefsgvchypbdwlwp") // <nil> #7
db.Get("aihwir") // ["zmsegmk"] <nil> #8
db.Merge("wrqhtxrb", "qwgolijxtdywtk") // <nil> #9

Strip out the comments and the second format is the almost identical to the first. In hindsight, I don't ever use that functionality, though. I only ever parse the first format. Something you're not seeing in the snippets here are how "objects" are dealt with. Each is given a unique name:

snap0 = db.NewSnapshot()
batch0 = db.NewBatch()
snap1 = db.NewSnapshot()
batch1 = db.NewIndexedBatch()
snap2 = db.NewSnapshot()
iter0 = db.NewIter("", "gmaidfbwe")
batch2 = db.NewBatch()
iter1 = snap0.NewIter("", "")
batch3 = db.NewIndexedBatch()
iter2 = snap2.NewIter("", "wrqhtxrb")

The Pebble test doesn't perform any concurrent operations. I haven't thought what the syntax for specifying that should be. Perhaps each worker should have a separate set of operations to execute. Or perhaps there should be blocks of operations that are executed concurrently where each operation is tagged with the worker to execute it on. I haven't really grok'ed all the requirements you're operating under.

@danhhz danhhz requested a review from tbg January 13, 2020 15:31
@danhhz
Copy link
Contributor Author

danhhz commented Jan 13, 2020

@tbg If you're looking for things to review, I had originally intended this one for you, but I didn't get it out in time.

Copy link
Member

@nvanbenschoten nvanbenschoten left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This mostly looks like I was expecting. Nice job putting this all together! My biggest question at this point is how you're planning to validate the log when you only have a partial order on the execution time of each operation.

Reviewed 22 of 22 files at r9, 7 of 7 files at r10, 17 of 17 files at r11.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @danhhz and @tbg)


pkg/kv/kvnemeses/applier.go, line 27 at r11 (raw file):

	db *client.DB
	mu struct {
		syncutil.Mutex

I find it strange that Applier provides internal synchronization but Stepper doesn't. Wouldn't it be cleaner to pull the mutex you have in kvnemesis.go into Stepper?


pkg/kv/kvnemeses/applier.go, line 61 at r11 (raw file):

		applyBatchOp(ctx, a.db, op)
	case *SplitOperation:
		expiration := a.db.Clock().Now().Add(time.Hour.Nanoseconds(), 0)

Do we want an expiration? I'd expect us to pass hlc.MaxTimestamp.


pkg/kv/kvnemeses/engine.go, line 44 at r11 (raw file):

	var value roachpb.Value
	e.kvs.AscendGreaterOrEqual(
		btreeItem(engine.MVCCKeyValue{Key: engine.MVCCKey{Key: key, Timestamp: ts}}),

nit throughout this file: do you need the cast or can you just construct a btreeItem directly? https://play.golang.org/p/SoWYEGpIRk3


pkg/kv/kvnemeses/kvnemeses.go, line 81 at r11 (raw file):

	var steps []Step
	for step := range stepCh {
		steps = append(steps, *step)

Are we ok with steps containing inversions of the execution order? For instance, I don't see any reason why two workers couldn't each write to key A in order 1, 2 but end up in steps in the order put(2), put(1).


pkg/kv/kvnemeses/stepper.go, line 121 at r11 (raw file):

	if prev != nil {
		switch t := prev.Op.GetValue().(type) {
		case *MergeOperation:

If we're going to do this then does it make sense to just update currentSplits when we hand out a MergeOperation?


pkg/kv/kvnemeses/stepper.go, line 164 at r11 (raw file):

func (s *Stepper) randGetMissing(rng *rand.Rand) Operation {
	return get(randKey(rng))

We're not making any assumptions that this key actually is missing, right? If so then we'll need to confirm that randKey is not in s.keys.


pkg/kv/kvnemeses/stepper.go, line 173 at r11 (raw file):

func (s *Stepper) randPutMissing(rng *rand.Rand, usedKeys map[string]struct{}) Operation {
	value := `v-` + strconv.Itoa(s.nextValue)

nit: consider pulling these two lines into a method.


pkg/kv/kvnemeses/stepper.go, line 194 at r11 (raw file):

func (s *Stepper) randSplit(rng *rand.Rand) Operation {
	if len(s.currentSplits) == 0 || rng.Intn(100) < 10 {

Why do we make this first split harder?


pkg/kv/kvnemeses/stepper.go, line 204 at r11 (raw file):

func (s *Stepper) randMerge(rng *rand.Rand) Operation {
	if len(s.currentSplits) == 0 || rng.Intn(100) < 10 {

This is supposed to be analogous to a randMergeMissing operation with 1/10th the probability of a randMergeExisting, right? Do we want to just add that as a separate op? That will also allow us to test missing merges before all split points get merged away.


pkg/kv/kvnemeses/stepper.go, line 211 at r11 (raw file):

}

func (s *Stepper) randBatch(rng *rand.Rand, usedKeys map[string]struct{}) Operation {

Can usedKeys be nil here?


pkg/kv/kvnemeses/validator.go, line 29 at r11 (raw file):

// Validate checks for violations of our kv api guarantees. The Steps must all
// have been applied and the kvs the result of those applications.
func Validate(steps []Step, kvs *Engine) []error {

Am I missing something or do we never actually call this outside of tests? Is that what you mean by "TODO: Validate the log"?

Copy link
Contributor Author

@danhhz danhhz left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

My biggest question at this point is how you're planning to validate the log when you only have a partial order on the execution time of each operation.

I tried to address this in a comment in Validate, but I very well may be missing something

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten, @petermattis, and @tbg)


pkg/kv/kvnemeses/applier.go, line 27 at r11 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

I find it strange that Applier provides internal synchronization but Stepper doesn't. Wouldn't it be cleaner to pull the mutex you have in kvnemesis.go into Stepper?

I find it much easier to not have to worry about locking in Stepper. The locking has to encompass from where we construct the candidate ops until after we've finished calculating the op type we've selected, which is the entirety of the only exported method on Stepper. I agree that the inline mu struct in RunNemeses is odd, what are your thoughts about a ConcurrentStepper wrapper that handles the locking?


pkg/kv/kvnemeses/applier.go, line 61 at r11 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Do we want an expiration? I'd expect us to pass hlc.MaxTimestamp.

Done


pkg/kv/kvnemeses/applier_test.go, line 60 at r11 (raw file):

Previously, petermattis (Peter Mattis) wrote…

The Pebble test has two similar formats. The first is the ops, without results:

Init(45 /* batches */, 43 /* iters */, 42 /* snapshots */)
db.Delete("orqbhr")
db.Get("orqbhr")
db.Delete("orqbhr")
db.Set("aihwir", "zmsegmk")
db.Set("wrqhtxrb", "cbjjefsgvchypbdwlwp")
db.Get("aihwir")
db.Merge("wrqhtxrb", "qwgolijxtdywtk")

After execution, this becomes something like:

// INFO: [JOB 1] MANIFEST created 000001
// INFO: [JOB 1] WAL created 000002
db.Open() // <nil> #1
Init(45 /* batches */, 43 /* iters */, 42 /* snapshots */) #2
db.Delete("orqbhr") // <nil> #3
db.Get("orqbhr") // [""] pebble: not found #4
db.Delete("orqbhr") // <nil> #5
db.Set("aihwir", "zmsegmk") // <nil> #6
db.Set("wrqhtxrb", "cbjjefsgvchypbdwlwp") // <nil> #7
db.Get("aihwir") // ["zmsegmk"] <nil> #8
db.Merge("wrqhtxrb", "qwgolijxtdywtk") // <nil> #9

Strip out the comments and the second format is the almost identical to the first. In hindsight, I don't ever use that functionality, though. I only ever parse the first format. Something you're not seeing in the snippets here are how "objects" are dealt with. Each is given a unique name:

snap0 = db.NewSnapshot()
batch0 = db.NewBatch()
snap1 = db.NewSnapshot()
batch1 = db.NewIndexedBatch()
snap2 = db.NewSnapshot()
iter0 = db.NewIter("", "gmaidfbwe")
batch2 = db.NewBatch()
iter1 = snap0.NewIter("", "")
batch3 = db.NewIndexedBatch()
iter2 = snap2.NewIter("", "wrqhtxrb")

The Pebble test doesn't perform any concurrent operations. I haven't thought what the syntax for specifying that should be. Perhaps each worker should have a separate set of operations to execute. Or perhaps there should be blocks of operations that are executed concurrently where each operation is tagged with the worker to execute it on. I haven't really grok'ed all the requirements you're operating under.

That's helpful, thanks! This has inspired me to keep track the executed steps separately per worker, which makes it possible to print them as something like the following when there is a failure, which maybe helps with writing a repro. Perhaps I should be doing what pebble is doing and parsing them when stressing for a repro, but I'd like to hold off on writing a parser for v1 of this test.

g := ctxgroup.WithContext(ctx)
g.GoCtx(func(ctx context.Context) error {
  // worker 1's operations
  ...
  return nil
})
g.GoCtx(func(ctx context.Context) error {
  // worker 2's operations
  ...
  return nil
})
g.Wait()

pkg/kv/kvnemeses/engine.go, line 44 at r11 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

nit throughout this file: do you need the cast or can you just construct a btreeItem directly? https://play.golang.org/p/SoWYEGpIRk3

TIL done


pkg/kv/kvnemeses/kvnemeses.go, line 81 at r11 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Are we ok with steps containing inversions of the execution order? For instance, I don't see any reason why two workers couldn't each write to key A in order 1, 2 but end up in steps in the order put(2), put(1).

I now keep these in order per worker


pkg/kv/kvnemeses/kvnemeses_test.go, line 37 at r11 (raw file):

	sqlDB := sqlutils.MakeSQLRunner(tc.ServerConn(0))
	sqlDB.Exec(t, `SET CLUSTER SETTING kv.rangefeed.enabled = true`)
	sqlDB.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '1s'`)

plumbed this down so that Watcher can control it, which 1) makes the test run WAY faster and 2) unbreaks the merge txn (though i haven't reenabled merges yet in the test because i'm still working through some other issues with them)


pkg/kv/kvnemeses/stepper.go, line 121 at r11 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

If we're going to do this then does it make sense to just update currentSplits when we hand out a MergeOperation?

Done. This also let me get rid of the awkward prev argument!


pkg/kv/kvnemeses/stepper.go, line 164 at r11 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

We're not making any assumptions that this key actually is missing, right? If so then we'll need to confirm that randKey is not in s.keys.

Right, missing vs existing is best effort anyway because of the nondeterminism introduced by concurrency. Do you think it's clearer if I name them all randGetProbablyMissing/randGetProbablyExisting or does that just get awkwardly wordy?


pkg/kv/kvnemeses/stepper.go, line 173 at r11 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

nit: consider pulling these two lines into a method.

Done


pkg/kv/kvnemeses/stepper.go, line 194 at r11 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Why do we make this first split harder?

Moved this to the new model, I think it's more clear now (plus the proportion of new split vs resplit is configurable now)


pkg/kv/kvnemeses/stepper.go, line 204 at r11 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

This is supposed to be analogous to a randMergeMissing operation with 1/10th the probability of a randMergeExisting, right? Do we want to just add that as a separate op? That will also allow us to test missing merges before all split points get merged away.

Done


pkg/kv/kvnemeses/stepper.go, line 211 at r11 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Can usedKeys be nil here?

Got rid of all usedKeys nil-ness


pkg/kv/kvnemeses/validator.go, line 29 at r11 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Am I missing something or do we never actually call this outside of tests? Is that what you mean by "TODO: Validate the log"?

"TODO: Validate the log" had already rotted, it was validating read-write transactions but not read-only transactions, so I updated the TODO.

Validate was being called in TestKVNemesesSingleNode instead of RunNemeses so I could use t.Log, but that is dumb because it would mean a duplicated block in every TestKVNemeses*. Fixed

Copy link
Contributor Author

@danhhz danhhz left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten, @petermattis, and @tbg)


pkg/kv/kvnemeses/stepper.go, line 113 at r8 (raw file):

Previously, danhhz (Daniel Harrison) wrote…

Yeah, this was always meant to be temporary. I've been imagining something more like how the changefeed nemeses works where the overall relative proportions are passed in and the test selects between which whichever ones are contextually appropriate. Hows this?

I've come to the reluctant conclusion that Stepper needs to avoid including two writes for the same key in a single batch/txn. (Because it breaks our assumptions that every Put shows up in kv, plus it errors when done in a batch.) When I pushed this revision, I was thinking this extra bookkeeping prevented me from doing the opGenFunc factorization, but I think that it's actually still possible and even makes the usedKey stuff more clear. I'll include this in the next revision.

I removed the bit that prevents Stepper from generating two keys and implemented the validator logic to check that only the last write for each key in a txn makes it to kv. This a) lets us actually check this behavior and b) trivially allows for your opGenFunc simplification, which reduces quite a bit of duplication. Thanks for the idea!

Copy link
Member

@tbg tbg left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This generally looks good! I wrote up some longer comments on the Stepper and Validator, but nothing that really suggests a change in direction here. For the "architectural" stuff I would like to discuss it before merging, but when it comes to implementation details or missing functionality (verifying reads etc) I think we should prefer separate PRs to get v1 merged soon.

The commit message needs some updates.

Reviewed 22 of 22 files at r9, 7 of 7 files at r10, 3 of 17 files at r11, 10 of 13 files at r12, 4 of 4 files at r13.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @danhhz, @nvanbenschoten, and @petermattis)


pkg/kv/kvnemeses/applier.go, line 116 at r13 (raw file):

		err := db.Put(ctx, o.Key, o.Value)
		o.Result = resultError(ctx, err)
	case *BatchOperation:

I think you want to get back to a world in which opening/committing/doing something with the open txn are individual steps, right? At that point it might make sense to do the same for batches (i.e. starting a batch is a step, and then Get and Put are put into an open batch if one is around, etc). I wonder if that's overall better, it does raise some new awkward problems.


pkg/kv/kvnemeses/applier.go, line 162 at r13 (raw file):

		return Result{Type: ResultType_NoError}
	}
	if strings.Contains(err.Error(), `already committed`) {

Add a comment.


pkg/kv/kvnemeses/applier_test.go, line 68 at r13 (raw file):

{
  b := &Batch{}
  b.Put(ctx, "b", 2) // aborted in distSender: context canceled

Why are operations on *Batch returning errors? I would've expected no comments here.


pkg/kv/kvnemeses/engine.go, line 27 at r13 (raw file):

// check the behavior of engine.ReadWriter. Engine is not threadsafe.
//
// TODO(dan): Actually implement engine.ReadWriter with a bunch of panics?

Is checking the correctness of the default impl of engine.ReadWriter a goal here? I would've thought that that's their unit tests' job (where we have metamorphic testing). I wouldn't feel bad about just using an empty in-mem engine here.


pkg/kv/kvnemeses/kvnemeses.go, line 30 at r13 (raw file):

// RunNemeses generates and applies a series of Operations to exercise the KV
// api. It returns the resulting log of inputs and outputs.

Stale comment about what's returned.


pkg/kv/kvnemeses/kvnemeses.go, line 72 at r13 (raw file):

			}
			buf.Reset()
			step.format(&buf, formatCtx{receiver: `db`, indent: `  ` + workerName + ` OPERATION `})

nit: I assume this gets printed a lot, so maybe OP and ERR instead of ERRORED and OPERATION.


pkg/kv/kvnemeses/kvnemeses.go, line 87 at r13 (raw file):

	}

	kvs, err := r.KVs(ctx, allSteps)

Can you avoid passing allSteps in? It's only used to wait until max(step.After) is closed out.

maybe just add somewhere

type Steps []Step
func (s Steps) After() time.Time {
  var ts hlc.Timestamp{}
  for _, step := range s {
    ts.Forward(step.After)
  }
  return ts
}

I don't feel too strongly about this, but note that I went to check out KVs just to make sure it was really only doing what I thought it was.


pkg/kv/kvnemeses/kvnemeses.go, line 114 at r13 (raw file):

// Resulter tracks the kv changes that happen over a span.
type Resulter struct {

Is Resulter useful to keep around? It never uses its db field and I also don't find its name very evocative. Can KVs sit on Watcher and we just have a Watcher where we have Resulter now?


pkg/kv/kvnemeses/stepper.go, line 35 at r13 (raw file):
"SplitAgain"?

Also add comments. If something got split and merged again, is it a re-split or a new split? Similar for some of the neighbors here.

Also mention the concurrency issues where they exist. For example:

OpMergeIsSplit merges at a key that is likely to currently represent a range boundary (according to previously generated steps).

OpGetMissing reads a key that is likely to be absent or deleted, according to previously generated steps.

The , according ... is probably best kept above the const block and then it's enough to say "likely" in each comment only.

I wish there were an easy way to make suggestion that you can simply patch in. Vanilla GH has that feature but it's a pain to write the suggestion in markdown plain-text, which is what they force you to do.


pkg/kv/kvnemeses/stepper.go, line 38 at r13 (raw file):

	OpPMergeNotSplit OpP = "MergeNotSplit"
	OpPMergeIsSplit  OpP = "MergeIsSplit"
)

DelRange will be an interesting one to add (though I wish we didn't have it in the API in the first place).


pkg/kv/kvnemeses/stepper.go, line 40 at r13 (raw file):

relative


pkg/kv/kvnemeses/stepper.go, line 64 at r13 (raw file):
Comment suggestion:

Stepper in itself is deterministic, but it's intended usage is that multiple worker goroutines take turns pulling steps (sequentially) which they then execute concurrently. To improve the efficiency of this pattern, Stepper will track which splits and merges could possibly have taken place and takes this into account when generating operations. For example, Stepper won't take a OpMergeIsSplit step if it has never previously emitted a split, but it may emit an OpMerge once it has produced an OpSplit even though the worker executing the split may find that the merge has not yet been executed.

All in all I'm sympathetic to Nathan's view that the Mutex should sit on *Stepper, since its intended concurrent use shapes its design. I personally would also move away from the name Stepper as this somehow lures at least my brain into thinking about steps that are executed (stepped) sequentially. But this is very much not the case, we're creating a sequence but the execution order is not fully constrained. Generator?


pkg/kv/kvnemeses/stepper.go, line 79 at r13 (raw file):

These races can result in a currentSplit that is not in fact a split at the KV level.


pkg/kv/kvnemeses/stepper.go, line 81 at r13 (raw file):

for which a split has been emitted, regardless of whether the split has since been applied or been merged away again.


pkg/kv/kvnemeses/validator.go, line 27 at r13 (raw file):

)

// Validate checks for violations of our kv api guarantees. The Steps must all

Let's be more specific about what's being checked. (I know we're going to update it, but still).

If I am reading the code correctly, reads aren't checked at all right now, right? That is, if a transaction commits at timestamp 100, and it read some key K getting value V, we don't check that V is indeed the right value to read at ts=100.

For validating read-only transactions, isn't it trivial to do the same if we know the commit timestamp? We should be able to learn the commit timestamp from the client.Txn (if we don't get that today, we should fix that instead of adding extra smarts here).
We can't do the same for nontransactional read-only ops (i.e. single read or batch of only reads) but again I'd be in favor of rectifying that (i.e. communicating timestamp back). However, if we're serious about cleaning up the KV API, I would also suggest removal of all nontransactional ops in general. (Curious what @andreimatei feels about this). Note that for ambiguous results, we've necessarily attempted to write, and can recover the commit timestamp from any of our writes (which we can identify since the values are unique).

The "intersecting read intervals" idea sounds good too, but I think I'll still avoid it and use the timestamp the DB claims to do the verification just because that's simpler and also gives us confidence that what the DB tells us the commit timestamp was is actually right, something that might become more important with causality tokens.

Going back to documenting what's being checked at the time of writing, something like this perhaps:

// For transactions, it is verified that all of their writes are present if and only if the transaction committed (which is inferred from the KV data on ambiguous results /* TODO // actually do that */). Non-transactional r/w operations are treated as though they had been wrapped in a transaction and are verified accordingly.
//
// TODO: verify that the results of reads match the data visible at the commit timestamp.
// TODO: verify that there is no causality inversion between steps. That is, if transactions corresponding to two steps are sequential (i.e. txn1CommitStep.After < txn2BeginStep.Before) then the commit timestamps need to reflect this ordering.
//
// Splits and merges are not verified for anything other than that they did not return an error.


pkg/kv/kvnemeses/validator.go, line 108 at r13 (raw file):

			// Accumulate all the writes for this transaction.
			v.kvsByTxn[*txnID] = append(v.kvsByTxn[*txnID], kv)
		} else {

Can we get to a place in which a non-transactional (possibly batch of) op is verified as if it had been a single transactional batch (that's how it should behave)? Instead of this else, you'd pull the same fakeTxnID trick as for ClosureTxnOperation below.


pkg/kv/kvnemeses/validator.go, line 148 at r13 (raw file):

			v.checkCommittedTxn(fakeTxnID)
		} else {
			v.checkUncommittedTxn(fakeTxnID)

Doesn't seem like this handles ambiguous results correctly?


pkg/kv/kvnemeses/validator_test.go, line 94 at r13 (raw file):

		{
			name:     "one retryable put with write (correctly) missing",
			steps:    []Step{step(withResult(put(`a`, `v1`), retryableError))},

add two cases for ambiguous results, one where the txn actually did commit and one where it didn't.

@danhhz danhhz force-pushed the kvnemeses branch 2 times, most recently from 63c7960 to fb0ce7f Compare January 16, 2020 20:55
Copy link
Contributor Author

@danhhz danhhz left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The commit message needs some updates.

Whoops, forgot this, will do. Everything else is done.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @andreimatei, @nvanbenschoten, @petermattis, and @tbg)


pkg/kv/kvnemeses/applier.go, line 116 at r13 (raw file):

Previously, tbg (Tobias Grieger) wrote…

I think you want to get back to a world in which opening/committing/doing something with the open txn are individual steps, right? At that point it might make sense to do the same for batches (i.e. starting a batch is a step, and then Get and Put are put into an open batch if one is around, etc). I wonder if that's overall better, it does raise some new awkward problems.

I would like to get back to something like that because I think it will generate interesting contention histories more often, but I haven't found a good way to deal with the deadlock issues.

Not sure it gets you anything for batches. Txns are interesting because starting one and using it does things in the DB, while a batch doesn't. That said, I'm not opposed to the parallel structure if you prefer it.


pkg/kv/kvnemeses/applier.go, line 162 at r13 (raw file):

Previously, tbg (Tobias Grieger) wrote…

Add a comment.

debugging detritus. removed


pkg/kv/kvnemeses/applier_test.go, line 68 at r13 (raw file):

Previously, tbg (Tobias Grieger) wrote…

Why are operations on *Batch returning errors? I would've expected no comments here.

The Put and Get themselves don't error, but this is what is in the Batch.Results after Run gets called


pkg/kv/kvnemeses/engine.go, line 27 at r13 (raw file):

Previously, tbg (Tobias Grieger) wrote…

Is checking the correctness of the default impl of engine.ReadWriter a goal here? I would've thought that that's their unit tests' job (where we have metamorphic testing). I wouldn't feel bad about just using an empty in-mem engine here.

This is an artifact of the path I took while figuring out exactly what kvnemeses would look like. I do like the simplified interface that this has over using an in-mem engine directly with all the iterators and MVCCGetOptions and such, so I'm inclined to keep it. If this gets any more complicated than it is, it should probably be swapped out


pkg/kv/kvnemeses/kvnemeses.go, line 30 at r13 (raw file):

Previously, tbg (Tobias Grieger) wrote…

Stale comment about what's returned.

Done


pkg/kv/kvnemeses/kvnemeses.go, line 72 at r13 (raw file):

Previously, tbg (Tobias Grieger) wrote…

nit: I assume this gets printed a lot, so maybe OP and ERR instead of ERRORED and OPERATION.

Done


pkg/kv/kvnemeses/kvnemeses.go, line 87 at r13 (raw file):

Previously, tbg (Tobias Grieger) wrote…

Can you avoid passing allSteps in? It's only used to wait until max(step.After) is closed out.

maybe just add somewhere

type Steps []Step
func (s Steps) After() time.Time {
  var ts hlc.Timestamp{}
  for _, step := range s {
    ts.Forward(step.After)
  }
  return ts
}

I don't feel too strongly about this, but note that I went to check out KVs just to make sure it was really only doing what I thought it was.

Done


pkg/kv/kvnemeses/kvnemeses.go, line 114 at r13 (raw file):

Previously, tbg (Tobias Grieger) wrote…

Is Resulter useful to keep around? It never uses its db field and I also don't find its name very evocative. Can KVs sit on Watcher and we just have a Watcher where we have Resulter now?

I think this makes more sense once the "Also slurp the splits" TODO is done. Happy to delete it until then though


pkg/kv/kvnemeses/stepper.go, line 35 at r13 (raw file):

Previously, tbg (Tobias Grieger) wrote…

"SplitAgain"?

Also add comments. If something got split and merged again, is it a re-split or a new split? Similar for some of the neighbors here.

Also mention the concurrency issues where they exist. For example:

OpMergeIsSplit merges at a key that is likely to currently represent a range boundary (according to previously generated steps).

OpGetMissing reads a key that is likely to be absent or deleted, according to previously generated steps.

The , according ... is probably best kept above the const block and then it's enough to say "likely" in each comment only.

I wish there were an easy way to make suggestion that you can simply patch in. Vanilla GH has that feature but it's a pain to write the suggestion in markdown plain-text, which is what they force you to do.

Done


pkg/kv/kvnemeses/stepper.go, line 38 at r13 (raw file):

Previously, tbg (Tobias Grieger) wrote…

DelRange will be an interesting one to add (though I wish we didn't have it in the API in the first place).

Agreed, it's already in the TODO list in doc.go


pkg/kv/kvnemeses/stepper.go, line 40 at r13 (raw file):

Previously, tbg (Tobias Grieger) wrote…

relative

Done


pkg/kv/kvnemeses/stepper.go, line 64 at r13 (raw file):

Previously, tbg (Tobias Grieger) wrote…

Comment suggestion:

Stepper in itself is deterministic, but it's intended usage is that multiple worker goroutines take turns pulling steps (sequentially) which they then execute concurrently. To improve the efficiency of this pattern, Stepper will track which splits and merges could possibly have taken place and takes this into account when generating operations. For example, Stepper won't take a OpMergeIsSplit step if it has never previously emitted a split, but it may emit an OpMerge once it has produced an OpSplit even though the worker executing the split may find that the merge has not yet been executed.

All in all I'm sympathetic to Nathan's view that the Mutex should sit on *Stepper, since its intended concurrent use shapes its design. I personally would also move away from the name Stepper as this somehow lures at least my brain into thinking about steps that are executed (stepped) sequentially. But this is very much not the case, we're creating a sequence but the execution order is not fully constrained. Generator?

Done and done and done. The opGenFunc that get passed around in the map need to be locked when they are called and I was having trouble making this sufficiently obvious. In the interest of getting this merged, I've used my wrapper struct idea and left a TODO for cleanup.


pkg/kv/kvnemeses/stepper.go, line 79 at r13 (raw file):

Previously, tbg (Tobias Grieger) wrote…

These races can result in a currentSplit that is not in fact a split at the KV level.

Done


pkg/kv/kvnemeses/stepper.go, line 81 at r13 (raw file):

Previously, tbg (Tobias Grieger) wrote…

for which a split has been emitted, regardless of whether the split has since been applied or been merged away again.

Done


pkg/kv/kvnemeses/validator.go, line 27 at r13 (raw file):

Previously, tbg (Tobias Grieger) wrote…

Let's be more specific about what's being checked. (I know we're going to update it, but still).

If I am reading the code correctly, reads aren't checked at all right now, right? That is, if a transaction commits at timestamp 100, and it read some key K getting value V, we don't check that V is indeed the right value to read at ts=100.

For validating read-only transactions, isn't it trivial to do the same if we know the commit timestamp? We should be able to learn the commit timestamp from the client.Txn (if we don't get that today, we should fix that instead of adding extra smarts here).
We can't do the same for nontransactional read-only ops (i.e. single read or batch of only reads) but again I'd be in favor of rectifying that (i.e. communicating timestamp back). However, if we're serious about cleaning up the KV API, I would also suggest removal of all nontransactional ops in general. (Curious what @andreimatei feels about this). Note that for ambiguous results, we've necessarily attempted to write, and can recover the commit timestamp from any of our writes (which we can identify since the values are unique).

The "intersecting read intervals" idea sounds good too, but I think I'll still avoid it and use the timestamp the DB claims to do the verification just because that's simpler and also gives us confidence that what the DB tells us the commit timestamp was is actually right, something that might become more important with causality tokens.

Going back to documenting what's being checked at the time of writing, something like this perhaps:

// For transactions, it is verified that all of their writes are present if and only if the transaction committed (which is inferred from the KV data on ambiguous results /* TODO // actually do that */). Non-transactional r/w operations are treated as though they had been wrapped in a transaction and are verified accordingly.
//
// TODO: verify that the results of reads match the data visible at the commit timestamp.
// TODO: verify that there is no causality inversion between steps. That is, if transactions corresponding to two steps are sequential (i.e. txn1CommitStep.After < txn2BeginStep.Before) then the commit timestamps need to reflect this ordering.
//
// Splits and merges are not verified for anything other than that they did not return an error.

Done. Left learning the commit timestamp from client.Txn as a TODO


pkg/kv/kvnemeses/validator.go, line 108 at r13 (raw file):

Previously, tbg (Tobias Grieger) wrote…

Can we get to a place in which a non-transactional (possibly batch of) op is verified as if it had been a single transactional batch (that's how it should behave)? Instead of this else, you'd pull the same fakeTxnID trick as for ClosureTxnOperation below.

Done, this cleaned things up considerably!


pkg/kv/kvnemeses/validator.go, line 148 at r13 (raw file):

Previously, tbg (Tobias Grieger) wrote…

Doesn't seem like this handles ambiguous results correctly?

Yeah, I knew. It wasn't seeing them in practice. Thanks for pushing me on this : - )


pkg/kv/kvnemeses/validator_test.go, line 94 at r13 (raw file):

Previously, tbg (Tobias Grieger) wrote…

add two cases for ambiguous results, one where the txn actually did commit and one where it didn't.

Done, plus ambiguous with validation error

No changes at all: everything is done in the next commit.

Release note: None
Rename spanFrontier to Frontier since it would stutter with the package
name.

Release note: None
@danhhz
Copy link
Contributor Author

danhhz commented Jan 21, 2020

Hey @tbg / @nvanbenschoten did either of you want to take another look at this before it merges?

@nvanbenschoten
Copy link
Member

Yes, I'm going to do one more pass today.

@tbg
Copy link
Member

tbg commented Jan 21, 2020 via email

Copy link
Member

@nvanbenschoten nvanbenschoten left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

:lgtm:

Reviewed 7 of 7 files at r15, 17 of 17 files at r16.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @andreimatei, @danhhz, @petermattis, and @tbg)


pkg/kv/kvnemeses/applier.go, line 27 at r11 (raw file):

what are your thoughts about a ConcurrentStepper wrapper that handles the locking?

I'm +1 on that.


pkg/kv/kvnemeses/operations.proto, line 24 at r16 (raw file):

}

enum ClosureTxnType {

Didn't you want to add a CommitInBatch variant here?


pkg/kv/kvnemeses/watcher.go, line 140 at r16 (raw file):

				// but it means that we'll won't catch it if we violate those semantics.
				// Consider first doing a Get and somehow failing if this exact key+ts
				// has previously been put with a different value.

We discussed last week that another option is to fix this by ensuring that all requests in a nontransactional batch are disjoint and upgrading to a transactional batch (see CrossRangeTxnWrapperSender) if they are. You can use roachpb.SpanGroup to efficiently check this.

Package kvnemeses exercises the KV api with random traffic and then
validates that the observed behaviors are consistent with our
guarantees.

A set of Operations are generated which represent usage of the public KV
api. These include both "workload" operations like Gets and Puts as well
as "admin" operations like rebalances. These Operations can be handed to
an Applier, which runs them against the KV api and records the results.

Operations do allow for concurrency (this testing is much less
interesting otherwise), which means that the state of the KV map is not
recoverable from _only_ the input. TODO(dan): We can use RangeFeed to
recover the exact KV history. This plus some Kyle magic can be used to
check our transactional guarantees.

TODO (in later commits)
- Validate the log
- CPut/InitPut/Increment/Delete
- DeleteRange/ClearRange/RevertRange/Scan/ReverseScan
- ChangeReplicas/TransferLease
- ExportRequest
- AddSSTable
- Root and leaf transactions
- GCRequest
- Protected timestamps

Release note: None
Copy link
Contributor Author

@danhhz danhhz left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the reviews!

bors r=nvanbenschoten/tbg

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @andreimatei, @danhhz, @nvanbenschoten, @petermattis, and @tbg)


pkg/kv/kvnemeses/applier.go, line 27 at r11 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

what are your thoughts about a ConcurrentStepper wrapper that handles the locking?

I'm +1 on that.

Ended up calling them Generator and generator


pkg/kv/kvnemeses/operations.proto, line 24 at r16 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Didn't you want to add a CommitInBatch variant here?

Left this as a TODO. I wrote it up real quick, but none of the Gets and Puts in the Batch seem to be doing anything? Maybe I just messed some something silly but what's the contract here again?


pkg/kv/kvnemeses/watcher.go, line 140 at r16 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

We discussed last week that another option is to fix this by ensuring that all requests in a nontransactional batch are disjoint and upgrading to a transactional batch (see CrossRangeTxnWrapperSender) if they are. You can use roachpb.SpanGroup to efficiently check this.

Added this comment in a slightly different place

craig bot pushed a commit that referenced this pull request Jan 22, 2020
43565: kvnemeses: begin scaffolding for a jepsen-style kv test r=nvanbenschoten/tbg a=danhhz

Package kvnemeses exercises the KV api with random traffic and then
validates that the observed behaviors are consistent with our
guarantees.

A set of Operations are generated which represent usage of the public KV
api. These include both "workload" operations like Gets and Puts as well
as "admin" operations like rebalances. These Operations can be handed to
an Applier, which runs them against the KV api and records the results.

Operations do allow for concurrency (this testing is much less
interesting otherwise), which means that the state of the KV map is not
recoverable from _only_ the input. TODO(dan): We can use RangeFeed to
recover the exact KV history. This plus some Kyle magic can be used to
check our transactional guarantees.

TODO (in later commits)
- Validate the log
- CPut/InitPut/Increment/Delete
- DeleteRange/ClearRange/RevertRange/Scan/ReverseScan
- ChangeReplicas/TransferLease
- ExportRequest
- AddSSTable
- Root and leaf transactions
- GCRequest
- Protected timestamps

Release note: None

44144: colexec: fix multiple starts of the wrapped processors r=yuzefovich a=yuzefovich

**colexec: fix multiple starts of the wrapped processors**

Previously, wrapped processors could be started multiple times if they
were in the input chain for the bufferOp (each of the CASE arms will
initialize its input - the bufferOp). Now this is fixed by tracking in
both Columnarizer and bufferOp whether Init has already been called.

Previous behavior could lead to a crash when rowexec.valuesProcessor was
wrapped because it sends a "bogus" metadata header on each call to
Start, and only single header is expected whereas with multiple Inits
they would be multiple headers.

Fixes: #44133.

Release note (bug fix): Previously, CockroachDB could crash in special
circumstances when vectorized execution engine is used (it was more
likely to happen if `vectorize=experimental_on` setting was used). Now
this has been fixed.

**execerror: catch panics coming from sql/execinfra package**

sql/execinfra is definitely a part of the vectorized engine as a whole,
so we should be catching panics coming from it when running vectorized
flows.

Release note: None

44169: sql/opt/optbuilder: resolve remaining comments from #44015 r=nvanbenschoten a=nvanbenschoten

This commit resolves a few typos that were missed before #44015 was merged.

Release note: None

44172: Include "/cockroach" into PATH in Docker image r=vladdy a=vladdy

This adds "/cockroach" into environment's PATH in Docker image to require less typing when invoking "cockroach" commands via running container.

Fixes: #44189

Co-authored-by: Daniel Harrison <daniel.harrison@gmail.com>
Co-authored-by: Yahor Yuzefovich <yahor@cockroachlabs.com>
Co-authored-by: Nathan VanBenschoten <nvanbenschoten@gmail.com>
Co-authored-by: Vlad Artamonov <742047+vladdy@users.noreply.github.com>
@craig
Copy link
Contributor

craig bot commented Jan 22, 2020

Build succeeded

@craig craig bot merged commit 9e1dfff into cockroachdb:master Jan 22, 2020
@danhhz danhhz deleted the kvnemeses branch February 20, 2020 15:53
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

6 participants