From 736a67e0d36cc545bf74d65db069ee895ff9bea0 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Wed, 15 Feb 2023 18:59:50 -0800 Subject: [PATCH] kvpb: extract requests and errors from roachpb into new package This commit extracts a new `pkg/kv/kvpb` package out of `pkg/roachpb`. We've wanted to do this for a while, and what prompted me to do this was the desire to make the ScanResponse depend on `pkg/col/coldata`. `api.proto` and `errors.proto` files are moved into the new package while preserving the `roachpb` as their protobuf package. This required also the movement of the following: - `roachpb/gen` and `roachpb/roachpbmock` subpackages - almost all of `api.go` (except for `roachpb.UserPriority` that was moved into `roachpb/data.go`) - miscellaneous error-specific files - extracting a couple of functions from `data.go` - `method.go`. In order to keep the backwards-compatibility for errors that are sent over the wire in the mixed-version clusters, for all error types that were moved from `roachpb` to `kv/kvpb` a type migration has been registered with the `errors` library. Clearly, essentially all of these changes were done automatically. The only manual change worth mentioning was when moving `errors.go`. Namely, type conversion of `ReadWithinUncertaintyIntervalError.ObservedTimestamps` to `roachpb.observedTimestampSlice` was removed in the loop since it seems to not do anything. After running `./dev gen bazel` the proto compiler in `roachpb/BUILD.bazel` changed. I'm assuming that it is expected. Epic: None Release note: None --- .github/CODEOWNERS | 27 +- BUILD.bazel | 8 +- build/bazelutil/check.sh | 4 +- build/bazelutil/nogo_config.json | 6 +- docs/generated/http/BUILD.bazel | 1 + pkg/BUILD.bazel | 15 +- pkg/base/serverident/server_ident.go | 2 +- pkg/ccl/backupccl/BUILD.bazel | 2 + pkg/ccl/backupccl/backup_intents_test.go | 4 +- pkg/ccl/backupccl/backup_job.go | 7 +- pkg/ccl/backupccl/backup_processor.go | 23 +- .../backupccl/backup_processor_planning.go | 7 +- pkg/ccl/backupccl/backup_test.go | 33 +- pkg/ccl/backupccl/backupinfo/BUILD.bazel | 1 + .../backupccl/backupinfo/backup_metadata.go | 25 +- .../backupccl/backupinfo/manifest_handling.go | 5 +- pkg/ccl/backupccl/datadriven_test.go | 5 +- pkg/ccl/backupccl/file_sst_sink.go | 4 +- pkg/ccl/backupccl/restore_data_processor.go | 11 +- .../backupccl/restore_data_processor_test.go | 7 +- pkg/ccl/backupccl/restore_job.go | 5 +- .../backupccl/restore_processor_planning.go | 5 +- .../backupccl/split_and_scatter_processor.go | 9 +- pkg/ccl/changefeedccl/BUILD.bazel | 1 + pkg/ccl/changefeedccl/cdceval/BUILD.bazel | 1 + .../changefeedccl/cdceval/expr_eval_test.go | 9 +- pkg/ccl/changefeedccl/cdctest/BUILD.bazel | 1 + pkg/ccl/changefeedccl/cdctest/row.go | 11 +- pkg/ccl/changefeedccl/changefeed_test.go | 19 +- .../changefeedccl/changefeedbase/BUILD.bazel | 2 +- .../changefeedccl/changefeedbase/errors.go | 4 +- .../changefeedccl/event_processing_test.go | 6 +- pkg/ccl/changefeedccl/kvevent/BUILD.bazel | 2 + pkg/ccl/changefeedccl/kvevent/bench_test.go | 7 +- .../kvevent/blocking_buffer_test.go | 7 +- pkg/ccl/changefeedccl/kvevent/event.go | 15 +- pkg/ccl/changefeedccl/kvfeed/BUILD.bazel | 2 + pkg/ccl/changefeedccl/kvfeed/kv_feed_test.go | 29 +- .../changefeedccl/kvfeed/physical_kv_feed.go | 10 +- pkg/ccl/changefeedccl/kvfeed/scanner.go | 11 +- pkg/ccl/changefeedccl/kvfeed/testing_knobs.go | 4 +- pkg/ccl/changefeedccl/schemafeed/BUILD.bazel | 3 +- .../changefeedccl/schemafeed/schema_feed.go | 17 +- .../schemafeed/schema_feed_test.go | 6 +- pkg/ccl/kvccl/kvfollowerreadsccl/BUILD.bazel | 2 + .../boundedstaleness_test.go | 6 +- .../kvccl/kvfollowerreadsccl/followerreads.go | 5 +- .../kvfollowerreadsccl/followerreads_test.go | 131 +- pkg/ccl/kvccl/kvtenantccl/BUILD.bazel | 2 + pkg/ccl/kvccl/kvtenantccl/connector.go | 17 +- pkg/ccl/kvccl/kvtenantccl/connector_test.go | 103 +- .../kvccl/kvtenantccl/setting_overrides.go | 10 +- .../kvtenantccl/setting_overrides_test.go | 45 +- .../kvccl/kvtenantccl/tenant_trace_test.go | 3 +- .../tenantcostclient/BUILD.bazel | 2 + .../query_ru_estimate_test.go | 4 +- .../tenantcostclient/tenant_side.go | 19 +- .../tenantcostclient/tenant_side_test.go | 13 +- .../tenantcostserver/BUILD.bazel | 2 + .../tenantcostserver/server_test.go | 7 +- .../tenantcostserver/system_table.go | 3 +- .../tenanttokenbucket/BUILD.bazel | 3 +- .../tenanttokenbucket/tenant_token_bucket.go | 7 +- .../tenant_token_bucket_test.go | 4 +- .../tenantcostserver/token_bucket.go | 19 +- pkg/ccl/storageccl/BUILD.bazel | 2 +- pkg/ccl/storageccl/external_sst_reader.go | 6 +- pkg/ccl/streamingccl/BUILD.bazel | 1 + pkg/ccl/streamingccl/event.go | 29 +- .../streamingccl/replicationutils/BUILD.bazel | 2 + .../streamingccl/replicationutils/utils.go | 5 +- .../replicationutils/utils_test.go | 3 +- pkg/ccl/streamingccl/streamclient/BUILD.bazel | 1 + .../streamclient/random_stream_client.go | 5 +- pkg/ccl/streamingccl/streamingest/BUILD.bazel | 2 + .../replication_random_client_test.go | 11 +- .../streamingest/stream_ingestion_job.go | 5 +- .../streamingest/stream_ingestion_job_test.go | 5 +- .../stream_ingestion_processor.go | 5 +- .../stream_ingestion_processor_test.go | 3 +- .../streamingccl/streamproducer/BUILD.bazel | 2 + .../streamproducer/event_stream.go | 29 +- .../streamproducer/replication_stream_test.go | 13 +- pkg/cli/BUILD.bazel | 2 + pkg/cli/debug_reset_quorum.go | 4 +- pkg/cli/debug_send_kv_batch.go | 16 +- pkg/cli/debug_send_kv_batch_test.go | 15 +- pkg/cli/zip_test.go | 3 +- pkg/cmd/roachtest/tests/BUILD.bazel | 1 + pkg/cmd/roachtest/tests/mvcc_gc.go | 27 +- pkg/gen/gomock.bzl | 2 +- pkg/gen/misc.bzl | 2 +- pkg/gen/protobuf.bzl | 1 + pkg/gen/stringer.bzl | 4 +- pkg/gossip/BUILD.bazel | 1 + pkg/gossip/gossip.go | 3 +- .../client/requestbatcher/BUILD.bazel | 2 + pkg/internal/client/requestbatcher/batcher.go | 29 +- .../client/requestbatcher/batcher_test.go | 65 +- pkg/jobs/BUILD.bazel | 2 + pkg/jobs/job_scheduler.go | 4 +- pkg/jobs/jobs_test.go | 7 +- pkg/jobs/jobspb/BUILD.bazel | 2 + pkg/jobs/jobspb/jobs.proto | 2 +- pkg/keys/BUILD.bazel | 2 + pkg/keys/keys.go | 5 +- pkg/keys/keys_test.go | 13 +- .../keyvissubscriber/BUILD.bazel | 1 + .../keyvissubscriber/boundary_subscriber.go | 5 +- pkg/kv/BUILD.bazel | 2 + pkg/kv/batch.go | 189 +- pkg/kv/bulk/BUILD.bazel | 2 + pkg/kv/bulk/buffering_adder.go | 7 +- pkg/kv/bulk/sst_batcher.go | 27 +- pkg/kv/bulk/sst_batcher_test.go | 11 +- pkg/kv/client_test.go | 41 +- pkg/kv/db.go | 65 +- pkg/kv/db_test.go | 11 +- pkg/kv/kvclient/BUILD.bazel | 1 + pkg/kv/kvclient/kvcoord/BUILD.bazel | 4 +- pkg/kv/kvclient/kvcoord/batch.go | 81 +- pkg/kv/kvclient/kvcoord/batch_test.go | 51 +- pkg/kv/kvclient/kvcoord/dist_sender.go | 251 ++- .../kvcoord/dist_sender_mux_rangefeed.go | 21 +- .../kvclient/kvcoord/dist_sender_rangefeed.go | 59 +- .../dist_sender_rangefeed_mock_test.go | 8 +- .../kvcoord/dist_sender_rangefeed_test.go | 29 +- .../kvcoord/dist_sender_server_test.go | 165 +- pkg/kv/kvclient/kvcoord/dist_sender_test.go | 721 +++---- pkg/kv/kvclient/kvcoord/integration_test.go | 15 +- .../kvcoord/local_test_cluster_util.go | 5 +- .../kvcoord/lock_spans_over_budget_error.go | 4 +- .../kvclient/kvcoord/mocks_generated_test.go | 5 +- pkg/kv/kvclient/kvcoord/rangefeed_message.go | 9 +- .../kvclient/kvcoord/replayed_commit_test.go | 9 +- pkg/kv/kvclient/kvcoord/send_test.go | 45 +- pkg/kv/kvclient/kvcoord/testdata/savepoints | 14 +- pkg/kv/kvclient/kvcoord/testing_knobs.go | 4 +- pkg/kv/kvclient/kvcoord/transport.go | 19 +- pkg/kv/kvclient/kvcoord/transport_race.go | 16 +- pkg/kv/kvclient/kvcoord/transport_test.go | 67 +- pkg/kv/kvclient/kvcoord/txn_coord_sender.go | 114 +- .../kvcoord/txn_coord_sender_savepoints.go | 5 +- .../txn_coord_sender_savepoints_test.go | 11 +- .../kvcoord/txn_coord_sender_server_test.go | 5 +- .../kvclient/kvcoord/txn_coord_sender_test.go | 323 ++-- .../kvcoord/txn_interceptor_committer.go | 67 +- .../kvcoord/txn_interceptor_committer_test.go | 145 +- .../kvcoord/txn_interceptor_heartbeater.go | 37 +- .../txn_interceptor_heartbeater_test.go | 167 +- .../txn_interceptor_metric_recorder.go | 5 +- .../kvcoord/txn_interceptor_pipeliner.go | 81 +- .../txn_interceptor_pipeliner_client_test.go | 9 +- .../kvcoord/txn_interceptor_pipeliner_test.go | 553 +++--- .../txn_interceptor_seq_num_allocator.go | 9 +- .../txn_interceptor_seq_num_allocator_test.go | 175 +- .../kvcoord/txn_interceptor_span_refresher.go | 79 +- .../txn_interceptor_span_refresher_test.go | 311 +-- .../kvclient/kvcoord/txn_lock_gatekeeper.go | 10 +- pkg/kv/kvclient/kvcoord/txn_test.go | 31 +- pkg/kv/kvclient/kvstreamer/BUILD.bazel | 2 + .../kvclient/kvstreamer/requests_provider.go | 5 +- .../kvstreamer/results_buffer_test.go | 5 +- pkg/kv/kvclient/kvstreamer/size.go | 19 +- pkg/kv/kvclient/kvstreamer/streamer.go | 54 +- pkg/kv/kvclient/kvstreamer/streamer_test.go | 35 +- pkg/kv/kvclient/kvtenant/BUILD.bazel | 1 + pkg/kv/kvclient/kvtenant/connector.go | 5 +- pkg/kv/kvclient/rangecache/BUILD.bazel | 1 + pkg/kv/kvclient/rangecache/range_cache.go | 9 +- .../rangecache/rangecachemock/BUILD.bazel | 1 + .../rangecachemock/mocks_generated.go | 3 +- pkg/kv/kvclient/rangefeed/BUILD.bazel | 2 + pkg/kv/kvclient/rangefeed/config.go | 7 +- pkg/kv/kvclient/rangefeed/db_adapter.go | 3 +- .../rangefeed/db_adapter_external_test.go | 3 +- pkg/kv/kvclient/rangefeed/rangefeed.go | 7 +- .../rangefeed/rangefeed_external_test.go | 71 +- .../kvclient/rangefeed/rangefeed_mock_test.go | 39 +- .../rangefeed/rangefeedbuffer/BUILD.bazel | 2 + .../kvclient/rangefeed/rangefeedbuffer/kvs.go | 5 +- .../rangefeed/rangefeedbuffer/kvs_test.go | 5 +- .../rangefeed/rangefeedcache/BUILD.bazel | 2 + .../rangefeedcache/cache_impl_test.go | 5 +- .../rangefeed/rangefeedcache/cache_test.go | 9 +- .../rangefeed/rangefeedcache/watcher.go | 5 +- pkg/kv/kvclient/rangefeed/scanner.go | 3 +- pkg/kv/kvclient/rangestats/BUILD.bazel | 1 + pkg/kv/kvclient/rangestats/fetcher.go | 9 +- pkg/kv/kvclient/revision_reader.go | 11 +- pkg/kv/kvnemesis/BUILD.bazel | 4 + pkg/kv/kvnemesis/applier.go | 13 +- pkg/kv/kvnemesis/env.go | 9 +- pkg/kv/kvnemesis/generator.go | 9 +- pkg/kv/kvnemesis/kvnemesis_test.go | 9 +- pkg/kv/kvnemesis/operations.proto | 2 +- pkg/kv/kvnemesis/validator.go | 7 +- pkg/kv/kvnemesis/validator_test.go | 7 +- pkg/kv/kvnemesis/watcher.go | 11 +- pkg/kv/kvpb/.gitattributes | 2 + pkg/kv/kvpb/BUILD.bazel | 153 ++ .../kvpb}/ambiguous_result_error.go | 17 +- pkg/{roachpb => kv/kvpb}/api.go | 118 +- pkg/{roachpb => kv/kvpb}/api.proto | 67 +- pkg/{roachpb => kv/kvpb}/api_requestheader.go | 2 +- pkg/{roachpb => kv/kvpb}/api_test.go | 95 +- pkg/{roachpb => kv/kvpb}/batch.go | 27 +- pkg/{roachpb => kv/kvpb}/batch_generated.go | 2 +- pkg/{roachpb => kv/kvpb}/batch_test.go | 81 +- pkg/kv/kvpb/data.go | 145 ++ .../kvpb}/errordetailtype_string.go | 2 +- pkg/{roachpb => kv/kvpb}/errors.go | 116 +- pkg/{roachpb => kv/kvpb}/errors.proto | 37 +- pkg/{roachpb => kv/kvpb}/errors_test.go | 31 +- pkg/{roachpb => kv/kvpb}/gen.bzl | 2 +- pkg/{roachpb => kv/kvpb}/gen/BUILD.bazel | 2 +- pkg/{roachpb => kv/kvpb}/gen/main.go | 2 +- .../kvpb/kvpbmock}/BUILD.bazel | 15 +- .../kvpb/kvpbmock}/mocks_generated.go | 57 +- pkg/{roachpb => kv/kvpb}/method.go | 2 +- pkg/{roachpb => kv/kvpb}/method_string.go | 2 +- .../kvpb}/replica_unavailable_error.go | 12 +- .../kvpb}/replica_unavailable_error_test.go | 2 +- pkg/kv/kvpb/string_test.go | 95 + .../kvpb}/testdata/ambiguous_result_error.txt | 0 .../testdata/replica_unavailable_error.txt | 0 pkg/kv/kvprober/BUILD.bazel | 1 + pkg/kv/kvprober/kvprober_integration_test.go | 14 +- pkg/kv/kvserver/BUILD.bazel | 2 + pkg/kv/kvserver/addressing_test.go | 3 +- pkg/kv/kvserver/allocation_op.go | 3 +- pkg/kv/kvserver/batcheval/BUILD.bazel | 2 + pkg/kv/kvserver/batcheval/cmd_add_sstable.go | 19 +- .../batcheval/cmd_add_sstable_test.go | 141 +- pkg/kv/kvserver/batcheval/cmd_barrier.go | 12 +- pkg/kv/kvserver/batcheval/cmd_clear_range.go | 15 +- .../batcheval/cmd_clear_range_test.go | 23 +- .../batcheval/cmd_compute_checksum.go | 13 +- .../kvserver/batcheval/cmd_conditional_put.go | 14 +- pkg/kv/kvserver/batcheval/cmd_delete.go | 10 +- pkg/kv/kvserver/batcheval/cmd_delete_range.go | 23 +- .../batcheval/cmd_delete_range_gchint_test.go | 9 +- .../batcheval/cmd_delete_range_test.go | 21 +- .../kvserver/batcheval/cmd_end_transaction.go | 49 +- .../batcheval/cmd_end_transaction_test.go | 29 +- pkg/kv/kvserver/batcheval/cmd_export.go | 29 +- pkg/kv/kvserver/batcheval/cmd_export_test.go | 85 +- pkg/kv/kvserver/batcheval/cmd_gc.go | 23 +- pkg/kv/kvserver/batcheval/cmd_get.go | 13 +- pkg/kv/kvserver/batcheval/cmd_get_test.go | 27 +- .../kvserver/batcheval/cmd_heartbeat_txn.go | 13 +- pkg/kv/kvserver/batcheval/cmd_increment.go | 10 +- pkg/kv/kvserver/batcheval/cmd_init_put.go | 8 +- .../kvserver/batcheval/cmd_is_span_empty.go | 10 +- .../batcheval/cmd_is_span_empty_test.go | 9 +- pkg/kv/kvserver/batcheval/cmd_lease.go | 7 +- pkg/kv/kvserver/batcheval/cmd_lease_info.go | 11 +- .../kvserver/batcheval/cmd_lease_request.go | 13 +- pkg/kv/kvserver/batcheval/cmd_lease_test.go | 11 +- .../kvserver/batcheval/cmd_lease_transfer.go | 11 +- pkg/kv/kvserver/batcheval/cmd_merge.go | 8 +- pkg/kv/kvserver/batcheval/cmd_migrate.go | 13 +- pkg/kv/kvserver/batcheval/cmd_probe.go | 12 +- pkg/kv/kvserver/batcheval/cmd_push_txn.go | 33 +- pkg/kv/kvserver/batcheval/cmd_put.go | 14 +- pkg/kv/kvserver/batcheval/cmd_query_intent.go | 18 +- .../batcheval/cmd_query_intent_test.go | 15 +- pkg/kv/kvserver/batcheval/cmd_query_locks.go | 13 +- .../batcheval/cmd_query_resolved_timestamp.go | 11 +- .../cmd_query_resolved_timestamp_test.go | 13 +- pkg/kv/kvserver/batcheval/cmd_query_txn.go | 15 +- pkg/kv/kvserver/batcheval/cmd_range_stats.go | 11 +- .../kvserver/batcheval/cmd_recompute_stats.go | 13 +- pkg/kv/kvserver/batcheval/cmd_recover_txn.go | 15 +- .../batcheval/cmd_recover_txn_test.go | 17 +- pkg/kv/kvserver/batcheval/cmd_refresh.go | 12 +- .../kvserver/batcheval/cmd_refresh_range.go | 13 +- .../batcheval/cmd_refresh_range_bench_test.go | 9 +- .../batcheval/cmd_refresh_range_test.go | 45 +- pkg/kv/kvserver/batcheval/cmd_refresh_test.go | 19 +- .../kvserver/batcheval/cmd_resolve_intent.go | 21 +- .../batcheval/cmd_resolve_intent_range.go | 13 +- .../batcheval/cmd_resolve_intent_test.go | 55 +- pkg/kv/kvserver/batcheval/cmd_reverse_scan.go | 20 +- pkg/kv/kvserver/batcheval/cmd_revert_range.go | 17 +- .../batcheval/cmd_revert_range_test.go | 23 +- pkg/kv/kvserver/batcheval/cmd_scan.go | 20 +- pkg/kv/kvserver/batcheval/cmd_scan_test.go | 41 +- pkg/kv/kvserver/batcheval/cmd_subsume.go | 14 +- pkg/kv/kvserver/batcheval/cmd_truncate_log.go | 11 +- .../batcheval/cmd_truncate_log_test.go | 5 +- pkg/kv/kvserver/batcheval/command.go | 25 +- pkg/kv/kvserver/batcheval/declare.go | 21 +- pkg/kv/kvserver/batcheval/declare_test.go | 13 +- pkg/kv/kvserver/batcheval/eval_context.go | 7 +- pkg/kv/kvserver/batcheval/intent.go | 13 +- pkg/kv/kvserver/batcheval/intent_test.go | 23 +- pkg/kv/kvserver/batcheval/ranges_test.go | 17 +- pkg/kv/kvserver/batcheval/result/BUILD.bazel | 1 + pkg/kv/kvserver/batcheval/result/result.go | 3 +- pkg/kv/kvserver/batcheval/transaction.go | 11 +- pkg/kv/kvserver/batcheval/transaction_test.go | 21 +- .../client_atomic_membership_change_test.go | 7 +- pkg/kv/kvserver/client_lease_test.go | 29 +- pkg/kv/kvserver/client_merge_test.go | 217 +-- pkg/kv/kvserver/client_metrics_test.go | 9 +- pkg/kv/kvserver/client_migration_test.go | 3 +- pkg/kv/kvserver/client_raft_helpers_test.go | 7 +- pkg/kv/kvserver/client_raft_test.go | 205 +- pkg/kv/kvserver/client_rangefeed_test.go | 3 +- pkg/kv/kvserver/client_relocate_range_test.go | 23 +- .../client_replica_backpressure_test.go | 5 +- .../client_replica_circuit_breaker_test.go | 87 +- pkg/kv/kvserver/client_replica_gc_test.go | 5 +- pkg/kv/kvserver/client_replica_test.go | 263 +-- pkg/kv/kvserver/client_split_burst_test.go | 3 +- pkg/kv/kvserver/client_split_test.go | 157 +- pkg/kv/kvserver/client_tenant_test.go | 15 +- pkg/kv/kvserver/client_test.go | 65 +- pkg/kv/kvserver/closed_timestamp_test.go | 65 +- pkg/kv/kvserver/concurrency/BUILD.bazel | 2 + .../concurrency/concurrency_control.go | 21 +- .../concurrency/concurrency_manager.go | 23 +- .../concurrency/concurrency_manager_test.go | 59 +- .../concurrency/datadriven_util_test.go | 15 +- pkg/kv/kvserver/concurrency/latch_manager.go | 8 +- pkg/kv/kvserver/concurrency/lock_table.go | 5 +- .../kvserver/concurrency/lock_table_waiter.go | 51 +- .../concurrency/lock_table_waiter_test.go | 61 +- pkg/kv/kvserver/consistency_queue.go | 5 +- pkg/kv/kvserver/consistency_queue_test.go | 49 +- pkg/kv/kvserver/gc/BUILD.bazel | 2 + pkg/kv/kvserver/gc/gc.go | 27 +- pkg/kv/kvserver/gc/gc_old_test.go | 7 +- pkg/kv/kvserver/gc/gc_random_test.go | 33 +- pkg/kv/kvserver/gc/gc_test.go | 43 +- pkg/kv/kvserver/helpers_test.go | 27 +- .../intent_resolver_integration_test.go | 37 +- pkg/kv/kvserver/intentresolver/BUILD.bazel | 2 + .../intentresolver/intent_resolver.go | 65 +- .../intentresolver/intent_resolver_test.go | 75 +- pkg/kv/kvserver/kvadmission/BUILD.bazel | 1 + pkg/kv/kvserver/kvadmission/kvadmission.go | 13 +- pkg/kv/kvserver/kvserverbase/BUILD.bazel | 1 + pkg/kv/kvserver/kvserverbase/base.go | 27 +- pkg/kv/kvserver/kvserverbase/bulk_adder.go | 9 +- pkg/kv/kvserver/kvserverbase/forced_error.go | 19 +- pkg/kv/kvserver/kvserverpb/BUILD.bazel | 2 + pkg/kv/kvserver/kvserverpb/proposer_kv.proto | 2 +- pkg/kv/kvserver/kvserverpb/raft.proto | 2 +- pkg/kv/kvserver/kvstorage/BUILD.bazel | 1 + pkg/kv/kvserver/kvstorage/replica_state.go | 5 +- pkg/kv/kvserver/liveness/BUILD.bazel | 1 + pkg/kv/kvserver/liveness/liveness.go | 11 +- pkg/kv/kvserver/loqrecovery/BUILD.bazel | 1 + .../loqrecovery/collect_raft_log_test.go | 3 +- pkg/kv/kvserver/merge_queue.go | 15 +- pkg/kv/kvserver/mvcc_gc_queue.go | 25 +- pkg/kv/kvserver/mvcc_gc_queue_test.go | 29 +- pkg/kv/kvserver/node_liveness_test.go | 23 +- .../kvserver/protectedts/ptcache/BUILD.bazel | 1 + .../protectedts/ptcache/cache_test.go | 33 +- pkg/kv/kvserver/queue.go | 5 +- pkg/kv/kvserver/queue_concurrency_test.go | 3 +- pkg/kv/kvserver/raft_log_queue.go | 5 +- pkg/kv/kvserver/raft_log_queue_test.go | 7 +- pkg/kv/kvserver/raft_transport.go | 11 +- pkg/kv/kvserver/raft_transport_test.go | 9 +- pkg/kv/kvserver/raftlog/encoding.go | 2 +- pkg/kv/kvserver/rangefeed/BUILD.bazel | 2 + pkg/kv/kvserver/rangefeed/catchup_scan.go | 15 +- .../rangefeed/catchup_scan_bench_test.go | 3 +- .../kvserver/rangefeed/catchup_scan_test.go | 9 +- pkg/kv/kvserver/rangefeed/processor.go | 45 +- pkg/kv/kvserver/rangefeed/processor_test.go | 95 +- pkg/kv/kvserver/rangefeed/registry.go | 64 +- pkg/kv/kvserver/rangefeed/registry_test.go | 75 +- pkg/kv/kvserver/rangefeed/task.go | 3 +- pkg/kv/kvserver/replica.go | 57 +- pkg/kv/kvserver/replica_app_batch.go | 15 +- pkg/kv/kvserver/replica_application_cmd.go | 8 +- pkg/kv/kvserver/replica_application_result.go | 17 +- .../replica_application_state_machine.go | 11 +- .../replica_application_state_machine_test.go | 9 +- pkg/kv/kvserver/replica_backpressure.go | 11 +- pkg/kv/kvserver/replica_batch_updates.go | 29 +- pkg/kv/kvserver/replica_batch_updates_test.go | 37 +- pkg/kv/kvserver/replica_circuit_breaker.go | 11 +- .../replica_closedts_internal_test.go | 77 +- pkg/kv/kvserver/replica_closedts_test.go | 55 +- pkg/kv/kvserver/replica_command.go | 122 +- pkg/kv/kvserver/replica_command_test.go | 91 +- pkg/kv/kvserver/replica_consistency.go | 47 +- pkg/kv/kvserver/replica_consistency_test.go | 11 +- pkg/kv/kvserver/replica_corruption.go | 8 +- pkg/kv/kvserver/replica_destroy.go | 5 +- pkg/kv/kvserver/replica_eval_context_span.go | 3 +- pkg/kv/kvserver/replica_evaluate.go | 73 +- pkg/kv/kvserver/replica_evaluate_test.go | 13 +- pkg/kv/kvserver/replica_follower_read.go | 11 +- pkg/kv/kvserver/replica_follower_read_test.go | 13 +- pkg/kv/kvserver/replica_gc_queue.go | 5 +- pkg/kv/kvserver/replica_gossip.go | 19 +- pkg/kv/kvserver/replica_learner_test.go | 21 +- pkg/kv/kvserver/replica_probe_test.go | 23 +- pkg/kv/kvserver/replica_proposal.go | 21 +- .../kvserver/replica_proposal_bench_test.go | 5 +- pkg/kv/kvserver/replica_proposal_buf.go | 9 +- pkg/kv/kvserver/replica_proposal_buf_test.go | 19 +- pkg/kv/kvserver/replica_raft.go | 53 +- pkg/kv/kvserver/replica_raftstorage.go | 5 +- pkg/kv/kvserver/replica_range_lease.go | 97 +- pkg/kv/kvserver/replica_rangefeed.go | 59 +- pkg/kv/kvserver/replica_rangefeed_test.go | 175 +- pkg/kv/kvserver/replica_rankings_test.go | 41 +- pkg/kv/kvserver/replica_rate_limit.go | 5 +- pkg/kv/kvserver/replica_read.go | 73 +- pkg/kv/kvserver/replica_send.go | 231 ++- pkg/kv/kvserver/replica_sideload_test.go | 11 +- pkg/kv/kvserver/replica_split_load.go | 11 +- pkg/kv/kvserver/replica_split_load_test.go | 161 +- pkg/kv/kvserver/replica_test.go | 1677 ++++++++--------- pkg/kv/kvserver/replica_tscache.go | 83 +- pkg/kv/kvserver/replica_write.go | 75 +- pkg/kv/kvserver/replicate_queue.go | 35 +- pkg/kv/kvserver/replicate_queue_test.go | 3 +- pkg/kv/kvserver/reports/BUILD.bazel | 2 + pkg/kv/kvserver/reports/reporter.go | 7 +- pkg/kv/kvserver/reports/reporter_test.go | 3 +- pkg/kv/kvserver/reset_quorum_test.go | 13 +- pkg/kv/kvserver/scatter_test.go | 7 +- pkg/kv/kvserver/spanlatch/BUILD.bazel | 1 + pkg/kv/kvserver/spanlatch/manager.go | 3 +- pkg/kv/kvserver/split_queue.go | 13 +- pkg/kv/kvserver/store.go | 23 +- pkg/kv/kvserver/store_create_replica.go | 9 +- pkg/kv/kvserver/store_raft.go | 37 +- pkg/kv/kvserver/store_remove_replica.go | 7 +- pkg/kv/kvserver/store_send.go | 65 +- pkg/kv/kvserver/store_snapshot.go | 7 +- pkg/kv/kvserver/store_test.go | 207 +- pkg/kv/kvserver/stores.go | 29 +- pkg/kv/kvserver/stores_test.go | 3 +- pkg/kv/kvserver/testing_knobs.go | 15 +- pkg/kv/kvserver/ts_maintenance_queue_test.go | 3 +- .../kvserver/txn_recovery_integration_test.go | 47 +- pkg/kv/kvserver/txn_wait_queue_test.go | 83 +- pkg/kv/kvserver/txnrecovery/BUILD.bazel | 2 + pkg/kv/kvserver/txnrecovery/manager.go | 25 +- pkg/kv/kvserver/txnrecovery/manager_test.go | 75 +- pkg/kv/kvserver/txnwait/BUILD.bazel | 2 + pkg/kv/kvserver/txnwait/queue.go | 77 +- pkg/kv/kvserver/txnwait/queue_test.go | 259 +-- pkg/kv/kvserver/uncertainty/BUILD.bazel | 2 + pkg/kv/kvserver/uncertainty/compute.go | 7 +- pkg/kv/kvserver/uncertainty/compute_test.go | 3 +- pkg/kv/kvserver/uncertainty/doc.go | 5 +- pkg/kv/mock_transactional_sender.go | 33 +- pkg/kv/range_lookup.go | 41 +- pkg/kv/range_lookup_test.go | 19 +- pkg/kv/sender.go | 39 +- pkg/kv/txn.go | 83 +- pkg/kv/txn_external_test.go | 43 +- pkg/kv/txn_test.go | 131 +- pkg/multitenant/BUILD.bazel | 1 + pkg/multitenant/mtinfopb/BUILD.bazel | 5 +- pkg/multitenant/mtinfopb/info.proto | 2 +- pkg/multitenant/tenant_usage.go | 5 +- .../tenantcapabilities/BUILD.bazel | 1 + .../tenantcapabilities/capabilities.go | 3 +- .../tenantcapabilitiesauthorizer/BUILD.bazel | 1 + .../authorizer.go | 5 +- .../tenantcapabilitiesauthorizer/noop.go | 3 +- .../tenantcapabilitiestestutils/BUILD.bazel | 1 + .../tenantcapabilitiestestutils/testutils.go | 9 +- .../tenantcapabilitieswatcher/BUILD.bazel | 1 + .../tenantcapabilitieswatcher/decoder.go | 3 +- pkg/multitenant/tenantcostmodel/BUILD.bazel | 2 +- pkg/multitenant/tenantcostmodel/model.go | 18 +- pkg/repstream/streampb/BUILD.bazel | 2 + pkg/repstream/streampb/stream.proto | 2 +- pkg/roachpb/.gitattributes | 1 - pkg/roachpb/BUILD.bazel | 62 +- pkg/roachpb/data.go | 155 +- pkg/roachpb/data_test.go | 11 +- pkg/roachpb/string_test.go | 71 - pkg/rpc/BUILD.bazel | 2 + pkg/rpc/auth_tenant.go | 89 +- pkg/rpc/auth_test.go | 141 +- pkg/rpc/codec_test.go | 9 +- pkg/rpc/context.go | 85 +- pkg/rpc/context_test.go | 83 +- pkg/rpc/nodedialer/BUILD.bazel | 2 + pkg/rpc/nodedialer/nodedialer.go | 11 +- pkg/rpc/nodedialer/nodedialer_test.go | 35 +- pkg/rpc/restricted_internal_client.go | 10 +- pkg/server/BUILD.bazel | 2 + pkg/server/admin.go | 9 +- pkg/server/authentication_test.go | 5 +- pkg/server/bench_test.go | 3 +- pkg/server/config.go | 3 +- pkg/server/import_ts.go | 5 +- pkg/server/init.go | 9 +- pkg/server/intent_test.go | 7 +- pkg/server/node.go | 171 +- pkg/server/node_test.go | 21 +- pkg/server/server.go | 3 +- pkg/server/server_test.go | 5 +- pkg/server/serverpb/BUILD.bazel | 2 + pkg/server/serverpb/admin.proto | 2 +- pkg/server/settingswatcher/BUILD.bazel | 2 + .../settingswatcher/settings_watcher.go | 5 +- .../settings_watcher_external_test.go | 11 +- pkg/server/status/BUILD.bazel | 1 + pkg/server/status/recorder.go | 3 +- pkg/server/systemconfigwatcher/BUILD.bazel | 1 + pkg/server/systemconfigwatcher/cache.go | 5 +- .../systemconfigwatchertest/BUILD.bazel | 1 + .../test_system_config_watcher.go | 5 +- pkg/server/tenantsettingswatcher/BUILD.bazel | 2 + .../tenantsettingswatcher/overrides_store.go | 15 +- .../overrides_store_test.go | 7 +- .../tenantsettingswatcher/row_decoder.go | 13 +- pkg/server/tenantsettingswatcher/watcher.go | 11 +- .../tenantsettingswatcher/watcher_test.go | 3 +- pkg/server/testserver.go | 29 +- .../spanconfigkvsubscriber/BUILD.bazel | 1 + .../spanconfigdecoder.go | 3 +- .../spanconfigsqlwatcher/BUILD.bazel | 1 + .../spanconfigsqlwatcher/sqlwatcher.go | 7 +- pkg/sql/BUILD.bazel | 2 + pkg/sql/ambiguous_commit_test.go | 25 +- pkg/sql/as_of_test.go | 14 +- pkg/sql/backfill.go | 3 +- pkg/sql/backfill/BUILD.bazel | 1 + pkg/sql/backfill/mvcc_index_merger.go | 13 +- pkg/sql/catalog/descs/BUILD.bazel | 2 +- pkg/sql/catalog/descs/collection_test.go | 8 +- pkg/sql/catalog/ingesting/BUILD.bazel | 1 + pkg/sql/catalog/ingesting/write_descs.go | 3 +- pkg/sql/catalog/lease/BUILD.bazel | 2 + pkg/sql/catalog/lease/kv_writer_test.go | 9 +- pkg/sql/catalog/lease/lease.go | 13 +- pkg/sql/catalog/lease/lease_test.go | 27 +- pkg/sql/catalog/lease/storage.go | 3 +- pkg/sql/colexecop/BUILD.bazel | 2 +- pkg/sql/colexecop/operator.go | 4 +- pkg/sql/colfetcher/BUILD.bazel | 1 + pkg/sql/colfetcher/colbatch_scan.go | 10 +- pkg/sql/colfetcher/index_join.go | 3 +- pkg/sql/colflow/colrpc/inbox.go | 2 +- pkg/sql/conn_executor.go | 9 +- pkg/sql/conn_executor_internal_test.go | 5 +- pkg/sql/conn_executor_test.go | 45 +- pkg/sql/consistencychecker/BUILD.bazel | 1 + .../consistencychecker/consistency_checker.go | 11 +- pkg/sql/contention/BUILD.bazel | 2 + pkg/sql/contention/event_store_test.go | 8 +- pkg/sql/contention/registry.go | 9 +- pkg/sql/contention/registry_test.go | 9 +- pkg/sql/contentionpb/BUILD.bazel | 3 +- pkg/sql/contentionpb/contention.proto | 2 +- pkg/sql/copy/BUILD.bazel | 2 +- pkg/sql/copy/copy_in_test.go | 12 +- pkg/sql/crdb_internal.go | 11 +- pkg/sql/crdb_internal_test.go | 5 +- pkg/sql/distsql_running.go | 5 +- pkg/sql/distsql_running_test.go | 13 +- pkg/sql/drop_test.go | 15 +- pkg/sql/exec_util.go | 3 +- pkg/sql/execinfra/BUILD.bazel | 1 + pkg/sql/execinfra/processorsbase.go | 8 +- pkg/sql/execinfrapb/BUILD.bazel | 2 + pkg/sql/execinfrapb/component_stats.proto | 2 +- pkg/sql/execinfrapb/data.proto | 2 +- pkg/sql/execinfrapb/processors_bulk_io.proto | 2 +- pkg/sql/execstats/BUILD.bazel | 4 +- pkg/sql/execstats/stats.go | 12 +- pkg/sql/execstats/traceanalyzer.go | 6 +- pkg/sql/execstats/traceanalyzer_test.go | 12 +- pkg/sql/flowinfra/BUILD.bazel | 1 + pkg/sql/flowinfra/cluster_test.go | 7 +- pkg/sql/gcjob/BUILD.bazel | 1 + pkg/sql/gcjob/gc_job.go | 5 +- pkg/sql/gcjob/table_garbage_collection.go | 13 +- pkg/sql/gcjob_test/BUILD.bazel | 1 + pkg/sql/gcjob_test/gc_job_test.go | 17 +- pkg/sql/importer/BUILD.bazel | 2 + pkg/sql/importer/exportcsv_test.go | 12 +- pkg/sql/importer/import_job.go | 17 +- pkg/sql/importer/import_processor.go | 13 +- pkg/sql/importer/import_processor_planning.go | 19 +- pkg/sql/importer/import_processor_test.go | 15 +- pkg/sql/importer/import_stmt_test.go | 5 +- pkg/sql/importer/read_import_base.go | 5 +- pkg/sql/insert_fast_path.go | 13 +- pkg/sql/internal.go | 3 +- pkg/sql/metric_test.go | 12 +- pkg/sql/mvcc_backfiller_test.go | 5 +- pkg/sql/opt/norm/BUILD.bazel | 2 +- pkg/sql/opt/norm/fold_constants_funcs.go | 6 +- pkg/sql/pgwire/pgerror/BUILD.bazel | 1 + pkg/sql/pgwire/pgerror/errors_test.go | 6 +- pkg/sql/pgwire/pgerror/flatten.go | 2 +- pkg/sql/pgwire/pgerror/flatten_test.go | 19 +- pkg/sql/pgwire/pgerror/pgcode.go | 4 +- pkg/sql/pgwire/pgerror/wrap_test.go | 6 +- pkg/sql/physicalplan/BUILD.bazel | 1 + pkg/sql/physicalplan/fake_span_resolver.go | 3 +- pkg/sql/relocate_range.go | 3 +- pkg/sql/repair.go | 7 +- pkg/sql/revert.go | 19 +- pkg/sql/revert_test.go | 8 +- pkg/sql/row/BUILD.bazel | 1 + pkg/sql/row/errors.go | 15 +- pkg/sql/row/expr_walker.go | 4 +- pkg/sql/row/fetcher.go | 3 +- pkg/sql/row/kv_batch_fetcher.go | 59 +- pkg/sql/row/kv_batch_streamer.go | 7 +- pkg/sql/row/kv_fetcher.go | 15 +- pkg/sql/rowcontainer/BUILD.bazel | 2 + .../kvstreamer_result_disk_buffer.go | 5 +- .../kvstreamer_result_disk_buffer_test.go | 5 +- pkg/sql/rowexec/BUILD.bazel | 2 + pkg/sql/rowexec/bulk_row_writer.go | 3 +- pkg/sql/rowexec/indexbackfiller.go | 3 +- pkg/sql/rowexec/processors_test.go | 23 +- pkg/sql/run_control_test.go | 11 +- pkg/sql/scatter.go | 7 +- pkg/sql/schema_changer.go | 7 +- pkg/sql/schema_changer_test.go | 13 +- pkg/sql/schemachanger/scrun/BUILD.bazel | 2 +- pkg/sql/schemachanger/scrun/scrun.go | 4 +- pkg/sql/schemachanger/sctest/BUILD.bazel | 2 +- pkg/sql/schemachanger/sctest/cumulative.go | 4 +- pkg/sql/sem/builtins/BUILD.bazel | 2 + pkg/sql/sem/builtins/builtins.go | 35 +- .../sem/builtins/fingerprint_builtin_test.go | 7 +- pkg/sql/sem/builtins/generator_builtins.go | 23 +- pkg/sql/sem/eval/BUILD.bazel | 2 + pkg/sql/sem/eval/context.go | 7 +- pkg/sql/sem/eval/timeconv_test.go | 5 +- pkg/sql/sequence.go | 4 +- .../sqlinstance/instancestorage/BUILD.bazel | 1 + .../instancestorage/instancereader.go | 3 +- pkg/sql/sqlliveness/slstorage/BUILD.bazel | 1 + .../sqlliveness/slstorage/slstorage_test.go | 28 +- .../sqlstats/persistedsqlstats/BUILD.bazel | 1 + .../persistedsqlstats/compaction_test.go | 9 +- pkg/sql/stats/BUILD.bazel | 3 +- pkg/sql/stats/create_stats_job_test.go | 8 +- pkg/sql/stats/stats_cache.go | 5 +- pkg/sql/stmtdiagnostics/BUILD.bazel | 1 + .../statement_diagnostics_test.go | 3 +- pkg/sql/tenant_creation.go | 3 +- pkg/sql/tenant_gc.go | 5 +- pkg/sql/tests/BUILD.bazel | 1 + pkg/sql/tests/command_filters.go | 6 +- pkg/sql/truncate.go | 9 +- pkg/sql/txn_restart_test.go | 57 +- pkg/sql/txn_state_test.go | 5 +- pkg/sql/upsert_test.go | 9 +- pkg/sql/user_test.go | 4 +- pkg/storage/BUILD.bazel | 2 + pkg/storage/bench_test.go | 5 +- pkg/storage/enginepb/mvcc3.proto | 2 +- pkg/storage/metamorphic/BUILD.bazel | 1 + pkg/storage/metamorphic/operations.go | 9 +- pkg/storage/mvcc.go | 114 +- pkg/storage/mvcc_history_test.go | 9 +- pkg/storage/mvcc_incremental_iterator.go | 9 +- pkg/storage/mvcc_incremental_iterator_test.go | 15 +- pkg/storage/mvcc_stats_test.go | 11 +- pkg/storage/mvcc_test.go | 135 +- pkg/storage/pebble_mvcc_scanner.go | 19 +- pkg/storage/row_counter.go | 5 +- pkg/storage/sst.go | 15 +- pkg/storage/sst_test.go | 3 +- .../testdata/mvcc_histories/conditional_put | 12 +- .../conditional_put_with_txn_enable_separated | 2 +- .../conditional_put_write_too_old | 8 +- .../testdata/mvcc_histories/delete_range | 2 +- .../mvcc_histories/delete_range_predicate | 8 +- pkg/storage/testdata/mvcc_histories/deletes | 8 +- pkg/storage/testdata/mvcc_histories/export | 6 +- .../mvcc_histories/export_fingerprint | 6 +- .../mvcc_histories/ignored_seq_nums_cput | 12 +- pkg/storage/testdata/mvcc_histories/increment | 4 +- .../mvcc_histories/range_tombstone_conflicts | 18 +- .../mvcc_histories/range_tombstone_gets | 30 +- .../range_tombstone_iter_incremental | 10 +- .../mvcc_histories/range_tombstone_scans | 32 +- .../mvcc_histories/range_tombstone_writes | 28 +- .../range_tombstone_writes_idempotent | 12 +- .../mvcc_histories/read_fail_on_more_recent | 44 +- .../testdata/mvcc_histories/skip_locked | 82 +- .../mvcc_histories/uncertainty_interval | 76 +- ...inty_interval_with_local_uncertainty_limit | 624 +++--- ...uncertainty_limit_and_synthetic_timestamps | 648 +++---- ...uncertainty_limit_disable_local_timestamps | 416 ++-- .../update_existing_key_diff_txn | 2 +- .../update_existing_key_old_version | 4 +- .../testdata/mvcc_histories/write_too_old | 4 +- pkg/testutils/BUILD.bazel | 2 +- pkg/testutils/error.go | 4 +- pkg/testutils/jobutils/BUILD.bazel | 2 +- pkg/testutils/jobutils/jobs_verification.go | 6 +- pkg/testutils/kvclientutils/BUILD.bazel | 1 + pkg/testutils/kvclientutils/txn_recovery.go | 9 +- pkg/testutils/lint/lint_test.go | 22 +- .../lint/passes/errwrap/functions.go | 2 +- .../lint/passes/fmtsafe/functions.go | 4 +- .../lint/passes/redactcheck/redactcheck.go | 6 +- .../lint/passes/returncheck/returncheck.go | 12 +- pkg/testutils/storageutils/BUILD.bazel | 1 + pkg/testutils/storageutils/mocking.go | 12 +- pkg/testutils/testcluster/BUILD.bazel | 2 + pkg/testutils/testcluster/testcluster.go | 9 +- pkg/testutils/testcluster/testcluster_test.go | 5 +- pkg/ts/BUILD.bazel | 2 + pkg/ts/db.go | 5 +- pkg/ts/pruning.go | 5 +- pkg/ts/server.go | 3 +- pkg/ts/server_test.go | 5 +- pkg/util/admission/admission.go | 2 +- pkg/util/hlc/doc.go | 8 +- pkg/util/protoutil/BUILD.bazel | 1 + pkg/util/protoutil/clone_test.go | 3 +- pkg/util/stop/BUILD.bazel | 4 +- pkg/util/stop/stopper.go | 4 +- pkg/util/stop/stopper_test.go | 4 +- 731 files changed, 10599 insertions(+), 9828 deletions(-) create mode 100644 pkg/kv/kvpb/.gitattributes create mode 100644 pkg/kv/kvpb/BUILD.bazel rename pkg/{roachpb => kv/kvpb}/ambiguous_result_error.go (84%) rename pkg/{roachpb => kv/kvpb}/api.go (94%) rename pkg/{roachpb => kv/kvpb}/api.proto (98%) rename pkg/{roachpb => kv/kvpb}/api_requestheader.go (99%) rename pkg/{roachpb => kv/kvpb}/api_test.go (78%) rename pkg/{roachpb => kv/kvpb}/batch.go (97%) rename pkg/{roachpb => kv/kvpb}/batch_generated.go (99%) rename pkg/{roachpb => kv/kvpb}/batch_test.go (87%) create mode 100644 pkg/kv/kvpb/data.go rename pkg/{roachpb => kv/kvpb}/errordetailtype_string.go (99%) rename pkg/{roachpb => kv/kvpb}/errors.go (89%) rename pkg/{roachpb => kv/kvpb}/errors.proto (95%) rename pkg/{roachpb => kv/kvpb}/errors_test.go (92%) rename pkg/{roachpb => kv/kvpb}/gen.bzl (90%) rename pkg/{roachpb => kv/kvpb}/gen/BUILD.bazel (87%) rename pkg/{roachpb => kv/kvpb}/gen/main.go (99%) rename pkg/{roachpb/roachpbmock => kv/kvpb/kvpbmock}/BUILD.bazel (69%) rename pkg/{roachpb/roachpbmock => kv/kvpb/kvpbmock}/mocks_generated.go (90%) rename pkg/{roachpb => kv/kvpb}/method.go (99%) rename pkg/{roachpb => kv/kvpb}/method_string.go (99%) rename pkg/{roachpb => kv/kvpb}/replica_unavailable_error.go (80%) rename pkg/{roachpb => kv/kvpb}/replica_unavailable_error_test.go (96%) create mode 100644 pkg/kv/kvpb/string_test.go rename pkg/{roachpb => kv/kvpb}/testdata/ambiguous_result_error.txt (100%) rename pkg/{roachpb => kv/kvpb}/testdata/replica_unavailable_error.txt (100%) diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index bfb82f5cee48..d36655228245 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -59,6 +59,7 @@ /pkg/sql/job_exec_context* @cockroachdb/sql-queries @cockroachdb/jobs-prs /pkg/sql/delegate/*job*.go @cockroachdb/jobs-prs +/pkg/sql/appstatspb @cockroachdb/sql-observability /pkg/sql/execstats/ @cockroachdb/sql-observability /pkg/sql/scheduledlogging/ @cockroachdb/sql-observability /pkg/sql/sqlstats/ @cockroachdb/sql-observability @@ -196,6 +197,19 @@ /pkg/kv/kvclient/kvstreamer @cockroachdb/sql-queries /pkg/kv/kvclient/rangefeed/ @cockroachdb/repl-prs /pkg/kv/kvnemesis/ @cockroachdb/kv-prs +/pkg/kv/kvpb/ @cockroachdb/kv-prs +/pkg/kv/kvpb/.gitattributes @cockroachdb/dev-inf +/pkg/kv/kvpb/ambiguous_* @cockroachdb/kv-prs +/pkg/kv/kvpb/api* @cockroachdb/kv-prs +/pkg/kv/kvpb/batch* @cockroachdb/kv-prs +/pkg/kv/kvpb/error* @cockroachdb/kv-prs +/pkg/kv/kvpb/gen @cockroachdb/dev-inf +/pkg/kv/kvpb/gen.bzl @cockroachdb/dev-inf +/pkg/kv/kvpb/kvpbmock/ @cockroachdb/kv-prs +/pkg/kv/kvpb/method* @cockroachdb/kv-prs +/pkg/kv/kvpb/string_test.go @cockroachdb/kv-prs +/pkg/kv/kvpb/testdata/ambi* @cockroachdb/kv-prs +/pkg/kv/kvpb/testdata/repl* @cockroachdb/kv-prs /pkg/kv/kvprober/ @cockroachdb/kv-prs # Same subdirectory rule as above for `/pkg/kv` /pkg/kv/kvserver/*.* @cockroachdb/kv-prs @@ -393,30 +407,17 @@ /pkg/multitenant/ @cockroachdb/multi-tenant /pkg/release/ @cockroachdb/dev-inf /pkg/roachpb/.gitattributes @cockroachdb/dev-inf -/pkg/roachpb/ambiguous_* @cockroachdb/kv-prs -/pkg/roachpb/api* @cockroachdb/kv-prs -/pkg/roachpb/batch* @cockroachdb/kv-prs #!/pkg/roachpb/BUILD.bazel @cockroachdb/kv-prs-noreview /pkg/roachpb/data* @cockroachdb/kv-prs -/pkg/roachpb/dep_test.go @cockroachdb/dev-inf -/pkg/roachpb/error* @cockroachdb/kv-prs -/pkg/roachpb/gen @cockroachdb/dev-inf -/pkg/roachpb/gen.bzl @cockroachdb/dev-inf -/pkg/roachpb/app* @cockroachdb/sql-observability /pkg/roachpb/index* @cockroachdb/sql-observability /pkg/roachpb/internal* @cockroachdb/kv-prs /pkg/roachpb/io-formats* @cockroachdb/disaster-recovery #!/pkg/roachpb/main_test.go @cockroachdb/kv-prs-noreview /pkg/roachpb/merge_spans* @cockroachdb/kv-prs /pkg/roachpb/metadata* @cockroachdb/kv-prs -/pkg/roachpb/method* @cockroachdb/kv-prs -/pkg/roachpb/roachpbmock/ @cockroachdb/kv-prs -/pkg/roachpb/replica_* @cockroachdb/kv-prs /pkg/roachpb/span* @cockroachdb/kv-prs /pkg/roachpb/string_test.go @cockroachdb/kv-prs /pkg/roachpb/tenant* @cockroachdb/kv-prs -/pkg/roachpb/testdata/ambi* @cockroachdb/kv-prs -/pkg/roachpb/testdata/repl* @cockroachdb/kv-prs #!/pkg/roachpb/version* @cockroachdb/unowned /pkg/roachprod/ @cockroachdb/test-eng /pkg/rpc/ @cockroachdb/kv-prs diff --git a/BUILD.bazel b/BUILD.bazel index 172d3f476c00..1205a706d375 100644 --- a/BUILD.bazel +++ b/BUILD.bazel @@ -58,18 +58,18 @@ exports_files([ # gazelle:resolve go google.golang.org/genproto/googleapis/pubsub/v1 @org_golang_google_genproto//googleapis/pubsub/v1:pubsub # gazelle:resolve go google.golang.org/genproto/googleapis/cloud/kms/v1 @org_golang_google_genproto//googleapis/cloud/kms/v1:kms -# See pkg/roachpb/gen/BUILD.bazel for more details. +# See pkg/kv/kvpb/gen/BUILD.bazel for more details. # -# gazelle:resolve proto go roachpb/api.proto //pkg/roachpb +# gazelle:resolve proto go kv/kvpb/api.proto //pkg/kv/kvpb +# gazelle:resolve proto go kv/kvpb/errors.proto //pkg/kv/kvpb # gazelle:resolve proto go roachpb/data.proto //pkg/roachpb -# gazelle:resolve proto go roachpb/errors.proto //pkg/roachpb # gazelle:resolve proto go roachpb/index_usage_stats.proto //pkg/roachpb # gazelle:resolve proto go roachpb/internal.proto //pkg/roachpb # gazelle:resolve proto go roachpb/internal_raft.proto //pkg/roachpb # gazelle:resolve proto go roachpb/io-formats.proto //pkg/roachpb # gazelle:resolve proto go roachpb/metadata.proto //pkg/roachpb # gazelle:resolve proto go roachpb/span_config.proto //pkg/roachpb -# gazelle:exclude pkg/roachpb/batch_generated.go +# gazelle:exclude pkg/kv/kvpb/batch_generated.go # See pkg/sql/opt/optgen/cmd/langgen/BUILD.bazel for more details. # diff --git a/build/bazelutil/check.sh b/build/bazelutil/check.sh index 117b2fa5bf47..6a957e354872 100755 --- a/build/bazelutil/check.sh +++ b/build/bazelutil/check.sh @@ -22,8 +22,8 @@ pkg/kv/kvclient/rangecache/range_cache.go://go:generate mockgen -package=rangeca pkg/kv/kvclient/rangefeed/rangefeed.go://go:generate mockgen -destination=mocks_generated_test.go --package=rangefeed . DB pkg/kv/kvserver/concurrency/lock_table.go://go:generate ../../../util/interval/generic/gen.sh *lockState concurrency pkg/kv/kvserver/spanlatch/manager.go://go:generate ../../../util/interval/generic/gen.sh *latch spanlatch -pkg/roachpb/api.go://go:generate mockgen -package=roachpbmock -destination=roachpbmock/mocks_generated.go . InternalClient,Internal_RangeFeedClient,Internal_MuxRangeFeedClient -pkg/roachpb/batch.go://go:generate go run gen/main.go --filename batch_generated.go *.pb.go +pkg/kv/kvpb/api.go://go:generate mockgen -package=kvpbmock -destination=kvpbmock/mocks_generated.go . InternalClient,Internal_RangeFeedClient,Internal_MuxRangeFeedClient +pkg/kv/kvpb/batch.go://go:generate go run gen/main.go --filename batch_generated.go *.pb.go pkg/security/certmgr/cert.go://go:generate mockgen -package=certmgr -destination=mocks_generated_test.go . Cert pkg/security/securitytest/securitytest.go://go:generate go-bindata -mode 0600 -modtime 1400000000 -pkg securitytest -o embedded.go -ignore README.md -ignore regenerate.sh test_certs pkg/security/securitytest/securitytest.go://go:generate gofmt -s -w embedded.go diff --git a/build/bazelutil/nogo_config.json b/build/bazelutil/nogo_config.json index 084f4f3b24e1..097a73513593 100644 --- a/build/bazelutil/nogo_config.json +++ b/build/bazelutil/nogo_config.json @@ -47,8 +47,8 @@ ".*\\.pb\\.gw\\.go$": "generated code", "cockroach/pkg/.*_generated\\.go$": "generated code", "cockroach/pkg/kv/kvclient/kvcoord/lock_spans_over_budget_error\\.go$": "invalid direct cast on error object", - "cockroach/pkg/roachpb/batch_generated-gen\\.go$": "invalid direct cast on error object", - "cockroach/pkg/roachpb/errors\\.go$": "invalid direct cast on error object", + "cockroach/pkg/kv/kvpb/batch_generated-gen\\.go$": "invalid direct cast on error object", + "cockroach/pkg/kv/kvpb/errors\\.go$": "invalid direct cast on error object", "cockroach/pkg/sql/pgwire/pgerror/constraint_name\\.go$": "invalid direct cast on error object", "cockroach/pkg/sql/pgwire/pgerror/pgcode\\.go$": "invalid direct cast on error object", "cockroach/pkg/testutils/lint/lint_test\\.go$": "invalid direct cast on error object", @@ -719,7 +719,7 @@ }, "SA1019": { "exclude_files": { - "cockroach/pkg/roachpb/api_test.go$": "same package that grpc-go imports", + "cockroach/pkg/kv/kvpb/api_test.go$": "same package that grpc-go imports", "cockroach/pkg/rpc/codec.go$": "rpc/codec.go imports the same proto package that grpc-go imports (as of crdb@dd87d1145 and grpc-go@7b167fd6).", "cockroach/pkg/rpc/stats_handler.go$": "Using deprecated WireLength call", "cockroach/pkg/.*\\.eg\\.go$": "generated code", diff --git a/docs/generated/http/BUILD.bazel b/docs/generated/http/BUILD.bazel index 2ad3dbaf2630..aaa68e421d2f 100644 --- a/docs/generated/http/BUILD.bazel +++ b/docs/generated/http/BUILD.bazel @@ -8,6 +8,7 @@ genrule( "//pkg/geo/geopb:geopb_proto", "//pkg/gossip:gossip_proto", "//pkg/jobs/jobspb:jobspb_proto", + "//pkg/kv/kvpb:kvpb_proto", "//pkg/kv/kvserver/concurrency/lock:lock_proto", "//pkg/kv/kvserver/kvserverpb:kvserverpb_proto", "//pkg/kv/kvserver/liveness/livenesspb:livenesspb_proto", diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 0c7c9811d7df..972889f84109 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -184,6 +184,8 @@ ALL_TESTS = [ "//pkg/kv/kvclient/rangefeed/rangefeedcache:rangefeedcache_test", "//pkg/kv/kvclient/rangefeed:rangefeed_test", "//pkg/kv/kvnemesis:kvnemesis_test", + "//pkg/kv/kvpb:kvpb_disallowed_imports_test", + "//pkg/kv/kvpb:kvpb_test", "//pkg/kv/kvprober:kvprober_test", "//pkg/kv/kvserver/abortspan:abortspan_test", "//pkg/kv/kvserver/allocator/allocatorimpl:allocatorimpl_test", @@ -1180,6 +1182,11 @@ GO_TARGETS = [ "//pkg/kv/kvnemesis/kvnemesisutil:kvnemesisutil", "//pkg/kv/kvnemesis:kvnemesis", "//pkg/kv/kvnemesis:kvnemesis_test", + "//pkg/kv/kvpb/gen:gen", + "//pkg/kv/kvpb/gen:gen_lib", + "//pkg/kv/kvpb/kvpbmock:kvpbmock", + "//pkg/kv/kvpb:kvpb", + "//pkg/kv/kvpb:kvpb_test", "//pkg/kv/kvprober:kvprober", "//pkg/kv/kvprober:kvprober_test", "//pkg/kv/kvserver/abortspan:abortspan", @@ -1345,9 +1352,6 @@ GO_TARGETS = [ "//pkg/release:release", "//pkg/repstream/streampb:streampb", "//pkg/repstream:repstream", - "//pkg/roachpb/gen:gen", - "//pkg/roachpb/gen:gen_lib", - "//pkg/roachpb/roachpbmock:roachpbmock", "//pkg/roachpb:roachpb", "//pkg/roachpb:roachpb_test", "//pkg/roachprod/cloud:cloud", @@ -2620,6 +2624,9 @@ GET_X_DATA_TARGETS = [ "//pkg/kv/kvclient/rangestats:get_x_data", "//pkg/kv/kvnemesis:get_x_data", "//pkg/kv/kvnemesis/kvnemesisutil:get_x_data", + "//pkg/kv/kvpb:get_x_data", + "//pkg/kv/kvpb/gen:get_x_data", + "//pkg/kv/kvpb/kvpbmock:get_x_data", "//pkg/kv/kvprober:get_x_data", "//pkg/kv/kvserver:get_x_data", "//pkg/kv/kvserver/abortspan:get_x_data", @@ -2722,8 +2729,6 @@ GET_X_DATA_TARGETS = [ "//pkg/repstream:get_x_data", "//pkg/repstream/streampb:get_x_data", "//pkg/roachpb:get_x_data", - "//pkg/roachpb/gen:get_x_data", - "//pkg/roachpb/roachpbmock:get_x_data", "//pkg/roachprod:get_x_data", "//pkg/roachprod/cloud:get_x_data", "//pkg/roachprod/config:get_x_data", diff --git a/pkg/base/serverident/server_ident.go b/pkg/base/serverident/server_ident.go index e49df6f52758..8ac3ca0dfd28 100644 --- a/pkg/base/serverident/server_ident.go +++ b/pkg/base/serverident/server_ident.go @@ -54,7 +54,7 @@ type ServerIdentificationPayload interface { // sense for this interface to live in log). // // Note that this tenant ID should not be confused with the one put in the - // context by roachpb.NewContextForTenant(): that one is used by a server + // context by roachpb.ContextWithClientTenant(): that one is used by a server // handling an RPC call, referring to the tenant that's the client of the RPC. TenantID() interface{} } diff --git a/pkg/ccl/backupccl/BUILD.bazel b/pkg/ccl/backupccl/BUILD.bazel index 0896e8f021b3..6b2d002a19a5 100644 --- a/pkg/ccl/backupccl/BUILD.bazel +++ b/pkg/ccl/backupccl/BUILD.bazel @@ -62,6 +62,7 @@ go_library( "//pkg/kv", "//pkg/kv/bulk", "//pkg/kv/kvclient", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/kv/kvserver/batcheval", "//pkg/kv/kvserver/concurrency/lock", @@ -226,6 +227,7 @@ go_test( "//pkg/kv", "//pkg/kv/bulk", "//pkg/kv/kvclient/kvcoord", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/kv/kvserver/closedts", "//pkg/kv/kvserver/kvserverbase", diff --git a/pkg/ccl/backupccl/backup_intents_test.go b/pkg/ccl/backupccl/backup_intents_test.go index c7efe706661a..e6c69b2cd9eb 100644 --- a/pkg/ccl/backupccl/backup_intents_test.go +++ b/pkg/ccl/backupccl/backup_intents_test.go @@ -16,8 +16,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/ccl" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -43,7 +43,7 @@ func TestCleanupIntentsDuringBackupPerformanceRegression(t *testing.T) { // Interceptor catches requests that cleanup transactions of size 1000 which are // test data transactions. All other transaction commits pass though. - interceptor := func(ctx context.Context, req *roachpb.BatchRequest) *roachpb.Error { + interceptor := func(ctx context.Context, req *kvpb.BatchRequest) *kvpb.Error { endTxn := req.Requests[0].GetEndTxn() if endTxn != nil && !endTxn.Commit && len(endTxn.LockSpans) == perTransactionRowCount { // If this is a rollback of one the test's SQL transactions, allow the diff --git a/pkg/ccl/backupccl/backup_job.go b/pkg/ccl/backupccl/backup_job.go index 23e606de5f80..112e2f3a7797 100644 --- a/pkg/ccl/backupccl/backup_job.go +++ b/pkg/ccl/backupccl/backup_job.go @@ -32,6 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/joberror" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/scheduledjobs" @@ -72,7 +73,7 @@ var BackupCheckpointInterval = settings.RegisterDurationSetting( var forceReadBackupManifest = util.ConstantWithMetamorphicTestBool("backup-read-manifest", false) -func countRows(raw roachpb.BulkOpSummary, pkIDs map[uint64]bool) roachpb.RowCount { +func countRows(raw kvpb.BulkOpSummary, pkIDs map[uint64]bool) roachpb.RowCount { res := roachpb.RowCount{DataSize: raw.DataSize} for id, count := range raw.EntryCounts { if _, ok := pkIDs[id]; ok { @@ -179,7 +180,7 @@ func backup( pkIDs := make(map[uint64]bool) for i := range backupManifest.Descriptors { if t, _, _, _, _ := descpb.GetDescriptors(&backupManifest.Descriptors[i]); t != nil { - pkIDs[roachpb.BulkOpSummaryID(uint64(t.ID), uint64(t.PrimaryIndex.ID))] = true + pkIDs[kvpb.BulkOpSummaryID(uint64(t.ID), uint64(t.PrimaryIndex.ID))] = true } } @@ -206,7 +207,7 @@ func backup( urisByLocalityKV, encryption, &kmsEnv, - roachpb.MVCCFilter(backupManifest.MVCCFilter), + kvpb.MVCCFilter(backupManifest.MVCCFilter), backupManifest.StartTime, backupManifest.EndTime, ) diff --git a/pkg/ccl/backupccl/backup_processor.go b/pkg/ccl/backupccl/backup_processor.go index bfceb7f37479..abaf65c9dcd5 100644 --- a/pkg/ccl/backupccl/backup_processor.go +++ b/pkg/ccl/backupccl/backup_processor.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb" "github.com/cockroachdb/cockroach/pkg/cloud" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" @@ -364,8 +365,8 @@ func runBackupProcessor( splitMidKey = true } - req := &roachpb.ExportRequest{ - RequestHeader: roachpb.RequestHeaderFromSpan(span.span), + req := &kvpb.ExportRequest{ + RequestHeader: kvpb.RequestHeaderFromSpan(span.span), ResumeKeyTS: span.firstKeyTS, StartTime: span.start, MVCCFilter: spec.MVCCFilter, @@ -394,7 +395,7 @@ func runBackupProcessor( priority = timeutil.Since(readTime) > priorityAfter.Get(&clusterSettings.SV) } - header := roachpb.Header{ + header := kvpb.Header{ // We set the DistSender response target bytes field to a sentinel // value. The sentinel value of 1 forces the ExportRequest to paginate // after creating a single SST. @@ -416,19 +417,19 @@ func runBackupProcessor( header.WaitPolicy = lock.WaitPolicy_Error } - admissionHeader := roachpb.AdmissionHeader{ + admissionHeader := kvpb.AdmissionHeader{ // Export requests are currently assigned BulkNormalPri. // // TODO(dt): Consider linking this to/from the UserPriority field. Priority: int32(admissionpb.BulkNormalPri), CreateTime: timeutil.Now().UnixNano(), - Source: roachpb.AdmissionHeader_FROM_SQL, + Source: kvpb.AdmissionHeader_FROM_SQL, NoMemoryReservedAtSource: true, } log.VEventf(ctx, 1, "sending ExportRequest for span %s (attempt %d, priority %s)", span.span, span.attempts+1, header.UserPriority.String()) - var rawResp roachpb.Response - var pErr *roachpb.Error + var rawResp kvpb.Response + var pErr *kvpb.Error requestSentAt := timeutil.Now() exportRequestErr := contextutil.RunWithTimeout(ctx, fmt.Sprintf("ExportRequest for span %s", span.span), @@ -441,7 +442,7 @@ func runBackupProcessor( return nil }) if exportRequestErr != nil { - if intentErr, ok := pErr.GetDetail().(*roachpb.WriteIntentError); ok { + if intentErr, ok := pErr.GetDetail().(*kvpb.WriteIntentError); ok { span.lastTried = timeutil.Now() span.attempts++ todo <- span @@ -458,7 +459,7 @@ func runBackupProcessor( } // BatchTimestampBeforeGCError is returned if the ExportRequest // attempts to read below the range's GC threshold. - if batchTimestampBeforeGCError, ok := pErr.GetDetail().(*roachpb.BatchTimestampBeforeGCError); ok { + if batchTimestampBeforeGCError, ok := pErr.GetDetail().(*kvpb.BatchTimestampBeforeGCError); ok { // If the range we are exporting is marked to be excluded from // backup, it is safe to ignore the error. It is likely that the // table has been configured with a low GC TTL, and so the data @@ -471,7 +472,7 @@ func runBackupProcessor( return errors.Wrapf(exportRequestErr, "exporting %s", span.span) } - resp := rawResp.(*roachpb.ExportResponse) + resp := rawResp.(*kvpb.ExportResponse) // If the reply has a resume span, we process it immediately. var resumeSpan spanAndTime @@ -558,7 +559,7 @@ func runBackupProcessor( // recordExportStats emits a StructuredEvent containing the stats about the // evaluated ExportRequest. -func recordExportStats(sp *tracing.Span, resp *roachpb.ExportResponse, requestSentAt time.Time) { +func recordExportStats(sp *tracing.Span, resp *kvpb.ExportResponse, requestSentAt time.Time) { if sp == nil { return } diff --git a/pkg/ccl/backupccl/backup_processor_planning.go b/pkg/ccl/backupccl/backup_processor_planning.go index dd85d17908be..5bc54356a569 100644 --- a/pkg/ccl/backupccl/backup_processor_planning.go +++ b/pkg/ccl/backupccl/backup_processor_planning.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cloud" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -41,7 +42,7 @@ func distBackupPlanSpecs( urisByLocalityKV map[string]string, encryption *jobspb.BackupEncryptionOptions, kmsEnv cloud.KMSEnv, - mvccFilter roachpb.MVCCFilter, + mvccFilter kvpb.MVCCFilter, startTime, endTime hlc.Timestamp, ) (map[base.SQLInstanceID]*execinfrapb.BackupDataSpec, error) { var span *tracing.Span @@ -85,9 +86,9 @@ func distBackupPlanSpecs( } // Wrap the relevant BackupEncryptionOptions to be used by the Backup // processor and KV ExportRequest. - var fileEncryption *roachpb.FileEncryptionOptions + var fileEncryption *kvpb.FileEncryptionOptions if encryption != nil { - fileEncryption = &roachpb.FileEncryptionOptions{Key: encryption.Key} + fileEncryption = &kvpb.FileEncryptionOptions{Key: encryption.Key} } // First construct spans based on span partitions. Then add on diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index 0a88451accba..7d61e0476a44 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -61,6 +61,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptutil" @@ -1344,7 +1345,7 @@ func checkInProgressBackupRestore( knobs := base.TestingKnobs{ DistSQL: &execinfra.TestingKnobs{ BackupRestoreTestingKnobs: &sql.BackupRestoreTestingKnobs{ - RunAfterExportingSpanEntry: func(_ context.Context, res *roachpb.ExportResponse) { + RunAfterExportingSpanEntry: func(_ context.Context, res *kvpb.ExportResponse) { <-allowResponse // If ResumeSpan is set to nil, it means that we have completed // exporting a span and the job will update its fraction progressed. @@ -3887,9 +3888,9 @@ func TestRestoreAsOfSystemTimeGCBounds(t *testing.T) { const dir = "nodelocal://0/" preGC := eval.TimestampToDecimalDatum(tc.Server(0).Clock().Now()).String() - gcr := roachpb.GCRequest{ + gcr := kvpb.GCRequest{ // Bogus span to make it a valid request. - RequestHeader: roachpb.RequestHeader{ + RequestHeader: kvpb.RequestHeader{ Key: keys.SystemSQLCodec.TablePrefix(bootstrap.TestingUserDescID(0)), EndKey: keys.MaxKey, }, @@ -6249,7 +6250,7 @@ func TestRestoreErrorPropagates(t *testing.T) { jobsTableKey := keys.SystemSQLCodec.TablePrefix(uint32(systemschema.JobsTable.GetID())) var shouldFail, failures int64 params.ServerArgs.Knobs.Store = &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(ctx context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + TestingRequestFilter: func(ctx context.Context, ba *kvpb.BatchRequest) *kvpb.Error { // Intercept Put and ConditionalPut requests to the jobs table // and, if shouldFail is positive, increment failures and return an // injected error. @@ -6259,10 +6260,10 @@ func TestRestoreErrorPropagates(t *testing.T) { for _, ru := range ba.Requests { r := ru.GetInner() switch r.(type) { - case *roachpb.ConditionalPutRequest, *roachpb.PutRequest: + case *kvpb.ConditionalPutRequest, *kvpb.PutRequest: key := r.Header().Key if bytes.HasPrefix(key, jobsTableKey) && atomic.LoadInt64(&shouldFail) > 0 { - return roachpb.NewError(errors.Errorf("boom %d", atomic.AddInt64(&failures, 1))) + return kvpb.NewError(errors.Errorf("boom %d", atomic.AddInt64(&failures, 1))) } } } @@ -6370,16 +6371,16 @@ func TestPaginatedBackupTenant(t *testing.T) { // Check if export request is from a lease for a descriptor to avoid picking // up on wrong export requests - isLeasingExportRequest := func(r *roachpb.ExportRequest) bool { + isLeasingExportRequest := func(r *kvpb.ExportRequest) bool { _, tenantID, _ := keys.DecodeTenantPrefix(r.Key) codec := keys.MakeSQLCodec(tenantID) return bytes.HasPrefix(r.Key, codec.DescMetadataPrefix()) && r.EndKey.Equal(r.Key.PrefixEnd()) } params.ServerArgs.Knobs.Store = &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(ctx context.Context, request *roachpb.BatchRequest) *roachpb.Error { + TestingRequestFilter: func(ctx context.Context, request *kvpb.BatchRequest) *kvpb.Error { for _, ru := range request.Requests { - if exportRequest, ok := ru.GetInner().(*roachpb.ExportRequest); ok && + if exportRequest, ok := ru.GetInner().(*kvpb.ExportRequest); ok && !isLeasingExportRequest(exportRequest) { mu.Lock() @@ -6395,11 +6396,11 @@ func TestPaginatedBackupTenant(t *testing.T) { } return nil }, - TestingResponseFilter: func(ctx context.Context, ba *roachpb.BatchRequest, br *roachpb.BatchResponse) *roachpb.Error { + TestingResponseFilter: func(ctx context.Context, ba *kvpb.BatchRequest, br *kvpb.BatchResponse) *kvpb.Error { for i, ru := range br.Responses { - if exportRequest, ok := ba.Requests[i].GetInner().(*roachpb.ExportRequest); ok && + if exportRequest, ok := ba.Requests[i].GetInner().(*kvpb.ExportRequest); ok && !isLeasingExportRequest(exportRequest) { - exportResponse := ru.GetInner().(*roachpb.ExportResponse) + exportResponse := ru.GetInner().(*kvpb.ExportResponse) // Every ExportResponse should have a single SST when running backup // within a tenant. require.Equal(t, 1, len(exportResponse.Files)) @@ -7325,10 +7326,10 @@ func TestClientDisconnect(t *testing.T) { blockBackupOrRestore(ctx) }}}, Store: &kvserver.StoreTestingKnobs{ - TestingResponseFilter: func(ctx context.Context, ba *roachpb.BatchRequest, br *roachpb.BatchResponse) *roachpb.Error { + TestingResponseFilter: func(ctx context.Context, ba *kvpb.BatchRequest, br *kvpb.BatchResponse) *kvpb.Error { for _, ru := range br.Responses { switch ru.GetInner().(type) { - case *roachpb.ExportResponse: + case *kvpb.ExportResponse: blockBackupOrRestore(ctx) } } @@ -8834,8 +8835,8 @@ DROP INDEX idx_3; // descriptor. var b kv.Batch descriptorTableSpan := makeTableSpan(codec, keys.DescriptorTableID) - b.AddRawRequest(&roachpb.RevertRangeRequest{ - RequestHeader: roachpb.RequestHeader{ + b.AddRawRequest(&kvpb.RevertRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: descriptorTableSpan.Key, EndKey: descriptorTableSpan.EndKey, }, diff --git a/pkg/ccl/backupccl/backupinfo/BUILD.bazel b/pkg/ccl/backupccl/backupinfo/BUILD.bazel index b32e46b4c4af..140f7b8b7b26 100644 --- a/pkg/ccl/backupccl/backupinfo/BUILD.bazel +++ b/pkg/ccl/backupccl/backupinfo/BUILD.bazel @@ -20,6 +20,7 @@ go_library( "//pkg/cloud/cloudpb", "//pkg/jobs/jobspb", "//pkg/keys", + "//pkg/kv/kvpb", "//pkg/multitenant/mtinfopb", "//pkg/roachpb", "//pkg/security/username", diff --git a/pkg/ccl/backupccl/backupinfo/backup_metadata.go b/pkg/ccl/backupccl/backupinfo/backup_metadata.go index 5bc6f2588d4e..2188284c3df3 100644 --- a/pkg/ccl/backupccl/backupinfo/backup_metadata.go +++ b/pkg/ccl/backupccl/backupinfo/backup_metadata.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cloud" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -751,13 +752,13 @@ func debugDumpFileSST( kmsEnv cloud.KMSEnv, out func(rawKey, readableKey string, value json.JSON) error, ) error { - var encOpts *roachpb.FileEncryptionOptions + var encOpts *kvpb.FileEncryptionOptions if enc != nil { key, err := backupencryption.GetEncryptionKey(ctx, enc, kmsEnv) if err != nil { return err } - encOpts = &roachpb.FileEncryptionOptions{Key: key} + encOpts = &kvpb.FileEncryptionOptions{Key: key} } iter, err := storageccl.ExternalSSTReader(ctx, []storageccl.StoreFile{{Store: store, FilePath: fileInfoPath}}, encOpts, iterOpts) if err != nil { @@ -802,13 +803,13 @@ func DebugDumpMetadataSST( kmsEnv cloud.KMSEnv, out func(rawKey, readableKey string, value json.JSON) error, ) error { - var encOpts *roachpb.FileEncryptionOptions + var encOpts *kvpb.FileEncryptionOptions if enc != nil { key, err := backupencryption.GetEncryptionKey(ctx, enc, kmsEnv) if err != nil { return err } - encOpts = &roachpb.FileEncryptionOptions{Key: key} + encOpts = &kvpb.FileEncryptionOptions{Key: key} } iter, err := storageccl.ExternalSSTReader(ctx, []storageccl.StoreFile{{Store: store, FilePath: path}}, encOpts, iterOpts) @@ -964,13 +965,13 @@ func NewBackupMetadata( encryption *jobspb.BackupEncryptionOptions, kmsEnv cloud.KMSEnv, ) (*BackupMetadata, error) { - var encOpts *roachpb.FileEncryptionOptions + var encOpts *kvpb.FileEncryptionOptions if encryption != nil { key, err := backupencryption.GetEncryptionKey(ctx, encryption, kmsEnv) if err != nil { return nil, err } - encOpts = &roachpb.FileEncryptionOptions{Key: key} + encOpts = &kvpb.FileEncryptionOptions{Key: key} } iter, err := storageccl.ExternalSSTReader(ctx, []storageccl.StoreFile{{Store: exportStore, FilePath: sstFileName}}, encOpts, iterOpts) @@ -1092,13 +1093,13 @@ func (b *BackupMetadata) NewFileIter( defer fileInfoIter.close() var storeFiles []storageccl.StoreFile - var encOpts *roachpb.FileEncryptionOptions + var encOpts *kvpb.FileEncryptionOptions if b.enc != nil { key, err := backupencryption.GetEncryptionKey(ctx, b.enc, b.kmsEnv) if err != nil { return nil, err } - encOpts = &roachpb.FileEncryptionOptions{Key: key} + encOpts = &kvpb.FileEncryptionOptions{Key: key} } result := resultWrapper{} @@ -1120,13 +1121,13 @@ func (b *BackupMetadata) NewFileIter( // NewFileSSTIter creates a new FileIterator to iterate over the storeFile. // It is the caller's responsibility to Close() the returned iterator. func NewFileSSTIter( - ctx context.Context, storeFile storageccl.StoreFile, encOpts *roachpb.FileEncryptionOptions, + ctx context.Context, storeFile storageccl.StoreFile, encOpts *kvpb.FileEncryptionOptions, ) (*FileIterator, error) { return newFileSSTIter(ctx, []storageccl.StoreFile{storeFile}, encOpts) } func newFileSSTIter( - ctx context.Context, storeFiles []storageccl.StoreFile, encOpts *roachpb.FileEncryptionOptions, + ctx context.Context, storeFiles []storageccl.StoreFile, encOpts *kvpb.FileEncryptionOptions, ) (*FileIterator, error) { iter, err := storageccl.ExternalSSTReader(ctx, storeFiles, encOpts, iterOpts) if err != nil { @@ -1496,13 +1497,13 @@ func makeBytesIter( useMVCCNext bool, kmsEnv cloud.KMSEnv, ) bytesIter { - var encOpts *roachpb.FileEncryptionOptions + var encOpts *kvpb.FileEncryptionOptions if enc != nil { key, err := backupencryption.GetEncryptionKey(ctx, enc, kmsEnv) if err != nil { return bytesIter{iterError: err} } - encOpts = &roachpb.FileEncryptionOptions{Key: key} + encOpts = &kvpb.FileEncryptionOptions{Key: key} } iter, err := storageccl.ExternalSSTReader(ctx, []storageccl.StoreFile{{Store: store, diff --git a/pkg/ccl/backupccl/backupinfo/manifest_handling.go b/pkg/ccl/backupccl/backupinfo/manifest_handling.go index f5f4ddaf3625..0069239e6989 100644 --- a/pkg/ccl/backupccl/backupinfo/manifest_handling.go +++ b/pkg/ccl/backupccl/backupinfo/manifest_handling.go @@ -28,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cloud/cloudpb" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/settings" @@ -1590,13 +1591,13 @@ func (f *IterFactory) NewFileIter( Store: f.store, FilePath: f.fileSSTPath, } - var encOpts *roachpb.FileEncryptionOptions + var encOpts *kvpb.FileEncryptionOptions if f.encryption != nil { key, err := backupencryption.GetEncryptionKey(ctx, f.encryption, f.kmsEnv) if err != nil { return nil, err } - encOpts = &roachpb.FileEncryptionOptions{Key: key} + encOpts = &kvpb.FileEncryptionOptions{Key: key} } return NewFileSSTIter(ctx, storeFile, encOpts) } diff --git a/pkg/ccl/backupccl/datadriven_test.go b/pkg/ccl/backupccl/datadriven_test.go index 0fc60ba41204..9e8b466bdf70 100644 --- a/pkg/ccl/backupccl/datadriven_test.go +++ b/pkg/ccl/backupccl/datadriven_test.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -895,9 +896,9 @@ func handleKVRequest( target).Scan(&tableID) require.NoError(t, err) bankSpan := makeTableSpan(keys.SystemSQLCodec, tableID) - dr := roachpb.DeleteRangeRequest{ + dr := kvpb.DeleteRangeRequest{ // Bogus span to make it a valid request. - RequestHeader: roachpb.RequestHeader{ + RequestHeader: kvpb.RequestHeader{ Key: bankSpan.Key, EndKey: bankSpan.EndKey, }, diff --git a/pkg/ccl/backupccl/file_sst_sink.go b/pkg/ccl/backupccl/file_sst_sink.go index 314b8a1a3fb9..67f98c130a48 100644 --- a/pkg/ccl/backupccl/file_sst_sink.go +++ b/pkg/ccl/backupccl/file_sst_sink.go @@ -18,7 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/storageccl" "github.com/cockroachdb/cockroach/pkg/cloud" "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" @@ -32,7 +32,7 @@ import ( type sstSinkConf struct { progCh chan execinfrapb.RemoteProducerMetadata_BulkProcessorProgress - enc *roachpb.FileEncryptionOptions + enc *kvpb.FileEncryptionOptions id base.SQLInstanceID settings *settings.Values } diff --git a/pkg/ccl/backupccl/restore_data_processor.go b/pkg/ccl/backupccl/restore_data_processor.go index 9cb148ad4dd1..ec93faae17b9 100644 --- a/pkg/ccl/backupccl/restore_data_processor.go +++ b/pkg/ccl/backupccl/restore_data_processor.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cloud" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/bulk" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql" @@ -391,10 +392,10 @@ func (rd *restoreDataProcessor) runRestoreWorkers(ctx context.Context, ssts chan func (rd *restoreDataProcessor) processRestoreSpanEntry( ctx context.Context, kr *KeyRewriter, sst mergedSST, -) (roachpb.BulkOpSummary, error) { +) (kvpb.BulkOpSummary, error) { db := rd.flowCtx.Cfg.DB evalCtx := rd.EvalCtx - var summary roachpb.BulkOpSummary + var summary kvpb.BulkOpSummary entry := sst.entry iter := sst.iter @@ -530,7 +531,7 @@ func (rd *restoreDataProcessor) processRestoreSpanEntry( } func makeProgressUpdate( - summary roachpb.BulkOpSummary, entry execinfrapb.RestoreSpanEntry, pkIDs map[uint64]bool, + summary kvpb.BulkOpSummary, entry execinfrapb.RestoreSpanEntry, pkIDs map[uint64]bool, ) (progDetails backuppb.RestoreProgress) { progDetails.Summary = countRows(summary, pkIDs) progDetails.ProgressIdx = entry.ProgressIdx @@ -594,7 +595,7 @@ type SSTBatcherExecutor interface { Reset(ctx context.Context) error Flush(ctx context.Context) error Close(ctx context.Context) - GetSummary() roachpb.BulkOpSummary + GetSummary() kvpb.BulkOpSummary } type sstBatcherNoop struct { @@ -624,7 +625,7 @@ func (b *sstBatcherNoop) Close(ctx context.Context) { } // GetSummary returns this batcher's total added rows/bytes/etc. -func (b *sstBatcherNoop) GetSummary() roachpb.BulkOpSummary { +func (b *sstBatcherNoop) GetSummary() kvpb.BulkOpSummary { return b.totalRows.BulkOpSummary } diff --git a/pkg/ccl/backupccl/restore_data_processor_test.go b/pkg/ccl/backupccl/restore_data_processor_test.go index 75f27d387251..1083092b23fe 100644 --- a/pkg/ccl/backupccl/restore_data_processor_test.go +++ b/pkg/ccl/backupccl/restore_data_processor_test.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/bulk" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -217,9 +218,9 @@ func runTestIngest(t *testing.T, init func(*cluster.Settings)) { remainingAmbiguousSubReqs := int64(initialAmbiguousSubReqs) knobs := base.TestingKnobs{Store: &kvserver.StoreTestingKnobs{ EvalKnobs: kvserverbase.BatchEvalTestingKnobs{ - TestingEvalFilter: func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { + TestingEvalFilter: func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { switch filterArgs.Req.(type) { - case *roachpb.AddSSTableRequest: + case *kvpb.AddSSTableRequest: // No-op. default: return nil @@ -228,7 +229,7 @@ func runTestIngest(t *testing.T, init func(*cluster.Settings)) { if r < 0 { return nil } - return roachpb.NewError(roachpb.NewAmbiguousResultErrorf("%d", r)) + return kvpb.NewError(kvpb.NewAmbiguousResultErrorf("%d", r)) }, }, }} diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index dda387ff725c..5499d9353340 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobsprotectedts" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" @@ -186,7 +187,7 @@ func restoreWithRetry( break } - if errors.HasType(err, &roachpb.InsufficientSpaceError{}) { + if errors.HasType(err, &kvpb.InsufficientSpaceError{}) { return roachpb.RowCount{}, jobs.MarkPauseRequestError(errors.UnwrapAll(err)) } @@ -1364,7 +1365,7 @@ func createImportingDescriptors( pkIDs := make(map[uint64]bool) for _, tbl := range tables { - pkIDs[roachpb.BulkOpSummaryID(uint64(tbl.GetID()), uint64(tbl.GetPrimaryIndexID()))] = true + pkIDs[kvpb.BulkOpSummaryID(uint64(tbl.GetID()), uint64(tbl.GetPrimaryIndexID()))] = true } dataToPreRestore = &restorationDataBase{ diff --git a/pkg/ccl/backupccl/restore_processor_planning.go b/pkg/ccl/backupccl/restore_processor_planning.go index 20c44aae4b2d..20da5a60f4ea 100644 --- a/pkg/ccl/backupccl/restore_processor_planning.go +++ b/pkg/ccl/backupccl/restore_processor_planning.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cloud" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql" @@ -101,9 +102,9 @@ func distRestore( } // Wrap the relevant BackupEncryptionOptions to be used by the Restore // processor. - var fileEncryption *roachpb.FileEncryptionOptions + var fileEncryption *kvpb.FileEncryptionOptions if encryption != nil { - fileEncryption = &roachpb.FileEncryptionOptions{Key: encryption.Key} + fileEncryption = &kvpb.FileEncryptionOptions{Key: encryption.Key} } makePlan := func(ctx context.Context, dsp *sql.DistSQLPlanner) (*sql.PhysicalPlan, *sql.PlanningCtx, error) { diff --git a/pkg/ccl/backupccl/split_and_scatter_processor.go b/pkg/ccl/backupccl/split_and_scatter_processor.go index eaef2eb4ff86..7821479597c4 100644 --- a/pkg/ccl/backupccl/split_and_scatter_processor.go +++ b/pkg/ccl/backupccl/split_and_scatter_processor.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catenumpb" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" @@ -125,8 +126,8 @@ func (s dbSplitAndScatterer) scatter( } log.VEventf(ctx, 1, "scattering new key %+v", newScatterKey) - req := &roachpb.AdminScatterRequest{ - RequestHeader: roachpb.RequestHeaderFromSpan(roachpb.Span{ + req := &kvpb.AdminScatterRequest{ + RequestHeader: kvpb.RequestHeaderFromSpan(roachpb.Span{ Key: newScatterKey, EndKey: newScatterKey.Next(), }), @@ -156,12 +157,12 @@ func (s dbSplitAndScatterer) scatter( return 0, nil } - return s.findDestination(res.(*roachpb.AdminScatterResponse)), nil + return s.findDestination(res.(*kvpb.AdminScatterResponse)), nil } // findDestination returns the node ID of the node of the destination of the // AdminScatter request. If the destination cannot be found, 0 is returned. -func (s dbSplitAndScatterer) findDestination(res *roachpb.AdminScatterResponse) roachpb.NodeID { +func (s dbSplitAndScatterer) findDestination(res *kvpb.AdminScatterResponse) roachpb.NodeID { if len(res.RangeInfos) > 0 { // If the lease is not populated, we return the 0 value anyway. We receive 1 // RangeInfo per range that was scattered. Since we send a scatter request diff --git a/pkg/ccl/changefeedccl/BUILD.bazel b/pkg/ccl/changefeedccl/BUILD.bazel index ae02db722c3f..3fe679e336b3 100644 --- a/pkg/ccl/changefeedccl/BUILD.bazel +++ b/pkg/ccl/changefeedccl/BUILD.bazel @@ -220,6 +220,7 @@ go_test( "//pkg/keys", "//pkg/kv", "//pkg/kv/kvclient/kvcoord", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/kv/kvserver/kvserverbase", "//pkg/kv/kvserver/protectedts", diff --git a/pkg/ccl/changefeedccl/cdceval/BUILD.bazel b/pkg/ccl/changefeedccl/cdceval/BUILD.bazel index a518b6a38528..4b0a5288b601 100644 --- a/pkg/ccl/changefeedccl/cdceval/BUILD.bazel +++ b/pkg/ccl/changefeedccl/cdceval/BUILD.bazel @@ -70,6 +70,7 @@ go_test( "//pkg/ccl/changefeedccl/changefeedbase", "//pkg/jobs/jobspb", "//pkg/keys", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/security/securityassets", "//pkg/security/securitytest", diff --git a/pkg/ccl/changefeedccl/cdceval/expr_eval_test.go b/pkg/ccl/changefeedccl/cdceval/expr_eval_test.go index 3d4d23d72597..ca2cbf90935b 100644 --- a/pkg/ccl/changefeedccl/cdceval/expr_eval_test.go +++ b/pkg/ccl/changefeedccl/cdceval/expr_eval_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdctest" "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql" @@ -682,7 +683,7 @@ func TestUnsupportedCDCFunctions(t *testing.T) { } func decodeRowErr( - decoder cdcevent.Decoder, v *roachpb.RangeFeedValue, rt cdcevent.RowType, + decoder cdcevent.Decoder, v *kvpb.RangeFeedValue, rt cdcevent.RowType, ) (cdcevent.Row, error) { keyVal := roachpb.KeyValue{Key: v.Key} if rt == cdcevent.PrevRow { @@ -695,7 +696,7 @@ func decodeRowErr( } func decodeRow( - t *testing.T, decoder cdcevent.Decoder, v *roachpb.RangeFeedValue, rt cdcevent.RowType, + t *testing.T, decoder cdcevent.Decoder, v *kvpb.RangeFeedValue, rt cdcevent.RowType, ) cdcevent.Row { r, err := decodeRowErr(decoder, v, rt) require.NoError(t, err) @@ -738,8 +739,8 @@ func randEncDatumPrimaryFamily( // readSortedRangeFeedValues reads n values, and sorts them based on key order. func readSortedRangeFeedValues( - t *testing.T, n int, row func(t *testing.T) *roachpb.RangeFeedValue, -) (res []roachpb.RangeFeedValue) { + t *testing.T, n int, row func(t *testing.T) *kvpb.RangeFeedValue, +) (res []kvpb.RangeFeedValue) { t.Helper() for i := 0; i < n; i++ { v := row(t) diff --git a/pkg/ccl/changefeedccl/cdctest/BUILD.bazel b/pkg/ccl/changefeedccl/cdctest/BUILD.bazel index e93e5f525ffb..5aa89288071d 100644 --- a/pkg/ccl/changefeedccl/cdctest/BUILD.bazel +++ b/pkg/ccl/changefeedccl/cdctest/BUILD.bazel @@ -20,6 +20,7 @@ go_library( "//pkg/jobs/jobspb", "//pkg/keys", "//pkg/kv/kvclient/rangefeed", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/sql", "//pkg/sql/catalog", diff --git a/pkg/ccl/changefeedccl/cdctest/row.go b/pkg/ccl/changefeedccl/cdctest/row.go index 7598ed5136f5..583ee37aa193 100644 --- a/pkg/ccl/changefeedccl/cdctest/row.go +++ b/pkg/ccl/changefeedccl/cdctest/row.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog" @@ -27,22 +28,22 @@ import ( ) // MakeRangeFeedValueReader starts rangefeed on the specified table and returns a function -// that returns the next *roachpb.RangeFeedValue from the table. +// that returns the next *kvpb.RangeFeedValue from the table. // This funciton is intended to be used in tests that wish to read low level roachpb.KeyValue(s). // Instead of trying to generate KVs ourselves (subject to encoding restrictions, etc), it is // simpler to just "INSERT ..." into the table, and then use this function to read next value. func MakeRangeFeedValueReader( t *testing.T, execCfgI interface{}, desc catalog.TableDescriptor, -) (func(t *testing.T) *roachpb.RangeFeedValue, func()) { +) (func(t *testing.T) *kvpb.RangeFeedValue, func()) { t.Helper() execCfg := execCfgI.(sql.ExecutorConfig) - rows := make(chan *roachpb.RangeFeedValue) + rows := make(chan *kvpb.RangeFeedValue) ctx, cleanup := context.WithCancel(context.Background()) _, err := execCfg.RangeFeedFactory.RangeFeed(ctx, "feed-"+desc.GetName(), []roachpb.Span{desc.PrimaryIndexSpan(keys.SystemSQLCodec)}, execCfg.Clock.Now(), - func(ctx context.Context, value *roachpb.RangeFeedValue) { + func(ctx context.Context, value *kvpb.RangeFeedValue) { select { case <-ctx.Done(): case rows <- value: @@ -59,7 +60,7 @@ func MakeRangeFeedValueReader( // Helper to read next rangefeed value. dups := make(map[string]struct{}) - return func(t *testing.T) *roachpb.RangeFeedValue { + return func(t *testing.T) *kvpb.RangeFeedValue { t.Helper() for { select { diff --git a/pkg/ccl/changefeedccl/changefeed_test.go b/pkg/ccl/changefeedccl/changefeed_test.go index 5ca4e75848f0..25afbffbde2b 100644 --- a/pkg/ccl/changefeedccl/changefeed_test.go +++ b/pkg/ccl/changefeedccl/changefeed_test.go @@ -45,6 +45,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" @@ -1714,7 +1715,7 @@ func TestChangefeedLaggingSpanCheckpointing(t *testing.T) { // Rangefeed will skip some of the checkpoints to simulate lagging spans. var laggingSpans roachpb.SpanGroup numLagging := 0 - knobs.FeedKnobs.ShouldSkipCheckpoint = func(checkpoint *roachpb.RangeFeedCheckpoint) bool { + knobs.FeedKnobs.ShouldSkipCheckpoint = func(checkpoint *kvpb.RangeFeedCheckpoint) bool { // Skip spans that were skipped before; otherwise skip some spans. seenBefore := laggingSpans.Encloses(checkpoint.Span) if seenBefore || (numLagging < 5 && rnd.Int()%3 == 0) { @@ -2289,23 +2290,23 @@ func fetchDescVersionModificationTime( t testing.TB, s TestServerWithSystem, tableName string, version int, ) hlc.Timestamp { tblKey := s.Codec.TablePrefix(keys.DescriptorTableID) - header := roachpb.RequestHeader{ + header := kvpb.RequestHeader{ Key: tblKey, EndKey: tblKey.PrefixEnd(), } dropColTblID := sqlutils.QueryTableID(t, s.DB, `d`, "public", tableName) - req := &roachpb.ExportRequest{ + req := &kvpb.ExportRequest{ RequestHeader: header, - MVCCFilter: roachpb.MVCCFilter_All, + MVCCFilter: kvpb.MVCCFilter_All, StartTime: hlc.Timestamp{}, } - hh := roachpb.Header{Timestamp: hlc.NewClockForTesting(nil).Now()} + hh := kvpb.Header{Timestamp: hlc.NewClockForTesting(nil).Now()} res, pErr := kv.SendWrappedWith(context.Background(), s.SystemServer.DB().NonTransactionalSender(), hh, req) if pErr != nil { t.Fatal(pErr.GoError()) } - for _, file := range res.(*roachpb.ExportResponse).Files { + for _, file := range res.(*kvpb.ExportResponse).Files { it, err := storage.NewMemSSTIterator(file.SST, false /* verify */, storage.IterOptions{ KeyTypes: storage.IterKeyTypePointsAndRanges, LowerBound: keys.MinKey, @@ -5186,12 +5187,12 @@ func TestChangefeedProtectedTimestamps(t *testing.T) { } } requestFilter = kvserverbase.ReplicaRequestFilter(func( - ctx context.Context, ba *roachpb.BatchRequest, - ) *roachpb.Error { + ctx context.Context, ba *kvpb.BatchRequest, + ) *kvpb.Error { if ba.Txn == nil || ba.Txn.Name != "changefeed backfill" { return nil } - scanReq, ok := ba.GetArg(roachpb.Scan) + scanReq, ok := ba.GetArg(kvpb.Scan) if !ok { return nil } diff --git a/pkg/ccl/changefeedccl/changefeedbase/BUILD.bazel b/pkg/ccl/changefeedccl/changefeedbase/BUILD.bazel index e55f7cb0ae78..4bfbe3d8344c 100644 --- a/pkg/ccl/changefeedccl/changefeedbase/BUILD.bazel +++ b/pkg/ccl/changefeedccl/changefeedbase/BUILD.bazel @@ -15,7 +15,7 @@ go_library( deps = [ "//pkg/jobs", "//pkg/jobs/jobspb", - "//pkg/roachpb", + "//pkg/kv/kvpb", "//pkg/settings", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/lease", diff --git a/pkg/ccl/changefeedccl/changefeedbase/errors.go b/pkg/ccl/changefeedccl/changefeedbase/errors.go index 3aaeea2493f0..53ad8edbc5a2 100644 --- a/pkg/ccl/changefeedccl/changefeedbase/errors.go +++ b/pkg/ccl/changefeedccl/changefeedbase/errors.go @@ -12,7 +12,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/jobs" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" "github.com/cockroachdb/errors" ) @@ -125,7 +125,7 @@ func AsTerminalError(ctx context.Context, lm *lease.Manager, cause error) (termE } // GC TTL errors are always fatal. - if errors.HasType(cause, (*roachpb.BatchTimestampBeforeGCError)(nil)) { + if errors.HasType(cause, (*kvpb.BatchTimestampBeforeGCError)(nil)) { return WithTerminalError(cause) } diff --git a/pkg/ccl/changefeedccl/event_processing_test.go b/pkg/ccl/changefeedccl/event_processing_test.go index 63b66f3bcee7..8d03e3ca0e51 100644 --- a/pkg/ccl/changefeedccl/event_processing_test.go +++ b/pkg/ccl/changefeedccl/event_processing_test.go @@ -16,7 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvevent" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc/keyside" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/encoding" @@ -235,8 +235,8 @@ func makeKVEventKeyOnly(rng *rand.Rand, upper int) kvevent.Event { panic(err) } - return kvevent.MakeKVEvent(&roachpb.RangeFeedEvent{ - Val: &roachpb.RangeFeedValue{ + return kvevent.MakeKVEvent(&kvpb.RangeFeedEvent{ + Val: &kvpb.RangeFeedValue{ Key: key, }, }) diff --git a/pkg/ccl/changefeedccl/kvevent/BUILD.bazel b/pkg/ccl/changefeedccl/kvevent/BUILD.bazel index a0710fb93de8..b6110086c3f0 100644 --- a/pkg/ccl/changefeedccl/kvevent/BUILD.bazel +++ b/pkg/ccl/changefeedccl/kvevent/BUILD.bazel @@ -18,6 +18,7 @@ go_library( "//pkg/ccl/changefeedccl/cdcutils", "//pkg/ccl/changefeedccl/changefeedbase", "//pkg/jobs/jobspb", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/settings", "//pkg/util/hlc", @@ -45,6 +46,7 @@ go_test( deps = [ "//pkg/jobs/jobspb", "//pkg/keys", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/settings/cluster", "//pkg/sql/randgen", diff --git a/pkg/ccl/changefeedccl/kvevent/bench_test.go b/pkg/ccl/changefeedccl/kvevent/bench_test.go index 3873f536f6d2..d8ac93dce94e 100644 --- a/pkg/ccl/changefeedccl/kvevent/bench_test.go +++ b/pkg/ccl/changefeedccl/kvevent/bench_test.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvevent" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/rowenc/keyside" @@ -112,7 +113,7 @@ func BenchmarkMemBuffer(b *testing.B) { _ = wg.Wait() // Ignore error -- this group returns context cancellation. } -func generateRangeFeedCheckpoint(rng *rand.Rand) *roachpb.RangeFeedEvent { +func generateRangeFeedCheckpoint(rng *rand.Rand) *kvpb.RangeFeedEvent { start := rng.Intn(2 << 20) end := start + rng.Intn(2<<20) startDatum := tree.NewDInt(tree.DInt(start)) @@ -137,8 +138,8 @@ func generateRangeFeedCheckpoint(rng *rand.Rand) *roachpb.RangeFeedEvent { panic(err) } - return &roachpb.RangeFeedEvent{ - Checkpoint: &roachpb.RangeFeedCheckpoint{ + return &kvpb.RangeFeedEvent{ + Checkpoint: &kvpb.RangeFeedCheckpoint{ Span: roachpb.Span{ Key: startKey, EndKey: endKey, diff --git a/pkg/ccl/changefeedccl/kvevent/blocking_buffer_test.go b/pkg/ccl/changefeedccl/kvevent/blocking_buffer_test.go index 79edc9acc5a3..458f55d830c2 100644 --- a/pkg/ccl/changefeedccl/kvevent/blocking_buffer_test.go +++ b/pkg/ccl/changefeedccl/kvevent/blocking_buffer_test.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvevent" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/randgen" @@ -34,7 +35,7 @@ import ( "github.com/stretchr/testify/require" ) -func makeRangeFeedEvent(rnd *rand.Rand, valSize int, prevValSize int) *roachpb.RangeFeedEvent { +func makeRangeFeedEvent(rnd *rand.Rand, valSize int, prevValSize int) *kvpb.RangeFeedEvent { const tableID = 42 key, err := keyside.Encode( @@ -46,8 +47,8 @@ func makeRangeFeedEvent(rnd *rand.Rand, valSize int, prevValSize int) *roachpb.R panic(err) } - e := roachpb.RangeFeedEvent{ - Val: &roachpb.RangeFeedValue{ + e := kvpb.RangeFeedEvent{ + Val: &kvpb.RangeFeedValue{ Key: key, Value: roachpb.Value{ RawBytes: randutil.RandBytes(rnd, valSize), diff --git a/pkg/ccl/changefeedccl/kvevent/event.go b/pkg/ccl/changefeedccl/kvevent/event.go index 5539d1949cd2..99db22e8c855 100644 --- a/pkg/ccl/changefeedccl/kvevent/event.go +++ b/pkg/ccl/changefeedccl/kvevent/event.go @@ -16,6 +16,7 @@ import ( "unsafe" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -101,7 +102,7 @@ const ( // Event represents an event emitted by a kvfeed. It is either a KV or a // resolved timestamp. type Event struct { - ev *roachpb.RangeFeedEvent + ev *kvpb.RangeFeedEvent et Type backfillTimestamp hlc.Timestamp bufferAddTimestamp time.Time @@ -239,7 +240,7 @@ func getTypeForBoundary(bt jobspb.ResolvedSpan_BoundaryType) Type { // MakeResolvedEvent returns resolved event constructed from existing RangeFeedEvent. func MakeResolvedEvent( - ev *roachpb.RangeFeedEvent, boundaryType jobspb.ResolvedSpan_BoundaryType, + ev *kvpb.RangeFeedEvent, boundaryType jobspb.ResolvedSpan_BoundaryType, ) Event { if ev.Checkpoint == nil { panic("expected initialized RangeFeedCheckpoint") @@ -252,8 +253,8 @@ func MakeResolvedEvent( func NewBackfillResolvedEvent( span roachpb.Span, ts hlc.Timestamp, boundaryType jobspb.ResolvedSpan_BoundaryType, ) Event { - rfe := &roachpb.RangeFeedEvent{ - Checkpoint: &roachpb.RangeFeedCheckpoint{ + rfe := &kvpb.RangeFeedEvent{ + Checkpoint: &kvpb.RangeFeedCheckpoint{ Span: span, ResolvedTS: ts, }, @@ -262,7 +263,7 @@ func NewBackfillResolvedEvent( } // MakeKVEvent returns KV event constructed from existing RangeFeedEvent. -func MakeKVEvent(ev *roachpb.RangeFeedEvent) Event { +func MakeKVEvent(ev *kvpb.RangeFeedEvent) Event { if ev.Val == nil { panic("expected initialized RangeFeedValue") } @@ -274,8 +275,8 @@ func MakeKVEvent(ev *roachpb.RangeFeedEvent) Event { func NewBackfillKVEvent( key []byte, ts hlc.Timestamp, val []byte, withDiff bool, backfillTS hlc.Timestamp, ) Event { - rfe := &roachpb.RangeFeedEvent{ - Val: &roachpb.RangeFeedValue{ + rfe := &kvpb.RangeFeedEvent{ + Val: &kvpb.RangeFeedValue{ Key: key, Value: roachpb.Value{ RawBytes: val, diff --git a/pkg/ccl/changefeedccl/kvfeed/BUILD.bazel b/pkg/ccl/changefeedccl/kvfeed/BUILD.bazel index 8b43db40c390..e002c4d14d01 100644 --- a/pkg/ccl/changefeedccl/kvfeed/BUILD.bazel +++ b/pkg/ccl/changefeedccl/kvfeed/BUILD.bazel @@ -20,6 +20,7 @@ go_library( "//pkg/keys", "//pkg/kv", "//pkg/kv/kvclient/kvcoord", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/settings", "//pkg/settings/cluster", @@ -61,6 +62,7 @@ go_test( "//pkg/keys", "//pkg/kv", "//pkg/kv/kvclient/kvcoord", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/security/securityassets", "//pkg/security/securitytest", diff --git a/pkg/ccl/changefeedccl/kvfeed/kv_feed_test.go b/pkg/ccl/changefeedccl/kvfeed/kv_feed_test.go index 3cbc8b49f1a0..1e73b666881c 100644 --- a/pkg/ccl/changefeedccl/kvfeed/kv_feed_test.go +++ b/pkg/ccl/changefeedccl/kvfeed/kv_feed_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/schemafeed/schematestutils" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog" @@ -64,10 +65,10 @@ func TestKVFeed(t *testing.T) { }, } } - kvEvent := func(codec keys.SQLCodec, tableID uint32, k, v string, ts hlc.Timestamp) roachpb.RangeFeedEvent { + kvEvent := func(codec keys.SQLCodec, tableID uint32, k, v string, ts hlc.Timestamp) kvpb.RangeFeedEvent { keyVal := kv(codec, tableID, k, v, ts) - return roachpb.RangeFeedEvent{ - Val: &roachpb.RangeFeedValue{ + return kvpb.RangeFeedEvent{ + Val: &kvpb.RangeFeedValue{ Key: keyVal.Key, Value: keyVal.Value, }, @@ -75,9 +76,9 @@ func TestKVFeed(t *testing.T) { Error: nil, } } - checkpointEvent := func(span roachpb.Span, ts hlc.Timestamp) roachpb.RangeFeedEvent { - return roachpb.RangeFeedEvent{ - Checkpoint: &roachpb.RangeFeedCheckpoint{ + checkpointEvent := func(span roachpb.Span, ts hlc.Timestamp) kvpb.RangeFeedEvent { + return kvpb.RangeFeedEvent{ + Checkpoint: &kvpb.RangeFeedCheckpoint{ Span: span, ResolvedTS: ts, }, @@ -93,7 +94,7 @@ func TestKVFeed(t *testing.T) { endTime hlc.Timestamp spans []roachpb.Span checkpoint []roachpb.Span - events []roachpb.RangeFeedEvent + events []kvpb.RangeFeedEvent descs []catalog.TableDescriptor @@ -190,7 +191,7 @@ func TestKVFeed(t *testing.T) { spans: []roachpb.Span{ tableSpan(codec, 42), }, - events: []roachpb.RangeFeedEvent{ + events: []kvpb.RangeFeedEvent{ kvEvent(codec, 42, "a", "b", ts(3)), }, expScans: []hlc.Timestamp{ @@ -210,7 +211,7 @@ func TestKVFeed(t *testing.T) { checkpoint: []roachpb.Span{ tableSpan(codec, 42), }, - events: []roachpb.RangeFeedEvent{ + events: []kvpb.RangeFeedEvent{ kvEvent(codec, 42, "a", "b", ts(3)), }, expScans: []hlc.Timestamp{}, @@ -228,7 +229,7 @@ func TestKVFeed(t *testing.T) { checkpoint: []roachpb.Span{ makeSpan(codec, 42, "a", "q"), }, - events: []roachpb.RangeFeedEvent{ + events: []kvpb.RangeFeedEvent{ kvEvent(codec, 42, "a", "val", ts(3)), kvEvent(codec, 42, "d", "val", ts(3)), }, @@ -246,7 +247,7 @@ func TestKVFeed(t *testing.T) { spans: []roachpb.Span{ tableSpan(codec, 42), }, - events: []roachpb.RangeFeedEvent{ + events: []kvpb.RangeFeedEvent{ kvEvent(codec, 42, "a", "b", ts(3)), checkpointEvent(tableSpan(codec, 42), ts(4)), kvEvent(codec, 42, "a", "b", ts(5)), @@ -272,7 +273,7 @@ func TestKVFeed(t *testing.T) { spans: []roachpb.Span{ tableSpan(codec, 42), }, - events: []roachpb.RangeFeedEvent{ + events: []kvpb.RangeFeedEvent{ kvEvent(codec, 42, "a", "b", ts(3).Next()), checkpointEvent(tableSpan(codec, 42), ts(4)), kvEvent(codec, 42, "a", "b", ts(5)), @@ -296,7 +297,7 @@ func TestKVFeed(t *testing.T) { spans: []roachpb.Span{ tableSpan(codec, 42), }, - events: []roachpb.RangeFeedEvent{ + events: []kvpb.RangeFeedEvent{ kvEvent(codec, 42, "a", "b", ts(3)), checkpointEvent(tableSpan(codec, 42), ts(4)), kvEvent(codec, 42, "a", "b", ts(5)), @@ -392,7 +393,7 @@ func (r *rawTableFeed) peekOrPop( return events, nil } -type rawEventFeed []roachpb.RangeFeedEvent +type rawEventFeed []kvpb.RangeFeedEvent func (f rawEventFeed) run( ctx context.Context, diff --git a/pkg/ccl/changefeedccl/kvfeed/physical_kv_feed.go b/pkg/ccl/changefeedccl/kvfeed/physical_kv_feed.go index 14a0d7db6476..190a5b7eecff 100644 --- a/pkg/ccl/changefeedccl/kvfeed/physical_kv_feed.go +++ b/pkg/ccl/changefeedccl/kvfeed/physical_kv_feed.go @@ -14,7 +14,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvevent" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/errors" @@ -91,7 +91,7 @@ func (p *rangefeed) addEventsToBuffer(ctx context.Context) error { select { case e := <-p.eventC: switch t := e.GetValue().(type) { - case *roachpb.RangeFeedValue: + case *kvpb.RangeFeedValue: if p.cfg.Knobs.OnRangeFeedValue != nil { if err := p.cfg.Knobs.OnRangeFeedValue(); err != nil { return err @@ -106,7 +106,7 @@ func (p *rangefeed) addEventsToBuffer(ctx context.Context) error { ); err != nil { return err } - case *roachpb.RangeFeedCheckpoint: + case *kvpb.RangeFeedCheckpoint: if p.knobs.ModifyTimestamps != nil { p.knobs.ModifyTimestamps(&t.ResolvedTS) } @@ -124,12 +124,12 @@ func (p *rangefeed) addEventsToBuffer(ctx context.Context) error { ); err != nil { return err } - case *roachpb.RangeFeedSSTable: + case *kvpb.RangeFeedSSTable: // For now, we just error on SST ingestion, since we currently don't // expect SST ingestion into spans with active changefeeds. return errors.Errorf("unexpected SST ingestion: %v", t) - case *roachpb.RangeFeedDeleteRange: + case *kvpb.RangeFeedDeleteRange: // For now, we just ignore on MVCC range tombstones. These are currently // only expected to be used by schema GC and IMPORT INTO, and such spans // should not have active changefeeds across them, at least at the times diff --git a/pkg/ccl/changefeedccl/kvfeed/scanner.go b/pkg/ccl/changefeedccl/kvfeed/scanner.go index 70dd5534ccd6..a4f25be05672 100644 --- a/pkg/ccl/changefeedccl/kvfeed/scanner.go +++ b/pkg/ccl/changefeedccl/kvfeed/scanner.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -188,10 +189,10 @@ func (p *scanRequestScanner) exportSpan( for remaining := &span; remaining != nil; { start := timeutil.Now() b := txn.NewBatch() - r := roachpb.NewScan(remaining.Key, remaining.EndKey, false /* forUpdate */).(*roachpb.ScanRequest) - r.ScanFormat = roachpb.BATCH_RESPONSE + r := kvpb.NewScan(remaining.Key, remaining.EndKey, false /* forUpdate */).(*kvpb.ScanRequest) + r.ScanFormat = kvpb.BATCH_RESPONSE b.Header.TargetBytes = targetBytesPerScan - b.AdmissionHeader = roachpb.AdmissionHeader{ + b.AdmissionHeader = kvpb.AdmissionHeader{ // TODO(irfansharif): Make this configurable if we want system table // scanners or support "high priority" changefeeds to run at higher // priorities. We use higher AC priorities for system-internal @@ -201,7 +202,7 @@ func (p *scanRequestScanner) exportSpan( // txn level) -- this way later batches from earlier txns don't just // out compete batches from newer txns. CreateTime: start.UnixNano(), - Source: roachpb.AdmissionHeader_FROM_SQL, + Source: kvpb.AdmissionHeader_FROM_SQL, NoMemoryReservedAtSource: true, } // NB: We use a raw request rather than the Scan() method because we want @@ -296,7 +297,7 @@ func getSpansToProcess( func slurpScanResponse( ctx context.Context, sink kvevent.Writer, - res *roachpb.ScanResponse, + res *kvpb.ScanResponse, backfillTS hlc.Timestamp, withDiff bool, span roachpb.Span, diff --git a/pkg/ccl/changefeedccl/kvfeed/testing_knobs.go b/pkg/ccl/changefeedccl/kvfeed/testing_knobs.go index 6fd5074e2fa5..1982d6cdb9d0 100644 --- a/pkg/ccl/changefeedccl/kvfeed/testing_knobs.go +++ b/pkg/ccl/changefeedccl/kvfeed/testing_knobs.go @@ -11,7 +11,7 @@ package kvfeed import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" ) @@ -23,7 +23,7 @@ type TestingKnobs struct { OnRangeFeedValue func() error // ShouldSkipCheckpoint invoked when rangefed receives a checkpoint. // Returns true if checkpoint should be skipped. - ShouldSkipCheckpoint func(*roachpb.RangeFeedCheckpoint) bool + ShouldSkipCheckpoint func(*kvpb.RangeFeedCheckpoint) bool // OnRangeFeedStart invoked when rangefeed starts. It is given // the list of SpanTimePairs. OnRangeFeedStart func(spans []kvcoord.SpanTimePair) diff --git a/pkg/ccl/changefeedccl/schemafeed/BUILD.bazel b/pkg/ccl/changefeedccl/schemafeed/BUILD.bazel index 90aab901008e..d4f406646d59 100644 --- a/pkg/ccl/changefeedccl/schemafeed/BUILD.bazel +++ b/pkg/ccl/changefeedccl/schemafeed/BUILD.bazel @@ -17,6 +17,7 @@ go_library( "//pkg/ccl/changefeedccl/changefeedvalidators", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/settings", "//pkg/settings/cluster", @@ -61,7 +62,7 @@ go_test( "//pkg/ccl/changefeedccl/schemafeed/schematestutils", "//pkg/jobs/jobspb", "//pkg/keys", - "//pkg/roachpb", + "//pkg/kv/kvpb", "//pkg/security/securityassets", "//pkg/security/securitytest", "//pkg/server", diff --git a/pkg/ccl/changefeedccl/schemafeed/schema_feed.go b/pkg/ccl/changefeedccl/schemafeed/schema_feed.go index b569c3d82b5f..8d6563a259cb 100644 --- a/pkg/ccl/changefeedccl/schemafeed/schema_feed.go +++ b/pkg/ccl/changefeedccl/schemafeed/schema_feed.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedvalidators" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -577,17 +578,17 @@ func fetchDescriptorsWithPriorityOverride( sender kv.Sender, codec keys.SQLCodec, startTS, endTS hlc.Timestamp, -) (roachpb.Response, error) { +) (kvpb.Response, error) { span := roachpb.Span{Key: codec.TablePrefix(keys.DescriptorTableID)} span.EndKey = span.Key.PrefixEnd() - header := roachpb.Header{Timestamp: endTS} - req := &roachpb.ExportRequest{ - RequestHeader: roachpb.RequestHeaderFromSpan(span), + header := kvpb.Header{Timestamp: endTS} + req := &kvpb.ExportRequest{ + RequestHeader: kvpb.RequestHeaderFromSpan(span), StartTime: startTS, - MVCCFilter: roachpb.MVCCFilter_All, + MVCCFilter: kvpb.MVCCFilter_All, } - fetchDescriptors := func(ctx context.Context) (roachpb.Response, error) { + fetchDescriptors := func(ctx context.Context) (kvpb.Response, error) { resp, pErr := kv.SendWrappedWith(ctx, sender, header, req) if pErr != nil { err := pErr.GoError() @@ -601,7 +602,7 @@ func fetchDescriptorsWithPriorityOverride( return fetchDescriptors(ctx) } - var resp roachpb.Response + var resp kvpb.Response err := contextutil.RunWithTimeout( ctx, "schema-feed", priorityAfter, func(ctx context.Context) error { @@ -642,7 +643,7 @@ func (tf *schemaFeed) fetchDescriptorVersions( var descriptors []catalog.Descriptor found := errors.New(``) - for _, file := range res.(*roachpb.ExportResponse).Files { + for _, file := range res.(*kvpb.ExportResponse).Files { if err := func() error { it, err := storage.NewMemSSTIterator(file.SST, false /* verify */, storage.IterOptions{ // NB: We assume there will be no MVCC range tombstones here. diff --git a/pkg/ccl/changefeedccl/schemafeed/schema_feed_test.go b/pkg/ccl/changefeedccl/schemafeed/schema_feed_test.go index 27f54c5ccfa5..ac3ceb32d6c7 100644 --- a/pkg/ccl/changefeedccl/schemafeed/schema_feed_test.go +++ b/pkg/ccl/changefeedccl/schemafeed/schema_feed_test.go @@ -15,7 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" @@ -167,14 +167,14 @@ func TestIssuesHighPriorityReadsIfBlocked(t *testing.T) { // Attempt to fetch descriptors; it should succeed withing reasonable time. priorityAfter := 500 * time.Millisecond highPriorityAfter.Override(ctx, &s.ClusterSettings().SV, priorityAfter) - var responseFiles []roachpb.ExportResponse_File + var responseFiles []kvpb.ExportResponse_File testutils.SucceedsWithin(t, func() error { resp, err := fetchDescriptorsWithPriorityOverride(ctx, s.ClusterSettings(), kvDB.NonTransactionalSender(), keys.SystemSQLCodec, hlc.Timestamp{}, s.Clock().Now()) if err != nil { return err } - responseFiles = resp.(*roachpb.ExportResponse).Files + responseFiles = resp.(*kvpb.ExportResponse).Files return nil }, 10*priorityAfter) require.Less(t, 0, len(responseFiles)) diff --git a/pkg/ccl/kvccl/kvfollowerreadsccl/BUILD.bazel b/pkg/ccl/kvccl/kvfollowerreadsccl/BUILD.bazel index 75c3fe91dedc..b55ef2af7b71 100644 --- a/pkg/ccl/kvccl/kvfollowerreadsccl/BUILD.bazel +++ b/pkg/ccl/kvccl/kvfollowerreadsccl/BUILD.bazel @@ -14,6 +14,7 @@ go_library( "//pkg/ccl/utilccl", "//pkg/kv", "//pkg/kv/kvclient/kvcoord", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/kv/kvserver/closedts", "//pkg/roachpb", @@ -53,6 +54,7 @@ go_test( "//pkg/kv", "//pkg/kv/kvbase", "//pkg/kv/kvclient/kvcoord", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/kv/kvserver/closedts", "//pkg/kv/kvserver/concurrency/lock", diff --git a/pkg/ccl/kvccl/kvfollowerreadsccl/boundedstaleness_test.go b/pkg/ccl/kvccl/kvfollowerreadsccl/boundedstaleness_test.go index 1a899961d170..043b702c3098 100644 --- a/pkg/ccl/kvccl/kvfollowerreadsccl/boundedstaleness_test.go +++ b/pkg/ccl/kvccl/kvfollowerreadsccl/boundedstaleness_test.go @@ -21,7 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvbase" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -136,7 +136,7 @@ func (ev *boundedStalenessTraceEvent) EventOutput() string { // transaction as a result of a nearest_only bounded staleness restart. type boundedStalenessRetryEvent struct { nodeIdx int - *roachpb.MinTimestampBoundUnsatisfiableError + *kvpb.MinTimestampBoundUnsatisfiableError asOf eval.AsOfSystemTime } @@ -224,7 +224,7 @@ func (bse *boundedStalenessEvents) onTxnRetry( if bse.mu.stmt == "" { return } - var minTSErr *roachpb.MinTimestampBoundUnsatisfiableError + var minTSErr *kvpb.MinTimestampBoundUnsatisfiableError if autoRetryReason != nil && errors.As(autoRetryReason, &minTSErr) { ev := &boundedStalenessRetryEvent{ nodeIdx: nodeIdx, diff --git a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads.go b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads.go index a350c2a68d09..6231100d5a0b 100644 --- a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads.go +++ b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/utilccl" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -132,7 +133,7 @@ func canSendToFollower( st *cluster.Settings, clock *hlc.Clock, ctPolicy roachpb.RangeClosedTimestampPolicy, - ba *roachpb.BatchRequest, + ba *kvpb.BatchRequest, ) bool { return kvserver.BatchCanBeEvaluatedOnFollower(ba) && closedTimestampLikelySufficient(st, clock, ctPolicy, ba.RequiredFrontier()) && @@ -180,7 +181,7 @@ func (o *followerReadOracle) useClosestOracle( txn *kv.Txn, ctPolicy roachpb.RangeClosedTimestampPolicy, ) bool { // NOTE: this logic is almost identical to canSendToFollower, except that it - // operates on a *kv.Txn instead of a roachpb.BatchRequest. As a result, the + // operates on a *kv.Txn instead of a kvpb.BatchRequest. As a result, the // function does not check batchCanBeEvaluatedOnFollower. This is because we // assume that if a request is going to be executed in a distributed DistSQL // flow (which is why it is consulting a replicaoracle.Oracle), then all of diff --git a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go index 9ec0c056483e..abf32a010966 100644 --- a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go +++ b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" @@ -108,19 +109,19 @@ func TestCanSendToFollower(t *testing.T) { txn.GlobalUncertaintyLimit = ts return txn } - batch := func(txn *roachpb.Transaction, reqs ...roachpb.Request) *roachpb.BatchRequest { - ba := &roachpb.BatchRequest{} + batch := func(txn *roachpb.Transaction, reqs ...kvpb.Request) *kvpb.BatchRequest { + ba := &kvpb.BatchRequest{} ba.Txn = txn for _, req := range reqs { ba.Add(req) } return ba } - withBatchTimestamp := func(ba *roachpb.BatchRequest, ts hlc.Timestamp) *roachpb.BatchRequest { + withBatchTimestamp := func(ba *kvpb.BatchRequest, ts hlc.Timestamp) *kvpb.BatchRequest { ba.Timestamp = ts return ba } - withServerSideBatchTimestamp := func(ba *roachpb.BatchRequest, ts hlc.Timestamp) *roachpb.BatchRequest { + withServerSideBatchTimestamp := func(ba *kvpb.BatchRequest, ts hlc.Timestamp) *kvpb.BatchRequest { ba = withBatchTimestamp(ba, ts) ba.TimestampFromServerClock = (*hlc.ClockTimestamp)(&ts) return ba @@ -128,7 +129,7 @@ func TestCanSendToFollower(t *testing.T) { testCases := []struct { name string - ba *roachpb.BatchRequest + ba *kvpb.BatchRequest ctPolicy roachpb.RangeClosedTimestampPolicy disabledEnterprise bool disabledFollowerReads bool @@ -137,334 +138,334 @@ func TestCanSendToFollower(t *testing.T) { }{ { name: "non-txn batch, without ts", - ba: batch(nil, &roachpb.GetRequest{}), + ba: batch(nil, &kvpb.GetRequest{}), exp: false, }, { name: "stale non-txn batch", - ba: withBatchTimestamp(batch(nil, &roachpb.GetRequest{}), stale), + ba: withBatchTimestamp(batch(nil, &kvpb.GetRequest{}), stale), exp: true, }, { name: "stale non-txn export batch", - ba: withBatchTimestamp(batch(nil, &roachpb.ExportRequest{}), stale), + ba: withBatchTimestamp(batch(nil, &kvpb.ExportRequest{}), stale), exp: true, }, { name: "stale non-txn multiple exports batch", - ba: withBatchTimestamp(batch(nil, &roachpb.ExportRequest{}, &roachpb.ExportRequest{}), stale), + ba: withBatchTimestamp(batch(nil, &kvpb.ExportRequest{}, &kvpb.ExportRequest{}), stale), exp: true, }, { name: "stale non-txn mixed export-scan batch", - ba: withBatchTimestamp(batch(nil, &roachpb.ExportRequest{}, &roachpb.ScanRequest{}), stale), + ba: withBatchTimestamp(batch(nil, &kvpb.ExportRequest{}, &kvpb.ScanRequest{}), stale), exp: true, }, { name: "current-time non-txn batch", - ba: withBatchTimestamp(batch(nil, &roachpb.GetRequest{}), current), + ba: withBatchTimestamp(batch(nil, &kvpb.GetRequest{}), current), exp: false, }, { name: "current-time non-txn export batch", - ba: withBatchTimestamp(batch(nil, &roachpb.ExportRequest{}), current), + ba: withBatchTimestamp(batch(nil, &kvpb.ExportRequest{}), current), exp: false, }, { name: "current-time non-txn multiple exports batch", - ba: withBatchTimestamp(batch(nil, &roachpb.ExportRequest{}, &roachpb.ExportRequest{}), current), + ba: withBatchTimestamp(batch(nil, &kvpb.ExportRequest{}, &kvpb.ExportRequest{}), current), exp: false, }, { name: "future non-txn batch", - ba: withBatchTimestamp(batch(nil, &roachpb.GetRequest{}), future), + ba: withBatchTimestamp(batch(nil, &kvpb.GetRequest{}), future), exp: false, }, { name: "future non-txn export batch", - ba: withBatchTimestamp(batch(nil, &roachpb.ExportRequest{}), future), + ba: withBatchTimestamp(batch(nil, &kvpb.ExportRequest{}), future), exp: false, }, { name: "stale non-txn batch, server-side ts", - ba: withServerSideBatchTimestamp(batch(nil, &roachpb.GetRequest{}), stale), + ba: withServerSideBatchTimestamp(batch(nil, &kvpb.GetRequest{}), stale), exp: false, }, { name: "current-time non-txn batch, server-side ts", - ba: withServerSideBatchTimestamp(batch(nil, &roachpb.GetRequest{}), current), + ba: withServerSideBatchTimestamp(batch(nil, &kvpb.GetRequest{}), current), exp: false, }, { name: "future non-txn batch, server-side ts", - ba: withServerSideBatchTimestamp(batch(nil, &roachpb.GetRequest{}), future), + ba: withServerSideBatchTimestamp(batch(nil, &kvpb.GetRequest{}), future), exp: false, }, { name: "stale read", - ba: batch(txn(stale), &roachpb.GetRequest{}), + ba: batch(txn(stale), &kvpb.GetRequest{}), exp: true, }, { name: "stale locking read", - ba: batch(txn(stale), &roachpb.GetRequest{KeyLocking: lock.Exclusive}), + ba: batch(txn(stale), &kvpb.GetRequest{KeyLocking: lock.Exclusive}), exp: false, }, { name: "stale scan", - ba: batch(txn(stale), &roachpb.ScanRequest{}), + ba: batch(txn(stale), &kvpb.ScanRequest{}), exp: true, }, { name: "stale reverse scan", - ba: batch(txn(stale), &roachpb.ReverseScanRequest{}), + ba: batch(txn(stale), &kvpb.ReverseScanRequest{}), exp: true, }, { name: "stale refresh", - ba: batch(txn(stale), &roachpb.RefreshRequest{}), + ba: batch(txn(stale), &kvpb.RefreshRequest{}), exp: true, }, { name: "stale refresh range", - ba: batch(txn(stale), &roachpb.RefreshRangeRequest{}), + ba: batch(txn(stale), &kvpb.RefreshRangeRequest{}), exp: true, }, { name: "stale write", - ba: batch(txn(stale), &roachpb.PutRequest{}), + ba: batch(txn(stale), &kvpb.PutRequest{}), exp: false, }, { name: "stale heartbeat txn", - ba: batch(txn(stale), &roachpb.HeartbeatTxnRequest{}), + ba: batch(txn(stale), &kvpb.HeartbeatTxnRequest{}), exp: false, }, { name: "stale end txn", - ba: batch(txn(stale), &roachpb.EndTxnRequest{}), + ba: batch(txn(stale), &kvpb.EndTxnRequest{}), exp: false, }, { name: "stale non-txn request", - ba: batch(txn(stale), &roachpb.QueryTxnRequest{}), + ba: batch(txn(stale), &kvpb.QueryTxnRequest{}), exp: false, }, { name: "stale read with current-time writes", - ba: batch(withWriteTimestamp(txn(stale), current), &roachpb.GetRequest{}), + ba: batch(withWriteTimestamp(txn(stale), current), &kvpb.GetRequest{}), exp: false, }, { name: "stale read with current-time uncertainty limit", - ba: batch(withUncertaintyLimit(txn(stale), current), &roachpb.GetRequest{}), + ba: batch(withUncertaintyLimit(txn(stale), current), &kvpb.GetRequest{}), exp: false, }, { name: "stale read when zero target_duration", - ba: batch(txn(stale), &roachpb.GetRequest{}), + ba: batch(txn(stale), &kvpb.GetRequest{}), zeroTargetDuration: true, exp: false, }, { name: "current-time read", - ba: batch(txn(current), &roachpb.GetRequest{}), + ba: batch(txn(current), &kvpb.GetRequest{}), exp: false, }, { name: "future read", - ba: batch(txn(future), &roachpb.GetRequest{}), + ba: batch(txn(future), &kvpb.GetRequest{}), exp: false, }, { name: "non-txn batch, without ts, global reads policy", - ba: batch(nil, &roachpb.GetRequest{}), + ba: batch(nil, &kvpb.GetRequest{}), ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: false, }, { name: "stale non-txn batch, global reads policy", - ba: withBatchTimestamp(batch(nil, &roachpb.GetRequest{}), stale), + ba: withBatchTimestamp(batch(nil, &kvpb.GetRequest{}), stale), ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: true, }, { name: "stale non-txn export batch, global reads policy", - ba: withBatchTimestamp(batch(nil, &roachpb.ExportRequest{}), stale), + ba: withBatchTimestamp(batch(nil, &kvpb.ExportRequest{}), stale), ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: true, }, { name: "current-time non-txn batch, global reads policy", - ba: withBatchTimestamp(batch(nil, &roachpb.GetRequest{}), current), + ba: withBatchTimestamp(batch(nil, &kvpb.GetRequest{}), current), ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: true, }, { name: "current-time non-txn export batch, global reads policy", - ba: withBatchTimestamp(batch(nil, &roachpb.ExportRequest{}), current), + ba: withBatchTimestamp(batch(nil, &kvpb.ExportRequest{}), current), ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: true, }, { name: "future non-txn batch, global reads policy", - ba: withBatchTimestamp(batch(nil, &roachpb.GetRequest{}), future), + ba: withBatchTimestamp(batch(nil, &kvpb.GetRequest{}), future), ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: false, }, { name: "future non-txn export batch, global reads policy", - ba: withBatchTimestamp(batch(nil, &roachpb.ExportRequest{}), future), + ba: withBatchTimestamp(batch(nil, &kvpb.ExportRequest{}), future), ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: false, }, { name: "stale non-txn batch, server-side ts, global reads policy", - ba: withServerSideBatchTimestamp(batch(nil, &roachpb.GetRequest{}), stale), + ba: withServerSideBatchTimestamp(batch(nil, &kvpb.GetRequest{}), stale), ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: false, }, { name: "current-time non-txn batch, server-side ts, global reads policy", - ba: withServerSideBatchTimestamp(batch(nil, &roachpb.GetRequest{}), current), + ba: withServerSideBatchTimestamp(batch(nil, &kvpb.GetRequest{}), current), ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: false, }, { name: "future non-txn batch, server-side ts, global reads policy", - ba: withServerSideBatchTimestamp(batch(nil, &roachpb.GetRequest{}), future), + ba: withServerSideBatchTimestamp(batch(nil, &kvpb.GetRequest{}), future), ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: false, }, { name: "stale read, global reads policy", - ba: batch(txn(stale), &roachpb.GetRequest{}), + ba: batch(txn(stale), &kvpb.GetRequest{}), ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: true, }, { name: "stale locking read, global reads policy", - ba: batch(txn(stale), &roachpb.GetRequest{KeyLocking: lock.Exclusive}), + ba: batch(txn(stale), &kvpb.GetRequest{KeyLocking: lock.Exclusive}), ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: false, }, { name: "stale scan, global reads policy", - ba: batch(txn(stale), &roachpb.ScanRequest{}), + ba: batch(txn(stale), &kvpb.ScanRequest{}), ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: true, }, { name: "stale reverse scan, global reads policy", - ba: batch(txn(stale), &roachpb.ReverseScanRequest{}), + ba: batch(txn(stale), &kvpb.ReverseScanRequest{}), ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: true, }, { name: "stale refresh, global reads policy", - ba: batch(txn(stale), &roachpb.RefreshRequest{}), + ba: batch(txn(stale), &kvpb.RefreshRequest{}), ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: true, }, { name: "stale refresh range, global reads policy", - ba: batch(txn(stale), &roachpb.RefreshRangeRequest{}), + ba: batch(txn(stale), &kvpb.RefreshRangeRequest{}), ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: true, }, { name: "stale write, global reads policy", - ba: batch(txn(stale), &roachpb.PutRequest{}), + ba: batch(txn(stale), &kvpb.PutRequest{}), ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: false, }, { name: "stale heartbeat txn, global reads policy", - ba: batch(txn(stale), &roachpb.HeartbeatTxnRequest{}), + ba: batch(txn(stale), &kvpb.HeartbeatTxnRequest{}), ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: false, }, { name: "stale end txn, global reads policy", - ba: batch(txn(stale), &roachpb.EndTxnRequest{}), + ba: batch(txn(stale), &kvpb.EndTxnRequest{}), ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: false, }, { name: "stale non-txn request, global reads policy", - ba: batch(txn(stale), &roachpb.QueryTxnRequest{}), + ba: batch(txn(stale), &kvpb.QueryTxnRequest{}), ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: false, }, { name: "stale read with current-time writes, global reads policy", - ba: batch(withWriteTimestamp(txn(stale), current), &roachpb.GetRequest{}), + ba: batch(withWriteTimestamp(txn(stale), current), &kvpb.GetRequest{}), ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: true, }, { name: "stale read with current-time uncertainty limit, global reads policy", - ba: batch(withUncertaintyLimit(txn(stale), current), &roachpb.GetRequest{}), + ba: batch(withUncertaintyLimit(txn(stale), current), &kvpb.GetRequest{}), ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: true, }, { name: "current-time read, global reads policy", - ba: batch(txn(current), &roachpb.GetRequest{}), + ba: batch(txn(current), &kvpb.GetRequest{}), ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: true, }, { name: "current-time scan, global reads policy", - ba: batch(txn(current), &roachpb.ScanRequest{}), + ba: batch(txn(current), &kvpb.ScanRequest{}), ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: true, }, { name: "current-time reverse scan, global reads policy", - ba: batch(txn(current), &roachpb.ReverseScanRequest{}), + ba: batch(txn(current), &kvpb.ReverseScanRequest{}), ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: true, }, { name: "current-time refresh, global reads policy", - ba: batch(txn(current), &roachpb.RefreshRequest{}), + ba: batch(txn(current), &kvpb.RefreshRequest{}), ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: true, }, { name: "current-time refresh range, global reads policy", - ba: batch(txn(current), &roachpb.RefreshRangeRequest{}), + ba: batch(txn(current), &kvpb.RefreshRangeRequest{}), ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: true, }, { name: "current-time read with future writes, global reads policy", - ba: batch(withWriteTimestamp(txn(current), future), &roachpb.GetRequest{}), + ba: batch(withWriteTimestamp(txn(current), future), &kvpb.GetRequest{}), ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: false, }, { name: "current-time read with future uncertainty limit, global reads policy", - ba: batch(withUncertaintyLimit(txn(current), future), &roachpb.GetRequest{}), + ba: batch(withUncertaintyLimit(txn(current), future), &kvpb.GetRequest{}), ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: false, }, { name: "future read, global reads policy", - ba: batch(txn(future), &roachpb.GetRequest{}), + ba: batch(txn(future), &kvpb.GetRequest{}), ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: false, }, { name: "non-enterprise", - ba: withBatchTimestamp(batch(nil, &roachpb.GetRequest{}), stale), + ba: withBatchTimestamp(batch(nil, &kvpb.GetRequest{}), stale), disabledEnterprise: true, exp: false, }, { name: "follower reads disabled", - ba: withBatchTimestamp(batch(nil, &roachpb.GetRequest{}), stale), + ba: withBatchTimestamp(batch(nil, &kvpb.GetRequest{}), stale), disabledFollowerReads: true, exp: false, }, diff --git a/pkg/ccl/kvccl/kvtenantccl/BUILD.bazel b/pkg/ccl/kvccl/kvtenantccl/BUILD.bazel index de21731b94bf..758d97a361e6 100644 --- a/pkg/ccl/kvccl/kvtenantccl/BUILD.bazel +++ b/pkg/ccl/kvccl/kvtenantccl/BUILD.bazel @@ -18,6 +18,7 @@ go_library( "//pkg/kv/kvclient/kvcoord", "//pkg/kv/kvclient/kvtenant", "//pkg/kv/kvclient/rangecache", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/rpc", "//pkg/server/serverpb", @@ -69,6 +70,7 @@ go_test( "//pkg/kv/kvclient/kvcoord", "//pkg/kv/kvclient/kvtenant", "//pkg/kv/kvclient/rangecache", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/kv/kvserver/kvserverbase", "//pkg/kv/kvserver/load", diff --git a/pkg/ccl/kvccl/kvtenantccl/connector.go b/pkg/ccl/kvccl/kvtenantccl/connector.go index d78f2931a743..88b77a34be22 100644 --- a/pkg/ccl/kvccl/kvtenantccl/connector.go +++ b/pkg/ccl/kvccl/kvtenantccl/connector.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvtenant" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangecache" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/server/serverpb" @@ -97,7 +98,7 @@ type Connector struct { // client represents an RPC client that proxies to a KV instance. type client struct { - roachpb.InternalClient + kvpb.InternalClient serverpb.StatusClient serverpb.AdminClient tspb.TimeSeriesClient @@ -230,7 +231,7 @@ func (c *Connector) runGossipSubscription(ctx context.Context, startupCh chan st if err != nil { continue } - stream, err := client.GossipSubscription(ctx, &roachpb.GossipSubscriptionRequest{ + stream, err := client.GossipSubscription(ctx, &kvpb.GossipSubscriptionRequest{ Patterns: gossipSubsPatterns, }) if err != nil { @@ -436,7 +437,7 @@ func (c *Connector) RangeLookup( if err != nil { continue } - resp, err := client.RangeLookup(ctx, &roachpb.RangeLookupRequest{ + resp, err := client.RangeLookup(ctx, &kvpb.RangeLookupRequest{ Key: key, // See the comment on (*kvcoord.DistSender).RangeLookup or kv.RangeLookup // for more discussion on the choice of ReadConsistency and its @@ -513,7 +514,7 @@ func (c *Connector) NewIterator( if err != nil { continue } - stream, err := client.GetRangeDescriptors(ctx, &roachpb.GetRangeDescriptorsRequest{ + stream, err := client.GetRangeDescriptors(ctx, &kvpb.GetRangeDescriptorsRequest{ Span: span, }) if err != nil { @@ -549,8 +550,8 @@ func (c *Connector) NewIterator( // TokenBucket implements the kvtenant.TokenBucketProvider interface. func (c *Connector) TokenBucket( - ctx context.Context, in *roachpb.TokenBucketRequest, -) (*roachpb.TokenBucketResponse, error) { + ctx context.Context, in *kvpb.TokenBucketRequest, +) (*kvpb.TokenBucketResponse, error) { // Proxy token bucket requests through the Internal service. ctx = c.AnnotateCtx(ctx) for ctx.Err() == nil { @@ -786,7 +787,7 @@ func (c *Connector) dialAddrs(ctx context.Context) (*client, error) { continue } return &client{ - InternalClient: roachpb.NewInternalClient(conn), + InternalClient: kvpb.NewInternalClient(conn), StatusClient: serverpb.NewStatusClient(conn), AdminClient: serverpb.NewAdminClient(conn), TimeSeriesClient: tspb.NewTimeSeriesClient(conn), @@ -807,7 +808,7 @@ func (c *Connector) dialAddr(ctx context.Context, addr string) (conn *grpc.Clien return conn, err } -func (c *Connector) tryForgetClient(ctx context.Context, client roachpb.InternalClient) { +func (c *Connector) tryForgetClient(ctx context.Context, client kvpb.InternalClient) { if ctx.Err() != nil { // Error (may be) due to context. Don't forget client. return diff --git a/pkg/ccl/kvccl/kvtenantccl/connector_test.go b/pkg/ccl/kvccl/kvtenantccl/connector_test.go index 440e2a71c8c2..3b6c4b0fc701 100644 --- a/pkg/ccl/kvccl/kvtenantccl/connector_test.go +++ b/pkg/ccl/kvccl/kvtenantccl/connector_test.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvtenant" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -44,32 +45,32 @@ var rpcRetryOpts = retry.Options{ MaxBackoff: 4 * time.Microsecond, } -var _ roachpb.InternalServer = &mockServer{} +var _ kvpb.InternalServer = &mockServer{} type mockServer struct { - rangeLookupFn func(context.Context, *roachpb.RangeLookupRequest) (*roachpb.RangeLookupResponse, error) - gossipSubFn func(*roachpb.GossipSubscriptionRequest, roachpb.Internal_GossipSubscriptionServer) error - tenantSettingsFn func(request *roachpb.TenantSettingsRequest, server roachpb.Internal_TenantSettingsServer) error + rangeLookupFn func(context.Context, *kvpb.RangeLookupRequest) (*kvpb.RangeLookupResponse, error) + gossipSubFn func(*kvpb.GossipSubscriptionRequest, kvpb.Internal_GossipSubscriptionServer) error + tenantSettingsFn func(request *kvpb.TenantSettingsRequest, server kvpb.Internal_TenantSettingsServer) error } func (m *mockServer) RangeLookup( - ctx context.Context, req *roachpb.RangeLookupRequest, -) (*roachpb.RangeLookupResponse, error) { + ctx context.Context, req *kvpb.RangeLookupRequest, +) (*kvpb.RangeLookupResponse, error) { return m.rangeLookupFn(ctx, req) } func (m *mockServer) GossipSubscription( - req *roachpb.GossipSubscriptionRequest, stream roachpb.Internal_GossipSubscriptionServer, + req *kvpb.GossipSubscriptionRequest, stream kvpb.Internal_GossipSubscriptionServer, ) error { return m.gossipSubFn(req, stream) } func (m *mockServer) TenantSettings( - req *roachpb.TenantSettingsRequest, stream roachpb.Internal_TenantSettingsServer, + req *kvpb.TenantSettingsRequest, stream kvpb.Internal_TenantSettingsServer, ) error { if m.tenantSettingsFn == nil { - return stream.Send(&roachpb.TenantSettingsEvent{ - Precedence: roachpb.SpecificTenantOverrides, + return stream.Send(&kvpb.TenantSettingsEvent{ + Precedence: kvpb.SpecificTenantOverrides, Incremental: false, Overrides: nil, }) @@ -78,32 +79,30 @@ func (m *mockServer) TenantSettings( } func (*mockServer) ResetQuorum( - context.Context, *roachpb.ResetQuorumRequest, -) (*roachpb.ResetQuorumResponse, error) { + context.Context, *kvpb.ResetQuorumRequest, +) (*kvpb.ResetQuorumResponse, error) { panic("unimplemented") } -func (*mockServer) Batch(context.Context, *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { +func (*mockServer) Batch(context.Context, *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { panic("unimplemented") } -func (*mockServer) RangeFeed(*roachpb.RangeFeedRequest, roachpb.Internal_RangeFeedServer) error { +func (*mockServer) RangeFeed(*kvpb.RangeFeedRequest, kvpb.Internal_RangeFeedServer) error { panic("unimplemented") } -func (m *mockServer) MuxRangeFeed(server roachpb.Internal_MuxRangeFeedServer) error { +func (m *mockServer) MuxRangeFeed(server kvpb.Internal_MuxRangeFeedServer) error { panic("implement me") } -func (*mockServer) Join( - context.Context, *roachpb.JoinNodeRequest, -) (*roachpb.JoinNodeResponse, error) { +func (*mockServer) Join(context.Context, *kvpb.JoinNodeRequest) (*kvpb.JoinNodeResponse, error) { panic("unimplemented") } func (*mockServer) TokenBucket( - ctx context.Context, in *roachpb.TokenBucketRequest, -) (*roachpb.TokenBucketResponse, error) { + ctx context.Context, in *kvpb.TokenBucketRequest, +) (*kvpb.TokenBucketResponse, error) { panic("unimplemented") } @@ -132,49 +131,49 @@ func (m *mockServer) SpanConfigConformance( } func (m *mockServer) GetRangeDescriptors( - *roachpb.GetRangeDescriptorsRequest, roachpb.Internal_GetRangeDescriptorsServer, + *kvpb.GetRangeDescriptorsRequest, kvpb.Internal_GetRangeDescriptorsServer, ) error { panic("unimplemented") } -func gossipEventForClusterID(clusterID uuid.UUID) *roachpb.GossipSubscriptionEvent { - return &roachpb.GossipSubscriptionEvent{ +func gossipEventForClusterID(clusterID uuid.UUID) *kvpb.GossipSubscriptionEvent { + return &kvpb.GossipSubscriptionEvent{ Key: gossip.KeyClusterID, Content: roachpb.MakeValueFromBytesAndTimestamp(clusterID.GetBytes(), hlc.Timestamp{}), PatternMatched: gossip.KeyClusterID, } } -func gossipEventForNodeDesc(desc *roachpb.NodeDescriptor) *roachpb.GossipSubscriptionEvent { +func gossipEventForNodeDesc(desc *roachpb.NodeDescriptor) *kvpb.GossipSubscriptionEvent { val, err := protoutil.Marshal(desc) if err != nil { panic(err) } - return &roachpb.GossipSubscriptionEvent{ + return &kvpb.GossipSubscriptionEvent{ Key: gossip.MakeNodeIDKey(desc.NodeID), Content: roachpb.MakeValueFromBytesAndTimestamp(val, hlc.Timestamp{}), PatternMatched: gossip.MakePrefixPattern(gossip.KeyNodeDescPrefix), } } -func gossipEventForStoreDesc(desc *roachpb.StoreDescriptor) *roachpb.GossipSubscriptionEvent { +func gossipEventForStoreDesc(desc *roachpb.StoreDescriptor) *kvpb.GossipSubscriptionEvent { val, err := protoutil.Marshal(desc) if err != nil { panic(err) } - return &roachpb.GossipSubscriptionEvent{ + return &kvpb.GossipSubscriptionEvent{ Key: gossip.MakeStoreDescKey(desc.StoreID), Content: roachpb.MakeValueFromBytesAndTimestamp(val, hlc.Timestamp{}), PatternMatched: gossip.MakePrefixPattern(gossip.KeyStoreDescPrefix), } } -func gossipEventForSystemConfig(cfg *config.SystemConfigEntries) *roachpb.GossipSubscriptionEvent { +func gossipEventForSystemConfig(cfg *config.SystemConfigEntries) *kvpb.GossipSubscriptionEvent { val, err := protoutil.Marshal(cfg) if err != nil { panic(err) } - return &roachpb.GossipSubscriptionEvent{ + return &kvpb.GossipSubscriptionEvent{ Key: gossip.KeyDeprecatedSystemConfig, Content: roachpb.MakeValueFromBytesAndTimestamp(val, hlc.Timestamp{}), PatternMatched: gossip.KeyDeprecatedSystemConfig, @@ -215,9 +214,9 @@ func TestConnectorGossipSubscription(t *testing.T) { clusterID := rpcContext.StorageClusterID.Get() rpcContext.StorageClusterID.Reset(uuid.Nil) - gossipSubC := make(chan *roachpb.GossipSubscriptionEvent) + gossipSubC := make(chan *kvpb.GossipSubscriptionEvent) defer close(gossipSubC) - gossipSubFn := func(req *roachpb.GossipSubscriptionRequest, stream roachpb.Internal_GossipSubscriptionServer) error { + gossipSubFn := func(req *kvpb.GossipSubscriptionRequest, stream kvpb.Internal_GossipSubscriptionServer) error { assert.Len(t, req.Patterns, 4) assert.Equal(t, "cluster-id", req.Patterns[0]) assert.Equal(t, "node:.*", req.Patterns[1]) @@ -230,7 +229,7 @@ func TestConnectorGossipSubscription(t *testing.T) { } return nil } - roachpb.RegisterInternalServer(s, &mockServer{gossipSubFn: gossipSubFn}) + kvpb.RegisterInternalServer(s, &mockServer{gossipSubFn: gossipSubFn}) ln, err := netutil.ListenAndServeGRPC(stopper, s, util.TestAddr) require.NoError(t, err) @@ -363,11 +362,11 @@ func TestConnectorRangeLookup(t *testing.T) { s, err := rpc.NewServer(rpcContext) require.NoError(t, err) - rangeLookupRespC := make(chan *roachpb.RangeLookupResponse, 1) - rangeLookupFn := func(_ context.Context, req *roachpb.RangeLookupRequest) (*roachpb.RangeLookupResponse, error) { + rangeLookupRespC := make(chan *kvpb.RangeLookupResponse, 1) + rangeLookupFn := func(_ context.Context, req *kvpb.RangeLookupRequest) (*kvpb.RangeLookupResponse, error) { // Validate request. assert.Equal(t, roachpb.RKey("a"), req.Key) - assert.Equal(t, roachpb.READ_UNCOMMITTED, req.ReadConsistency) + assert.Equal(t, kvpb.READ_UNCOMMITTED, req.ReadConsistency) assert.Equal(t, int64(kvcoord.RangeLookupPrefetchCount), req.PrefetchNum) assert.Equal(t, false, req.PrefetchReverse) @@ -375,7 +374,7 @@ func TestConnectorRangeLookup(t *testing.T) { return <-rangeLookupRespC, nil } server := &mockServer{rangeLookupFn: rangeLookupFn} - roachpb.RegisterInternalServer(s, server) + kvpb.RegisterInternalServer(s, server) ln, err := netutil.ListenAndServeGRPC(stopper, s, util.TestAddr) require.NoError(t, err) @@ -392,18 +391,18 @@ func TestConnectorRangeLookup(t *testing.T) { // Success case. descs := []roachpb.RangeDescriptor{{RangeID: 1}, {RangeID: 2}} preDescs := []roachpb.RangeDescriptor{{RangeID: 3}, {RangeID: 4}} - rangeLookupRespC <- &roachpb.RangeLookupResponse{ + rangeLookupRespC <- &kvpb.RangeLookupResponse{ Descriptors: descs, PrefetchedDescriptors: preDescs, } - const rc = roachpb.READ_UNCOMMITTED + const rc = kvpb.READ_UNCOMMITTED resDescs, resPreDescs, err := c.RangeLookup(ctx, roachpb.RKey("a"), rc, false /* useReverseScan */) require.Equal(t, descs, resDescs) require.Equal(t, preDescs, resPreDescs) require.NoError(t, err) // Error case. - rangeLookupRespC <- &roachpb.RangeLookupResponse{ - Error: roachpb.NewErrorf("hit error"), + rangeLookupRespC <- &kvpb.RangeLookupResponse{ + Error: kvpb.NewErrorf("hit error"), } resDescs, resPreDescs, err = c.RangeLookup(ctx, roachpb.RKey("a"), rc, false /* useReverseScan */) require.Nil(t, resDescs) @@ -413,7 +412,7 @@ func TestConnectorRangeLookup(t *testing.T) { // Context cancelation. canceledCtx, cancel := context.WithCancel(ctx) blockingC := make(chan struct{}) - server.rangeLookupFn = func(ctx context.Context, _ *roachpb.RangeLookupRequest) (*roachpb.RangeLookupResponse, error) { + server.rangeLookupFn = func(ctx context.Context, _ *kvpb.RangeLookupRequest) (*kvpb.RangeLookupResponse, error) { <-blockingC <-ctx.Done() return nil, ctx.Err() @@ -451,12 +450,12 @@ func TestConnectorRetriesUnreachable(t *testing.T) { node1 := &roachpb.NodeDescriptor{NodeID: 1, Address: util.MakeUnresolvedAddr("tcp", "1.1.1.1")} node2 := &roachpb.NodeDescriptor{NodeID: 2, Address: util.MakeUnresolvedAddr("tcp", "2.2.2.2")} - gossipSubEvents := []*roachpb.GossipSubscriptionEvent{ + gossipSubEvents := []*kvpb.GossipSubscriptionEvent{ gossipEventForClusterID(rpcContext.StorageClusterID.Get()), gossipEventForNodeDesc(node1), gossipEventForNodeDesc(node2), } - gossipSubFn := func(req *roachpb.GossipSubscriptionRequest, stream roachpb.Internal_GossipSubscriptionServer) error { + gossipSubFn := func(req *kvpb.GossipSubscriptionRequest, stream kvpb.Internal_GossipSubscriptionServer) error { assert.Len(t, req.Patterns, 4) assert.Equal(t, "cluster-id", req.Patterns[0]) assert.Equal(t, "node:.*", req.Patterns[1]) @@ -470,7 +469,7 @@ func TestConnectorRetriesUnreachable(t *testing.T) { <-stream.Context().Done() return stream.Context().Err() } - roachpb.RegisterInternalServer(s, &mockServer{gossipSubFn: gossipSubFn}) + kvpb.RegisterInternalServer(s, &mockServer{gossipSubFn: gossipSubFn}) // Decompose netutil.ListenAndServeGRPC so we can listen before serving. ln, err := net.Listen(util.TestAddr.Network(), util.TestAddr.String()) require.NoError(t, err) @@ -539,12 +538,12 @@ func TestConnectorRetriesError(t *testing.T) { // Returns address on which server is listening for use in connector. createServer := func( t *testing.T, - gossipSubFn func(req *roachpb.GossipSubscriptionRequest, stream roachpb.Internal_GossipSubscriptionServer) error, - rangeLookupFn func(_ context.Context, req *roachpb.RangeLookupRequest) (*roachpb.RangeLookupResponse, error), + gossipSubFn func(req *kvpb.GossipSubscriptionRequest, stream kvpb.Internal_GossipSubscriptionServer) error, + rangeLookupFn func(_ context.Context, req *kvpb.RangeLookupRequest) (*kvpb.RangeLookupResponse, error), ) string { internalServer, err := rpc.NewServer(rpcContext) require.NoError(t, err) - roachpb.RegisterInternalServer(internalServer, &mockServer{rangeLookupFn: rangeLookupFn, gossipSubFn: gossipSubFn}) + kvpb.RegisterInternalServer(internalServer, &mockServer{rangeLookupFn: rangeLookupFn, gossipSubFn: gossipSubFn}) ln, err := net.Listen(util.TestAddr.Network(), util.TestAddr.String()) require.NoError(t, err) stopper.AddCloser(stop.CloserFn(internalServer.Stop)) @@ -568,20 +567,20 @@ func TestConnectorRetriesError(t *testing.T) { } { t.Run(fmt.Sprintf("error %v retries %v", spec.code, spec.shouldRetry), func(t *testing.T) { - gossipSubFn := func(req *roachpb.GossipSubscriptionRequest, stream roachpb.Internal_GossipSubscriptionServer) error { + gossipSubFn := func(req *kvpb.GossipSubscriptionRequest, stream kvpb.Internal_GossipSubscriptionServer) error { return stream.Send(gossipEventForClusterID(rpcContext.StorageClusterID.Get())) } - rangeLookupFn := func(_ context.Context, req *roachpb.RangeLookupRequest) (*roachpb.RangeLookupResponse, error) { + rangeLookupFn := func(_ context.Context, req *kvpb.RangeLookupRequest) (*kvpb.RangeLookupResponse, error) { descs := []roachpb.RangeDescriptor{{RangeID: 1}, {RangeID: 2}} preDescs := []roachpb.RangeDescriptor{{RangeID: 3}, {RangeID: 4}} - return &roachpb.RangeLookupResponse{ + return &kvpb.RangeLookupResponse{ Descriptors: descs, PrefetchedDescriptors: preDescs, }, nil } var errorsReported int32 = 0 - rangeLookupRejectorFn := func(_ context.Context, req *roachpb.RangeLookupRequest) (*roachpb.RangeLookupResponse, error) { + rangeLookupRejectorFn := func(_ context.Context, req *kvpb.RangeLookupRequest) (*kvpb.RangeLookupResponse, error) { // Respond with error always atomic.AddInt32(&errorsReported, 1) return nil, grpcstatus.Errorf(spec.code, "range lookup rejected") @@ -607,7 +606,7 @@ func TestConnectorRetriesError(t *testing.T) { // to check if it was retried. for i := 0; i < 100; i++ { _, _, err := c.RangeLookup( - ctx, roachpb.RKey("a"), roachpb.READ_UNCOMMITTED, false, + ctx, roachpb.RKey("a"), kvpb.READ_UNCOMMITTED, false, ) if atomic.LoadInt32(&errorsReported) == 0 { continue diff --git a/pkg/ccl/kvccl/kvtenantccl/setting_overrides.go b/pkg/ccl/kvccl/kvtenantccl/setting_overrides.go index 6cea5b27687a..2f8a3ce70681 100644 --- a/pkg/ccl/kvccl/kvtenantccl/setting_overrides.go +++ b/pkg/ccl/kvccl/kvtenantccl/setting_overrides.go @@ -12,7 +12,7 @@ import ( "context" "io" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" @@ -28,7 +28,7 @@ func (c *Connector) runTenantSettingsSubscription(ctx context.Context, startupCh if err != nil { continue } - stream, err := client.TenantSettings(ctx, &roachpb.TenantSettingsRequest{ + stream, err := client.TenantSettings(ctx, &kvpb.TenantSettingsRequest{ TenantID: c.tenantID, }) if err != nil { @@ -71,7 +71,7 @@ func (c *Connector) runTenantSettingsSubscription(ctx context.Context, startupCh // processSettingsEvent updates the setting overrides based on the event. func (c *Connector) processSettingsEvent( - e *roachpb.TenantSettingsEvent, firstEventInStream bool, + e *kvpb.TenantSettingsEvent, firstEventInStream bool, ) error { if firstEventInStream && e.Incremental { return errors.Newf("first event must not be Incremental") @@ -81,9 +81,9 @@ func (c *Connector) processSettingsEvent( var m map[string]settings.EncodedValue switch e.Precedence { - case roachpb.AllTenantsOverrides: + case kvpb.AllTenantsOverrides: m = c.settingsMu.allTenantOverrides - case roachpb.SpecificTenantOverrides: + case kvpb.SpecificTenantOverrides: m = c.settingsMu.specificOverrides default: return errors.Newf("unknown precedence value %d", e.Precedence) diff --git a/pkg/ccl/kvccl/kvtenantccl/setting_overrides_test.go b/pkg/ccl/kvccl/kvtenantccl/setting_overrides_test.go index b557394104ad..55a606aeb697 100644 --- a/pkg/ccl/kvccl/kvtenantccl/setting_overrides_test.go +++ b/pkg/ccl/kvccl/kvtenantccl/setting_overrides_test.go @@ -16,6 +16,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvtenant" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/settings" @@ -42,12 +43,12 @@ func TestConnectorSettingOverrides(t *testing.T) { require.NoError(t, err) tenantID := roachpb.MustMakeTenantID(5) - gossipSubFn := func(req *roachpb.GossipSubscriptionRequest, stream roachpb.Internal_GossipSubscriptionServer) error { + gossipSubFn := func(req *kvpb.GossipSubscriptionRequest, stream kvpb.Internal_GossipSubscriptionServer) error { return stream.Send(gossipEventForClusterID(rpcContext.StorageClusterID.Get())) } - eventCh := make(chan *roachpb.TenantSettingsEvent) + eventCh := make(chan *kvpb.TenantSettingsEvent) defer close(eventCh) - settingsFn := func(req *roachpb.TenantSettingsRequest, stream roachpb.Internal_TenantSettingsServer) error { + settingsFn := func(req *kvpb.TenantSettingsRequest, stream kvpb.Internal_TenantSettingsServer) error { if req.TenantID != tenantID { t.Errorf("invalid tenantID %s (expected %s)", req.TenantID, tenantID) } @@ -58,7 +59,7 @@ func TestConnectorSettingOverrides(t *testing.T) { } return nil } - roachpb.RegisterInternalServer(s, &mockServer{ + kvpb.RegisterInternalServer(s, &mockServer{ gossipSubFn: gossipSubFn, tenantSettingsFn: settingsFn, }) @@ -89,7 +90,7 @@ func TestConnectorSettingOverrides(t *testing.T) { // We should always get an initial notification. waitForSettings(t, ch) - ev := &roachpb.TenantSettingsEvent{ + ev := &kvpb.TenantSettingsEvent{ Precedence: 1, Incremental: false, Overrides: nil, @@ -100,58 +101,58 @@ func TestConnectorSettingOverrides(t *testing.T) { waitForSettings(t, ch) expectSettings(t, c, "foo=default bar=default baz=default") - st := func(name, val string) roachpb.TenantSetting { - return roachpb.TenantSetting{ + st := func(name, val string) kvpb.TenantSetting { + return kvpb.TenantSetting{ Name: name, Value: settings.EncodedValue{Value: val}, } } // Set some all-tenant overrides. - ev = &roachpb.TenantSettingsEvent{ - Precedence: roachpb.AllTenantsOverrides, + ev = &kvpb.TenantSettingsEvent{ + Precedence: kvpb.AllTenantsOverrides, Incremental: true, - Overrides: []roachpb.TenantSetting{st("foo", "all"), st("bar", "all")}, + Overrides: []kvpb.TenantSetting{st("foo", "all"), st("bar", "all")}, } eventCh <- ev waitForSettings(t, ch) expectSettings(t, c, "foo=all bar=all baz=default") // Set some tenant-specific overrides, with all-tenant overlap. - ev = &roachpb.TenantSettingsEvent{ - Precedence: roachpb.SpecificTenantOverrides, + ev = &kvpb.TenantSettingsEvent{ + Precedence: kvpb.SpecificTenantOverrides, Incremental: true, - Overrides: []roachpb.TenantSetting{st("foo", "specific"), st("baz", "specific")}, + Overrides: []kvpb.TenantSetting{st("foo", "specific"), st("baz", "specific")}, } eventCh <- ev waitForSettings(t, ch) expectSettings(t, c, "foo=specific bar=all baz=specific") // Remove an all-tenant override that has a specific override. - ev = &roachpb.TenantSettingsEvent{ - Precedence: roachpb.AllTenantsOverrides, + ev = &kvpb.TenantSettingsEvent{ + Precedence: kvpb.AllTenantsOverrides, Incremental: true, - Overrides: []roachpb.TenantSetting{st("foo", "")}, + Overrides: []kvpb.TenantSetting{st("foo", "")}, } eventCh <- ev waitForSettings(t, ch) expectSettings(t, c, "foo=specific bar=all baz=specific") // Remove a specific override. - ev = &roachpb.TenantSettingsEvent{ - Precedence: roachpb.SpecificTenantOverrides, + ev = &kvpb.TenantSettingsEvent{ + Precedence: kvpb.SpecificTenantOverrides, Incremental: true, - Overrides: []roachpb.TenantSetting{st("foo", "")}, + Overrides: []kvpb.TenantSetting{st("foo", "")}, } eventCh <- ev waitForSettings(t, ch) expectSettings(t, c, "foo=default bar=all baz=specific") // Non-incremental change to all-tenants override. - ev = &roachpb.TenantSettingsEvent{ - Precedence: roachpb.AllTenantsOverrides, + ev = &kvpb.TenantSettingsEvent{ + Precedence: kvpb.AllTenantsOverrides, Incremental: true, - Overrides: []roachpb.TenantSetting{st("bar", "all")}, + Overrides: []kvpb.TenantSetting{st("bar", "all")}, } eventCh <- ev waitForSettings(t, ch) diff --git a/pkg/ccl/kvccl/kvtenantccl/tenant_trace_test.go b/pkg/ccl/kvccl/kvtenantccl/tenant_trace_test.go index 8348afdd88d0..561a0a20923f 100644 --- a/pkg/ccl/kvccl/kvtenantccl/tenant_trace_test.go +++ b/pkg/ccl/kvccl/kvtenantccl/tenant_trace_test.go @@ -14,6 +14,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -58,7 +59,7 @@ func testTenantTracesAreRedactedImpl(t *testing.T, redactable bool) { Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ EvalKnobs: kvserverbase.BatchEvalTestingKnobs{ - TestingEvalFilter: func(args kvserverbase.FilterArgs) *roachpb.Error { + TestingEvalFilter: func(args kvserverbase.FilterArgs) *kvpb.Error { log.Eventf(args.Ctx, "%v", sensitiveString) log.Eventf(args.Ctx, "%v", redact.Safe(visibleString)) return nil diff --git a/pkg/ccl/multitenantccl/tenantcostclient/BUILD.bazel b/pkg/ccl/multitenantccl/tenantcostclient/BUILD.bazel index 3bba37ebac2f..4f27f01e9ff6 100644 --- a/pkg/ccl/multitenantccl/tenantcostclient/BUILD.bazel +++ b/pkg/ccl/multitenantccl/tenantcostclient/BUILD.bazel @@ -14,6 +14,7 @@ go_library( deps = [ "//pkg/base", "//pkg/kv/kvclient/kvtenant", + "//pkg/kv/kvpb", "//pkg/multitenant", "//pkg/multitenant/tenantcostmodel", "//pkg/roachpb", @@ -59,6 +60,7 @@ go_test( "//pkg/jobs/jobstest", "//pkg/keys", "//pkg/kv/kvclient/kvtenant", + "//pkg/kv/kvpb", "//pkg/multitenant", "//pkg/multitenant/multitenantio", "//pkg/multitenant/tenantcostmodel", diff --git a/pkg/ccl/multitenantccl/tenantcostclient/query_ru_estimate_test.go b/pkg/ccl/multitenantccl/tenantcostclient/query_ru_estimate_test.go index d5a4e2176316..aa1fde2644b0 100644 --- a/pkg/ccl/multitenantccl/tenantcostclient/query_ru_estimate_test.go +++ b/pkg/ccl/multitenantccl/tenantcostclient/query_ru_estimate_test.go @@ -21,7 +21,7 @@ import ( _ "github.com/cockroachdb/cockroach/pkg/ccl/kvccl/kvtenantccl" "github.com/cockroachdb/cockroach/pkg/ccl/multitenantccl/tenantcostclient" _ "github.com/cockroachdb/cockroach/pkg/ccl/multitenantccl/tenantcostserver" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/stats" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" @@ -152,7 +152,7 @@ func TestEstimateQueryRUConsumption(t *testing.T) { // tenant_usage table. time.Sleep(time.Second) var consumptionBytes []byte - var consumption roachpb.TenantConsumption + var consumption kvpb.TenantConsumption var tenantRUs float64 rows := sysDB.Query(t, fmt.Sprintf( diff --git a/pkg/ccl/multitenantccl/tenantcostclient/tenant_side.go b/pkg/ccl/multitenantccl/tenantcostclient/tenant_side.go index 6ff3d785a704..6fc67e2de6c7 100644 --- a/pkg/ccl/multitenantccl/tenantcostclient/tenant_side.go +++ b/pkg/ccl/multitenantccl/tenantcostclient/tenant_side.go @@ -14,6 +14,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvtenant" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/multitenant" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcostmodel" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -170,7 +171,7 @@ func newTenantSideCostController( settings: st, tenantID: tenantID, provider: provider, - responseChan: make(chan *roachpb.TokenBucketResponse, 1), + responseChan: make(chan *kvpb.TokenBucketResponse, 1), lowRUNotifyChan: make(chan struct{}, 1), } c.limiter.Init(timeSource, c.lowRUNotifyChan) @@ -264,7 +265,7 @@ type tenantSideCostController struct { // consumption records the amount of resources consumed by the tenant. // It is read and written on multiple goroutines and so must be protected // by a mutex. - consumption roachpb.TenantConsumption + consumption kvpb.TenantConsumption // avgCPUPerSec is an exponentially-weighted moving average of the CPU usage // per second; used to estimate the CPU usage of a query. It is only written @@ -278,7 +279,7 @@ type tenantSideCostController struct { // responseChan is used to receive results from token bucket requests, which // are run in a separate goroutine. A nil response indicates an error. - responseChan chan *roachpb.TokenBucketResponse + responseChan chan *kvpb.TokenBucketResponse // run contains the state that is updated by the main loop. It doesn't need a // mutex since the main loop runs on a single goroutine. @@ -289,7 +290,7 @@ type tenantSideCostController struct { // externalUsage stores the last value returned by externalUsageFn. externalUsage multitenant.ExternalUsage // consumption stores the last value of mu.consumption. - consumption roachpb.TenantConsumption + consumption kvpb.TenantConsumption // targetPeriod stores the value of the TargetPeriodSetting setting at the // last update. targetPeriod time.Duration @@ -303,7 +304,7 @@ type tenantSideCostController struct { // requestInProgress is the token bucket request that is in progress, or // nil if there is no call in progress. It gets set to nil when we process // the response (in the main loop), even in error cases. - requestInProgress *roachpb.TokenBucketRequest + requestInProgress *kvpb.TokenBucketRequest // shouldSendRequest is set if the last token bucket request encountered an // error. This triggers a retry attempt on the next tick. // @@ -316,7 +317,7 @@ type tenantSideCostController struct { lastRequestTime time.Time // lastReportedConsumption is the set of tenant resource consumption // metrics last sent to the token bucket server. - lastReportedConsumption roachpb.TenantConsumption + lastReportedConsumption kvpb.TenantConsumption // lastRate is the token bucket fill rate that was last configured. lastRate float64 @@ -523,7 +524,7 @@ func (c *tenantSideCostController) sendTokenBucketRequest(ctx context.Context) { } } - req := &roachpb.TokenBucketRequest{ + req := &kvpb.TokenBucketRequest{ TenantID: c.tenantID.ToUint64(), InstanceID: uint32(c.instanceID), InstanceLease: c.sessionID.UnsafeBytes(), @@ -567,7 +568,7 @@ func (c *tenantSideCostController) sendTokenBucketRequest(ctx context.Context) { } func (c *tenantSideCostController) handleTokenBucketResponse( - ctx context.Context, req *roachpb.TokenBucketRequest, resp *roachpb.TokenBucketResponse, + ctx context.Context, req *kvpb.TokenBucketRequest, resp *kvpb.TokenBucketResponse, ) { if log.ExpensiveLogEnabled(ctx, 1) { log.Infof( @@ -793,7 +794,7 @@ func (c *tenantSideCostController) OnResponseWait( if multitenant.TenantRUEstimateEnabled.Get(&c.settings.SV) { if sp := tracing.SpanFromContext(ctx); sp != nil && sp.RecordingType() != tracingpb.RecordingOff { - sp.RecordStructured(&roachpb.TenantConsumption{ + sp.RecordStructured(&kvpb.TenantConsumption{ RU: float64(totalRU), }) } diff --git a/pkg/ccl/multitenantccl/tenantcostclient/tenant_side_test.go b/pkg/ccl/multitenantccl/tenantcostclient/tenant_side_test.go index 8924be5bab18..a64d2ff23836 100644 --- a/pkg/ccl/multitenantccl/tenantcostclient/tenant_side_test.go +++ b/pkg/ccl/multitenantccl/tenantcostclient/tenant_side_test.go @@ -34,6 +34,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobstest" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvtenant" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/multitenant" "github.com/cockroachdb/cockroach/pkg/multitenant/multitenantio" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcostmodel" @@ -632,7 +633,7 @@ func (ew *eventWaiter) WaitForEvent(typ tenantcostclient.TestEventType) bool { type testProvider struct { mu struct { syncutil.Mutex - consumption roachpb.TenantConsumption + consumption kvpb.TenantConsumption lastSeqNum int64 @@ -685,7 +686,7 @@ func (tp *testProvider) waitForRequest(t *testing.T) { } } -func (tp *testProvider) consumption() roachpb.TenantConsumption { +func (tp *testProvider) consumption() kvpb.TenantConsumption { tp.mu.Lock() defer tp.mu.Unlock() return tp.mu.consumption @@ -693,7 +694,7 @@ func (tp *testProvider) consumption() roachpb.TenantConsumption { // waitForConsumption waits for the next TokenBucket request and returns the // total consumption. -func (tp *testProvider) waitForConsumption(t *testing.T) roachpb.TenantConsumption { +func (tp *testProvider) waitForConsumption(t *testing.T) kvpb.TenantConsumption { tp.waitForRequest(t) // it is possible that the TokenBucket request was in the process of being // prepared; we have to wait for another one to make sure the latest @@ -717,8 +718,8 @@ func (tp *testProvider) unblockRequest(t *testing.T) { // TokenBucket implements the kvtenant.TokenBucketProvider interface. func (tp *testProvider) TokenBucket( - _ context.Context, in *roachpb.TokenBucketRequest, -) (*roachpb.TokenBucketResponse, error) { + _ context.Context, in *kvpb.TokenBucketRequest, +) (*kvpb.TokenBucketResponse, error) { tp.mu.Lock() defer tp.mu.Unlock() select { @@ -744,7 +745,7 @@ func (tp *testProvider) TokenBucket( } tp.mu.consumption.Add(&in.ConsumptionSinceLastRequest) - res := &roachpb.TokenBucketResponse{} + res := &kvpb.TokenBucketResponse{} rate := tp.mu.cfg.Throttle if rate >= 0 { diff --git a/pkg/ccl/multitenantccl/tenantcostserver/BUILD.bazel b/pkg/ccl/multitenantccl/tenantcostserver/BUILD.bazel index 32655f8f15fa..3c70626bf569 100644 --- a/pkg/ccl/multitenantccl/tenantcostserver/BUILD.bazel +++ b/pkg/ccl/multitenantccl/tenantcostserver/BUILD.bazel @@ -16,6 +16,7 @@ go_library( "//pkg/base", "//pkg/ccl/multitenantccl/tenantcostserver/tenanttokenbucket", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/multitenant", "//pkg/roachpb", "//pkg/server", @@ -49,6 +50,7 @@ go_test( deps = [ "//pkg/base", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/multitenant", "//pkg/roachpb", "//pkg/security/securityassets", diff --git a/pkg/ccl/multitenantccl/tenantcostserver/server_test.go b/pkg/ccl/multitenantccl/tenantcostserver/server_test.go index 241cd4b29f98..9f9a2afd5233 100644 --- a/pkg/ccl/multitenantccl/tenantcostserver/server_test.go +++ b/pkg/ccl/multitenantccl/tenantcostserver/server_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/ccl/multitenantccl/tenantcostserver" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/multitenant" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/isql" @@ -172,13 +173,13 @@ func (ts *testState) tokenBucketRequest(t *testing.T, d *datadriven.TestData) st if err != nil { d.Fatalf(t, "failed to parse duration: %v", args.Period) } - req := roachpb.TokenBucketRequest{ + req := kvpb.TokenBucketRequest{ TenantID: tenantID, InstanceID: args.InstanceID, InstanceLease: []byte(args.InstanceLease), NextLiveInstanceID: args.NextLiveInstanceID, SeqNum: args.SeqNum, - ConsumptionSinceLastRequest: roachpb.TenantConsumption{ + ConsumptionSinceLastRequest: kvpb.TenantConsumption{ RU: args.Consumption.RU, KVRU: args.Consumption.KVRU, ReadBatches: args.Consumption.ReadBatches, @@ -352,7 +353,7 @@ func TestInstanceCleanup(t *testing.T) { // Send one token bucket update from each instance, in random order. instances := liveset.Ordered() for _, i := range rand.Perm(len(instances)) { - req := roachpb.TokenBucketRequest{ + req := kvpb.TokenBucketRequest{ TenantID: 5, InstanceID: uint32(instances[i]), } diff --git a/pkg/ccl/multitenantccl/tenantcostserver/system_table.go b/pkg/ccl/multitenantccl/tenantcostserver/system_table.go index 729a4ba84127..18b01d32bd4d 100644 --- a/pkg/ccl/multitenantccl/tenantcostserver/system_table.go +++ b/pkg/ccl/multitenantccl/tenantcostserver/system_table.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/ccl/multitenantccl/tenantcostserver/tenanttokenbucket" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/isql" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -43,7 +44,7 @@ type tenantState struct { Bucket tenanttokenbucket.State // Current consumption information. - Consumption roachpb.TenantConsumption + Consumption kvpb.TenantConsumption } // defaultRefillRate is the default refill rate if it is never configured (via diff --git a/pkg/ccl/multitenantccl/tenantcostserver/tenanttokenbucket/BUILD.bazel b/pkg/ccl/multitenantccl/tenantcostserver/tenanttokenbucket/BUILD.bazel index 264df69d9564..807116d1270a 100644 --- a/pkg/ccl/multitenantccl/tenantcostserver/tenanttokenbucket/BUILD.bazel +++ b/pkg/ccl/multitenantccl/tenantcostserver/tenanttokenbucket/BUILD.bazel @@ -7,6 +7,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/ccl/multitenantccl/tenantcostserver/tenanttokenbucket", visibility = ["//visibility:public"], deps = [ + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/util/log", ], @@ -19,7 +20,7 @@ go_test( data = glob(["testdata/**"]), embed = [":tenanttokenbucket"], deps = [ - "//pkg/roachpb", + "//pkg/kv/kvpb", "//pkg/testutils/datapathutils", "//pkg/util/leaktest", "@com_github_cockroachdb_datadriven//:datadriven", diff --git a/pkg/ccl/multitenantccl/tenantcostserver/tenanttokenbucket/tenant_token_bucket.go b/pkg/ccl/multitenantccl/tenantcostserver/tenanttokenbucket/tenant_token_bucket.go index d2b7dc401f14..d8755533b968 100644 --- a/pkg/ccl/multitenantccl/tenantcostserver/tenanttokenbucket/tenant_token_bucket.go +++ b/pkg/ccl/multitenantccl/tenantcostserver/tenanttokenbucket/tenant_token_bucket.go @@ -16,6 +16,7 @@ import ( "math" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/log" ) @@ -56,9 +57,9 @@ func (s *State) Update(since time.Duration) { // Request processes a request for more tokens and updates the State // accordingly. func (s *State) Request( - ctx context.Context, req *roachpb.TokenBucketRequest, -) roachpb.TokenBucketResponse { - var res roachpb.TokenBucketResponse + ctx context.Context, req *kvpb.TokenBucketRequest, +) kvpb.TokenBucketResponse { + var res kvpb.TokenBucketResponse // Calculate the fallback rate. res.FallbackRate = s.RURefillRate diff --git a/pkg/ccl/multitenantccl/tenantcostserver/tenanttokenbucket/tenant_token_bucket_test.go b/pkg/ccl/multitenantccl/tenantcostserver/tenanttokenbucket/tenant_token_bucket_test.go index 690d58b09a71..98949a30560e 100644 --- a/pkg/ccl/multitenantccl/tenantcostserver/tenanttokenbucket/tenant_token_bucket_test.go +++ b/pkg/ccl/multitenantccl/tenantcostserver/tenanttokenbucket/tenant_token_bucket_test.go @@ -15,7 +15,7 @@ import ( "testing" "time" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/testutils/datapathutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/datadriven" @@ -92,7 +92,7 @@ func (ts *testState) request(t *testing.T, d *datadriven.TestData) string { if err := yaml.UnmarshalStrict([]byte(d.Input), &vals); err != nil { d.Fatalf(t, "failed to unmarshal init values: %v", err) } - req := roachpb.TokenBucketRequest{ + req := kvpb.TokenBucketRequest{ RequestedRU: vals.RU, TargetRequestPeriod: parseDuration(t, d, vals.Period), } diff --git a/pkg/ccl/multitenantccl/tenantcostserver/token_bucket.go b/pkg/ccl/multitenantccl/tenantcostserver/token_bucket.go index 925ed03553c0..d679a1672d30 100644 --- a/pkg/ccl/multitenantccl/tenantcostserver/token_bucket.go +++ b/pkg/ccl/multitenantccl/tenantcostserver/token_bucket.go @@ -12,6 +12,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/isql" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -23,21 +24,21 @@ import ( // runs on the host cluster to service requests coming from tenants (through the // kvtenant.Connector). func (s *instance) TokenBucketRequest( - ctx context.Context, tenantID roachpb.TenantID, in *roachpb.TokenBucketRequest, -) *roachpb.TokenBucketResponse { + ctx context.Context, tenantID roachpb.TenantID, in *kvpb.TokenBucketRequest, +) *kvpb.TokenBucketResponse { if tenantID == roachpb.SystemTenantID { - return &roachpb.TokenBucketResponse{ + return &kvpb.TokenBucketResponse{ Error: errors.EncodeError(ctx, errors.New("token bucket request for system tenant")), } } instanceID := base.SQLInstanceID(in.InstanceID) if instanceID < 1 { - return &roachpb.TokenBucketResponse{ + return &kvpb.TokenBucketResponse{ Error: errors.EncodeError(ctx, errors.Errorf("invalid instance ID %d", instanceID)), } } if in.RequestedRU < 0 { - return &roachpb.TokenBucketResponse{ + return &kvpb.TokenBucketResponse{ Error: errors.EncodeError(ctx, errors.Errorf("negative requested RUs")), } } @@ -50,10 +51,10 @@ func (s *instance) TokenBucketRequest( metrics.mutex.Lock() defer metrics.mutex.Unlock() - result := &roachpb.TokenBucketResponse{} - var consumption roachpb.TenantConsumption + result := &kvpb.TokenBucketResponse{} + var consumption kvpb.TenantConsumption if err := s.ief.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { - *result = roachpb.TokenBucketResponse{} + *result = kvpb.TokenBucketResponse{} h := makeSysTableHelper(ctx, tenantID) tenant, instance, err := h.readTenantAndInstanceState(txn, instanceID) @@ -116,7 +117,7 @@ func (s *instance) TokenBucketRequest( consumption = tenant.Consumption return nil }); err != nil { - return &roachpb.TokenBucketResponse{ + return &kvpb.TokenBucketResponse{ Error: errors.EncodeError(ctx, err), } } diff --git a/pkg/ccl/storageccl/BUILD.bazel b/pkg/ccl/storageccl/BUILD.bazel index 6fea79cb3fd3..e5a9121006ec 100644 --- a/pkg/ccl/storageccl/BUILD.bazel +++ b/pkg/ccl/storageccl/BUILD.bazel @@ -12,7 +12,7 @@ go_library( deps = [ "//pkg/base", "//pkg/cloud", - "//pkg/roachpb", + "//pkg/kv/kvpb", "//pkg/settings", "//pkg/storage", "//pkg/util/ioctx", diff --git a/pkg/ccl/storageccl/external_sst_reader.go b/pkg/ccl/storageccl/external_sst_reader.go index c8296f9dd093..0c9ca9d09810 100644 --- a/pkg/ccl/storageccl/external_sst_reader.go +++ b/pkg/ccl/storageccl/external_sst_reader.go @@ -16,7 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/cloud" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/ioctx" @@ -73,7 +73,7 @@ type StoreFile struct { func newMemPebbleSSTReader( ctx context.Context, storeFiles []StoreFile, - encryption *roachpb.FileEncryptionOptions, + encryption *kvpb.FileEncryptionOptions, iterOps storage.IterOptions, ) (storage.SimpleMVCCIterator, error) { @@ -112,7 +112,7 @@ func newMemPebbleSSTReader( func ExternalSSTReader( ctx context.Context, storeFiles []StoreFile, - encryption *roachpb.FileEncryptionOptions, + encryption *kvpb.FileEncryptionOptions, iterOpts storage.IterOptions, ) (storage.SimpleMVCCIterator, error) { // TODO(jackson): Change the interface to accept a two-dimensional diff --git a/pkg/ccl/streamingccl/BUILD.bazel b/pkg/ccl/streamingccl/BUILD.bazel index 2d2b85135ef2..5adcdfec3bff 100644 --- a/pkg/ccl/streamingccl/BUILD.bazel +++ b/pkg/ccl/streamingccl/BUILD.bazel @@ -13,6 +13,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/jobs/jobspb", + "//pkg/kv/kvpb", "//pkg/repstream/streampb", "//pkg/roachpb", "//pkg/settings", diff --git a/pkg/ccl/streamingccl/event.go b/pkg/ccl/streamingccl/event.go index 4491b7cad25f..033ccc9eed95 100644 --- a/pkg/ccl/streamingccl/event.go +++ b/pkg/ccl/streamingccl/event.go @@ -10,6 +10,7 @@ package streamingccl import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" ) @@ -43,10 +44,10 @@ type Event interface { GetKV() *roachpb.KeyValue // GetSSTable returns a AddSSTable event if the EventType is SSTableEvent. - GetSSTable() *roachpb.RangeFeedSSTable + GetSSTable() *kvpb.RangeFeedSSTable // GetDeleteRange returns a DeleteRange event if the EventType is DeleteRangeEvent. - GetDeleteRange() *roachpb.RangeFeedDeleteRange + GetDeleteRange() *kvpb.RangeFeedDeleteRange // GetResolvedSpans returns a list of span-time pairs indicating the time for // which all KV events within that span has been emitted. @@ -71,12 +72,12 @@ func (kve kvEvent) GetKV() *roachpb.KeyValue { } // GetSSTable implements the Event interface. -func (kve kvEvent) GetSSTable() *roachpb.RangeFeedSSTable { +func (kve kvEvent) GetSSTable() *kvpb.RangeFeedSSTable { return nil } // GetDeleteRange implements the Event interface. -func (kve kvEvent) GetDeleteRange() *roachpb.RangeFeedDeleteRange { +func (kve kvEvent) GetDeleteRange() *kvpb.RangeFeedDeleteRange { return nil } @@ -87,7 +88,7 @@ func (kve kvEvent) GetResolvedSpans() []jobspb.ResolvedSpan { // sstableEvent is a sstable that needs to be ingested. type sstableEvent struct { - sst roachpb.RangeFeedSSTable + sst kvpb.RangeFeedSSTable } // Type implements the Event interface. @@ -101,12 +102,12 @@ func (sste sstableEvent) GetKV() *roachpb.KeyValue { } // GetSSTable implements the Event interface. -func (sste sstableEvent) GetSSTable() *roachpb.RangeFeedSSTable { +func (sste sstableEvent) GetSSTable() *kvpb.RangeFeedSSTable { return &sste.sst } // GetDeleteRange implements the Event interface. -func (sste sstableEvent) GetDeleteRange() *roachpb.RangeFeedDeleteRange { +func (sste sstableEvent) GetDeleteRange() *kvpb.RangeFeedDeleteRange { return nil } @@ -119,7 +120,7 @@ var _ Event = sstableEvent{} // delRangeEvent is a DeleteRange event that needs to be ingested. type delRangeEvent struct { - delRange roachpb.RangeFeedDeleteRange + delRange kvpb.RangeFeedDeleteRange } // Type implements the Event interface. @@ -133,12 +134,12 @@ func (dre delRangeEvent) GetKV() *roachpb.KeyValue { } // GetSSTable implements the Event interface. -func (dre delRangeEvent) GetSSTable() *roachpb.RangeFeedSSTable { +func (dre delRangeEvent) GetSSTable() *kvpb.RangeFeedSSTable { return nil } // GetDeleteRange implements the Event interface. -func (dre delRangeEvent) GetDeleteRange() *roachpb.RangeFeedDeleteRange { +func (dre delRangeEvent) GetDeleteRange() *kvpb.RangeFeedDeleteRange { return &dre.delRange } @@ -168,12 +169,12 @@ func (ce checkpointEvent) GetKV() *roachpb.KeyValue { } // GetSSTable implements the Event interface. -func (ce checkpointEvent) GetSSTable() *roachpb.RangeFeedSSTable { +func (ce checkpointEvent) GetSSTable() *kvpb.RangeFeedSSTable { return nil } // GetDeleteRange implements the Event interface. -func (ce checkpointEvent) GetDeleteRange() *roachpb.RangeFeedDeleteRange { +func (ce checkpointEvent) GetDeleteRange() *kvpb.RangeFeedDeleteRange { return nil } @@ -188,12 +189,12 @@ func MakeKVEvent(kv roachpb.KeyValue) Event { } // MakeSSTableEvent creates an Event from a SSTable. -func MakeSSTableEvent(sst roachpb.RangeFeedSSTable) Event { +func MakeSSTableEvent(sst kvpb.RangeFeedSSTable) Event { return sstableEvent{sst: sst} } // MakeDeleteRangeEvent creates an Event from a DeleteRange. -func MakeDeleteRangeEvent(delRange roachpb.RangeFeedDeleteRange) Event { +func MakeDeleteRangeEvent(delRange kvpb.RangeFeedDeleteRange) Event { return delRangeEvent{delRange: delRange} } diff --git a/pkg/ccl/streamingccl/replicationutils/BUILD.bazel b/pkg/ccl/streamingccl/replicationutils/BUILD.bazel index db2b9d9b0560..7e2c1f04dd88 100644 --- a/pkg/ccl/streamingccl/replicationutils/BUILD.bazel +++ b/pkg/ccl/streamingccl/replicationutils/BUILD.bazel @@ -9,6 +9,7 @@ go_library( deps = [ "//pkg/ccl/streamingccl/streamclient", "//pkg/jobs/jobspb", + "//pkg/kv/kvpb", "//pkg/repstream/streampb", "//pkg/roachpb", "//pkg/storage", @@ -28,6 +29,7 @@ go_test( embed = [":replicationutils"], deps = [ "//pkg/clusterversion", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/settings/cluster", "//pkg/storage", diff --git a/pkg/ccl/streamingccl/replicationutils/utils.go b/pkg/ccl/streamingccl/replicationutils/utils.go index d390ad2ba610..c6b75443ad5a 100644 --- a/pkg/ccl/streamingccl/replicationutils/utils.go +++ b/pkg/ccl/streamingccl/replicationutils/utils.go @@ -14,6 +14,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/streamclient" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/repstream/streampb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" @@ -29,7 +30,7 @@ import ( // 'scanWithin' boundaries and execute given operations on each // emitted MVCCKeyValue and MVCCRangeKeyValue. func ScanSST( - sst *roachpb.RangeFeedSSTable, + sst *kvpb.RangeFeedSSTable, scanWithin roachpb.Span, mvccKeyValOp func(key storage.MVCCKeyValue) error, mvccRangeKeyValOp func(rangeKeyVal storage.MVCCRangeKeyValue) error, @@ -37,7 +38,7 @@ func ScanSST( rangeKVs := make([]*storage.MVCCRangeKeyValue, 0) timestampToRangeKey := make(map[hlc.Timestamp]*storage.MVCCRangeKeyValue) // Iterator may release fragmented ranges, we try to de-fragment them - // before we release roachpb.RangeFeedDeleteRange events. + // before we release kvpb.RangeFeedDeleteRange events. mergeRangeKV := func(rangeKV storage.MVCCRangeKeyValue) { // Range keys are emitted with increasing order in terms of start key, // so we only need to check if the current range key can be concatenated behind diff --git a/pkg/ccl/streamingccl/replicationutils/utils_test.go b/pkg/ccl/streamingccl/replicationutils/utils_test.go index d799895aad8c..222ebb4f0d4c 100644 --- a/pkg/ccl/streamingccl/replicationutils/utils_test.go +++ b/pkg/ccl/streamingccl/replicationutils/utils_test.go @@ -13,6 +13,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage" @@ -76,7 +77,7 @@ func TestScanSST(t *testing.T) { ) { actualPointKVs := make([]storage.MVCCKeyValue, 0, len(expectedPointKVs)) actualRangeKVs := make([]storage.MVCCRangeKey, 0, len(expectedRangeKeys)) - require.NoError(t, ScanSST(&roachpb.RangeFeedSSTable{ + require.NoError(t, ScanSST(&kvpb.RangeFeedSSTable{ Data: data, Span: roachpb.Span{Key: start, EndKey: end}, WriteTS: hlc.Timestamp{WallTime: timeutil.Now().UnixNano()}, diff --git a/pkg/ccl/streamingccl/streamclient/BUILD.bazel b/pkg/ccl/streamingccl/streamclient/BUILD.bazel index db8113aa239f..be30bce331c5 100644 --- a/pkg/ccl/streamingccl/streamclient/BUILD.bazel +++ b/pkg/ccl/streamingccl/streamclient/BUILD.bazel @@ -15,6 +15,7 @@ go_library( "//pkg/cloud/externalconn", "//pkg/jobs/jobspb", "//pkg/keys", + "//pkg/kv/kvpb", "//pkg/repstream/streampb", "//pkg/roachpb", "//pkg/security/username", diff --git a/pkg/ccl/streamingccl/streamclient/random_stream_client.go b/pkg/ccl/streamingccl/streamclient/random_stream_client.go index 61e7e3462678..1c5452119e05 100644 --- a/pkg/ccl/streamingccl/streamclient/random_stream_client.go +++ b/pkg/ccl/streamingccl/streamclient/random_stream_client.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/repstream/streampb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" @@ -102,7 +103,7 @@ type HeartbeatInterceptFn func(timestamp hlc.Timestamp) // SSTableMakerFn is a function that generates RangeFeedSSTable event // with a given list of roachpb.KeyValue. -type SSTableMakerFn func(keyValues []roachpb.KeyValue) roachpb.RangeFeedSSTable +type SSTableMakerFn func(keyValues []roachpb.KeyValue) kvpb.RangeFeedSSTable // randomStreamConfig specifies the variables that controls the rate and type of // events that the generated stream emits. @@ -638,7 +639,7 @@ func duplicateEvent(event streamingccl.Event) streamingccl.Event { sst := event.GetSSTable() dataCopy := make([]byte, len(sst.Data)) copy(dataCopy, sst.Data) - dup = streamingccl.MakeSSTableEvent(roachpb.RangeFeedSSTable{ + dup = streamingccl.MakeSSTableEvent(kvpb.RangeFeedSSTable{ Data: dataCopy, Span: sst.Span.Clone(), WriteTS: sst.WriteTS, diff --git a/pkg/ccl/streamingccl/streamingest/BUILD.bazel b/pkg/ccl/streamingccl/streamingest/BUILD.bazel index 6dfc6c78c374..3fa078ba0e60 100644 --- a/pkg/ccl/streamingccl/streamingest/BUILD.bazel +++ b/pkg/ccl/streamingccl/streamingest/BUILD.bazel @@ -31,6 +31,7 @@ go_library( "//pkg/keys", "//pkg/kv", "//pkg/kv/bulk", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/protectedts", "//pkg/kv/kvserver/protectedts/ptpb", "//pkg/multitenant/mtinfopb", @@ -107,6 +108,7 @@ go_test( "//pkg/jobs", "//pkg/jobs/jobspb", "//pkg/keys", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/kv/kvserver/protectedts", "//pkg/multitenant/tenantcapabilities", diff --git a/pkg/ccl/streamingccl/streamingest/replication_random_client_test.go b/pkg/ccl/streamingccl/streamingest/replication_random_client_test.go index 8c2c92e2bf1c..4399bc79e0bf 100644 --- a/pkg/ccl/streamingccl/streamingest/replication_random_client_test.go +++ b/pkg/ccl/streamingccl/streamingest/replication_random_client_test.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" clustersettings "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -70,7 +71,7 @@ func getTestRandomClientURI(tenantID roachpb.TenantID, tenantName roachpb.Tenant dupProbability, tenantID, tenantName) } -func sstMaker(t *testing.T, keyValues []roachpb.KeyValue) roachpb.RangeFeedSSTable { +func sstMaker(t *testing.T, keyValues []roachpb.KeyValue) kvpb.RangeFeedSSTable { sort.Slice(keyValues, func(i, j int) bool { return keyValues[i].Key.Compare(keyValues[j].Key) < 0 }) @@ -89,7 +90,7 @@ func sstMaker(t *testing.T, keyValues []roachpb.KeyValue) roachpb.RangeFeedSSTab }) } data, start, end := storageutils.MakeSST(t, clustersettings.MakeTestingClusterSettings(), kvs) - return roachpb.RangeFeedSSTable{ + return kvpb.RangeFeedSSTable{ Data: data, Span: roachpb.Span{ Key: start, @@ -192,7 +193,7 @@ func TestStreamIngestionJobWithRandomClient(t *testing.T) { client.ClearInterceptors() client.RegisterInterception(completeJobAfterCheckpoints) client.RegisterInterception(validateFnWithValidator(t, streamValidator)) - client.RegisterSSTableGenerator(func(keyValues []roachpb.KeyValue) roachpb.RangeFeedSSTable { + client.RegisterSSTableGenerator(func(keyValues []roachpb.KeyValue) kvpb.RangeFeedSSTable { return sstMaker(t, keyValues) }) @@ -207,10 +208,10 @@ func TestStreamIngestionJobWithRandomClient(t *testing.T) { }, } params.ServerArgs.Knobs.Store = &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(_ context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + TestingRequestFilter: func(_ context.Context, ba *kvpb.BatchRequest) *kvpb.Error { for _, req := range ba.Requests { switch r := req.GetInner().(type) { - case *roachpb.RevertRangeRequest: + case *kvpb.RevertRangeRequest: revertRangeTargetTime = r.TargetTime <-receivedRevertRequest } diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_job.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_job.go index c760aeb0591a..9dbcec1e96cd 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_job.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_job.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/jobs/jobsprotectedts" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" @@ -566,8 +567,8 @@ func maybeRevertToCutoverTimestamp( } var b kv.Batch for _, span := range spans { - b.AddRawRequest(&roachpb.RevertRangeRequest{ - RequestHeader: roachpb.RequestHeader{ + b.AddRawRequest(&kvpb.RevertRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: span.Key, EndKey: span.EndKey, }, diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_job_test.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_job_test.go index 04e54ab6806d..e99c18332749 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_job_test.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_job_test.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/repstream/streampb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -416,10 +417,10 @@ func TestCutoverFractionProgressed(t *testing.T) { s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{ Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ - TestingResponseFilter: func(ctx context.Context, ba *roachpb.BatchRequest, br *roachpb.BatchResponse) *roachpb.Error { + TestingResponseFilter: func(ctx context.Context, ba *kvpb.BatchRequest, br *kvpb.BatchResponse) *kvpb.Error { for _, ru := range br.Responses { switch ru.GetInner().(type) { - case *roachpb.RevertRangeResponse: + case *kvpb.RevertRangeResponse: respRecvd <- struct{}{} <-continueRevert } diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor.go index b516c258e2fe..cc24ff23367e 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/bulk" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/repstream/streampb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" @@ -599,7 +600,7 @@ func (sip *streamIngestionProcessor) consumeEvents() (*jobspb.ResolvedSpans, err return nil, nil } -func (sip *streamIngestionProcessor) bufferSST(sst *roachpb.RangeFeedSSTable) error { +func (sip *streamIngestionProcessor) bufferSST(sst *kvpb.RangeFeedSSTable) error { // TODO(casper): we currently buffer all keys in an SST at once even for large SSTs. // If in the future we decide buffer them in separate batches, we need to be // careful with checkpoints: we can only send checkpoint whose TS >= SST batch TS @@ -632,7 +633,7 @@ func (sip *streamIngestionProcessor) rekey(key roachpb.Key) ([]byte, error) { return rekey, nil } -func (sip *streamIngestionProcessor) bufferDelRange(delRange *roachpb.RangeFeedDeleteRange) error { +func (sip *streamIngestionProcessor) bufferDelRange(delRange *kvpb.RangeFeedDeleteRange) error { tombstoneVal, err := storage.EncodeMVCCValue(storage.MVCCValue{ MVCCValueHeader: enginepb.MVCCValueHeader{ LocalTimestamp: hlc.ClockTimestamp{ diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor_test.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor_test.go index 219a0c0454a6..68b98241553a 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor_test.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor_test.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/repstream/streampb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -487,7 +488,7 @@ func TestRandomClientGeneration(t *testing.T) { streamValidator := newStreamClientValidator(rekeyer) randomStreamClient.ClearInterceptors() - randomStreamClient.RegisterSSTableGenerator(func(keyValues []roachpb.KeyValue) roachpb.RangeFeedSSTable { + randomStreamClient.RegisterSSTableGenerator(func(keyValues []roachpb.KeyValue) kvpb.RangeFeedSSTable { return sstMaker(t, keyValues) }) randomStreamClient.RegisterInterception(cancelAfterCheckpoints) diff --git a/pkg/ccl/streamingccl/streamproducer/BUILD.bazel b/pkg/ccl/streamingccl/streamproducer/BUILD.bazel index ad4e50c74099..68a6bb77da2d 100644 --- a/pkg/ccl/streamingccl/streamproducer/BUILD.bazel +++ b/pkg/ccl/streamingccl/streamproducer/BUILD.bazel @@ -22,6 +22,7 @@ go_library( "//pkg/kv", "//pkg/kv/kvclient/kvcoord", "//pkg/kv/kvclient/rangefeed", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/kv/kvserver/protectedts", "//pkg/kv/kvserver/protectedts/ptpb", @@ -78,6 +79,7 @@ go_test( "//pkg/jobs/jobsprotectedts", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/protectedts", "//pkg/kv/kvserver/protectedts/ptpb", "//pkg/repstream/streampb", diff --git a/pkg/ccl/streamingccl/streamproducer/event_stream.go b/pkg/ccl/streamingccl/streamproducer/event_stream.go index e76de613640d..5ae42350f172 100644 --- a/pkg/ccl/streamingccl/streamproducer/event_stream.go +++ b/pkg/ccl/streamingccl/streamproducer/event_stream.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/repstream/streampb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql" @@ -226,24 +227,24 @@ func (s *eventStream) Close(ctx context.Context) { s.sp.Finish() } -func (s *eventStream) onValue(ctx context.Context, value *roachpb.RangeFeedValue) { +func (s *eventStream) onValue(ctx context.Context, value *kvpb.RangeFeedValue) { select { case <-ctx.Done(): - case s.eventsCh <- kvcoord.RangeFeedMessage{RangeFeedEvent: &roachpb.RangeFeedEvent{Val: value}}: + case s.eventsCh <- kvcoord.RangeFeedMessage{RangeFeedEvent: &kvpb.RangeFeedEvent{Val: value}}: log.VInfof(ctx, 1, "onValue: %s@%s", value.Key, value.Value.Timestamp) } } -func (s *eventStream) onCheckpoint(ctx context.Context, checkpoint *roachpb.RangeFeedCheckpoint) { +func (s *eventStream) onCheckpoint(ctx context.Context, checkpoint *kvpb.RangeFeedCheckpoint) { select { case <-ctx.Done(): - case s.eventsCh <- kvcoord.RangeFeedMessage{RangeFeedEvent: &roachpb.RangeFeedEvent{Checkpoint: checkpoint}}: + case s.eventsCh <- kvcoord.RangeFeedMessage{RangeFeedEvent: &kvpb.RangeFeedEvent{Checkpoint: checkpoint}}: log.VInfof(ctx, 1, "onCheckpoint: %s@%s", checkpoint.Span, checkpoint.ResolvedTS) } } func (s *eventStream) onInitialScanSpanCompleted(ctx context.Context, sp roachpb.Span) error { - checkpoint := roachpb.RangeFeedCheckpoint{ + checkpoint := kvpb.RangeFeedCheckpoint{ Span: sp, ResolvedTS: s.spec.InitialScanTimestamp, } @@ -251,7 +252,7 @@ func (s *eventStream) onInitialScanSpanCompleted(ctx context.Context, sp roachpb case <-ctx.Done(): return ctx.Err() case s.eventsCh <- kvcoord.RangeFeedMessage{ - RangeFeedEvent: &roachpb.RangeFeedEvent{Checkpoint: &checkpoint}, + RangeFeedEvent: &kvpb.RangeFeedEvent{Checkpoint: &checkpoint}, }: log.VInfof(ctx, 1, "onSpanCompleted: %s@%s", checkpoint.Span, checkpoint.ResolvedTS) return nil @@ -259,12 +260,12 @@ func (s *eventStream) onInitialScanSpanCompleted(ctx context.Context, sp roachpb } func (s *eventStream) onSSTable( - ctx context.Context, sst *roachpb.RangeFeedSSTable, registeredSpan roachpb.Span, + ctx context.Context, sst *kvpb.RangeFeedSSTable, registeredSpan roachpb.Span, ) { select { case <-ctx.Done(): case s.eventsCh <- kvcoord.RangeFeedMessage{ - RangeFeedEvent: &roachpb.RangeFeedEvent{SST: sst}, + RangeFeedEvent: &kvpb.RangeFeedEvent{SST: sst}, RegisteredSpan: registeredSpan, }: log.VInfof(ctx, 1, "onSSTable: %s@%s with registered span %s", @@ -272,10 +273,10 @@ func (s *eventStream) onSSTable( } } -func (s *eventStream) onDeleteRange(ctx context.Context, delRange *roachpb.RangeFeedDeleteRange) { +func (s *eventStream) onDeleteRange(ctx context.Context, delRange *kvpb.RangeFeedDeleteRange) { select { case <-ctx.Done(): - case s.eventsCh <- kvcoord.RangeFeedMessage{RangeFeedEvent: &roachpb.RangeFeedEvent{DeleteRange: delRange}}: + case s.eventsCh <- kvcoord.RangeFeedMessage{RangeFeedEvent: &kvpb.RangeFeedEvent{DeleteRange: delRange}}: log.VInfof(ctx, 1, "onDeleteRange: %s@%s", delRange.Span, delRange.Timestamp) } } @@ -356,7 +357,7 @@ func (p *checkpointPacer) shouldCheckpoint( // Add a RangeFeedSSTable into current batch and return number of bytes added. func (s *eventStream) addSST( - sst *roachpb.RangeFeedSSTable, registeredSpan roachpb.Span, batch *streampb.StreamEvent_Batch, + sst *kvpb.RangeFeedSSTable, registeredSpan roachpb.Span, batch *streampb.StreamEvent_Batch, ) (int, error) { // We send over the whole SSTable if the sst span is within // the registered span boundaries. @@ -384,7 +385,7 @@ func (s *eventStream) addSST( size += batch.KeyValues[len(batch.KeyValues)-1].Size() return nil }, func(rangeKeyVal storage.MVCCRangeKeyValue) error { - batch.DelRanges = append(batch.DelRanges, roachpb.RangeFeedDeleteRange{ + batch.DelRanges = append(batch.DelRanges, kvpb.RangeFeedDeleteRange{ Span: roachpb.Span{ Key: rangeKeyVal.RangeKey.StartKey, EndKey: rangeKeyVal.RangeKey.EndKey, @@ -406,7 +407,7 @@ func (s *eventStream) streamLoop(ctx context.Context, frontier *span.Frontier) e var batch streampb.StreamEvent_Batch batchSize := 0 - addValue := func(v *roachpb.RangeFeedValue) { + addValue := func(v *kvpb.RangeFeedValue) { keyValue := roachpb.KeyValue{ Key: v.Key, Value: v.Value, @@ -415,7 +416,7 @@ func (s *eventStream) streamLoop(ctx context.Context, frontier *span.Frontier) e batchSize += keyValue.Size() } - addDelRange := func(delRange *roachpb.RangeFeedDeleteRange) error { + addDelRange := func(delRange *kvpb.RangeFeedDeleteRange) error { // DelRange's span is already trimmed to enclosed within // the subscribed span, just emit it. batch.DelRanges = append(batch.DelRanges, *delRange) diff --git a/pkg/ccl/streamingccl/streamproducer/replication_stream_test.go b/pkg/ccl/streamingccl/streamproducer/replication_stream_test.go index c416b85911b9..69557138c390 100644 --- a/pkg/ccl/streamingccl/streamproducer/replication_stream_test.go +++ b/pkg/ccl/streamingccl/streamproducer/replication_stream_test.go @@ -28,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" "github.com/cockroachdb/cockroach/pkg/repstream/streampb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -364,8 +365,8 @@ USE d; // Send a ClearRange to trigger rows cursor to return internal error from rangefeed. // Choose 't2' so that it doesn't trigger error on other registered span in rangefeeds, // affecting other tests. - _, err := kv.SendWrapped(ctx, h.SysServer.DB().NonTransactionalSender(), &roachpb.ClearRangeRequest{ - RequestHeader: roachpb.RequestHeader{ + _, err := kv.SendWrapped(ctx, h.SysServer.DB().NonTransactionalSender(), &kvpb.ClearRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: subscribedSpan.Key, EndKey: subscribedSpan.EndKey, }, @@ -631,7 +632,7 @@ func TestCompleteStreamReplication(t *testing.T) { } } -func sortDelRanges(receivedDelRanges []roachpb.RangeFeedDeleteRange) { +func sortDelRanges(receivedDelRanges []kvpb.RangeFeedDeleteRange) { sort.Slice(receivedDelRanges, func(i, j int) bool { if !receivedDelRanges[i].Timestamp.Equal(receivedDelRanges[j].Timestamp) { return receivedDelRanges[i].Timestamp.Compare(receivedDelRanges[j].Timestamp) < 0 @@ -699,7 +700,7 @@ USE d; expectedDelRangeSpan3 := roachpb.Span{Key: t2Span.Key, EndKey: t2Span.Key.Next()} codec := source.mu.codec.(*partitionStreamDecoder) - receivedDelRanges := make([]roachpb.RangeFeedDeleteRange, 0, 3) + receivedDelRanges := make([]kvpb.RangeFeedDeleteRange, 0, 3) for { source.mu.Lock() require.True(t, source.mu.rows.Next()) @@ -733,8 +734,8 @@ USE d; // Delete range for t3s - t3e, emitting nothing. storageutils.RangeKV(string(t3Span.Key), string(t3Span.EndKey), ts, ""), }) - expectedDelRange1 := roachpb.RangeFeedDeleteRange{Span: t1Span, Timestamp: batchHLCTime} - expectedDelRange2 := roachpb.RangeFeedDeleteRange{Span: t2Span, Timestamp: batchHLCTime} + expectedDelRange1 := kvpb.RangeFeedDeleteRange{Span: t1Span, Timestamp: batchHLCTime} + expectedDelRange2 := kvpb.RangeFeedDeleteRange{Span: t2Span, Timestamp: batchHLCTime} require.Equal(t, t1Span.Key, start) require.Equal(t, t3Span.EndKey, end) diff --git a/pkg/cli/BUILD.bazel b/pkg/cli/BUILD.bazel index ebf97493d560..38bef575ba85 100644 --- a/pkg/cli/BUILD.bazel +++ b/pkg/cli/BUILD.bazel @@ -125,6 +125,7 @@ go_library( "//pkg/jobs", "//pkg/jobs/jobspb", "//pkg/keys", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/kv/kvserver/gc", "//pkg/kv/kvserver/kvstorage", @@ -362,6 +363,7 @@ go_test( "//pkg/jobs/jobspb", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/kv/kvserver/allocator/storepool", "//pkg/kv/kvserver/liveness/livenesspb", diff --git a/pkg/cli/debug_reset_quorum.go b/pkg/cli/debug_reset_quorum.go index 8c0fefe03892..a15c353860dd 100644 --- a/pkg/cli/debug_reset_quorum.go +++ b/pkg/cli/debug_reset_quorum.go @@ -16,7 +16,7 @@ import ( "strconv" "github.com/cockroachdb/cockroach/pkg/cli/clierrorplus" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/spf13/cobra" ) @@ -58,7 +58,7 @@ func runDebugResetQuorum(cmd *cobra.Command, args []string) error { defer finish() // Call ResetQuorum to reset quorum for given range on target node. - _, err = roachpb.NewInternalClient(cc).ResetQuorum(ctx, &roachpb.ResetQuorumRequest{ + _, err = kvpb.NewInternalClient(cc).ResetQuorum(ctx, &kvpb.ResetQuorumRequest{ RangeID: int32(rangeID), }) if err != nil { diff --git a/pkg/cli/debug_send_kv_batch.go b/pkg/cli/debug_send_kv_batch.go index c6d3ca36c9c1..1503587ef3de 100644 --- a/pkg/cli/debug_send_kv_batch.go +++ b/pkg/cli/debug_send_kv_batch.go @@ -17,7 +17,7 @@ import ( "io" "os" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" @@ -57,7 +57,7 @@ Requires the admin role. The request is logged to the system event log. This command can modify internal system state. Incorrect use can cause severe irreversible damage including permanent data loss. -For more information on requests, see roachpb/api.proto. Unknown or invalid +For more information on requests, see kv/kvpb/api.proto. Unknown or invalid fields will error. Binary fields ([]byte) are base64-encoded. Requests spanning multiple ranges are wrapped in a transaction. @@ -89,9 +89,9 @@ This would yield the following response: To generate JSON requests with Go (see also debug_send_kv_batch_test.go): func TestSendKVBatchExample(t *testing.T) { - var ba roachpb.BatchRequest - ba.Add(roachpb.NewPut(roachpb.Key("foo"), roachpb.MakeValueFromString("bar"))) - ba.Add(roachpb.NewGet(roachpb.Key("foo"), false /* forUpdate */)) + var ba kvpb.BatchRequest + ba.Add(kvpb.NewPut(roachpb.Key("foo"), roachpb.MakeValueFromString("bar"))) + ba.Add(kvpb.NewGet(roachpb.Key("foo"), false /* forUpdate */)) jsonpb := protoutil.JSONPb{} jsonProto, err := jsonpb.Marshal(&ba) @@ -159,7 +159,7 @@ func runSendKVBatch(cmd *cobra.Command, args []string) error { return errors.Wrapf(err, "failed to read input") } - var ba roachpb.BatchRequest + var ba kvpb.BatchRequest if err := jsonpb.Unmarshal(baJSON, &ba); err != nil { return errors.Wrap(err, "invalid JSON") } @@ -226,8 +226,8 @@ func runSendKVBatch(cmd *cobra.Command, args []string) error { } func sendKVBatchRequestWithTracingOption( - ctx context.Context, verboseTrace bool, admin serverpb.AdminClient, ba *roachpb.BatchRequest, -) (br *roachpb.BatchResponse, rec tracingpb.Recording, err error) { + ctx context.Context, verboseTrace bool, admin serverpb.AdminClient, ba *kvpb.BatchRequest, +) (br *kvpb.BatchResponse, rec tracingpb.Recording, err error) { var sp *tracing.Span if verboseTrace { // Set up a tracing span and enable verbose tracing if requested by diff --git a/pkg/cli/debug_send_kv_batch_test.go b/pkg/cli/debug_send_kv_batch_test.go index 47554f0ba1e4..9dca99252e9b 100644 --- a/pkg/cli/debug_send_kv_batch_test.go +++ b/pkg/cli/debug_send_kv_batch_test.go @@ -19,6 +19,7 @@ import ( "strings" "testing" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -34,9 +35,9 @@ import ( func TestSendKVBatchExample(t *testing.T) { defer leaktest.AfterTest(t)() - var ba roachpb.BatchRequest - ba.Add(roachpb.NewPut(roachpb.Key("foo"), roachpb.MakeValueFromString("bar"))) - ba.Add(roachpb.NewGet(roachpb.Key("foo"), false /* forUpdate */)) + var ba kvpb.BatchRequest + ba.Add(kvpb.NewPut(roachpb.Key("foo"), roachpb.MakeValueFromString("bar"))) + ba.Add(kvpb.NewGet(roachpb.Key("foo"), false /* forUpdate */)) // NOTE: This cannot be marshaled using the standard Go JSON marshaler, // since it does not correctly (un)marshal the JSON as mandated by the @@ -69,9 +70,9 @@ func TestSendKVBatch(t *testing.T) { // NOTE: This cannot be marshaled using the standard Go JSON marshaler, // since it does not correctly (un)marshal the JSON as mandated by the // Protobuf spec. Instead, use the JSON marshaler shipped with Protobuf. - var ba roachpb.BatchRequest - ba.Add(roachpb.NewPut(roachpb.Key("foo"), roachpb.MakeValueFromString("bar"))) - ba.Add(roachpb.NewGet(roachpb.Key("foo"), false /* forUpdate */)) + var ba kvpb.BatchRequest + ba.Add(kvpb.NewPut(roachpb.Key("foo"), roachpb.MakeValueFromString("bar"))) + ba.Add(kvpb.NewGet(roachpb.Key("foo"), false /* forUpdate */)) jsonpb := protoutil.JSONPb{} jsonProto, err := jsonpb.Marshal(&ba) @@ -140,7 +141,7 @@ func TestSendKVBatch(t *testing.T) { // Check that the log entry contains the BatchRequest as JSON, following // a Protobuf marshaling roundtrip (for normalization). - var ba roachpb.BatchRequest + var ba kvpb.BatchRequest require.NoError(t, jsonpb.Unmarshal(jsonRequest, &ba)) expectLogJSON, err := jsonpb.Marshal(&ba) require.NoError(t, err) diff --git a/pkg/cli/zip_test.go b/pkg/cli/zip_test.go index 7daf8baa1a09..3e98d80fd962 100644 --- a/pkg/cli/zip_test.go +++ b/pkg/cli/zip_test.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -273,7 +274,7 @@ func TestUnavailableZip(t *testing.T) { close(closedCh) unavailableCh.Store(closedCh) knobs := &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(ctx context.Context, _ *roachpb.BatchRequest) *roachpb.Error { + TestingRequestFilter: func(ctx context.Context, _ *kvpb.BatchRequest) *kvpb.Error { select { case <-unavailableCh.Load().(chan struct{}): case <-ctx.Done(): diff --git a/pkg/cmd/roachtest/tests/BUILD.bazel b/pkg/cmd/roachtest/tests/BUILD.bazel index 9f1b01edb86d..444c541f4d65 100644 --- a/pkg/cmd/roachtest/tests/BUILD.bazel +++ b/pkg/cmd/roachtest/tests/BUILD.bazel @@ -196,6 +196,7 @@ go_library( "//pkg/jobs", "//pkg/jobs/jobspb", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/multitenant/mtinfopb", "//pkg/roachpb", "//pkg/roachprod", diff --git a/pkg/cmd/roachtest/tests/mvcc_gc.go b/pkg/cmd/roachtest/tests/mvcc_gc.go index a3c8eb33d2df..f2a129703fd4 100644 --- a/pkg/cmd/roachtest/tests/mvcc_gc.go +++ b/pkg/cmd/roachtest/tests/mvcc_gc.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/roachprod/install" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -254,7 +255,7 @@ func checkRangesConsistentAndHaveNoData( return errors.Errorf("table ranges contain live data %s", totals.String()) } for id, d := range details { - if d.status != roachpb.CheckConsistencyResponse_RANGE_CONSISTENT.String() { + if d.status != kvpb.CheckConsistencyResponse_RANGE_CONSISTENT.String() { return errors.Errorf("consistency check failed for r%d: %s detail: %s", id, d.status, d.detail) } @@ -458,7 +459,7 @@ func deleteAllTableDataWithOverlappingTombstones( return encodeKey(right) } - var ba roachpb.BatchRequest + var ba kvpb.BatchRequest for i := 0; i < fragments; i++ { startKey := leftBound(i) endKey := rightBound(i) @@ -492,7 +493,7 @@ func deleteSomeTableDataWithOverlappingTombstones( return key } - var ba roachpb.BatchRequest + var ba kvpb.BatchRequest for i := 0; i < rangeKeys; i++ { startPK := math.MinInt64 + int64(rng.Uint64()) endPK := math.MinInt64 + int64(rng.Uint64()) @@ -534,9 +535,9 @@ func queryRowOrFatal( } } -func addDeleteRangeUsingTombstone(ba *roachpb.BatchRequest, startKey, endKey roachpb.Key) { - r := roachpb.DeleteRangeRequest{ - RequestHeader: roachpb.RequestHeader{ +func addDeleteRangeUsingTombstone(ba *kvpb.BatchRequest, startKey, endKey roachpb.Key) { + r := kvpb.DeleteRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: startKey, EndKey: endKey, }, @@ -546,31 +547,31 @@ func addDeleteRangeUsingTombstone(ba *roachpb.BatchRequest, startKey, endKey roa } func sendBatchRequest( - ctx context.Context, t test.Test, c cluster.Cluster, node int, ba roachpb.BatchRequest, -) (roachpb.BatchResponse, error) { + ctx context.Context, t test.Test, c cluster.Cluster, node int, ba kvpb.BatchRequest, +) (kvpb.BatchResponse, error) { reqArg, err := batchToJSONOrFatal(ba) if err != nil { - return roachpb.BatchResponse{}, err + return kvpb.BatchResponse{}, err } requestFileName := "request-" + uuid.FastMakeV4().String() + ".json" if err := c.PutString(ctx, reqArg, requestFileName, 0755, c.Node(node)); err != nil { - return roachpb.BatchResponse{}, err + return kvpb.BatchResponse{}, err } res, err := c.RunWithDetailsSingleNode(ctx, t.L(), c.Node(node), "./cockroach", "debug", "send-kv-batch", "--insecure", requestFileName) if err != nil { - return roachpb.BatchResponse{}, err + return kvpb.BatchResponse{}, err } return jsonToResponseOrFatal(res.Stdout) } -func batchToJSONOrFatal(ba roachpb.BatchRequest) (string, error) { +func batchToJSONOrFatal(ba kvpb.BatchRequest) (string, error) { jsonpb := protoutil.JSONPb{} jsonProto, err := jsonpb.Marshal(&ba) return string(jsonProto), err } -func jsonToResponseOrFatal(json string) (br roachpb.BatchResponse, err error) { +func jsonToResponseOrFatal(json string) (br kvpb.BatchResponse, err error) { jsonpb := protoutil.JSONPb{} err = jsonpb.Unmarshal([]byte(json), &br) return br, err diff --git a/pkg/gen/gomock.bzl b/pkg/gen/gomock.bzl index b41a907f77a9..aa4794abc40c 100644 --- a/pkg/gen/gomock.bzl +++ b/pkg/gen/gomock.bzl @@ -6,7 +6,7 @@ GOMOCK_SRCS = [ "//pkg/kv/kvclient/kvcoord:mocks_generated_test.go", "//pkg/kv/kvclient/rangecache/rangecachemock:mocks_generated.go", "//pkg/kv/kvclient/rangefeed:mocks_generated_test.go", - "//pkg/roachpb/roachpbmock:mocks_generated.go", + "//pkg/kv/kvpb/kvpbmock:mocks_generated.go", "//pkg/security/certmgr:mocks_generated_test.go", "//pkg/sql/schemachanger/scexec:mocks_generated_test.go", "//pkg/util/log:mocks_generated_test.go", diff --git a/pkg/gen/misc.bzl b/pkg/gen/misc.bzl index 99ae3acdf3b6..774fcce9e261 100644 --- a/pkg/gen/misc.bzl +++ b/pkg/gen/misc.bzl @@ -1,11 +1,11 @@ # Generated by genbzl MISC_SRCS = [ + "//pkg/kv/kvpb:batch_generated.go", "//pkg/kv/kvserver/concurrency:lockstate_interval_btree.go", "//pkg/kv/kvserver/concurrency:lockstate_interval_btree_test.go", "//pkg/kv/kvserver/spanlatch:latch_interval_btree.go", "//pkg/kv/kvserver/spanlatch:latch_interval_btree_test.go", - "//pkg/roachpb:batch_generated.go", "//pkg/roachprod/vm/aws:terraform/main.tf", "//pkg/spanconfig/spanconfigstore:entry_interval_btree.go", "//pkg/spanconfig/spanconfigstore:entry_interval_btree_test.go", diff --git a/pkg/gen/protobuf.bzl b/pkg/gen/protobuf.bzl index 42809e019e69..4aac989b5e90 100644 --- a/pkg/gen/protobuf.bzl +++ b/pkg/gen/protobuf.bzl @@ -27,6 +27,7 @@ PROTOBUF_SRCS = [ "//pkg/keyvisualizer/keyvispb:keyvispb_go_proto", "//pkg/kv/bulk/bulkpb:bulkpb_go_proto", "//pkg/kv/kvnemesis:kvnemesis_go_proto", + "//pkg/kv/kvpb:kvpb_go_proto", "//pkg/kv/kvserver/closedts/ctpb:ctpb_go_proto", "//pkg/kv/kvserver/concurrency/lock:lock_go_proto", "//pkg/kv/kvserver/concurrency/poison:poison_go_proto", diff --git a/pkg/gen/stringer.bzl b/pkg/gen/stringer.bzl index dec031ce3f08..4d89f66af5a9 100644 --- a/pkg/gen/stringer.bzl +++ b/pkg/gen/stringer.bzl @@ -7,10 +7,10 @@ STRINGER_SRCS = [ "//pkg/ccl/sqlproxyccl:errorcode_string.go", "//pkg/cli:keytype_string.go", "//pkg/kv/kvclient/kvcoord:txnstate_string.go", + "//pkg/kv/kvpb:errordetailtype_string.go", + "//pkg/kv/kvpb:method_string.go", "//pkg/kv/kvserver/closedts/sidetransport:cantclosereason_string.go", "//pkg/kv/kvserver:refreshraftreason_string.go", - "//pkg/roachpb:errordetailtype_string.go", - "//pkg/roachpb:method_string.go", "//pkg/sql/catalog/catalogkeys:commenttype_string.go", "//pkg/sql/catalog/catpb:privilegedescversion_string.go", "//pkg/sql/catalog/descpb:formatversion_string.go", diff --git a/pkg/gossip/BUILD.bazel b/pkg/gossip/BUILD.bazel index 161b0cf7162a..4d06c27969b6 100644 --- a/pkg/gossip/BUILD.bazel +++ b/pkg/gossip/BUILD.bazel @@ -24,6 +24,7 @@ go_library( "//pkg/base", "//pkg/config", "//pkg/config/zonepb", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/rpc", "//pkg/rpc/nodedialer", diff --git a/pkg/gossip/gossip.go b/pkg/gossip/gossip.go index 924fa8348a1d..dbbf1ba09489 100644 --- a/pkg/gossip/gossip.go +++ b/pkg/gossip/gossip.go @@ -61,6 +61,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" @@ -533,7 +534,7 @@ func (g *Gossip) GetStoreDescriptor(storeID roachpb.StoreID) (*roachpb.StoreDesc desc := (*roachpb.StoreDescriptor)(value) return desc, nil } - return nil, roachpb.NewStoreNotFoundError(storeID) + return nil, kvpb.NewStoreNotFoundError(storeID) } // LogStatus logs the current status of gossip such as the incoming and diff --git a/pkg/internal/client/requestbatcher/BUILD.bazel b/pkg/internal/client/requestbatcher/BUILD.bazel index dac8835030d9..54c09ec45d81 100644 --- a/pkg/internal/client/requestbatcher/BUILD.bazel +++ b/pkg/internal/client/requestbatcher/BUILD.bazel @@ -8,6 +8,7 @@ go_library( visibility = ["//pkg:__subpackages__"], deps = [ "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/util/contextutil", "//pkg/util/log", @@ -23,6 +24,7 @@ go_test( args = ["-test.timeout=55s"], embed = [":requestbatcher"], deps = [ + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/testutils", "//pkg/util/leaktest", diff --git a/pkg/internal/client/requestbatcher/batcher.go b/pkg/internal/client/requestbatcher/batcher.go index 90012e3616a2..543694d418af 100644 --- a/pkg/internal/client/requestbatcher/batcher.go +++ b/pkg/internal/client/requestbatcher/batcher.go @@ -30,6 +30,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/contextutil" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -184,7 +185,7 @@ type RequestBatcher struct { // Response is exported for use with the channel-oriented SendWithChan method. // At least one of Resp or Err will be populated for every sent Response. type Response struct { - Resp roachpb.Response + Resp kvpb.Response Err error } @@ -226,7 +227,7 @@ func validateConfig(cfg *Config) { // insufficiently buffered channel can lead to deadlocks and unintended delays // processing requests inside the RequestBatcher. func (b *RequestBatcher) SendWithChan( - ctx context.Context, respChan chan<- Response, rangeID roachpb.RangeID, req roachpb.Request, + ctx context.Context, respChan chan<- Response, rangeID roachpb.RangeID, req kvpb.Request, ) error { select { case b.requestChan <- b.pool.newRequest(ctx, rangeID, req, respChan): @@ -242,8 +243,8 @@ func (b *RequestBatcher) SendWithChan( // is canceled before the sending of the request completes. The context with // the latest deadline for a batch is used to send the underlying batch request. func (b *RequestBatcher) Send( - ctx context.Context, rangeID roachpb.RangeID, req roachpb.Request, -) (roachpb.Response, error) { + ctx context.Context, rangeID roachpb.RangeID, req kvpb.Request, +) (kvpb.Response, error) { responseChan := b.pool.getResponseChan() if err := b.SendWithChan(ctx, responseChan, rangeID, req); err != nil { return nil, err @@ -272,9 +273,9 @@ func (b *RequestBatcher) sendDone(ba *batch) { func (b *RequestBatcher) sendBatch(ctx context.Context, ba *batch) { if err := b.cfg.Stopper.RunAsyncTask(ctx, "send-batch", func(ctx context.Context) { defer b.sendDone(ba) - var br *roachpb.BatchResponse + var br *kvpb.BatchResponse send := func(ctx context.Context) error { - var pErr *roachpb.Error + var pErr *kvpb.Error if br, pErr = b.cfg.Sender.Send(ctx, ba.batchRequest(&b.cfg)); pErr != nil { return pErr.GoError() } @@ -295,7 +296,7 @@ func (b *RequestBatcher) sendBatch(ctx context.Context, ba *batch) { // be ordered nor guaranteed to be non-overlapping, so we can make no // assumptions about the requests that will result in full responses // (with no resume spans) vs. partial responses vs. empty responses (see - // the comment on roachpb.Header.MaxSpanRequestKeys). + // the comment on kvpb.Header.MaxSpanRequestKeys). // // To accommodate this, we keep track of all partial responses from // previous iterations. After receiving a batch of responses during an @@ -304,7 +305,7 @@ func (b *RequestBatcher) sendBatch(ctx context.Context, ba *batch) { // resume spans are removed. Responses that have resume spans are // updated appropriately and sent again in the next iteration. The loop // proceeds until all requests have been run to completion. - var prevResps []roachpb.Response + var prevResps []kvpb.Response for len(ba.reqs) > 0 { err := send(ctx) nextReqs, nextPrevResps := ba.reqs[:0], prevResps[:0] @@ -314,7 +315,7 @@ func (b *RequestBatcher) sendBatch(ctx context.Context, ba *batch) { resp := br.Responses[i].GetInner() if prevResps != nil { prevResp := prevResps[i] - if cErr := roachpb.CombineResponses(prevResp, resp); cErr != nil { + if cErr := kvpb.CombineResponses(prevResp, resp); cErr != nil { log.Fatalf(ctx, "%v", cErr) } resp = prevResp @@ -494,7 +495,7 @@ func (b *RequestBatcher) run(ctx context.Context) { type request struct { ctx context.Context - req roachpb.Request + req kvpb.Request rangeID roachpb.RangeID responseChan chan<- Response } @@ -526,10 +527,10 @@ func (b *batch) rangeID() roachpb.RangeID { return b.reqs[0].rangeID } -func (b *batch) batchRequest(cfg *Config) *roachpb.BatchRequest { - req := &roachpb.BatchRequest{ +func (b *batch) batchRequest(cfg *Config) *kvpb.BatchRequest { + req := &kvpb.BatchRequest{ // Preallocate the Requests slice. - Requests: make([]roachpb.RequestUnion, 0, len(b.reqs)), + Requests: make([]kvpb.RequestUnion, 0, len(b.reqs)), } for _, r := range b.reqs { req.Add(r.req) @@ -571,7 +572,7 @@ func (p *pool) putResponseChan(r chan Response) { } func (p *pool) newRequest( - ctx context.Context, rangeID roachpb.RangeID, req roachpb.Request, responseChan chan<- Response, + ctx context.Context, rangeID roachpb.RangeID, req kvpb.Request, responseChan chan<- Response, ) *request { r := p.requestPool.Get().(*request) *r = request{ diff --git a/pkg/internal/client/requestbatcher/batcher_test.go b/pkg/internal/client/requestbatcher/batcher_test.go index 2ab6be9b40b2..40960168f83b 100644 --- a/pkg/internal/client/requestbatcher/batcher_test.go +++ b/pkg/internal/client/requestbatcher/batcher_test.go @@ -19,6 +19,7 @@ import ( "testing" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -31,32 +32,32 @@ import ( type batchResp struct { // TODO(ajwerner): we never actually test that this result is what we expect // it to be. We should add a test that does so. - br *roachpb.BatchResponse - pe *roachpb.Error + br *kvpb.BatchResponse + pe *kvpb.Error } type batchSend struct { ctx context.Context - ba *roachpb.BatchRequest + ba *kvpb.BatchRequest respChan chan<- batchResp } type chanSender chan batchSend func (c chanSender) Send( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, *kvpb.Error) { respChan := make(chan batchResp, 1) select { case c <- batchSend{ctx: ctx, ba: ba, respChan: respChan}: case <-ctx.Done(): - return nil, roachpb.NewError(ctx.Err()) + return nil, kvpb.NewError(ctx.Err()) } select { case resp := <-respChan: return resp.br, resp.pe case <-ctx.Done(): - return nil, roachpb.NewError(ctx.Err()) + return nil, kvpb.NewError(ctx.Err()) } } @@ -65,7 +66,7 @@ type senderGroup struct { g errgroup.Group } -func (g *senderGroup) Send(rangeID roachpb.RangeID, request roachpb.Request) { +func (g *senderGroup) Send(rangeID roachpb.RangeID, request kvpb.Request) { g.g.Go(func() error { _, err := g.b.Send(context.Background(), rangeID, request) return err @@ -103,8 +104,8 @@ func TestBatcherSendOnSizeWithReset(t *testing.T) { Stopper: stopper, }) g := senderGroup{b: b} - g.Send(1, &roachpb.GetRequest{}) - g.Send(1, &roachpb.GetRequest{}) + g.Send(1, &kvpb.GetRequest{}) + g.Send(1, &kvpb.GetRequest{}) s := <-sc s.respChan <- batchResp{} // See the comment above wait. In rare cases the batch will be sent before the @@ -145,7 +146,7 @@ func TestBatchesAtTheSameTime(t *testing.T) { const N = 20 sendChan := make(chan Response, N) for i := 0; i < N; i++ { - assert.Nil(t, b.SendWithChan(context.Background(), sendChan, roachpb.RangeID(i), &roachpb.GetRequest{})) + assert.Nil(t, b.SendWithChan(context.Background(), sendChan, roachpb.RangeID(i), &kvpb.GetRequest{})) } for i := 0; i < N; i++ { bs := <-sc @@ -170,12 +171,12 @@ func TestBackpressure(t *testing.T) { // These 3 should all send without blocking but should put the batcher into // back pressure. sendChan := make(chan Response, 6) - assert.Nil(t, b.SendWithChan(context.Background(), sendChan, 1, &roachpb.GetRequest{})) - assert.Nil(t, b.SendWithChan(context.Background(), sendChan, 2, &roachpb.GetRequest{})) - assert.Nil(t, b.SendWithChan(context.Background(), sendChan, 3, &roachpb.GetRequest{})) + assert.Nil(t, b.SendWithChan(context.Background(), sendChan, 1, &kvpb.GetRequest{})) + assert.Nil(t, b.SendWithChan(context.Background(), sendChan, 2, &kvpb.GetRequest{})) + assert.Nil(t, b.SendWithChan(context.Background(), sendChan, 3, &kvpb.GetRequest{})) var sent int64 send := func() { - assert.Nil(t, b.SendWithChan(context.Background(), sendChan, 4, &roachpb.GetRequest{})) + assert.Nil(t, b.SendWithChan(context.Background(), sendChan, 4, &kvpb.GetRequest{})) atomic.AddInt64(&sent, 1) } go send() @@ -229,11 +230,11 @@ func TestBatcherSend(t *testing.T) { // MaxMsgsPerBatch configuration. The range 1 batch will be sent after the // MaxWait timeout expires. g := senderGroup{b: b} - g.Send(1, &roachpb.GetRequest{}) - g.Send(2, &roachpb.GetRequest{}) - g.Send(1, &roachpb.GetRequest{}) - g.Send(2, &roachpb.GetRequest{}) - g.Send(2, &roachpb.GetRequest{}) + g.Send(1, &kvpb.GetRequest{}) + g.Send(2, &kvpb.GetRequest{}) + g.Send(1, &kvpb.GetRequest{}) + g.Send(2, &kvpb.GetRequest{}) + g.Send(2, &kvpb.GetRequest{}) // Wait for the range 2 request and ensure it contains 3 requests. s := <-sc assert.Len(t, s.ba.Requests, 3) @@ -257,7 +258,7 @@ func TestSendAfterStopped(t *testing.T) { Stopper: stopper, }) stopper.Stop(context.Background()) - _, err := b.Send(context.Background(), 1, &roachpb.GetRequest{}) + _, err := b.Send(context.Background(), 1, &kvpb.GetRequest{}) assert.Equal(t, err, stop.ErrUnavailable) } @@ -272,7 +273,7 @@ func TestSendAfterCanceled(t *testing.T) { }) ctx, cancel := context.WithCancel(context.Background()) cancel() - _, err := b.Send(ctx, 1, &roachpb.GetRequest{}) + _, err := b.Send(ctx, 1, &kvpb.GetRequest{}) assert.Equal(t, err, ctx.Err()) } @@ -289,7 +290,7 @@ func TestStopDuringSend(t *testing.T) { }) errChan := make(chan error) go func() { - _, err := b.Send(context.Background(), 1, &roachpb.GetRequest{}) + _, err := b.Send(context.Background(), 1, &kvpb.GetRequest{}) errChan <- err }() // Wait for the request to get sent. @@ -337,7 +338,7 @@ func TestBatchTimeout(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), timeout) defer cancel() respChan := make(chan Response, 1) - if err := b.SendWithChan(ctx, respChan, 1, &roachpb.GetRequest{}); err != nil { + if err := b.SendWithChan(ctx, respChan, 1, &kvpb.GetRequest{}); err != nil { testutils.IsError(err, context.DeadlineExceeded.Error()) return } @@ -375,11 +376,11 @@ func TestBatchTimeout(t *testing.T) { var err1, err2 error err1Chan := make(chan error, 1) go func() { - _, err1 = b.Send(ctx1, 1, &roachpb.GetRequest{}) + _, err1 = b.Send(ctx1, 1, &kvpb.GetRequest{}) err1Chan <- err1 wg.Done() }() - go func() { _, err2 = b.Send(ctx2, 1, &roachpb.GetRequest{}); wg.Done() }() + go func() { _, err2 = b.Send(ctx2, 1, &kvpb.GetRequest{}); wg.Done() }() select { case s := <-sc: assert.Len(t, s.ba.Requests, 2) @@ -415,13 +416,13 @@ func TestIdleAndMaxTimeoutDisabled(t *testing.T) { // the requests, they should only be sent when the MaxMsgsPerBatch limit is // reached, because no MaxWait timeout is configured. g := senderGroup{b: b} - g.Send(1, &roachpb.GetRequest{}) + g.Send(1, &kvpb.GetRequest{}) select { case <-sc: t.Fatalf("RequestBatcher should not sent based on time") case <-time.After(10 * time.Millisecond): } - g.Send(1, &roachpb.GetRequest{}) + g.Send(1, &kvpb.GetRequest{}) s := <-sc assert.Len(t, s.ba.Requests, 2) s.respChan <- batchResp{} @@ -465,13 +466,13 @@ func TestMaxKeysPerBatchReq(t *testing.T) { type span [2]string // [key, endKey] type spanMap map[span]span var nilResumeSpan span - makeReq := func(sp span) *roachpb.ResolveIntentRangeRequest { - var req roachpb.ResolveIntentRangeRequest + makeReq := func(sp span) *kvpb.ResolveIntentRangeRequest { + var req kvpb.ResolveIntentRangeRequest req.Key = roachpb.Key(sp[0]) req.EndKey = roachpb.Key(sp[1]) return &req } - makeResp := func(ba *roachpb.BatchRequest, resumeSpans spanMap) *roachpb.BatchResponse { + makeResp := func(ba *kvpb.BatchRequest, resumeSpans spanMap) *kvpb.BatchResponse { br := ba.CreateReply() for i, ru := range ba.Requests { req := ru.GetResolveIntentRange() @@ -487,7 +488,7 @@ func TestMaxKeysPerBatchReq(t *testing.T) { resp.ResumeSpan = &roachpb.Span{ Key: roachpb.Key(resumeSp[0]), EndKey: roachpb.Key(resumeSp[1]), } - resp.ResumeReason = roachpb.RESUME_KEY_LIMIT + resp.ResumeReason = kvpb.RESUME_KEY_LIMIT } return br } diff --git a/pkg/jobs/BUILD.bazel b/pkg/jobs/BUILD.bazel index def651b5f524..ee5c316b6570 100644 --- a/pkg/jobs/BUILD.bazel +++ b/pkg/jobs/BUILD.bazel @@ -32,6 +32,7 @@ go_library( "//pkg/clusterversion", "//pkg/jobs/jobspb", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/multitenant", "//pkg/roachpb", "//pkg/scheduledjobs", @@ -107,6 +108,7 @@ go_test( "//pkg/keys", "//pkg/keyvisualizer", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/roachpb", "//pkg/scheduledjobs", diff --git a/pkg/jobs/job_scheduler.go b/pkg/jobs/job_scheduler.go index d308a66e99d0..0ce4b47a9971 100644 --- a/pkg/jobs/job_scheduler.go +++ b/pkg/jobs/job_scheduler.go @@ -19,8 +19,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/multitenant" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/scheduledjobs" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql/isql" @@ -213,7 +213,7 @@ func withSavePoint(ctx context.Context, txn *kv.Txn, fn func() error) error { return nil } - if errors.HasType(execErr, (*roachpb.TransactionRetryWithProtoRefreshError)(nil)) { + if errors.HasType(execErr, (*kvpb.TransactionRetryWithProtoRefreshError)(nil)) { // If function execution failed because transaction was restarted, // treat this error as a savePointError so that the execution code bails out // and retries scheduling loop. diff --git a/pkg/jobs/jobs_test.go b/pkg/jobs/jobs_test.go index 85dbb8ed2392..cddf600a2eef 100644 --- a/pkg/jobs/jobs_test.go +++ b/pkg/jobs/jobs_test.go @@ -33,6 +33,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keyvisualizer" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" @@ -2716,15 +2717,15 @@ func TestStartableJobTxnRetry(t *testing.T) { haveInjectedRetry := false params := base.TestServerArgs{} params.Knobs.Store = &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(ctx context.Context, r *roachpb.BatchRequest) *roachpb.Error { + TestingRequestFilter: func(ctx context.Context, r *kvpb.BatchRequest) *kvpb.Error { if r.Txn == nil || r.Txn.Name != txnName { return nil } - if _, ok := r.GetArg(roachpb.EndTxn); ok { + if _, ok := r.GetArg(kvpb.EndTxn); ok { if !haveInjectedRetry { haveInjectedRetry = true // Force a retry error the first time. - return roachpb.NewError(roachpb.NewTransactionRetryError(roachpb.RETRY_REASON_UNKNOWN, "injected error")) + return kvpb.NewError(kvpb.NewTransactionRetryError(kvpb.RETRY_REASON_UNKNOWN, "injected error")) } } return nil diff --git a/pkg/jobs/jobspb/BUILD.bazel b/pkg/jobs/jobspb/BUILD.bazel index 52677f1acc61..bbd4e42e80b9 100644 --- a/pkg/jobs/jobspb/BUILD.bazel +++ b/pkg/jobs/jobspb/BUILD.bazel @@ -29,6 +29,7 @@ proto_library( visibility = ["//visibility:public"], deps = [ "//pkg/clusterversion:clusterversion_proto", + "//pkg/kv/kvpb:kvpb_proto", "//pkg/multitenant/mtinfopb:mtinfopb_proto", "//pkg/roachpb:roachpb_proto", "//pkg/sql/catalog/descpb:descpb_proto", @@ -49,6 +50,7 @@ go_proto_library( visibility = ["//visibility:public"], deps = [ "//pkg/clusterversion", + "//pkg/kv/kvpb", "//pkg/multitenant/mtinfopb", "//pkg/roachpb", "//pkg/security/username", # keep diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index c6c544f58d0c..fa5ea3f416d2 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -14,7 +14,7 @@ option go_package = "jobspb"; import "errorspb/errors.proto"; import "gogoproto/gogo.proto"; -import "roachpb/api.proto"; +import "kv/kvpb/api.proto"; import "roachpb/data.proto"; import "roachpb/metadata.proto"; import "roachpb/io-formats.proto"; diff --git a/pkg/keys/BUILD.bazel b/pkg/keys/BUILD.bazel index 3bd12ac1b83e..47513301d44c 100644 --- a/pkg/keys/BUILD.bazel +++ b/pkg/keys/BUILD.bazel @@ -15,6 +15,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/keys", visibility = ["//visibility:public"], deps = [ + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/util/encoding", "//pkg/util/uuid", @@ -34,6 +35,7 @@ go_test( args = ["-test.timeout=55s"], embed = [":keys"], deps = [ + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/testutils", "//pkg/util/bitarray", diff --git a/pkg/keys/keys.go b/pkg/keys/keys.go index 6a31638b454e..f2c2dcb26351 100644 --- a/pkg/keys/keys.go +++ b/pkg/keys/keys.go @@ -15,6 +15,7 @@ import ( "fmt" "math" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -949,13 +950,13 @@ func EnsureSafeSplitKey(key roachpb.Key) (roachpb.Key, error) { } // Range returns a key range encompassing the key ranges of all requests. -func Range(reqs []roachpb.RequestUnion) (roachpb.RSpan, error) { +func Range(reqs []kvpb.RequestUnion) (roachpb.RSpan, error) { from := roachpb.RKeyMax to := roachpb.RKeyMin for _, arg := range reqs { req := arg.GetInner() h := req.Header() - if !roachpb.IsRange(req) && len(h.EndKey) != 0 { + if !kvpb.IsRange(req) && len(h.EndKey) != 0 { return roachpb.RSpan{}, errors.Errorf("end key specified for non-range operation: %s", req) } diff --git a/pkg/keys/keys_test.go b/pkg/keys/keys_test.go index 1bbf9fd79f3c..39957add50c5 100644 --- a/pkg/keys/keys_test.go +++ b/pkg/keys/keys_test.go @@ -17,6 +17,7 @@ import ( "reflect" "testing" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/encoding" @@ -541,9 +542,9 @@ func TestBatchRange(t *testing.T) { } for i, c := range testCases { - var ba roachpb.BatchRequest + var ba kvpb.BatchRequest for _, pair := range c.req { - ba.Add(&roachpb.ScanRequest{RequestHeader: roachpb.RequestHeader{ + ba.Add(&kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{ Key: roachpb.Key(pair[0]), EndKey: roachpb.Key(pair[1]), }}) } @@ -572,8 +573,8 @@ func TestBatchError(t *testing.T) { } for i, c := range testCases { - var ba roachpb.BatchRequest - ba.Add(&roachpb.ScanRequest{RequestHeader: roachpb.RequestHeader{ + var ba kvpb.BatchRequest + ba.Add(&kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{ Key: roachpb.Key(c.req[0]), EndKey: roachpb.Key(c.req[1]), }}) if _, err := Range(ba.Requests); !testutils.IsError(err, c.errMsg) { @@ -582,8 +583,8 @@ func TestBatchError(t *testing.T) { } // Test a case where a non-range request has an end key. - var ba roachpb.BatchRequest - ba.Add(&roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{ + var ba kvpb.BatchRequest + ba.Add(&kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{ Key: roachpb.Key("a"), EndKey: roachpb.Key("b"), }}) if _, err := Range(ba.Requests); !testutils.IsError(err, "end key specified for non-range operation") { diff --git a/pkg/keyvisualizer/keyvissubscriber/BUILD.bazel b/pkg/keyvisualizer/keyvissubscriber/BUILD.bazel index 1b4b5edb3bcd..6a3ab47bde6d 100644 --- a/pkg/keyvisualizer/keyvissubscriber/BUILD.bazel +++ b/pkg/keyvisualizer/keyvissubscriber/BUILD.bazel @@ -11,6 +11,7 @@ go_library( "//pkg/keyvisualizer/keyvispb", "//pkg/kv", "//pkg/kv/kvclient/rangefeed", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/settings/cluster", "//pkg/sql/catalog", diff --git a/pkg/keyvisualizer/keyvissubscriber/boundary_subscriber.go b/pkg/keyvisualizer/keyvissubscriber/boundary_subscriber.go index 055e181a94a4..b3baa8a022c9 100644 --- a/pkg/keyvisualizer/keyvissubscriber/boundary_subscriber.go +++ b/pkg/keyvisualizer/keyvissubscriber/boundary_subscriber.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keyvisualizer/keyvispb" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog" @@ -68,7 +69,7 @@ func Start( // decodeRow decodes a row of the system.span_stats_tenant_boundaries table. decodeRow := func( ctx context.Context, - kv *roachpb.RangeFeedValue, + kv *kvpb.RangeFeedValue, ) (*roachpb.TenantID, *keyvispb.UpdateBoundariesRequest, error) { // First, decode the tenant_id column. @@ -121,7 +122,7 @@ func Start( "tenant-boundaries-watcher", []roachpb.Span{tableSpan}, initialTimestamp, - func(ctx context.Context, kv *roachpb.RangeFeedValue) { + func(ctx context.Context, kv *kvpb.RangeFeedValue) { tID, update, err := decodeRow(ctx, kv) if err != nil { log.Warningf(ctx, diff --git a/pkg/kv/BUILD.bazel b/pkg/kv/BUILD.bazel index 7a9b69c04917..a22e1081323f 100644 --- a/pkg/kv/BUILD.bazel +++ b/pkg/kv/BUILD.bazel @@ -20,6 +20,7 @@ go_library( "//pkg/base", "//pkg/keys", "//pkg/kv/kvbase", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/closedts", "//pkg/roachpb", "//pkg/settings", @@ -65,6 +66,7 @@ go_test( "//pkg/base", "//pkg/config/zonepb", "//pkg/keys", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/kv/kvserver/concurrency/lock", "//pkg/kv/kvserver/kvserverbase", diff --git a/pkg/kv/batch.go b/pkg/kv/batch.go index b999f20fa1f0..447e2970bdc9 100644 --- a/pkg/kv/batch.go +++ b/pkg/kv/batch.go @@ -13,6 +13,7 @@ package kv import ( "context" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -47,11 +48,11 @@ type Batch struct { Results []Result // The Header which will be used to send the resulting BatchRequest. // To be modified directly. - Header roachpb.Header + Header kvpb.Header // The AdmissionHeader which will be used when sending the resulting // BatchRequest. To be modified directly. - AdmissionHeader roachpb.AdmissionHeader - reqs []roachpb.RequestUnion + AdmissionHeader kvpb.AdmissionHeader + reqs []kvpb.RequestUnion // approxMutationReqBytes tracks the approximate size of keys and values in // mutations added to this batch via Put, CPut, InitPut, Del, etc. @@ -60,9 +61,9 @@ type Batch struct { // operations renders the batch unusable. raw bool // Once received, the response from a successful batch. - response *roachpb.BatchResponse + response *kvpb.BatchResponse // Once received, any error encountered sending the batch. - pErr *roachpb.Error + pErr *kvpb.Error // We use pre-allocated buffers to avoid dynamic allocations for small batches. resultsBuf [8]Result @@ -79,19 +80,19 @@ func (b *Batch) ApproximateMutationBytes() int { } // Requests exposes the requests stashed in the batch thus far. -func (b *Batch) Requests() []roachpb.RequestUnion { +func (b *Batch) Requests() []kvpb.RequestUnion { return b.reqs } // RawResponse returns the BatchResponse which was the result of a successful // execution of the batch, and nil otherwise. -func (b *Batch) RawResponse() *roachpb.BatchResponse { +func (b *Batch) RawResponse() *kvpb.BatchResponse { return b.response } // MustPErr returns the structured error resulting from a failed execution of // the batch, asserting that that error is non-nil. -func (b *Batch) MustPErr() *roachpb.Error { +func (b *Batch) MustPErr() *kvpb.Error { if b.pErr == nil { panic(errors.Errorf("expected non-nil pErr for batch %+v", b)) } @@ -104,7 +105,7 @@ func (b *Batch) validate() error { if err != nil { // Set pErr just as sendAndFill does, so that higher layers can find it // using MustPErr. - b.pErr = roachpb.NewError(err) + b.pErr = kvpb.NewError(err) } return err } @@ -165,7 +166,7 @@ func (b *Batch) fillResults(ctx context.Context) { for k := 0; k < result.calls; k++ { args := b.reqs[offset+k].GetInner() - var reply roachpb.Response + var reply kvpb.Response // It's possible that result.Err was populated early, for example // when PutProto is called and the proto marshaling errored out. // In that case, we don't want to mutate this result's error @@ -178,7 +179,7 @@ func (b *Batch) fillResults(ctx context.Context) { // this pass. if b.response != nil && offset+k < len(b.response.Responses) { reply = b.response.Responses[offset+k].GetInner() - } else if args.Method() != roachpb.EndTxn { + } else if args.Method() != kvpb.EndTxn { // TODO(tschottdorf): EndTxn is special-cased here // because it may be elided (r/o txns). Might prefer to // simulate an EndTxn response instead; this effectively @@ -191,41 +192,41 @@ func (b *Batch) fillResults(ctx context.Context) { } switch req := args.(type) { - case *roachpb.GetRequest: + case *kvpb.GetRequest: row := &result.Rows[k] row.Key = []byte(req.Key) if result.Err == nil { - row.Value = reply.(*roachpb.GetResponse).Value + row.Value = reply.(*kvpb.GetResponse).Value } - case *roachpb.PutRequest: + case *kvpb.PutRequest: row := &result.Rows[k] row.Key = []byte(req.Key) if result.Err == nil { row.Value = &req.Value } - case *roachpb.ConditionalPutRequest: + case *kvpb.ConditionalPutRequest: row := &result.Rows[k] row.Key = []byte(req.Key) if result.Err == nil { row.Value = &req.Value } - case *roachpb.InitPutRequest: + case *kvpb.InitPutRequest: row := &result.Rows[k] row.Key = []byte(req.Key) if result.Err == nil { row.Value = &req.Value } - case *roachpb.IncrementRequest: + case *kvpb.IncrementRequest: row := &result.Rows[k] row.Key = []byte(req.Key) if result.Err == nil { - t := reply.(*roachpb.IncrementResponse) + t := reply.(*kvpb.IncrementResponse) row.Value = &roachpb.Value{} row.Value.SetInt(t.NewValue) } - case *roachpb.ScanRequest: + case *kvpb.ScanRequest: if result.Err == nil { - t := reply.(*roachpb.ScanResponse) + t := reply.(*kvpb.ScanResponse) result.Rows = make([]KeyValue, len(t.Rows)) for j := range t.Rows { src := &t.Rows[j] @@ -234,9 +235,9 @@ func (b *Batch) fillResults(ctx context.Context) { dst.Value = &src.Value } } - case *roachpb.ReverseScanRequest: + case *kvpb.ReverseScanRequest: if result.Err == nil { - t := reply.(*roachpb.ReverseScanResponse) + t := reply.(*kvpb.ReverseScanResponse) result.Rows = make([]KeyValue, len(t.Rows)) for j := range t.Rows { src := &t.Rows[j] @@ -245,45 +246,45 @@ func (b *Batch) fillResults(ctx context.Context) { dst.Value = &src.Value } } - case *roachpb.DeleteRequest: + case *kvpb.DeleteRequest: if result.Err == nil { - resp := reply.(*roachpb.DeleteResponse) + resp := reply.(*kvpb.DeleteResponse) if resp.FoundKey { // Accumulate all keys that were deleted as part of a // single Del() operation. - result.Keys = append(result.Keys, args.(*roachpb.DeleteRequest).Key) + result.Keys = append(result.Keys, args.(*kvpb.DeleteRequest).Key) } } - case *roachpb.DeleteRangeRequest: + case *kvpb.DeleteRangeRequest: if result.Err == nil { - result.Keys = reply.(*roachpb.DeleteRangeResponse).Keys + result.Keys = reply.(*kvpb.DeleteRangeResponse).Keys } // Nothing to do for all methods below as they do not generate // any rows. - case *roachpb.EndTxnRequest: - case *roachpb.AdminMergeRequest: - case *roachpb.AdminSplitRequest: - case *roachpb.AdminUnsplitRequest: - case *roachpb.AdminTransferLeaseRequest: - case *roachpb.AdminChangeReplicasRequest: - case *roachpb.AdminRelocateRangeRequest: - case *roachpb.HeartbeatTxnRequest: - case *roachpb.GCRequest: - case *roachpb.LeaseInfoRequest: - case *roachpb.PushTxnRequest: - case *roachpb.QueryTxnRequest: - case *roachpb.QueryIntentRequest: - case *roachpb.ResolveIntentRequest: - case *roachpb.ResolveIntentRangeRequest: - case *roachpb.MergeRequest: - case *roachpb.TruncateLogRequest: - case *roachpb.RequestLeaseRequest: - case *roachpb.CheckConsistencyRequest: - case *roachpb.AdminScatterRequest: - case *roachpb.AddSSTableRequest: - case *roachpb.MigrateRequest: - case *roachpb.QueryResolvedTimestampRequest: - case *roachpb.BarrierRequest: + case *kvpb.EndTxnRequest: + case *kvpb.AdminMergeRequest: + case *kvpb.AdminSplitRequest: + case *kvpb.AdminUnsplitRequest: + case *kvpb.AdminTransferLeaseRequest: + case *kvpb.AdminChangeReplicasRequest: + case *kvpb.AdminRelocateRangeRequest: + case *kvpb.HeartbeatTxnRequest: + case *kvpb.GCRequest: + case *kvpb.LeaseInfoRequest: + case *kvpb.PushTxnRequest: + case *kvpb.QueryTxnRequest: + case *kvpb.QueryIntentRequest: + case *kvpb.ResolveIntentRequest: + case *kvpb.ResolveIntentRangeRequest: + case *kvpb.MergeRequest: + case *kvpb.TruncateLogRequest: + case *kvpb.RequestLeaseRequest: + case *kvpb.CheckConsistencyRequest: + case *kvpb.AdminScatterRequest: + case *kvpb.AddSSTableRequest: + case *kvpb.MigrateRequest: + case *kvpb.QueryResolvedTimestampRequest: + case *kvpb.BarrierRequest: default: if result.Err == nil { result.Err = errors.Errorf("unsupported reply: %T for %T", @@ -323,14 +324,14 @@ func (b *Batch) growReqs(n int) { for newSize < len(b.reqs)+n { newSize *= 2 } - newReqs := make([]roachpb.RequestUnion, len(b.reqs), newSize) + newReqs := make([]kvpb.RequestUnion, len(b.reqs), newSize) copy(newReqs, b.reqs) b.reqs = newReqs } b.reqs = b.reqs[:len(b.reqs)+n] } -func (b *Batch) appendReqs(args ...roachpb.Request) { +func (b *Batch) appendReqs(args ...kvpb.Request) { n := len(b.reqs) b.growReqs(len(args)) for i := range args { @@ -341,16 +342,16 @@ func (b *Batch) appendReqs(args ...roachpb.Request) { // AddRawRequest adds the specified requests to the batch. No responses will // be allocated for them, and using any of the non-raw operations will result // in an error when running the batch. -func (b *Batch) AddRawRequest(reqs ...roachpb.Request) { +func (b *Batch) AddRawRequest(reqs ...kvpb.Request) { b.raw = true for _, args := range reqs { numRows := 0 switch args.(type) { - case *roachpb.GetRequest, - *roachpb.PutRequest, - *roachpb.ConditionalPutRequest, - *roachpb.IncrementRequest, - *roachpb.DeleteRequest: + case *kvpb.GetRequest, + *kvpb.PutRequest, + *kvpb.ConditionalPutRequest, + *kvpb.IncrementRequest, + *kvpb.DeleteRequest: numRows = 1 } b.appendReqs(args) @@ -364,7 +365,7 @@ func (b *Batch) get(key interface{}, forUpdate bool) { b.initResult(0, 1, notRaw, err) return } - b.appendReqs(roachpb.NewGet(k, forUpdate)) + b.appendReqs(kvpb.NewGet(k, forUpdate)) b.initResult(1, 1, notRaw, nil) } @@ -403,9 +404,9 @@ func (b *Batch) put(key, value interface{}, inline bool) { return } if inline { - b.appendReqs(roachpb.NewPutInline(k, v)) + b.appendReqs(kvpb.NewPutInline(k, v)) } else { - b.appendReqs(roachpb.NewPut(k, v)) + b.appendReqs(kvpb.NewPut(k, v)) } b.approxMutationReqBytes += len(k) + len(v.RawBytes) b.initResult(1, 1, notRaw, nil) @@ -502,9 +503,9 @@ func (b *Batch) cputInternal( return } if inline { - b.appendReqs(roachpb.NewConditionalPutInline(k, v, expValue, allowNotExist)) + b.appendReqs(kvpb.NewConditionalPutInline(k, v, expValue, allowNotExist)) } else { - b.appendReqs(roachpb.NewConditionalPut(k, v, expValue, allowNotExist)) + b.appendReqs(kvpb.NewConditionalPut(k, v, expValue, allowNotExist)) } b.approxMutationReqBytes += len(k) + len(v.RawBytes) b.initResult(1, 1, notRaw, nil) @@ -529,7 +530,7 @@ func (b *Batch) InitPut(key, value interface{}, failOnTombstones bool) { b.initResult(0, 1, notRaw, err) return } - b.appendReqs(roachpb.NewInitPut(k, v, failOnTombstones)) + b.appendReqs(kvpb.NewInitPut(k, v, failOnTombstones)) b.approxMutationReqBytes += len(k) + len(v.RawBytes) b.initResult(1, 1, notRaw, nil) } @@ -548,7 +549,7 @@ func (b *Batch) Inc(key interface{}, value int64) { b.initResult(0, 1, notRaw, err) return } - b.appendReqs(roachpb.NewIncrement(k, value)) + b.appendReqs(kvpb.NewIncrement(k, value)) b.initResult(1, 1, notRaw, nil) } @@ -564,9 +565,9 @@ func (b *Batch) scan(s, e interface{}, isReverse, forUpdate bool) { return } if !isReverse { - b.appendReqs(roachpb.NewScan(begin, end, forUpdate)) + b.appendReqs(kvpb.NewScan(begin, end, forUpdate)) } else { - b.appendReqs(roachpb.NewReverseScan(begin, end, forUpdate)) + b.appendReqs(kvpb.NewReverseScan(begin, end, forUpdate)) } b.initResult(1, 0, notRaw, nil) } @@ -625,14 +626,14 @@ func (b *Batch) ReverseScanForUpdate(s, e interface{}) { // // key can be either a byte slice or a string. func (b *Batch) Del(keys ...interface{}) { - reqs := make([]roachpb.Request, 0, len(keys)) + reqs := make([]kvpb.Request, 0, len(keys)) for _, key := range keys { k, err := marshalKey(key) if err != nil { b.initResult(len(keys), 0, notRaw, err) return } - reqs = append(reqs, roachpb.NewDelete(k)) + reqs = append(reqs, kvpb.NewDelete(k)) b.approxMutationReqBytes += len(k) } b.appendReqs(reqs...) @@ -656,7 +657,7 @@ func (b *Batch) DelRange(s, e interface{}, returnKeys bool) { b.initResult(0, 0, notRaw, err) return } - b.appendReqs(roachpb.NewDeleteRange(begin, end, returnKeys)) + b.appendReqs(kvpb.NewDeleteRange(begin, end, returnKeys)) b.initResult(1, 0, notRaw, nil) } @@ -674,8 +675,8 @@ func (b *Batch) DelRangeUsingTombstone(s, e interface{}) { b.initResult(0, 0, notRaw, err) return } - b.appendReqs(&roachpb.DeleteRangeRequest{ - RequestHeader: roachpb.RequestHeader{ + b.appendReqs(&kvpb.DeleteRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: start, EndKey: end, }, @@ -692,8 +693,8 @@ func (b *Batch) adminMerge(key interface{}) { b.initResult(0, 0, notRaw, err) return } - req := &roachpb.AdminMergeRequest{ - RequestHeader: roachpb.RequestHeader{ + req := &kvpb.AdminMergeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: k, }, } @@ -711,8 +712,8 @@ func (b *Batch) adminSplit( b.initResult(0, 0, notRaw, err) return } - req := &roachpb.AdminSplitRequest{ - RequestHeader: roachpb.RequestHeader{ + req := &kvpb.AdminSplitRequest{ + RequestHeader: kvpb.RequestHeader{ Key: splitKey, }, SplitKey: splitKey, @@ -728,8 +729,8 @@ func (b *Batch) adminUnsplit(splitKeyIn interface{}) { if err != nil { b.initResult(0, 0, notRaw, err) } - req := &roachpb.AdminUnsplitRequest{ - RequestHeader: roachpb.RequestHeader{ + req := &kvpb.AdminUnsplitRequest{ + RequestHeader: kvpb.RequestHeader{ Key: splitKey, }, } @@ -747,8 +748,8 @@ func (b *Batch) adminTransferLease( b.initResult(0, 0, notRaw, err) return } - req := &roachpb.AdminTransferLeaseRequest{ - RequestHeader: roachpb.RequestHeader{ + req := &kvpb.AdminTransferLeaseRequest{ + RequestHeader: kvpb.RequestHeader{ Key: k, }, Target: target, @@ -761,15 +762,15 @@ func (b *Batch) adminTransferLease( // adminChangeReplicas is only exported on DB. It is here for symmetry with the // other operations. func (b *Batch) adminChangeReplicas( - key interface{}, expDesc roachpb.RangeDescriptor, chgs []roachpb.ReplicationChange, + key interface{}, expDesc roachpb.RangeDescriptor, chgs []kvpb.ReplicationChange, ) { k, err := marshalKey(key) if err != nil { b.initResult(0, 0, notRaw, err) return } - req := &roachpb.AdminChangeReplicasRequest{ - RequestHeader: roachpb.RequestHeader{ + req := &kvpb.AdminChangeReplicasRequest{ + RequestHeader: kvpb.RequestHeader{ Key: k, }, ExpDesc: expDesc, @@ -792,8 +793,8 @@ func (b *Batch) adminRelocateRange( b.initResult(0, 0, notRaw, err) return } - req := &roachpb.AdminRelocateRangeRequest{ - RequestHeader: roachpb.RequestHeader{ + req := &kvpb.AdminRelocateRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: k, }, VoterTargets: voterTargets, @@ -826,8 +827,8 @@ func (b *Batch) addSSTable( b.initResult(0, 0, notRaw, err) return } - req := &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{ + req := &kvpb.AddSSTableRequest{ + RequestHeader: kvpb.RequestHeader{ Key: begin, EndKey: end, }, @@ -855,8 +856,8 @@ func (b *Batch) migrate(s, e interface{}, version roachpb.Version) { b.initResult(0, 0, notRaw, err) return } - req := &roachpb.MigrateRequest{ - RequestHeader: roachpb.RequestHeader{ + req := &kvpb.MigrateRequest{ + RequestHeader: kvpb.RequestHeader{ Key: begin, EndKey: end, }, @@ -878,8 +879,8 @@ func (b *Batch) queryResolvedTimestamp(s, e interface{}) { b.initResult(0, 0, notRaw, err) return } - req := &roachpb.QueryResolvedTimestampRequest{ - RequestHeader: roachpb.RequestHeader{ + req := &kvpb.QueryResolvedTimestampRequest{ + RequestHeader: kvpb.RequestHeader{ Key: begin, EndKey: end, }, @@ -899,8 +900,8 @@ func (b *Batch) barrier(s, e interface{}) { b.initResult(0, 0, notRaw, err) return } - req := &roachpb.BarrierRequest{ - RequestHeader: roachpb.RequestHeader{ + req := &kvpb.BarrierRequest{ + RequestHeader: kvpb.RequestHeader{ Key: begin, EndKey: end, }, diff --git a/pkg/kv/bulk/BUILD.bazel b/pkg/kv/bulk/BUILD.bazel index 9c81db6ee82c..56c397e49de3 100644 --- a/pkg/kv/bulk/BUILD.bazel +++ b/pkg/kv/bulk/BUILD.bazel @@ -17,6 +17,7 @@ go_library( "//pkg/kv", "//pkg/kv/bulk/bulkpb", "//pkg/kv/kvclient/rangecache", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/kvserverbase", "//pkg/roachpb", "//pkg/settings", @@ -54,6 +55,7 @@ go_test( "//pkg/kv", "//pkg/kv/kvclient/kvcoord", "//pkg/kv/kvclient/rangecache", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/kvserverbase", "//pkg/roachpb", "//pkg/security/securityassets", diff --git a/pkg/kv/bulk/buffering_adder.go b/pkg/kv/bulk/buffering_adder.go index 0aff47420ec6..97858f1058ab 100644 --- a/pkg/kv/bulk/buffering_adder.go +++ b/pkg/kv/bulk/buffering_adder.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/bulk/bulkpb" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangecache" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -60,7 +61,7 @@ type BufferingAdder struct { bulkMon *mon.BytesMonitor memAcc mon.BoundAccount - onFlush func(summary roachpb.BulkOpSummary) + onFlush func(summary kvpb.BulkOpSummary) // underfill tracks how much capacity was remaining in curBuf when it was // flushed due to size, e.g. how much its mis-allocated entries vs slab. underfill sz @@ -130,7 +131,7 @@ func MakeBulkAdder( } // SetOnFlush sets a callback to run after the buffering adder flushes. -func (b *BufferingAdder) SetOnFlush(fn func(summary roachpb.BulkOpSummary)) { +func (b *BufferingAdder) SetOnFlush(fn func(summary kvpb.BulkOpSummary)) { b.onFlush = fn } @@ -446,6 +447,6 @@ func (b *BufferingAdder) createInitialSplits(ctx context.Context) error { } // GetSummary returns this batcher's total added rows/bytes/etc. -func (b *BufferingAdder) GetSummary() roachpb.BulkOpSummary { +func (b *BufferingAdder) GetSummary() kvpb.BulkOpSummary { return b.sink.GetSummary() } diff --git a/pkg/kv/bulk/sst_batcher.go b/pkg/kv/bulk/sst_batcher.go index 92f7f75bb8ac..8703fbe1164e 100644 --- a/pkg/kv/bulk/sst_batcher.go +++ b/pkg/kv/bulk/sst_batcher.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/bulk/bulkpb" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangecache" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" @@ -103,7 +104,7 @@ type SSTBatcher struct { mem mon.BoundAccount limiter limit.ConcurrentRequestLimiter - // disallowShadowingBelow is described on roachpb.AddSSTableRequest. + // disallowShadowingBelow is described on kvpb.AddSSTableRequest. disallowShadowingBelow hlc.Timestamp // skips duplicate keys (iff they are buffered together). This is true when @@ -180,7 +181,7 @@ type SSTBatcher struct { syncutil.Mutex maxWriteTS hlc.Timestamp - totalRows roachpb.BulkOpSummary + totalRows kvpb.BulkOpSummary // totalStats contain the stats over the entire lifetime of the SST Batcher. // As rows accumulate, the corresponding stats initially start out in // currentStats. After each flush, the contents of currentStats are combined @@ -663,12 +664,12 @@ func (b *SSTBatcher) Close(ctx context.Context) { } // GetBatchSummary returns this batcher's total added rows/bytes/etc. -func (b *SSTBatcher) GetBatchSummary() roachpb.BulkOpSummary { +func (b *SSTBatcher) GetBatchSummary() kvpb.BulkOpSummary { return b.rowCounter.BulkOpSummary } // GetSummary returns this batcher's total added rows/bytes/etc. -func (b *SSTBatcher) GetSummary() roachpb.BulkOpSummary { +func (b *SSTBatcher) GetSummary() kvpb.BulkOpSummary { b.mu.Lock() defer b.mu.Unlock() return b.mu.totalRows @@ -744,8 +745,8 @@ func (b *SSTBatcher) addSSTable( ingestAsWriteBatch = true } - req := &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: item.start, EndKey: item.end}, + req := &kvpb.AddSSTableRequest{ + RequestHeader: kvpb.RequestHeader{Key: item.start, EndKey: item.end}, Data: item.sstBytes, DisallowShadowing: !b.disallowShadowingBelow.IsEmpty(), DisallowShadowingBelow: b.disallowShadowingBelow, @@ -757,12 +758,12 @@ func (b *SSTBatcher) addSSTable( req.SSTTimestampToRequestTimestamp = batchTS } - ba := &roachpb.BatchRequest{ - Header: roachpb.Header{Timestamp: batchTS, ClientRangeInfo: roachpb.ClientRangeInfo{ExplicitlyRequested: true}}, - AdmissionHeader: roachpb.AdmissionHeader{ + ba := &kvpb.BatchRequest{ + Header: kvpb.Header{Timestamp: batchTS, ClientRangeInfo: roachpb.ClientRangeInfo{ExplicitlyRequested: true}}, + AdmissionHeader: kvpb.AdmissionHeader{ Priority: int32(admissionpb.BulkNormalPri), CreateTime: timeutil.Now().UnixNano(), - Source: roachpb.AdmissionHeader_FROM_SQL, + Source: kvpb.AdmissionHeader_FROM_SQL, NoMemoryReservedAtSource: true, }, } @@ -784,7 +785,7 @@ func (b *SSTBatcher) addSSTable( } if pErr == nil { - resp := br.Responses[0].GetInner().(*roachpb.AddSSTableResponse) + resp := br.Responses[0].GetInner().(*kvpb.AddSSTableResponse) b.mu.Lock() if b.writeAtBatchTS { b.mu.maxWriteTS.Forward(br.Timestamp) @@ -809,12 +810,12 @@ func (b *SSTBatcher) addSSTable( err = pErr.GoError() // Retry on AmbiguousResult. - if errors.HasType(err, (*roachpb.AmbiguousResultError)(nil)) { + if errors.HasType(err, (*kvpb.AmbiguousResultError)(nil)) { log.Warningf(ctx, "addsstable [%s,%s) attempt %d failed: %+v", start, end, i, err) continue } // This range has split -- we need to split the SST to try again. - if m := (*roachpb.RangeKeyMismatchError)(nil); errors.As(err, &m) { + if m := (*kvpb.RangeKeyMismatchError)(nil); errors.As(err, &m) { // TODO(andrei): We just use the first of m.Ranges; presumably we // should be using all of them to avoid further retries. mr, err := m.MismatchedRange() diff --git a/pkg/kv/bulk/sst_batcher_test.go b/pkg/kv/bulk/sst_batcher_test.go index d199f9455fd6..4acb9d1b6d84 100644 --- a/pkg/kv/bulk/sst_batcher_test.go +++ b/pkg/kv/bulk/sst_batcher_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/bulk" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangecache" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" @@ -57,20 +58,20 @@ func TestDuplicateHandling(t *testing.T) { defer s.Stopper().Stop(ctx) expectRevisionCount := func(startKey roachpb.Key, endKey roachpb.Key, count int) { - req := &roachpb.ExportRequest{ - RequestHeader: roachpb.RequestHeader{ + req := &kvpb.ExportRequest{ + RequestHeader: kvpb.RequestHeader{ Key: startKey, EndKey: endKey, }, - MVCCFilter: roachpb.MVCCFilter_All, + MVCCFilter: kvpb.MVCCFilter_All, StartTime: hlc.Timestamp{}, } - header := roachpb.Header{Timestamp: s.Clock().Now()} + header := kvpb.Header{Timestamp: s.Clock().Now()} resp, err := kv.SendWrappedWith(ctx, kvDB.NonTransactionalSender(), header, req) require.NoError(t, err.GoError()) keyCount := 0 - for _, file := range resp.(*roachpb.ExportResponse).Files { + for _, file := range resp.(*kvpb.ExportResponse).Files { iterOpts := storage.IterOptions{ KeyTypes: storage.IterKeyTypePointsOnly, LowerBound: keys.LocalMax, diff --git a/pkg/kv/client_test.go b/pkg/kv/client_test.go index c15b7445ea8e..1fa7ebdf1256 100644 --- a/pkg/kv/client_test.go +++ b/pkg/kv/client_test.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -103,10 +104,10 @@ func TestClientRetryNonTxn(t *testing.T) { }{ m: make(map[string]struct{}), } - filter := func(args kvserverbase.FilterArgs) *roachpb.Error { + filter := func(args kvserverbase.FilterArgs) *kvpb.Error { mu.Lock() defer mu.Unlock() - pushArg, ok := args.Req.(*roachpb.PushTxnRequest) + pushArg, ok := args.Req.(*kvpb.PushTxnRequest) if !ok || !strings.HasPrefix(string(pushArg.PusheeTxn.Key), "key-") { return nil } @@ -126,16 +127,16 @@ func TestClientRetryNonTxn(t *testing.T) { defer s.Stopper().Stop(context.Background()) testCases := []struct { - args roachpb.Request + args kvpb.Request canPush bool expAttempts int }{ // Write/write conflicts. - {&roachpb.PutRequest{}, true, 2}, - {&roachpb.PutRequest{}, false, 1}, + {&kvpb.PutRequest{}, true, 2}, + {&kvpb.PutRequest{}, false, 1}, // Read/write conflicts. - {&roachpb.GetRequest{}, true, 1}, - {&roachpb.GetRequest{}, false, 1}, + {&kvpb.GetRequest{}, true, 1}, + {&kvpb.GetRequest{}, false, 1}, } // Lay down a write intent using a txn and attempt to access the same // key from our test client, with priorities set up so that the Push @@ -169,9 +170,9 @@ func TestClientRetryNonTxn(t *testing.T) { // We must try the non-txn put or get in a goroutine because // it might have to retry and will only succeed immediately in // the event we can push. - go func(i int, args roachpb.Request) { + go func(i int, args kvpb.Request) { var err error - if _, ok := args.(*roachpb.GetRequest); ok { + if _, ok := args.(*kvpb.GetRequest); ok { _, err = db.Get(nonTxnCtx, key) } else { err = db.Put(nonTxnCtx, key, "value") @@ -218,7 +219,7 @@ func TestClientRetryNonTxn(t *testing.T) { t.Fatalf("%d: expected success getting %q: %s", i, key, err) } - if _, isGet := test.args.(*roachpb.GetRequest); isGet || test.canPush { + if _, isGet := test.args.(*kvpb.GetRequest); isGet || test.canPush { if !bytes.Equal(gr.ValueBytes(), []byte("txn-value")) { t.Errorf("%d: expected \"txn-value\"; got %q", i, gr.ValueBytes()) } @@ -750,10 +751,10 @@ func TestReadConsistencyTypes(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, rc := range []roachpb.ReadConsistencyType{ - roachpb.CONSISTENT, - roachpb.READ_UNCOMMITTED, - roachpb.INCONSISTENT, + for _, rc := range []kvpb.ReadConsistencyType{ + kvpb.CONSISTENT, + kvpb.READ_UNCOMMITTED, + kvpb.INCONSISTENT, } { t.Run(rc.String(), func(t *testing.T) { ctx := context.Background() @@ -762,10 +763,10 @@ func TestReadConsistencyTypes(t *testing.T) { // Mock out DistSender's sender function to check the read consistency for // outgoing BatchRequests and return an empty reply. factory := kv.NonTransactionalFactoryFunc( - func(_ context.Context, ba *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, *roachpb.Error) { + func(_ context.Context, ba *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, *kvpb.Error) { if ba.ReadConsistency != rc { - return nil, roachpb.NewErrorf("BatchRequest has unexpected ReadConsistency %s", ba.ReadConsistency) + return nil, kvpb.NewErrorf("BatchRequest has unexpected ReadConsistency %s", ba.ReadConsistency) } return ba.CreateReply(), nil }) @@ -908,7 +909,7 @@ func TestNodeIDAndObservedTimestamps(t *testing.T) { // Mock out sender function to check that created transactions // have the observed timestamp set for the configured node ID. factory := kv.MakeMockTxnSenderFactory( - func(_ context.Context, _ *roachpb.Transaction, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + func(_ context.Context, _ *roachpb.Transaction, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { return ba.CreateReply(), nil }) @@ -1086,11 +1087,11 @@ func TestRollbackWithCanceledContextInsidious(t *testing.T) { key := roachpb.Key("a") ctx, cancel := context.WithCancel(context.Background()) var rollbacks int - storeKnobs.TestingRequestFilter = func(_ context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + storeKnobs.TestingRequestFilter = func(_ context.Context, ba *kvpb.BatchRequest) *kvpb.Error { if !ba.IsSingleEndTxnRequest() { return nil } - et := ba.Requests[0].GetInner().(*roachpb.EndTxnRequest) + et := ba.Requests[0].GetInner().(*kvpb.EndTxnRequest) if !et.Commit && et.Key.Equal(key) { rollbacks++ cancel() diff --git a/pkg/kv/db.go b/pkg/kv/db.go index 30279b1cc8d5..005f61fe4209 100644 --- a/pkg/kv/db.go +++ b/pkg/kv/db.go @@ -16,6 +16,7 @@ import ( "strings" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" @@ -144,7 +145,7 @@ type Result struct { ResumeSpan *roachpb.Span // When ResumeSpan is populated, this specifies the reason why the operation // wasn't completed and needs to be resumed. - ResumeReason roachpb.ResumeReason + ResumeReason kvpb.ResumeReason // ResumeNextBytes is the size of the next result when ResumeSpan is populated. ResumeNextBytes int64 } @@ -213,14 +214,14 @@ var _ Sender = &CrossRangeTxnWrapperSender{} // Send implements the Sender interface. func (s *CrossRangeTxnWrapperSender) Send( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, *kvpb.Error) { if ba.Txn != nil { log.Fatalf(ctx, "CrossRangeTxnWrapperSender can't handle transactional requests") } br, pErr := s.wrapped.Send(ctx, ba) - if _, ok := pErr.GetDetail().(*roachpb.OpRequiresTxnError); !ok { + if _, ok := pErr.GetDetail().(*kvpb.OpRequiresTxnError); !ok { return br, pErr } @@ -237,7 +238,7 @@ func (s *CrossRangeTxnWrapperSender) Send( return err }) if err != nil { - return nil, roachpb.NewError(err) + return nil, kvpb.NewError(err) } br.Txn = nil // hide the evidence return br, nil @@ -463,7 +464,7 @@ func (db *DB) scan( maxRows int64, isReverse bool, forUpdate bool, - readConsistency roachpb.ReadConsistencyType, + readConsistency kvpb.ReadConsistencyType, ) ([]KeyValue, error) { b := &Batch{} b.Header.ReadConsistency = readConsistency @@ -482,7 +483,7 @@ func (db *DB) scan( // // key can be either a byte slice or a string. func (db *DB) Scan(ctx context.Context, begin, end interface{}, maxRows int64) ([]KeyValue, error) { - return db.scan(ctx, begin, end, maxRows, false /* isReverse */, false /* forUpdate */, roachpb.CONSISTENT) + return db.scan(ctx, begin, end, maxRows, false /* isReverse */, false /* forUpdate */, kvpb.CONSISTENT) } // ScanForUpdate retrieves the rows between begin (inclusive) and end @@ -495,7 +496,7 @@ func (db *DB) Scan(ctx context.Context, begin, end interface{}, maxRows int64) ( func (db *DB) ScanForUpdate( ctx context.Context, begin, end interface{}, maxRows int64, ) ([]KeyValue, error) { - return db.scan(ctx, begin, end, maxRows, false /* isReverse */, true /* forUpdate */, roachpb.CONSISTENT) + return db.scan(ctx, begin, end, maxRows, false /* isReverse */, true /* forUpdate */, kvpb.CONSISTENT) } // ReverseScan retrieves the rows between begin (inclusive) and end (exclusive) @@ -507,7 +508,7 @@ func (db *DB) ScanForUpdate( func (db *DB) ReverseScan( ctx context.Context, begin, end interface{}, maxRows int64, ) ([]KeyValue, error) { - return db.scan(ctx, begin, end, maxRows, true /* isReverse */, false /* forUpdate */, roachpb.CONSISTENT) + return db.scan(ctx, begin, end, maxRows, true /* isReverse */, false /* forUpdate */, kvpb.CONSISTENT) } // ReverseScanForUpdate retrieves the rows between begin (inclusive) and end @@ -520,7 +521,7 @@ func (db *DB) ReverseScan( func (db *DB) ReverseScanForUpdate( ctx context.Context, begin, end interface{}, maxRows int64, ) ([]KeyValue, error) { - return db.scan(ctx, begin, end, maxRows, true /* isReverse */, true /* forUpdate */, roachpb.CONSISTENT) + return db.scan(ctx, begin, end, maxRows, true /* isReverse */, true /* forUpdate */, kvpb.CONSISTENT) } // Del deletes one or more keys. @@ -604,9 +605,9 @@ func (db *DB) AdminSplit( // if the range is large. func (db *DB) AdminScatter( ctx context.Context, key roachpb.Key, maxSize int64, -) (*roachpb.AdminScatterResponse, error) { - scatterReq := &roachpb.AdminScatterRequest{ - RequestHeader: roachpb.RequestHeaderFromSpan(roachpb.Span{Key: key, EndKey: key.Next()}), +) (*kvpb.AdminScatterResponse, error) { + scatterReq := &kvpb.AdminScatterRequest{ + RequestHeader: kvpb.RequestHeaderFromSpan(roachpb.Span{Key: key, EndKey: key.Next()}), RandomizeLeases: true, MaxSize: maxSize, } @@ -614,7 +615,7 @@ func (db *DB) AdminScatter( if pErr != nil { return nil, pErr.GoError() } - resp, ok := raw.(*roachpb.AdminScatterResponse) + resp, ok := raw.(*kvpb.AdminScatterResponse) if !ok { return nil, errors.Errorf("unexpected response of type %T for AdminScatter", raw) } @@ -668,7 +669,7 @@ func (db *DB) AdminChangeReplicas( ctx context.Context, key interface{}, expDesc roachpb.RangeDescriptor, - chgs []roachpb.ReplicationChange, + chgs []kvpb.ReplicationChange, ) (*roachpb.RangeDescriptor, error) { b := &Batch{} b.adminChangeReplicas(key, expDesc, chgs) @@ -679,7 +680,7 @@ func (db *DB) AdminChangeReplicas( if len(responses) == 0 { return nil, errors.Errorf("unexpected empty responses for AdminChangeReplicas") } - resp, ok := responses[0].GetInner().(*roachpb.AdminChangeReplicasResponse) + resp, ok := responses[0].GetInner().(*kvpb.AdminChangeReplicasResponse) if !ok { return nil, errors.Errorf("unexpected response of type %T for AdminChangeReplicas", responses[0].GetInner()) @@ -716,7 +717,7 @@ func (db *DB) AddSSTable( ingestAsWrites bool, batchTs hlc.Timestamp, ) (roachpb.Span, int64, error) { - b := &Batch{Header: roachpb.Header{Timestamp: batchTs}} + b := &Batch{Header: kvpb.Header{Timestamp: batchTs}} b.addSSTable(begin, end, data, disallowConflicts, disallowShadowing, disallowShadowingBelow, stats, ingestAsWrites, hlc.Timestamp{} /* sstTimestampToRequestTimestamp */) err := getOneErr(db.Run(ctx, b), b) @@ -747,7 +748,7 @@ func (db *DB) AddSSTableAtBatchTimestamp( ingestAsWrites bool, batchTs hlc.Timestamp, ) (hlc.Timestamp, roachpb.Span, int64, error) { - b := &Batch{Header: roachpb.Header{Timestamp: batchTs}} + b := &Batch{Header: kvpb.Header{Timestamp: batchTs}} b.addSSTable(begin, end, data, disallowConflicts, disallowShadowing, disallowShadowingBelow, stats, ingestAsWrites, batchTs) err := getOneErr(db.Run(ctx, b), b) @@ -784,7 +785,7 @@ func (db *DB) QueryResolvedTimestamp( b := &Batch{} b.queryResolvedTimestamp(begin, end) if nearest { - b.Header.RoutingPolicy = roachpb.RoutingPolicy_NEAREST + b.Header.RoutingPolicy = kvpb.RoutingPolicy_NEAREST } if err := getOneErr(db.Run(ctx, b), b); err != nil { return hlc.Timestamp{}, err @@ -806,7 +807,7 @@ func (db *DB) Barrier(ctx context.Context, begin, end interface{}) (hlc.Timestam if len(responses) == 0 { return hlc.Timestamp{}, errors.Errorf("unexpected empty response for Barrier") } - resp, ok := responses[0].GetInner().(*roachpb.BarrierResponse) + resp, ok := responses[0].GetInner().(*kvpb.BarrierResponse) if !ok { return hlc.Timestamp{}, errors.Errorf("unexpected response of type %T for Barrier", responses[0].GetInner()) @@ -823,14 +824,14 @@ func sendAndFill(ctx context.Context, send SenderFunc, b *Batch) error { // fails. But send() also returns its own errors, so there's some dancing // here to do because we want to run fillResults() so that the individual // result gets initialized with an error from the corresponding call. - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Requests = b.reqs ba.Header = b.Header ba.AdmissionHeader = b.AdmissionHeader b.response, b.pErr = send(ctx, ba) b.fillResults(ctx) if b.pErr == nil { - b.pErr = roachpb.NewError(b.resultErr()) + b.pErr = kvpb.NewError(b.resultErr()) } return b.pErr.GoError() } @@ -894,7 +895,7 @@ func (db *DB) NewTxn(ctx context.Context, debugName string) *Txn { // conscious about what they want. func (db *DB) Txn(ctx context.Context, retryable func(context.Context, *Txn) error) error { return db.TxnWithAdmissionControl( - ctx, roachpb.AdmissionHeader_OTHER, admissionpb.NormalPri, + ctx, kvpb.AdmissionHeader_OTHER, admissionpb.NormalPri, SteppingDisabled, retryable, ) } @@ -903,7 +904,7 @@ func (db *DB) Txn(ctx context.Context, retryable func(context.Context, *Txn) err // control source and priority. func (db *DB) TxnWithAdmissionControl( ctx context.Context, - source roachpb.AdmissionHeader_Source, + source kvpb.AdmissionHeader_Source, priority admissionpb.WorkPriority, steppingMode SteppingMode, retryable func(context.Context, *Txn) error, @@ -966,7 +967,7 @@ func runTxn(ctx context.Context, txn *Txn, retryable func(context.Context, *Txn) // Terminate TransactionRetryWithProtoRefreshError here, so it doesn't cause a higher-level // txn to be retried. We don't do this in any of the other functions in DB; I // guess we should. - if errors.HasType(err, (*roachpb.TransactionRetryWithProtoRefreshError)(nil)) { + if errors.HasType(err, (*kvpb.TransactionRetryWithProtoRefreshError)(nil)) { return errors.Wrapf(err, "terminated retryable error") } return err @@ -974,21 +975,19 @@ func runTxn(ctx context.Context, txn *Txn, retryable func(context.Context, *Txn) // send runs the specified calls synchronously in a single batch and returns // any errors. Returns (nil, nil) for an empty batch. -func (db *DB) send( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, *roachpb.Error) { +func (db *DB) send(ctx context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { return db.sendUsingSender(ctx, ba, db.NonTransactionalSender()) } // sendUsingSender uses the specified sender to send the batch request. func (db *DB) sendUsingSender( - ctx context.Context, ba *roachpb.BatchRequest, sender Sender, -) (*roachpb.BatchResponse, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, sender Sender, +) (*kvpb.BatchResponse, *kvpb.Error) { if len(ba.Requests) == 0 { return nil, nil } if err := ba.ReadConsistency.SupportsBatch(ba); err != nil { - return nil, roachpb.NewError(err) + return nil, kvpb.NewError(err) } if ba.UserPriority == 0 && db.ctx.UserPriority != 1 { ba.UserPriority = db.ctx.UserPriority @@ -1049,8 +1048,8 @@ func IncrementValRetryable(ctx context.Context, db *DB, key roachpb.Key, inc int var res KeyValue for r := retry.Start(base.DefaultRetryOptions()); r.Next(); { res, err = db.Inc(ctx, key, inc) - if errors.HasType(err, (*roachpb.UnhandledRetryableError)(nil)) || - errors.HasType(err, (*roachpb.AmbiguousResultError)(nil)) { + if errors.HasType(err, (*kvpb.UnhandledRetryableError)(nil)) || + errors.HasType(err, (*kvpb.AmbiguousResultError)(nil)) { continue } break diff --git a/pkg/kv/db_test.go b/pkg/kv/db_test.go index 867c50adaf9b..5e72d00ade19 100644 --- a/pkg/kv/db_test.go +++ b/pkg/kv/db_test.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" @@ -721,7 +722,7 @@ func TestGenerateForcedRetryableError(t *testing.T) { txn := db.NewTxn(ctx, "test: TestGenerateForcedRetryableError") require.Equal(t, 0, int(txn.Epoch())) err := txn.GenerateForcedRetryableError(ctx, "testing TestGenerateForcedRetryableError") - var retryErr *roachpb.TransactionRetryWithProtoRefreshError + var retryErr *kvpb.TransactionRetryWithProtoRefreshError require.True(t, errors.As(err, &retryErr)) require.Equal(t, 1, int(retryErr.Transaction.Epoch)) } @@ -769,12 +770,12 @@ func TestDB_TxnRetry(t *testing.T) { if runNumber == 0 { // First run, we should get a retryable error. require.Zero(t, runNumber) - require.IsType(t, &roachpb.TransactionRetryWithProtoRefreshError{}, err) + require.IsType(t, &kvpb.TransactionRetryWithProtoRefreshError{}, err) require.Equal(t, []byte(nil), r.ValueBytes()) // At this point txn is poisoned, and any op returns the same (poisoning) error. r, err = txn.Get(ctx, keyB) - require.IsType(t, &roachpb.TransactionRetryWithProtoRefreshError{}, err) + require.IsType(t, &kvpb.TransactionRetryWithProtoRefreshError{}, err) require.Equal(t, []byte(nil), r.ValueBytes()) } else { // The retry should succeed. @@ -839,8 +840,8 @@ func TestPreservingSteppingOnSenderReplacement(t *testing.T) { _, err := txn.Get(ctx, keyA) require.NotNil(t, err) - require.IsType(t, &roachpb.TransactionRetryWithProtoRefreshError{}, err) - pErr := (*roachpb.TransactionRetryWithProtoRefreshError)(nil) + require.IsType(t, &kvpb.TransactionRetryWithProtoRefreshError{}, err) + pErr := (*kvpb.TransactionRetryWithProtoRefreshError)(nil) require.ErrorAs(t, err, &pErr) require.Equal(t, txn.ID(), pErr.TxnID) diff --git a/pkg/kv/kvclient/BUILD.bazel b/pkg/kv/kvclient/BUILD.bazel index 225629ff8773..ea0689a8b271 100644 --- a/pkg/kv/kvclient/BUILD.bazel +++ b/pkg/kv/kvclient/BUILD.bazel @@ -13,6 +13,7 @@ go_library( deps = [ "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/storage", "//pkg/util/hlc", diff --git a/pkg/kv/kvclient/kvcoord/BUILD.bazel b/pkg/kv/kvclient/kvcoord/BUILD.bazel index 9fb193fbc55f..ced73d0b9909 100644 --- a/pkg/kv/kvclient/kvcoord/BUILD.bazel +++ b/pkg/kv/kvclient/kvcoord/BUILD.bazel @@ -47,6 +47,7 @@ go_library( "//pkg/kv", "//pkg/kv/kvbase", "//pkg/kv/kvclient/rangecache", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/closedts", "//pkg/kv/kvserver/concurrency/lock", "//pkg/kv/kvserver/txnwait", @@ -163,6 +164,8 @@ go_test( "//pkg/kv/kvbase", "//pkg/kv/kvclient/rangecache", "//pkg/kv/kvclient/rangecache/rangecachemock", + "//pkg/kv/kvpb", + "//pkg/kv/kvpb/kvpbmock", "//pkg/kv/kvserver", "//pkg/kv/kvserver/closedts", "//pkg/kv/kvserver/concurrency/lock", @@ -172,7 +175,6 @@ go_test( "//pkg/multitenant", "//pkg/multitenant/tenantcostmodel", "//pkg/roachpb", - "//pkg/roachpb/roachpbmock", # keep "//pkg/rpc", "//pkg/rpc/nodedialer", "//pkg/security/securityassets", diff --git a/pkg/kv/kvclient/kvcoord/batch.go b/pkg/kv/kvclient/kvcoord/batch.go index e36ca77de358..16f561679667 100644 --- a/pkg/kv/kvclient/kvcoord/batch.go +++ b/pkg/kv/kvclient/kvcoord/batch.go @@ -15,6 +15,7 @@ import ( "unsafe" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/buildutil" "github.com/cockroachdb/errors" @@ -80,7 +81,7 @@ type BatchTruncationHelper struct { scanDir ScanDirection // requests are the original requests this helper needs to process (possibly // in non-original order). - requests []roachpb.RequestUnion + requests []kvpb.RequestUnion // ownRequestsSlice indicates whether a separate slice was allocated for // requests. It is used for the purposes of the memory accounting. // @@ -113,7 +114,7 @@ type BatchTruncationHelper struct { // startKey.Next()). // // All keys in the headers are global. - headers []roachpb.RequestHeader + headers []kvpb.RequestHeader // positions stores the corresponding indices of requests in the original // requests slice. Once request is fully processed, it's position value // becomes negative. @@ -182,7 +183,7 @@ func (h descBatchTruncationHelper) Less(i, j int) bool { // mutated in any way. func NewBatchTruncationHelper( scanDir ScanDirection, - requests []roachpb.RequestUnion, + requests []kvpb.RequestUnion, mustPreserveOrder bool, canReorderRequestsSlice bool, ) (*BatchTruncationHelper, error) { @@ -196,7 +197,7 @@ func NewBatchTruncationHelper( // Init sets up the helper for the provided requests. It can be called multiple // times, and it will reuse as much internal allocations as possible. -func (h *BatchTruncationHelper) Init(requests []roachpb.RequestUnion) error { +func (h *BatchTruncationHelper) Init(requests []kvpb.RequestUnion) error { // Determine whether we can use the optimized strategy before making any // allocations. h.foundLocalKey = false @@ -215,7 +216,7 @@ func (h *BatchTruncationHelper) Init(requests []roachpb.RequestUnion) error { } else { // If we can't reorder the original requests slice, we must make a copy. if cap(h.requests) < len(requests) { - h.requests = make([]roachpb.RequestUnion, len(requests)) + h.requests = make([]kvpb.RequestUnion, len(requests)) h.ownRequestsSlice = true } else { if len(requests) < len(h.requests) { @@ -227,7 +228,7 @@ func (h *BatchTruncationHelper) Init(requests []roachpb.RequestUnion) error { // everything past the length is already nil-ed out. oldRequests := h.requests[len(requests):len(h.requests)] for i := range oldRequests { - oldRequests[i] = roachpb.RequestUnion{} + oldRequests[i] = kvpb.RequestUnion{} } } h.requests = h.requests[:len(requests)] @@ -235,7 +236,7 @@ func (h *BatchTruncationHelper) Init(requests []roachpb.RequestUnion) error { copy(h.requests, requests) } if cap(h.headers) < len(requests) { - h.headers = make([]roachpb.RequestHeader, len(requests)) + h.headers = make([]kvpb.RequestHeader, len(requests)) } else { if len(requests) < len(h.headers) { // Ensure that we lose references to the old header that will @@ -246,7 +247,7 @@ func (h *BatchTruncationHelper) Init(requests []roachpb.RequestUnion) error { // past the length is already nil-ed out. oldHeaders := h.headers[len(requests):len(h.headers)] for i := range oldHeaders { - oldHeaders[i] = roachpb.RequestHeader{} + oldHeaders[i] = kvpb.RequestHeader{} } } h.headers = h.headers[:len(requests)] @@ -267,7 +268,7 @@ func (h *BatchTruncationHelper) Init(requests []roachpb.RequestUnion) error { req := requests[i].GetInner() h.headers[i] = req.Header() h.positions[i] = i - h.isRange[i] = roachpb.IsRange(req) + h.isRange[i] = kvpb.IsRange(req) if h.isRange[i] { // We're dealing with a range-spanning request. if l, r := keys.IsLocal(h.headers[i].Key), keys.IsLocal(h.headers[i].EndKey); (l && !r) || (!l && r) { @@ -297,8 +298,8 @@ func (h *BatchTruncationHelper) Init(requests []roachpb.RequestUnion) error { } const ( - requestUnionOverhead = int64(unsafe.Sizeof(roachpb.RequestUnion{})) - requestHeaderOverhead = int64(unsafe.Sizeof(roachpb.RequestHeader{})) + requestUnionOverhead = int64(unsafe.Sizeof(kvpb.RequestUnion{})) + requestHeaderOverhead = int64(unsafe.Sizeof(kvpb.RequestHeader{})) intOverhead = int64(unsafe.Sizeof(int(0))) boolOverhead = int64(unsafe.Sizeof(false)) ) @@ -345,8 +346,8 @@ func (h *BatchTruncationHelper) MemUsage() int64 { // 2. rs is intersected with the current range boundaries. func (h *BatchTruncationHelper) Truncate( rs roachpb.RSpan, -) ([]roachpb.RequestUnion, []int, roachpb.RKey, error) { - var truncReqs []roachpb.RequestUnion +) ([]kvpb.RequestUnion, []int, roachpb.RKey, error) { + var truncReqs []kvpb.RequestUnion var positions []int var err error if !h.foundLocalKey { @@ -479,10 +480,8 @@ func (h *BatchTruncationHelper) Truncate( // extract out the differences into an interface; however, this leads to // non-trivial slowdown and increase in allocations, so we choose to duplicate // the code for performance. -func (h *BatchTruncationHelper) truncateAsc( - rs roachpb.RSpan, -) ([]roachpb.RequestUnion, []int, error) { - var truncReqs []roachpb.RequestUnion +func (h *BatchTruncationHelper) truncateAsc(rs roachpb.RSpan) ([]kvpb.RequestUnion, []int, error) { + var truncReqs []kvpb.RequestUnion var positions []int for i := h.startIdx; i < len(h.positions); i++ { pos := h.positions[i] @@ -506,7 +505,7 @@ func (h *BatchTruncationHelper) truncateAsc( // processed". truncReqs = append(truncReqs, h.requests[i]) positions = append(positions, pos) - h.headers[i] = roachpb.RequestHeader{} + h.headers[i] = kvpb.RequestHeader{} h.positions[i] = -1 continue } @@ -524,7 +523,7 @@ func (h *BatchTruncationHelper) truncateAsc( if header.EndKey.Compare(ek) <= 0 { // This is the last part of this request since it is fully contained // within this range, so we mark the request as "fully processed". - h.headers[i] = roachpb.RequestHeader{} + h.headers[i] = kvpb.RequestHeader{} h.positions[i] = -1 if origStartKey := inner.Header().Key; origStartKey.Equal(header.Key) { // This range-spanning request fits within a single range, so we @@ -541,7 +540,7 @@ func (h *BatchTruncationHelper) truncateAsc( } shallowCopy := inner.ShallowCopy() shallowCopy.SetHeader(header) - truncReqs = append(truncReqs, roachpb.RequestUnion{}) + truncReqs = append(truncReqs, kvpb.RequestUnion{}) truncReqs[len(truncReqs)-1].MustSetInner(shallowCopy) positions = append(positions, pos) } @@ -640,10 +639,8 @@ func (h *BatchTruncationHelper) truncateAsc( // extract out the differences into an interface; however, this leads to // non-trivial slowdown and increase in allocations, so we choose to duplicate // the code for performance. -func (h *BatchTruncationHelper) truncateDesc( - rs roachpb.RSpan, -) ([]roachpb.RequestUnion, []int, error) { - var truncReqs []roachpb.RequestUnion +func (h *BatchTruncationHelper) truncateDesc(rs roachpb.RSpan) ([]kvpb.RequestUnion, []int, error) { + var truncReqs []kvpb.RequestUnion var positions []int for i := h.startIdx; i < len(h.positions); i++ { pos := h.positions[i] @@ -667,7 +664,7 @@ func (h *BatchTruncationHelper) truncateDesc( // processed". truncReqs = append(truncReqs, h.requests[i]) positions = append(positions, pos) - h.headers[i] = roachpb.RequestHeader{} + h.headers[i] = kvpb.RequestHeader{} h.positions[i] = -1 continue } @@ -685,7 +682,7 @@ func (h *BatchTruncationHelper) truncateDesc( if header.Key.Compare(sk) >= 0 { // This is the last part of this request since it is fully contained // within this range, so we mark the request as "fully processed". - h.headers[i] = roachpb.RequestHeader{} + h.headers[i] = kvpb.RequestHeader{} h.positions[i] = -1 if origEndKey := inner.Header().EndKey; len(origEndKey) == 0 || origEndKey.Equal(header.EndKey) { // This range-spanning request fits within a single range, so we @@ -702,14 +699,14 @@ func (h *BatchTruncationHelper) truncateDesc( } shallowCopy := inner.ShallowCopy() shallowCopy.SetHeader(header) - truncReqs = append(truncReqs, roachpb.RequestUnion{}) + truncReqs = append(truncReqs, kvpb.RequestUnion{}) truncReqs[len(truncReqs)-1].MustSetInner(shallowCopy) positions = append(positions, pos) } return truncReqs, positions, nil } -var emptyHeader = roachpb.RequestHeader{} +var emptyHeader = kvpb.RequestHeader{} // truncateLegacy restricts all requests to the given key range and returns new, // truncated, requests. All returned requests are "truncated" to the given span, @@ -722,11 +719,11 @@ var emptyHeader = roachpb.RequestHeader{} // // then truncateLegacy(reqs,rs) returns (Put[a], Put[b]) and positions [0,2]. func truncateLegacy( - reqs []roachpb.RequestUnion, rs roachpb.RSpan, -) ([]roachpb.RequestUnion, []int, error) { - truncateOne := func(args roachpb.Request) (hasRequest bool, changed bool, _ roachpb.RequestHeader, _ error) { + reqs []kvpb.RequestUnion, rs roachpb.RSpan, +) ([]kvpb.RequestUnion, []int, error) { + truncateOne := func(args kvpb.Request) (hasRequest bool, changed bool, _ kvpb.RequestHeader, _ error) { header := args.Header() - if !roachpb.IsRange(args) { + if !kvpb.IsRange(args) { // This is a point request. if len(header.EndKey) > 0 { return false, false, emptyHeader, errors.AssertionFailedf("%T is not a range command, but EndKey is set", args) @@ -792,7 +789,7 @@ func truncateLegacy( // slice, only when something changed (copy-on-write). var positions []int - var truncReqs []roachpb.RequestUnion + var truncReqs []kvpb.RequestUnion for pos, arg := range reqs { inner := arg.GetInner() hasRequest, changed, newHeader, err := truncateOne(inner) @@ -801,7 +798,7 @@ func truncateLegacy( if changed { shallowCopy := inner.ShallowCopy() shallowCopy.SetHeader(newHeader) - truncReqs = append(truncReqs, roachpb.RequestUnion{}) + truncReqs = append(truncReqs, kvpb.RequestUnion{}) truncReqs[len(truncReqs)-1].MustSetInner(shallowCopy) } else { truncReqs = append(truncReqs, arg) @@ -854,7 +851,7 @@ func (h *BatchTruncationHelper) prev(k roachpb.RKey) (roachpb.RKey, error) { // Informally, a call `prevLegacy(reqs, k)` means: we've already executed the // parts of `reqs` that intersect `[k, KeyMax)`; please tell me how far to the // left the next relevant request begins. -func prevLegacy(reqs []roachpb.RequestUnion, k roachpb.RKey) (roachpb.RKey, error) { +func prevLegacy(reqs []kvpb.RequestUnion, k roachpb.RKey) (roachpb.RKey, error) { candidate := roachpb.RKeyMin for _, union := range reqs { inner := union.GetInner() @@ -952,7 +949,7 @@ func (h *BatchTruncationHelper) next(k roachpb.RKey) (roachpb.RKey, error) { // Informally, a call `nextLegacy(reqs, k)` means: we've already executed the // parts of `reqs` that intersect `[KeyMin, k)`; please tell me how far to the // right the next relevant request begins. -func nextLegacy(reqs []roachpb.RequestUnion, k roachpb.RKey) (roachpb.RKey, error) { +func nextLegacy(reqs []kvpb.RequestUnion, k roachpb.RKey) (roachpb.RKey, error) { candidate := roachpb.RKeyMax for _, union := range reqs { inner := union.GetInner() @@ -994,7 +991,7 @@ func nextLegacy(reqs []roachpb.RequestUnion, k roachpb.RKey) (roachpb.RKey, erro type orderRestorationHelper struct { // scratch is reused on the next call to restoreOrder() if it has enough // capacity. - scratch []roachpb.RequestUnion + scratch []kvpb.RequestUnion // found is used as a map indicating whether a request for a particular // positions value is included into the truncated requests and at what // index, -1 if the corresponding request is not found in the truncated @@ -1041,19 +1038,19 @@ func (h *orderRestorationHelper) memUsage() int64 { // 4. found[3] = 0 -> toReturn = [Get(b), Scan(a, c)] positions = [0, 3] // 5. found[4] = 2 -> toReturn = [Get(b), Scan(a, c), Scan(c, d)] positions = [0, 3, 4] func (h *orderRestorationHelper) restoreOrder( - truncReqs []roachpb.RequestUnion, positions []int, -) ([]roachpb.RequestUnion, []int) { + truncReqs []kvpb.RequestUnion, positions []int, +) ([]kvpb.RequestUnion, []int) { if len(truncReqs) == 0 { return truncReqs, positions } for i, pos := range positions { h.found[pos] = i } - var toReturn []roachpb.RequestUnion + var toReturn []kvpb.RequestUnion if cap(h.scratch) >= len(positions) { toReturn = h.scratch[:0] } else { - toReturn = make([]roachpb.RequestUnion, 0, len(positions)) + toReturn = make([]kvpb.RequestUnion, 0, len(positions)) } positions = positions[:0] for pos, found := range h.found { @@ -1066,7 +1063,7 @@ func (h *orderRestorationHelper) restoreOrder( positions = append(positions, pos) // Lose the reference to the request so that we can keep truncReqs as // the scratch space for the next call. - truncReqs[found] = roachpb.RequestUnion{} + truncReqs[found] = kvpb.RequestUnion{} // Make sure that the found map is set up for the next call. h.found[pos] = -1 } diff --git a/pkg/kv/kvclient/kvcoord/batch_test.go b/pkg/kv/kvclient/kvcoord/batch_test.go index 1c199d0cfa27..17f4ee795ede 100644 --- a/pkg/kv/kvclient/kvcoord/batch_test.go +++ b/pkg/kv/kvclient/kvcoord/batch_test.go @@ -20,6 +20,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -202,9 +203,9 @@ func TestBatchPrevNext(t *testing.T) { for _, test := range testCases { t.Run("", func(t *testing.T) { - var ba roachpb.BatchRequest + var ba kvpb.BatchRequest for _, span := range test.spans { - args := &roachpb.ScanRequest{} + args := &kvpb.ScanRequest{} args.Key, args.EndKey = span.Key, span.EndKey ba.Add(args) } @@ -237,7 +238,7 @@ func TestBatchPrevNext(t *testing.T) { } type requestsWithPositions struct { - reqs []roachpb.RequestUnion + reqs []kvpb.RequestUnion positions []int } @@ -375,23 +376,23 @@ func TestTruncate(t *testing.T) { } for i, test := range testCases { - goldenOriginal := roachpb.BatchRequest{} + goldenOriginal := kvpb.BatchRequest{} for _, ks := range test.keys { if len(ks[1]) > 0 { - goldenOriginal.Add(&roachpb.ResolveIntentRangeRequest{ - RequestHeader: roachpb.RequestHeader{ + goldenOriginal.Add(&kvpb.ResolveIntentRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: roachpb.Key(ks[0]), EndKey: roachpb.Key(ks[1]), }, IntentTxn: enginepb.TxnMeta{ID: uuid.MakeV4()}, }) } else { - goldenOriginal.Add(&roachpb.GetRequest{ - RequestHeader: roachpb.RequestHeader{Key: roachpb.Key(ks[0])}, + goldenOriginal.Add(&kvpb.GetRequest{ + RequestHeader: kvpb.RequestHeader{Key: roachpb.Key(ks[0])}, }) } } - original := roachpb.BatchRequest{Requests: make([]roachpb.RequestUnion, len(goldenOriginal.Requests))} + original := kvpb.BatchRequest{Requests: make([]kvpb.RequestUnion, len(goldenOriginal.Requests))} for i, request := range goldenOriginal.Requests { original.Requests[i].MustSetInner(request.GetInner().ShallowCopy()) } @@ -512,17 +513,17 @@ func TestTruncateLoop(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - makeGetRequest := func(key string) roachpb.RequestUnion { - var req roachpb.RequestUnion - req.MustSetInner(&roachpb.GetRequest{ - RequestHeader: roachpb.RequestHeader{Key: roachpb.Key(key)}, + makeGetRequest := func(key string) kvpb.RequestUnion { + var req kvpb.RequestUnion + req.MustSetInner(&kvpb.GetRequest{ + RequestHeader: kvpb.RequestHeader{Key: roachpb.Key(key)}, }) return req } - makeScanRequest := func(start, end string) roachpb.RequestUnion { - var req roachpb.RequestUnion - req.MustSetInner(&roachpb.ScanRequest{ - RequestHeader: roachpb.RequestHeader{ + makeScanRequest := func(start, end string) kvpb.RequestUnion { + var req kvpb.RequestUnion + req.MustSetInner(&kvpb.ScanRequest{ + RequestHeader: kvpb.RequestHeader{ Key: roachpb.Key(start), EndKey: roachpb.Key(end), }, }) @@ -535,8 +536,8 @@ func TestTruncateLoop(t *testing.T) { return string(randomKey(rng, keyLength)) } - makeRandomTestCase := func() ([]roachpb.RequestUnion, []roachpb.RSpan) { - var requests []roachpb.RequestUnion + makeRandomTestCase := func() ([]kvpb.RequestUnion, []roachpb.RSpan) { + var requests []kvpb.RequestUnion numRequests := rng.Intn(20) + 1 for i := 0; i < numRequests; i++ { if rng.Float64() < 0.5 { @@ -671,17 +672,17 @@ func BenchmarkTruncateLoop(b *testing.B) { "%s%s/reqs=%d/ranges=%d/type=%s", scanDir, orderStr, numRequests, numRanges, requestType, ), func(b *testing.B) { - reqs := make([]roachpb.RequestUnion, numRequests) + reqs := make([]kvpb.RequestUnion, numRequests) switch requestType { case "get": for i := 0; i < numRequests; i++ { - var get roachpb.GetRequest + var get kvpb.GetRequest get.Key = randomKey(rng, keyLength) reqs[i].MustSetInner(&get) } case "scan": for i := 0; i < numRequests; i++ { - var scan roachpb.ScanRequest + var scan kvpb.ScanRequest startKey := randomKey(rng, keyLength) endKey := randomKey(rng, keyLength) for bytes.Equal(startKey, endKey) { @@ -727,17 +728,17 @@ func BenchmarkTruncateLegacy(b *testing.B) { for _, numRequests := range []int{1 << 5, 1 << 10, 1 << 15} { for _, requestType := range []string{"get", "scan"} { b.Run(fmt.Sprintf("reqs=%d/type=%s", numRequests, requestType), func(b *testing.B) { - reqs := make([]roachpb.RequestUnion, numRequests) + reqs := make([]kvpb.RequestUnion, numRequests) switch requestType { case "get": for i := 0; i < numRequests; i++ { - var get roachpb.GetRequest + var get kvpb.GetRequest get.Key = []byte{randomKeyByte(rng)} reqs[i].MustSetInner(&get) } case "scan": for i := 0; i < numRequests; i++ { - var scan roachpb.ScanRequest + var scan kvpb.ScanRequest startKey := randomKeyByte(rng) endKey := randomKeyByte(rng) if endKey < startKey { diff --git a/pkg/kv/kvclient/kvcoord/dist_sender.go b/pkg/kv/kvclient/kvcoord/dist_sender.go index 5f34b5c47d6c..e9d75698bed2 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangecache" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/multitenant" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcostmodel" @@ -182,7 +183,7 @@ var CanSendToFollower = func( _ *cluster.Settings, _ *hlc.Clock, _ roachpb.RangeClosedTimestampPolicy, - _ *roachpb.BatchRequest, + _ *kvpb.BatchRequest, ) bool { return false } @@ -239,8 +240,8 @@ type DistSenderMetrics struct { RangefeedCatchupRanges *metric.Gauge RangefeedErrorCatchup *metric.Counter RangefeedRestartStuck *metric.Counter - MethodCounts [roachpb.NumMethods]*metric.Counter - ErrCounts [roachpb.NumErrors]*metric.Counter + MethodCounts [kvpb.NumMethods]*metric.Counter + ErrCounts [kvpb.NumErrors]*metric.Counter } func makeDistSenderMetrics() DistSenderMetrics { @@ -262,14 +263,14 @@ func makeDistSenderMetrics() DistSenderMetrics { RangefeedRestartStuck: metric.NewCounter(metaDistSenderRangefeedRestartStuck), } for i := range m.MethodCounts { - method := roachpb.Method(i).String() + method := kvpb.Method(i).String() meta := metaDistSenderMethodCountTmpl meta.Name = fmt.Sprintf(meta.Name, strings.ToLower(method)) meta.Help = fmt.Sprintf(meta.Help, method) m.MethodCounts[i] = metric.NewCounter(meta) } for i := range m.ErrCounts { - errType := roachpb.ErrorDetailType(i).String() + errType := kvpb.ErrorDetailType(i).String() meta := metaDistSenderErrCountTmpl meta.Name = fmt.Sprintf(meta.Name, strings.ToLower(errType)) meta.Help = fmt.Sprintf(meta.Help, errType) @@ -346,7 +347,7 @@ type DistSender struct { // LatencyFunc is used to estimate the latency to other nodes. latencyFunc LatencyFunc - onRangeSpanningNonTxnalBatch func(ba *roachpb.BatchRequest) *roachpb.Error + onRangeSpanningNonTxnalBatch func(ba *kvpb.BatchRequest) *kvpb.Error // locality is the description of the topography of the server on which the // DistSender is running. It is used to estimate the latency to other nodes @@ -558,7 +559,7 @@ func (ds *DistSender) RangeLookup( ) ([]roachpb.RangeDescriptor, []roachpb.RangeDescriptor, error) { ds.metrics.RangeLookups.Inc(1) switch rc { - case roachpb.INCONSISTENT, roachpb.READ_UNCOMMITTED: + case kvpb.INCONSISTENT, kvpb.READ_UNCOMMITTED: default: return nil, nil, errors.AssertionFailedf("invalid consistency level %v", rc) } @@ -660,9 +661,7 @@ func (ds *DistSender) getRoutingInfo( // initAndVerifyBatch initializes timestamp-related information and // verifies batch constraints before splitting. -func (ds *DistSender) initAndVerifyBatch( - ctx context.Context, ba *roachpb.BatchRequest, -) *roachpb.Error { +func (ds *DistSender) initAndVerifyBatch(ctx context.Context, ba *kvpb.BatchRequest) *kvpb.Error { // Attach the local node ID to each request. if ba.GatewayNodeID == 0 { ba.GatewayNodeID = ds.getNodeID() @@ -674,12 +673,12 @@ func (ds *DistSender) initAndVerifyBatch( // In the event that timestamp isn't set and read consistency isn't // required, set the timestamp using the local clock. - if ba.ReadConsistency != roachpb.CONSISTENT && ba.Timestamp.IsEmpty() { + if ba.ReadConsistency != kvpb.CONSISTENT && ba.Timestamp.IsEmpty() { ba.Timestamp = ba.Now.ToTimestamp() } if len(ba.Requests) < 1 { - return roachpb.NewErrorf("empty batch") + return kvpb.NewErrorf("empty batch") } if ba.MaxSpanRequestKeys != 0 || ba.TargetBytes != 0 { @@ -690,26 +689,26 @@ func (ds *DistSender) initAndVerifyBatch( for _, req := range ba.Requests { inner := req.GetInner() switch inner.(type) { - case *roachpb.ScanRequest, *roachpb.ResolveIntentRangeRequest, - *roachpb.DeleteRangeRequest, *roachpb.RevertRangeRequest, - *roachpb.ExportRequest, *roachpb.QueryLocksRequest, *roachpb.IsSpanEmptyRequest: + case *kvpb.ScanRequest, *kvpb.ResolveIntentRangeRequest, + *kvpb.DeleteRangeRequest, *kvpb.RevertRangeRequest, + *kvpb.ExportRequest, *kvpb.QueryLocksRequest, *kvpb.IsSpanEmptyRequest: // Accepted forward range requests. foundForward = true - case *roachpb.ReverseScanRequest: + case *kvpb.ReverseScanRequest: // Accepted reverse range requests. foundReverse = true - case *roachpb.QueryIntentRequest, *roachpb.EndTxnRequest, - *roachpb.GetRequest, *roachpb.DeleteRequest: + case *kvpb.QueryIntentRequest, *kvpb.EndTxnRequest, + *kvpb.GetRequest, *kvpb.DeleteRequest: // Accepted point requests that can be in batches with limit. default: - return roachpb.NewErrorf("batch with limit contains %s request", inner.Method()) + return kvpb.NewErrorf("batch with limit contains %s request", inner.Method()) } } if foundForward && foundReverse { - return roachpb.NewErrorf("batch with limit contains both forward and reverse scans") + return kvpb.NewErrorf("batch with limit contains both forward and reverse scans") } } @@ -719,17 +718,17 @@ func (ds *DistSender) initAndVerifyBatch( case lock.WaitPolicy_SkipLocked: for _, req := range ba.Requests { inner := req.GetInner() - if !roachpb.CanSkipLocked(inner) { + if !kvpb.CanSkipLocked(inner) { switch inner.(type) { - case *roachpb.QueryIntentRequest, *roachpb.EndTxnRequest: + case *kvpb.QueryIntentRequest, *kvpb.EndTxnRequest: // Not directly supported, but can be part of the same batch. default: - return roachpb.NewErrorf("batch with SkipLocked wait policy contains %s request", inner.Method()) + return kvpb.NewErrorf("batch with SkipLocked wait policy contains %s request", inner.Method()) } } } default: - return roachpb.NewErrorf("unknown wait policy %s", ba.WaitPolicy) + return kvpb.NewErrorf("unknown wait policy %s", ba.WaitPolicy) } return nil @@ -739,7 +738,7 @@ func (ds *DistSender) initAndVerifyBatch( // commit because it spans multiple ranges and must be split into at // least two parts, with the final part containing the EndTxn // request. -var errNo1PCTxn = roachpb.NewErrorf("cannot send 1PC txn to multiple ranges") +var errNo1PCTxn = kvpb.NewErrorf("cannot send 1PC txn to multiple ranges") // splitBatchAndCheckForRefreshSpans splits the batch according to the // canSplitET parameter and checks whether the batch can forward its @@ -747,8 +746,8 @@ var errNo1PCTxn = roachpb.NewErrorf("cannot send 1PC txn to multiple ranges") // set but is being split across multiple sub-batches then the flag in // the batch header is unset. func splitBatchAndCheckForRefreshSpans( - ba *roachpb.BatchRequest, canSplitET bool, -) [][]roachpb.RequestUnion { + ba *kvpb.BatchRequest, canSplitET bool, +) [][]kvpb.RequestUnion { parts := ba.Split(canSplitET) // If the batch is split and the header has its CanForwardReadTimestamp flag @@ -771,13 +770,13 @@ func splitBatchAndCheckForRefreshSpans( // different range would also need to refresh. Such behavior could cause // a transaction to observe an inconsistent snapshot and violate // serializability. -func unsetCanForwardReadTimestampFlag(ba *roachpb.BatchRequest) { +func unsetCanForwardReadTimestampFlag(ba *kvpb.BatchRequest) { if !ba.CanForwardReadTimestamp { // Already unset. return } for _, req := range ba.Requests { - if roachpb.NeedsRefresh(req.GetInner()) { + if kvpb.NeedsRefresh(req.GetInner()) { // Unset the flag. ba.CanForwardReadTimestamp = false return @@ -793,8 +792,8 @@ func unsetCanForwardReadTimestampFlag(ba *roachpb.BatchRequest) { // When the request spans ranges, it is split by range and a partial // subset of the batch request is sent to affected ranges in parallel. func (ds *DistSender) Send( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, *kvpb.Error) { ds.incrementBatchCounters(ba) if pErr := ds.initAndVerifyBatch(ctx, ba); pErr != nil { @@ -808,7 +807,7 @@ func (ds *DistSender) Send( splitET := false var require1PC bool lastReq := ba.Requests[len(ba.Requests)-1].GetInner() - if et, ok := lastReq.(*roachpb.EndTxnRequest); ok && et.Require1PC { + if et, ok := lastReq.(*kvpb.EndTxnRequest); ok && et.Require1PC { require1PC = true } // To ensure that we lay down intents to prevent starvation, always @@ -824,7 +823,7 @@ func (ds *DistSender) Send( log.Fatalf(ctx, "batch with MaxSpanRequestKeys=%d, TargetBytes=%d needs splitting", redact.Safe(ba.MaxSpanRequestKeys), redact.Safe(ba.TargetBytes)) } - var singleRplChunk [1]*roachpb.BatchResponse + var singleRplChunk [1]*kvpb.BatchResponse rplChunks := singleRplChunk[:0:1] onePart := len(parts) == 1 @@ -840,21 +839,21 @@ func (ds *DistSender) Send( // (for example, non-range requests with EndKey, or empty key ranges). rs, err := keys.Range(ba.Requests) if err != nil { - return nil, roachpb.NewError(err) + return nil, kvpb.NewError(err) } isReverse := ba.IsReverse() // Determine whether this part of the BatchRequest contains a committing // EndTxn request. var withCommit, withParallelCommit bool - if etArg, ok := ba.GetArg(roachpb.EndTxn); ok { - et := etArg.(*roachpb.EndTxnRequest) + if etArg, ok := ba.GetArg(kvpb.EndTxn); ok { + et := etArg.(*kvpb.EndTxnRequest) withCommit = et.Commit withParallelCommit = et.IsParallelCommit() } - var rpl *roachpb.BatchResponse - var pErr *roachpb.Error + var rpl *kvpb.BatchResponse + var pErr *kvpb.Error if withParallelCommit { rpl, pErr = ds.divideAndSendParallelCommit(ctx, ba, rs, isReverse, 0 /* batchIdx */) } else { @@ -893,7 +892,7 @@ func (ds *DistSender) Send( } } - var reply *roachpb.BatchResponse + var reply *kvpb.BatchResponse if len(rplChunks) > 0 { reply = rplChunks[0] for _, rpl := range rplChunks[1:] { @@ -910,7 +909,7 @@ func (ds *DistSender) Send( // incrementBatchCounters increments the appropriate counters to track the // batch and its composite request methods. -func (ds *DistSender) incrementBatchCounters(ba *roachpb.BatchRequest) { +func (ds *DistSender) incrementBatchCounters(ba *kvpb.BatchRequest) { ds.metrics.BatchCount.Inc(1) for _, ru := range ba.Requests { m := ru.GetInner().Method() @@ -919,9 +918,9 @@ func (ds *DistSender) incrementBatchCounters(ba *roachpb.BatchRequest) { } type response struct { - reply *roachpb.BatchResponse + reply *kvpb.BatchResponse positions []int - pErr *roachpb.Error + pErr *kvpb.Error } // divideAndSendParallelCommit divides a parallel-committing batch into @@ -948,14 +947,14 @@ type response struct { // method is never invoked recursively, but it is exposed to maintain symmetry // with divideAndSendBatchToRanges. func (ds *DistSender) divideAndSendParallelCommit( - ctx context.Context, ba *roachpb.BatchRequest, rs roachpb.RSpan, isReverse bool, batchIdx int, -) (br *roachpb.BatchResponse, pErr *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, rs roachpb.RSpan, isReverse bool, batchIdx int, +) (br *kvpb.BatchResponse, pErr *kvpb.Error) { // Search backwards, looking for the first pre-commit QueryIntent. swapIdx := -1 lastIdx := len(ba.Requests) - 1 for i := lastIdx - 1; i >= 0; i-- { req := ba.Requests[i].GetInner() - if req.Method() == roachpb.QueryIntent { + if req.Method() == kvpb.QueryIntent { swapIdx = i } else { break @@ -978,7 +977,7 @@ func (ds *DistSender) divideAndSendParallelCommit( // txnPipeliner.chainToInFlightWrites). These will introduce pipeline stalls // and undo most of the benefit of this method, but luckily they are rare in // practice. - swappedReqs := append([]roachpb.RequestUnion(nil), ba.Requests...) + swappedReqs := append([]kvpb.RequestUnion(nil), ba.Requests...) swappedReqs[swapIdx], swappedReqs[lastIdx] = swappedReqs[lastIdx], swappedReqs[swapIdx] // Create a new pre-commit QueryIntent-only batch and issue it @@ -988,7 +987,7 @@ func (ds *DistSender) divideAndSendParallelCommit( qiBa.Requests = swappedReqs[swapIdx+1:] qiRS, err := keys.Range(qiBa.Requests) if err != nil { - return br, roachpb.NewError(err) + return br, kvpb.NewError(err) } qiIsReverse := false // QueryIntentRequests do not carry the isReverse flag qiBatchIdx := batchIdx + 1 @@ -1022,7 +1021,7 @@ func (ds *DistSender) divideAndSendParallelCommit( reply, pErr := ds.divideAndSendBatchToRanges(ctx, qiBa, qiRS, qiIsReverse, true /* withCommit */, qiBatchIdx) qiResponseCh <- response{reply: reply, positions: positions, pErr: pErr} }); err != nil { - return nil, roachpb.NewError(err) + return nil, kvpb.NewError(err) } // Adjust the original batch request to ignore the pre-commit @@ -1032,7 +1031,7 @@ func (ds *DistSender) divideAndSendParallelCommit( ba.Requests = swappedReqs[:swapIdx+1] rs, err = keys.Range(ba.Requests) if err != nil { - return nil, roachpb.NewError(err) + return nil, kvpb.NewError(err) } // Note that we don't need to recompute isReverse for the updated batch // since we only separated out QueryIntentRequests which don't carry the @@ -1057,12 +1056,12 @@ func (ds *DistSender) divideAndSendParallelCommit( if qiPErr := qiReply.pErr; qiPErr != nil { // The batch with the pre-commit QueryIntent requests returned an error. ignoreMissing := false - if _, ok := qiPErr.GetDetail().(*roachpb.IntentMissingError); ok { + if _, ok := qiPErr.GetDetail().(*kvpb.IntentMissingError); ok { // If the error is an IntentMissingError, detect whether this is due // to intent resolution and can be safely ignored. ignoreMissing, err = ds.detectIntentMissingDueToIntentResolution(ctx, br.Txn) if err != nil { - return nil, roachpb.NewErrorWithTxn(err, br.Txn) + return nil, kvpb.NewErrorWithTxn(err, br.Txn) } } if !ignoreMissing { @@ -1079,12 +1078,12 @@ func (ds *DistSender) divideAndSendParallelCommit( } // Both halves of the split batch succeeded. Piece them back together. - resps := make([]roachpb.ResponseUnion, len(swappedReqs)) + resps := make([]kvpb.ResponseUnion, len(swappedReqs)) copy(resps, br.Responses) resps[swapIdx], resps[lastIdx] = resps[lastIdx], resps[swapIdx] br.Responses = resps if err := br.Combine(qiReply.reply, qiReply.positions); err != nil { - return nil, roachpb.NewError(err) + return nil, kvpb.NewError(err) } return br, nil } @@ -1113,10 +1112,10 @@ func (ds *DistSender) divideAndSendParallelCommit( func (ds *DistSender) detectIntentMissingDueToIntentResolution( ctx context.Context, txn *roachpb.Transaction, ) (bool, error) { - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Timestamp = ds.clock.Now() - ba.Add(&roachpb.QueryTxnRequest{ - RequestHeader: roachpb.RequestHeader{ + ba.Add(&kvpb.QueryTxnRequest{ + RequestHeader: kvpb.RequestHeader{ Key: txn.TxnMeta.Key, }, Txn: txn.TxnMeta, @@ -1127,7 +1126,7 @@ func (ds *DistSender) detectIntentMissingDueToIntentResolution( // We weren't able to determine whether the intent missing error is // due to intent resolution or not, so it is still ambiguous whether // the commit succeeded. - return false, roachpb.NewAmbiguousResultErrorf("error=%s [intent missing]", pErr) + return false, kvpb.NewAmbiguousResultErrorf("error=%s [intent missing]", pErr) } resp := br.Responses[0].GetQueryTxn() respTxn := &resp.QueriedTxn @@ -1151,9 +1150,9 @@ func (ds *DistSender) detectIntentMissingDueToIntentResolution( // commit because a committed record cannot be GC'ed and the recreated as // ABORTED. if resp.TxnRecordExists { - return false, roachpb.NewTransactionAbortedError(roachpb.ABORT_REASON_ABORTED_RECORD_FOUND) + return false, kvpb.NewTransactionAbortedError(kvpb.ABORT_REASON_ABORTED_RECORD_FOUND) } - return false, roachpb.NewAmbiguousResultErrorf("intent missing and record aborted") + return false, kvpb.NewAmbiguousResultErrorf("intent missing and record aborted") default: // The transaction has not been finalized yet, so the missing intent // error must have been caused by a real missing intent. Propagate the @@ -1166,7 +1165,7 @@ func (ds *DistSender) detectIntentMissingDueToIntentResolution( // maybeSwapErrorIndex swaps the error index from a to b or b to a if the // error's index is set and is equal to one of these to values. -func maybeSwapErrorIndex(pErr *roachpb.Error, a, b int) { +func maybeSwapErrorIndex(pErr *kvpb.Error, a, b int) { if pErr.Index == nil { return } @@ -1179,9 +1178,9 @@ func maybeSwapErrorIndex(pErr *roachpb.Error, a, b int) { // mergeErrors merges the two errors, combining their transaction state and // returning the error with the highest priority. -func mergeErrors(pErr1, pErr2 *roachpb.Error) *roachpb.Error { +func mergeErrors(pErr1, pErr2 *kvpb.Error) *kvpb.Error { ret, drop := pErr1, pErr2 - if roachpb.ErrPriority(drop.GoError()) > roachpb.ErrPriority(ret.GoError()) { + if kvpb.ErrPriority(drop.GoError()) > kvpb.ErrPriority(ret.GoError()) { ret, drop = drop, ret } ret.UpdateTxn(drop.GetTxn()) @@ -1209,12 +1208,12 @@ func mergeErrors(pErr1, pErr2 *roachpb.Error) *roachpb.Error { // this method is invoked recursively. func (ds *DistSender) divideAndSendBatchToRanges( ctx context.Context, - ba *roachpb.BatchRequest, + ba *kvpb.BatchRequest, rs roachpb.RSpan, isReverse bool, withCommit bool, batchIdx int, -) (br *roachpb.BatchResponse, pErr *roachpb.Error) { +) (br *kvpb.BatchResponse, pErr *kvpb.Error) { // Clone the BatchRequest's transaction so that future mutations to the // proto don't affect the proto in this batch. if ba.Txn != nil { @@ -1233,7 +1232,7 @@ func (ds *DistSender) divideAndSendBatchToRanges( ri := MakeRangeIterator(ds) ri.Seek(ctx, seekKey, scanDir) if !ri.Valid() { - return nil, roachpb.NewError(ri.Error()) + return nil, kvpb.NewError(ri.Error()) } // Take the fast path if this batch fits within a single range. if !ri.NeedAnother(rs) { @@ -1250,20 +1249,20 @@ func (ds *DistSender) divideAndSendBatchToRanges( // turning single-range queries into multi-range queries for no good // reason. if ba.IsUnsplittable() { - mismatch := roachpb.NewRangeKeyMismatchErrorWithCTPolicy(ctx, + mismatch := kvpb.NewRangeKeyMismatchErrorWithCTPolicy(ctx, rs.Key.AsRawKey(), rs.EndKey.AsRawKey(), ri.Desc(), nil, /* lease */ ri.ClosedTimestampPolicy(), ) - return nil, roachpb.NewError(mismatch) + return nil, kvpb.NewError(mismatch) } // If there's no transaction and ba spans ranges, possibly re-run as part of // a transaction for consistency. The case where we don't need to re-run is // if the read consistency is not required. if ba.Txn == nil { - if ba.IsTransactional() && ba.ReadConsistency == roachpb.CONSISTENT { + if ba.IsTransactional() && ba.ReadConsistency == kvpb.CONSISTENT { // NB: this check isn't quite right. We enter this if there's *any* transactional // request here, but there could be a mix (for example a DeleteRangeUsingTombstone // and a Put). DeleteRangeUsingTombstone gets split non-transactionally across @@ -1272,7 +1271,7 @@ func (ds *DistSender) divideAndSendBatchToRanges( // Revisit if this ever becomes something we actually want to do, for now such // batches will fail (re-wrapped in txn and then fail because some requests // don't support txns). - return nil, roachpb.NewError(&roachpb.OpRequiresTxnError{}) + return nil, kvpb.NewError(&kvpb.OpRequiresTxnError{}) } if fn := ds.onRangeSpanningNonTxnalBatch; fn != nil { if pErr := fn(ba); pErr != nil { @@ -1288,8 +1287,8 @@ func (ds *DistSender) divideAndSendBatchToRanges( // but even when that's true, we still need to search for the EndTxn in the // batch. if withCommit { - etArg, ok := ba.GetArg(roachpb.EndTxn) - if ok && !etArg.(*roachpb.EndTxnRequest).IsParallelCommit() { + etArg, ok := ba.GetArg(kvpb.EndTxn) + if ok && !etArg.(*kvpb.EndTxnRequest).IsParallelCommit() { return nil, errNo1PCTxn } } @@ -1298,8 +1297,8 @@ func (ds *DistSender) divideAndSendBatchToRanges( // Make an empty slice of responses which will be populated with responses // as they come in via Combine(). - br = &roachpb.BatchResponse{ - Responses: make([]roachpb.ResponseUnion, len(ba.Requests)), + br = &kvpb.BatchResponse{ + Responses: make([]kvpb.ResponseUnion, len(ba.Requests)), } // This function builds a channel of responses for each range // implicated in the span (rs) and combines them into a single @@ -1310,7 +1309,7 @@ func (ds *DistSender) divideAndSendBatchToRanges( // If couldHaveSkippedResponses is set, resumeReason indicates the reason why // the ResumeSpan is necessary. This reason is common to all individual // responses that carry a ResumeSpan. - var resumeReason roachpb.ResumeReason + var resumeReason kvpb.ResumeReason defer func() { if r := recover(); r != nil { // If we're in the middle of a panic, don't wait on responseChs. @@ -1342,7 +1341,7 @@ func (ds *DistSender) divideAndSendBatchToRanges( // the headers) if we haven't yet seen an error. if pErr == nil { if err := br.Combine(resp.reply, resp.positions); err != nil { - pErr = roachpb.NewError(err) + pErr = kvpb.NewError(err) } } else { // Update the error's transaction with any new information from @@ -1362,7 +1361,7 @@ func (ds *DistSender) divideAndSendBatchToRanges( if ba.IsSingleCheckConsistencyRequest() { // Don't parallelize full checksum requests as they have to touch the // entirety of each replica of each range they touch. - isExpensive := ba.Requests[0].GetCheckConsistency().Mode == roachpb.ChecksumMode_CHECK_FULL + isExpensive := ba.Requests[0].GetCheckConsistency().Mode == kvpb.ChecksumMode_CHECK_FULL canParallelize = canParallelize && !isExpensive } @@ -1381,7 +1380,7 @@ func (ds *DistSender) divideAndSendBatchToRanges( scanDir, ba.Requests, mustPreserveOrder, canReorderRequestsSlice, ) if err != nil { - return nil, roachpb.NewError(err) + return nil, kvpb.NewError(err) } // Iterate over the ranges that the batch touches. The iteration is done in // key order - the order of requests in the batch is not relevant for the @@ -1405,7 +1404,7 @@ func (ds *DistSender) divideAndSendBatchToRanges( // Truncate the request to range descriptor. curRangeRS, err := rs.Intersect(ri.Token().Desc().RSpan()) if err != nil { - responseCh <- response{pErr: roachpb.NewError(err)} + responseCh <- response{pErr: kvpb.NewError(err)} return } curRangeBatch := ba.ShallowCopy() @@ -1416,7 +1415,7 @@ func (ds *DistSender) divideAndSendBatchToRanges( err = errors.Newf("truncation resulted in empty batch on %s: %s", rs, ba) } if err != nil { - responseCh <- response{pErr: roachpb.NewError(err)} + responseCh <- response{pErr: kvpb.NewError(err)} return } nextRS := rs @@ -1470,7 +1469,7 @@ func (ds *DistSender) divideAndSendBatchToRanges( ba.MaxSpanRequestKeys -= replyKeys if ba.MaxSpanRequestKeys <= 0 { couldHaveSkippedResponses = true - resumeReason = roachpb.RESUME_KEY_LIMIT + resumeReason = kvpb.RESUME_KEY_LIMIT return } } @@ -1478,7 +1477,7 @@ func (ds *DistSender) divideAndSendBatchToRanges( ba.TargetBytes -= replyBytes if ba.TargetBytes <= 0 { couldHaveSkippedResponses = true - resumeReason = roachpb.RESUME_BYTE_LIMIT + resumeReason = kvpb.RESUME_BYTE_LIMIT return } } @@ -1486,7 +1485,7 @@ func (ds *DistSender) divideAndSendBatchToRanges( // do this after checking the limits, so that they take precedence. if ba.Header.ReturnOnRangeBoundary && replyKeys > 0 && !lastRange { couldHaveSkippedResponses = true - resumeReason = roachpb.RESUME_RANGE_BOUNDARY + resumeReason = kvpb.RESUME_RANGE_BOUNDARY return } } @@ -1507,7 +1506,7 @@ func (ds *DistSender) divideAndSendBatchToRanges( // We've exited early. Return the range iterator error. responseCh := make(chan response, 1) - responseCh <- response{pErr: roachpb.NewError(ri.Error())} + responseCh <- response{pErr: kvpb.NewError(ri.Error())} responseChs = append(responseChs, responseCh) return } @@ -1518,7 +1517,7 @@ func (ds *DistSender) divideAndSendBatchToRanges( // sent. func (ds *DistSender) sendPartialBatchAsync( ctx context.Context, - ba *roachpb.BatchRequest, + ba *kvpb.BatchRequest, rs roachpb.RSpan, isReverse bool, withCommit bool, @@ -1550,12 +1549,12 @@ func (ds *DistSender) sendPartialBatchAsync( func slowRangeRPCWarningStr( s *redact.StringBuilder, - ba *roachpb.BatchRequest, + ba *kvpb.BatchRequest, dur time.Duration, attempts int64, desc *roachpb.RangeDescriptor, err error, - br *roachpb.BatchResponse, + br *kvpb.BatchResponse, ) { resp := interface{}(err) if resp == nil { @@ -1591,7 +1590,7 @@ func slowRangeRPCReturnWarningStr(s *redact.StringBuilder, dur time.Duration, at // the ranges in the span and resend to each. func (ds *DistSender) sendPartialBatch( ctx context.Context, - ba *roachpb.BatchRequest, + ba *kvpb.BatchRequest, rs roachpb.RSpan, isReverse bool, withCommit bool, @@ -1604,8 +1603,8 @@ func (ds *DistSender) sendPartialBatch( } else if batchIdx > 1 { ds.metrics.PartialBatchCount.Inc(1) } - var reply *roachpb.BatchResponse - var pErr *roachpb.Error + var reply *kvpb.BatchResponse + var pErr *kvpb.Error var err error // Start a retry loop for sending the batch to the range. Each iteration of @@ -1641,7 +1640,7 @@ func (ds *DistSender) sendPartialBatch( log.VErrEventf(ctx, 1, "range descriptor re-lookup failed: %s", err) // We set pErr if we encountered an error getting the descriptor in // order to return the most recent error when we are out of retries. - pErr = roachpb.NewError(err) + pErr = kvpb.NewError(err) if !rangecache.IsRangeLookupErrorRetryable(err) { return response{pErr: pErr} } @@ -1655,7 +1654,7 @@ func (ds *DistSender) sendPartialBatch( // larger response slice with unknown mapping to our truncated reply). intersection, err := rs.Intersect(routingTok.Desc().RSpan()) if err != nil { - return response{pErr: roachpb.NewError(err)} + return response{pErr: kvpb.NewError(err)} } if !intersection.Equal(rs) { log.Eventf(ctx, "range shrunk; sub-dividing the request") @@ -1691,7 +1690,7 @@ func (ds *DistSender) sendPartialBatch( if err != nil { // Set pErr so that, if we don't perform any more retries, the // deduceRetryEarlyExitError() call below the loop is inhibited. - pErr = roachpb.NewError(err) + pErr = kvpb.NewError(err) switch { case errors.HasType(err, sendError{}): // We've tried all the replicas without success. Either they're all @@ -1742,7 +1741,7 @@ func (ds *DistSender) sendPartialBatch( // row and the range descriptor hasn't changed, return the error // to our caller. switch tErr := pErr.GetDetail().(type) { - case *roachpb.RangeKeyMismatchError: + case *kvpb.RangeKeyMismatchError: // Range descriptor might be out of date - evict it. This is likely the // result of a range split. If we have new range descriptors, insert them // instead. @@ -1751,7 +1750,7 @@ func (ds *DistSender) sendPartialBatch( // descriptor and putting it in the cache would be bad, as we'd go through // an infinite loops of retries. if routingTok.Desc().RSpan().Equal(ri.Desc.RSpan()) { - return response{pErr: roachpb.NewError(errors.AssertionFailedf( + return response{pErr: kvpb.NewError(errors.AssertionFailedf( "mismatched range suggestion not different from original desc. desc: %s. suggested: %s. err: %s", routingTok.Desc(), ri.Desc, pErr))} } @@ -1778,7 +1777,7 @@ func (ds *DistSender) sendPartialBatch( if err := ds.deduceRetryEarlyExitError(ctx); err == nil { log.Fatal(ctx, "exited retry loop without an error") } else { - pErr = roachpb.NewError(err) + pErr = kvpb.NewError(err) } } @@ -1789,7 +1788,7 @@ func (ds *DistSender) deduceRetryEarlyExitError(ctx context.Context) error { select { case <-ds.rpcRetryOptions.Closer: // Typically happens during shutdown. - return &roachpb.NodeUnavailableError{} + return &kvpb.NodeUnavailableError{} case <-ctx.Done(): // Happens when the client request is canceled. return errors.Wrap(ctx.Err(), "aborted in DistSender") @@ -1807,10 +1806,10 @@ func (ds *DistSender) deduceRetryEarlyExitError(ctx context.Context) error { // nextKey is the first key that was not processed. This will be used when // filling up the ResumeSpan's. func fillSkippedResponses( - ba *roachpb.BatchRequest, - br *roachpb.BatchResponse, + ba *kvpb.BatchRequest, + br *kvpb.BatchResponse, nextKey roachpb.RKey, - resumeReason roachpb.ResumeReason, + resumeReason kvpb.ResumeReason, isReverse bool, ) { // Some requests might have no response at all if we used a batch-wide @@ -1819,9 +1818,9 @@ func fillSkippedResponses( // limit, and add any other requests at higher keys at the end of the // batch -- they'll all come back without any response since they never // execute. - var scratchBA roachpb.BatchRequest + var scratchBA kvpb.BatchRequest for i := range br.Responses { - if br.Responses[i] != (roachpb.ResponseUnion{}) { + if br.Responses[i] != (kvpb.ResponseUnion{}) { continue } req := ba.Requests[i].GetInner() @@ -1829,9 +1828,9 @@ func fillSkippedResponses( // most efficient) way is to use (*BatchRequest).CreateReply. // // TODO(tschottdorf): can autogenerate CreateReply for individual - // requests, see roachpb/gen_batch.go. + // requests, see kv/kvpb/gen/main.go. if scratchBA.Requests == nil { - scratchBA.Requests = make([]roachpb.RequestUnion, 1) + scratchBA.Requests = make([]kvpb.RequestUnion, 1) } scratchBA.Requests[0].MustSetInner(req) br.Responses[i] = scratchBA.CreateReply().Responses[0] @@ -1854,9 +1853,9 @@ func fillSkippedResponses( // // nextKey is the first key that was not processed. func maybeSetResumeSpan( - req roachpb.Request, hdr *roachpb.ResponseHeader, nextKey roachpb.RKey, isReverse bool, + req kvpb.Request, hdr *kvpb.ResponseHeader, nextKey roachpb.RKey, isReverse bool, ) { - if _, ok := req.(*roachpb.GetRequest); ok { + if _, ok := req.(*kvpb.GetRequest); ok { // This is a Get request. There are three possibilities: // // 1. The request was completed. In this case we don't want a ResumeSpan. @@ -1888,7 +1887,7 @@ func maybeSetResumeSpan( return } - if !roachpb.IsRange(req) { + if !kvpb.IsRange(req) { return } @@ -1940,7 +1939,7 @@ func maybeSetResumeSpan( // the error that the last attempt to execute the request returned. func noMoreReplicasErr(ambiguousErr, lastAttemptErr error) error { if ambiguousErr != nil { - return roachpb.NewAmbiguousResultErrorf("error=%s [exhausted]", ambiguousErr) + return kvpb.NewAmbiguousResultErrorf("error=%s [exhausted]", ambiguousErr) } // TODO(bdarnell): The error from the last attempt is not necessarily the best @@ -1960,7 +1959,7 @@ const defaultSendClosedTimestampPolicy = roachpb.LEAD_FOR_GLOBAL_READS // sendToReplicas sends a batch to the replicas of a range. Replicas are tried one // at a time (generally the leaseholder first). The result of this call is // either a BatchResponse or an error. In the former case, the BatchResponse -// wraps either a response or a *roachpb.Error; this error will come from a +// wraps either a response or a *kvpb.Error; this error will come from a // replica authorized to evaluate the request (for example ConditionFailedError) // and can be seen as "data" returned from the request. In the latter case, // DistSender was unable to get a response from a replica willing to evaluate @@ -1984,30 +1983,30 @@ const defaultSendClosedTimestampPolicy = roachpb.LEAD_FOR_GLOBAL_READS // that do not definitively rule out the possibility that the batch could have // succeeded are transformed into AmbiguousResultErrors. func (ds *DistSender) sendToReplicas( - ctx context.Context, ba *roachpb.BatchRequest, routing rangecache.EvictionToken, withCommit bool, -) (*roachpb.BatchResponse, error) { + ctx context.Context, ba *kvpb.BatchRequest, routing rangecache.EvictionToken, withCommit bool, +) (*kvpb.BatchResponse, error) { // If this request can be sent to a follower to perform a consistent follower // read under the closed timestamp, promote its routing policy to NEAREST. // If we don't know the closed timestamp policy, we ought to optimistically // assume that it's LEAD_FOR_GLOBAL_READS, because if it is, and we assumed // otherwise, we may send a request to a remote region unnecessarily. - if ba.RoutingPolicy == roachpb.RoutingPolicy_LEASEHOLDER && + if ba.RoutingPolicy == kvpb.RoutingPolicy_LEASEHOLDER && CanSendToFollower( ds.logicalClusterID.Get(), ds.st, ds.clock, routing.ClosedTimestampPolicy(defaultSendClosedTimestampPolicy), ba, ) { ba = ba.ShallowCopy() - ba.RoutingPolicy = roachpb.RoutingPolicy_NEAREST + ba.RoutingPolicy = kvpb.RoutingPolicy_NEAREST } // Filter the replicas to only those that are relevant to the routing policy. // NB: When changing leaseholder policy constraint_status_report should be // updated appropriately. var replicaFilter ReplicaSliceFilter switch ba.RoutingPolicy { - case roachpb.RoutingPolicy_LEASEHOLDER: + case kvpb.RoutingPolicy_LEASEHOLDER: replicaFilter = OnlyPotentialLeaseholders - case roachpb.RoutingPolicy_NEAREST: + case kvpb.RoutingPolicy_NEAREST: replicaFilter = AllExtantReplicas default: log.Fatalf(ctx, "unknown routing policy: %s", ba.RoutingPolicy) @@ -2023,7 +2022,7 @@ func (ds *DistSender) sendToReplicas( // policy. var leaseholderFirst bool switch ba.RoutingPolicy { - case roachpb.RoutingPolicy_LEASEHOLDER: + case kvpb.RoutingPolicy_LEASEHOLDER: // First order by latency, then move the leaseholder to the front of the // list, if it is known. if !ds.dontReorderReplicas { @@ -2043,7 +2042,7 @@ func (ds *DistSender) sendToReplicas( log.VEvent(ctx, 2, "routing to nearest replica; leaseholder not known") } - case roachpb.RoutingPolicy_NEAREST: + case kvpb.RoutingPolicy_NEAREST: // Order by latency. log.VEvent(ctx, 2, "routing to nearest replica; leaseholder not required") replicas.OptimizeReplicaOrder(ds.getNodeID(), ds.latencyFunc, ds.locality) @@ -2082,7 +2081,7 @@ func (ds *DistSender) sendToReplicas( // This loop will retry operations that fail with errors that reflect // per-replica state and may succeed on other replicas. var ambiguousError error - var br *roachpb.BatchResponse + var br *kvpb.BatchResponse for first := true; ; first = false { if !first { ds.metrics.NextReplicaErrCount.Inc(1) @@ -2146,7 +2145,7 @@ func (ds *DistSender) sendToReplicas( if grpcutil.IsAuthError(err) { // Authentication or authorization error. Propagate. if ambiguousError != nil { - return nil, roachpb.NewAmbiguousResultErrorf("error=%s [propagate]", ambiguousError) + return nil, kvpb.NewAmbiguousResultErrorf("error=%s [propagate]", ambiguousError) } return nil, err } @@ -2272,10 +2271,10 @@ func (ds *DistSender) sendToReplicas( // different replica without updating our caches. This means that future // requests will attempt the same useless replicas. switch tErr := br.Error.GetDetail().(type) { - case *roachpb.StoreNotFoundError, *roachpb.NodeUnavailableError: + case *kvpb.StoreNotFoundError, *kvpb.NodeUnavailableError: // These errors are likely to be unique to the replica that reported // them, so no action is required before the next retry. - case *roachpb.RangeNotFoundError: + case *kvpb.RangeNotFoundError: // The store we routed to doesn't have this replica. This can happen when // our descriptor is outright outdated, but it can also be caused by a // replica that has just been added but needs a snapshot to be caught up. @@ -2283,7 +2282,7 @@ func (ds *DistSender) sendToReplicas( // We'll try other replicas which typically gives us the leaseholder, either // via the NotLeaseHolderError or nil error paths, both of which update the // leaseholder in the range cache. - case *roachpb.NotLeaseHolderError: + case *kvpb.NotLeaseHolderError: ds.metrics.NotLeaseHolderErrCount.Inc(1) // If we got some lease information, we use it. If not, we loop around // and try the next replica. @@ -2365,7 +2364,7 @@ func (ds *DistSender) sendToReplicas( } default: if ambiguousError != nil { - return nil, roachpb.NewAmbiguousResultErrorf("error=%s [propagate]", ambiguousError) + return nil, kvpb.NewAmbiguousResultErrorf("error=%s [propagate]", ambiguousError) } // The error received is likely not specific to this @@ -2385,7 +2384,7 @@ func (ds *DistSender) sendToReplicas( // sender changed its mind or the request timed out. if ambiguousError != nil { - err = roachpb.NewAmbiguousResultError(errors.Wrapf(ambiguousError, "context done during DistSender.Send")) + err = kvpb.NewAmbiguousResultError(errors.Wrapf(ambiguousError, "context done during DistSender.Send")) } else { err = errors.Wrap(ctx.Err(), "aborted during DistSender.Send") } @@ -2517,14 +2516,14 @@ func (ds *DistSender) computeSendRUMultiplier( return res } -func (ds *DistSender) maybeIncrementErrCounters(br *roachpb.BatchResponse, err error) { +func (ds *DistSender) maybeIncrementErrCounters(br *kvpb.BatchResponse, err error) { if err == nil && br.Error == nil { return } if err != nil { - ds.metrics.ErrCounts[roachpb.CommunicationErrType].Inc(1) + ds.metrics.ErrCounts[kvpb.CommunicationErrType].Inc(1) } else { - typ := roachpb.InternalErrType + typ := kvpb.InternalErrType if detail := br.Error.GetDetail(); detail != nil { typ = detail.Type() } diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_mux_rangefeed.go b/pkg/kv/kvclient/kvcoord/dist_sender_mux_rangefeed.go index b6a82bed3777..fb97943a36d0 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_mux_rangefeed.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_mux_rangefeed.go @@ -16,6 +16,7 @@ import ( "sync/atomic" "unsafe" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" @@ -32,7 +33,7 @@ import ( // each range feed request on an appropriate node. type rangefeedMuxer struct { // eventCh receives events from all active muxStreams. - eventCh chan *roachpb.MuxRangeFeedEvent + eventCh chan *kvpb.MuxRangeFeedEvent // Context group controlling execution of MuxRangeFeed calls. When this group // cancels, the entire muxer shuts down. The goroutines started in `g` will @@ -68,7 +69,7 @@ type muxClientState struct { doneCtx terminationContext // signaled when client shuts down. // RPC state. Valid only after initCtx.Done(). - client roachpb.Internal_MuxRangeFeedClient + client kvpb.Internal_MuxRangeFeedClient cancel context.CancelFunc // Number of consumers (ranges) running on this node; accessed under rangefeedMuxer lock. @@ -77,7 +78,7 @@ type muxClientState struct { func newRangefeedMuxer(g ctxgroup.Group) *rangefeedMuxer { m := &rangefeedMuxer{ - eventCh: make(chan *roachpb.MuxRangeFeedEvent), + eventCh: make(chan *kvpb.MuxRangeFeedEvent), demuxLoopDone: make(chan struct{}), g: g, } @@ -104,14 +105,14 @@ type channelRangeFeedEventProducer struct { callerCtx context.Context muxClientCtx terminationContext - streamID int64 // stream ID for this producer. - eventCh chan *roachpb.RangeFeedEvent // consumer event channel. + streamID int64 // stream ID for this producer. + eventCh chan *kvpb.RangeFeedEvent // consumer event channel. } -var _ roachpb.RangeFeedEventProducer = (*channelRangeFeedEventProducer)(nil) +var _ kvpb.RangeFeedEventProducer = (*channelRangeFeedEventProducer)(nil) // Recv implements rangeFeedEventProducer interface. -func (c *channelRangeFeedEventProducer) Recv() (*roachpb.RangeFeedEvent, error) { +func (c *channelRangeFeedEventProducer) Recv() (*kvpb.RangeFeedEvent, error) { select { case <-c.callerCtx.Done(): return nil, c.callerCtx.Err() @@ -126,8 +127,8 @@ func (c *channelRangeFeedEventProducer) Recv() (*roachpb.RangeFeedEvent, error) // RangeFeedRequest. // The passed in client is only needed to establish MuxRangeFeed RPC. func (m *rangefeedMuxer) startMuxRangeFeed( - ctx context.Context, client rpc.RestrictedInternalClient, req *roachpb.RangeFeedRequest, -) (roachpb.RangeFeedEventProducer, func(), error) { + ctx context.Context, client rpc.RestrictedInternalClient, req *kvpb.RangeFeedRequest, +) (kvpb.RangeFeedEventProducer, func(), error) { ms, err := m.establishMuxConnection(ctx, client, req.Replica.NodeID) if err != nil { return nil, nil, err @@ -139,7 +140,7 @@ func (m *rangefeedMuxer) startMuxRangeFeed( callerCtx: streamCtx, muxClientCtx: ms.doneCtx, streamID: req.StreamID, - eventCh: make(chan *roachpb.RangeFeedEvent), + eventCh: make(chan *kvpb.RangeFeedEvent), } m.producers.Store(req.StreamID, unsafe.Pointer(producer)) diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go b/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go index e25dcc700671..2c75ef7c72ab 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangecache" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" @@ -291,7 +292,7 @@ type activeRangeFeed struct { } func (a *activeRangeFeed) onRangeEvent( - nodeID roachpb.NodeID, rangeID roachpb.RangeID, event *roachpb.RangeFeedEvent, + nodeID roachpb.NodeID, rangeID roachpb.RangeID, event *kvpb.RangeFeedEvent, ) { a.Lock() defer a.Unlock() @@ -435,8 +436,8 @@ func (ds *DistSender) partialRangeFeed( span, startAfter, timeutil.Since(startAfter.GoTime()), err) } switch { - case errors.HasType(err, (*roachpb.StoreNotFoundError)(nil)) || - errors.HasType(err, (*roachpb.NodeUnavailableError)(nil)): + case errors.HasType(err, (*kvpb.StoreNotFoundError)(nil)) || + errors.HasType(err, (*kvpb.NodeUnavailableError)(nil)): // These errors are likely to be unique to the replica that // reported them, so no action is required before the next // retry. @@ -451,31 +452,31 @@ func (ds *DistSender) partialRangeFeed( token.Evict(ctx) token = rangecache.EvictionToken{} continue - case IsSendError(err), errors.HasType(err, (*roachpb.RangeNotFoundError)(nil)): + case IsSendError(err), errors.HasType(err, (*kvpb.RangeNotFoundError)(nil)): // Evict the descriptor from the cache and reload on next attempt. token.Evict(ctx) token = rangecache.EvictionToken{} continue - case errors.HasType(err, (*roachpb.RangeKeyMismatchError)(nil)): + case errors.HasType(err, (*kvpb.RangeKeyMismatchError)(nil)): // Evict the descriptor from the cache. token.Evict(ctx) return ds.divideAndSendRangeFeedToRanges(ctx, rs, startAfter, rangeCh) - case errors.HasType(err, (*roachpb.RangeFeedRetryError)(nil)): - var t *roachpb.RangeFeedRetryError + case errors.HasType(err, (*kvpb.RangeFeedRetryError)(nil)): + var t *kvpb.RangeFeedRetryError if ok := errors.As(err, &t); !ok { return errors.AssertionFailedf("wrong error type: %T", err) } switch t.Reason { - case roachpb.RangeFeedRetryError_REASON_REPLICA_REMOVED, - roachpb.RangeFeedRetryError_REASON_RAFT_SNAPSHOT, - roachpb.RangeFeedRetryError_REASON_LOGICAL_OPS_MISSING, - roachpb.RangeFeedRetryError_REASON_SLOW_CONSUMER: + case kvpb.RangeFeedRetryError_REASON_REPLICA_REMOVED, + kvpb.RangeFeedRetryError_REASON_RAFT_SNAPSHOT, + kvpb.RangeFeedRetryError_REASON_LOGICAL_OPS_MISSING, + kvpb.RangeFeedRetryError_REASON_SLOW_CONSUMER: // Try again with same descriptor. These are transient // errors that should not show up again. continue - case roachpb.RangeFeedRetryError_REASON_RANGE_SPLIT, - roachpb.RangeFeedRetryError_REASON_RANGE_MERGED, - roachpb.RangeFeedRetryError_REASON_NO_LEASEHOLDER: + case kvpb.RangeFeedRetryError_REASON_RANGE_SPLIT, + kvpb.RangeFeedRetryError_REASON_RANGE_MERGED, + kvpb.RangeFeedRetryError_REASON_NO_LEASEHOLDER: // Evict the descriptor from the cache. token.Evict(ctx) return ds.divideAndSendRangeFeedToRanges(ctx, rs, startAfter, rangeCh) @@ -492,7 +493,7 @@ func (ds *DistSender) partialRangeFeed( // onRangeEventCb is invoked for each non-error range event. // nodeID identifies the node ID which generated the event. -type onRangeEventCb func(nodeID roachpb.NodeID, rangeID roachpb.RangeID, event *roachpb.RangeFeedEvent) +type onRangeEventCb func(nodeID roachpb.NodeID, rangeID roachpb.RangeID, event *kvpb.RangeFeedEvent) // singleRangeFeed gathers and rearranges the replicas, and makes a RangeFeed // RPC call. Results will be sent on the provided channel. Returns the timestamp @@ -525,19 +526,19 @@ func (ds *DistSender) singleRangeFeed( if cfg.overSystemTable { admissionPri = admissionpb.NormalPri } - args := roachpb.RangeFeedRequest{ + args := kvpb.RangeFeedRequest{ Span: span, - Header: roachpb.Header{ + Header: kvpb.Header{ Timestamp: startAfter, RangeID: desc.RangeID, }, WithDiff: withDiff, - AdmissionHeader: roachpb.AdmissionHeader{ + AdmissionHeader: kvpb.AdmissionHeader{ // NB: AdmissionHeader is used only at the start of the range feed // stream since the initial catch-up scan is expensive. Priority: int32(admissionPri), CreateTime: timeutil.Now().UnixNano(), - Source: roachpb.AdmissionHeader_FROM_SQL, + Source: kvpb.AdmissionHeader_FROM_SQL, NoMemoryReservedAtSource: true, }, } @@ -624,7 +625,7 @@ func (ds *DistSender) singleRangeFeed( ctx = logtags.AddTag(ctx, "dest_r", args.RangeID) ctx, restore := pprofutil.SetProfilerLabelsFromCtxTags(ctx) - var stream roachpb.RangeFeedEventProducer + var stream kvpb.RangeFeedEventProducer stream, streamCleanup, err = streamProducerFactory(ctx, client, &args) if err != nil { restore() @@ -643,7 +644,7 @@ func (ds *DistSender) singleRangeFeed( } } - var event *roachpb.RangeFeedEvent + var event *kvpb.RangeFeedEvent for { if err := stuckWatcher.do(func() (err error) { event, err = stream.Recv() @@ -662,7 +663,7 @@ func (ds *DistSender) singleRangeFeed( msg := RangeFeedMessage{RangeFeedEvent: event, RegisteredSpan: span} switch t := event.GetValue().(type) { - case *roachpb.RangeFeedCheckpoint: + case *kvpb.RangeFeedCheckpoint: if t.Span.Contains(args.Span) { // If we see the first non-empty checkpoint, we know we're done with the catchup scan. if !t.ResolvedTS.IsEmpty() && catchupRes != nil { @@ -675,8 +676,8 @@ func (ds *DistSender) singleRangeFeed( // that timestamp. args.Timestamp.Forward(t.ResolvedTS) } - case *roachpb.RangeFeedSSTable: - case *roachpb.RangeFeedError: + case *kvpb.RangeFeedSSTable: + case *kvpb.RangeFeedError: log.VErrEventf(ctx, 2, "RangeFeedError: %s", t.Error.GoError()) if catchupRes != nil { ds.metrics.RangefeedErrorCatchup.Inc(1) @@ -711,21 +712,21 @@ func connectionClass(sv *settings.Values) rpc.ConnectionClass { type rangeFeedEventProducerFactory func( ctx context.Context, client rpc.RestrictedInternalClient, - req *roachpb.RangeFeedRequest, -) (roachpb.RangeFeedEventProducer, func(), error) + req *kvpb.RangeFeedRequest, +) (kvpb.RangeFeedEventProducer, func(), error) // legacyRangeFeedEventProducer is a rangeFeedEventProducerFactory using // legacy RangeFeed RPC. func legacyRangeFeedEventProducer( - ctx context.Context, client rpc.RestrictedInternalClient, req *roachpb.RangeFeedRequest, -) (producer roachpb.RangeFeedEventProducer, cleanup func(), err error) { + ctx context.Context, client rpc.RestrictedInternalClient, req *kvpb.RangeFeedRequest, +) (producer kvpb.RangeFeedEventProducer, cleanup func(), err error) { cleanup = func() {} producer, err = client.RangeFeed(ctx, req) return producer, cleanup, err } func (ds *DistSender) handleStuckEvent( - args *roachpb.RangeFeedRequest, afterCatchupScan bool, threshold time.Duration, + args *kvpb.RangeFeedRequest, afterCatchupScan bool, threshold time.Duration, ) error { ds.metrics.RangefeedRestartStuck.Inc(1) if afterCatchupScan { diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed_mock_test.go b/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed_mock_test.go index 9482a1d57740..480b8992ea28 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed_mock_test.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed_mock_test.go @@ -19,8 +19,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangecache/rangecachemock" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb/kvpbmock" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/roachpb/roachpbmock" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -129,15 +129,15 @@ func TestDistSenderRangeFeedRetryOnTransportErrors(t *testing.T) { // cancels the context and closes the range feed stream. if spec.expectRetry { rangeDB.EXPECT().FirstRange().Return(&desc, nil) - client := roachpbmock.NewMockInternalClient(ctrl) + client := kvpbmock.NewMockInternalClient(ctrl) if useMuxRangeFeed { - stream := roachpbmock.NewMockInternal_MuxRangeFeedClient(ctrl) + stream := kvpbmock.NewMockInternal_MuxRangeFeedClient(ctrl) stream.EXPECT().Send(gomock.Any()).Return(nil) stream.EXPECT().Recv().Do(cancel).Return(nil, io.EOF) client.EXPECT().MuxRangeFeed(gomock.Any()).Return(stream, nil) } else { - stream := roachpbmock.NewMockInternal_RangeFeedClient(ctrl) + stream := kvpbmock.NewMockInternal_RangeFeedClient(ctrl) stream.EXPECT().Recv().Do(cancel).Return(nil, io.EOF) client.EXPECT().RangeFeed(gomock.Any(), gomock.Any()).Return(stream, nil) } diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed_test.go b/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed_test.go index 9c4a00662569..3b6ecaf1369c 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed_test.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed_test.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" @@ -43,7 +44,7 @@ import ( "google.golang.org/grpc/metadata" ) -type wrapRangeFeedClientFn func(client roachpb.Internal_RangeFeedClient) roachpb.Internal_RangeFeedClient +type wrapRangeFeedClientFn func(client kvpb.Internal_RangeFeedClient) kvpb.Internal_RangeFeedClient type testRangefeedClient struct { rpc.RestrictedInternalClient muxRangeFeedEnabled bool @@ -52,8 +53,8 @@ type testRangefeedClient struct { } func (c *testRangefeedClient) RangeFeed( - ctx context.Context, args *roachpb.RangeFeedRequest, opts ...grpc.CallOption, -) (roachpb.Internal_RangeFeedClient, error) { + ctx context.Context, args *kvpb.RangeFeedRequest, opts ...grpc.CallOption, +) (kvpb.Internal_RangeFeedClient, error) { defer c.count() if c.muxRangeFeedEnabled && ctx.Value(useMuxRangeFeedCtxKey{}) != nil { @@ -72,7 +73,7 @@ func (c *testRangefeedClient) RangeFeed( func (c *testRangefeedClient) MuxRangeFeed( ctx context.Context, opts ...grpc.CallOption, -) (roachpb.Internal_MuxRangeFeedClient, error) { +) (kvpb.Internal_MuxRangeFeedClient, error) { defer c.count() if !c.muxRangeFeedEnabled || ctx.Value(useMuxRangeFeedCtxKey{}) == nil { @@ -109,8 +110,8 @@ func (c *countConnectionsTransport) IsExhausted() bool { } func (c *countConnectionsTransport) SendNext( - ctx context.Context, request *roachpb.BatchRequest, -) (*roachpb.BatchResponse, error) { + ctx context.Context, request *kvpb.BatchRequest, +) (*kvpb.BatchResponse, error) { return c.wrapped.SendNext(ctx, request) } @@ -383,13 +384,13 @@ func TestMuxRangeFeedConnectsToNodeOnce(t *testing.T) { } type blockRecvRangeFeedClient struct { - roachpb.Internal_RangeFeedClient + kvpb.Internal_RangeFeedClient numRecvRemainingUntilBlocked int ctxCanceled bool } -func (b *blockRecvRangeFeedClient) Recv() (*roachpb.RangeFeedEvent, error) { +func (b *blockRecvRangeFeedClient) Recv() (*kvpb.RangeFeedEvent, error) { if !b.ctxCanceled { ctx := b.Internal_RangeFeedClient.Context() b.numRecvRemainingUntilBlocked-- @@ -406,7 +407,7 @@ func (b *blockRecvRangeFeedClient) Recv() (*roachpb.RangeFeedEvent, error) { return b.Internal_RangeFeedClient.Recv() } -var _ roachpb.Internal_RangeFeedClient = (*blockRecvRangeFeedClient)(nil) +var _ kvpb.Internal_RangeFeedClient = (*blockRecvRangeFeedClient)(nil) func TestRestartsStuckRangeFeeds(t *testing.T) { defer leaktest.AfterTest(t)() @@ -415,7 +416,7 @@ func TestRestartsStuckRangeFeeds(t *testing.T) { ctx := context.Background() blockingClient := &blockRecvRangeFeedClient{} - var wrapRfClient wrapRangeFeedClientFn = func(client roachpb.Internal_RangeFeedClient) roachpb.Internal_RangeFeedClient { + var wrapRfClient wrapRangeFeedClientFn = func(client kvpb.Internal_RangeFeedClient) kvpb.Internal_RangeFeedClient { blockingClient.Internal_RangeFeedClient = client blockingClient.numRecvRemainingUntilBlocked = 1 // let first Recv through, then block return blockingClient @@ -474,18 +475,18 @@ func TestRestartsStuckRangeFeedsSecondImplementation(t *testing.T) { ServerArgs: base.TestServerArgs{ Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ - TestingRangefeedFilter: func(args *roachpb.RangeFeedRequest, stream roachpb.RangeFeedEventSink) *roachpb.Error { + TestingRangefeedFilter: func(args *kvpb.RangeFeedRequest, stream kvpb.RangeFeedEventSink) *kvpb.Error { md, ok := metadata.FromIncomingContext(stream.Context()) if (!ok || len(md[t.Name()]) == 0) && stream.Context().Value(testKey{}) == nil { return nil } if atomic.LoadInt32(&canceled) != 0 { - return roachpb.NewError(doneErr) + return kvpb.NewError(doneErr) } t.Logf("intercepting %s", args) // Send a first response to "arm" the stuck detector in DistSender. - if assert.NoError(t, stream.Send(&roachpb.RangeFeedEvent{Checkpoint: &roachpb.RangeFeedCheckpoint{ + if assert.NoError(t, stream.Send(&kvpb.RangeFeedEvent{Checkpoint: &kvpb.RangeFeedCheckpoint{ Span: args.Span, ResolvedTS: hlc.Timestamp{Logical: 1}, }})) { @@ -493,7 +494,7 @@ func TestRestartsStuckRangeFeedsSecondImplementation(t *testing.T) { } select { case <-time.After(testutils.DefaultSucceedsSoonDuration): - return roachpb.NewErrorf("timed out waiting for stuck rangefeed's ctx cancellation") + return kvpb.NewErrorf("timed out waiting for stuck rangefeed's ctx cancellation") case <-stream.Context().Done(): t.Log("server side rangefeed canceled (as expected)") atomic.StoreInt32(&canceled, 1) diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go b/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go index 73cd37a19bf4..9bccc049a5a7 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -199,7 +200,7 @@ func checkResumeSpanScanResults( reverse bool, expResults [][]string, expSatisfied map[int]struct{}, - expReason roachpb.ResumeReason, + expReason kvpb.ResumeReason, opt checkOptions, ) { t.Helper() @@ -259,7 +260,7 @@ func checkScanResults( results []kv.Result, expResults [][]string, expSatisfied map[int]struct{}, - expReason roachpb.ResumeReason, + expReason kvpb.ResumeReason, opt checkOptions, ) { t.Helper() @@ -273,7 +274,7 @@ func checkReverseScanResults( results []kv.Result, expResults [][]string, expSatisfied map[int]struct{}, - expReason roachpb.ResumeReason, + expReason kvpb.ResumeReason, opt checkOptions, ) { t.Helper() @@ -329,7 +330,7 @@ func TestMultiRangeBoundedBatchScanSimple(t *testing.T) { expSatisfied[i] = struct{}{} } - checkScanResults(t, spans, b.Results, expResults, expSatisfied, roachpb.RESUME_KEY_LIMIT, checkOptions{mode: Strict}) + checkScanResults(t, spans, b.Results, expResults, expSatisfied, kvpb.RESUME_KEY_LIMIT, checkOptions{mode: Strict}) }) } } @@ -467,7 +468,7 @@ func TestMultiRangeBoundedBatchScan(t *testing.T) { require.NoError(t, db.Run(ctx, b)) // Compute the range boundary. - expReason := roachpb.RESUME_KEY_LIMIT + expReason := kvpb.RESUME_KEY_LIMIT expCount := maxExpCount if bound < maxExpCount { expCount = bound @@ -481,7 +482,7 @@ func TestMultiRangeBoundedBatchScan(t *testing.T) { } if threshold < expCount { expCount = threshold - expReason = roachpb.RESUME_RANGE_BOUNDARY + expReason = kvpb.RESUME_RANGE_BOUNDARY } } // Compute the satisfied scans. @@ -571,7 +572,7 @@ func TestMultiRangeBoundedBatchScanPartialResponses(t *testing.T) { returnOnRangeBoundary bool expResults [][]string expSatisfied []int - expReason roachpb.ResumeReason + expReason kvpb.ResumeReason }{ { name: "unsorted, non-overlapping, neither satisfied", @@ -721,7 +722,7 @@ func TestMultiRangeBoundedBatchScanPartialResponses(t *testing.T) { expResults: [][]string{ {"e1", "e2", "e3"}, {"e2", "e3"}, }, - expReason: roachpb.RESUME_RANGE_BOUNDARY, + expReason: kvpb.RESUME_RANGE_BOUNDARY, }, { name: "range boundary, overlapping, neither satisfied", @@ -733,7 +734,7 @@ func TestMultiRangeBoundedBatchScanPartialResponses(t *testing.T) { expResults: [][]string{ {"e1", "e2", "e3"}, {"e2", "e3"}, }, - expReason: roachpb.RESUME_RANGE_BOUNDARY, + expReason: kvpb.RESUME_RANGE_BOUNDARY, }, { name: "range boundary, overlapping, neither satisfied, bounded below boundary", @@ -745,7 +746,7 @@ func TestMultiRangeBoundedBatchScanPartialResponses(t *testing.T) { expResults: [][]string{ {"e1", "e2", "e3"}, {"e2"}, }, - expReason: roachpb.RESUME_KEY_LIMIT, + expReason: kvpb.RESUME_KEY_LIMIT, }, { name: "range boundary, overlapping, neither satisfied, bounded at boundary", @@ -757,7 +758,7 @@ func TestMultiRangeBoundedBatchScanPartialResponses(t *testing.T) { expResults: [][]string{ {"e1", "e2", "e3"}, {"e2", "e3"}, }, - expReason: roachpb.RESUME_KEY_LIMIT, + expReason: kvpb.RESUME_KEY_LIMIT, }, { name: "range boundary, overlapping, neither satisfied, bounded above boundary", @@ -769,7 +770,7 @@ func TestMultiRangeBoundedBatchScanPartialResponses(t *testing.T) { expResults: [][]string{ {"e1", "e2", "e3"}, {"e2", "e3"}, }, - expReason: roachpb.RESUME_RANGE_BOUNDARY, + expReason: kvpb.RESUME_RANGE_BOUNDARY, }, { name: "range boundary, non-overlapping, first satisfied", @@ -782,7 +783,7 @@ func TestMultiRangeBoundedBatchScanPartialResponses(t *testing.T) { {"e1", "e2"}, {}, }, expSatisfied: []int{0}, - expReason: roachpb.RESUME_RANGE_BOUNDARY, + expReason: kvpb.RESUME_RANGE_BOUNDARY, }, { name: "range boundary, non-overlapping, second satisfied", @@ -795,7 +796,7 @@ func TestMultiRangeBoundedBatchScanPartialResponses(t *testing.T) { {"e3"}, {"e1"}, }, expSatisfied: []int{1}, - expReason: roachpb.RESUME_RANGE_BOUNDARY, + expReason: kvpb.RESUME_RANGE_BOUNDARY, }, { name: "range boundary, non-overlapping, both satisfied", @@ -819,7 +820,7 @@ func TestMultiRangeBoundedBatchScanPartialResponses(t *testing.T) { expResults: [][]string{ {"c1", "c2", "c3"}, {}, }, - expReason: roachpb.RESUME_RANGE_BOUNDARY, + expReason: kvpb.RESUME_RANGE_BOUNDARY, }, { name: "range boundary, separate ranges, first satisfied", @@ -832,7 +833,7 @@ func TestMultiRangeBoundedBatchScanPartialResponses(t *testing.T) { {"c1", "c2", "c3"}, {}, }, expSatisfied: []int{0}, - expReason: roachpb.RESUME_RANGE_BOUNDARY, + expReason: kvpb.RESUME_RANGE_BOUNDARY, }, { name: "range boundary, separate ranges, second satisfied", @@ -845,7 +846,7 @@ func TestMultiRangeBoundedBatchScanPartialResponses(t *testing.T) { {}, {"c2", "c3"}, }, expSatisfied: []int{1}, - expReason: roachpb.RESUME_RANGE_BOUNDARY, + expReason: kvpb.RESUME_RANGE_BOUNDARY, }, { name: "range boundary, separate ranges, first empty", @@ -858,7 +859,7 @@ func TestMultiRangeBoundedBatchScanPartialResponses(t *testing.T) { {}, {"f1"}, }, expSatisfied: []int{0}, - expReason: roachpb.RESUME_RANGE_BOUNDARY, + expReason: kvpb.RESUME_RANGE_BOUNDARY, }, { name: "range boundary, separate ranges, second empty", @@ -871,7 +872,7 @@ func TestMultiRangeBoundedBatchScanPartialResponses(t *testing.T) { {"f1"}, {}, }, expSatisfied: []int{1}, - expReason: roachpb.RESUME_RANGE_BOUNDARY, + expReason: kvpb.RESUME_RANGE_BOUNDARY, }, { name: "range boundary, separate ranges, all empty", @@ -896,7 +897,7 @@ func TestMultiRangeBoundedBatchScanPartialResponses(t *testing.T) { {}, {}, {}, {}, {"f1"}, {"f1"}, }, expSatisfied: []int{0, 1, 2, 3, 5}, - expReason: roachpb.RESUME_RANGE_BOUNDARY, + expReason: kvpb.RESUME_RANGE_BOUNDARY, }, } { t.Run(tc.name, func(t *testing.T) { @@ -913,8 +914,8 @@ func TestMultiRangeBoundedBatchScanPartialResponses(t *testing.T) { expSatisfied[exp] = struct{}{} } opts := checkOptions{mode: Strict} - expReason := roachpb.RESUME_KEY_LIMIT - if tc.expReason != roachpb.RESUME_UNKNOWN { + expReason := kvpb.RESUME_KEY_LIMIT + if tc.expReason != kvpb.RESUME_UNKNOWN { expReason = tc.expReason } checkScanResults(t, tc.spans, b.Results, tc.expResults, expSatisfied, expReason, opts) @@ -1084,9 +1085,9 @@ func TestMultiRangeScanReverseScanInconsistent(t *testing.T) { defer log.Scope(t).Close(t) - for _, rc := range []roachpb.ReadConsistencyType{ - roachpb.READ_UNCOMMITTED, - roachpb.INCONSISTENT, + for _, rc := range []kvpb.ReadConsistencyType{ + kvpb.READ_UNCOMMITTED, + kvpb.INCONSISTENT, } { t.Run(rc.String(), func(t *testing.T) { s, _ := startNoSplitMergeServer(t) @@ -1127,9 +1128,9 @@ func TestMultiRangeScanReverseScanInconsistent(t *testing.T) { // it does the read at its local clock and doesn't receive an // OpRequiresTxnError. We set the local clock to the timestamp of // just above the first key to verify it's used to read only key "a". - for i, request := range []roachpb.Request{ - roachpb.NewScan(roachpb.Key("a"), roachpb.Key("c"), false), - roachpb.NewReverseScan(roachpb.Key("a"), roachpb.Key("c"), false), + for i, request := range []kvpb.Request{ + kvpb.NewScan(roachpb.Key("a"), roachpb.Key("c"), false), + kvpb.NewReverseScan(roachpb.Key("a"), roachpb.Key("c"), false), } { clock := hlc.NewClockForTesting(timeutil.NewManualTime(ts[0].GoTime().Add(1))) ds := kvcoord.NewDistSender(kvcoord.DistSenderConfig{ @@ -1142,7 +1143,7 @@ func TestMultiRangeScanReverseScanInconsistent(t *testing.T) { FirstRangeProvider: s.Gossip(), }) - reply, err := kv.SendWrappedWith(context.Background(), ds, roachpb.Header{ + reply, err := kv.SendWrappedWith(context.Background(), ds, kvpb.Header{ ReadConsistency: rc, }, request) if err != nil { @@ -1151,9 +1152,9 @@ func TestMultiRangeScanReverseScanInconsistent(t *testing.T) { var rows []roachpb.KeyValue switch r := reply.(type) { - case *roachpb.ScanResponse: + case *kvpb.ScanResponse: rows = r.Rows - case *roachpb.ReverseScanResponse: + case *kvpb.ReverseScanResponse: rows = r.Rows default: t.Fatalf("unexpected response %T: %v", reply, reply) @@ -1189,8 +1190,8 @@ func TestMultiRangeScanDeleteRange(t *testing.T) { t.Fatal(err) } writes := []roachpb.Key{roachpb.Key("a"), roachpb.Key("z")} - get := &roachpb.GetRequest{ - RequestHeader: roachpb.RequestHeader{Key: writes[0]}, + get := &kvpb.GetRequest{ + RequestHeader: kvpb.RequestHeader{Key: writes[0]}, } get.EndKey = writes[len(writes)-1] if _, err := kv.SendWrapped(ctx, tds, get); err == nil { @@ -1198,16 +1199,16 @@ func TestMultiRangeScanDeleteRange(t *testing.T) { } var delTS hlc.Timestamp for i, k := range writes { - put := roachpb.NewPut(k, roachpb.MakeValueFromBytes(k)) + put := kvpb.NewPut(k, roachpb.MakeValueFromBytes(k)) if _, err := kv.SendWrapped(ctx, tds, put); err != nil { t.Fatal(err) } - scan := roachpb.NewScan(writes[0], writes[len(writes)-1].Next(), false) + scan := kvpb.NewScan(writes[0], writes[len(writes)-1].Next(), false) reply, err := kv.SendWrapped(ctx, tds, scan) if err != nil { t.Fatal(err) } - sr := reply.(*roachpb.ScanResponse) + sr := reply.(*kvpb.ScanResponse) if sr.Txn != nil { // This was the other way around at some point in the past. // Same below for Delete, etc. @@ -1218,18 +1219,18 @@ func TestMultiRangeScanDeleteRange(t *testing.T) { } } - del := &roachpb.DeleteRangeRequest{ - RequestHeader: roachpb.RequestHeader{ + del := &kvpb.DeleteRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: writes[0], EndKey: writes[len(writes)-1].Next(), }, ReturnKeys: true, } - reply, err := kv.SendWrappedWith(ctx, tds, roachpb.Header{Timestamp: delTS}, del) + reply, err := kv.SendWrappedWith(ctx, tds, kvpb.Header{Timestamp: delTS}, del) if err != nil { t.Fatal(err) } - dr := reply.(*roachpb.DeleteRangeResponse) + dr := reply.(*kvpb.DeleteRangeResponse) if dr.Txn != nil { t.Errorf("expected no transaction in response header") } @@ -1241,9 +1242,9 @@ func TestMultiRangeScanDeleteRange(t *testing.T) { txnProto := roachpb.MakeTransaction("MyTxn", nil, 0, now.ToTimestamp(), 0, int32(s.SQLInstanceID())) txn := kv.NewTxnFromProto(ctx, db, s.NodeID(), now, kv.RootTxn, &txnProto) - scan := roachpb.NewScan(writes[0], writes[len(writes)-1].Next(), false) - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txnProto} + scan := kvpb.NewScan(writes[0], writes[len(writes)-1].Next(), false) + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txnProto} ba.Add(scan) br, pErr := txn.Send(ctx, ba) if pErr != nil { @@ -1253,7 +1254,7 @@ func TestMultiRangeScanDeleteRange(t *testing.T) { if replyTxn == nil || replyTxn.Name != "MyTxn" { t.Errorf("wanted Txn to persist, but it changed to %v", txn) } - sr := br.Responses[0].GetInner().(*roachpb.ScanResponse) + sr := br.Responses[0].GetInner().(*kvpb.ScanResponse) if rows := sr.Rows; len(rows) > 0 { t.Fatalf("scan after delete returned rows: %v", rows) } @@ -1348,7 +1349,7 @@ func TestMultiRangeScanWithPagination(t *testing.T) { } for _, k := range tc.keys { - put := roachpb.NewPut(k, roachpb.MakeValueFromBytes(k)) + put := kvpb.NewPut(k, roachpb.MakeValueFromBytes(k)) if _, err := kv.SendWrapped(ctx, tds, put); err != nil { t.Fatal(err) } @@ -1359,7 +1360,7 @@ func TestMultiRangeScanWithPagination(t *testing.T) { // happens above this. var maxTargetBytes int64 { - scan := roachpb.NewScan(tc.keys[0], tc.keys[len(tc.keys)-1].Next(), false) + scan := kvpb.NewScan(tc.keys[0], tc.keys[len(tc.keys)-1].Next(), false) resp, pErr := kv.SendWrapped(ctx, tds, scan) require.Nil(t, pErr) require.Nil(t, resp.Header().ResumeSpan) @@ -1395,16 +1396,16 @@ func TestMultiRangeScanWithPagination(t *testing.T) { numPages++ // Build the batch. - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} for _, span := range operations { - var req roachpb.Request + var req kvpb.Request switch { case span.EndKey == nil: - req = roachpb.NewGet(span.Key, false /* forUpdate */) + req = kvpb.NewGet(span.Key, false /* forUpdate */) case reverse: - req = roachpb.NewReverseScan(span.Key, span.EndKey, false /* forUpdate */) + req = kvpb.NewReverseScan(span.Key, span.EndKey, false /* forUpdate */) default: - req = roachpb.NewScan(span.Key, span.EndKey, false /* forUpdate */) + req = kvpb.NewScan(span.Key, span.EndKey, false /* forUpdate */) } ba.Add(req) } @@ -1669,8 +1670,8 @@ func TestBatchPutWithConcurrentSplit(t *testing.T) { FirstRangeProvider: s.Gossip(), }) for _, key := range []string{"c"} { - req := &roachpb.AdminSplitRequest{ - RequestHeader: roachpb.RequestHeader{ + req := &kvpb.AdminSplitRequest{ + RequestHeader: kvpb.RequestHeader{ Key: roachpb.Key(key), }, SplitKey: roachpb.Key(key), @@ -1799,21 +1800,21 @@ func TestPropagateTxnOnError(t *testing.T) { var numCPuts int32 var storeKnobs kvserver.StoreTestingKnobs storeKnobs.EvalKnobs.TestingEvalFilter = - func(fArgs kvserverbase.FilterArgs) *roachpb.Error { + func(fArgs kvserverbase.FilterArgs) *kvpb.Error { k := fArgs.Req.Header().Key switch fArgs.Req.(type) { - case *roachpb.PutRequest: + case *kvpb.PutRequest: if k.Equal(keyA) { fArgs.Hdr.Txn.UpdateObservedTimestamp(ot1.NodeID, ot1.Timestamp) } else if k.Equal(keyC) { fArgs.Hdr.Txn.UpdateObservedTimestamp(ot2.NodeID, ot2.Timestamp) } - case *roachpb.ConditionalPutRequest: + case *kvpb.ConditionalPutRequest: if k.Equal(keyB) { if atomic.AddInt32(&numCPuts, 1) == 1 { - pErr := roachpb.NewReadWithinUncertaintyIntervalError( + pErr := kvpb.NewReadWithinUncertaintyIntervalError( hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, hlc.Timestamp{}, hlc.ClockTimestamp{}) - return roachpb.NewErrorWithTxn(pErr, fArgs.Hdr.Txn) + return kvpb.NewErrorWithTxn(pErr, fArgs.Hdr.Txn) } } } @@ -1867,7 +1868,7 @@ func TestPropagateTxnOnError(t *testing.T) { b.Put(keyC, "val2") err := txn.CommitInBatch(ctx, b) if epoch == 1 { - if retErr := (*roachpb.TransactionRetryWithProtoRefreshError)(nil); errors.As(err, &retErr) { + if retErr := (*kvpb.TransactionRetryWithProtoRefreshError)(nil); errors.As(err, &retErr) { if !testutils.IsError(retErr, "ReadWithinUncertaintyIntervalError") { t.Errorf("expected ReadWithinUncertaintyIntervalError, but got: %v", retErr) } @@ -1954,10 +1955,10 @@ func TestAsyncAbortPoisons(t *testing.T) { var storeKnobs kvserver.StoreTestingKnobs keyA, keyB := roachpb.Key("a"), roachpb.Key("b") commitCh := make(chan error, 1) - storeKnobs.TestingRequestFilter = func(_ context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + storeKnobs.TestingRequestFilter = func(_ context.Context, ba *kvpb.BatchRequest) *kvpb.Error { for _, req := range ba.Requests { switch r := req.GetInner().(type) { - case *roachpb.EndTxnRequest: + case *kvpb.EndTxnRequest: if r.Key.Equal(keyA) { if r.Poison { close(commitCh) @@ -1996,7 +1997,7 @@ func TestAsyncAbortPoisons(t *testing.T) { _, err := txn.Get(ctx, keyA) require.Error(t, err) - require.IsType(t, err, &roachpb.TransactionRetryWithProtoRefreshError{}) + require.IsType(t, err, &kvpb.TransactionRetryWithProtoRefreshError{}) require.Contains(t, err.Error(), "TransactionAbortedError") require.NoError(t, <-commitCh) } @@ -2011,9 +2012,9 @@ func TestTxnCoordSenderRetries(t *testing.T) { var filterFn atomic.Value var storeKnobs kvserver.StoreTestingKnobs storeKnobs.EvalKnobs.TestingEvalFilter = - func(fArgs kvserverbase.FilterArgs) *roachpb.Error { + func(fArgs kvserverbase.FilterArgs) *kvpb.Error { fnVal := filterFn.Load() - if fn, ok := fnVal.(func(kvserverbase.FilterArgs) *roachpb.Error); ok && fn != nil { + if fn, ok := fnVal.(func(kvserverbase.FilterArgs) *kvpb.Error); ok && fn != nil { return fn(fArgs) } return nil @@ -2038,17 +2039,17 @@ func TestTxnCoordSenderRetries(t *testing.T) { ctx := context.Background() defer s.Stopper().Stop(ctx) - newUncertaintyFilter := func(key roachpb.Key) func(kvserverbase.FilterArgs) *roachpb.Error { + newUncertaintyFilter := func(key roachpb.Key) func(kvserverbase.FilterArgs) *kvpb.Error { var count int32 - return func(fArgs kvserverbase.FilterArgs) *roachpb.Error { + return func(fArgs kvserverbase.FilterArgs) *kvpb.Error { if (fArgs.Req.Header().Key.Equal(key) || fArgs.Req.Header().Span().ContainsKey(key)) && fArgs.Hdr.Txn != nil { if atomic.AddInt32(&count, 1) > 1 { return nil } - err := roachpb.NewReadWithinUncertaintyIntervalError( + err := kvpb.NewReadWithinUncertaintyIntervalError( fArgs.Hdr.Timestamp, hlc.ClockTimestamp{}, fArgs.Hdr.Txn, s.Clock().Now(), hlc.ClockTimestamp{}) - return roachpb.NewErrorWithTxn(err, fArgs.Hdr.Txn) + return kvpb.NewErrorWithTxn(err, fArgs.Hdr.Txn) } return nil } @@ -2065,7 +2066,7 @@ func TestTxnCoordSenderRetries(t *testing.T) { beforeTxnStart func(context.Context, *kv.DB) error // called before the txn starts afterTxnStart func(context.Context, *kv.DB) error // called after the txn chooses a timestamp retryable func(context.Context, *kv.Txn) error // called during the txn; may be retried - filter func(kvserverbase.FilterArgs) *roachpb.Error + filter func(kvserverbase.FilterArgs) *kvpb.Error refreshSpansCondenseFilter func() bool priorReads bool tsLeaked bool @@ -3138,9 +3139,9 @@ func TestTxnCoordSenderRetries(t *testing.T) { // directly. This should be picked up by the transaction's // QueryIntent when chaining on to the pipelined write to // key "a". - ba := &roachpb.BatchRequest{} - ba.Add(&roachpb.ResolveIntentRequest{ - RequestHeader: roachpb.RequestHeader{ + ba := &kvpb.BatchRequest{} + ba.Add(&kvpb.ResolveIntentRequest{ + RequestHeader: kvpb.RequestHeader{ Key: roachpb.Key("a"), }, IntentTxn: txn.TestingCloneTxn().TxnMeta, @@ -3164,9 +3165,9 @@ func TestTxnCoordSenderRetries(t *testing.T) { // Simulate a failed intent write by resolving the intent // directly. This should be picked up by the transaction's // pre-commit QueryIntent for the pipelined write to key "a". - ba := &roachpb.BatchRequest{} - ba.Add(&roachpb.ResolveIntentRequest{ - RequestHeader: roachpb.RequestHeader{ + ba := &kvpb.BatchRequest{} + ba.Add(&kvpb.ResolveIntentRequest{ + RequestHeader: kvpb.RequestHeader{ Key: roachpb.Key("a"), }, IntentTxn: txn.TestingCloneTxn().TxnMeta, @@ -3192,7 +3193,7 @@ func TestTxnCoordSenderRetries(t *testing.T) { if tc.filter != nil { filterFn.Store(tc.filter) - defer filterFn.Store((func(kvserverbase.FilterArgs) *roachpb.Error)(nil)) + defer filterFn.Store((func(kvserverbase.FilterArgs) *kvpb.Error)(nil)) } if tc.refreshSpansCondenseFilter != nil { refreshSpansCondenseFilter.Store(tc.refreshSpansCondenseFilter) @@ -3293,9 +3294,9 @@ func TestTxnCoordSenderRetriesAcrossEndTxn(t *testing.T) { var filterFn atomic.Value var storeKnobs kvserver.StoreTestingKnobs storeKnobs.EvalKnobs.TestingEvalFilter = - func(fArgs kvserverbase.FilterArgs) *roachpb.Error { + func(fArgs kvserverbase.FilterArgs) *kvpb.Error { fnVal := filterFn.Load() - if fn, ok := fnVal.(func(kvserverbase.FilterArgs) *roachpb.Error); ok && fn != nil { + if fn, ok := fnVal.(func(kvserverbase.FilterArgs) *kvpb.Error); ok && fn != nil { return fn(fArgs) } return nil @@ -3424,8 +3425,8 @@ func TestTxnCoordSenderRetriesAcrossEndTxn(t *testing.T) { // Install a filter which will reject requests touching // secondAttemptRejectKey on the retry. var count int32 - filterFn.Store(func(args kvserverbase.FilterArgs) *roachpb.Error { - put, ok := args.Req.(*roachpb.ConditionalPutRequest) + filterFn.Store(func(args kvserverbase.FilterArgs) *kvpb.Error { + put, ok := args.Req.(*kvpb.ConditionalPutRequest) if !ok { return nil } @@ -3435,7 +3436,7 @@ func TestTxnCoordSenderRetriesAcrossEndTxn(t *testing.T) { count++ // Reject the right request on the 2nd attempt. if count == 2 { - return roachpb.NewErrorf("injected error; test rejecting request") + return kvpb.NewErrorf("injected error; test rejecting request") } return nil }) @@ -3551,7 +3552,7 @@ func BenchmarkReturnOnRangeBoundary(b *testing.B) { ctx := context.Background() scanCtx := context.WithValue(ctx, scanKey{}, "scan") - reqFilter := func(ctx context.Context, _ *roachpb.BatchRequest) *roachpb.Error { + reqFilter := func(ctx context.Context, _ *kvpb.BatchRequest) *kvpb.Error { if ctx.Value(scanKey{}) != nil && Latency > 0 { time.Sleep(Latency) } diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_test.go b/pkg/kv/kvclient/kvcoord/dist_sender_test.go index 3f51e501b3ec..39355c8edd66 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_test.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_test.go @@ -31,6 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangecache" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/multitenant" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcostmodel" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -125,14 +126,14 @@ var ( var testAddress = util.NewUnresolvedAddr("tcp", "node1") // simpleSendFn is the function type used to dispatch RPC calls in simpleTransportAdapter. -type simpleSendFn func(context.Context, *roachpb.BatchRequest) (*roachpb.BatchResponse, error) +type simpleSendFn func(context.Context, *kvpb.BatchRequest) (*kvpb.BatchResponse, error) // stubRPCSendFn is an rpcSendFn that simply creates a reply for the // BatchRequest without performing an RPC call or triggering any // test instrumentation. var stubRPCSendFn simpleSendFn = func( - _ context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, error) { + _ context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, error) { return ba.CreateReply(), nil } @@ -169,8 +170,8 @@ func (l *simpleTransportAdapter) IsExhausted() bool { } func (l *simpleTransportAdapter) SendNext( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, error) { ba = ba.ShallowCopy() ba.Replica = l.replicas[l.nextReplicaIdx] l.nextReplicaIdx++ @@ -283,7 +284,7 @@ func TestSendRPCOrder(t *testing.T) { testCases := []struct { name string - routingPolicy roachpb.RoutingPolicy + routingPolicy kvpb.RoutingPolicy tiers []roachpb.Tier leaseHolder int32 // 0 for not caching a lease holder. expReplica []roachpb.NodeID // 0 elements ignored @@ -293,21 +294,21 @@ func TestSendRPCOrder(t *testing.T) { }{ { name: "route to leaseholder, without matching attributes", - routingPolicy: roachpb.RoutingPolicy_LEASEHOLDER, + routingPolicy: kvpb.RoutingPolicy_LEASEHOLDER, tiers: []roachpb.Tier{}, // No ordering. expReplica: []roachpb.NodeID{1, 2, 3, 4, 5}, }, { name: "route to leaseholder, with matching attributes", - routingPolicy: roachpb.RoutingPolicy_LEASEHOLDER, + routingPolicy: kvpb.RoutingPolicy_LEASEHOLDER, tiers: nodeTiers[5], // Order nearest first. expReplica: []roachpb.NodeID{5, 4, 0, 0, 0}, }, { name: "route to leaseholder, without matching attributes, known leaseholder", - routingPolicy: roachpb.RoutingPolicy_LEASEHOLDER, + routingPolicy: kvpb.RoutingPolicy_LEASEHOLDER, tiers: []roachpb.Tier{}, leaseHolder: 2, // Order leaseholder first. @@ -315,7 +316,7 @@ func TestSendRPCOrder(t *testing.T) { }, { name: "route to leaseholder, without matching attributes, non-voters", - routingPolicy: roachpb.RoutingPolicy_LEASEHOLDER, + routingPolicy: kvpb.RoutingPolicy_LEASEHOLDER, tiers: []roachpb.Tier{}, leaseHolder: 2, // Order leaseholder first, omits the non-voters. @@ -327,7 +328,7 @@ func TestSendRPCOrder(t *testing.T) { }, { name: "route to leaseholder, with matching attributes, known leaseholder", - routingPolicy: roachpb.RoutingPolicy_LEASEHOLDER, + routingPolicy: kvpb.RoutingPolicy_LEASEHOLDER, tiers: nodeTiers[5], leaseHolder: 2, // Order leaseholder first, then nearest. @@ -335,21 +336,21 @@ func TestSendRPCOrder(t *testing.T) { }, { name: "route to nearest, without matching attributes", - routingPolicy: roachpb.RoutingPolicy_NEAREST, + routingPolicy: kvpb.RoutingPolicy_NEAREST, tiers: []roachpb.Tier{}, // No ordering. expReplica: []roachpb.NodeID{1, 2, 3, 4, 5}, }, { name: "route to nearest, with matching attributes", - routingPolicy: roachpb.RoutingPolicy_NEAREST, + routingPolicy: kvpb.RoutingPolicy_NEAREST, tiers: nodeTiers[5], // Order nearest first. expReplica: []roachpb.NodeID{5, 4, 0, 0, 0}, }, { name: "route to nearest, without matching attributes, known leaseholder", - routingPolicy: roachpb.RoutingPolicy_NEAREST, + routingPolicy: kvpb.RoutingPolicy_NEAREST, tiers: []roachpb.Tier{}, leaseHolder: 2, // No ordering. @@ -357,7 +358,7 @@ func TestSendRPCOrder(t *testing.T) { }, { name: "route to nearest, with matching attributes, known leaseholder", - routingPolicy: roachpb.RoutingPolicy_NEAREST, + routingPolicy: kvpb.RoutingPolicy_NEAREST, tiers: nodeTiers[5], leaseHolder: 2, // Order nearest first. @@ -365,7 +366,7 @@ func TestSendRPCOrder(t *testing.T) { }, { name: "route to leaseholder, no known leaseholder, uses non-voters", - routingPolicy: roachpb.RoutingPolicy_LEASEHOLDER, + routingPolicy: kvpb.RoutingPolicy_LEASEHOLDER, tiers: nodeTiers[5], // Order nearest first, includes the non-voter despite the leaseholder // routing policy. @@ -387,7 +388,7 @@ func TestSendRPCOrder(t *testing.T) { _ *cluster.Settings, _ *hlc.Clock, p roachpb.RangeClosedTimestampPolicy, - ba *roachpb.BatchRequest, + ba *kvpb.BatchRequest, ) bool { return !ba.IsLocking() && p == roachpb.LEAD_FOR_GLOBAL_READS } @@ -424,7 +425,7 @@ func TestSendRPCOrder(t *testing.T) { return nil, err } return adaptSimpleTransport( - func(ctx context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + func(ctx context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { return ba.CreateReply(), nil })(opts, dialer, replicas) } @@ -483,11 +484,11 @@ func TestSendRPCOrder(t *testing.T) { ds.rangeCache.Insert(ctx, ri) // Issue the request. - header := roachpb.Header{ + header := kvpb.Header{ RangeID: rangeID, // Not used in this test, but why not. RoutingPolicy: tc.routingPolicy, } - req := roachpb.NewScan(roachpb.Key("a"), roachpb.Key("b"), false) + req := kvpb.NewScan(roachpb.Key("a"), roachpb.Key("b"), false) _, pErr := kv.SendWrappedWith(ctx, ds, header, req) require.Nil(t, pErr) }) @@ -502,7 +503,7 @@ func TestSendRPCOrder(t *testing.T) { type MockRangeDescriptorDB func(roachpb.RKey, bool) (rs, preRs []roachpb.RangeDescriptor, err error) func (mdb MockRangeDescriptorDB) RangeLookup( - ctx context.Context, key roachpb.RKey, _ roachpb.ReadConsistencyType, useReverseScan bool, + ctx context.Context, key roachpb.RKey, _ kvpb.ReadConsistencyType, useReverseScan bool, ) ([]roachpb.RangeDescriptor, []roachpb.RangeDescriptor, error) { return mdb(key, useReverseScan) } @@ -590,8 +591,8 @@ func TestImmutableBatchArgs(t *testing.T) { rpcContext := rpc.NewInsecureTestingContext(ctx, clock, stopper) g := makeGossip(t, stopper, rpcContext) var testFn simpleSendFn = func( - _ context.Context, args *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, error) { + _ context.Context, args *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, error) { reply := args.CreateReply() reply.Txn = args.Txn.Clone() reply.Txn.WriteTimestamp = hlc.MaxTimestamp @@ -623,8 +624,8 @@ func TestImmutableBatchArgs(t *testing.T) { // so make sure we're not in that case. txn.UpdateObservedTimestamp(1, hlc.MaxClockTimestamp) - put := roachpb.NewPut(roachpb.Key("don't"), roachpb.Value{}) - if _, pErr := kv.SendWrappedWith(context.Background(), ds, roachpb.Header{ + put := kvpb.NewPut(roachpb.Key("don't"), roachpb.Value{}) + if _, pErr := kv.SendWrappedWith(context.Background(), ds, kvpb.Header{ Txn: &txn, }, put); pErr != nil { t.Fatal(pErr) @@ -655,13 +656,13 @@ func TestRetryOnNotLeaseHolderError(t *testing.T) { // information is present in it, we expect the cache to be updated. tests := []struct { name string - nlhe roachpb.NotLeaseHolderError + nlhe kvpb.NotLeaseHolderError expLeaseholder *roachpb.ReplicaDescriptor expLease bool }{ { name: "leaseholder in desc", - nlhe: roachpb.NotLeaseHolderError{ + nlhe: kvpb.NotLeaseHolderError{ RangeID: testUserRangeDescriptor3Replicas.RangeID, Lease: &roachpb.Lease{Replica: recognizedLeaseHolder, Sequence: 1}, RangeDesc: testUserRangeDescriptor3Replicas, @@ -673,7 +674,7 @@ func TestRetryOnNotLeaseHolderError(t *testing.T) { // TODO(arul): This is only possible in 22.{1,2} mixed version clusters; // remove once we get rid of the LeaseHolder field in 23.1. name: "leaseholder in desc, no lease", - nlhe: roachpb.NotLeaseHolderError{ + nlhe: kvpb.NotLeaseHolderError{ RangeID: testUserRangeDescriptor3Replicas.RangeID, DeprecatedLeaseHolder: &recognizedLeaseHolder, RangeDesc: testUserRangeDescriptor3Replicas, @@ -683,7 +684,7 @@ func TestRetryOnNotLeaseHolderError(t *testing.T) { }, { name: "leaseholder not in desc", - nlhe: roachpb.NotLeaseHolderError{ + nlhe: kvpb.NotLeaseHolderError{ RangeID: testUserRangeDescriptor3Replicas.RangeID, Lease: &roachpb.Lease{Replica: unrecognizedLeaseHolder, Sequence: 2}, RangeDesc: testUserRangeDescriptor3Replicas, @@ -692,7 +693,7 @@ func TestRetryOnNotLeaseHolderError(t *testing.T) { }, { name: "leaseholder in desc with different type", - nlhe: roachpb.NotLeaseHolderError{ + nlhe: kvpb.NotLeaseHolderError{ RangeID: testUserRangeDescriptor3Replicas.RangeID, Lease: &roachpb.Lease{Replica: recognizedLeaseHolderIncoming, Sequence: 1}, RangeDesc: testUserRangeDescriptor3Replicas, @@ -702,7 +703,7 @@ func TestRetryOnNotLeaseHolderError(t *testing.T) { }, { name: "leaseholder unknown", - nlhe: roachpb.NotLeaseHolderError{ + nlhe: kvpb.NotLeaseHolderError{ RangeID: testUserRangeDescriptor3Replicas.RangeID, RangeDesc: testUserRangeDescriptor3Replicas, }, @@ -729,19 +730,19 @@ func TestRetryOnNotLeaseHolderError(t *testing.T) { var retryReplica roachpb.ReplicaDescriptor var testFn simpleSendFn = func( - _ context.Context, args *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, error) { + _ context.Context, args *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, error) { attempts++ - reply := &roachpb.BatchResponse{} + reply := &kvpb.BatchResponse{} if attempts == 1 { - reply.Error = roachpb.NewError(&tc.nlhe) + reply.Error = kvpb.NewError(&tc.nlhe) return reply, nil } // Return an error to avoid activating a code path that would update the // cache with the leaseholder from the successful response. That's not // what this test wants to test. retryReplica = args.Header.Replica - reply.Error = roachpb.NewErrorf("boom") + reply.Error = kvpb.NewErrorf("boom") return reply, nil } @@ -759,7 +760,7 @@ func TestRetryOnNotLeaseHolderError(t *testing.T) { } ds := NewDistSender(cfg) v := roachpb.MakeValueFromString("value") - put := roachpb.NewPut(roachpb.Key("a"), v) + put := kvpb.NewPut(roachpb.Key("a"), v) if _, pErr := kv.SendWrapped(ctx, ds, put); !testutils.IsPError(pErr, "boom") { t.Fatalf("unexpected error: %v", pErr) } @@ -813,8 +814,8 @@ func TestBackoffOnNotLeaseHolderErrorDuringTransfer(t *testing.T) { } } var sequences []roachpb.LeaseSequence - var testFn simpleSendFn = func(_ context.Context, args *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { - reply := &roachpb.BatchResponse{} + var testFn simpleSendFn = func(_ context.Context, args *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { + reply := &kvpb.BatchResponse{} if len(sequences) > 0 { seq := sequences[0] sequences = sequences[1:] @@ -826,15 +827,15 @@ func TestBackoffOnNotLeaseHolderErrorDuringTransfer(t *testing.T) { Replica: repls[int(seq)%2], } } - reply.Error = roachpb.NewError( - &roachpb.NotLeaseHolderError{ + reply.Error = kvpb.NewError( + &kvpb.NotLeaseHolderError{ Replica: repls[int(seq)%2], Lease: lease, }) return reply, nil } // Return an error to bail out of retries. - reply.Error = roachpb.NewErrorf("boom") + reply.Error = kvpb.NewErrorf("boom") return reply, nil } @@ -866,7 +867,7 @@ func TestBackoffOnNotLeaseHolderErrorDuringTransfer(t *testing.T) { sequences = c.leaseSequences ds := NewDistSender(cfg) v := roachpb.MakeValueFromString("value") - put := roachpb.NewPut(roachpb.Key("a"), v) + put := kvpb.NewPut(roachpb.Key("a"), v) if _, pErr := kv.SendWrapped(ctx, ds, put); !testutils.IsPError(pErr, "boom") { t.Fatalf("%d: unexpected error: %v", i, pErr) } @@ -895,7 +896,7 @@ func TestNoBackoffOnNotLeaseHolderErrorFromFollowerRead(t *testing.T) { _ *cluster.Settings, _ *hlc.Clock, _ roachpb.RangeClosedTimestampPolicy, - ba *roachpb.BatchRequest, + ba *kvpb.BatchRequest, ) bool { return true } @@ -905,11 +906,11 @@ func TestNoBackoffOnNotLeaseHolderErrorFromFollowerRead(t *testing.T) { Replica: testUserRangeDescriptor3Replicas.InternalReplicas[1], Sequence: 1, } - testFn := func(_ context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + testFn := func(_ context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { sentTo = append(sentTo, ba.Replica.NodeID) br := ba.CreateReply() if ba.Replica != lease.Replica { - br.Error = roachpb.NewError(&roachpb.NotLeaseHolderError{ + br.Error = kvpb.NewError(&kvpb.NotLeaseHolderError{ Replica: ba.Replica, Lease: &lease, }) @@ -948,7 +949,7 @@ func TestNoBackoffOnNotLeaseHolderErrorFromFollowerRead(t *testing.T) { Lease: lease, }) - get := roachpb.NewGet(roachpb.Key("a"), false /* forUpdate */) + get := kvpb.NewGet(roachpb.Key("a"), false /* forUpdate */) _, pErr := kv.SendWrapped(ctx, ds, get) require.Nil(t, pErr) require.Equal(t, []roachpb.NodeID{1, 2}, sentTo) @@ -978,11 +979,11 @@ func TestNoBackoffOnNotLeaseHolderErrorWithoutLease(t *testing.T) { // n1 and n2 return an NLHE without lease information, n3 returns success. // Record which replicas the request was sent to. var sentTo []roachpb.NodeID - sendFn := func(_ context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + sendFn := func(_ context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { sentTo = append(sentTo, ba.Replica.NodeID) br := ba.CreateReply() if ba.Replica != replicas[2] { - br.Error = roachpb.NewError(&roachpb.NotLeaseHolderError{ + br.Error = kvpb.NewError(&kvpb.NotLeaseHolderError{ Replica: ba.Replica, }) } @@ -1020,7 +1021,7 @@ func TestNoBackoffOnNotLeaseHolderErrorWithoutLease(t *testing.T) { // Send a request. It should try all three replicas once: the first two fail // with NLHE, the third one succeeds. None of them should trigger backoffs. - _, pErr := kv.SendWrapped(ctx, ds, roachpb.NewGet(roachpb.Key("a"), false /* forUpdate */)) + _, pErr := kv.SendWrapped(ctx, ds, kvpb.NewGet(roachpb.Key("a"), false /* forUpdate */)) require.NoError(t, pErr.GoError()) require.Equal(t, []roachpb.NodeID{1, 2, 3}, sentTo) require.Equal(t, int64(0), ds.Metrics().InLeaseTransferBackoffs.Count()) @@ -1085,12 +1086,12 @@ func TestDistSenderMovesOnFromReplicaWithStaleLease(t *testing.T) { // replica, which will return a success. var callsToNode2 int - sendFn := func(ctx context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + sendFn := func(ctx context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { if ba.Replica.NodeID == 2 { callsToNode2++ - reply := &roachpb.BatchResponse{} - err := &roachpb.NotLeaseHolderError{Lease: &staleLease} - reply.Error = roachpb.NewError(err) + reply := &kvpb.BatchResponse{} + err := &kvpb.NotLeaseHolderError{Lease: &staleLease} + reply.Error = kvpb.NewError(err) return reply, nil } require.Equal(t, ba.Replica.NodeID, roachpb.NodeID(1)) @@ -1116,7 +1117,7 @@ func TestDistSenderMovesOnFromReplicaWithStaleLease(t *testing.T) { Lease: cachedLease, }) - get := roachpb.NewGet(roachpb.Key("a"), false /* forUpdate */) + get := kvpb.NewGet(roachpb.Key("a"), false /* forUpdate */) _, pErr := kv.SendWrapped(ctx, ds, get) require.Nil(t, pErr) @@ -1135,7 +1136,7 @@ func TestDistSenderIgnoresNLHEBasedOnOldRangeGeneration(t *testing.T) { name string nlheLeaseSequence int // if 0, NLHE returns a speculative LeaseHolder instead of a full lease. cachedLeaseSequence int - nlhe *roachpb.NotLeaseHolderError + nlhe *kvpb.NotLeaseHolderError }{ { name: "speculative lease", @@ -1181,7 +1182,7 @@ func TestDistSenderIgnoresNLHEBasedOnOldRangeGeneration(t *testing.T) { }, } - nlhe := &roachpb.NotLeaseHolderError{ + nlhe := &kvpb.NotLeaseHolderError{ RangeDesc: roachpb.RangeDescriptor{ Generation: oldGeneration, }, @@ -1208,11 +1209,11 @@ func TestDistSenderIgnoresNLHEBasedOnOldRangeGeneration(t *testing.T) { // routed there. That replica will reply with an NLHE with an old descriptor // generation value, which should make the DistSender try the next replica. var calls []roachpb.NodeID - sendFn := func(ctx context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + sendFn := func(ctx context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { calls = append(calls, ba.Replica.NodeID) if ba.Replica.NodeID == 2 { - reply := &roachpb.BatchResponse{} - reply.Error = roachpb.NewError(nlhe) + reply := &kvpb.BatchResponse{} + reply.Error = kvpb.NewError(nlhe) return reply, nil } require.Equal(t, ba.Replica.NodeID, roachpb.NodeID(1)) @@ -1238,7 +1239,7 @@ func TestDistSenderIgnoresNLHEBasedOnOldRangeGeneration(t *testing.T) { Lease: cachedLease, }) - get := roachpb.NewGet(roachpb.Key("a"), false /* forUpdate */) + get := kvpb.NewGet(roachpb.Key("a"), false /* forUpdate */) _, pErr := kv.SendWrapped(ctx, ds, get) require.Nil(t, pErr) @@ -1314,7 +1315,7 @@ func TestDistSenderRetryOnTransportErrors(t *testing.T) { // how transport errors are retried by dist sender. secondReplicaTried := false - sendFn := func(ctx context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + sendFn := func(ctx context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { if ba.Replica.NodeID == 2 { return nil, errutil.WithMessage( netutil.NewInitialHeartBeatFailedError( @@ -1347,7 +1348,7 @@ func TestDistSenderRetryOnTransportErrors(t *testing.T) { Lease: cachedLease, }) - get := roachpb.NewGet(roachpb.Key("a"), false /* forUpdate */) + get := kvpb.NewGet(roachpb.Key("a"), false /* forUpdate */) _, pErr := kv.SendWrapped(ctx, ds, get) if spec.shouldRetry { require.True(t, secondReplicaTried, "Second replica was not retried") @@ -1402,7 +1403,7 @@ func TestDistSenderDownNodeEvictLeaseholder(t *testing.T) { Sequence: 2, } - transport := func(ctx context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + transport := func(ctx context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { switch ba.Replica.StoreID { case 1: assert.Equal(t, desc.Generation, ba.ClientRangeInfo.DescriptorGeneration) @@ -1452,9 +1453,9 @@ func TestDistSenderDownNodeEvictLeaseholder(t *testing.T) { ClosedTimestampPolicy: roachpb.LEAD_FOR_GLOBAL_READS, }) - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.RangeID = 1 - get := &roachpb.GetRequest{} + get := &kvpb.GetRequest{} get.Key = roachpb.Key("a") ba.Add(get) @@ -1513,7 +1514,7 @@ func TestRetryOnDescriptorLookupError(t *testing.T) { Settings: cluster.MakeTestingClusterSettings(), } ds := NewDistSender(cfg) - put := roachpb.NewPut(roachpb.Key("a"), roachpb.MakeValueFromString("value")) + put := kvpb.NewPut(roachpb.Key("a"), roachpb.MakeValueFromString("value")) // Error on descriptor lookup, second attempt successful. if _, pErr := kv.SendWrapped(context.Background(), ds, put); pErr != nil { t.Errorf("unexpected error: %s", pErr) @@ -1539,8 +1540,8 @@ func TestEvictOnFirstRangeGossip(t *testing.T) { g := makeGossip(t, stopper, rpcContext) sender := func( - _ context.Context, ba *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, *roachpb.Error) { + _ context.Context, ba *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, *kvpb.Error) { return ba.CreateReply(), nil } @@ -1661,7 +1662,7 @@ func TestEvictCacheOnError(t *testing.T) { }, } - rangeMismachErr := roachpb.NewRangeKeyMismatchError( + rangeMismachErr := kvpb.NewRangeKeyMismatchError( context.Background(), nil, nil, &lhs, nil /* lease */) rangeMismachErr.AppendRangeInfo(context.Background(), roachpb.RangeInfo{Desc: rhs, Lease: roachpb.Lease{}}) @@ -1671,11 +1672,11 @@ func TestEvictCacheOnError(t *testing.T) { shouldClearLeaseHolder bool shouldClearReplica bool }{ - {false, errors.New(errString), false, false}, // non-retryable replica error - {false, rangeMismachErr, false, false}, // RangeKeyMismatch replica error - {false, &roachpb.RangeNotFoundError{}, false, false}, // RangeNotFound replica error - {false, nil, false, false}, // RPC error - {true, nil, false, false}, // canceled context + {false, errors.New(errString), false, false}, // non-retryable replica error + {false, rangeMismachErr, false, false}, // RangeKeyMismatch replica error + {false, &kvpb.RangeNotFoundError{}, false, false}, // RangeNotFound replica error + {false, nil, false, false}, // RPC error + {true, nil, false, false}, // canceled context } for _, tc := range testCases { @@ -1695,7 +1696,7 @@ func TestEvictCacheOnError(t *testing.T) { ctx, cancel := context.WithCancel(ctx) - var testFn simpleSendFn = func(ctx context.Context, args *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + var testFn simpleSendFn = func(ctx context.Context, args *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { if !first { return args.CreateReply(), nil } @@ -1707,8 +1708,8 @@ func TestEvictCacheOnError(t *testing.T) { if tc.replicaError == nil { return nil, errors.New(errString) } - reply := &roachpb.BatchResponse{} - reply.Error = roachpb.NewError(tc.replicaError) + reply := &kvpb.BatchResponse{} + reply.Error = kvpb.NewError(tc.replicaError) return reply, nil } @@ -1734,7 +1735,7 @@ func TestEvictCacheOnError(t *testing.T) { }) key := roachpb.Key("b") - put := roachpb.NewPut(key, roachpb.MakeValueFromString("value")) + put := kvpb.NewPut(key, roachpb.MakeValueFromString("value")) if _, pErr := kv.SendWrapped(ctx, ds, put); pErr != nil && !testutils.IsPError(pErr, errString) && !testutils.IsError(pErr.GoError(), ctx.Err().Error()) { t.Errorf("put encountered unexpected error: %s", pErr) @@ -1769,22 +1770,22 @@ func TestEvictCacheOnUnknownLeaseHolder(t *testing.T) { } var count int32 - testFn := func(_ context.Context, args *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + testFn := func(_ context.Context, args *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { var err error switch count { case 0, 1: - err = &roachpb.NotLeaseHolderError{ + err = &kvpb.NotLeaseHolderError{ Lease: &roachpb.Lease{ Replica: roachpb.ReplicaDescriptor{NodeID: 99, StoreID: 999}}, } case 2: - err = roachpb.NewRangeNotFoundError(0, 0) + err = kvpb.NewRangeNotFoundError(0, 0) default: return args.CreateReply(), nil } count++ - reply := &roachpb.BatchResponse{} - reply.Error = roachpb.NewError(err) + reply := &kvpb.BatchResponse{} + reply.Error = kvpb.NewError(err) return reply, nil } @@ -1802,7 +1803,7 @@ func TestEvictCacheOnUnknownLeaseHolder(t *testing.T) { } ds := NewDistSender(cfg) key := roachpb.Key("a") - put := roachpb.NewPut(key, roachpb.MakeValueFromString("value")) + put := kvpb.NewPut(key, roachpb.MakeValueFromString("value")) if _, pErr := kv.SendWrapped(context.Background(), ds, put); pErr != nil { t.Errorf("put encountered unexpected error: %s", pErr) @@ -1836,15 +1837,15 @@ func TestRetryOnWrongReplicaError(t *testing.T) { newRangeDescriptor.EndKey = badEndKey descStale := true - var testFn simpleSendFn = func(ctx context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + var testFn simpleSendFn = func(ctx context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { rs, err := keys.Range(ba.Requests) if err != nil { t.Fatal(err) } if kv.TestingIsRangeLookup(ba) { if bytes.HasPrefix(rs.Key, keys.Meta1Prefix) { - br := &roachpb.BatchResponse{} - r := &roachpb.ScanResponse{} + br := &kvpb.BatchResponse{} + r := &kvpb.ScanResponse{} var kv roachpb.KeyValue if err := kv.Value.SetProto(&TestMetaRangeDescriptor); err != nil { t.Fatal(err) @@ -1858,8 +1859,8 @@ func TestRetryOnWrongReplicaError(t *testing.T) { t.Fatalf("unexpected extra lookup for non-stale replica descriptor at %s", rs.Key) } - br := &roachpb.BatchResponse{} - r := &roachpb.ScanResponse{} + br := &kvpb.BatchResponse{} + r := &kvpb.ScanResponse{} var kv roachpb.KeyValue if err := kv.Value.SetProto(&newRangeDescriptor); err != nil { t.Fatal(err) @@ -1880,7 +1881,7 @@ func TestRetryOnWrongReplicaError(t *testing.T) { // When the Scan first turns up, update the descriptor for future // range descriptor lookups. if !newRangeDescriptor.EndKey.Equal(goodEndKey) { - return nil, &roachpb.RangeKeyMismatchError{ + return nil, &kvpb.RangeKeyMismatchError{ RequestStartKey: rs.Key.AsRawKey(), RequestEndKey: rs.EndKey.AsRawKey(), } @@ -1901,7 +1902,7 @@ func TestRetryOnWrongReplicaError(t *testing.T) { Settings: cluster.MakeTestingClusterSettings(), } ds := NewDistSender(cfg) - scan := roachpb.NewScan(roachpb.Key("a"), roachpb.Key("d"), false) + scan := kvpb.NewScan(roachpb.Key("a"), roachpb.Key("d"), false) if _, err := kv.SendWrapped(context.Background(), ds, scan); err != nil { t.Errorf("scan encountered error: %s", err) } @@ -1939,15 +1940,15 @@ func TestRetryOnWrongReplicaErrorWithSuggestion(t *testing.T) { rhsDesc.Generation = staleDesc.Generation + 2 firstLookup := true - var testFn simpleSendFn = func(ctx context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + var testFn simpleSendFn = func(ctx context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { rs, err := keys.Range(ba.Requests) if err != nil { panic(err) } if kv.TestingIsRangeLookup(ba) { if bytes.HasPrefix(rs.Key, keys.Meta1Prefix) { - br := &roachpb.BatchResponse{} - r := &roachpb.ScanResponse{} + br := &kvpb.BatchResponse{} + r := &kvpb.ScanResponse{} var kv roachpb.KeyValue if err := kv.Value.SetProto(&TestMetaRangeDescriptor); err != nil { panic(err) @@ -1958,14 +1959,14 @@ func TestRetryOnWrongReplicaErrorWithSuggestion(t *testing.T) { } if !firstLookup { - br := &roachpb.BatchResponse{} - br.Error = roachpb.NewErrorf("unexpected extra lookup for non-stale replica descriptor at %s", rs.Key) + br := &kvpb.BatchResponse{} + br.Error = kvpb.NewErrorf("unexpected extra lookup for non-stale replica descriptor at %s", rs.Key) return br, nil } firstLookup = false - br := &roachpb.BatchResponse{} - r := &roachpb.ScanResponse{} + br := &kvpb.BatchResponse{} + r := &kvpb.ScanResponse{} var kv roachpb.KeyValue if err := kv.Value.SetProto(&staleDesc); err != nil { panic(err) @@ -1978,10 +1979,10 @@ func TestRetryOnWrongReplicaErrorWithSuggestion(t *testing.T) { // When the Scan first turns up, provide the correct descriptor as a // suggestion for future range descriptor lookups. if ba.RangeID == staleDesc.RangeID { - var br roachpb.BatchResponse - err := roachpb.NewRangeKeyMismatchError(ctx, rs.Key.AsRawKey(), rs.EndKey.AsRawKey(), &rhsDesc, nil /* lease */) + var br kvpb.BatchResponse + err := kvpb.NewRangeKeyMismatchError(ctx, rs.Key.AsRawKey(), rs.EndKey.AsRawKey(), &rhsDesc, nil /* lease */) err.AppendRangeInfo(ctx, roachpb.RangeInfo{Desc: lhsDesc, Lease: roachpb.Lease{}}) - br.Error = roachpb.NewError(err) + br.Error = kvpb.NewError(err) return &br, nil } else if ba.RangeID != lhsDesc.RangeID { t.Fatalf("unexpected RangeID %d provided in request %v. expected: %s", ba.RangeID, ba, lhsDesc.RangeID) @@ -2005,7 +2006,7 @@ func TestRetryOnWrongReplicaErrorWithSuggestion(t *testing.T) { RPCRetryOptions: &retry.Options{MaxRetries: 1}, } ds := NewDistSender(cfg) - scan := roachpb.NewScan(roachpb.Key("a"), roachpb.Key("d"), false) + scan := kvpb.NewScan(roachpb.Key("a"), roachpb.Key("d"), false) if _, err := kv.SendWrapped(context.Background(), ds, scan); err != nil { t.Errorf("scan encountered error: %s", err) } @@ -2106,9 +2107,9 @@ func TestSendRPCRetry(t *testing.T) { descriptor, ) - var testFn simpleSendFn = func(ctx context.Context, args *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { - batchReply := &roachpb.BatchResponse{} - reply := &roachpb.ScanResponse{} + var testFn simpleSendFn = func(ctx context.Context, args *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { + batchReply := &kvpb.BatchResponse{} + reply := &kvpb.ScanResponse{} batchReply.Add(reply) reply.Rows = append([]roachpb.KeyValue{}, roachpb.KeyValue{Key: roachpb.Key("b"), Value: roachpb.Value{}}) return batchReply, nil @@ -2125,12 +2126,12 @@ func TestSendRPCRetry(t *testing.T) { Settings: cluster.MakeTestingClusterSettings(), } ds := NewDistSender(cfg) - scan := roachpb.NewScan(roachpb.Key("a"), roachpb.Key("d"), false) - sr, err := kv.SendWrappedWith(ctx, ds, roachpb.Header{MaxSpanRequestKeys: 1}, scan) + scan := kvpb.NewScan(roachpb.Key("a"), roachpb.Key("d"), false) + sr, err := kv.SendWrappedWith(ctx, ds, kvpb.Header{MaxSpanRequestKeys: 1}, scan) if err != nil { t.Fatal(err) } - if l := len(sr.(*roachpb.ScanResponse).Rows); l != 1 { + if l := len(sr.(*kvpb.ScanResponse).Rows); l != 1 { t.Fatalf("expected 1 row; got %d", l) } } @@ -2227,9 +2228,9 @@ func TestDistSenderDescriptorUpdatesOnSuccessfulRPCs(t *testing.T) { } { t.Run("", func(t *testing.T) { descDB := mockRangeDescriptorDBForDescs(TestMetaRangeDescriptor, desc) - var testFn simpleSendFn = func(ctx context.Context, args *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { - batchReply := &roachpb.BatchResponse{} - reply := &roachpb.GetResponse{} + var testFn simpleSendFn = func(ctx context.Context, args *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { + batchReply := &kvpb.BatchResponse{} + reply := &kvpb.GetResponse{} batchReply.Add(reply) // Return updated descriptors. batchReply.RangeInfos = tc @@ -2251,8 +2252,8 @@ func TestDistSenderDescriptorUpdatesOnSuccessfulRPCs(t *testing.T) { // Send a request that's going to receive a response with a RangeInfo. k := roachpb.Key("a") - get := roachpb.NewGet(k, false /* forUpdate */) - ba := &roachpb.BatchRequest{} + get := kvpb.NewGet(k, false /* forUpdate */) + ba := &kvpb.BatchRequest{} ba.Add(get) _, pErr := ds.Send(ctx, ba) require.Nil(t, pErr) @@ -2322,10 +2323,10 @@ func TestSendRPCRangeNotFoundError(t *testing.T) { seen := map[roachpb.ReplicaID]struct{}{} var leaseholderStoreID roachpb.StoreID var ds *DistSender - var testFn simpleSendFn = func(ctx context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + var testFn simpleSendFn = func(ctx context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { br := ba.CreateReply() if _, ok := seen[ba.Replica.ReplicaID]; ok { - br.Error = roachpb.NewErrorf("visited replica %+v twice", ba.Replica) + br.Error = kvpb.NewErrorf("visited replica %+v twice", ba.Replica) return br, nil } seen[ba.Replica.ReplicaID] = struct{}{} @@ -2339,7 +2340,7 @@ func TestSendRPCRangeNotFoundError(t *testing.T) { Lease: roachpb.Lease{Replica: ba.Replica}, }) } - br.Error = roachpb.NewError(roachpb.NewRangeNotFoundError(ba.RangeID, ba.Replica.StoreID)) + br.Error = kvpb.NewError(kvpb.NewRangeNotFoundError(ba.RangeID, ba.Replica.StoreID)) return br, nil } leaseholderStoreID = ba.Replica.StoreID @@ -2364,7 +2365,7 @@ func TestSendRPCRangeNotFoundError(t *testing.T) { Settings: cluster.MakeTestingClusterSettings(), } ds = NewDistSender(cfg) - get := roachpb.NewGet(roachpb.Key("b"), false /* forUpdate */) + get := kvpb.NewGet(roachpb.Key("b"), false /* forUpdate */) _, err := kv.SendWrapped(ctx, ds, get) if err != nil { t.Fatal(err) @@ -2408,7 +2409,7 @@ func TestMultiRangeGapReverse(t *testing.T) { } sender := kv.SenderFunc( - func(_ context.Context, args *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + func(_ context.Context, args *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { rb := args.CreateReply() return rb, nil }) @@ -2456,10 +2457,10 @@ func TestMultiRangeGapReverse(t *testing.T) { 1, // coordinatorNodeID ) - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Txn = &txn - ba.Add(roachpb.NewReverseScan(splits[0], splits[1], false)) - ba.Add(roachpb.NewReverseScan(splits[2], splits[3], false)) + ba.Add(kvpb.NewReverseScan(splits[0], splits[1], false)) + ba.Add(kvpb.NewReverseScan(splits[2], splits[3], false)) // Before fixing https://github.com/cockroachdb/cockroach/issues/18174, this // would error with: @@ -2517,13 +2518,13 @@ func TestMultiRangeMergeStaleDescriptor(t *testing.T) { {Key: roachpb.Key("a"), Value: roachpb.MakeValueFromString("1")}, {Key: roachpb.Key("c"), Value: roachpb.MakeValueFromString("2")}, } - testFn := func(_ context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + testFn := func(_ context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { rs, err := keys.Range(ba.Requests) if err != nil { t.Fatal(err) } - batchReply := &roachpb.BatchResponse{} - reply := &roachpb.ScanResponse{} + batchReply := &kvpb.BatchResponse{} + reply := &kvpb.ScanResponse{} batchReply.Add(reply) results := []roachpb.KeyValue{} for _, curKV := range existingKVs { @@ -2560,16 +2561,16 @@ func TestMultiRangeMergeStaleDescriptor(t *testing.T) { Settings: cluster.MakeTestingClusterSettings(), } ds := NewDistSender(cfg) - scan := roachpb.NewScan(roachpb.Key("a"), roachpb.Key("d"), false) + scan := kvpb.NewScan(roachpb.Key("a"), roachpb.Key("d"), false) // Set the Txn info to avoid an OpRequiresTxnError. - reply, err := kv.SendWrappedWith(ctx, ds, roachpb.Header{ + reply, err := kv.SendWrappedWith(ctx, ds, kvpb.Header{ MaxSpanRequestKeys: 10, Txn: &roachpb.Transaction{}, }, scan) if err != nil { t.Fatalf("scan encountered error: %s", err) } - sr := reply.(*roachpb.ScanResponse) + sr := reply.(*kvpb.ScanResponse) if !reflect.DeepEqual(existingKVs, sr.Rows) { t.Fatalf("expect get %v, actual get %v", existingKVs, sr.Rows) } @@ -2607,8 +2608,8 @@ func TestRangeLookupOptionOnReverseScan(t *testing.T) { Settings: cluster.MakeTestingClusterSettings(), } ds := NewDistSender(cfg) - rScan := &roachpb.ReverseScanRequest{ - RequestHeader: roachpb.RequestHeader{Key: roachpb.Key("a"), EndKey: roachpb.Key("b")}, + rScan := &kvpb.ReverseScanRequest{ + RequestHeader: kvpb.RequestHeader{Key: roachpb.Key("a"), EndKey: roachpb.Key("b")}, } if _, err := kv.SendWrapped(ctx, ds, rScan); err != nil { t.Fatal(err) @@ -2638,10 +2639,10 @@ func TestClockUpdateOnResponse(t *testing.T) { } ds := NewDistSender(cfg) - expectedErr := roachpb.NewError(errors.New("boom")) + expectedErr := kvpb.NewError(errors.New("boom")) // Prepare the test function - put := roachpb.NewPut(roachpb.Key("a"), roachpb.MakeValueFromString("value")) + put := kvpb.NewPut(roachpb.Key("a"), roachpb.MakeValueFromString("value")) doCheck := func(sender kv.Sender, fakeTime hlc.ClockTimestamp) { ds.transportFactory = SenderTransportFactory(tracing.NewTracer(), sender) _, err := kv.SendWrapped(ctx, ds, put) @@ -2657,7 +2658,7 @@ func TestClockUpdateOnResponse(t *testing.T) { // Test timestamp propagation on valid BatchResults. fakeTime := ds.clock.Now().Add(10000000000 /*10s*/, 0).UnsafeToClockTimestamp() replyNormal := kv.SenderFunc( - func(_ context.Context, args *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + func(_ context.Context, args *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { rb := args.CreateReply() rb.Now = fakeTime return rb, nil @@ -2667,7 +2668,7 @@ func TestClockUpdateOnResponse(t *testing.T) { // Test timestamp propagation on errors. fakeTime = ds.clock.Now().Add(10000000000 /*10s*/, 0).UnsafeToClockTimestamp() replyError := kv.SenderFunc( - func(_ context.Context, _ *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + func(_ context.Context, _ *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { pErr := expectedErr pErr.Now = fakeTime return nil, pErr @@ -2742,7 +2743,7 @@ func TestTruncateWithSpanAndDescriptor(t *testing.T) { // requests. Because of parallelization, there's no guarantee // on the ordering of requests. var haveA, haveB bool - sendStub := func(ctx context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + sendStub := func(ctx context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { rs, err := keys.Range(ba.Requests) if err != nil { t.Fatal(err) @@ -2755,8 +2756,8 @@ func TestTruncateWithSpanAndDescriptor(t *testing.T) { t.Fatalf("Unexpected span %s", rs) } - batchReply := &roachpb.BatchResponse{} - reply := &roachpb.PutResponse{} + batchReply := &kvpb.BatchResponse{} + reply := &kvpb.PutResponse{} batchReply.Add(reply) return batchReply, nil } @@ -2782,15 +2783,15 @@ func TestTruncateWithSpanAndDescriptor(t *testing.T) { // In the second attempt, The range of the descriptor found in // the cache is ["a", "c"), but the put on "a" will not be // present. The request is truncated to contain only the put on "b". - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Txn = &roachpb.Transaction{Name: "test"} { val := roachpb.MakeValueFromString("val") - ba.Add(roachpb.NewPut(keys.MakeRangeKeyPrefix(roachpb.RKey("a")), val)) + ba.Add(kvpb.NewPut(keys.MakeRangeKeyPrefix(roachpb.RKey("a")), val)) } { val := roachpb.MakeValueFromString("val") - ba.Add(roachpb.NewPut(keys.MakeRangeKeyPrefix(roachpb.RKey("b")), val)) + ba.Add(kvpb.NewPut(keys.MakeRangeKeyPrefix(roachpb.RKey("b")), val)) } if _, pErr := ds.Send(ctx, ba); pErr != nil { @@ -2870,7 +2871,7 @@ func TestTruncateWithLocalSpanAndDescriptor(t *testing.T) { // Define our rpcSend stub which checks the span of the batch // requests. haveRequest := []bool{false, false, false} - sendStub := func(ctx context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + sendStub := func(ctx context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { h := ba.Requests[0].GetInner().Header() if h.Key.Equal(keys.RangeDescriptorKey(roachpb.RKey("a"))) && h.EndKey.Equal(keys.MakeRangeKeyPrefix(roachpb.RKey("b"))) { haveRequest[0] = true @@ -2882,8 +2883,8 @@ func TestTruncateWithLocalSpanAndDescriptor(t *testing.T) { t.Fatalf("Unexpected span [%s,%s)", h.Key, h.EndKey) } - batchReply := &roachpb.BatchResponse{} - reply := &roachpb.ScanResponse{} + batchReply := &kvpb.BatchResponse{} + reply := &kvpb.ScanResponse{} batchReply.Add(reply) return batchReply, nil } @@ -2909,9 +2910,9 @@ func TestTruncateWithLocalSpanAndDescriptor(t *testing.T) { // In the second attempt, The range of the descriptor found in // the cache is ["b", "d"), The request is truncated to contain // only the scan on local keys that address from "b" to "d". - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Txn = &roachpb.Transaction{Name: "test"} - ba.Add(roachpb.NewScan( + ba.Add(kvpb.NewScan( keys.RangeDescriptorKey(roachpb.RKey("a")), keys.RangeDescriptorKey(roachpb.RKey("c")), false /* forUpdate */)) @@ -2944,7 +2945,7 @@ func TestMultiRangeWithEndTxn(t *testing.T) { testCases := []struct { put1, put2, et roachpb.Key parCommit bool - exp [][]roachpb.Method + exp [][]kvpb.Method }{ { // Everything hits the first range, so we get a 1PC txn. @@ -2952,7 +2953,7 @@ func TestMultiRangeWithEndTxn(t *testing.T) { put2: roachpb.Key("a2"), et: roachpb.Key("a3"), parCommit: false, - exp: [][]roachpb.Method{{roachpb.Put, roachpb.Put, roachpb.EndTxn}}, + exp: [][]kvpb.Method{{kvpb.Put, kvpb.Put, kvpb.EndTxn}}, }, { // Everything hits the first range, so we get a 1PC txn. @@ -2961,7 +2962,7 @@ func TestMultiRangeWithEndTxn(t *testing.T) { put2: roachpb.Key("a2"), et: roachpb.Key("a3"), parCommit: true, - exp: [][]roachpb.Method{{roachpb.Put, roachpb.Put, roachpb.EndTxn}}, + exp: [][]kvpb.Method{{kvpb.Put, kvpb.Put, kvpb.EndTxn}}, }, { // Only EndTxn hits the second range. @@ -2969,7 +2970,7 @@ func TestMultiRangeWithEndTxn(t *testing.T) { put2: roachpb.Key("a2"), et: roachpb.Key("b"), parCommit: false, - exp: [][]roachpb.Method{{roachpb.Put, roachpb.Put}, {roachpb.EndTxn}}, + exp: [][]kvpb.Method{{kvpb.Put, kvpb.Put}, {kvpb.EndTxn}}, }, { // Only EndTxn hits the second range. However, since the EndTxn is @@ -2981,7 +2982,7 @@ func TestMultiRangeWithEndTxn(t *testing.T) { put2: roachpb.Key("a2"), et: roachpb.Key("b"), parCommit: true, - exp: [][]roachpb.Method{{roachpb.Put, roachpb.Put}, {roachpb.EndTxn}}, + exp: [][]kvpb.Method{{kvpb.Put, kvpb.Put}, {kvpb.EndTxn}}, }, { // One write hits the second range, so EndTxn has to be split off. @@ -2992,7 +2993,7 @@ func TestMultiRangeWithEndTxn(t *testing.T) { put2: roachpb.Key("b1"), et: roachpb.Key("a1"), parCommit: false, - exp: [][]roachpb.Method{{roachpb.Put}, {roachpb.Put}, {roachpb.EndTxn}}, + exp: [][]kvpb.Method{{kvpb.Put}, {kvpb.Put}, {kvpb.EndTxn}}, }, { // One write hits the second range. Again, EndTxn does not need to @@ -3002,7 +3003,7 @@ func TestMultiRangeWithEndTxn(t *testing.T) { put2: roachpb.Key("b1"), et: roachpb.Key("a1"), parCommit: true, - exp: [][]roachpb.Method{{roachpb.Put, roachpb.EndTxn}, {roachpb.Put}}, + exp: [][]kvpb.Method{{kvpb.Put, kvpb.EndTxn}, {kvpb.Put}}, }, { // Both writes go to the second range, but not EndTxn. It is split @@ -3011,7 +3012,7 @@ func TestMultiRangeWithEndTxn(t *testing.T) { put2: roachpb.Key("b2"), et: roachpb.Key("a1"), parCommit: false, - exp: [][]roachpb.Method{{roachpb.Put, roachpb.Put}, {roachpb.EndTxn}}, + exp: [][]kvpb.Method{{kvpb.Put, kvpb.Put}, {kvpb.EndTxn}}, }, { // Both writes go to the second range, but not EndTxn. Since the @@ -3023,7 +3024,7 @@ func TestMultiRangeWithEndTxn(t *testing.T) { put2: roachpb.Key("b2"), et: roachpb.Key("a1"), parCommit: true, - exp: [][]roachpb.Method{{roachpb.EndTxn}, {roachpb.Put, roachpb.Put}}, + exp: [][]kvpb.Method{{kvpb.EndTxn}, {kvpb.Put, kvpb.Put}}, }, } @@ -3069,9 +3070,9 @@ func TestMultiRangeWithEndTxn(t *testing.T) { ) for i, test := range testCases { - var act [][]roachpb.Method - testFn := func(ctx context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { - var cur []roachpb.Method + var act [][]kvpb.Method + testFn := func(ctx context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { + var cur []kvpb.Method for _, union := range ba.Requests { cur = append(cur, union.GetInner().Method()) } @@ -3094,12 +3095,12 @@ func TestMultiRangeWithEndTxn(t *testing.T) { ds.DisableParallelBatches() // Send a batch request containing two puts. - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Txn = &roachpb.Transaction{Name: "test"} - ba.Add(roachpb.NewPut(test.put1, roachpb.MakeValueFromString("val1"))) - ba.Add(roachpb.NewPut(test.put2, roachpb.MakeValueFromString("val2"))) - et := &roachpb.EndTxnRequest{ - RequestHeader: roachpb.RequestHeader{Key: test.et}, + ba.Add(kvpb.NewPut(test.put1, roachpb.MakeValueFromString("val1"))) + ba.Add(kvpb.NewPut(test.put2, roachpb.MakeValueFromString("val2"))) + et := &kvpb.EndTxnRequest{ + RequestHeader: kvpb.RequestHeader{Key: test.et}, Commit: true, } if test.parCommit { @@ -3141,70 +3142,70 @@ func TestParallelCommitSplitFromQueryIntents(t *testing.T) { g := makeGossip(t, stopper, rpcContext) keyA, keyB := roachpb.Key("a"), roachpb.Key("ab") - put1 := roachpb.NewPut(keyA, roachpb.MakeValueFromString("val1")) - put2 := roachpb.NewPut(keyB, roachpb.MakeValueFromString("val2")) - qi := &roachpb.QueryIntentRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}} - et := &roachpb.EndTxnRequest{ - RequestHeader: roachpb.RequestHeader{Key: keyA}, + put1 := kvpb.NewPut(keyA, roachpb.MakeValueFromString("val1")) + put2 := kvpb.NewPut(keyB, roachpb.MakeValueFromString("val2")) + qi := &kvpb.QueryIntentRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}} + et := &kvpb.EndTxnRequest{ + RequestHeader: kvpb.RequestHeader{Key: keyA}, Commit: true, } - etPar := &roachpb.EndTxnRequest{ - RequestHeader: roachpb.RequestHeader{Key: keyA}, + etPar := &kvpb.EndTxnRequest{ + RequestHeader: kvpb.RequestHeader{Key: keyA}, Commit: true, InFlightWrites: []roachpb.SequencedWrite{{Key: keyA, Sequence: 1}, {Key: keyB, Sequence: 2}}, } testCases := []struct { name string - reqs []roachpb.Request - exp [][]roachpb.Method + reqs []kvpb.Request + exp [][]kvpb.Method }{ { name: "no parallel commits or query intents", - reqs: []roachpb.Request{put1, put2, et}, - exp: [][]roachpb.Method{{roachpb.Put, roachpb.Put, roachpb.EndTxn}}, + reqs: []kvpb.Request{put1, put2, et}, + exp: [][]kvpb.Method{{kvpb.Put, kvpb.Put, kvpb.EndTxn}}, }, { name: "no parallel commits, but regular and pre-commit query intents", - reqs: []roachpb.Request{qi, put1, put2, qi, et}, - exp: [][]roachpb.Method{ - {roachpb.QueryIntent, roachpb.Put, roachpb.Put, roachpb.QueryIntent, roachpb.EndTxn}, + reqs: []kvpb.Request{qi, put1, put2, qi, et}, + exp: [][]kvpb.Method{ + {kvpb.QueryIntent, kvpb.Put, kvpb.Put, kvpb.QueryIntent, kvpb.EndTxn}, }, }, { name: "parallel commits without query intents", - reqs: []roachpb.Request{put1, put2, etPar}, - exp: [][]roachpb.Method{{roachpb.Put, roachpb.Put, roachpb.EndTxn}}, + reqs: []kvpb.Request{put1, put2, etPar}, + exp: [][]kvpb.Method{{kvpb.Put, kvpb.Put, kvpb.EndTxn}}, }, { name: "parallel commits with pre-commit query intents", - reqs: []roachpb.Request{put1, put2, qi, qi, etPar}, - exp: [][]roachpb.Method{ - {roachpb.QueryIntent, roachpb.QueryIntent}, - {roachpb.Put, roachpb.Put, roachpb.EndTxn}, + reqs: []kvpb.Request{put1, put2, qi, qi, etPar}, + exp: [][]kvpb.Method{ + {kvpb.QueryIntent, kvpb.QueryIntent}, + {kvpb.Put, kvpb.Put, kvpb.EndTxn}, }, }, { name: "parallel commits with regular query intents", - reqs: []roachpb.Request{qi, put1, qi, put2, etPar}, - exp: [][]roachpb.Method{ - {roachpb.QueryIntent, roachpb.Put, roachpb.QueryIntent, roachpb.Put, roachpb.EndTxn}, + reqs: []kvpb.Request{qi, put1, qi, put2, etPar}, + exp: [][]kvpb.Method{ + {kvpb.QueryIntent, kvpb.Put, kvpb.QueryIntent, kvpb.Put, kvpb.EndTxn}, }, }, { name: "parallel commits with regular and pre-commit query intents", - reqs: []roachpb.Request{qi, put1, put2, qi, qi, qi, etPar}, - exp: [][]roachpb.Method{ - {roachpb.QueryIntent, roachpb.QueryIntent, roachpb.QueryIntent}, - {roachpb.QueryIntent, roachpb.Put, roachpb.Put, roachpb.EndTxn}, + reqs: []kvpb.Request{qi, put1, put2, qi, qi, qi, etPar}, + exp: [][]kvpb.Method{ + {kvpb.QueryIntent, kvpb.QueryIntent, kvpb.QueryIntent}, + {kvpb.QueryIntent, kvpb.Put, kvpb.Put, kvpb.EndTxn}, }, }, } for _, test := range testCases { t.Run(test.name, func(t *testing.T) { - var act [][]roachpb.Method - testFn := func(ctx context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { - var cur []roachpb.Method + var act [][]kvpb.Method + testFn := func(ctx context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { + var cur []kvpb.Method for _, union := range ba.Requests { cur = append(cur, union.GetInner().Method()) } @@ -3227,7 +3228,7 @@ func TestParallelCommitSplitFromQueryIntents(t *testing.T) { ds.DisableParallelBatches() // Send a batch request containing the requests. - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Txn = &roachpb.Transaction{Name: "test"} ba.Add(test.reqs...) @@ -3314,15 +3315,15 @@ func TestParallelCommitsDetectIntentMissingCause(t *testing.T) { } for _, test := range testCases { t.Run(test.name, func(t *testing.T) { - testFn := func(ctx context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + testFn := func(ctx context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { br := ba.CreateReply() switch ba.Requests[0].GetInner().Method() { - case roachpb.QueryIntent: - br.Error = roachpb.NewError(roachpb.NewIntentMissingError(key, nil)) - case roachpb.QueryTxn: + case kvpb.QueryIntent: + br.Error = kvpb.NewError(kvpb.NewIntentMissingError(key, nil)) + case kvpb.QueryTxn: status, txnRecordPresent, err := test.queryTxnFn() if err != nil { - br.Error = roachpb.NewError(err) + br.Error = kvpb.NewError(err) } else { if !txnRecordPresent { // A missing txn record doesn't make sense for some statuses. @@ -3334,7 +3335,7 @@ func TestParallelCommitsDetectIntentMissingCause(t *testing.T) { resp.QueriedTxn = respTxn resp.TxnRecordExists = txnRecordPresent } - case roachpb.EndTxn: + case kvpb.EndTxn: br.Txn = ba.Txn.Clone() br.Txn.Status = roachpb.STAGING } @@ -3355,15 +3356,15 @@ func TestParallelCommitsDetectIntentMissingCause(t *testing.T) { ds := NewDistSender(cfg) // Send a parallel commit batch request. - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Txn = txn.Clone() - ba.Add(&roachpb.QueryIntentRequest{ - RequestHeader: roachpb.RequestHeader{Key: key}, + ba.Add(&kvpb.QueryIntentRequest{ + RequestHeader: kvpb.RequestHeader{Key: key}, Txn: txn.TxnMeta, ErrorIfMissing: true, }) - ba.Add(&roachpb.EndTxnRequest{ - RequestHeader: roachpb.RequestHeader{Key: key}, + ba.Add(&kvpb.EndTxnRequest{ + RequestHeader: kvpb.RequestHeader{Key: key}, Commit: true, InFlightWrites: []roachpb.SequencedWrite{{Key: key, Sequence: 1}}, }) @@ -3476,15 +3477,15 @@ func TestSenderTransport(t *testing.T) { kv.SenderFunc( func( _ context.Context, - _ *roachpb.BatchRequest, - ) (r *roachpb.BatchResponse, e *roachpb.Error) { + _ *kvpb.BatchRequest, + ) (r *kvpb.BatchResponse, e *kvpb.Error) { return }, ))(SendOptions{}, &nodedialer.Dialer{}, ReplicaSlice{{}}) if err != nil { t.Fatal(err) } - _, err = transport.SendNext(context.Background(), &roachpb.BatchRequest{}) + _, err = transport.SendNext(context.Background(), &kvpb.BatchRequest{}) if err != nil { t.Fatal(err) } @@ -3514,7 +3515,7 @@ func TestGatewayNodeID(t *testing.T) { } var observedNodeID roachpb.NodeID - testFn := func(ctx context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + testFn := func(ctx context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { observedNodeID = ba.Header.GatewayNodeID return ba.CreateReply(), nil } @@ -3531,8 +3532,8 @@ func TestGatewayNodeID(t *testing.T) { Settings: cluster.MakeTestingClusterSettings(), } ds := NewDistSender(cfg) - ba := &roachpb.BatchRequest{} - ba.Add(roachpb.NewPut(roachpb.Key("a"), roachpb.MakeValueFromString("value"))) + ba := &kvpb.BatchRequest{} + ba.Add(kvpb.NewPut(roachpb.Key("a"), roachpb.MakeValueFromString("value"))) if _, err := ds.Send(context.Background(), ba); err != nil { t.Fatalf("put encountered error: %s", err) } @@ -3605,13 +3606,13 @@ func TestMultipleErrorsMerged(t *testing.T) { err1WriteTimestamp := txn.WriteTimestamp.Add(100, 0) err2WriteTimestamp := txn.WriteTimestamp.Add(200, 0) - retryErr := roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE, "test err") - abortErr := roachpb.NewTransactionAbortedError(roachpb.ABORT_REASON_ABORTED_RECORD_FOUND) - conditionFailedErr := &roachpb.ConditionFailedError{} - writeIntentErr := &roachpb.WriteIntentError{} + retryErr := kvpb.NewTransactionRetryError(kvpb.RETRY_SERIALIZABLE, "test err") + abortErr := kvpb.NewTransactionAbortedError(kvpb.ABORT_REASON_ABORTED_RECORD_FOUND) + conditionFailedErr := &kvpb.ConditionFailedError{} + writeIntentErr := &kvpb.WriteIntentError{} sendErr := sendError{} - ambiguousErr := &roachpb.AmbiguousResultError{} - randomErr := &roachpb.IntegerOverflowError{} + ambiguousErr := &kvpb.AmbiguousResultError{} + randomErr := &kvpb.IntegerOverflowError{} testCases := []struct { err1, err2 error @@ -3708,7 +3709,7 @@ func TestMultipleErrorsMerged(t *testing.T) { tc.err2 = err1 } - testFn := func(ctx context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + testFn := func(ctx context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { reply := ba.CreateReply() if delRng := ba.Requests[0].GetDeleteRange(); delRng == nil { return nil, errors.Errorf("expected DeleteRange request, found %v", ba.Requests[0]) @@ -3716,13 +3717,13 @@ func TestMultipleErrorsMerged(t *testing.T) { if tc.err1 != nil { errTxn := ba.Txn.Clone() errTxn.WriteTimestamp = err1WriteTimestamp - reply.Error = roachpb.NewErrorWithTxn(tc.err1, errTxn) + reply.Error = kvpb.NewErrorWithTxn(tc.err1, errTxn) } } else if delRng.Key.Equal(roachpb.Key("b")) { if tc.err2 != nil { errTxn := ba.Txn.Clone() errTxn.WriteTimestamp = err2WriteTimestamp - reply.Error = roachpb.NewErrorWithTxn(tc.err2, errTxn) + reply.Error = kvpb.NewErrorWithTxn(tc.err2, errTxn) } } else { return nil, errors.Errorf("unexpected DeleteRange boundaries") @@ -3744,9 +3745,9 @@ func TestMultipleErrorsMerged(t *testing.T) { } ds := NewDistSender(cfg) - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Txn = txn.Clone() - ba.Add(roachpb.NewDeleteRange(roachpb.Key("a"), roachpb.Key("c"), false /* returnKeys */)) + ba.Add(kvpb.NewDeleteRange(roachpb.Key("a"), roachpb.Key("c"), false /* returnKeys */)) expWriteTimestamp := txn.WriteTimestamp if tc.err1 != nil { @@ -3770,7 +3771,7 @@ func TestMultipleErrorsMerged(t *testing.T) { // Regression test for #20067. // If a batch is partitioned into multiple partial batches, the -// roachpb.Error.Index of each batch should correspond to its original index in +// kvpb.Error.Index of each batch should correspond to its original index in // the overall batch. func TestErrorIndexAlignment(t *testing.T) { defer leaktest.AfterTest(t)() @@ -3855,14 +3856,14 @@ func TestErrorIndexAlignment(t *testing.T) { t.Run(strconv.Itoa(i), func(t *testing.T) { nthRequest := 0 - var testFn simpleSendFn = func(ctx context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + var testFn simpleSendFn = func(ctx context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { reply := ba.CreateReply() if nthRequest == tc.nthPartialBatch { - reply.Error = roachpb.NewErrorf("foo") + reply.Error = kvpb.NewErrorf("foo") // The relative index is always 0 since // we return an error for the first // request of the nthPartialBatch. - reply.Error.Index = &roachpb.ErrPosition{Index: 0} + reply.Error.Index = &kvpb.ErrPosition{Index: 0} } nthRequest++ return reply, nil @@ -3882,21 +3883,21 @@ func TestErrorIndexAlignment(t *testing.T) { ds := NewDistSender(cfg) ds.DisableParallelBatches() - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Txn = &roachpb.Transaction{Name: "test"} // First batch has 1 request. val := roachpb.MakeValueFromString("val") - ba.Add(roachpb.NewPut(roachpb.Key("a"), val)) + ba.Add(kvpb.NewPut(roachpb.Key("a"), val)) // Second batch has 2 requests. val = roachpb.MakeValueFromString("val") - ba.Add(roachpb.NewPut(roachpb.Key("b"), val)) + ba.Add(kvpb.NewPut(roachpb.Key("b"), val)) val = roachpb.MakeValueFromString("val") - ba.Add(roachpb.NewPut(roachpb.Key("bb"), val)) + ba.Add(kvpb.NewPut(roachpb.Key("bb"), val)) // Third batch has 1 request. val = roachpb.MakeValueFromString("val") - ba.Add(roachpb.NewPut(roachpb.Key("c"), val)) + ba.Add(kvpb.NewPut(roachpb.Key("c"), val)) _, pErr := ds.Send(ctx, ba) if pErr == nil { @@ -3926,7 +3927,7 @@ func TestCanSendToFollower(t *testing.T) { _ *cluster.Settings, _ *hlc.Clock, _ roachpb.RangeClosedTimestampPolicy, - ba *roachpb.BatchRequest, + ba *kvpb.BatchRequest, ) bool { return !ba.IsLocking() && canSend } @@ -3945,7 +3946,7 @@ func TestCanSendToFollower(t *testing.T) { } } var sentTo roachpb.ReplicaDescriptor - testFn := func(_ context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + testFn := func(_ context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { sentTo = ba.Replica return ba.CreateReply(), nil } @@ -3967,36 +3968,36 @@ func TestCanSendToFollower(t *testing.T) { } for i, c := range []struct { canSendToFollower bool - header roachpb.Header - msg roachpb.Request + header kvpb.Header + msg kvpb.Request expectedNode roachpb.NodeID }{ { true, - roachpb.Header{ + kvpb.Header{ Txn: &roachpb.Transaction{}, }, - roachpb.NewPut(roachpb.Key("a"), roachpb.Value{}), + kvpb.NewPut(roachpb.Key("a"), roachpb.Value{}), 2, }, { true, - roachpb.Header{ + kvpb.Header{ Txn: &roachpb.Transaction{}, }, - roachpb.NewGet(roachpb.Key("a"), false /* forUpdate */), + kvpb.NewGet(roachpb.Key("a"), false /* forUpdate */), 1, }, { true, - roachpb.Header{}, - roachpb.NewGet(roachpb.Key("a"), false /* forUpdate */), + kvpb.Header{}, + kvpb.NewGet(roachpb.Key("a"), false /* forUpdate */), 1, }, { false, - roachpb.Header{}, - roachpb.NewGet(roachpb.Key("a"), false /* forUpdate */), + kvpb.Header{}, + kvpb.NewGet(roachpb.Key("a"), false /* forUpdate */), 2, }, } { @@ -4089,7 +4090,7 @@ func TestEvictMetaRange(t *testing.T) { isStale := false - testFn := func(ctx context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + testFn := func(ctx context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { rs, err := keys.Range(ba.Requests) if err != nil { t.Fatal(err) @@ -4100,8 +4101,8 @@ func TestEvictMetaRange(t *testing.T) { if bytes.HasPrefix(rs.Key, keys.Meta1Prefix) { // Querying meta 1 range. - br := &roachpb.BatchResponse{} - r := &roachpb.ScanResponse{} + br := &kvpb.BatchResponse{} + r := &kvpb.ScanResponse{} var kv roachpb.KeyValue if rs.Key.Equal(keys.RangeMetaKey(keys.RangeMetaKey(roachpb.RKey("a")).Next()).Next()) { // Scan request is [/Meta1/a - /Meta2), so return the first meta1 @@ -4123,8 +4124,8 @@ func TestEvictMetaRange(t *testing.T) { return br, nil } // Querying meta2 range. - br := &roachpb.BatchResponse{} - r := &roachpb.ScanResponse{} + br := &kvpb.BatchResponse{} + r := &kvpb.ScanResponse{} var kv roachpb.KeyValue if rs.Key.Equal(keys.RangeMetaKey(roachpb.RKey("a")).Next()) { // Scan request is [/Meta2/a - /Meta2/b), so return the first @@ -4150,7 +4151,7 @@ func TestEvictMetaRange(t *testing.T) { reply := ba.CreateReply() // Return a RangeKeyMismatchError to simulate the range being stale. - err := roachpb.NewRangeKeyMismatchError( + err := kvpb.NewRangeKeyMismatchError( ctx, rs.Key.AsRawKey(), rs.EndKey.AsRawKey(), &testMeta2RangeDescriptor1, nil /* lease */) if hasSuggestedRange { ri := roachpb.RangeInfo{ @@ -4159,7 +4160,7 @@ func TestEvictMetaRange(t *testing.T) { } err.AppendRangeInfo(ctx, ri) } - reply.Error = roachpb.NewError(err) + reply.Error = kvpb.NewError(err) return reply, nil } else { // Scan request is [/Meta2/b - /Meta2/c) and the range descriptor is @@ -4187,7 +4188,7 @@ func TestEvictMetaRange(t *testing.T) { } ds := NewDistSender(cfg) - scan := roachpb.NewScan(roachpb.Key("a"), roachpb.Key("b"), false) + scan := kvpb.NewScan(roachpb.Key("a"), roachpb.Key("b"), false) if _, pErr := kv.SendWrapped(ctx, ds, scan); pErr != nil { t.Fatalf("scan encountered error: %s", pErr) } @@ -4202,7 +4203,7 @@ func TestEvictMetaRange(t *testing.T) { // Simulate a split on the meta2 range and mark it as stale. isStale = true - scan = roachpb.NewScan(roachpb.Key("b"), roachpb.Key("c"), false) + scan = kvpb.NewScan(roachpb.Key("b"), roachpb.Key("c"), false) if _, pErr := kv.SendWrapped(ctx, ds, scan); pErr != nil { t.Fatalf("scan encountered error: %s", pErr) } @@ -4271,7 +4272,7 @@ func TestConnectionClass(t *testing.T) { ) (Transport, error) { class = opts.class return adaptSimpleTransport( - func(_ context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + func(_ context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { return ba.CreateReply(), nil })(opts, dialer, replicas) } @@ -4304,9 +4305,9 @@ func TestConnectionClass(t *testing.T) { keys.SystemSQLCodec.TablePrefix(1234), // A non-system table } { t.Run(key.String(), func(t *testing.T) { - ba := &roachpb.BatchRequest{} - ba.Add(&roachpb.GetRequest{ - RequestHeader: roachpb.RequestHeader{ + ba := &kvpb.BatchRequest{} + ba.Add(&kvpb.GetRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, }, }) @@ -4375,11 +4376,11 @@ func TestEvictionTokenCoalesce(t *testing.T) { waitForInitialMeta2Scans := makeBarrier(2) var queriedMetaKeys sync.Map var ds *DistSender - testFn := func(ctx context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + testFn := func(ctx context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { rs, err := keys.Range(ba.Requests) br := ba.CreateReply() if err != nil { - br.Error = roachpb.NewError(err) + br.Error = kvpb.NewError(err) return br, nil } if !kv.TestingIsRangeLookup(ba) { @@ -4393,11 +4394,11 @@ func TestEvictionTokenCoalesce(t *testing.T) { if bytes.HasPrefix(rs.Key, keys.Meta1Prefix) { // Querying meta 1 range. - br = &roachpb.BatchResponse{} - r := &roachpb.ScanResponse{} + br = &kvpb.BatchResponse{} + r := &kvpb.ScanResponse{} var kv roachpb.KeyValue if err := kv.Value.SetProto(&TestMetaRangeDescriptor); err != nil { - br.Error = roachpb.NewError(err) + br.Error = kvpb.NewError(err) return br, nil } r.Rows = append(r.Rows, kv) @@ -4406,11 +4407,11 @@ func TestEvictionTokenCoalesce(t *testing.T) { } waitForInitialMeta2Scans() // Querying meta2 range. - br = &roachpb.BatchResponse{} - r := &roachpb.ScanResponse{} + br = &kvpb.BatchResponse{} + r := &kvpb.ScanResponse{} var kv roachpb.KeyValue if err := kv.Value.SetProto(&testUserRangeDescriptor); err != nil { - br.Error = roachpb.NewError(err) + br.Error = kvpb.NewError(err) return br, nil } r.Rows = append(r.Rows, kv) @@ -4428,7 +4429,7 @@ func TestEvictionTokenCoalesce(t *testing.T) { } return nil }); err != nil { - br.Error = roachpb.NewError(err) + br.Error = kvpb.NewError(err) return br, nil } } @@ -4456,7 +4457,7 @@ func TestEvictionTokenCoalesce(t *testing.T) { var batchWaitGroup sync.WaitGroup putFn := func(key, value string) { defer batchWaitGroup.Done() - put := roachpb.NewPut(roachpb.Key(key), roachpb.MakeValueFromString("c")) + put := kvpb.NewPut(roachpb.Key(key), roachpb.MakeValueFromString("c")) if _, pErr := kv.SendWrapped(context.Background(), ds, put); pErr != nil { t.Errorf("put encountered error: %s", pErr) } @@ -4474,12 +4475,12 @@ func TestDistSenderSlowLogMessage(t *testing.T) { dur = 8158 * time.Millisecond attempts = 120 ) - ba := &roachpb.BatchRequest{} - get := &roachpb.GetRequest{} + ba := &kvpb.BatchRequest{} + get := &kvpb.GetRequest{} get.Key = roachpb.Key("a") ba.Add(get) - br := &roachpb.BatchResponse{} - br.Error = roachpb.NewError(errors.New("boom")) + br := &kvpb.BatchResponse{} + br.Error = kvpb.NewError(errors.New("boom")) desc := &roachpb.RangeDescriptor{RangeID: 9, StartKey: roachpb.RKey("x"), EndKey: roachpb.RKey("z")} { exp := `have been waiting 8.16s (120 attempts) for RPC Get [‹"a"›,/Min) to` + @@ -4515,18 +4516,18 @@ func TestRequestSubdivisionAfterDescriptorChange(t *testing.T) { keyC := roachpb.Key("c") splitKey := keys.MustAddr(keyB) - get := func(k roachpb.Key) roachpb.Request { - return roachpb.NewGet(k, false /* forUpdate */) + get := func(k roachpb.Key) kvpb.Request { + return kvpb.NewGet(k, false /* forUpdate */) } - scan := func(k roachpb.Key) roachpb.Request { - return roachpb.NewScan(k, k.Next(), false /* forUpdate */) + scan := func(k roachpb.Key) kvpb.Request { + return kvpb.NewScan(k, k.Next(), false /* forUpdate */) } - revScan := func(k roachpb.Key) roachpb.Request { - return roachpb.NewReverseScan(k, k.Next(), false /* forUpdate */) + revScan := func(k roachpb.Key) kvpb.Request { + return kvpb.NewReverseScan(k, k.Next(), false /* forUpdate */) } for _, tc := range []struct { - req1, req2 func(roachpb.Key) roachpb.Request + req1, req2 func(roachpb.Key) kvpb.Request }{ {get, get}, {scan, get}, @@ -4583,14 +4584,14 @@ func TestRequestSubdivisionAfterDescriptorChange(t *testing.T) { } returnErr := true - transportFn := func(_ context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + transportFn := func(_ context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { if returnErr { // First time around we return an RPC error. Next time around, make sure // the DistSender tries gets the split descriptors. if len(ba.Requests) != 2 { // Sanity check - first attempt should have the unsplit batch. rep := ba.CreateReply() - rep.Error = roachpb.NewErrorf("expected divided batches with one request each, got: %s", ba) + rep.Error = kvpb.NewErrorf("expected divided batches with one request each, got: %s", ba) return rep, nil } switchToSplitDesc() @@ -4599,7 +4600,7 @@ func TestRequestSubdivisionAfterDescriptorChange(t *testing.T) { } rep := ba.CreateReply() if len(ba.Requests) != 1 { - rep.Error = roachpb.NewErrorf("expected divided batches with one request each, got: %s", ba) + rep.Error = kvpb.NewErrorf("expected divided batches with one request each, got: %s", ba) } return rep, nil } @@ -4627,11 +4628,11 @@ func TestRequestSubdivisionAfterDescriptorChange(t *testing.T) { // moment on, we check that the sent batches only consist of single requests - // which proves that the original batch was split. - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Add(tc.req1(keyA), tc.req2(keyC)) // Inconsistent read because otherwise the batch will ask to be re-sent in a // txn when split. - ba.ReadConsistency = roachpb.INCONSISTENT + ba.ReadConsistency = kvpb.INCONSISTENT _, pErr := ds.Send(ctx, ba) require.Nil(t, pErr) @@ -4654,8 +4655,8 @@ func TestRequestSubdivisionAfterDescriptorChangeWithUnavailableReplicasTerminate keyC := roachpb.Key("c") splitKey := keys.MustAddr(keyB) - get := func(k roachpb.Key) roachpb.Request { - return roachpb.NewGet(k, false /* forUpdate */) + get := func(k roachpb.Key) kvpb.Request { + return kvpb.NewGet(k, false /* forUpdate */) } ctx := context.Background() @@ -4688,7 +4689,7 @@ func TestRequestSubdivisionAfterDescriptorChangeWithUnavailableReplicasTerminate splitRDB := mockRangeDescriptorDBForDescs(splitDescs...) var numAttempts int32 - transportFn := func(_ context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + transportFn := func(_ context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { atomic.AddInt32(&numAttempts, 1) require.Equal(t, 1, len(ba.Requests)) return nil, newSendError("boom") @@ -4711,11 +4712,11 @@ func TestRequestSubdivisionAfterDescriptorChangeWithUnavailableReplicasTerminate ds := NewDistSender(cfg) - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Add(get(keyA), get(keyC)) // Inconsistent read because otherwise the batch will ask to be re-sent in a // txn when split. - ba.ReadConsistency = roachpb.INCONSISTENT + ba.ReadConsistency = kvpb.INCONSISTENT _, pErr := ds.Send(ctx, ba) require.NotNil(t, pErr) require.True(t, testutils.IsError(pErr.GoError(), "boom")) @@ -4747,18 +4748,18 @@ func TestDescriptorChangeAfterRequestSubdivision(t *testing.T) { laterSplitKey1 := keys.MustAddr(keyB) laterSplitKey2 := keys.MustAddr(keyD) - get := func(k roachpb.Key) roachpb.Request { - return roachpb.NewGet(k, false /* forUpdate */) + get := func(k roachpb.Key) kvpb.Request { + return kvpb.NewGet(k, false /* forUpdate */) } - scan := func(k roachpb.Key) roachpb.Request { - return roachpb.NewScan(k, k.Next(), false /* forUpdate */) + scan := func(k roachpb.Key) kvpb.Request { + return kvpb.NewScan(k, k.Next(), false /* forUpdate */) } - revScan := func(k roachpb.Key) roachpb.Request { - return roachpb.NewReverseScan(k, k.Next(), false /* forUpdate */) + revScan := func(k roachpb.Key) kvpb.Request { + return kvpb.NewReverseScan(k, k.Next(), false /* forUpdate */) } for _, tc := range []struct { - req1, req2 func(roachpb.Key) roachpb.Request + req1, req2 func(roachpb.Key) kvpb.Request }{ {get, get}, {scan, get}, @@ -4849,7 +4850,7 @@ func TestDescriptorChangeAfterRequestSubdivision(t *testing.T) { } var successes int32 - transportFn := func(ctx context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + transportFn := func(ctx context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { require.Len(t, ba.Requests, 1) switch ba.ClientRangeInfo.DescriptorGeneration { case 1: @@ -4885,11 +4886,11 @@ func TestDescriptorChangeAfterRequestSubdivision(t *testing.T) { // from the cache. Then, we'll switch the descriptor db that the DistSender // uses to the version that returns four ranges. - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Add(tc.req1(keyA), tc.req2(keyE)) // Inconsistent read because otherwise the batch will ask to be re-sent in a // txn when split. - ba.ReadConsistency = roachpb.INCONSISTENT + ba.ReadConsistency = kvpb.INCONSISTENT _, pErr := ds.Send(ctx, ba) require.Nil(t, pErr) @@ -5063,9 +5064,9 @@ func TestSendToReplicasSkipsStaleReplicas(t *testing.T) { tok := rc.MakeEvictionToken(&ent) numCalled := 0 - transportFn := func(_ context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + transportFn := func(_ context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { numCalled++ - nlhe := &roachpb.NotLeaseHolderError{ + nlhe := &kvpb.NotLeaseHolderError{ RangeID: tc.initialDesc.RangeID, Lease: &roachpb.Lease{ Replica: roachpb.ReplicaDescriptor{ @@ -5085,8 +5086,8 @@ func TestSendToReplicasSkipsStaleReplicas(t *testing.T) { } else { nlhe.RangeDesc = tc.updatedDesc } - br := &roachpb.BatchResponse{} - br.Error = roachpb.NewError(nlhe) + br := &kvpb.BatchResponse{} + br.Error = kvpb.NewError(nlhe) return br, nil } @@ -5111,8 +5112,8 @@ func TestSendToReplicasSkipsStaleReplicas(t *testing.T) { ds := NewDistSender(cfg) - ba := &roachpb.BatchRequest{} - get := &roachpb.GetRequest{} + ba := &kvpb.BatchRequest{} + get := &kvpb.GetRequest{} get.Key = roachpb.Key("a") ba.Add(get) _, err = ds.sendToReplicas(ctx, ba, tok, false /* withCommit */) @@ -5457,19 +5458,19 @@ func TestDistSenderDescEvictionAfterLeaseUpdate(t *testing.T) { // We'll send a request that first gets a NLHE, and then a RangeNotFoundError. We // then expect an updated descriptor to be used and return success. call := 0 - var transportFn = func(_ context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { - br := &roachpb.BatchResponse{} + var transportFn = func(_ context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { + br := &kvpb.BatchResponse{} switch call { case 0: expRepl := desc1.Replicas().Descriptors()[0] require.Equal(t, expRepl, ba.Replica) - br.Error = roachpb.NewError(&roachpb.NotLeaseHolderError{ + br.Error = kvpb.NewError(&kvpb.NotLeaseHolderError{ Lease: &roachpb.Lease{Replica: desc1.Replicas().Descriptors()[1]}, }) case 1: expRep := desc1.Replicas().Descriptors()[1] require.Equal(t, ba.Replica, expRep) - br.Error = roachpb.NewError(roachpb.NewRangeNotFoundError(ba.RangeID, ba.Replica.StoreID)) + br.Error = kvpb.NewError(kvpb.NewRangeNotFoundError(ba.RangeID, ba.Replica.StoreID)) case 2: expRep := desc2.Replicas().Descriptors()[0] require.Equal(t, ba.Replica, expRep) @@ -5511,8 +5512,8 @@ func TestDistSenderDescEvictionAfterLeaseUpdate(t *testing.T) { } ds := NewDistSender(cfg) - ba := &roachpb.BatchRequest{} - get := &roachpb.GetRequest{} + ba := &kvpb.BatchRequest{} + get := &kvpb.GetRequest{} get.Key = roachpb.Key("a") ba.Add(get) @@ -5548,14 +5549,14 @@ func TestDistSenderRPCMetrics(t *testing.T) { // We'll send a request that first gets a NLHE, and then a ConditionFailedError. call := 0 - var transportFn = func(_ context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { - br := &roachpb.BatchResponse{} + var transportFn = func(_ context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { + br := &kvpb.BatchResponse{} if call == 0 { - br.Error = roachpb.NewError(&roachpb.NotLeaseHolderError{ + br.Error = kvpb.NewError(&kvpb.NotLeaseHolderError{ Lease: &roachpb.Lease{Replica: desc.Replicas().Descriptors()[1]}, }) } else { - br.Error = roachpb.NewError(&roachpb.ConditionFailedError{}) + br.Error = kvpb.NewError(&kvpb.ConditionFailedError{}) } call++ return br, nil @@ -5584,18 +5585,18 @@ func TestDistSenderRPCMetrics(t *testing.T) { Replica: desc.Replicas().Descriptors()[0], }, }) - ba := &roachpb.BatchRequest{} - get := &roachpb.GetRequest{} + ba := &kvpb.BatchRequest{} + get := &kvpb.GetRequest{} get.Key = roachpb.Key("a") ba.Add(get) _, err := ds.Send(ctx, ba) require.Regexp(t, "unexpected value", err) - require.Equal(t, ds.metrics.MethodCounts[roachpb.Get].Count(), int64(1)) + require.Equal(t, ds.metrics.MethodCounts[kvpb.Get].Count(), int64(1)) // Expect that the metrics for both of the returned errors were incremented. - require.Equal(t, ds.metrics.ErrCounts[roachpb.NotLeaseHolderErrType].Count(), int64(1)) - require.Equal(t, ds.metrics.ErrCounts[roachpb.ConditionFailedErrType].Count(), int64(1)) + require.Equal(t, ds.metrics.ErrCounts[kvpb.NotLeaseHolderErrType].Count(), int64(1)) + require.Equal(t, ds.metrics.ErrCounts[kvpb.ConditionFailedErrType].Count(), int64(1)) } // TestDistSenderNLHEFromUninitializedReplicaDoesNotCauseUnboundedBackoff @@ -5655,27 +5656,27 @@ func TestDistSenderNLHEFromUninitializedReplicaDoesNotCauseUnboundedBackoff(t *t } call := 0 - var transportFn = func(_ context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { - br := &roachpb.BatchResponse{} + var transportFn = func(_ context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { + br := &kvpb.BatchResponse{} switch call { case 0: // We return an empty range descriptor in the NLHE like an // uninitialized replica would. expRepl := desc.Replicas().Descriptors()[0] require.Equal(t, expRepl, ba.Replica) - nlhe := &roachpb.NotLeaseHolderError{ + nlhe := &kvpb.NotLeaseHolderError{ RangeDesc: roachpb.RangeDescriptor{}, } if returnSpeculativeLease { nlhe.DeprecatedLeaseHolder = &roachpb.ReplicaDescriptor{NodeID: 5, StoreID: 5, ReplicaID: 5} } - br.Error = roachpb.NewError(nlhe) + br.Error = kvpb.NewError(nlhe) case 1: // We expect the client to discard information from the NLHE above and // instead just try the next replica. expRepl := desc.Replicas().Descriptors()[1] require.Equal(t, expRepl, ba.Replica) - br.Error = roachpb.NewError(&roachpb.NotLeaseHolderError{ + br.Error = kvpb.NewError(&kvpb.NotLeaseHolderError{ RangeDesc: desc, Lease: &leaseResp, }) @@ -5718,8 +5719,8 @@ func TestDistSenderNLHEFromUninitializedReplicaDoesNotCauseUnboundedBackoff(t *t } ds := NewDistSender(cfg) - ba := &roachpb.BatchRequest{} - get := &roachpb.GetRequest{} + ba := &kvpb.BatchRequest{} + get := &kvpb.GetRequest{} get.Key = roachpb.Key("a") ba.Add(get) @@ -5738,11 +5739,11 @@ func TestOptimisticRangeDescriptorLookups(t *testing.T) { defer leaktest.AfterTest(t)() type batchResponse struct { - br *roachpb.BatchResponse + br *kvpb.BatchResponse err error } type batchRequest struct { - ba *roachpb.BatchRequest + ba *kvpb.BatchRequest resp chan batchResponse } @@ -5772,7 +5773,7 @@ func TestOptimisticRangeDescriptorLookups(t *testing.T) { fr := mockFirstRangeProvider{d: firstRange} sendCh := make(chan batchRequest) - transportFn := func(ctx context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + transportFn := func(ctx context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { r := batchRequest{ba: ba, resp: make(chan batchResponse, 1)} select { case sendCh <- r: @@ -5802,10 +5803,10 @@ func TestOptimisticRangeDescriptorLookups(t *testing.T) { return sendCh, ds, stopper } - send := func(ctx context.Context, ds *DistSender, ba *roachpb.BatchRequest) func() batchResponse { + send := func(ctx context.Context, ds *DistSender, ba *kvpb.BatchRequest) func() batchResponse { var ( - br *roachpb.BatchResponse - pErr *roachpb.Error + br *kvpb.BatchResponse + pErr *kvpb.Error ) var wg sync.WaitGroup wg.Add(1) @@ -5821,24 +5822,24 @@ func TestOptimisticRangeDescriptorLookups(t *testing.T) { mkKey := func(i uint32) roachpb.Key { return keys.SystemSQLCodec.TablePrefix(i) } - mkGet := func(k roachpb.Key) *roachpb.BatchRequest { - ba := roachpb.BatchRequest{} - ba.Add(&roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: k}}) + mkGet := func(k roachpb.Key) *kvpb.BatchRequest { + ba := kvpb.BatchRequest{} + ba.Add(&kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{Key: k}}) return &ba } - expectSingleScan := func(t *testing.T, ba *roachpb.BatchRequest) *roachpb.ScanRequest { + expectSingleScan := func(t *testing.T, ba *kvpb.BatchRequest) *kvpb.ScanRequest { require.Len(t, ba.Requests, 1) - scanReq, ok := ba.GetArg(roachpb.Scan) + scanReq, ok := ba.GetArg(kvpb.Scan) require.True(t, ok) - scan := scanReq.(*roachpb.ScanRequest) + scan := scanReq.(*kvpb.ScanRequest) return scan } - expectSingleGet := func(t *testing.T, ba *roachpb.BatchRequest) *roachpb.GetRequest { + expectSingleGet := func(t *testing.T, ba *kvpb.BatchRequest) *kvpb.GetRequest { require.Len(t, ba.Requests, 1) - getReq, ok := ba.GetArg(roachpb.Get) + getReq, ok := ba.GetArg(kvpb.Get) require.True(t, ok) - get := getReq.(*roachpb.GetRequest) + get := getReq.(*kvpb.GetRequest) return get } @@ -5859,8 +5860,8 @@ func TestOptimisticRangeDescriptorLookups(t *testing.T) { })) return meta2RangeDesc, &next } - mkBatchResponseWithResponses := func(resps ...roachpb.Response) *roachpb.BatchResponse { - var br roachpb.BatchResponse + mkBatchResponseWithResponses := func(resps ...kvpb.Response) *kvpb.BatchResponse { + var br kvpb.BatchResponse for _, resp := range resps { br.Add(resp) } @@ -5868,26 +5869,26 @@ func TestOptimisticRangeDescriptorLookups(t *testing.T) { } mkBatchResponseWithRangeDescriptor := func( t *testing.T, k roachpb.Key, d *roachpb.RangeDescriptor, - ) *roachpb.BatchResponse { + ) *kvpb.BatchResponse { var retKV roachpb.KeyValue retKV.Key = k require.NoError(t, retKV.Value.SetProto(d)) return mkBatchResponseWithResponses( - &roachpb.ScanResponse{Rows: []roachpb.KeyValue{retKV}}, + &kvpb.ScanResponse{Rows: []roachpb.KeyValue{retKV}}, ) } checkBatch := func( - t *testing.T, ba *roachpb.BatchRequest, expDesc *roachpb.RangeDescriptor, - consistency roachpb.ReadConsistencyType) { + t *testing.T, ba *kvpb.BatchRequest, expDesc *roachpb.RangeDescriptor, + consistency kvpb.ReadConsistencyType) { require.Equal(t, expDesc.RangeID, ba.RangeID) require.Equal(t, expDesc.Replicas().Descriptors()[0], ba.Replica) require.Equal(t, consistency, ba.ReadConsistency) } checkScan := func( expDesc *roachpb.RangeDescriptor, - consistency roachpb.ReadConsistencyType, key roachpb.Key, - ) func(*testing.T, *roachpb.BatchRequest) { - return func(t *testing.T, ba *roachpb.BatchRequest) { + consistency kvpb.ReadConsistencyType, key roachpb.Key, + ) func(*testing.T, *kvpb.BatchRequest) { + return func(t *testing.T, ba *kvpb.BatchRequest) { checkBatch(t, ba, expDesc, consistency) scan := expectSingleScan(t, ba) require.Equal(t, key, scan.Key) @@ -5896,9 +5897,9 @@ func TestOptimisticRangeDescriptorLookups(t *testing.T) { checkGet := func( expDesc *roachpb.RangeDescriptor, - consistency roachpb.ReadConsistencyType, key roachpb.Key, - ) func(*testing.T, *roachpb.BatchRequest) { - return func(t *testing.T, ba *roachpb.BatchRequest) { + consistency kvpb.ReadConsistencyType, key roachpb.Key, + ) func(*testing.T, *kvpb.BatchRequest) { + return func(t *testing.T, ba *kvpb.BatchRequest) { get := expectSingleGet(t, ba) require.Equal(t, key, get.Key) checkBatch(t, ba, expDesc, consistency) @@ -5924,7 +5925,7 @@ func TestOptimisticRangeDescriptorLookups(t *testing.T) { }), ) for _, step := range []struct { - check func(*testing.T, *roachpb.BatchRequest) + check func(*testing.T, *kvpb.BatchRequest) next batchResponse }{ // The first request we expect is a scan to meta1 to find the meta2 for our @@ -5932,7 +5933,7 @@ func TestOptimisticRangeDescriptorLookups(t *testing.T) { // k.Next(), so when scanning meta1, we'll be at k.Next().Next(). { // 0 checkScan( - firstRange, roachpb.INCONSISTENT, + firstRange, kvpb.INCONSISTENT, meta1Key.Next().Next().AsRawKey(), ), batchResponse{br: mkBatchResponseWithRangeDescriptor( @@ -5945,15 +5946,15 @@ func TestOptimisticRangeDescriptorLookups(t *testing.T) { // the same generation number, then we should get a consistent scan. { // 1 checkScan( - meta2Initial, roachpb.INCONSISTENT, meta2Key.Next().AsRawKey(), + meta2Initial, kvpb.INCONSISTENT, meta2Key.Next().AsRawKey(), ), - batchResponse{err: roachpb.NewRangeNotFoundError(2, 3)}, + batchResponse{err: kvpb.NewRangeNotFoundError(2, 3)}, }, // Now we should get another scan to meta1 to look up the meta2 range again. // Send the same response as the first time around. { // 2 checkScan( - firstRange, roachpb.INCONSISTENT, + firstRange, kvpb.INCONSISTENT, meta1Key.Next().Next().AsRawKey(), ), batchResponse{br: mkBatchResponseWithRangeDescriptor( @@ -5964,7 +5965,7 @@ func TestOptimisticRangeDescriptorLookups(t *testing.T) { // scan. { // 3 checkScan( - firstRange, roachpb.READ_UNCOMMITTED, + firstRange, kvpb.READ_UNCOMMITTED, meta1Key.Next().Next().AsRawKey(), ), batchResponse{br: mkBatchResponseWithRangeDescriptor( @@ -5975,7 +5976,7 @@ func TestOptimisticRangeDescriptorLookups(t *testing.T) { // location. { // 4 checkScan( - meta2NextGen, roachpb.INCONSISTENT, + meta2NextGen, kvpb.INCONSISTENT, meta2Key.Next().AsRawKey(), ), batchResponse{ @@ -5986,9 +5987,9 @@ func TestOptimisticRangeDescriptorLookups(t *testing.T) { }, // Finally the request gets where it needs to go. { // 5 - checkGet(tableDataRange, roachpb.CONSISTENT, k), + checkGet(tableDataRange, kvpb.CONSISTENT, k), batchResponse{ - br: mkBatchResponseWithResponses(&roachpb.GetResponse{}), + br: mkBatchResponseWithResponses(&kvpb.GetResponse{}), }, }, } { diff --git a/pkg/kv/kvclient/kvcoord/integration_test.go b/pkg/kv/kvclient/kvcoord/integration_test.go index 0bc38cb1fe48..0587faf09ceb 100644 --- a/pkg/kv/kvclient/kvcoord/integration_test.go +++ b/pkg/kv/kvclient/kvcoord/integration_test.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/txnwait" @@ -71,15 +72,15 @@ func TestWaiterOnRejectedCommit(t *testing.T) { Store: &kvserver.StoreTestingKnobs{ DisableMergeQueue: true, DisableSplitQueue: true, - TestingProposalFilter: func(args kvserverbase.ProposalFilterArgs) *roachpb.Error { + TestingProposalFilter: func(args kvserverbase.ProposalFilterArgs) *kvpb.Error { // We'll recognize the attempt to commit our transaction and store the // respective command id. ba := args.Req - etReq, ok := ba.GetArg(roachpb.EndTxn) + etReq, ok := ba.GetArg(kvpb.EndTxn) if !ok { return nil } - if !etReq.(*roachpb.EndTxnRequest).Commit { + if !etReq.(*kvpb.EndTxnRequest).Commit { return nil } v := txnID.Load() @@ -92,7 +93,7 @@ func TestWaiterOnRejectedCommit(t *testing.T) { commitCmdID.Store(args.CmdID) return nil }, - TestingApplyCalledTwiceFilter: func(args kvserverbase.ApplyFilterArgs) (int, *roachpb.Error) { + TestingApplyCalledTwiceFilter: func(args kvserverbase.ApplyFilterArgs) (int, *kvpb.Error) { // We'll trap the processing of the commit command and return an error // for it. v := commitCmdID.Load() @@ -104,15 +105,15 @@ func TestWaiterOnRejectedCommit(t *testing.T) { if illegalLeaseIndex { illegalLeaseIndex = false // NB: 1 is proposalIllegalLeaseIndex. - return 1, roachpb.NewErrorf("test injected err (illegal lease index)") + return 1, kvpb.NewErrorf("test injected err (illegal lease index)") } // NB: 0 is proposalNoReevaluation. - return 0, roachpb.NewErrorf("test injected err") + return 0, kvpb.NewErrorf("test injected err") } return 0, nil }, TxnWaitKnobs: txnwait.TestingKnobs{ - OnPusherBlocked: func(ctx context.Context, push *roachpb.PushTxnRequest) { + OnPusherBlocked: func(ctx context.Context, push *kvpb.PushTxnRequest) { // We'll trap a reader entering the wait queue for our txn. v := txnID.Load() if v == nil { diff --git a/pkg/kv/kvclient/kvcoord/local_test_cluster_util.go b/pkg/kv/kvclient/kvcoord/local_test_cluster_util.go index 13414de95a51..975747c689ab 100644 --- a/pkg/kv/kvclient/kvcoord/local_test_cluster_util.go +++ b/pkg/kv/kvclient/kvcoord/local_test_cluster_util.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" @@ -36,8 +37,8 @@ type localTestClusterTransport struct { } func (l *localTestClusterTransport) SendNext( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, error) { if l.latency > 0 { time.Sleep(l.latency) } diff --git a/pkg/kv/kvclient/kvcoord/lock_spans_over_budget_error.go b/pkg/kv/kvclient/kvcoord/lock_spans_over_budget_error.go index eb9c4ecde885..76f20ea4445d 100644 --- a/pkg/kv/kvclient/kvcoord/lock_spans_over_budget_error.go +++ b/pkg/kv/kvclient/kvcoord/lock_spans_over_budget_error.go @@ -15,7 +15,7 @@ import ( "fmt" "strconv" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/errors" "github.com/cockroachdb/errors/errorspb" "github.com/gogo/protobuf/proto" @@ -31,7 +31,7 @@ type lockSpansOverBudgetError struct { } func newLockSpansOverBudgetError( - lockSpansBytes, limitBytes int64, ba *roachpb.BatchRequest, + lockSpansBytes, limitBytes int64, ba *kvpb.BatchRequest, ) lockSpansOverBudgetError { return lockSpansOverBudgetError{ lockSpansBytes: lockSpansBytes, diff --git a/pkg/kv/kvclient/kvcoord/mocks_generated_test.go b/pkg/kv/kvclient/kvcoord/mocks_generated_test.go index 8c52fd0bef14..564899f21e84 100644 --- a/pkg/kv/kvclient/kvcoord/mocks_generated_test.go +++ b/pkg/kv/kvclient/kvcoord/mocks_generated_test.go @@ -8,6 +8,7 @@ import ( context "context" reflect "reflect" + kvpb "github.com/cockroachdb/cockroach/pkg/kv/kvpb" roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" rpc "github.com/cockroachdb/cockroach/pkg/rpc" gomock "github.com/golang/mock/gomock" @@ -106,10 +107,10 @@ func (mr *MockTransportMockRecorder) Release() *gomock.Call { } // SendNext mocks base method. -func (m *MockTransport) SendNext(arg0 context.Context, arg1 *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { +func (m *MockTransport) SendNext(arg0 context.Context, arg1 *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "SendNext", arg0, arg1) - ret0, _ := ret[0].(*roachpb.BatchResponse) + ret0, _ := ret[0].(*kvpb.BatchResponse) ret1, _ := ret[1].(error) return ret0, ret1 } diff --git a/pkg/kv/kvclient/kvcoord/rangefeed_message.go b/pkg/kv/kvclient/kvcoord/rangefeed_message.go index 5d4f222194d8..3d1068e8f297 100644 --- a/pkg/kv/kvclient/kvcoord/rangefeed_message.go +++ b/pkg/kv/kvclient/kvcoord/rangefeed_message.go @@ -10,13 +10,16 @@ package kvcoord -import "github.com/cockroachdb/cockroach/pkg/roachpb" +import ( + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" + "github.com/cockroachdb/cockroach/pkg/roachpb" +) -// RangeFeedMessage is a type that encapsulates the roachpb.RangeFeedEvent. +// RangeFeedMessage is a type that encapsulates the kvpb.RangeFeedEvent. type RangeFeedMessage struct { // RangeFeed event this message holds. - *roachpb.RangeFeedEvent + *kvpb.RangeFeedEvent // The span of the rangefeed registration that overlaps with the key or span // in the RangeFeed event. diff --git a/pkg/kv/kvclient/kvcoord/replayed_commit_test.go b/pkg/kv/kvclient/kvcoord/replayed_commit_test.go index 08ae5704739c..4cac1f9a116e 100644 --- a/pkg/kv/kvclient/kvcoord/replayed_commit_test.go +++ b/pkg/kv/kvclient/kvcoord/replayed_commit_test.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -34,12 +35,12 @@ import ( type interceptingTransport struct { kvcoord.Transport - intercept func(context.Context, *roachpb.BatchRequest, *roachpb.BatchResponse, error) (*roachpb.BatchResponse, error) + intercept func(context.Context, *kvpb.BatchRequest, *kvpb.BatchResponse, error) (*kvpb.BatchResponse, error) } func (f *interceptingTransport) SendNext( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, error) { br, err := f.Transport.SendNext(ctx, ba) return f.intercept(ctx, ba, br, err) } @@ -75,7 +76,7 @@ func TestCommitSanityCheckAssertionFiresOnUndetectedAmbiguousCommit(t *testing.T } return &interceptingTransport{ Transport: tf, - intercept: func(ctx context.Context, ba *roachpb.BatchRequest, br *roachpb.BatchResponse, err error) (*roachpb.BatchResponse, error) { + intercept: func(ctx context.Context, ba *kvpb.BatchRequest, br *kvpb.BatchResponse, err error) (*kvpb.BatchResponse, error) { if err != nil || ba.Txn == nil || br.Txn == nil || ba.Txn.Status != roachpb.PENDING || br.Txn.Status != roachpb.COMMITTED || !keys.ScratchRangeMin.Equal(br.Txn.Key) { diff --git a/pkg/kv/kvclient/kvcoord/send_test.go b/pkg/kv/kvclient/kvcoord/send_test.go index ee6a9857d4d2..4d32eba87e7c 100644 --- a/pkg/kv/kvclient/kvcoord/send_test.go +++ b/pkg/kv/kvclient/kvcoord/send_test.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangecache" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" @@ -34,52 +35,50 @@ import ( "github.com/stretchr/testify/require" ) -var _ roachpb.InternalServer = Node(0) +var _ kvpb.InternalServer = Node(0) type Node time.Duration -func (n Node) Batch( - ctx context.Context, args *roachpb.BatchRequest, -) (*roachpb.BatchResponse, error) { +func (n Node) Batch(ctx context.Context, args *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { if n > 0 { time.Sleep(time.Duration(n)) } - return &roachpb.BatchResponse{}, nil + return &kvpb.BatchResponse{}, nil } func (n Node) RangeLookup( - _ context.Context, _ *roachpb.RangeLookupRequest, -) (*roachpb.RangeLookupResponse, error) { + _ context.Context, _ *kvpb.RangeLookupRequest, +) (*kvpb.RangeLookupResponse, error) { panic("unimplemented") } -func (n Node) RangeFeed(_ *roachpb.RangeFeedRequest, _ roachpb.Internal_RangeFeedServer) error { +func (n Node) RangeFeed(_ *kvpb.RangeFeedRequest, _ kvpb.Internal_RangeFeedServer) error { panic("unimplemented") } -func (n Node) MuxRangeFeed(server roachpb.Internal_MuxRangeFeedServer) error { +func (n Node) MuxRangeFeed(server kvpb.Internal_MuxRangeFeedServer) error { panic("unimplemented") } func (n Node) GossipSubscription( - _ *roachpb.GossipSubscriptionRequest, _ roachpb.Internal_GossipSubscriptionServer, + _ *kvpb.GossipSubscriptionRequest, _ kvpb.Internal_GossipSubscriptionServer, ) error { panic("unimplemented") } -func (n Node) Join(context.Context, *roachpb.JoinNodeRequest) (*roachpb.JoinNodeResponse, error) { +func (n Node) Join(context.Context, *kvpb.JoinNodeRequest) (*kvpb.JoinNodeResponse, error) { panic("unimplemented") } func (n Node) ResetQuorum( - context.Context, *roachpb.ResetQuorumRequest, -) (*roachpb.ResetQuorumResponse, error) { + context.Context, *kvpb.ResetQuorumRequest, +) (*kvpb.ResetQuorumResponse, error) { panic("unimplemented") } func (n Node) TokenBucket( - ctx context.Context, in *roachpb.TokenBucketRequest, -) (*roachpb.TokenBucketResponse, error) { + ctx context.Context, in *kvpb.TokenBucketRequest, +) (*kvpb.TokenBucketResponse, error) { panic("unimplemented") } @@ -108,13 +107,13 @@ func (n Node) SpanConfigConformance( } func (n Node) TenantSettings( - *roachpb.TenantSettingsRequest, roachpb.Internal_TenantSettingsServer, + *kvpb.TenantSettingsRequest, kvpb.Internal_TenantSettingsServer, ) error { panic("unimplemented") } func (n Node) GetRangeDescriptors( - *roachpb.GetRangeDescriptorsRequest, roachpb.Internal_GetRangeDescriptorsServer, + *kvpb.GetRangeDescriptorsRequest, kvpb.Internal_GetRangeDescriptorsServer, ) error { panic("unimplemented") } @@ -138,7 +137,7 @@ func TestSendToOneClient(t *testing.T) { s, err := rpc.NewServer(rpcContext) require.NoError(t, err) - roachpb.RegisterInternalServer(s, Node(0)) + kvpb.RegisterInternalServer(s, Node(0)) ln, err := netutil.ListenAndServeGRPC(rpcContext.Stopper, s, util.TestAddr) require.NoError(t, err) nodeDialer := nodedialer.New(rpcContext, func(roachpb.NodeID) (net.Addr, error) { @@ -167,14 +166,14 @@ func (f *firstNErrorTransport) IsExhausted() bool { func (f *firstNErrorTransport) Release() {} func (f *firstNErrorTransport) SendNext( - _ context.Context, _ *roachpb.BatchRequest, -) (*roachpb.BatchResponse, error) { + _ context.Context, _ *kvpb.BatchRequest, +) (*kvpb.BatchResponse, error) { var err error if f.numSent < f.numErrors { err = errors.New("firstNErrorTransport injected error") } f.numSent++ - return &roachpb.BatchResponse{}, err + return &kvpb.BatchResponse{}, err } func (f *firstNErrorTransport) NextInternalClient( @@ -345,7 +344,7 @@ func sendBatch( addrs []net.Addr, rpcContext *rpc.Context, nodeDialer *nodedialer.Dialer, -) (*roachpb.BatchResponse, error) { +) (*kvpb.BatchResponse, error) { stopper := stop.NewStopper() defer stopper.Stop(ctx) g := makeGossip(t, stopper, rpcContext) @@ -387,5 +386,5 @@ func sendBatch( routing, err := ds.getRoutingInfo(ctx, desc.StartKey, rangecache.EvictionToken{}, false /* useReverseScan */) require.NoError(t, err) - return ds.sendToReplicas(ctx, &roachpb.BatchRequest{}, routing, false /* withCommit */) + return ds.sendToReplicas(ctx, &kvpb.BatchRequest{}, routing, false /* withCommit */) } diff --git a/pkg/kv/kvclient/kvcoord/testdata/savepoints b/pkg/kv/kvclient/kvcoord/testdata/savepoints index 861e7d5c28d2..b2b4fa48ff70 100644 --- a/pkg/kv/kvclient/kvcoord/testdata/savepoints +++ b/pkg/kv/kvclient/kvcoord/testdata/savepoints @@ -363,7 +363,7 @@ savepoint x cput k v bogus_expected ---- -(*roachpb.ConditionFailedError) unexpected value +(*kvpb.ConditionFailedError) unexpected value rollback x ---- @@ -396,7 +396,7 @@ savepoint x get conflict-key locking nowait ---- -(*roachpb.WriteIntentError) conflicting intents on "conflict-key" [reason=wait_policy] +(*kvpb.WriteIntentError) conflicting intents on "conflict-key" [reason=wait_policy] rollback x ---- @@ -404,7 +404,7 @@ rollback x put conflict-key b nowait ---- -(*roachpb.WriteIntentError) conflicting intents on "conflict-key" [reason=wait_policy] +(*kvpb.WriteIntentError) conflicting intents on "conflict-key" [reason=wait_policy] rollback x ---- @@ -437,7 +437,7 @@ savepoint x get conflict-key-2 lock-timeout ---- -(*roachpb.WriteIntentError) conflicting intents on "conflict-key-2" [reason=lock_timeout] +(*kvpb.WriteIntentError) conflicting intents on "conflict-key-2" [reason=lock_timeout] rollback x ---- @@ -445,7 +445,7 @@ rollback x put conflict-key-2 b lock-timeout ---- -(*roachpb.WriteIntentError) conflicting intents on "conflict-key-2" [reason=lock_timeout] +(*kvpb.WriteIntentError) conflicting intents on "conflict-key-2" [reason=lock_timeout] rollback x ---- @@ -490,7 +490,7 @@ savepoint x abort ---- -(*roachpb.TransactionRetryWithProtoRefreshError) +(*kvpb.TransactionRetryWithProtoRefreshError) txn id not changed reset @@ -529,6 +529,6 @@ epoch: 0 -> 1 rollback x ---- -(*roachpb.TransactionRetryWithProtoRefreshError) TransactionRetryWithProtoRefreshError: cannot rollback to savepoint after a transaction restart +(*kvpb.TransactionRetryWithProtoRefreshError) TransactionRetryWithProtoRefreshError: cannot rollback to savepoint after a transaction restart subtest end diff --git a/pkg/kv/kvclient/kvcoord/testing_knobs.go b/pkg/kv/kvclient/kvcoord/testing_knobs.go index 1da19d7a75ed..58bb2efafe1d 100644 --- a/pkg/kv/kvclient/kvcoord/testing_knobs.go +++ b/pkg/kv/kvclient/kvcoord/testing_knobs.go @@ -12,7 +12,7 @@ package kvcoord import ( "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" ) // ClientTestingKnobs contains testing options that dictate the behavior @@ -60,7 +60,7 @@ type ClientTestingKnobs struct { // a non-transactional batch across a range boundary. The method may inject an // error which, if non-nil, becomes the result of the batch. Otherwise, execution // continues. - OnRangeSpanningNonTxnalBatch func(ba *roachpb.BatchRequest) *roachpb.Error + OnRangeSpanningNonTxnalBatch func(ba *kvpb.BatchRequest) *kvpb.Error } var _ base.ModuleTestingKnobs = &ClientTestingKnobs{} diff --git a/pkg/kv/kvclient/kvcoord/transport.go b/pkg/kv/kvclient/kvcoord/transport.go index 641f0ef26dc7..8ed7b43d0e2d 100644 --- a/pkg/kv/kvclient/kvcoord/transport.go +++ b/pkg/kv/kvclient/kvcoord/transport.go @@ -17,6 +17,7 @@ import ( "sync" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" @@ -64,7 +65,7 @@ type Transport interface { // // SendNext is also in charge of importing the remotely collected spans (if // any) into the local trace. - SendNext(context.Context, *roachpb.BatchRequest) (*roachpb.BatchResponse, error) + SendNext(context.Context, *kvpb.BatchRequest) (*kvpb.BatchResponse, error) // NextInternalClient returns the InternalClient to use for making RPC // calls. @@ -178,8 +179,8 @@ func (gt *grpcTransport) IsExhausted() bool { // client is ready. On success, the reply is sent on the channel; // otherwise an error is sent. func (gt *grpcTransport) SendNext( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, error) { r := gt.replicas[gt.nextReplicaIdx] iface, err := gt.NextInternalClient(ctx) if err != nil { @@ -193,8 +194,8 @@ func (gt *grpcTransport) sendBatch( ctx context.Context, nodeID roachpb.NodeID, iface rpc.RestrictedInternalClient, - ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, error) { + ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, error) { // Bail out early if the context is already canceled. (GRPC will // detect this pretty quickly, but the first check of the context // in the local server comes pretty late) @@ -339,8 +340,8 @@ func (s *senderTransport) IsExhausted() bool { } func (s *senderTransport) SendNext( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, error) { if s.called { panic("called an exhausted transport") } @@ -351,10 +352,10 @@ func (s *senderTransport) SendNext( log.Eventf(ctx, "%v", ba.String()) br, pErr := s.sender.Send(ctx, ba) if br == nil { - br = &roachpb.BatchResponse{} + br = &kvpb.BatchResponse{} } if br.Error != nil { - panic(roachpb.ErrorUnexpectedlySet(s.sender, br)) + panic(kvpb.ErrorUnexpectedlySet(s.sender, br)) } br.Error = pErr if pErr != nil { diff --git a/pkg/kv/kvclient/kvcoord/transport_race.go b/pkg/kv/kvclient/kvcoord/transport_race.go index 86473b8bab72..eac19cef5ac5 100644 --- a/pkg/kv/kvclient/kvcoord/transport_race.go +++ b/pkg/kv/kvclient/kvcoord/transport_race.go @@ -22,17 +22,17 @@ import ( "sync/atomic" "time" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" ) var running int32 // atomically updated -var incoming chan *roachpb.BatchRequest +var incoming chan *kvpb.BatchRequest func init() { - incoming = make(chan *roachpb.BatchRequest, 100) + incoming = make(chan *kvpb.BatchRequest, 100) } const defaultRaceInterval = 150 * time.Microsecond @@ -54,18 +54,18 @@ type raceTransport struct { } func (tr raceTransport) SendNext( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, error) { // Make a copy of the requests slice, and shallow copies of the requests. // The caller is allowed to mutate the request after the call returns. Since // this transport has no way of checking who's doing mutations (the client - // which is allowed, or the server - which is not). So, for now, we exclude // the slice and the requests from any checks, since those are the parts that // the client currently mutates. - requestsCopy := make([]roachpb.RequestUnion, len(ba.Requests)) + requestsCopy := make([]kvpb.RequestUnion, len(ba.Requests)) for i, ru := range ba.Requests { // ru is a RequestUnion interface, so we need some hoops to dereference it. - requestsCopy[i] = reflect.Indirect(reflect.ValueOf(ru)).Interface().(roachpb.RequestUnion) + requestsCopy[i] = reflect.Indirect(reflect.ValueOf(ru)).Interface().(kvpb.RequestUnion) } ba.Requests = requestsCopy select { @@ -110,7 +110,7 @@ func GRPCTransportFactory( // Make a fixed-size slice of *BatchRequest. When full, entries // are evicted in FIFO order. const size = 1000 - bas := make([]*roachpb.BatchRequest, size) + bas := make([]*kvpb.BatchRequest, size) encoder := json.NewEncoder(io.Discard) for { iters++ diff --git a/pkg/kv/kvclient/kvcoord/transport_test.go b/pkg/kv/kvclient/kvcoord/transport_test.go index 7e29c58ffd34..7c57363b75ae 100644 --- a/pkg/kv/kvclient/kvcoord/transport_test.go +++ b/pkg/kv/kvclient/kvcoord/transport_test.go @@ -15,6 +15,7 @@ import ( "fmt" "testing" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/caller" @@ -120,7 +121,7 @@ func TestSpanImport(t *testing.T) { server := mockInternalClient{} // Let's spice things up and simulate an error from the server. expectedErr := "my expected error" - server.pErr = roachpb.NewErrorf(expectedErr /* nolint:fmtsafe */) + server.pErr = kvpb.NewErrorf(expectedErr /* nolint:fmtsafe */) recCtx, getRecAndFinish := tracing.ContextWithRecordingSpan( ctx, tracing.NewTracer(), "test") @@ -128,7 +129,7 @@ func TestSpanImport(t *testing.T) { server.tr = tracing.SpanFromContext(recCtx).Tracer() - br, err := gt.sendBatch(recCtx, roachpb.NodeID(1), &server, &roachpb.BatchRequest{}) + br, err := gt.sendBatch(recCtx, roachpb.NodeID(1), &server, &kvpb.BatchRequest{}) if err != nil { t.Fatal(err) } @@ -152,11 +153,11 @@ func TestResponseVerifyFailure(t *testing.T) { }, } - ba := &roachpb.BatchRequest{} - req := roachpb.NewScan(roachpb.KeyMin, roachpb.KeyMax, false /* forUpdate */) + ba := &kvpb.BatchRequest{} + req := kvpb.NewScan(roachpb.KeyMin, roachpb.KeyMax, false /* forUpdate */) ba.Add(req) br := ba.CreateReply() - resp := br.Responses[0].GetInner().(*roachpb.ScanResponse) + resp := br.Responses[0].GetInner().(*kvpb.ScanResponse) val := roachpb.MakeValueFromString("hi") val.InitChecksum(roachpb.Key("not the right key")) resp.Rows = append(resp.Rows, roachpb.KeyValue{ @@ -173,26 +174,26 @@ func TestResponseVerifyFailure(t *testing.T) { require.ErrorContains(t, err, "invalid checksum") } -// mockInternalClient is an implementation of roachpb.InternalClient. +// mockInternalClient is an implementation of kvpb.InternalClient. // It simulates aspects of how the Node normally handles tracing in gRPC calls. type mockInternalClient struct { tr *tracing.Tracer - br *roachpb.BatchResponse - pErr *roachpb.Error + br *kvpb.BatchResponse + pErr *kvpb.Error } -var _ roachpb.InternalClient = &mockInternalClient{} +var _ kvpb.InternalClient = &mockInternalClient{} func (*mockInternalClient) ResetQuorum( - context.Context, *roachpb.ResetQuorumRequest, ...grpc.CallOption, -) (*roachpb.ResetQuorumResponse, error) { + context.Context, *kvpb.ResetQuorumRequest, ...grpc.CallOption, +) (*kvpb.ResetQuorumResponse, error) { panic("unimplemented") } -// Batch is part of the roachpb.InternalClient interface. +// Batch is part of the kvpb.InternalClient interface. func (m *mockInternalClient) Batch( - ctx context.Context, in *roachpb.BatchRequest, opts ...grpc.CallOption, -) (*roachpb.BatchResponse, error) { + ctx context.Context, in *kvpb.BatchRequest, opts ...grpc.CallOption, +) (*kvpb.BatchResponse, error) { var sp *tracing.Span if m.tr != nil { sp = m.tr.StartSpan("mock", tracing.WithRecording(tracingpb.RecordingVerbose)) @@ -203,7 +204,7 @@ func (m *mockInternalClient) Batch( log.Eventf(ctx, "mockInternalClient processing batch") br := m.br if br == nil { - br = &roachpb.BatchResponse{} + br = &kvpb.BatchResponse{} } br.Error = m.pErr @@ -215,42 +216,42 @@ func (m *mockInternalClient) Batch( return br, nil } -// RangeLookup implements the roachpb.InternalClient interface. +// RangeLookup implements the kvpb.InternalClient interface. func (m *mockInternalClient) RangeLookup( - ctx context.Context, rl *roachpb.RangeLookupRequest, _ ...grpc.CallOption, -) (*roachpb.RangeLookupResponse, error) { + ctx context.Context, rl *kvpb.RangeLookupRequest, _ ...grpc.CallOption, +) (*kvpb.RangeLookupResponse, error) { return nil, fmt.Errorf("unsupported RangeLookup call") } -// RangeFeed is part of the roachpb.InternalClient interface. +// RangeFeed is part of the kvpb.InternalClient interface. func (m *mockInternalClient) RangeFeed( - ctx context.Context, in *roachpb.RangeFeedRequest, opts ...grpc.CallOption, -) (roachpb.Internal_RangeFeedClient, error) { + ctx context.Context, in *kvpb.RangeFeedRequest, opts ...grpc.CallOption, +) (kvpb.Internal_RangeFeedClient, error) { return nil, fmt.Errorf("unsupported RangeFeed call") } func (m *mockInternalClient) MuxRangeFeed( ctx context.Context, opts ...grpc.CallOption, -) (roachpb.Internal_MuxRangeFeedClient, error) { +) (kvpb.Internal_MuxRangeFeedClient, error) { return nil, fmt.Errorf("unsupported MuxRangeFeed call") } -// GossipSubscription is part of the roachpb.InternalClient interface. +// GossipSubscription is part of the kvpb.InternalClient interface. func (m *mockInternalClient) GossipSubscription( - ctx context.Context, args *roachpb.GossipSubscriptionRequest, _ ...grpc.CallOption, -) (roachpb.Internal_GossipSubscriptionClient, error) { + ctx context.Context, args *kvpb.GossipSubscriptionRequest, _ ...grpc.CallOption, +) (kvpb.Internal_GossipSubscriptionClient, error) { return nil, fmt.Errorf("unsupported GossipSubscripion call") } func (m *mockInternalClient) Join( - context.Context, *roachpb.JoinNodeRequest, ...grpc.CallOption, -) (*roachpb.JoinNodeResponse, error) { + context.Context, *kvpb.JoinNodeRequest, ...grpc.CallOption, +) (*kvpb.JoinNodeResponse, error) { return nil, fmt.Errorf("unsupported Join call") } func (m *mockInternalClient) TokenBucket( - ctx context.Context, in *roachpb.TokenBucketRequest, _ ...grpc.CallOption, -) (*roachpb.TokenBucketResponse, error) { + ctx context.Context, in *kvpb.TokenBucketRequest, _ ...grpc.CallOption, +) (*kvpb.TokenBucketResponse, error) { return nil, fmt.Errorf("unsupported TokenBucket call") } @@ -279,13 +280,13 @@ func (m *mockInternalClient) UpdateSpanConfigs( } func (m *mockInternalClient) TenantSettings( - context.Context, *roachpb.TenantSettingsRequest, ...grpc.CallOption, -) (roachpb.Internal_TenantSettingsClient, error) { + context.Context, *kvpb.TenantSettingsRequest, ...grpc.CallOption, +) (kvpb.Internal_TenantSettingsClient, error) { return nil, fmt.Errorf("unsupported TenantSettings call") } func (n *mockInternalClient) GetRangeDescriptors( - context.Context, *roachpb.GetRangeDescriptorsRequest, ...grpc.CallOption, -) (roachpb.Internal_GetRangeDescriptorsClient, error) { + context.Context, *kvpb.GetRangeDescriptorsRequest, ...grpc.CallOption, +) (kvpb.Internal_GetRangeDescriptorsClient, error) { return nil, fmt.Errorf("unsupported GetRangeDescriptors call") } diff --git a/pkg/kv/kvclient/kvcoord/txn_coord_sender.go b/pkg/kv/kvclient/kvcoord/txn_coord_sender.go index 1ed92b8ef089..834c0f91574e 100644 --- a/pkg/kv/kvclient/kvcoord/txn_coord_sender.go +++ b/pkg/kv/kvclient/kvcoord/txn_coord_sender.go @@ -15,6 +15,7 @@ import ( "runtime/debug" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/envutil" @@ -113,11 +114,11 @@ type TxnCoordSender struct { // storedRetryableErr is set when txnState == txnRetryableError. This // storedRetryableErr is returned to clients on Send(). - storedRetryableErr *roachpb.TransactionRetryWithProtoRefreshError + storedRetryableErr *kvpb.TransactionRetryWithProtoRefreshError // storedErr is set when txnState == txnError. This storedErr is returned to // clients on Send(). - storedErr *roachpb.Error + storedErr *kvpb.Error // active is set whenever the transaction has sent any requests. Rolling // back to a savepoint taken before the TxnCoordSender became active resets @@ -420,15 +421,13 @@ func (tc *TxnCoordSender) DisablePipelining() error { return nil } -func generateTxnDeadlineExceededErr( - txn *roachpb.Transaction, deadline hlc.Timestamp, -) *roachpb.Error { +func generateTxnDeadlineExceededErr(txn *roachpb.Transaction, deadline hlc.Timestamp) *kvpb.Error { exceededBy := txn.WriteTimestamp.GoTime().Sub(deadline.GoTime()) extraMsg := redact.Sprintf( "txn timestamp pushed too much; deadline exceeded by %s (%s > %s)", exceededBy, txn.WriteTimestamp, deadline) - return roachpb.NewErrorWithTxn( - roachpb.NewTransactionRetryError(roachpb.RETRY_COMMIT_DEADLINE_EXCEEDED, extraMsg), txn) + return kvpb.NewErrorWithTxn( + kvpb.NewTransactionRetryError(kvpb.RETRY_COMMIT_DEADLINE_EXCEEDED, extraMsg), txn) } // finalizeNonLockingTxnLocked finalizes a non-locking txn, either marking it as @@ -442,8 +441,8 @@ func generateTxnDeadlineExceededErr( // sendLockedWithElidedEndTxn method, but we would want to confirm // that doing so doesn't cut into the speed-up we see from this fast-path. func (tc *TxnCoordSender) finalizeNonLockingTxnLocked( - ctx context.Context, ba *roachpb.BatchRequest, -) *roachpb.Error { + ctx context.Context, ba *kvpb.BatchRequest, +) *kvpb.Error { et := ba.Requests[0].GetEndTxn() if et.Commit { deadline := et.Deadline @@ -464,7 +463,7 @@ func (tc *TxnCoordSender) finalizeNonLockingTxnLocked( tc.finalizeAndCleanupTxnLocked(ctx) if et.Commit { if err := tc.maybeCommitWait(ctx, false /* deferred */); err != nil { - return roachpb.NewError(err) + return kvpb.NewError(err) } } return nil @@ -472,8 +471,8 @@ func (tc *TxnCoordSender) finalizeNonLockingTxnLocked( // Send is part of the client.TxnSender interface. func (tc *TxnCoordSender) Send( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, *kvpb.Error) { // NOTE: The locking here is unusual. Although it might look like it, we are // NOT holding the lock continuously for the duration of the Send. We lock // here, and unlock at the botton of the interceptor stack, in the @@ -508,8 +507,8 @@ func (tc *TxnCoordSender) Send( // It doesn't make sense to use inconsistent reads in a transaction. However, // we still need to accept it as a parameter for this to compile. - if ba.ReadConsistency != roachpb.CONSISTENT { - return nil, roachpb.NewErrorf("cannot use %s ReadConsistency in txn", + if ba.ReadConsistency != kvpb.CONSISTENT { + return nil, kvpb.NewErrorf("cannot use %s ReadConsistency in txn", ba.ReadConsistency) } @@ -529,13 +528,13 @@ func (tc *TxnCoordSender) Send( // If we succeeded to commit, or we attempted to rollback, we move to // txnFinalized. - if req, ok := ba.GetArg(roachpb.EndTxn); ok { - et := req.(*roachpb.EndTxnRequest) + if req, ok := ba.GetArg(kvpb.EndTxn); ok { + et := req.(*kvpb.EndTxnRequest) if (et.Commit && pErr == nil) || !et.Commit { tc.finalizeAndCleanupTxnLocked(ctx) if et.Commit { if err := tc.maybeCommitWait(ctx, false /* deferred */); err != nil { - return nil, roachpb.NewError(err) + return nil, kvpb.NewError(err) } } } @@ -546,7 +545,7 @@ func (tc *TxnCoordSender) Send( } if br != nil && br.Error != nil { - panic(roachpb.ErrorUnexpectedlySet(nil /* culprit */, br)) + panic(kvpb.ErrorUnexpectedlySet(nil /* culprit */, br)) } return br, nil @@ -675,8 +674,8 @@ func (tc *TxnCoordSender) maybeCommitWait(ctx context.Context, deferred bool) er // ba is the batch that the client is trying to send. It's inspected because // rollbacks are always allowed. Can be nil. func (tc *TxnCoordSender) maybeRejectClientLocked( - ctx context.Context, ba *roachpb.BatchRequest, -) *roachpb.Error { + ctx context.Context, ba *kvpb.BatchRequest, +) *kvpb.Error { if ba != nil && ba.IsSingleAbortTxnRequest() && tc.mu.txn.Status != roachpb.COMMITTED { // As a special case, we allow rollbacks to be sent at any time. Any // rollback attempt moves the TxnCoordSender state to txnFinalized, but higher @@ -695,7 +694,7 @@ func (tc *TxnCoordSender) maybeRejectClientLocked( case txnPending: // All good. case txnRetryableError: - return roachpb.NewError(tc.mu.storedRetryableErr) + return kvpb.NewError(tc.mu.storedRetryableErr) case txnError: return tc.mu.storedErr case txnFinalized: @@ -703,11 +702,11 @@ func (tc *TxnCoordSender) maybeRejectClientLocked( "Trying to execute: %s", ba.Summary()) stack := string(debug.Stack()) log.Errorf(ctx, "%s. stack:\n%s", msg, stack) - reason := roachpb.TransactionStatusError_REASON_UNKNOWN + reason := kvpb.TransactionStatusError_REASON_UNKNOWN if tc.mu.txn.Status == roachpb.COMMITTED { - reason = roachpb.TransactionStatusError_REASON_TXN_COMMITTED + reason = kvpb.TransactionStatusError_REASON_TXN_COMMITTED } - return roachpb.NewErrorWithTxn(roachpb.NewTransactionStatusError(reason, msg), &tc.mu.txn) + return kvpb.NewErrorWithTxn(kvpb.NewTransactionStatusError(reason, msg), &tc.mu.txn) } // Check the transaction proto state, along with any finalized transaction @@ -723,12 +722,12 @@ func (tc *TxnCoordSender) maybeRejectClientLocked( // this was not due to a synchronous transaction commit and transaction // record garbage collection. // See the comment on txnHeartbeater.mu.finalObservedStatus for more details. - abortedErr := roachpb.NewErrorWithTxn( - roachpb.NewTransactionAbortedError(roachpb.ABORT_REASON_CLIENT_REJECT), &tc.mu.txn) - return roachpb.NewError(tc.handleRetryableErrLocked(ctx, abortedErr)) + abortedErr := kvpb.NewErrorWithTxn( + kvpb.NewTransactionAbortedError(kvpb.ABORT_REASON_CLIENT_REJECT), &tc.mu.txn) + return kvpb.NewError(tc.handleRetryableErrLocked(ctx, abortedErr)) case protoStatus != roachpb.PENDING || hbObservedStatus != roachpb.PENDING: // The transaction proto is in an unexpected state. - return roachpb.NewErrorf( + return kvpb.NewErrorf( "unexpected txn state: %s; heartbeat observed status: %s", tc.mu.txn, hbObservedStatus) default: // All good. @@ -757,11 +756,11 @@ func (tc *TxnCoordSender) cleanupTxnLocked(ctx context.Context) { // UpdateStateOnRemoteRetryableErr is part of the TxnSender interface. func (tc *TxnCoordSender) UpdateStateOnRemoteRetryableErr( - ctx context.Context, pErr *roachpb.Error, -) *roachpb.Error { + ctx context.Context, pErr *kvpb.Error, +) *kvpb.Error { tc.mu.Lock() defer tc.mu.Unlock() - return roachpb.NewError(tc.handleRetryableErrLocked(ctx, pErr)) + return kvpb.NewError(tc.handleRetryableErrLocked(ctx, pErr)) } // handleRetryableErrLocked takes a retriable error and creates a @@ -772,47 +771,47 @@ func (tc *TxnCoordSender) UpdateStateOnRemoteRetryableErr( // expected to check the ID of the resulting transaction. If the TxnCoordSender // can still be used, it will have been prepared for a new epoch. func (tc *TxnCoordSender) handleRetryableErrLocked( - ctx context.Context, pErr *roachpb.Error, -) *roachpb.TransactionRetryWithProtoRefreshError { + ctx context.Context, pErr *kvpb.Error, +) *kvpb.TransactionRetryWithProtoRefreshError { // If the error is a transaction retry error, update metrics to // reflect the reason for the restart. More details about the // different error types are documented above on the metaRestart // variables. switch tErr := pErr.GetDetail().(type) { - case *roachpb.TransactionRetryError: + case *kvpb.TransactionRetryError: switch tErr.Reason { - case roachpb.RETRY_WRITE_TOO_OLD: + case kvpb.RETRY_WRITE_TOO_OLD: tc.metrics.RestartsWriteTooOld.Inc() - case roachpb.RETRY_SERIALIZABLE: + case kvpb.RETRY_SERIALIZABLE: tc.metrics.RestartsSerializable.Inc() - case roachpb.RETRY_ASYNC_WRITE_FAILURE: + case kvpb.RETRY_ASYNC_WRITE_FAILURE: tc.metrics.RestartsAsyncWriteFailure.Inc() - case roachpb.RETRY_COMMIT_DEADLINE_EXCEEDED: + case kvpb.RETRY_COMMIT_DEADLINE_EXCEEDED: tc.metrics.RestartsCommitDeadlineExceeded.Inc() default: tc.metrics.RestartsUnknown.Inc() } - case *roachpb.WriteTooOldError: + case *kvpb.WriteTooOldError: tc.metrics.RestartsWriteTooOldMulti.Inc() - case *roachpb.ReadWithinUncertaintyIntervalError: + case *kvpb.ReadWithinUncertaintyIntervalError: tc.metrics.RestartsReadWithinUncertainty.Inc() - case *roachpb.TransactionAbortedError: + case *kvpb.TransactionAbortedError: tc.metrics.RestartsTxnAborted.Inc() - case *roachpb.TransactionPushError: + case *kvpb.TransactionPushError: tc.metrics.RestartsTxnPush.Inc() default: tc.metrics.RestartsUnknown.Inc() } errTxnID := pErr.GetTxn().ID - newTxn := roachpb.PrepareTransactionForRetry(ctx, pErr, tc.mu.userPriority, tc.clock) + newTxn := kvpb.PrepareTransactionForRetry(ctx, pErr, tc.mu.userPriority, tc.clock) // We'll pass a TransactionRetryWithProtoRefreshError up to the next layer. - retErr := roachpb.NewTransactionRetryWithProtoRefreshError( + retErr := kvpb.NewTransactionRetryWithProtoRefreshError( redact.Sprint(pErr), errTxnID, // the id of the transaction that encountered the error newTxn) @@ -852,8 +851,8 @@ func (tc *TxnCoordSender) handleRetryableErrLocked( // cases. It also updates retryable errors with the updated transaction for use // by client restarts. func (tc *TxnCoordSender) updateStateLocked( - ctx context.Context, ba *roachpb.BatchRequest, br *roachpb.BatchResponse, pErr *roachpb.Error, -) *roachpb.Error { + ctx context.Context, ba *kvpb.BatchRequest, br *kvpb.BatchResponse, pErr *kvpb.Error, +) *kvpb.Error { // We handle a couple of different cases: // 1) A successful response. If that response carries a transaction proto, @@ -878,7 +877,7 @@ func (tc *TxnCoordSender) updateStateLocked( return nil } - if pErr.TransactionRestart() != roachpb.TransactionRestart_NONE { + if pErr.TransactionRestart() != kvpb.TransactionRestart_NONE { if tc.typ == kv.LeafTxn { // Leaves handle retriable errors differently than roots. The leaf // transaction is not supposed to be used any more after a retriable @@ -904,7 +903,7 @@ func (tc *TxnCoordSender) updateStateLocked( log.Fatalf(ctx, "retryable error for the wrong txn. ba.Txn: %s. pErr: %s", ba.Txn, pErr) } - return roachpb.NewError(tc.handleRetryableErrLocked(ctx, pErr)) + return kvpb.NewError(tc.handleRetryableErrLocked(ctx, pErr)) } // This is the non-retriable error case. @@ -913,9 +912,9 @@ func (tc *TxnCoordSender) updateStateLocked( // rollback), but some errors are safe to allow continuing (in particular // ConditionFailedError). In particular, SQL can recover by rolling back to a // savepoint. - if roachpb.ErrPriority(pErr.GoError()) != roachpb.ErrorScoreUnambiguousError { + if kvpb.ErrPriority(pErr.GoError()) != kvpb.ErrorScoreUnambiguousError { tc.mu.txnState = txnError - tc.mu.storedErr = roachpb.NewError(&roachpb.TxnAlreadyEncounteredErrorError{ + tc.mu.storedErr = kvpb.NewError(&kvpb.TxnAlreadyEncounteredErrorError{ PrevError: pErr.String(), }) } @@ -923,7 +922,7 @@ func (tc *TxnCoordSender) updateStateLocked( // Update our transaction with any information the error has. if errTxn := pErr.GetTxn(); errTxn != nil { if err := sanityCheckErrWithTxn(ctx, pErr, ba, &tc.testingKnobs); err != nil { - return roachpb.NewError(err) + return kvpb.NewError(err) } tc.mu.txn.Update(errTxn) } @@ -941,10 +940,7 @@ func (tc *TxnCoordSender) updateStateLocked( // The assertion is known to have failed in the wild, see: // https://github.com/cockroachdb/cockroach/issues/67765 func sanityCheckErrWithTxn( - ctx context.Context, - pErrWithTxn *roachpb.Error, - ba *roachpb.BatchRequest, - knobs *ClientTestingKnobs, + ctx context.Context, pErrWithTxn *kvpb.Error, ba *kvpb.BatchRequest, knobs *ClientTestingKnobs, ) error { txn := pErrWithTxn.GetTxn() if txn.Status != roachpb.COMMITTED { @@ -1278,7 +1274,7 @@ func (tc *TxnCoordSender) PrepareRetryableError( if tc.mu.txnState != txnPending { return errors.AssertionFailedf("cannot set a retryable error. current state: %s", tc.mu.txnState) } - pErr := roachpb.NewTransactionRetryWithProtoRefreshError( + pErr := kvpb.NewTransactionRetryWithProtoRefreshError( msg, tc.mu.txn.ID, tc.mu.txn) tc.mu.storedRetryableErr = pErr tc.mu.txnState = txnRetryableError @@ -1333,14 +1329,14 @@ func (tc *TxnCoordSender) ManualRefresh(ctx context.Context) error { // needs the transaction proto. The function then returns a BatchRequest // with the updated transaction proto. We use this updated proto to call // into updateStateLocked directly. - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Txn = tc.mu.txn.Clone() const force = true refreshedBa, pErr := tc.interceptorAlloc.txnSpanRefresher.maybeRefreshPreemptivelyLocked(ctx, ba, force) if pErr != nil { pErr = tc.updateStateLocked(ctx, ba, nil, pErr) } else { - var br roachpb.BatchResponse + var br kvpb.BatchResponse br.Txn = refreshedBa.Txn pErr = tc.updateStateLocked(ctx, ba, &br, nil) } @@ -1366,7 +1362,7 @@ func (tc *TxnCoordSender) DeferCommitWait(ctx context.Context) func(context.Cont // GetTxnRetryableErr is part of the TxnSender interface. func (tc *TxnCoordSender) GetTxnRetryableErr( ctx context.Context, -) *roachpb.TransactionRetryWithProtoRefreshError { +) *kvpb.TransactionRetryWithProtoRefreshError { tc.mu.Lock() defer tc.mu.Unlock() if tc.mu.txnState == txnRetryableError { diff --git a/pkg/kv/kvclient/kvcoord/txn_coord_sender_savepoints.go b/pkg/kv/kvclient/kvcoord/txn_coord_sender_savepoints.go index 8c54c5a9b7d8..fabd6fe14157 100644 --- a/pkg/kv/kvclient/kvcoord/txn_coord_sender_savepoints.go +++ b/pkg/kv/kvclient/kvcoord/txn_coord_sender_savepoints.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" @@ -123,7 +124,7 @@ func (tc *TxnCoordSender) RollbackToSavepoint(ctx context.Context, s kv.Savepoin err := tc.checkSavepointLocked(sp) if err != nil { if errors.Is(err, errSavepointInvalidAfterTxnRestart) { - err = roachpb.NewTransactionRetryWithProtoRefreshError( + err = kvpb.NewTransactionRetryWithProtoRefreshError( "cannot rollback to savepoint after a transaction restart", tc.mu.txn.ID, tc.mu.txn, @@ -166,7 +167,7 @@ func (tc *TxnCoordSender) ReleaseSavepoint(ctx context.Context, s kv.SavepointTo sp := s.(*savepoint) err := tc.checkSavepointLocked(sp) if errors.Is(err, errSavepointInvalidAfterTxnRestart) { - err = roachpb.NewTransactionRetryWithProtoRefreshError( + err = kvpb.NewTransactionRetryWithProtoRefreshError( "cannot release savepoint after a transaction restart", tc.mu.txn.ID, tc.mu.txn, diff --git a/pkg/kv/kvclient/kvcoord/txn_coord_sender_savepoints_test.go b/pkg/kv/kvclient/kvcoord/txn_coord_sender_savepoints_test.go index 94d2f4ec564f..76899f19776d 100644 --- a/pkg/kv/kvclient/kvcoord/txn_coord_sender_savepoints_test.go +++ b/pkg/kv/kvclient/kvcoord/txn_coord_sender_savepoints_test.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" @@ -51,14 +52,14 @@ func TestSavepoints(t *testing.T) { var doAbort int64 params.Knobs.Store = &kvserver.StoreTestingKnobs{ EvalKnobs: kvserverbase.BatchEvalTestingKnobs{ - TestingEvalFilter: func(args kvserverbase.FilterArgs) *roachpb.Error { + TestingEvalFilter: func(args kvserverbase.FilterArgs) *kvpb.Error { key := args.Req.Header().Key if atomic.LoadInt64(&doAbort) != 0 && key.Equal(abortKey) { - return roachpb.NewErrorWithTxn( - roachpb.NewTransactionAbortedError(roachpb.ABORT_REASON_UNKNOWN), args.Hdr.Txn) + return kvpb.NewErrorWithTxn( + kvpb.NewTransactionAbortedError(kvpb.ABORT_REASON_UNKNOWN), args.Hdr.Txn) } if key.Equal(errKey) { - return roachpb.NewErrorf("injected error") + return kvpb.NewErrorf("injected error") } return nil }, @@ -160,7 +161,7 @@ func TestSavepoints(t *testing.T) { []byte(td.CmdArgs[1].Key), expVal, ); err != nil { - if errors.HasType(err, (*roachpb.ConditionFailedError)(nil)) { + if errors.HasType(err, (*kvpb.ConditionFailedError)(nil)) { // Print an easier to match message. fmt.Fprintf(&buf, "(%T) unexpected value\n", err) } else { diff --git a/pkg/kv/kvclient/kvcoord/txn_coord_sender_server_test.go b/pkg/kv/kvclient/kvcoord/txn_coord_sender_server_test.go index 4899ad9ad56c..fc1f8a2ef18c 100644 --- a/pkg/kv/kvclient/kvcoord/txn_coord_sender_server_test.go +++ b/pkg/kv/kvclient/kvcoord/txn_coord_sender_server_test.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvbase" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -52,11 +53,11 @@ func TestHeartbeatFindsOutAboutAbortedTransaction(t *testing.T) { s, _, origDB := serverutils.StartServer(t, base.TestServerArgs{ Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ - TestingProposalFilter: func(args kvserverbase.ProposalFilterArgs) *roachpb.Error { + TestingProposalFilter: func(args kvserverbase.ProposalFilterArgs) *kvpb.Error { // We'll eventually expect to see an EndTxn(commit=false) // with the right intents. if args.Req.IsSingleEndTxnRequest() { - et := args.Req.Requests[0].GetInner().(*roachpb.EndTxnRequest) + et := args.Req.Requests[0].GetInner().(*kvpb.EndTxnRequest) if !et.Commit && et.Key.Equal(key) && reflect.DeepEqual(et.LockSpans, []roachpb.Span{{Key: key}, {Key: key2}}) { atomic.StoreInt64(&cleanupSeen, 1) diff --git a/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go b/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go index 3913a3c5493c..7738d566f772 100644 --- a/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go +++ b/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -329,7 +330,7 @@ func TestDB_PrepareForRetryAfterHeartbeatFailure(t *testing.T) { if err == nil { return errors.New("the handle is not poisoned yet") } - require.IsType(t, &roachpb.TransactionRetryWithProtoRefreshError{}, err) + require.IsType(t, &kvpb.TransactionRetryWithProtoRefreshError{}, err) return nil }) @@ -343,16 +344,16 @@ func TestDB_PrepareForRetryAfterHeartbeatFailure(t *testing.T) { } // getTxn fetches the requested key and returns the transaction info. -func getTxn(ctx context.Context, txn *kv.Txn) (*roachpb.Transaction, *roachpb.Error) { +func getTxn(ctx context.Context, txn *kv.Txn) (*roachpb.Transaction, *kvpb.Error) { txnMeta := txn.TestingCloneTxn().TxnMeta - qt := &roachpb.QueryTxnRequest{ - RequestHeader: roachpb.RequestHeader{ + qt := &kvpb.QueryTxnRequest{ + RequestHeader: kvpb.RequestHeader{ Key: txnMeta.Key, }, Txn: txnMeta, } - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Timestamp = txnMeta.WriteTimestamp ba.Add(qt) @@ -363,7 +364,7 @@ func getTxn(ctx context.Context, txn *kv.Txn) (*roachpb.Transaction, *roachpb.Er if pErr != nil { return nil, pErr } - return &br.Responses[0].GetInner().(*roachpb.QueryTxnResponse).QueriedTxn, nil + return &br.Responses[0].GetInner().(*kvpb.QueryTxnResponse).QueriedTxn, nil } func verifyCleanup( @@ -487,9 +488,9 @@ func TestTxnCoordSenderCommitCanceled(t *testing.T) { // a txn ID, and the value is a ready channel (chan struct) that will be // closed when the commit has been received and blocked. var blockCommits sync.Map - responseFilter := func(_ context.Context, ba *roachpb.BatchRequest, _ *roachpb.BatchResponse) *roachpb.Error { - if arg, ok := ba.GetArg(roachpb.EndTxn); ok && ba.Txn != nil { - et := arg.(*roachpb.EndTxnRequest) + responseFilter := func(_ context.Context, ba *kvpb.BatchRequest, _ *kvpb.BatchResponse) *kvpb.Error { + if arg, ok := ba.GetArg(kvpb.EndTxn); ok && ba.Txn != nil { + et := arg.(*kvpb.EndTxnRequest) readyC, ok := blockCommits.Load(ba.Txn.ID) if ok && et.Commit && len(et.InFlightWrites) == 0 { close(readyC.(chan struct{})) // notify test that commit is received and blocked @@ -539,13 +540,13 @@ func TestTxnCoordSenderCommitCanceled(t *testing.T) { // EndTxn(commit=false) async. We instead replicate what Txn.Rollback() would // do here (i.e. send a EndTxn(commit=false)) and assert that we receive the // expected error. - ba := &roachpb.BatchRequest{} - ba.Add(&roachpb.EndTxnRequest{Commit: false}) + ba := &kvpb.BatchRequest{} + ba.Add(&kvpb.EndTxnRequest{Commit: false}) _, pErr := txn.Send(ctx, ba) require.NotNil(t, pErr) - require.IsType(t, &roachpb.TransactionStatusError{}, pErr.GetDetail()) - txnErr := pErr.GetDetail().(*roachpb.TransactionStatusError) - require.Equal(t, roachpb.TransactionStatusError_REASON_TXN_COMMITTED, txnErr.Reason) + require.IsType(t, &kvpb.TransactionStatusError{}, pErr.GetDetail()) + txnErr := pErr.GetDetail().(*kvpb.TransactionStatusError) + require.Equal(t, kvpb.TransactionStatusError_REASON_TXN_COMMITTED, txnErr.Reason) } // TestTxnCoordSenderAddLockOnError verifies that locks are tracked if the @@ -569,7 +570,7 @@ func TestTxnCoordSenderAddLockOnError(t *testing.T) { } { err := txn.CPut(ctx, key, []byte("x"), kvclientutils.StrToCPutExistingValue("born to fail")) - if !errors.HasType(err, (*roachpb.ConditionFailedError)(nil)) { + if !errors.HasType(err, (*kvpb.ConditionFailedError)(nil)) { t.Fatal(err) } } @@ -586,7 +587,7 @@ func TestTxnCoordSenderAddLockOnError(t *testing.T) { func assertTransactionRetryError(t *testing.T, e error) { t.Helper() - if retErr := (*roachpb.TransactionRetryWithProtoRefreshError)(nil); errors.As(e, &retErr) { + if retErr := (*kvpb.TransactionRetryWithProtoRefreshError)(nil); errors.As(e, &retErr) { if !testutils.IsError(retErr, "TransactionRetryError") { t.Fatalf("expected the cause to be TransactionRetryError, but got %s", retErr) @@ -597,7 +598,7 @@ func assertTransactionRetryError(t *testing.T, e error) { } func assertTransactionAbortedError(t *testing.T, e error) { - if retErr := (*roachpb.TransactionRetryWithProtoRefreshError)(nil); errors.As(e, &retErr) { + if retErr := (*kvpb.TransactionRetryWithProtoRefreshError)(nil); errors.As(e, &retErr) { if !testutils.IsError(retErr, "TransactionAbortedError") { t.Fatalf("expected the cause to be TransactionAbortedError, but got %s", retErr) @@ -676,12 +677,12 @@ func TestTxnCoordSenderGCWithAmbiguousResultErr(t *testing.T) { testutils.RunTrueAndFalse(t, "errOnFirst", func(t *testing.T, errOnFirst bool) { key := roachpb.Key("a") - are := roachpb.NewAmbiguousResultErrorf("very ambiguous") + are := kvpb.NewAmbiguousResultErrorf("very ambiguous") knobs := &kvserver.StoreTestingKnobs{ - TestingResponseFilter: func(ctx context.Context, ba *roachpb.BatchRequest, br *roachpb.BatchResponse) *roachpb.Error { + TestingResponseFilter: func(ctx context.Context, ba *kvpb.BatchRequest, br *kvpb.BatchResponse) *kvpb.Error { for _, req := range ba.Requests { - if putReq, ok := req.GetInner().(*roachpb.PutRequest); ok && putReq.Key.Equal(key) { - return roachpb.NewError(are) + if putReq, ok := req.GetInner().(*kvpb.PutRequest); ok && putReq.Key.Equal(key) { + return kvpb.NewError(are) } } return nil @@ -732,7 +733,7 @@ func TestTxnCoordSenderTxnUpdatedOnError(t *testing.T) { testCases := []struct { // The test's name. name string - pErrGen func(txn *roachpb.Transaction) *roachpb.Error + pErrGen func(txn *roachpb.Transaction) *kvpb.Error callPrepareForRetry bool expEpoch enginepb.TxnEpoch expPri enginepb.TxnPriority @@ -744,7 +745,7 @@ func TestTxnCoordSenderTxnUpdatedOnError(t *testing.T) { { // No error, so nothing interesting either. name: "nil", - pErrGen: func(_ *roachpb.Transaction) *roachpb.Error { return nil }, + pErrGen: func(_ *roachpb.Transaction) *kvpb.Error { return nil }, expEpoch: 0, expPri: 1, expWriteTS: origTS, @@ -754,9 +755,9 @@ func TestTxnCoordSenderTxnUpdatedOnError(t *testing.T) { // On uncertainty error, new epoch begins. Timestamp moves ahead of // the existing write and LocalUncertaintyLimit, if one exists. name: "ReadWithinUncertaintyIntervalError without LocalUncertaintyLimit", - pErrGen: func(txn *roachpb.Transaction) *roachpb.Error { - pErr := roachpb.NewErrorWithTxn( - roachpb.NewReadWithinUncertaintyIntervalError( + pErrGen: func(txn *roachpb.Transaction) *kvpb.Error { + pErr := kvpb.NewErrorWithTxn( + kvpb.NewReadWithinUncertaintyIntervalError( origTS, // readTS hlc.ClockTimestamp{}, // localUncertaintyLimit txn, // txn @@ -775,9 +776,9 @@ func TestTxnCoordSenderTxnUpdatedOnError(t *testing.T) { // On uncertainty error, new epoch begins. Timestamp moves ahead of // the existing write and LocalUncertaintyLimit, if one exists. name: "ReadWithinUncertaintyIntervalError with LocalUncertaintyLimit", - pErrGen: func(txn *roachpb.Transaction) *roachpb.Error { - pErr := roachpb.NewErrorWithTxn( - roachpb.NewReadWithinUncertaintyIntervalError( + pErrGen: func(txn *roachpb.Transaction) *kvpb.Error { + pErr := kvpb.NewErrorWithTxn( + kvpb.NewReadWithinUncertaintyIntervalError( origTS, // readTS hlc.ClockTimestamp(plus20), // localUncertaintyLimit txn, // txn @@ -796,10 +797,10 @@ func TestTxnCoordSenderTxnUpdatedOnError(t *testing.T) { // On abort, nothing changes - we are left with a poisoned txn (unless we // call PrepareForRetry as in the next test case). name: "TransactionAbortedError", - pErrGen: func(txn *roachpb.Transaction) *roachpb.Error { + pErrGen: func(txn *roachpb.Transaction) *kvpb.Error { txn.WriteTimestamp = plus20 txn.Priority = 10 - return roachpb.NewErrorWithTxn(&roachpb.TransactionAbortedError{}, txn) + return kvpb.NewErrorWithTxn(&kvpb.TransactionAbortedError{}, txn) }, expPri: 1, expWriteTS: origTS, @@ -809,10 +810,10 @@ func TestTxnCoordSenderTxnUpdatedOnError(t *testing.T) { // On abort, reset the txn by calling PrepareForRetry, and then we get a // new priority to use for the next attempt. name: "TransactionAbortedError with PrepareForRetry", - pErrGen: func(txn *roachpb.Transaction) *roachpb.Error { + pErrGen: func(txn *roachpb.Transaction) *kvpb.Error { txn.WriteTimestamp = plus20 txn.Priority = 10 - return roachpb.NewErrorWithTxn(&roachpb.TransactionAbortedError{}, txn) + return kvpb.NewErrorWithTxn(&kvpb.TransactionAbortedError{}, txn) }, callPrepareForRetry: true, expNewTransaction: true, @@ -824,8 +825,8 @@ func TestTxnCoordSenderTxnUpdatedOnError(t *testing.T) { // On failed push, new epoch begins just past the pushed timestamp. // Additionally, priority ratchets up to just below the pusher's. name: "TransactionPushError", - pErrGen: func(txn *roachpb.Transaction) *roachpb.Error { - return roachpb.NewErrorWithTxn(&roachpb.TransactionPushError{ + pErrGen: func(txn *roachpb.Transaction) *kvpb.Error { + return kvpb.NewErrorWithTxn(&kvpb.TransactionPushError{ PusheeTxn: roachpb.Transaction{ TxnMeta: enginepb.TxnMeta{WriteTimestamp: plus10, Priority: 10}, }, @@ -839,10 +840,10 @@ func TestTxnCoordSenderTxnUpdatedOnError(t *testing.T) { { // On retry, restart with new epoch, timestamp and priority. name: "TransactionRetryError", - pErrGen: func(txn *roachpb.Transaction) *roachpb.Error { + pErrGen: func(txn *roachpb.Transaction) *kvpb.Error { txn.WriteTimestamp = plus10 txn.Priority = 10 - return roachpb.NewErrorWithTxn(&roachpb.TransactionRetryError{}, txn) + return kvpb.NewErrorWithTxn(&kvpb.TransactionRetryError{}, txn) }, expEpoch: 1, expPri: 10, @@ -859,9 +860,9 @@ func TestTxnCoordSenderTxnUpdatedOnError(t *testing.T) { clock := hlc.NewClockForTesting(manual) var senderFn kv.SenderFunc = func( - _ context.Context, ba *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, *roachpb.Error) { - var reply *roachpb.BatchResponse + _ context.Context, ba *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, *kvpb.Error) { + var reply *kvpb.BatchResponse pErr := test.pErrGen(ba.Txn) if pErr == nil { reply = ba.CreateReply() @@ -1040,7 +1041,7 @@ func TestTxnMultipleCoord(t *testing.T) { require.Equal(t, []roachpb.Span{{Key: key}, {Key: key2}}, refreshSpans) ba := txn.NewBatch() - ba.AddRawRequest(&roachpb.EndTxnRequest{Commit: true}) + ba.AddRawRequest(&kvpb.EndTxnRequest{Commit: true}) if err := txn.Run(ctx, ba); err != nil { t.Fatal(err) } @@ -1058,12 +1059,12 @@ func TestTxnCoordSenderNoDuplicateLockSpans(t *testing.T) { var expectedLockSpans []roachpb.Span - var senderFn kv.SenderFunc = func(_ context.Context, ba *roachpb.BatchRequest) ( - *roachpb.BatchResponse, *roachpb.Error) { + var senderFn kv.SenderFunc = func(_ context.Context, ba *kvpb.BatchRequest) ( + *kvpb.BatchResponse, *kvpb.Error) { br := ba.CreateReply() br.Txn = ba.Txn.Clone() - if rArgs, ok := ba.GetArg(roachpb.EndTxn); ok { - et := rArgs.(*roachpb.EndTxnRequest) + if rArgs, ok := ba.GetArg(kvpb.EndTxn); ok { + et := rArgs.(*kvpb.EndTxnRequest) if !reflect.DeepEqual(et.LockSpans, expectedLockSpans) { t.Errorf("Invalid lock spans: %+v; expected %+v", et.LockSpans, expectedLockSpans) } @@ -1458,8 +1459,8 @@ func TestTxnCommitWait(t *testing.T) { ts := txn.TestingCloneTxn().WriteTimestamp. Add(futureOffset.Nanoseconds(), 0). WithSynthetic(true) - h := roachpb.Header{Timestamp: ts} - put := roachpb.NewPut(key, roachpb.Value{}) + h := kvpb.Header{Timestamp: ts} + put := kvpb.NewPut(key, roachpb.Value{}) if _, pErr := kv.SendWrappedWith(ctx, s.DB.NonTransactionalSender(), h, put); pErr != nil { return pErr.GoError() } @@ -1570,27 +1571,27 @@ func TestAbortTransactionOnCommitErrors(t *testing.T) { testCases := []struct { err error - errFn func(roachpb.Transaction) *roachpb.Error + errFn func(roachpb.Transaction) *kvpb.Error asyncAbort bool }{ { - errFn: func(txn roachpb.Transaction) *roachpb.Error { + errFn: func(txn roachpb.Transaction) *kvpb.Error { const nodeID = 0 // ReadWithinUncertaintyIntervalErrors need a clock to have been // recorded on the origin. txn.UpdateObservedTimestamp(nodeID, makeTS(123, 0).UnsafeToClockTimestamp()) - return roachpb.NewErrorWithTxn( - roachpb.NewReadWithinUncertaintyIntervalError( + return kvpb.NewErrorWithTxn( + kvpb.NewReadWithinUncertaintyIntervalError( hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, hlc.Timestamp{}, hlc.ClockTimestamp{}), &txn) }, asyncAbort: false}, - {err: &roachpb.TransactionAbortedError{}, asyncAbort: true}, - {err: &roachpb.TransactionPushError{}, asyncAbort: false}, - {err: &roachpb.TransactionRetryError{}, asyncAbort: false}, - {err: &roachpb.RangeNotFoundError{}, asyncAbort: false}, - {err: &roachpb.RangeKeyMismatchError{}, asyncAbort: false}, - {err: &roachpb.TransactionStatusError{}, asyncAbort: false}, + {err: &kvpb.TransactionAbortedError{}, asyncAbort: true}, + {err: &kvpb.TransactionPushError{}, asyncAbort: false}, + {err: &kvpb.TransactionRetryError{}, asyncAbort: false}, + {err: &kvpb.RangeNotFoundError{}, asyncAbort: false}, + {err: &kvpb.RangeKeyMismatchError{}, asyncAbort: false}, + {err: &kvpb.TransactionStatusError{}, asyncAbort: false}, } for _, test := range testCases { @@ -1602,18 +1603,18 @@ func TestAbortTransactionOnCommitErrors(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(ctx) var senderFn kv.SenderFunc = func( - _ context.Context, ba *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, *roachpb.Error) { + _ context.Context, ba *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, *kvpb.Error) { br := ba.CreateReply() br.Txn = ba.Txn.Clone() - if et, hasET := ba.GetArg(roachpb.EndTxn); hasET { - if et.(*roachpb.EndTxnRequest).Commit { + if et, hasET := ba.GetArg(kvpb.EndTxn); hasET { + if et.(*kvpb.EndTxnRequest).Commit { commit.Store(true) if test.errFn != nil { return nil, test.errFn(*ba.Txn) } - return nil, roachpb.NewErrorWithTxn(test.err, ba.Txn) + return nil, kvpb.NewErrorWithTxn(test.err, ba.Txn) } abort.Store(true) } @@ -1664,7 +1665,7 @@ type mockSender struct { var _ kv.Sender = &mockSender{} -type matcher func(*roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) +type matcher func(*kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) // match adds a matcher to the list of matchers. func (s *mockSender) match(m matcher) { @@ -1673,8 +1674,8 @@ func (s *mockSender) match(m matcher) { // Send implements the client.Sender interface. func (s *mockSender) Send( - _ context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, *roachpb.Error) { + _ context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, *kvpb.Error) { for _, m := range s.matchers { br, pErr := m(ba) if br != nil || pErr != nil { @@ -1711,22 +1712,22 @@ func TestRollbackErrorStopsHeartbeat(t *testing.T) { ) db := kv.NewDB(ambient, factory, clock, stopper) - sender.match(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { - if _, ok := ba.GetArg(roachpb.EndTxn); !ok { + sender.match(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { + if _, ok := ba.GetArg(kvpb.EndTxn); !ok { resp := ba.CreateReply() resp.Txn = ba.Txn return resp, nil } - return nil, roachpb.NewErrorf("injected err") + return nil, kvpb.NewErrorf("injected err") }) txn := kv.NewTxn(ctx, db, roachpb.NodeID(1)) - txnHeader := roachpb.Header{ + txnHeader := kvpb.Header{ Txn: txn.TestingCloneTxn(), } if _, pErr := kv.SendWrappedWith( - ctx, txn, txnHeader, &roachpb.PutRequest{ - RequestHeader: roachpb.RequestHeader{ + ctx, txn, txnHeader, &kvpb.PutRequest{ + RequestHeader: kvpb.RequestHeader{ Key: roachpb.Key("a"), }, }, @@ -1739,7 +1740,7 @@ func TestRollbackErrorStopsHeartbeat(t *testing.T) { if _, pErr := kv.SendWrappedWith( ctx, txn, txnHeader, - &roachpb.EndTxnRequest{Commit: false}, + &kvpb.EndTxnRequest{Commit: false}, ); !testutils.IsPError(pErr, "injected err") { t.Fatal(pErr) } @@ -1781,28 +1782,28 @@ func TestOnePCErrorTracking(t *testing.T) { keyA, keyB, keyC := roachpb.Key("a"), roachpb.Key("b"), roachpb.Key("c") // Register a matcher catching the commit attempt. - sender.match(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { - if et, ok := ba.GetArg(roachpb.EndTxn); !ok { + sender.match(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { + if et, ok := ba.GetArg(kvpb.EndTxn); !ok { return nil, nil - } else if !et.(*roachpb.EndTxnRequest).Commit { + } else if !et.(*kvpb.EndTxnRequest).Commit { return nil, nil } - return nil, roachpb.NewErrorf("injected err") + return nil, kvpb.NewErrorf("injected err") }) // Register a matcher catching the rollback attempt. - sender.match(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { - et, ok := ba.GetArg(roachpb.EndTxn) + sender.match(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { + et, ok := ba.GetArg(kvpb.EndTxn) if !ok { return nil, nil } - etReq := et.(*roachpb.EndTxnRequest) + etReq := et.(*kvpb.EndTxnRequest) if etReq.Commit { return nil, nil } expLocks := []roachpb.Span{{Key: keyA}, {Key: keyB, EndKey: keyC}} locks := etReq.LockSpans if !reflect.DeepEqual(locks, expLocks) { - return nil, roachpb.NewErrorf("expected locks %s, got: %s", expLocks, locks) + return nil, kvpb.NewErrorf("expected locks %s, got: %s", expLocks, locks) } resp := ba.CreateReply() // Set the response's txn to the Aborted status (as the server would). This @@ -1813,7 +1814,7 @@ func TestOnePCErrorTracking(t *testing.T) { }) txn := kv.NewTxn(ctx, db, roachpb.NodeID(1)) - txnHeader := roachpb.Header{ + txnHeader := kvpb.Header{ Txn: txn.TestingCloneTxn(), } b := txn.NewBatch() @@ -1827,7 +1828,7 @@ func TestOnePCErrorTracking(t *testing.T) { // to it. if _, pErr := kv.SendWrappedWith( ctx, txn, txnHeader, - &roachpb.EndTxnRequest{Commit: false}, + &kvpb.EndTxnRequest{Commit: false}, ); pErr != nil { t.Fatal(pErr) } @@ -1853,8 +1854,8 @@ func TestCommitReadOnlyTransaction(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(ctx) - var calls []roachpb.Method - sender.match(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + var calls []kvpb.Method + sender.match(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { calls = append(calls, ba.Methods()...) return nil, nil }) @@ -1885,9 +1886,9 @@ func TestCommitReadOnlyTransaction(t *testing.T) { t.Fatal(err) } - expectedCalls := []roachpb.Method(nil) + expectedCalls := []kvpb.Method(nil) if withGet { - expectedCalls = append(expectedCalls, roachpb.Get) + expectedCalls = append(expectedCalls, kvpb.Get) } if !reflect.DeepEqual(expectedCalls, calls) { t.Fatalf("expected %s, got %s", expectedCalls, calls) @@ -1908,8 +1909,8 @@ func TestCommitMutatingTransaction(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(ctx) - var calls []roachpb.Method - sender.match(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + var calls []kvpb.Method + sender.match(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { br := ba.CreateReply() br.Txn = ba.Txn.Clone() @@ -1917,8 +1918,8 @@ func TestCommitMutatingTransaction(t *testing.T) { if !bytes.Equal(ba.Txn.Key, roachpb.Key("a")) { t.Errorf("expected transaction key to be \"a\"; got %s", ba.Txn.Key) } - if et, ok := ba.GetArg(roachpb.EndTxn); ok { - if !et.(*roachpb.EndTxnRequest).Commit { + if et, ok := ba.GetArg(kvpb.EndTxn); ok { + if !et.(*kvpb.EndTxnRequest).Commit { t.Errorf("expected commit to be true") } br.Txn.Status = roachpb.COMMITTED @@ -1939,7 +1940,7 @@ func TestCommitMutatingTransaction(t *testing.T) { // Test all transactional write methods. testArgs := []struct { f func(ctx context.Context, txn *kv.Txn) error - expMethod roachpb.Method + expMethod kvpb.Method // pointWrite is set if the method is a "point write", which means that it // will be pipelined and we should expect a QueryIntent request at commit // time. @@ -1947,12 +1948,12 @@ func TestCommitMutatingTransaction(t *testing.T) { }{ { f: func(ctx context.Context, txn *kv.Txn) error { return txn.Put(ctx, "a", "b") }, - expMethod: roachpb.Put, + expMethod: kvpb.Put, pointWrite: true, }, { f: func(ctx context.Context, txn *kv.Txn) error { return txn.CPut(ctx, "a", "b", nil) }, - expMethod: roachpb.ConditionalPut, + expMethod: kvpb.ConditionalPut, pointWrite: true, }, { @@ -1960,7 +1961,7 @@ func TestCommitMutatingTransaction(t *testing.T) { _, err := txn.Inc(ctx, "a", 1) return err }, - expMethod: roachpb.Increment, + expMethod: kvpb.Increment, pointWrite: true, }, { @@ -1968,7 +1969,7 @@ func TestCommitMutatingTransaction(t *testing.T) { _, err := txn.Del(ctx, "a") return err }, - expMethod: roachpb.Delete, + expMethod: kvpb.Delete, pointWrite: true, }, { @@ -1976,7 +1977,7 @@ func TestCommitMutatingTransaction(t *testing.T) { _, err := txn.DelRange(ctx, "a", "b", false /* returnKeys */) return err }, - expMethod: roachpb.DeleteRange, + expMethod: kvpb.DeleteRange, pointWrite: false, }, } @@ -1987,11 +1988,11 @@ func TestCommitMutatingTransaction(t *testing.T) { if err := db.Txn(ctx, test.f); err != nil { t.Fatalf("%d: unexpected error on commit: %s", i, err) } - expectedCalls := []roachpb.Method{test.expMethod} + expectedCalls := []kvpb.Method{test.expMethod} if test.pointWrite { - expectedCalls = append(expectedCalls, roachpb.QueryIntent) + expectedCalls = append(expectedCalls, kvpb.QueryIntent) } - expectedCalls = append(expectedCalls, roachpb.EndTxn) + expectedCalls = append(expectedCalls, kvpb.EndTxn) if !reflect.DeepEqual(expectedCalls, calls) { t.Fatalf("%d: expected %s, got %s", i, expectedCalls, calls) } @@ -2011,8 +2012,8 @@ func TestAbortReadOnlyTransaction(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(ctx) - var calls []roachpb.Method - sender.match(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + var calls []kvpb.Method + sender.match(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { calls = append(calls, ba.Methods()...) return nil, nil }) @@ -2052,18 +2053,18 @@ func TestEndWriteRestartReadOnlyTransaction(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(ctx) - var calls []roachpb.Method - sender.match(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + var calls []kvpb.Method + sender.match(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { br := ba.CreateReply() br.Txn = ba.Txn.Clone() calls = append(calls, ba.Methods()...) switch ba.Requests[0].GetInner().Method() { - case roachpb.Put, roachpb.Scan: - return nil, roachpb.NewErrorWithTxn( - roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE, "test err"), + case kvpb.Put, kvpb.Scan: + return nil, kvpb.NewErrorWithTxn( + kvpb.NewTransactionRetryError(kvpb.RETRY_SERIALIZABLE, "test err"), ba.Txn) - case roachpb.EndTxn: + case kvpb.EndTxn: br.Txn.Status = roachpb.COMMITTED } return br, nil @@ -2110,11 +2111,11 @@ func TestEndWriteRestartReadOnlyTransaction(t *testing.T) { t.Fatalf("expected error: %t, got error: %v", !success, err) } - var expCalls []roachpb.Method + var expCalls []kvpb.Method if write { - expCalls = []roachpb.Method{roachpb.Put, roachpb.EndTxn} + expCalls = []kvpb.Method{kvpb.Put, kvpb.EndTxn} } else { - expCalls = []roachpb.Method{roachpb.Scan, roachpb.EndTxn} + expCalls = []kvpb.Method{kvpb.Scan, kvpb.EndTxn} } if !reflect.DeepEqual(expCalls, calls) { t.Fatalf("expected %v, got %v", expCalls, calls) @@ -2138,18 +2139,18 @@ func TestTransactionKeyNotChangedInRestart(t *testing.T) { keys := []string{"first", "second"} attempt := 0 - sender.match(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + sender.match(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { br := ba.CreateReply() br.Txn = ba.Txn.Clone() // Ignore the final EndTxnRequest. - if _, ok := ba.GetArg(roachpb.EndTxn); ok { + if _, ok := ba.GetArg(kvpb.EndTxn); ok { br.Txn.Status = roachpb.COMMITTED return br, nil } // Both attempts should have a PutRequest. - if _, ok := ba.GetArg(roachpb.Put); !ok { + if _, ok := ba.GetArg(kvpb.Put); !ok { t.Fatalf("failed to find a put request: %v", ba) } @@ -2160,8 +2161,8 @@ func TestTransactionKeyNotChangedInRestart(t *testing.T) { } if attempt == 0 { - return nil, roachpb.NewErrorWithTxn( - roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE, "test err"), + return nil, kvpb.NewErrorWithTxn( + kvpb.NewTransactionRetryError(kvpb.RETRY_SERIALIZABLE, "test err"), ba.Txn) } return br, nil @@ -2204,10 +2205,10 @@ func TestSequenceNumbers(t *testing.T) { defer stopper.Stop(ctx) var expSequence enginepb.TxnSeq - sender.match(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + sender.match(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { for _, ru := range ba.Requests { args := ru.GetInner() - if args.Method() == roachpb.QueryIntent { + if args.Method() == kvpb.QueryIntent { // QueryIntent requests don't have sequence numbers. continue } @@ -2235,9 +2236,9 @@ func TestSequenceNumbers(t *testing.T) { txn := kv.NewTxn(ctx, db, 0 /* gatewayNodeID */) for i := 0; i < 5; i++ { - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} for j := 0; j < i; j++ { - ba.Add(roachpb.NewPut(roachpb.Key("a"), roachpb.MakeValueFromString("foo")).(*roachpb.PutRequest)) + ba.Add(kvpb.NewPut(roachpb.Key("a"), roachpb.MakeValueFromString("foo")).(*kvpb.PutRequest)) } if _, pErr := txn.Send(ctx, ba); pErr != nil { t.Fatal(pErr) @@ -2258,15 +2259,15 @@ func TestConcurrentTxnRequestsProhibited(t *testing.T) { defer stopper.Stop(ctx) putSync := make(chan struct{}) - sender.match(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { - if _, ok := ba.GetArg(roachpb.Put); ok { + sender.match(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { + if _, ok := ba.GetArg(kvpb.Put); ok { // Block the Put until the Get runs. putSync <- struct{}{} <-putSync } br := ba.CreateReply() br.Txn = ba.Txn.Clone() - if _, ok := ba.GetArg(roachpb.EndTxn); ok { + if _, ok := ba.GetArg(kvpb.EndTxn); ok { br.Txn.Status = roachpb.COMMITTED } return br, nil @@ -2338,10 +2339,10 @@ func TestTxnRequestTxnTimestamp(t *testing.T) { {hlc.Timestamp{WallTime: 20, Logical: 1}, hlc.Timestamp{WallTime: 20, Logical: 1}}, } - sender.match(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + sender.match(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { req := requests[curReq] if req.expRequestTS != ba.Txn.WriteTimestamp { - return nil, roachpb.NewErrorf("%d: expected ts %s got %s", + return nil, kvpb.NewErrorf("%d: expected ts %s got %s", curReq, req.expRequestTS, ba.Txn.WriteTimestamp) } @@ -2380,8 +2381,8 @@ func TestReadOnlyTxnObeysDeadline(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(ctx) - sender.match(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { - if _, ok := ba.GetArg(roachpb.Get); ok { + sender.match(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { + if _, ok := ba.GetArg(kvpb.Get); ok { manual.Advance(100) br := ba.CreateReply() br.Txn = ba.Txn.Clone() @@ -2449,14 +2450,14 @@ func TestTxnCoordSenderPipelining(t *testing.T) { defer s.Stop() distSender := s.DB.GetFactory().(*kvcoord.TxnCoordSenderFactory).NonTransactionalSender() - var calls []roachpb.Method + var calls []kvpb.Method var senderFn kv.SenderFunc = func( - ctx context.Context, ba *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, *kvpb.Error) { calls = append(calls, ba.Methods()...) - if et, ok := ba.GetArg(roachpb.EndTxn); ok { + if et, ok := ba.GetArg(kvpb.EndTxn); ok { // Ensure that no transactions enter a STAGING state. - et.(*roachpb.EndTxnRequest).InFlightWrites = nil + et.(*kvpb.EndTxnRequest).InFlightWrites = nil } return distSender.Send(ctx, ba) } @@ -2489,9 +2490,9 @@ func TestTxnCoordSenderPipelining(t *testing.T) { t.Fatal(err) } - require.Equal(t, []roachpb.Method{ - roachpb.Put, roachpb.QueryIntent, roachpb.EndTxn, - roachpb.Put, roachpb.EndTxn, + require.Equal(t, []kvpb.Method{ + kvpb.Put, kvpb.QueryIntent, kvpb.EndTxn, + kvpb.Put, kvpb.EndTxn, }, calls) for _, action := range []func(ctx context.Context, txn *kv.Txn) error{ @@ -2526,14 +2527,14 @@ func TestAnchorKey(t *testing.T) { key2 := roachpb.Key("b") var senderFn kv.SenderFunc = func( - ctx context.Context, ba *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, *kvpb.Error) { if !roachpb.Key(ba.Txn.Key).Equal(key2) { t.Fatalf("expected anchor %q, got %q", key2, ba.Txn.Key) } br := ba.CreateReply() br.Txn = ba.Txn.Clone() - if _, ok := ba.GetArg(roachpb.EndTxn); ok { + if _, ok := ba.GetArg(kvpb.EndTxn); ok { br.Txn.Status = roachpb.COMMITTED } return br, nil @@ -2574,12 +2575,12 @@ func TestLeafTxnClientRejectError(t *testing.T) { // where the first one gets a TransactionAbortedError. errKey := roachpb.Key("a") knobs := &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(_ context.Context, ba *roachpb.BatchRequest) *roachpb.Error { - if g, ok := ba.GetArg(roachpb.Get); ok && g.(*roachpb.GetRequest).Key.Equal(errKey) { + TestingRequestFilter: func(_ context.Context, ba *kvpb.BatchRequest) *kvpb.Error { + if g, ok := ba.GetArg(kvpb.Get); ok && g.(*kvpb.GetRequest).Key.Equal(errKey) { txn := ba.Txn.Clone() txn.Status = roachpb.ABORTED - return roachpb.NewErrorWithTxn( - roachpb.NewTransactionAbortedError(roachpb.ABORT_REASON_UNKNOWN), txn, + return kvpb.NewErrorWithTxn( + kvpb.NewTransactionAbortedError(kvpb.ABORT_REASON_UNKNOWN), txn, ) } return nil @@ -2606,7 +2607,7 @@ func TestLeafTxnClientRejectError(t *testing.T) { // test is interested in demonstrating is that it's not a // TransactionRetryWithProtoRefreshError. _, err := leafTxn.Get(ctx, roachpb.Key("a")) - if !errors.HasType(err, (*roachpb.UnhandledRetryableError)(nil)) { + if !errors.HasType(err, (*kvpb.UnhandledRetryableError)(nil)) { t.Fatalf("expected UnhandledRetryableError(TransactionAbortedError), got: (%T) %v", err, err) } } @@ -2661,9 +2662,9 @@ func TestPutsInStagingTxn(t *testing.T) { var putInStagingSeen bool var storeKnobs kvserver.StoreTestingKnobs - storeKnobs.TestingRequestFilter = func(ctx context.Context, ba *roachpb.BatchRequest) *roachpb.Error { - put, ok := ba.GetArg(roachpb.Put) - if !ok || !put.(*roachpb.PutRequest).Key.Equal(keyB) { + storeKnobs.TestingRequestFilter = func(ctx context.Context, ba *kvpb.BatchRequest) *kvpb.Error { + put, ok := ba.GetArg(kvpb.Put) + if !ok || !put.(*kvpb.PutRequest).Key.Equal(keyB) { return nil } txn := ba.Txn @@ -2725,11 +2726,11 @@ func TestTxnManualRefresh(t *testing.T) { // launch some requests from the client and then pass control flow of handling // those requests back to the test. type resp struct { - br *roachpb.BatchResponse - pErr *roachpb.Error + br *kvpb.BatchResponse + pErr *kvpb.Error } type req struct { - ba *roachpb.BatchRequest + ba *kvpb.BatchRequest respCh chan resp } type testCase struct { @@ -2757,7 +2758,7 @@ func TestTxnManualRefresh(t *testing.T) { }() { r := <-reqCh - _, ok := r.ba.GetArg(roachpb.Get) + _, ok := r.ba.GetArg(kvpb.Get) require.True(t, ok) br := r.ba.CreateReply() br.Txn = r.ba.Txn @@ -2785,7 +2786,7 @@ func TestTxnManualRefresh(t *testing.T) { }() { r := <-reqCh - _, ok := r.ba.GetArg(roachpb.Get) + _, ok := r.ba.GetArg(kvpb.Get) require.True(t, ok) br := r.ba.CreateReply() br.Txn = r.ba.Txn @@ -2798,7 +2799,7 @@ func TestTxnManualRefresh(t *testing.T) { }() { r := <-reqCh - _, ok := r.ba.GetArg(roachpb.Put) + _, ok := r.ba.GetArg(kvpb.Put) require.True(t, ok) br := r.ba.CreateReply() br.Txn = r.ba.Txn.Clone() @@ -2814,7 +2815,7 @@ func TestTxnManualRefresh(t *testing.T) { }() { r := <-reqCh - _, ok := r.ba.GetArg(roachpb.Refresh) + _, ok := r.ba.GetArg(kvpb.Refresh) require.True(t, ok) br := r.ba.CreateReply() br.Txn = r.ba.Txn.Clone() @@ -2841,7 +2842,7 @@ func TestTxnManualRefresh(t *testing.T) { }() { r := <-reqCh - _, ok := r.ba.GetArg(roachpb.Get) + _, ok := r.ba.GetArg(kvpb.Get) require.True(t, ok) br := r.ba.CreateReply() br.Txn = r.ba.Txn @@ -2854,7 +2855,7 @@ func TestTxnManualRefresh(t *testing.T) { }() { r := <-reqCh - _, ok := r.ba.GetArg(roachpb.Put) + _, ok := r.ba.GetArg(kvpb.Put) require.True(t, ok) br := r.ba.CreateReply() br.Txn = r.ba.Txn.Clone() @@ -2870,11 +2871,11 @@ func TestTxnManualRefresh(t *testing.T) { }() { r := <-reqCh - _, ok := r.ba.GetArg(roachpb.Refresh) + _, ok := r.ba.GetArg(kvpb.Refresh) require.True(t, ok) // Rejects the refresh due to a conflicting write. - pErr := roachpb.NewError(roachpb.NewRefreshFailedError( - roachpb.RefreshFailedError_REASON_COMMITTED_VALUE, roachpb.Key("a"), hlc.Timestamp{WallTime: 1})) + pErr := kvpb.NewError(kvpb.NewRefreshFailedError( + kvpb.RefreshFailedError_REASON_COMMITTED_VALUE, roachpb.Key("a"), hlc.Timestamp{WallTime: 1})) r.respCh <- resp{pErr: pErr} } require.Regexp(t, "TransactionRetryError: retry txn \\(RETRY_SERIALIZABLE - failed preemptive "+ @@ -2890,8 +2891,8 @@ func TestTxnManualRefresh(t *testing.T) { defer stopper.Stop(ctx) reqCh := make(chan req) - var senderFn kv.SenderFunc = func(_ context.Context, ba *roachpb.BatchRequest) ( - *roachpb.BatchResponse, *roachpb.Error) { + var senderFn kv.SenderFunc = func(_ context.Context, ba *kvpb.BatchRequest) ( + *kvpb.BatchResponse, *kvpb.Error) { r := req{ ba: ba, respCh: make(chan resp), @@ -2899,13 +2900,13 @@ func TestTxnManualRefresh(t *testing.T) { select { case reqCh <- r: case <-ctx.Done(): - return nil, roachpb.NewError(ctx.Err()) + return nil, kvpb.NewError(ctx.Err()) } select { case rr := <-r.respCh: return rr.br, rr.pErr case <-ctx.Done(): - return nil, roachpb.NewError(ctx.Err()) + return nil, kvpb.NewError(ctx.Err()) } } ambient := log.MakeTestingAmbientCtxWithNewTracer() diff --git a/pkg/kv/kvclient/kvcoord/txn_interceptor_committer.go b/pkg/kv/kvclient/kvcoord/txn_interceptor_committer.go index cd132876e3ae..ae82fca11dea 100644 --- a/pkg/kv/kvclient/kvcoord/txn_interceptor_committer.go +++ b/pkg/kv/kvclient/kvcoord/txn_interceptor_committer.go @@ -14,6 +14,7 @@ import ( "context" "sync" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/multitenant" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" @@ -121,17 +122,17 @@ type txnCommitter struct { // SendLocked implements the lockedSender interface. func (tc *txnCommitter) SendLocked( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, *kvpb.Error) { // If the batch does not include an EndTxn request, pass it through. - rArgs, hasET := ba.GetArg(roachpb.EndTxn) + rArgs, hasET := ba.GetArg(kvpb.EndTxn) if !hasET { return tc.wrapped.SendLocked(ctx, ba) } - et := rArgs.(*roachpb.EndTxnRequest) + et := rArgs.(*kvpb.EndTxnRequest) if err := tc.validateEndTxnBatch(ba); err != nil { - return nil, roachpb.NewError(err) + return nil, kvpb.NewError(err) } // Determine whether we can elide the EndTxn entirely. We can do so if the @@ -230,7 +231,7 @@ func (tc *txnCommitter) SendLocked( // and there were no writes in the batch request. return br, nil default: - return nil, roachpb.NewErrorf("unexpected response status without error: %v", br.Txn) + return nil, kvpb.NewErrorf("unexpected response status without error: %v", br.Txn) } // Determine whether the transaction needs to either retry or refresh. When @@ -263,7 +264,7 @@ func (tc *txnCommitter) SendLocked( } // validateEndTxnBatch runs sanity checks on a commit or rollback request. -func (tc *txnCommitter) validateEndTxnBatch(ba *roachpb.BatchRequest) error { +func (tc *txnCommitter) validateEndTxnBatch(ba *kvpb.BatchRequest) error { // Check that we don't combine a limited DeleteRange with a commit. We cannot // attempt to run such a batch as a 1PC because, if it gets split and thus // doesn't run as a 1PC, resolving the intents will be very expensive. @@ -276,9 +277,9 @@ func (tc *txnCommitter) validateEndTxnBatch(ba *roachpb.BatchRequest) error { if ba.Header.MaxSpanRequestKeys == 0 { return nil } - e, endTxn := ba.GetArg(roachpb.EndTxn) - _, delRange := ba.GetArg(roachpb.DeleteRange) - if delRange && endTxn && !e.(*roachpb.EndTxnRequest).Require1PC { + e, endTxn := ba.GetArg(kvpb.EndTxn) + _, delRange := ba.GetArg(kvpb.DeleteRange) + if delRange && endTxn && !e.(*kvpb.EndTxnRequest).Require1PC { return errors.Errorf("possible 1PC batch cannot contain EndTxn without setting Require1PC; see #37457") } return nil @@ -292,8 +293,8 @@ func (tc *txnCommitter) validateEndTxnBatch(ba *roachpb.BatchRequest) error { // The method is used for read-only transactions, which never need to write a // transaction record. func (tc *txnCommitter) sendLockedWithElidedEndTxn( - ctx context.Context, ba *roachpb.BatchRequest, et *roachpb.EndTxnRequest, -) (br *roachpb.BatchResponse, pErr *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, et *kvpb.EndTxnRequest, +) (br *kvpb.BatchResponse, pErr *kvpb.Error) { // Send the batch without its final request, which we know to be the EndTxn // request that we're eliding. If this would result in us sending an empty // batch, mock out a reply instead of sending anything. @@ -305,7 +306,7 @@ func (tc *txnCommitter) sendLockedWithElidedEndTxn( return nil, pErr } } else { - br = &roachpb.BatchResponse{} + br = &kvpb.BatchResponse{} // NB: there's no need to clone the txn proto here because we already // call cloneWithStatus below. br.Txn = ba.Txn @@ -331,7 +332,7 @@ func (tc *txnCommitter) sendLockedWithElidedEndTxn( br.Txn = cloneWithStatus(br.Txn, status) // Synthesize and append an EndTxn response. - br.Add(&roachpb.EndTxnResponse{}) + br.Add(&kvpb.EndTxnResponse{}) return br, nil } @@ -351,7 +352,7 @@ const ( // writes, which all should have corresponding QueryIntent requests in the // batch. func (tc *txnCommitter) canCommitInParallel( - ctx context.Context, ba *roachpb.BatchRequest, et *roachpb.EndTxnRequest, etAttempt endTxnAttempt, + ctx context.Context, ba *kvpb.BatchRequest, et *kvpb.EndTxnRequest, etAttempt endTxnAttempt, ) bool { if !parallelCommitsEnabled.Get(&tc.st.SV) { return false @@ -380,8 +381,8 @@ func (tc *txnCommitter) canCommitInParallel( for _, ru := range ba.Requests[:len(ba.Requests)-1] { req := ru.GetInner() switch { - case roachpb.IsIntentWrite(req): - if roachpb.IsRange(req) { + case kvpb.IsIntentWrite(req): + if kvpb.IsRange(req) { // Similar to how we can't pipeline ranged writes, we also can't // commit in parallel with them. The reason for this is that the // status resolution process for STAGING transactions wouldn't @@ -393,7 +394,7 @@ func (tc *txnCommitter) canCommitInParallel( // process for STAGING transactions. Populating InFlightWrites // has already been done by the txnPipeliner. - case req.Method() == roachpb.QueryIntent: + case req.Method() == kvpb.QueryIntent: // QueryIntent requests are compatable with parallel commits. The // intents being queried are also attached to the EndTxn request's // InFlightWrites set and are visible to the status resolution @@ -426,17 +427,17 @@ func mergeIntoSpans(s []roachpb.Span, ws []roachpb.SequencedWrite) ([]roachpb.Sp // needTxnRetryAfterStaging determines whether the transaction needs to refresh // (see txnSpanRefresher) or retry based on the batch response of a parallel // commit attempt. -func needTxnRetryAfterStaging(br *roachpb.BatchResponse) *roachpb.Error { +func needTxnRetryAfterStaging(br *kvpb.BatchResponse) *kvpb.Error { if len(br.Responses) == 0 { - return roachpb.NewErrorf("no responses in BatchResponse: %v", br) + return kvpb.NewErrorf("no responses in BatchResponse: %v", br) } lastResp := br.Responses[len(br.Responses)-1].GetInner() - etResp, ok := lastResp.(*roachpb.EndTxnResponse) + etResp, ok := lastResp.(*kvpb.EndTxnResponse) if !ok { - return roachpb.NewErrorf("unexpected response in BatchResponse: %v", lastResp) + return kvpb.NewErrorf("unexpected response in BatchResponse: %v", lastResp) } if etResp.StagingTimestamp.IsEmpty() { - return roachpb.NewErrorf("empty StagingTimestamp in EndTxnResponse: %v", etResp) + return kvpb.NewErrorf("empty StagingTimestamp in EndTxnResponse: %v", etResp) } if etResp.StagingTimestamp.Less(br.Txn.WriteTimestamp) { // If the timestamp that the transaction record was staged at @@ -449,14 +450,14 @@ func needTxnRetryAfterStaging(br *roachpb.BatchResponse) *roachpb.Error { // Note that we leave the transaction record that we wrote in the STAGING // state, which is not ideal. But as long as we continue heartbeating the // txn record, it being PENDING or STAGING does not make a difference. - reason := roachpb.RETRY_SERIALIZABLE + reason := kvpb.RETRY_SERIALIZABLE if br.Txn.WriteTooOld { - reason = roachpb.RETRY_WRITE_TOO_OLD + reason = kvpb.RETRY_WRITE_TOO_OLD } - err := roachpb.NewTransactionRetryError( + err := kvpb.NewTransactionRetryError( reason, "serializability failure concurrent with STAGING") txn := cloneWithStatus(br.Txn, roachpb.PENDING) - return roachpb.NewErrorWithTxn(err, txn) + return kvpb.NewErrorWithTxn(err, txn) } return nil } @@ -500,9 +501,9 @@ func makeTxnCommitExplicitLocked( txn = txn.Clone() // Construct a new batch with just an EndTxn request. - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: txn} - et := roachpb.EndTxnRequest{Commit: true} + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: txn} + et := kvpb.EndTxnRequest{Commit: true} et.Key = txn.Key et.LockSpans = lockSpans ba.Add(&et) @@ -510,13 +511,13 @@ func makeTxnCommitExplicitLocked( _, pErr := s.SendLocked(ctx, ba) if pErr != nil { switch t := pErr.GetDetail().(type) { - case *roachpb.TransactionStatusError: + case *kvpb.TransactionStatusError: // Detect whether the error indicates that someone else beat // us to explicitly committing the transaction record. - if t.Reason == roachpb.TransactionStatusError_REASON_TXN_COMMITTED { + if t.Reason == kvpb.TransactionStatusError_REASON_TXN_COMMITTED { return nil } - case *roachpb.TransactionRetryError: + case *kvpb.TransactionRetryError: logFunc := log.Errorf if util.RaceEnabled { logFunc = log.Fatalf diff --git a/pkg/kv/kvclient/kvcoord/txn_interceptor_committer_test.go b/pkg/kv/kvclient/kvcoord/txn_interceptor_committer_test.go index b9c581d9c526..c393c002c6ad 100644 --- a/pkg/kv/kvclient/kvcoord/txn_interceptor_committer_test.go +++ b/pkg/kv/kvclient/kvcoord/txn_interceptor_committer_test.go @@ -14,6 +14,7 @@ import ( "context" "testing" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -57,16 +58,16 @@ func TestTxnCommitterElideEndTxn(t *testing.T) { // Test the case where the EndTxn request is part of a larger batch of // requests. - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - ba.Add(&roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.EndTxnRequest{Commit: commit, LockSpans: nil}) + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + ba.Add(&kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.EndTxnRequest{Commit: commit, LockSpans: nil}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 2) - require.IsType(t, &roachpb.GetRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.GetRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[1].GetInner()) br := ba.CreateReply() br.Txn = ba.Txn @@ -84,9 +85,9 @@ func TestTxnCommitterElideEndTxn(t *testing.T) { // Test the case where the EndTxn request is alone. ba.Requests = nil - ba.Add(&roachpb.EndTxnRequest{Commit: commit, LockSpans: nil}) + ba.Add(&kvpb.EndTxnRequest{Commit: commit, LockSpans: nil}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Fail(t, "should not have issued batch request", ba) return nil, nil }) @@ -116,12 +117,12 @@ func TestTxnCommitterAttachesTxnKey(t *testing.T) { intents := []roachpb.Span{{Key: keyA}} // Verify that the txn key is attached to committing EndTxn requests. - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.EndTxnRequest{Commit: true, LockSpans: intents}) + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.EndTxnRequest{Commit: true, LockSpans: intents}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 2) require.Equal(t, keyA, ba.Requests[0].GetInner().Header().Key) require.Equal(t, roachpb.Key(txn.Key), ba.Requests[1].GetInner().Header().Key) @@ -138,9 +139,9 @@ func TestTxnCommitterAttachesTxnKey(t *testing.T) { // Verify that the txn key is attached to aborting EndTxn requests. ba.Requests = nil - ba.Add(&roachpb.EndTxnRequest{Commit: false, LockSpans: intents}) + ba.Add(&kvpb.EndTxnRequest{Commit: false, LockSpans: intents}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) require.Equal(t, roachpb.Key(txn.Key), ba.Requests[0].GetInner().Header().Key) @@ -176,11 +177,11 @@ func TestTxnCommitterStripsInFlightWrites(t *testing.T) { // Verify that the QueryIntent and the Put are both attached as lock spans // to the committing EndTxn request when expected. - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - qiArgs := roachpb.QueryIntentRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}} - putArgs := roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyB}} - etArgs := roachpb.EndTxnRequest{Commit: true} + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + qiArgs := kvpb.QueryIntentRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}} + putArgs := kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyB}} + etArgs := kvpb.EndTxnRequest{Commit: true} qiArgs.Txn.Sequence = 1 putArgs.Sequence = 2 etArgs.Sequence = 3 @@ -190,11 +191,11 @@ func TestTxnCommitterStripsInFlightWrites(t *testing.T) { etArgsCopy := etArgs ba.Add(&putArgs, &qiArgs, &etArgsCopy) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 3) - require.IsType(t, &roachpb.EndTxnRequest{}, ba.Requests[2].GetInner()) + require.IsType(t, &kvpb.EndTxnRequest{}, ba.Requests[2].GetInner()) - et := ba.Requests[2].GetInner().(*roachpb.EndTxnRequest) + et := ba.Requests[2].GetInner().(*kvpb.EndTxnRequest) require.True(t, et.Commit) require.Len(t, et.LockSpans, 2) require.Equal(t, []roachpb.Span{{Key: keyA}, {Key: keyB}}, et.LockSpans) @@ -217,11 +218,11 @@ func TestTxnCommitterStripsInFlightWrites(t *testing.T) { etArgsCopy = etArgs ba.Add(&putArgs, &qiArgs, &etArgsCopy) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 3) - require.IsType(t, &roachpb.EndTxnRequest{}, ba.Requests[2].GetInner()) + require.IsType(t, &kvpb.EndTxnRequest{}, ba.Requests[2].GetInner()) - et := ba.Requests[2].GetInner().(*roachpb.EndTxnRequest) + et := ba.Requests[2].GetInner().(*kvpb.EndTxnRequest) require.True(t, et.Commit) require.Len(t, et.InFlightWrites, 2) require.Equal(t, roachpb.SequencedWrite{Key: keyA, Sequence: 1}, et.InFlightWrites[0]) @@ -247,11 +248,11 @@ func TestTxnCommitterStripsInFlightWrites(t *testing.T) { } ba.Add(&putArgs, &qiArgs, &etArgsWithTrigger) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 3) - require.IsType(t, &roachpb.EndTxnRequest{}, ba.Requests[2].GetInner()) + require.IsType(t, &kvpb.EndTxnRequest{}, ba.Requests[2].GetInner()) - et := ba.Requests[2].GetInner().(*roachpb.EndTxnRequest) + et := ba.Requests[2].GetInner().(*kvpb.EndTxnRequest) require.True(t, et.Commit) require.Len(t, et.LockSpans, 2) require.Equal(t, []roachpb.Span{{Key: keyA}, {Key: keyB}}, et.LockSpans) @@ -271,18 +272,18 @@ func TestTxnCommitterStripsInFlightWrites(t *testing.T) { // In-flight writes should not be attached because ranged writes cannot // be parallelized with a commit. ba.Requests = nil - delRngArgs := roachpb.DeleteRangeRequest{RequestHeader: roachpb.RequestHeader{Key: keyA, EndKey: keyB}} + delRngArgs := kvpb.DeleteRangeRequest{RequestHeader: kvpb.RequestHeader{Key: keyA, EndKey: keyB}} delRngArgs.Sequence = 2 etArgsWithRangedIntentSpan := etArgs etArgsWithRangedIntentSpan.LockSpans = []roachpb.Span{{Key: keyA, EndKey: keyB}} etArgsWithRangedIntentSpan.InFlightWrites = []roachpb.SequencedWrite{{Key: keyA, Sequence: 1}} ba.Add(&delRngArgs, &qiArgs, &etArgsWithRangedIntentSpan) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 3) - require.IsType(t, &roachpb.EndTxnRequest{}, ba.Requests[2].GetInner()) + require.IsType(t, &kvpb.EndTxnRequest{}, ba.Requests[2].GetInner()) - et := ba.Requests[2].GetInner().(*roachpb.EndTxnRequest) + et := ba.Requests[2].GetInner().(*kvpb.EndTxnRequest) require.True(t, et.Commit) require.Len(t, et.LockSpans, 1) require.Equal(t, []roachpb.Span{{Key: keyA, EndKey: keyB}}, et.LockSpans) @@ -302,16 +303,16 @@ func TestTxnCommitterStripsInFlightWrites(t *testing.T) { // In-flight writes should not be attached because read-only requests // cannot be parallelized with a commit. ba.Requests = nil - getArgs := roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}} + getArgs := kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}} getArgs.Sequence = 2 etArgsCopy = etArgs ba.Add(&getArgs, &qiArgs, &etArgsCopy) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 3) - require.IsType(t, &roachpb.EndTxnRequest{}, ba.Requests[2].GetInner()) + require.IsType(t, &kvpb.EndTxnRequest{}, ba.Requests[2].GetInner()) - et := ba.Requests[2].GetInner().(*roachpb.EndTxnRequest) + et := ba.Requests[2].GetInner().(*kvpb.EndTxnRequest) require.True(t, et.Commit) require.Len(t, et.LockSpans, 2) require.Equal(t, []roachpb.Span{{Key: keyA}, {Key: keyB}}, et.LockSpans) @@ -343,10 +344,10 @@ func TestTxnCommitterAsyncExplicitCommitTask(t *testing.T) { // Verify that the Put is attached as in-flight write to the committing // EndTxn request. - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - putArgs := roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}} - etArgs := roachpb.EndTxnRequest{Commit: true} + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + putArgs := kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}} + etArgs := kvpb.EndTxnRequest{Commit: true} putArgs.Sequence = 1 etArgs.Sequence = 2 etArgs.InFlightWrites = []roachpb.SequencedWrite{{Key: keyA, Sequence: 1}} @@ -357,13 +358,13 @@ func TestTxnCommitterAsyncExplicitCommitTask(t *testing.T) { ba.Header.CanForwardReadTimestamp = true explicitCommitCh := make(chan struct{}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 2) require.True(t, ba.CanForwardReadTimestamp) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.EndTxnRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.EndTxnRequest{}, ba.Requests[1].GetInner()) - et := ba.Requests[1].GetInner().(*roachpb.EndTxnRequest) + et := ba.Requests[1].GetInner().(*kvpb.EndTxnRequest) require.True(t, et.Commit) require.Len(t, et.InFlightWrites, 1) require.Equal(t, roachpb.SequencedWrite{Key: keyA, Sequence: 1}, et.InFlightWrites[0]) @@ -371,17 +372,17 @@ func TestTxnCommitterAsyncExplicitCommitTask(t *testing.T) { br := ba.CreateReply() br.Txn = ba.Txn br.Txn.Status = roachpb.STAGING - br.Responses[1].GetInner().(*roachpb.EndTxnResponse).StagingTimestamp = br.Txn.WriteTimestamp + br.Responses[1].GetInner().(*kvpb.EndTxnResponse).StagingTimestamp = br.Txn.WriteTimestamp // Before returning, mock out the sender again to test against the async // task that should be sent to make the implicit txn commit explicit. - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { defer close(explicitCommitCh) require.Len(t, ba.Requests, 1) require.False(t, ba.CanForwardReadTimestamp) - require.IsType(t, &roachpb.EndTxnRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.EndTxnRequest{}, ba.Requests[0].GetInner()) - et := ba.Requests[0].GetInner().(*roachpb.EndTxnRequest) + et := ba.Requests[0].GetInner().(*kvpb.EndTxnRequest) require.True(t, et.Commit) require.Len(t, et.InFlightWrites, 0) @@ -415,21 +416,21 @@ func TestTxnCommitterRetryAfterStaging(t *testing.T) { txn := makeTxnProto() keyA := roachpb.Key("a") - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - putArgs := roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}} - etArgs := roachpb.EndTxnRequest{Commit: true} + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + putArgs := kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}} + etArgs := kvpb.EndTxnRequest{Commit: true} putArgs.Sequence = 1 etArgs.Sequence = 2 etArgs.InFlightWrites = []roachpb.SequencedWrite{{Key: keyA, Sequence: 1}} ba.Add(&putArgs, &etArgs) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 2) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.EndTxnRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.EndTxnRequest{}, ba.Requests[1].GetInner()) - et := ba.Requests[1].GetInner().(*roachpb.EndTxnRequest) + et := ba.Requests[1].GetInner().(*kvpb.EndTxnRequest) require.True(t, et.Commit) require.Len(t, et.InFlightWrites, 1) require.Equal(t, roachpb.SequencedWrite{Key: keyA, Sequence: 1}, et.InFlightWrites[0]) @@ -437,7 +438,7 @@ func TestTxnCommitterRetryAfterStaging(t *testing.T) { br := ba.CreateReply() br.Txn = ba.Txn br.Txn.Status = roachpb.STAGING - br.Responses[1].GetInner().(*roachpb.EndTxnResponse).StagingTimestamp = br.Txn.WriteTimestamp + br.Responses[1].GetInner().(*kvpb.EndTxnResponse).StagingTimestamp = br.Txn.WriteTimestamp // Pretend the PutRequest was split and sent to a different Range. It // could hit the timestamp cache, or a WriteTooOld error (which sets the @@ -451,12 +452,12 @@ func TestTxnCommitterRetryAfterStaging(t *testing.T) { br, pErr := tc.SendLocked(ctx, ba) require.Nil(t, br) require.NotNil(t, pErr) - require.IsType(t, &roachpb.TransactionRetryError{}, pErr.GetDetail()) - expReason := roachpb.RETRY_SERIALIZABLE + require.IsType(t, &kvpb.TransactionRetryError{}, pErr.GetDetail()) + expReason := kvpb.RETRY_SERIALIZABLE if writeTooOld { - expReason = roachpb.RETRY_WRITE_TOO_OLD + expReason = kvpb.RETRY_WRITE_TOO_OLD } - require.Equal(t, expReason, pErr.GetDetail().(*roachpb.TransactionRetryError).Reason) + require.Equal(t, expReason, pErr.GetDetail().(*kvpb.TransactionRetryError).Reason) }) } @@ -473,10 +474,10 @@ func TestTxnCommitterNoParallelCommitsOnRetry(t *testing.T) { txn := makeTxnProto() keyA := roachpb.Key("a") - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - putArgs := roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}} - etArgs := roachpb.EndTxnRequest{Commit: true} + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + putArgs := kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}} + etArgs := kvpb.EndTxnRequest{Commit: true} putArgs.Sequence = 1 etArgs.Sequence = 2 etArgs.InFlightWrites = []roachpb.SequencedWrite{{Key: keyA, Sequence: 1}} @@ -487,12 +488,12 @@ func TestTxnCommitterNoParallelCommitsOnRetry(t *testing.T) { ba.Add(&putArgs, &etArgs) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 2) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.EndTxnRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.EndTxnRequest{}, ba.Requests[1].GetInner()) - et := ba.Requests[1].GetInner().(*roachpb.EndTxnRequest) + et := ba.Requests[1].GetInner().(*kvpb.EndTxnRequest) require.True(t, et.Commit) require.Len(t, et.InFlightWrites, 0, "expected parallel commit to be inhibited") diff --git a/pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater.go b/pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater.go index 2aa47c1bb46d..c7a8cbf724c6 100644 --- a/pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater.go +++ b/pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater.go @@ -16,6 +16,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/kv/kvbase" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/txnwait" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/contextutil" @@ -154,8 +155,8 @@ type txnHeartbeater struct { } type abortTxnAsyncResult struct { - br *roachpb.BatchResponse - pErr *roachpb.Error + br *kvpb.BatchResponse + pErr *kvpb.Error } // init initializes the txnHeartbeater. This method exists instead of a @@ -182,9 +183,9 @@ func (h *txnHeartbeater) init( // SendLocked is part of the txnInterceptor interface. func (h *txnHeartbeater) SendLocked( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, *roachpb.Error) { - etArg, hasET := ba.GetArg(roachpb.EndTxn) + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, *kvpb.Error) { + etArg, hasET := ba.GetArg(kvpb.EndTxn) firstLockingIndex, pErr := firstLockingIndex(ba) if pErr != nil { return nil, pErr @@ -208,7 +209,7 @@ func (h *txnHeartbeater) SendLocked( } if hasET { - et := etArg.(*roachpb.EndTxnRequest) + et := etArg.(*kvpb.EndTxnRequest) // Preemptively stop the heartbeat loop in case of transaction abort. // In case of transaction commit we don't want to do this because commit @@ -232,7 +233,7 @@ func (h *txnHeartbeater) SendLocked( case res := <-resultC: return res.br, res.pErr case <-ctx.Done(): - return nil, roachpb.NewError(ctx.Err()) + return nil, kvpb.NewError(ctx.Err()) } } } @@ -431,10 +432,10 @@ func (h *txnHeartbeater) heartbeatLocked(ctx context.Context) bool { if txn.Key == nil { log.Fatalf(ctx, "attempting to heartbeat txn without anchor key: %v", txn) } - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Txn = txn - ba.Add(&roachpb.HeartbeatTxnRequest{ - RequestHeader: roachpb.RequestHeader{ + ba.Add(&kvpb.HeartbeatTxnRequest{ + RequestHeader: kvpb.RequestHeader{ Key: txn.Key, }, Now: h.clock.Now(), @@ -460,7 +461,7 @@ func (h *txnHeartbeater) heartbeatLocked(ctx context.Context) bool { // // TODO(nvanbenschoten): Make this the only case where we get back an // Aborted txn. - if _, ok := pErr.GetDetail().(*roachpb.TransactionAbortedError); ok { + if _, ok := pErr.GetDetail().(*kvpb.TransactionAbortedError); ok { // Note that it's possible that the txn actually committed but its // record got GC'ed. In that case, aborting won't hurt anyone though, // since all intents have already been resolved. @@ -514,9 +515,9 @@ func (h *txnHeartbeater) abortTxnAsyncLocked(ctx context.Context) { // Construct a batch with an EndTxn request. txn := h.mu.txn.Clone() - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: txn} - ba.Add(&roachpb.EndTxnRequest{ + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: txn} + ba.Add(&kvpb.EndTxnRequest{ Commit: false, // Resolved intents should maintain an abort span entry to prevent // concurrent requests from failing to notice the transaction was aborted. @@ -592,15 +593,15 @@ func (h *txnHeartbeater) abortTxnAsyncLocked(ctx context.Context) { // in the BatchRequest. Returns -1 if the batch has no intention to acquire // locks. It also verifies that if an EndTxnRequest is included, then it is the // last request in the batch. -func firstLockingIndex(ba *roachpb.BatchRequest) (int, *roachpb.Error) { +func firstLockingIndex(ba *kvpb.BatchRequest) (int, *kvpb.Error) { for i, ru := range ba.Requests { args := ru.GetInner() if i < len(ba.Requests)-1 /* if not last*/ { - if _, ok := args.(*roachpb.EndTxnRequest); ok { - return -1, roachpb.NewErrorf("%s sent as non-terminal call", args.Method()) + if _, ok := args.(*kvpb.EndTxnRequest); ok { + return -1, kvpb.NewErrorf("%s sent as non-terminal call", args.Method()) } } - if roachpb.IsLocking(args) { + if kvpb.IsLocking(args) { return i, nil } } diff --git a/pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater_test.go b/pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater_test.go index 87ae841c8751..880167be07a9 100644 --- a/pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater_test.go +++ b/pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater_test.go @@ -17,6 +17,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/txnwait" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -74,12 +75,12 @@ func TestTxnHeartbeaterSetsTransactionKey(t *testing.T) { // No key is set on a read-only batch. keyA, keyB := roachpb.Key("a"), roachpb.Key("b") - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: txn.Clone()} - ba.Add(&roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: keyB}}) + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: txn.Clone()} + ba.Add(&kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{Key: keyB}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 2) require.Equal(t, keyA, ba.Requests[0].GetInner().Header().Key) require.Equal(t, keyB, ba.Requests[1].GetInner().Header().Key) @@ -99,10 +100,10 @@ func TestTxnHeartbeaterSetsTransactionKey(t *testing.T) { // The key of the first write is set as the transaction key. ba.Requests = nil - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyB}}) - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyB}}) + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 2) require.Equal(t, keyB, ba.Requests[0].GetInner().Header().Key) require.Equal(t, keyA, ba.Requests[1].GetInner().Header().Key) @@ -122,9 +123,9 @@ func TestTxnHeartbeaterSetsTransactionKey(t *testing.T) { // The transaction key is not changed on subsequent batches. ba.Requests = nil - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) require.Equal(t, keyA, ba.Requests[0].GetInner().Header().Key) @@ -156,10 +157,10 @@ func TestTxnHeartbeaterLoopStartedOnFirstLock(t *testing.T) { // Read-only requests don't start the heartbeat loop. keyA := roachpb.Key("a") - keyAHeader := roachpb.RequestHeader{Key: keyA} - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: txn.Clone()} - ba.Add(&roachpb.GetRequest{RequestHeader: keyAHeader}) + keyAHeader := kvpb.RequestHeader{Key: keyA} + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: txn.Clone()} + ba.Add(&kvpb.GetRequest{RequestHeader: keyAHeader}) br, pErr := th.SendLocked(ctx, ba) require.Nil(t, pErr) @@ -173,9 +174,9 @@ func TestTxnHeartbeaterLoopStartedOnFirstLock(t *testing.T) { // The heartbeat loop is started on the first locking request. ba.Requests = nil if write { - ba.Add(&roachpb.PutRequest{RequestHeader: keyAHeader}) + ba.Add(&kvpb.PutRequest{RequestHeader: keyAHeader}) } else { - ba.Add(&roachpb.ScanRequest{RequestHeader: keyAHeader, KeyLocking: lock.Exclusive}) + ba.Add(&kvpb.ScanRequest{RequestHeader: keyAHeader, KeyLocking: lock.Exclusive}) } br, pErr = th.SendLocked(ctx, ba) @@ -190,11 +191,11 @@ func TestTxnHeartbeaterLoopStartedOnFirstLock(t *testing.T) { // The interceptor indicates whether the heartbeat loop is // running on EndTxn requests. ba.Requests = nil - ba.Add(&roachpb.EndTxnRequest{Commit: true}) + ba.Add(&kvpb.EndTxnRequest{Commit: true}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) - require.IsType(t, &roachpb.EndTxnRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.EndTxnRequest{}, ba.Requests[0].GetInner()) br = ba.CreateReply() br.Txn = ba.Txn @@ -313,17 +314,17 @@ func TestTxnHeartbeaterLoopStartsBeforeExpiry(t *testing.T) { th.mu.Unlock() count := 0 - mockGatekeeper.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockGatekeeper.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) - require.IsType(t, &roachpb.HeartbeatTxnRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.HeartbeatTxnRequest{}, ba.Requests[0].GetInner()) - hbReq := ba.Requests[0].GetInner().(*roachpb.HeartbeatTxnRequest) + hbReq := ba.Requests[0].GetInner().(*kvpb.HeartbeatTxnRequest) require.Equal(t, &txn, ba.Txn) require.Equal(t, roachpb.Key(txn.Key), hbReq.Key) // Check that this transaction isn't already considered expired. if !test.consideredExpired && txnwait.IsExpired(clock.Now(), ba.Txn) { - return nil, roachpb.NewError(errors.New("transaction expired before heartbeat")) + return nil, kvpb.NewError(errors.New("transaction expired before heartbeat")) } log.Infof(ctx, "received heartbeat request") @@ -339,11 +340,11 @@ func TestTxnHeartbeaterLoopStartsBeforeExpiry(t *testing.T) { // The heartbeat loop is started on the first locking request, in this case // a GetForUpdate request. - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: txn.Clone()} + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: txn.Clone()} keyA := roachpb.Key("a") - keyAHeader := roachpb.RequestHeader{Key: keyA} - ba.Add(&roachpb.GetRequest{RequestHeader: keyAHeader, KeyLocking: lock.Exclusive}) + keyAHeader := kvpb.RequestHeader{Key: keyA} + ba.Add(&kvpb.GetRequest{RequestHeader: keyAHeader, KeyLocking: lock.Exclusive}) br, pErr := th.SendLocked(ctx, ba) require.Nil(t, pErr) @@ -404,15 +405,15 @@ func TestTxnHeartbeaterLoopStartedFor1PC(t *testing.T) { defer th.stopper.Stop(ctx) keyA := roachpb.Key("a") - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: txn.Clone()} - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.EndTxnRequest{Commit: true}) + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: txn.Clone()} + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.EndTxnRequest{Commit: true}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 2) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.EndTxnRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.EndTxnRequest{}, ba.Requests[1].GetInner()) br := ba.CreateReply() br.Txn = ba.Txn @@ -447,11 +448,11 @@ func TestTxnHeartbeaterLoopRequests(t *testing.T) { var count int var lastTime hlc.Timestamp - mockGatekeeper.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockGatekeeper.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) - require.IsType(t, &roachpb.HeartbeatTxnRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.HeartbeatTxnRequest{}, ba.Requests[0].GetInner()) - hbReq := ba.Requests[0].GetInner().(*roachpb.HeartbeatTxnRequest) + hbReq := ba.Requests[0].GetInner().(*kvpb.HeartbeatTxnRequest) require.Equal(t, &txn, ba.Txn) require.Equal(t, roachpb.Key(txn.Key), hbReq.Key) require.True(t, lastTime.Less(hbReq.Now)) @@ -466,9 +467,9 @@ func TestTxnHeartbeaterLoopRequests(t *testing.T) { // Kick off the heartbeat loop. keyA := roachpb.Key("a") - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: txn.Clone()} - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: txn.Clone()} + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) br, pErr := th.SendLocked(ctx, ba) require.Nil(t, pErr) @@ -489,9 +490,9 @@ func TestTxnHeartbeaterLoopRequests(t *testing.T) { // Mark the coordinator's transaction record as COMMITTED while a heartbeat // is in-flight. This should cause the heartbeat loop to shut down. th.mu.Lock() - mockGatekeeper.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockGatekeeper.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) - require.IsType(t, &roachpb.HeartbeatTxnRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.HeartbeatTxnRequest{}, ba.Requests[0].GetInner()) br := ba.CreateReply() br.Txn = ba.Txn @@ -531,16 +532,16 @@ func TestTxnHeartbeaterAsyncAbort(t *testing.T) { defer th.stopper.Stop(ctx) putDone, asyncAbortDone := make(chan struct{}), make(chan struct{}) - mockGatekeeper.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockGatekeeper.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { // Wait for the Put to finish to avoid a data race. <-putDone require.Len(t, ba.Requests, 1) - require.IsType(t, &roachpb.HeartbeatTxnRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.HeartbeatTxnRequest{}, ba.Requests[0].GetInner()) if abortedErr { - return nil, roachpb.NewErrorWithTxn( - roachpb.NewTransactionAbortedError(roachpb.ABORT_REASON_UNKNOWN), ba.Txn, + return nil, kvpb.NewErrorWithTxn( + kvpb.NewTransactionAbortedError(kvpb.ABORT_REASON_UNKNOWN), ba.Txn, ) } br := ba.CreateReply() @@ -551,21 +552,21 @@ func TestTxnHeartbeaterAsyncAbort(t *testing.T) { // Kick off the heartbeat loop. keyA := roachpb.Key("a") - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: txn.Clone()} - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: txn.Clone()} + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) br, pErr := th.SendLocked(ctx, ba) require.Nil(t, pErr) require.NotNil(t, br) // Test that the transaction is rolled back. - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { defer close(asyncAbortDone) require.Len(t, ba.Requests, 1) - require.IsType(t, &roachpb.EndTxnRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.EndTxnRequest{}, ba.Requests[0].GetInner()) - etReq := ba.Requests[0].GetInner().(*roachpb.EndTxnRequest) + etReq := ba.Requests[0].GetInner().(*kvpb.EndTxnRequest) require.Equal(t, &txn, ba.Txn) require.Nil(t, etReq.Key) // set in txnCommitter require.False(t, etReq.Commit) @@ -606,12 +607,12 @@ func TestTxnHeartbeaterAsyncAbortWaitsForInFlight(t *testing.T) { // putReady then return an aborted txn and signal hbAborted. putReady := make(chan struct{}) hbAborted := make(chan struct{}) - mockGatekeeper.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockGatekeeper.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { <-putReady defer close(hbAborted) require.Len(t, ba.Requests, 1) - require.IsType(t, &roachpb.HeartbeatTxnRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.HeartbeatTxnRequest{}, ba.Requests[0].GetInner()) br := ba.CreateReply() br.Txn = ba.Txn @@ -624,12 +625,12 @@ func TestTxnHeartbeaterAsyncAbortWaitsForInFlight(t *testing.T) { mockSender.ChainMockSend( // Mock a Put, which signals putReady and then waits for putResume // before returning a response. - func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { th.mu.Unlock() // without txnLockGatekeeper, we must unlock manually defer th.mu.Lock() close(putReady) require.Len(t, ba.Requests, 1) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[0].GetInner()) <-putResume @@ -638,12 +639,12 @@ func TestTxnHeartbeaterAsyncAbortWaitsForInFlight(t *testing.T) { return br, nil }, // Mock an EndTxn, which signals rollbackSent. - func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { defer close(rollbackSent) require.Len(t, ba.Requests, 1) - require.IsType(t, &roachpb.EndTxnRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.EndTxnRequest{}, ba.Requests[0].GetInner()) - etReq := ba.Requests[0].GetInner().(*roachpb.EndTxnRequest) + etReq := ba.Requests[0].GetInner().(*kvpb.EndTxnRequest) require.Equal(t, &txn, ba.Txn) require.False(t, etReq.Commit) require.True(t, etReq.Poison) @@ -657,9 +658,9 @@ func TestTxnHeartbeaterAsyncAbortWaitsForInFlight(t *testing.T) { // Spawn a goroutine to send the Put. require.NoError(t, th.stopper.RunAsyncTask(ctx, "put", func(ctx context.Context) { - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: txn.Clone()} - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: roachpb.Key("a")}}) + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: txn.Clone()} + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: roachpb.Key("a")}}) th.mu.Lock() // without TxnCoordSender, we must lock manually defer th.mu.Unlock() @@ -696,11 +697,11 @@ func TestTxnHeartbeaterAsyncAbortCollapsesRequests(t *testing.T) { // Mock the heartbeat request, which simply aborts and signals hbAborted. hbAborted := make(chan struct{}) - mockGatekeeper.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockGatekeeper.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { defer close(hbAborted) require.Len(t, ba.Requests, 1) - require.IsType(t, &roachpb.HeartbeatTxnRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.HeartbeatTxnRequest{}, ba.Requests[0].GetInner()) br := ba.CreateReply() br.Txn = ba.Txn @@ -714,25 +715,25 @@ func TestTxnHeartbeaterAsyncAbortCollapsesRequests(t *testing.T) { rollbackUnblock := make(chan struct{}) mockSender.ChainMockSend( // The first Put request is expected and should just return. - func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[0].GetInner()) br := ba.CreateReply() br.Txn = ba.Txn return br, nil }, // The first EndTxn request from the heartbeater is expected, so block and return. - func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { th.mu.Unlock() // manually unlock for concurrency, no txnLockGatekeeper defer th.mu.Lock() close(rollbackReady) require.Len(t, ba.Requests, 1) - require.IsType(t, &roachpb.EndTxnRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.EndTxnRequest{}, ba.Requests[0].GetInner()) <-rollbackUnblock - etReq := ba.Requests[0].GetInner().(*roachpb.EndTxnRequest) + etReq := ba.Requests[0].GetInner().(*kvpb.EndTxnRequest) require.Equal(t, &txn, ba.Txn) require.False(t, etReq.Commit) require.True(t, etReq.Poison) @@ -744,15 +745,15 @@ func TestTxnHeartbeaterAsyncAbortCollapsesRequests(t *testing.T) { }, // The second EndTxn request from the client is unexpected, so // return an error response. - func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { - return nil, roachpb.NewError(errors.Errorf("unexpected request: %v", ba)) + func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { + return nil, kvpb.NewError(errors.Errorf("unexpected request: %v", ba)) }, ) // Kick off the heartbeat loop. - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: txn.Clone()} - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: roachpb.Key("a")}}) + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: txn.Clone()} + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: roachpb.Key("a")}}) th.mu.Lock() // manually lock, there's no TxnCoordSender br, pErr := th.SendLocked(ctx, ba) @@ -772,9 +773,9 @@ func TestTxnHeartbeaterAsyncAbortCollapsesRequests(t *testing.T) { close(rollbackUnblock) })) - ba = &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: txn.Clone()} - ba.Add(&roachpb.EndTxnRequest{Commit: false}) + ba = &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: txn.Clone()} + ba.Add(&kvpb.EndTxnRequest{Commit: false}) th.mu.Lock() // manually lock, there's no TxnCoordSender br, pErr = th.SendLocked(ctx, ba) @@ -814,9 +815,9 @@ func TestTxnHeartbeaterEndTxnLoopHandling(t *testing.T) { // Kick off the heartbeat loop. key := roachpb.Key("a") - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: txn.Clone()} - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: key}}) + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: txn.Clone()} + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: key}}) br, pErr := th.SendLocked(ctx, ba) require.Nil(t, pErr) @@ -824,9 +825,9 @@ func TestTxnHeartbeaterEndTxnLoopHandling(t *testing.T) { require.True(t, heartbeaterRunning(&th), "heartbeat running") // End transaction to validate heartbeat state. - ba2 := &roachpb.BatchRequest{} - ba2.Header = roachpb.Header{Txn: txn.Clone()} - ba2.Add(&roachpb.EndTxnRequest{RequestHeader: roachpb.RequestHeader{Key: key}, Commit: tc.transactionCommit}) + ba2 := &kvpb.BatchRequest{} + ba2.Header = kvpb.Header{Txn: txn.Clone()} + ba2.Add(&kvpb.EndTxnRequest{RequestHeader: kvpb.RequestHeader{Key: key}, Commit: tc.transactionCommit}) th.mu.Lock() br, pErr = th.SendLocked(ctx, ba2) diff --git a/pkg/kv/kvclient/kvcoord/txn_interceptor_metric_recorder.go b/pkg/kv/kvclient/kvcoord/txn_interceptor_metric_recorder.go index ee2ab945c7ed..10f5d7f4ccc0 100644 --- a/pkg/kv/kvclient/kvcoord/txn_interceptor_metric_recorder.go +++ b/pkg/kv/kvclient/kvcoord/txn_interceptor_metric_recorder.go @@ -13,6 +13,7 @@ package kvcoord import ( "context" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -37,8 +38,8 @@ type txnMetricRecorder struct { // SendLocked is part of the txnInterceptor interface. func (m *txnMetricRecorder) SendLocked( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, *kvpb.Error) { if m.txnStartNanos == 0 { m.txnStartNanos = timeutil.Now().UnixNano() } diff --git a/pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go b/pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go index edd2ad6a073b..8515a555be94 100644 --- a/pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go +++ b/pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go @@ -14,6 +14,7 @@ import ( "context" "sort" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" @@ -258,8 +259,8 @@ func (f rangeIteratorFactory) newRangeIterator() condensableSpanSetRangeIterator // SendLocked implements the lockedSender interface. func (tp *txnPipeliner) SendLocked( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, *kvpb.Error) { // If an EndTxn request is part of this batch, attach the in-flight writes // and the lock footprint to it. ba, pErr := tp.attachLocksToEndTxn(ctx, ba) @@ -277,7 +278,7 @@ func (tp *txnPipeliner) SendLocked( maxBytes := TrackedWritesMaxSize.Get(&tp.st.SV) if rejectOverBudget { if err := tp.maybeRejectOverBudget(ba, maxBytes); err != nil { - return nil, roachpb.NewError(err) + return nil, kvpb.NewError(err) } } @@ -321,7 +322,7 @@ func (tp *txnPipeliner) SendLocked( // the transaction commits. If it fails, then we'd add the lock spans to our // tracking and exceed the budget. It's easier for this code and more // predictable for the user if we just reject this batch, though. -func (tp *txnPipeliner) maybeRejectOverBudget(ba *roachpb.BatchRequest, maxBytes int64) error { +func (tp *txnPipeliner) maybeRejectOverBudget(ba *kvpb.BatchRequest, maxBytes int64) error { // Bail early if the current request is not locking, even if we are already // over budget. In particular, we definitely want to permit rollbacks. We also // want to permit lone commits, since the damage in taking too much memory has @@ -354,18 +355,18 @@ func (tp *txnPipeliner) maybeRejectOverBudget(ba *roachpb.BatchRequest, maxBytes // provided batch. It augments these sets with locking requests from the current // batch. func (tp *txnPipeliner) attachLocksToEndTxn( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchRequest, *roachpb.Error) { - args, hasET := ba.GetArg(roachpb.EndTxn) + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchRequest, *kvpb.Error) { + args, hasET := ba.GetArg(kvpb.EndTxn) if !hasET { return ba, nil } - et := args.(*roachpb.EndTxnRequest) + et := args.(*kvpb.EndTxnRequest) if len(et.LockSpans) > 0 { - return ba, roachpb.NewErrorf("client must not pass intents to EndTxn") + return ba, kvpb.NewErrorf("client must not pass intents to EndTxn") } if len(et.InFlightWrites) > 0 { - return ba, roachpb.NewErrorf("client must not pass in-flight writes to EndTxn") + return ba, kvpb.NewErrorf("client must not pass in-flight writes to EndTxn") } // Populate et.LockSpans and et.InFlightWrites. @@ -384,7 +385,7 @@ func (tp *txnPipeliner) attachLocksToEndTxn( for _, ru := range ba.Requests[:len(ba.Requests)-1] { req := ru.GetInner() h := req.Header() - if roachpb.IsLocking(req) { + if kvpb.IsLocking(req) { // Ranged writes are added immediately to the lock spans because // it's not clear where they will actually leave intents. Point // writes are added to the in-flight writes set. All other locking @@ -394,7 +395,7 @@ func (tp *txnPipeliner) attachLocksToEndTxn( // will fold the in-flight writes into the lock spans immediately // and forgo a parallel commit, but let's not break that abstraction // boundary here. - if roachpb.IsIntentWrite(req) && !roachpb.IsRange(req) { + if kvpb.IsIntentWrite(req) && !kvpb.IsRange(req) { w := roachpb.SequencedWrite{Key: h.Key, Sequence: h.Sequence} et.InFlightWrites = append(et.InFlightWrites, w) } else { @@ -420,13 +421,13 @@ func (tp *txnPipeliner) attachLocksToEndTxn( // canUseAsyncConsensus checks the conditions necessary for this batch to be // allowed to set the AsyncConsensus flag. -func (tp *txnPipeliner) canUseAsyncConsensus(ctx context.Context, ba *roachpb.BatchRequest) bool { +func (tp *txnPipeliner) canUseAsyncConsensus(ctx context.Context, ba *kvpb.BatchRequest) bool { // Short-circuit for EndTransactions; it's common enough to have batches // containing a prefix of writes (which, by themselves, are all eligible for // async consensus) and then an EndTxn (which is not eligible). Note that // ba.GetArg() is efficient for EndTransactions, having its own internal // optimization. - if _, hasET := ba.GetArg(roachpb.EndTxn); hasET { + if _, hasET := ba.GetArg(kvpb.EndTxn); hasET { return false } @@ -456,7 +457,7 @@ func (tp *txnPipeliner) canUseAsyncConsensus(ctx context.Context, ba *roachpb.Ba // Determine whether the current request prevents us from performing async // consensus on the batch. - if !roachpb.IsIntentWrite(req) || roachpb.IsRange(req) { + if !kvpb.IsIntentWrite(req) || kvpb.IsRange(req) { // Only allow batches consisting of solely transactional point // writes to perform consensus asynchronously. // TODO(nvanbenschoten): We could allow batches with reads and point @@ -493,7 +494,7 @@ func (tp *txnPipeliner) canUseAsyncConsensus(ctx context.Context, ba *roachpb.Ba // a write succeeded before depending on its existence. We later prune down the // list of writes we proved to exist that are no longer "in-flight" in // updateLockTracking. -func (tp *txnPipeliner) chainToInFlightWrites(ba *roachpb.BatchRequest) *roachpb.BatchRequest { +func (tp *txnPipeliner) chainToInFlightWrites(ba *kvpb.BatchRequest) *kvpb.BatchRequest { // If there are no in-flight writes, there's nothing to chain to. if tp.ifWrites.len() == 0 { return ba @@ -519,7 +520,7 @@ func (tp *txnPipeliner) chainToInFlightWrites(ba *roachpb.BatchRequest) *roachpb // so fork it before modifying it. if !forked { ba = ba.ShallowCopy() - ba.Requests = append([]roachpb.RequestUnion(nil), ba.Requests[:i]...) + ba.Requests = append([]kvpb.RequestUnion(nil), ba.Requests[:i]...) forked = true } @@ -530,8 +531,8 @@ func (tp *txnPipeliner) chainToInFlightWrites(ba *roachpb.BatchRequest) *roachpb // chain on to the success of the in-flight write. meta := ba.Txn.TxnMeta meta.Sequence = w.Sequence - ba.Add(&roachpb.QueryIntentRequest{ - RequestHeader: roachpb.RequestHeader{ + ba.Add(&kvpb.QueryIntentRequest{ + RequestHeader: kvpb.RequestHeader{ Key: w.Key, }, Txn: meta, @@ -547,13 +548,13 @@ func (tp *txnPipeliner) chainToInFlightWrites(ba *roachpb.BatchRequest) *roachpb } } - if !roachpb.IsTransactional(req) { + if !kvpb.IsTransactional(req) { // Non-transactional requests require that we stall the entire // pipeline by chaining on to all in-flight writes. This is // because their request header is often insufficient to // determine all of the keys that they will interact with. tp.ifWrites.ascend(writeIter) - } else if et, ok := req.(*roachpb.EndTxnRequest); ok { + } else if et, ok := req.(*kvpb.EndTxnRequest); ok { if et.Commit { // EndTxns need to prove all in-flight writes before being // allowed to succeed themselves. @@ -598,9 +599,9 @@ func (tp *txnPipeliner) chainToInFlightWrites(ba *roachpb.BatchRequest) *roachpb // transaction cleans up. func (tp *txnPipeliner) updateLockTracking( ctx context.Context, - ba *roachpb.BatchRequest, - br *roachpb.BatchResponse, - pErr *roachpb.Error, + ba *kvpb.BatchRequest, + br *kvpb.BatchResponse, + pErr *kvpb.Error, maxBytes int64, condenseLocksIfOverBudget bool, ) { @@ -642,7 +643,7 @@ func (tp *txnPipeliner) updateLockTracking( } func (tp *txnPipeliner) updateLockTrackingInner( - ctx context.Context, ba *roachpb.BatchRequest, br *roachpb.BatchResponse, pErr *roachpb.Error, + ctx context.Context, ba *kvpb.BatchRequest, br *kvpb.BatchResponse, pErr *kvpb.Error, ) { // If the request failed, add all lock acquisitions attempts directly to the // lock footprint. This reduces the likelihood of dangling locks blocking @@ -660,8 +661,8 @@ func (tp *txnPipeliner) updateLockTrackingInner( // these transactions from adding even more load to the contended key by // trying to perform unnecessary intent resolution. baStripped := *ba - if roachpb.ErrPriority(pErr.GoError()) <= roachpb.ErrorScoreUnambiguousError && pErr.Index != nil { - baStripped.Requests = make([]roachpb.RequestUnion, len(ba.Requests)-1) + if kvpb.ErrPriority(pErr.GoError()) <= kvpb.ErrorScoreUnambiguousError && pErr.Index != nil { + baStripped.Requests = make([]kvpb.RequestUnion, len(ba.Requests)-1) copy(baStripped.Requests, ba.Requests[:pErr.Index.Index]) copy(baStripped.Requests[pErr.Index.Index:], ba.Requests[pErr.Index.Index+1:]) } @@ -697,18 +698,18 @@ func (tp *txnPipeliner) updateLockTrackingInner( req := ru.GetInner() resp := br.Responses[i].GetInner() - if qiReq, ok := req.(*roachpb.QueryIntentRequest); ok { + if qiReq, ok := req.(*kvpb.QueryIntentRequest); ok { // Remove any in-flight writes that were proven to exist. // It shouldn't be possible for a QueryIntentRequest with // the ErrorIfMissing option set to return without error // and with with FoundIntent=false, but we handle that // case here because it happens a lot in tests. - if resp.(*roachpb.QueryIntentResponse).FoundIntent { + if resp.(*kvpb.QueryIntentResponse).FoundIntent { tp.ifWrites.remove(qiReq.Key, qiReq.Txn.Sequence) // Move to lock footprint. tp.lockFootprint.insert(roachpb.Span{Key: qiReq.Key}) } - } else if roachpb.IsLocking(req) { + } else if kvpb.IsLocking(req) { // If the request intended to acquire locks, track its lock spans. if ba.AsyncConsensus { // Record any writes that were performed asynchronously. We'll @@ -718,7 +719,7 @@ func (tp *txnPipeliner) updateLockTrackingInner( // The request is not expected to be a ranged one, as we're only // tracking one key in the ifWrites. Ranged requests do not admit // ba.AsyncConsensus. - if roachpb.IsRange(req) { + if kvpb.IsRange(req) { log.Fatalf(ctx, "unexpected range request with AsyncConsensus: %s", req) } } else { @@ -726,7 +727,7 @@ func (tp *txnPipeliner) updateLockTrackingInner( // then add them directly to our lock footprint. Locking read // requests will always hit this path because they will never // use async consensus. - if sp, ok := roachpb.ActualSpan(req, resp); ok { + if sp, ok := kvpb.ActualSpan(req, resp); ok { tp.lockFootprint.insert(sp) } } @@ -741,7 +742,7 @@ func (tp *txnPipeliner) trackLocks(s roachpb.Span, _ lock.Durability) { // stripQueryIntents adjusts the BatchResponse to hide the fact that this // interceptor added new requests to the batch. It returns an adjusted batch // response without the responses that correspond to these added requests. -func (tp *txnPipeliner) stripQueryIntents(br *roachpb.BatchResponse) *roachpb.BatchResponse { +func (tp *txnPipeliner) stripQueryIntents(br *kvpb.BatchResponse) *kvpb.BatchResponse { j := 0 for i, ru := range br.Responses { if ru.GetQueryIntent() != nil { @@ -760,14 +761,14 @@ func (tp *txnPipeliner) stripQueryIntents(br *roachpb.BatchResponse) *roachpb.Ba // It transforms any IntentMissingError into a TransactionRetryError and fixes // the error's index position. func (tp *txnPipeliner) adjustError( - ctx context.Context, ba *roachpb.BatchRequest, pErr *roachpb.Error, -) *roachpb.Error { + ctx context.Context, ba *kvpb.BatchRequest, pErr *kvpb.Error, +) *kvpb.Error { // Fix the error index to hide the impact of any QueryIntent requests. if pErr.Index != nil { before := int32(0) for _, ru := range ba.Requests[:int(pErr.Index.Index)] { req := ru.GetInner() - if req.Method() == roachpb.QueryIntent { + if req.Method() == kvpb.QueryIntent { before++ } } @@ -775,11 +776,11 @@ func (tp *txnPipeliner) adjustError( } // Turn an IntentMissingError into a transactional retry error. - if ime, ok := pErr.GetDetail().(*roachpb.IntentMissingError); ok { + if ime, ok := pErr.GetDetail().(*kvpb.IntentMissingError); ok { log.VEventf(ctx, 2, "transforming intent missing error into retry: %v", ime) - err := roachpb.NewTransactionRetryError( - roachpb.RETRY_ASYNC_WRITE_FAILURE, redact.Sprintf("missing intent on: %s", ime.Key)) - retryErr := roachpb.NewErrorWithTxn(err, pErr.GetTxn()) + err := kvpb.NewTransactionRetryError( + kvpb.RETRY_ASYNC_WRITE_FAILURE, redact.Sprintf("missing intent on: %s", ime.Key)) + retryErr := kvpb.NewErrorWithTxn(err, pErr.GetTxn()) retryErr.Index = pErr.Index return retryErr } diff --git a/pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner_client_test.go b/pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner_client_test.go index 33cd10f35c31..f474d01bc7c0 100644 --- a/pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner_client_test.go +++ b/pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner_client_test.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -88,14 +89,14 @@ func TestTxnPipelinerCondenseLockSpans(t *testing.T) { // Check end transaction locks, which should be condensed and split // at range boundaries. expLocks := []roachpb.Span{aToBClosed, cToEClosed, fTog1} - sendFn := func(_ context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + sendFn := func(_ context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { resp := ba.CreateReply() resp.Txn = ba.Txn - if req, ok := ba.GetArg(roachpb.EndTxn); ok { - if !req.(*roachpb.EndTxnRequest).Commit { + if req, ok := ba.GetArg(kvpb.EndTxn); ok { + if !req.(*kvpb.EndTxnRequest).Commit { t.Errorf("expected commit to be true") } - et := req.(*roachpb.EndTxnRequest) + et := req.(*kvpb.EndTxnRequest) if a, e := et.LockSpans, expLocks; !reflect.DeepEqual(a, e) { t.Errorf("expected end transaction to have locks %+v; got %+v", e, a) } diff --git a/pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner_test.go b/pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner_test.go index ee96dff0ca90..dd9a6ff5c93e 100644 --- a/pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner_test.go +++ b/pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner_test.go @@ -18,6 +18,7 @@ import ( "testing" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -35,12 +36,12 @@ import ( // mock out and adjust the SendLocked method. If no mock function is set, a call // to SendLocked will return the default successful response. type mockLockedSender struct { - mockFn func(*roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) + mockFn func(*kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) } func (m *mockLockedSender) SendLocked( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, *kvpb.Error) { if m.mockFn == nil { br := ba.CreateReply() br.Txn = ba.Txn @@ -51,7 +52,7 @@ func (m *mockLockedSender) SendLocked( // MockSend sets the mockLockedSender mocking function. func (m *mockLockedSender) MockSend( - fn func(*roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error), + fn func(*kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error), ) { m.mockFn = fn } @@ -60,12 +61,12 @@ func (m *mockLockedSender) MockSend( // The provided mocking functions are set in the order that they are provided // and a given mocking function is set after the previous one has been called. func (m *mockLockedSender) ChainMockSend( - fns ...func(*roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error), + fns ...func(*kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error), ) { for i := range fns { i := i fn := fns[i] - fns[i] = func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + fns[i] = func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { if i < len(fns)-1 { m.mockFn = fns[i+1] } @@ -117,30 +118,30 @@ func TestTxnPipeliner1PCTransaction(t *testing.T) { keyA, keyB := roachpb.Key("a"), roachpb.Key("b") keyC, keyD := roachpb.Key("c"), roachpb.Key("d") - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - scanArgs := roachpb.ScanRequest{ - RequestHeader: roachpb.RequestHeader{Key: keyA, EndKey: keyB}, + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + scanArgs := kvpb.ScanRequest{ + RequestHeader: kvpb.RequestHeader{Key: keyA, EndKey: keyB}, KeyLocking: lock.Exclusive, } ba.Add(&scanArgs) - putArgs := roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}} + putArgs := kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}} putArgs.Sequence = 1 ba.Add(&putArgs) - delRngArgs := roachpb.DeleteRangeRequest{ - RequestHeader: roachpb.RequestHeader{Key: keyC, EndKey: keyD}, + delRngArgs := kvpb.DeleteRangeRequest{ + RequestHeader: kvpb.RequestHeader{Key: keyC, EndKey: keyD}, } delRngArgs.Sequence = 2 ba.Add(&delRngArgs) - ba.Add(&roachpb.EndTxnRequest{Commit: true}) + ba.Add(&kvpb.EndTxnRequest{Commit: true}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 4) require.False(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.ScanRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[1].GetInner()) - require.IsType(t, &roachpb.DeleteRangeRequest{}, ba.Requests[2].GetInner()) - require.IsType(t, &roachpb.EndTxnRequest{}, ba.Requests[3].GetInner()) + require.IsType(t, &kvpb.ScanRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.DeleteRangeRequest{}, ba.Requests[2].GetInner()) + require.IsType(t, &kvpb.EndTxnRequest{}, ba.Requests[3].GetInner()) etReq := ba.Requests[3].GetEndTxn() expLocks := []roachpb.Span{ @@ -178,16 +179,16 @@ func TestTxnPipelinerTrackInFlightWrites(t *testing.T) { txn := makeTxnProto() keyA := roachpb.Key("a") - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - putArgs := roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}} + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + putArgs := kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}} putArgs.Sequence = 1 ba.Add(&putArgs) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) require.True(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[0].GetInner()) br := ba.CreateReply() br.Txn = ba.Txn @@ -206,29 +207,29 @@ func TestTxnPipelinerTrackInFlightWrites(t *testing.T) { // More writes, one that replaces the other's sequence number. keyB, keyC := roachpb.Key("b"), roachpb.Key("c") ba.Requests = nil - cputArgs := roachpb.ConditionalPutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}} + cputArgs := kvpb.ConditionalPutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}} cputArgs.Sequence = 2 ba.Add(&cputArgs) - initPutArgs := roachpb.InitPutRequest{RequestHeader: roachpb.RequestHeader{Key: keyB}} + initPutArgs := kvpb.InitPutRequest{RequestHeader: kvpb.RequestHeader{Key: keyB}} initPutArgs.Sequence = 3 ba.Add(&initPutArgs) - incArgs := roachpb.IncrementRequest{RequestHeader: roachpb.RequestHeader{Key: keyC}} + incArgs := kvpb.IncrementRequest{RequestHeader: kvpb.RequestHeader{Key: keyC}} incArgs.Sequence = 4 ba.Add(&incArgs) // Write at the same key as another write in the same batch. Will only // result in a single in-flight write, at the larger sequence number. - delArgs := roachpb.DeleteRequest{RequestHeader: roachpb.RequestHeader{Key: keyC}} + delArgs := kvpb.DeleteRequest{RequestHeader: kvpb.RequestHeader{Key: keyC}} delArgs.Sequence = 5 ba.Add(&delArgs) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 5) require.True(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.QueryIntentRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.ConditionalPutRequest{}, ba.Requests[1].GetInner()) - require.IsType(t, &roachpb.InitPutRequest{}, ba.Requests[2].GetInner()) - require.IsType(t, &roachpb.IncrementRequest{}, ba.Requests[3].GetInner()) - require.IsType(t, &roachpb.DeleteRequest{}, ba.Requests[4].GetInner()) + require.IsType(t, &kvpb.QueryIntentRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.ConditionalPutRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.InitPutRequest{}, ba.Requests[2].GetInner()) + require.IsType(t, &kvpb.IncrementRequest{}, ba.Requests[3].GetInner()) + require.IsType(t, &kvpb.DeleteRequest{}, ba.Requests[4].GetInner()) qiReq := ba.Requests[0].GetQueryIntent() require.Equal(t, keyA, qiReq.Key) @@ -249,10 +250,10 @@ func TestTxnPipelinerTrackInFlightWrites(t *testing.T) { br, pErr = tp.SendLocked(ctx, ba) require.NotNil(t, br) require.Len(t, br.Responses, 4) // QueryIntent response stripped - require.IsType(t, &roachpb.ConditionalPutResponse{}, br.Responses[0].GetInner()) - require.IsType(t, &roachpb.InitPutResponse{}, br.Responses[1].GetInner()) - require.IsType(t, &roachpb.IncrementResponse{}, br.Responses[2].GetInner()) - require.IsType(t, &roachpb.DeleteResponse{}, br.Responses[3].GetInner()) + require.IsType(t, &kvpb.ConditionalPutResponse{}, br.Responses[0].GetInner()) + require.IsType(t, &kvpb.InitPutResponse{}, br.Responses[1].GetInner()) + require.IsType(t, &kvpb.IncrementResponse{}, br.Responses[2].GetInner()) + require.IsType(t, &kvpb.DeleteResponse{}, br.Responses[3].GetInner()) require.Nil(t, pErr) require.Equal(t, 3, tp.ifWrites.len()) @@ -267,21 +268,21 @@ func TestTxnPipelinerTrackInFlightWrites(t *testing.T) { // all in-flight writes. Should NOT use async consensus. keyD := roachpb.Key("d") ba.Requests = nil - putArgs2 := roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyD}} + putArgs2 := kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyD}} putArgs2.Sequence = 6 ba.Add(&putArgs2) - etArgs := roachpb.EndTxnRequest{Commit: true} + etArgs := kvpb.EndTxnRequest{Commit: true} etArgs.Sequence = 7 ba.Add(&etArgs) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 5) require.False(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.QueryIntentRequest{}, ba.Requests[1].GetInner()) - require.IsType(t, &roachpb.QueryIntentRequest{}, ba.Requests[2].GetInner()) - require.IsType(t, &roachpb.QueryIntentRequest{}, ba.Requests[3].GetInner()) - require.IsType(t, &roachpb.EndTxnRequest{}, ba.Requests[4].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.QueryIntentRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.QueryIntentRequest{}, ba.Requests[2].GetInner()) + require.IsType(t, &kvpb.QueryIntentRequest{}, ba.Requests[3].GetInner()) + require.IsType(t, &kvpb.EndTxnRequest{}, ba.Requests[4].GetInner()) qiReq1 := ba.Requests[1].GetQueryIntent() qiReq2 := ba.Requests[2].GetQueryIntent() @@ -332,14 +333,14 @@ func TestTxnPipelinerReads(t *testing.T) { keyA, keyC := roachpb.Key("a"), roachpb.Key("c") // Read-only. - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - ba.Add(&roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + ba.Add(&kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) require.False(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.GetRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.GetRequest{}, ba.Requests[0].GetInner()) br := ba.CreateReply() br.Txn = ba.Txn @@ -352,14 +353,14 @@ func TestTxnPipelinerReads(t *testing.T) { // Read before write. ba.Requests = nil - ba.Add(&roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyC}}) + ba.Add(&kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyC}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 2) require.False(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.GetRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.GetRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[1].GetInner()) br = ba.CreateReply() br.Txn = ba.Txn @@ -372,14 +373,14 @@ func TestTxnPipelinerReads(t *testing.T) { // Read after write. ba.Requests = nil - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyC}}) - ba.Add(&roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyC}}) + ba.Add(&kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 2) require.False(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.GetRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.GetRequest{}, ba.Requests[1].GetInner()) br = ba.CreateReply() br.Txn = ba.Txn @@ -396,13 +397,13 @@ func TestTxnPipelinerReads(t *testing.T) { // Read-only with conflicting in-flight write. ba.Requests = nil - ba.Add(&roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 2) require.False(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.QueryIntentRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.GetRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.QueryIntentRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.GetRequest{}, ba.Requests[1].GetInner()) qiReq := ba.Requests[0].GetQueryIntent() require.Equal(t, keyA, qiReq.Key) @@ -435,16 +436,16 @@ func TestTxnPipelinerRangedWrites(t *testing.T) { txn := makeTxnProto() keyA, keyD := roachpb.Key("a"), roachpb.Key("d") - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.DeleteRangeRequest{RequestHeader: roachpb.RequestHeader{Key: keyA, EndKey: keyD}}) + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.DeleteRangeRequest{RequestHeader: kvpb.RequestHeader{Key: keyA, EndKey: keyD}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 2) require.False(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.DeleteRangeRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.DeleteRangeRequest{}, ba.Requests[1].GetInner()) br := ba.CreateReply() br.Txn = ba.Txn @@ -469,14 +470,14 @@ func TestTxnPipelinerRangedWrites(t *testing.T) { tp.ifWrites.insert(roachpb.Key("e"), 13) require.Equal(t, 5, tp.ifWrites.len()) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 5) require.False(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.QueryIntentRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[1].GetInner()) - require.IsType(t, &roachpb.QueryIntentRequest{}, ba.Requests[2].GetInner()) - require.IsType(t, &roachpb.QueryIntentRequest{}, ba.Requests[3].GetInner()) - require.IsType(t, &roachpb.DeleteRangeRequest{}, ba.Requests[4].GetInner()) + require.IsType(t, &kvpb.QueryIntentRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.QueryIntentRequest{}, ba.Requests[2].GetInner()) + require.IsType(t, &kvpb.QueryIntentRequest{}, ba.Requests[3].GetInner()) + require.IsType(t, &kvpb.DeleteRangeRequest{}, ba.Requests[4].GetInner()) qiReq1 := ba.Requests[0].GetQueryIntent() qiReq2 := ba.Requests[2].GetQueryIntent() @@ -519,16 +520,16 @@ func TestTxnPipelinerNonTransactionalRequests(t *testing.T) { txn := makeTxnProto() keyA, keyC := roachpb.Key("a"), roachpb.Key("c") - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyC}}) + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyC}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 2) require.True(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[1].GetInner()) br := ba.CreateReply() br.Txn = ba.Txn @@ -544,16 +545,16 @@ func TestTxnPipelinerNonTransactionalRequests(t *testing.T) { // all in-flight writes, even if its header doesn't imply any interaction. keyRangeDesc := roachpb.Key("rangeDesc") ba.Requests = nil - ba.Add(&roachpb.SubsumeRequest{ - RequestHeader: roachpb.RequestHeader{Key: keyRangeDesc}, + ba.Add(&kvpb.SubsumeRequest{ + RequestHeader: kvpb.RequestHeader{Key: keyRangeDesc}, }) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 3) require.False(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.QueryIntentRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.QueryIntentRequest{}, ba.Requests[1].GetInner()) - require.IsType(t, &roachpb.SubsumeRequest{}, ba.Requests[2].GetInner()) + require.IsType(t, &kvpb.QueryIntentRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.QueryIntentRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.SubsumeRequest{}, ba.Requests[2].GetInner()) qiReq1 := ba.Requests[0].GetQueryIntent() qiReq2 := ba.Requests[1].GetQueryIntent() @@ -591,19 +592,19 @@ func TestTxnPipelinerManyWrites(t *testing.T) { makeSeq := func(i int) enginepb.TxnSeq { return enginepb.TxnSeq(i) + 1 } txn := makeTxnProto() - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} for i := 0; i < writes; i++ { - putArgs := roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: makeKey(i)}} + putArgs := kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: makeKey(i)}} putArgs.Sequence = makeSeq(i) ba.Add(&putArgs) } - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, writes) require.True(t, ba.AsyncConsensus) for i := 0; i < writes; i++ { - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[i].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[i].GetInner()) } br := ba.CreateReply() @@ -621,18 +622,18 @@ func TestTxnPipelinerManyWrites(t *testing.T) { for i := 0; i < writes; i++ { if i%2 == 0 { key := makeKey(i) - ba.Add(&roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: key}}) + ba.Add(&kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{Key: key}}) } } - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, writes) require.False(t, ba.AsyncConsensus) for i := 0; i < writes; i++ { if i%2 == 0 { key := makeKey(i) - require.IsType(t, &roachpb.QueryIntentRequest{}, ba.Requests[i].GetInner()) - require.IsType(t, &roachpb.GetRequest{}, ba.Requests[i+1].GetInner()) + require.IsType(t, &kvpb.QueryIntentRequest{}, ba.Requests[i].GetInner()) + require.IsType(t, &kvpb.GetRequest{}, ba.Requests[i+1].GetInner()) qiReq := ba.Requests[i].GetQueryIntent() require.Equal(t, key, qiReq.Key) @@ -680,16 +681,16 @@ func TestTxnPipelinerTransactionAbort(t *testing.T) { txn := makeTxnProto() keyA := roachpb.Key("a") - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - putArgs := roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}} + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + putArgs := kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}} putArgs.Sequence = 1 ba.Add(&putArgs) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) require.True(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[0].GetInner()) br := ba.CreateReply() br.Txn = ba.Txn @@ -708,14 +709,14 @@ func TestTxnPipelinerTransactionAbort(t *testing.T) { // We'll unrealistically return a PENDING transaction, which won't allow the // txnPipeliner to clean up. ba.Requests = nil - etArgs := roachpb.EndTxnRequest{Commit: false} + etArgs := kvpb.EndTxnRequest{Commit: false} etArgs.Sequence = 2 ba.Add(&etArgs) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) require.False(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.EndTxnRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.EndTxnRequest{}, ba.Requests[0].GetInner()) etReq := ba.Requests[0].GetEndTxn() require.Len(t, etReq.LockSpans, 0) @@ -736,14 +737,14 @@ func TestTxnPipelinerTransactionAbort(t *testing.T) { // ABORTED transaction. This will allow the txnPipeliner to remove all // in-flight writes because they are now uncommittable. ba.Requests = nil - etArgs = roachpb.EndTxnRequest{Commit: false} + etArgs = kvpb.EndTxnRequest{Commit: false} etArgs.Sequence = 2 ba.Add(&etArgs) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) require.False(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.EndTxnRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.EndTxnRequest{}, ba.Requests[0].GetInner()) etReq := ba.Requests[0].GetEndTxn() require.Len(t, etReq.LockSpans, 0) @@ -792,11 +793,11 @@ func TestTxnPipelinerIntentMissingError(t *testing.T) { keyA, keyB := roachpb.Key("a"), roachpb.Key("b") keyC, keyD := roachpb.Key("c"), roachpb.Key("d") - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.DeleteRangeRequest{RequestHeader: roachpb.RequestHeader{Key: keyB, EndKey: keyD}}) - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyD}}) + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.DeleteRangeRequest{RequestHeader: kvpb.RequestHeader{Key: keyB, EndKey: keyD}}) + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyD}}) // Insert in-flight writes into the in-flight write set so that each request // will need to chain on with a QueryIntent. @@ -812,19 +813,19 @@ func TestTxnPipelinerIntentMissingError(t *testing.T) { 5: 2, // intent on key "d" missing } { t.Run(fmt.Sprintf("errIdx=%d", errIdx), func(t *testing.T) { - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 7) require.False(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.QueryIntentRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[1].GetInner()) - require.IsType(t, &roachpb.QueryIntentRequest{}, ba.Requests[2].GetInner()) - require.IsType(t, &roachpb.QueryIntentRequest{}, ba.Requests[3].GetInner()) - require.IsType(t, &roachpb.DeleteRangeRequest{}, ba.Requests[4].GetInner()) - require.IsType(t, &roachpb.QueryIntentRequest{}, ba.Requests[5].GetInner()) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[6].GetInner()) - - err := roachpb.NewIntentMissingError(nil /* key */, nil /* intent */) - pErr := roachpb.NewErrorWithTxn(err, &txn) + require.IsType(t, &kvpb.QueryIntentRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.QueryIntentRequest{}, ba.Requests[2].GetInner()) + require.IsType(t, &kvpb.QueryIntentRequest{}, ba.Requests[3].GetInner()) + require.IsType(t, &kvpb.DeleteRangeRequest{}, ba.Requests[4].GetInner()) + require.IsType(t, &kvpb.QueryIntentRequest{}, ba.Requests[5].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[6].GetInner()) + + err := kvpb.NewIntentMissingError(nil /* key */, nil /* intent */) + pErr := kvpb.NewErrorWithTxn(err, &txn) pErr.SetErrorIndex(errIdx) return nil, pErr }) @@ -834,8 +835,8 @@ func TestTxnPipelinerIntentMissingError(t *testing.T) { require.NotNil(t, pErr) require.Equal(t, &txn, pErr.GetTxn()) require.Equal(t, resErrIdx, pErr.Index.Index) - require.IsType(t, &roachpb.TransactionRetryError{}, pErr.GetDetail()) - require.Equal(t, roachpb.RETRY_ASYNC_WRITE_FAILURE, pErr.GetDetail().(*roachpb.TransactionRetryError).Reason) + require.IsType(t, &kvpb.TransactionRetryError{}, pErr.GetDetail()) + require.Equal(t, kvpb.RETRY_ASYNC_WRITE_FAILURE, pErr.GetDetail().(*kvpb.TransactionRetryError).Reason) }) } } @@ -854,16 +855,16 @@ func TestTxnPipelinerEnableDisableMixTxn(t *testing.T) { txn := makeTxnProto() keyA, keyC := roachpb.Key("a"), roachpb.Key("c") - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - putArgs := roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}} + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + putArgs := kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}} putArgs.Sequence = 1 ba.Add(&putArgs) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) require.False(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[0].GetInner()) br := ba.CreateReply() br.Txn = ba.Txn @@ -879,18 +880,18 @@ func TestTxnPipelinerEnableDisableMixTxn(t *testing.T) { pipelinedWritesEnabled.Override(ctx, &tp.st.SV, true) ba.Requests = nil - putArgs2 := roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}} + putArgs2 := kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}} putArgs2.Sequence = 2 ba.Add(&putArgs2) - putArgs3 := roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyC}} + putArgs3 := kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyC}} putArgs3.Sequence = 3 ba.Add(&putArgs3) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 2) require.True(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[1].GetInner()) br = ba.CreateReply() br.Txn = ba.Txn @@ -907,15 +908,15 @@ func TestTxnPipelinerEnableDisableMixTxn(t *testing.T) { pipelinedWritesEnabled.Override(ctx, &tp.st.SV, false) ba.Requests = nil - putArgs4 := roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}} + putArgs4 := kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}} putArgs4.Sequence = 4 ba.Add(&putArgs4) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 2) require.False(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.QueryIntentRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.QueryIntentRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[1].GetInner()) qiReq := ba.Requests[0].GetQueryIntent() require.Equal(t, keyA, qiReq.Key) @@ -935,15 +936,15 @@ func TestTxnPipelinerEnableDisableMixTxn(t *testing.T) { // Commit the txn. Again with pipeling disabled. Again, in-flight writes // should be proven first. ba.Requests = nil - etArgs := roachpb.EndTxnRequest{Commit: true} + etArgs := kvpb.EndTxnRequest{Commit: true} etArgs.Sequence = 5 ba.Add(&etArgs) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 2) require.False(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.QueryIntentRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.EndTxnRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.QueryIntentRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.EndTxnRequest{}, ba.Requests[1].GetInner()) qiReq := ba.Requests[0].GetQueryIntent() require.Equal(t, keyC, qiReq.Key) @@ -991,14 +992,14 @@ func TestTxnPipelinerMaxInFlightSize(t *testing.T) { keyC, keyD := roachpb.Key("c"), roachpb.Key("d") // Send a batch that would exceed the limit. - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyB}}) - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyC}}) - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyD}}) - - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyB}}) + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyC}}) + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyD}}) + + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 4) require.False(t, ba.AsyncConsensus) @@ -1016,11 +1017,11 @@ func TestTxnPipelinerMaxInFlightSize(t *testing.T) { // Send a batch that is equal to the limit. tp.lockFootprint.clear() // Hackily forget about the past. ba.Requests = nil - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyB}}) - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyC}}) + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyB}}) + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyC}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 3) require.True(t, ba.AsyncConsensus) @@ -1036,9 +1037,9 @@ func TestTxnPipelinerMaxInFlightSize(t *testing.T) { // Send a batch that would be under the limit if we weren't already at it. ba.Requests = nil - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyD}}) + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyD}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) require.False(t, ba.AsyncConsensus) @@ -1055,16 +1056,16 @@ func TestTxnPipelinerMaxInFlightSize(t *testing.T) { // Send a batch that proves two of the in-flight writes. tp.lockFootprint.clear() // hackily disregard the locks ba.Requests = nil - ba.Add(&roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: keyB}}) + ba.Add(&kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{Key: keyB}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 4) require.False(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.QueryIntentRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.GetRequest{}, ba.Requests[1].GetInner()) - require.IsType(t, &roachpb.QueryIntentRequest{}, ba.Requests[2].GetInner()) - require.IsType(t, &roachpb.GetRequest{}, ba.Requests[3].GetInner()) + require.IsType(t, &kvpb.QueryIntentRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.GetRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.QueryIntentRequest{}, ba.Requests[2].GetInner()) + require.IsType(t, &kvpb.GetRequest{}, ba.Requests[3].GetInner()) br = ba.CreateReply() br.Txn = ba.Txn @@ -1082,15 +1083,15 @@ func TestTxnPipelinerMaxInFlightSize(t *testing.T) { // write and immediately writes it again, along with a second write. tp.lockFootprint.clear() // hackily disregard the locks ba.Requests = nil - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyB}}) - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyC}}) + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyB}}) + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyC}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 3) require.True(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.QueryIntentRequest{}, ba.Requests[1].GetInner()) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[2].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.QueryIntentRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[2].GetInner()) br = ba.CreateReply() br.Txn = ba.Txn @@ -1106,13 +1107,13 @@ func TestTxnPipelinerMaxInFlightSize(t *testing.T) { // Send the same batch again. Even though it would prove two in-flight // writes while performing two others, we won't allow it to perform async // consensus because the estimation is conservative. - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 4) require.False(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.QueryIntentRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[1].GetInner()) - require.IsType(t, &roachpb.QueryIntentRequest{}, ba.Requests[2].GetInner()) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[3].GetInner()) + require.IsType(t, &kvpb.QueryIntentRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.QueryIntentRequest{}, ba.Requests[2].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[3].GetInner()) br = ba.CreateReply() br.Txn = ba.Txn @@ -1132,12 +1133,12 @@ func TestTxnPipelinerMaxInFlightSize(t *testing.T) { // The original batch with 4 writes should succeed. ba.Requests = nil - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyB}}) - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyC}}) - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyD}}) + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyB}}) + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyC}}) + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyD}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 4) require.True(t, ba.AsyncConsensus) @@ -1172,14 +1173,14 @@ func TestTxnPipelinerMaxBatchSize(t *testing.T) { keyA, keyC := roachpb.Key("a"), roachpb.Key("c") // Batch below limit. - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) require.True(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[0].GetInner()) br := ba.CreateReply() br.Txn = ba.Txn @@ -1193,15 +1194,15 @@ func TestTxnPipelinerMaxBatchSize(t *testing.T) { // Batch above limit. ba.Requests = nil - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyC}}) + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyC}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 3) require.False(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.QueryIntentRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[1].GetInner()) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[2].GetInner()) + require.IsType(t, &kvpb.QueryIntentRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[2].GetInner()) br = ba.CreateReply() br.Txn = ba.Txn @@ -1218,11 +1219,11 @@ func TestTxnPipelinerMaxBatchSize(t *testing.T) { pipelinedWritesMaxBatchSize.Override(ctx, &tp.st.SV, 2) // Same batch now below limit. - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 2) require.True(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[1].GetInner()) br = ba.CreateReply() br.Txn = ba.Txn @@ -1250,19 +1251,19 @@ func TestTxnPipelinerRecordsLocksOnFailure(t *testing.T) { // Return an error for a point write, a range write, and a range locking // read. - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.DeleteRangeRequest{RequestHeader: roachpb.RequestHeader{Key: keyB, EndKey: keyB.Next()}}) - ba.Add(&roachpb.ScanRequest{RequestHeader: roachpb.RequestHeader{Key: keyC, EndKey: keyC.Next()}, KeyLocking: lock.Exclusive}) - - mockPErr := roachpb.NewErrorf("boom") - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.DeleteRangeRequest{RequestHeader: kvpb.RequestHeader{Key: keyB, EndKey: keyB.Next()}}) + ba.Add(&kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyC, EndKey: keyC.Next()}, KeyLocking: lock.Exclusive}) + + mockPErr := kvpb.NewErrorf("boom") + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 3) require.False(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.DeleteRangeRequest{}, ba.Requests[1].GetInner()) - require.IsType(t, &roachpb.ScanRequest{}, ba.Requests[2].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.DeleteRangeRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.ScanRequest{}, ba.Requests[2].GetInner()) return nil, mockPErr }) @@ -1281,16 +1282,16 @@ func TestTxnPipelinerRecordsLocksOnFailure(t *testing.T) { // Return an ABORTED transaction record for a point write, a range write, // and a range locking read. ba.Requests = nil - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyD}}) - ba.Add(&roachpb.DeleteRangeRequest{RequestHeader: roachpb.RequestHeader{Key: keyE, EndKey: keyE.Next()}}) - ba.Add(&roachpb.ScanRequest{RequestHeader: roachpb.RequestHeader{Key: keyF, EndKey: keyF.Next()}, KeyLocking: lock.Exclusive}) + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyD}}) + ba.Add(&kvpb.DeleteRangeRequest{RequestHeader: kvpb.RequestHeader{Key: keyE, EndKey: keyE.Next()}}) + ba.Add(&kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyF, EndKey: keyF.Next()}, KeyLocking: lock.Exclusive}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 3) require.False(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.DeleteRangeRequest{}, ba.Requests[1].GetInner()) - require.IsType(t, &roachpb.ScanRequest{}, ba.Requests[2].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.DeleteRangeRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.ScanRequest{}, ba.Requests[2].GetInner()) br = ba.CreateReply() br.Txn = ba.Txn @@ -1309,11 +1310,11 @@ func TestTxnPipelinerRecordsLocksOnFailure(t *testing.T) { // The lock spans are all attached to the EndTxn request when one is sent. ba.Requests = nil - ba.Add(&roachpb.EndTxnRequest{Commit: false}) + ba.Add(&kvpb.EndTxnRequest{Commit: false}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) - require.IsType(t, &roachpb.EndTxnRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.EndTxnRequest{}, ba.Requests[0].GetInner()) etReq := ba.Requests[0].GetEndTxn() require.Equal(t, expLocks, etReq.LockSpans) @@ -1347,20 +1348,20 @@ func TestTxnPipelinerIgnoresLocksOnUnambiguousFailure(t *testing.T) { // Return a ConditionalFailed error for a CPut. The lock spans correspond to // the CPut are not added to the lock footprint, but the lock spans for all // other requests in the batch are. - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - ba.Add(&roachpb.ConditionalPutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.DeleteRangeRequest{RequestHeader: roachpb.RequestHeader{Key: keyB, EndKey: keyB.Next()}}) - ba.Add(&roachpb.ScanRequest{RequestHeader: roachpb.RequestHeader{Key: keyC, EndKey: keyC.Next()}, KeyLocking: lock.Exclusive}) + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + ba.Add(&kvpb.ConditionalPutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.DeleteRangeRequest{RequestHeader: kvpb.RequestHeader{Key: keyB, EndKey: keyB.Next()}}) + ba.Add(&kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyC, EndKey: keyC.Next()}, KeyLocking: lock.Exclusive}) - condFailedErr := roachpb.NewError(&roachpb.ConditionFailedError{}) + condFailedErr := kvpb.NewError(&kvpb.ConditionFailedError{}) condFailedErr.SetErrorIndex(0) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 3) require.False(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.ConditionalPutRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.DeleteRangeRequest{}, ba.Requests[1].GetInner()) - require.IsType(t, &roachpb.ScanRequest{}, ba.Requests[2].GetInner()) + require.IsType(t, &kvpb.ConditionalPutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.DeleteRangeRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.ScanRequest{}, ba.Requests[2].GetInner()) return nil, condFailedErr }) @@ -1379,18 +1380,18 @@ func TestTxnPipelinerIgnoresLocksOnUnambiguousFailure(t *testing.T) { // are not added to the lock footprint, but the lock spans for all other // requests in the batch are. ba.Requests = nil - ba.Add(&roachpb.ConditionalPutRequest{RequestHeader: roachpb.RequestHeader{Key: keyD}}) - ba.Add(&roachpb.DeleteRangeRequest{RequestHeader: roachpb.RequestHeader{Key: keyE, EndKey: keyE.Next()}}) - ba.Add(&roachpb.ScanRequest{RequestHeader: roachpb.RequestHeader{Key: keyF, EndKey: keyF.Next()}, KeyLocking: lock.Exclusive}) + ba.Add(&kvpb.ConditionalPutRequest{RequestHeader: kvpb.RequestHeader{Key: keyD}}) + ba.Add(&kvpb.DeleteRangeRequest{RequestHeader: kvpb.RequestHeader{Key: keyE, EndKey: keyE.Next()}}) + ba.Add(&kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyF, EndKey: keyF.Next()}, KeyLocking: lock.Exclusive}) - writeIntentErr := roachpb.NewError(&roachpb.WriteIntentError{}) + writeIntentErr := kvpb.NewError(&kvpb.WriteIntentError{}) writeIntentErr.SetErrorIndex(2) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 3) require.False(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.ConditionalPutRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.DeleteRangeRequest{}, ba.Requests[1].GetInner()) - require.IsType(t, &roachpb.ScanRequest{}, ba.Requests[2].GetInner()) + require.IsType(t, &kvpb.ConditionalPutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.DeleteRangeRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.ScanRequest{}, ba.Requests[2].GetInner()) return nil, writeIntentErr }) @@ -1433,16 +1434,16 @@ func TestTxnPipelinerSavepoints(t *testing.T) { // Now verify one of the writes. When we'll rollback to the savepoint below, // we'll check that the verified write stayed verified. txn := makeTxnProto() - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - ba.Add(&roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: roachpb.Key("a")}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + ba.Add(&kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{Key: roachpb.Key("a")}}) + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 2) require.False(t, ba.AsyncConsensus) - require.IsType(t, &roachpb.QueryIntentRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.GetRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.QueryIntentRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.GetRequest{}, ba.Requests[1].GetInner()) - qiReq := ba.Requests[0].GetInner().(*roachpb.QueryIntentRequest) + qiReq := ba.Requests[0].GetInner().(*kvpb.QueryIntentRequest) require.Equal(t, roachpb.Key("a"), qiReq.Key) require.Equal(t, enginepb.TxnSeq(10), qiReq.Txn.Sequence) @@ -1514,7 +1515,7 @@ func TestTxnPipelinerCondenseLockSpans2(t *testing.T) { // The budget. maxBytes int64 // The request that the test sends. - req *roachpb.BatchRequest + req *kvpb.BatchRequest // The expected state after the request returns. expLockSpans []span expIfWrites []string @@ -1581,13 +1582,13 @@ func TestTxnPipelinerCondenseLockSpans2(t *testing.T) { } txn := makeTxnProto() - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { br := ba.CreateReply() br.Txn = ba.Txn return br, nil }) - tc.req.Header = roachpb.Header{Txn: &txn} + tc.req.Header = kvpb.Header{Txn: &txn} _, pErr := tp.SendLocked(ctx, tc.req) require.Nil(t, pErr) @@ -1615,10 +1616,10 @@ func TestTxnPipelinerCondenseLockSpans2(t *testing.T) { } } -func putBatch(key roachpb.Key, value []byte) *roachpb.BatchRequest { - ba := &roachpb.BatchRequest{} - ba.Add(&roachpb.PutRequest{ - RequestHeader: roachpb.RequestHeader{ +func putBatch(key roachpb.Key, value []byte) *kvpb.BatchRequest { + ba := &kvpb.BatchRequest{} + ba.Add(&kvpb.PutRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, }, Value: roachpb.MakeValueFromBytes(value), @@ -1629,10 +1630,10 @@ func putBatch(key roachpb.Key, value []byte) *roachpb.BatchRequest { // putBatchNoAsyncConsesnsus returns a PutRequest addressed to the default // replica for the specified key / value. The batch also contains a Get, which // inhibits the asyncConsensus flag. -func putBatchNoAsyncConsensus(key roachpb.Key, value []byte) *roachpb.BatchRequest { +func putBatchNoAsyncConsensus(key roachpb.Key, value []byte) *kvpb.BatchRequest { ba := putBatch(key, value) - ba.Add(&roachpb.GetRequest{ - RequestHeader: roachpb.RequestHeader{ + ba.Add(&kvpb.GetRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, }, }) @@ -1655,7 +1656,7 @@ func (s descriptorDBRangeIterator) Valid() bool { } func (s *descriptorDBRangeIterator) Seek(ctx context.Context, key roachpb.RKey, dir ScanDirection) { - descs, _, err := s.db.RangeLookup(ctx, key, roachpb.READ_UNCOMMITTED, dir == Descending) + descs, _, err := s.db.RangeLookup(ctx, key, kvpb.READ_UNCOMMITTED, dir == Descending) if err != nil { panic(err) } @@ -1683,16 +1684,16 @@ func TestTxnPipelinerRejectAboveBudget(t *testing.T) { largeWrite := putBatch(largeAs, nil) mediumWrite := putBatch(largeAs[:5], nil) - delRange := &roachpb.BatchRequest{} + delRange := &kvpb.BatchRequest{} delRange.Header.MaxSpanRequestKeys = 1 - delRange.Add(&roachpb.DeleteRangeRequest{ - RequestHeader: roachpb.RequestHeader{ + delRange.Add(&kvpb.DeleteRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: roachpb.Key("a"), EndKey: roachpb.Key("b"), }, }) delRangeResp := delRange.CreateReply() - delRangeResp.Responses[0].GetInner().(*roachpb.DeleteRangeResponse).ResumeSpan = &roachpb.Span{ + delRangeResp.Responses[0].GetInner().(*kvpb.DeleteRangeResponse).ResumeSpan = &roachpb.Span{ Key: largeAs, EndKey: roachpb.Key("b"), } @@ -1700,23 +1701,23 @@ func TestTxnPipelinerRejectAboveBudget(t *testing.T) { testCases := []struct { name string // The requests to be sent one by one. - reqs []*roachpb.BatchRequest + reqs []*kvpb.BatchRequest // The responses for reqs. If an entry is nil, a response is automatically // generated for it. Requests past the end of the resp array are also // generated automatically. - resp []*roachpb.BatchResponse + resp []*kvpb.BatchResponse // The 0-based index of the request that's expected to be rejected. -1 if no // request is expected to be rejected. expRejectIdx int maxSize int64 }{ {name: "large request", - reqs: []*roachpb.BatchRequest{largeWrite}, + reqs: []*kvpb.BatchRequest{largeWrite}, expRejectIdx: 0, maxSize: int64(len(largeAs)) - 1 + roachpb.SpanOverhead, }, {name: "requests that add up", - reqs: []*roachpb.BatchRequest{ + reqs: []*kvpb.BatchRequest{ putBatchNoAsyncConsensus(roachpb.Key("aaaa"), nil), putBatchNoAsyncConsensus(roachpb.Key("bbbb"), nil), putBatchNoAsyncConsensus(roachpb.Key("cccc"), nil)}, @@ -1729,7 +1730,7 @@ func TestTxnPipelinerRejectAboveBudget(t *testing.T) { // Like the previous test, but this time the requests run with async // consensus. Being tracked as in-flight writes, this test shows that // in-flight writes count towards the budget. - reqs: []*roachpb.BatchRequest{ + reqs: []*kvpb.BatchRequest{ putBatch(roachpb.Key("aaaa"), nil), putBatch(roachpb.Key("bbbb"), nil), putBatch(roachpb.Key("cccc"), nil)}, @@ -1740,8 +1741,8 @@ func TestTxnPipelinerRejectAboveBudget(t *testing.T) { name: "response goes over budget, next request rejected", // A request returns a response with a large resume span, which takes up // the budget. Then the next request will be rejected. - reqs: []*roachpb.BatchRequest{delRange, putBatch(roachpb.Key("a"), nil)}, - resp: []*roachpb.BatchResponse{delRangeResp}, + reqs: []*kvpb.BatchRequest{delRange, putBatch(roachpb.Key("a"), nil)}, + resp: []*kvpb.BatchResponse{delRangeResp}, expRejectIdx: 1, maxSize: 10 + roachpb.SpanOverhead, }, @@ -1750,15 +1751,15 @@ func TestTxnPipelinerRejectAboveBudget(t *testing.T) { // Like the previous test, except here we don't have a followup request // once we're above budget. The test runner will commit the txn, and this // test checks that committing is allowed. - reqs: []*roachpb.BatchRequest{delRange}, - resp: []*roachpb.BatchResponse{delRangeResp}, + reqs: []*kvpb.BatchRequest{delRange}, + resp: []*kvpb.BatchResponse{delRangeResp}, expRejectIdx: -1, maxSize: 10 + roachpb.SpanOverhead, }, { // Request keys overlap, so they don't count twice. name: "overlapping requests", - reqs: []*roachpb.BatchRequest{mediumWrite, mediumWrite, mediumWrite}, + reqs: []*kvpb.BatchRequest{mediumWrite, mediumWrite, mediumWrite}, expRejectIdx: -1, // Our estimation logic for rejecting requests based on size // consults both the in-flight write set (which doesn't account for @@ -1780,7 +1781,7 @@ func TestTxnPipelinerRejectAboveBudget(t *testing.T) { txn := makeTxnProto() var respIdx int - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { // Handle rollbacks and commits separately. if ba.IsSingleAbortTxnRequest() || ba.IsSingleCommitRequest() { br := ba.CreateReply() @@ -1788,7 +1789,7 @@ func TestTxnPipelinerRejectAboveBudget(t *testing.T) { return br, nil } - var resp *roachpb.BatchResponse + var resp *kvpb.BatchResponse if respIdx < len(tc.resp) { resp = tc.resp[respIdx] } @@ -1804,7 +1805,7 @@ func TestTxnPipelinerRejectAboveBudget(t *testing.T) { }) for i, ba := range tc.reqs { - ba.Header = roachpb.Header{Txn: &txn} + ba.Header = kvpb.Header{Txn: &txn} _, pErr := tp.SendLocked(ctx, ba) if i == tc.expRejectIdx { require.NotNil(t, pErr, "expected rejection, but request succeeded") @@ -1817,8 +1818,8 @@ func TestTxnPipelinerRejectAboveBudget(t *testing.T) { require.Equal(t, int64(1), tp.txnMetrics.TxnsRejectedByLockSpanBudget.Count()) // Make sure rolling back the txn works. - rollback := &roachpb.BatchRequest{} - rollback.Add(&roachpb.EndTxnRequest{Commit: false}) + rollback := &kvpb.BatchRequest{} + rollback.Add(&kvpb.EndTxnRequest{Commit: false}) rollback.Txn = &txn _, pErr = tp.SendLocked(ctx, rollback) require.Nil(t, pErr) @@ -1831,8 +1832,8 @@ func TestTxnPipelinerRejectAboveBudget(t *testing.T) { // to be over budget and the response surprised us with a large // ResumeSpan). Committing in these situations is allowed, since the // harm has already been done. - commit := &roachpb.BatchRequest{} - commit.Add(&roachpb.EndTxnRequest{Commit: true}) + commit := &kvpb.BatchRequest{} + commit.Add(&kvpb.EndTxnRequest{Commit: true}) commit.Txn = &txn _, pErr = tp.SendLocked(ctx, commit) require.Nil(t, pErr) diff --git a/pkg/kv/kvclient/kvcoord/txn_interceptor_seq_num_allocator.go b/pkg/kv/kvclient/kvcoord/txn_interceptor_seq_num_allocator.go index c867417e11d5..2854581785f4 100644 --- a/pkg/kv/kvclient/kvcoord/txn_interceptor_seq_num_allocator.go +++ b/pkg/kv/kvclient/kvcoord/txn_interceptor_seq_num_allocator.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/errors" @@ -79,14 +80,14 @@ type txnSeqNumAllocator struct { // SendLocked is part of the txnInterceptor interface. func (s *txnSeqNumAllocator) SendLocked( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, *kvpb.Error) { for _, ru := range ba.Requests { req := ru.GetInner() // Only increment the sequence number generator for requests that // will leave intents or requests that will commit the transaction. // This enables ba.IsCompleteTransaction to work properly. - if roachpb.IsIntentWrite(req) || req.Method() == roachpb.EndTxn { + if kvpb.IsIntentWrite(req) || req.Method() == kvpb.EndTxn { s.writeSeq++ } @@ -95,7 +96,7 @@ func (s *txnSeqNumAllocator) SendLocked( // latest write seqnum. oldHeader := req.Header() oldHeader.Sequence = s.writeSeq - if s.steppingModeEnabled && roachpb.IsReadOnly(req) { + if s.steppingModeEnabled && kvpb.IsReadOnly(req) { oldHeader.Sequence = s.readSeq } req.SetHeader(oldHeader) diff --git a/pkg/kv/kvclient/kvcoord/txn_interceptor_seq_num_allocator_test.go b/pkg/kv/kvclient/kvcoord/txn_interceptor_seq_num_allocator_test.go index f677450afa2f..cd376d360456 100644 --- a/pkg/kv/kvclient/kvcoord/txn_interceptor_seq_num_allocator_test.go +++ b/pkg/kv/kvclient/kvcoord/txn_interceptor_seq_num_allocator_test.go @@ -15,6 +15,7 @@ import ( "fmt" "testing" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -43,12 +44,12 @@ func TestSequenceNumberAllocation(t *testing.T) { keyA, keyB := roachpb.Key("a"), roachpb.Key("b") // Read-only requests are not given unique sequence numbers. - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - ba.Add(&roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.ScanRequest{RequestHeader: roachpb.RequestHeader{Key: keyA, EndKey: keyB}}) + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + ba.Add(&kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyA, EndKey: keyB}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 2) require.Equal(t, enginepb.TxnSeq(0), ba.Requests[0].GetInner().Header().Sequence) require.Equal(t, enginepb.TxnSeq(0), ba.Requests[1].GetInner().Header().Sequence) @@ -64,12 +65,12 @@ func TestSequenceNumberAllocation(t *testing.T) { // Write requests each get a unique sequence number. ba.Requests = nil - ba.Add(&roachpb.ConditionalPutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.InitPutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.ScanRequest{RequestHeader: roachpb.RequestHeader{Key: keyA, EndKey: keyB}}) + ba.Add(&kvpb.ConditionalPutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.InitPutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyA, EndKey: keyB}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 4) require.Equal(t, enginepb.TxnSeq(1), ba.Requests[0].GetInner().Header().Sequence) require.Equal(t, enginepb.TxnSeq(1), ba.Requests[1].GetInner().Header().Sequence) @@ -87,11 +88,11 @@ func TestSequenceNumberAllocation(t *testing.T) { // EndTxn requests also get a unique sequence number. ba.Requests = nil - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.ScanRequest{RequestHeader: roachpb.RequestHeader{Key: keyA, EndKey: keyB}}) - ba.Add(&roachpb.EndTxnRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyA, EndKey: keyB}}) + ba.Add(&kvpb.EndTxnRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 3) require.Equal(t, enginepb.TxnSeq(3), ba.Requests[0].GetInner().Header().Sequence) require.Equal(t, enginepb.TxnSeq(3), ba.Requests[1].GetInner().Header().Sequence) @@ -133,12 +134,12 @@ func TestSequenceNumberAllocationWithStep(t *testing.T) { t.Run(fmt.Sprintf("step %d", i), func(t *testing.T) { currentStepSeqNum := s.writeSeq - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - ba.Add(&roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.ScanRequest{RequestHeader: roachpb.RequestHeader{Key: keyA, EndKey: keyB}}) + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + ba.Add(&kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyA, EndKey: keyB}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 2) require.Equal(t, currentStepSeqNum, ba.Requests[0].GetInner().Header().Sequence) require.Equal(t, currentStepSeqNum, ba.Requests[1].GetInner().Header().Sequence) @@ -155,12 +156,12 @@ func TestSequenceNumberAllocationWithStep(t *testing.T) { // Write requests each get a unique sequence number. The read-only requests // remain at the last step seqnum. ba.Requests = nil - ba.Add(&roachpb.ConditionalPutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.InitPutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.ScanRequest{RequestHeader: roachpb.RequestHeader{Key: keyA, EndKey: keyB}}) + ba.Add(&kvpb.ConditionalPutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.InitPutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyA, EndKey: keyB}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 4) require.Equal(t, currentStepSeqNum+1, ba.Requests[0].GetInner().Header().Sequence) require.Equal(t, currentStepSeqNum, ba.Requests[1].GetInner().Header().Sequence) @@ -179,11 +180,11 @@ func TestSequenceNumberAllocationWithStep(t *testing.T) { // EndTxn requests also get a unique sequence number. Meanwhile read-only // requests remain at the last step. ba.Requests = nil - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.ScanRequest{RequestHeader: roachpb.RequestHeader{Key: keyA, EndKey: keyB}}) - ba.Add(&roachpb.EndTxnRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyA, EndKey: keyB}}) + ba.Add(&kvpb.EndTxnRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 3) require.Equal(t, currentStepSeqNum+3, ba.Requests[0].GetInner().Header().Sequence) require.Equal(t, currentStepSeqNum, ba.Requests[1].GetInner().Header().Sequence) @@ -204,14 +205,14 @@ func TestSequenceNumberAllocationWithStep(t *testing.T) { s.configureSteppingLocked(false /* enabled */) currentStepSeqNum := s.writeSeq - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Requests = nil - ba.Add(&roachpb.ConditionalPutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.InitPutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.ScanRequest{RequestHeader: roachpb.RequestHeader{Key: keyA, EndKey: keyB}}) + ba.Add(&kvpb.ConditionalPutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.InitPutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyA, EndKey: keyB}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 4) require.Equal(t, currentStepSeqNum+1, ba.Requests[0].GetInner().Header().Sequence) require.Equal(t, currentStepSeqNum+1, ba.Requests[1].GetInner().Header().Sequence) @@ -242,7 +243,7 @@ func TestModifyReadSeqNum(t *testing.T) { keyA := roachpb.Key("a") s.configureSteppingLocked(true /* enabled */) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { br := ba.CreateReply() br.Txn = ba.Txn return br, nil @@ -262,9 +263,9 @@ func TestModifyReadSeqNum(t *testing.T) { if err := s.stepLocked(ctx); err != nil { t.Fatal(err) } - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) br, pErr := s.SendLocked(ctx, ba) require.Nil(t, pErr) require.NotNil(t, br) @@ -276,9 +277,9 @@ func TestModifyReadSeqNum(t *testing.T) { if err := s.stepLocked(ctx); err != nil { t.Fatal(err) } - ba = &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) + ba = &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) br, pErr = s.SendLocked(ctx, ba) require.Nil(t, pErr) require.NotNil(t, br) @@ -290,10 +291,10 @@ func TestModifyReadSeqNum(t *testing.T) { t.Fatal(err) } s.readSeq = cursorSeqNum - ba = &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - ba.Add(&roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + ba = &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + ba.Add(&kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Equal(t, cursorSeqNum, ba.Requests[0].GetGet().RequestHeader.Sequence) br := ba.CreateReply() br.Txn = ba.Txn @@ -306,10 +307,10 @@ func TestModifyReadSeqNum(t *testing.T) { if err := s.stepLocked(ctx); err != nil { t.Fatal(err) } - ba = &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - ba.Add(&roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + ba = &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + ba.Add(&kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Equal(t, curReadSeq, ba.Requests[0].GetGet().RequestHeader.Sequence) br := ba.CreateReply() br.Txn = ba.Txn @@ -321,10 +322,10 @@ func TestModifyReadSeqNum(t *testing.T) { if err := s.stepLocked(ctx); err != nil { t.Fatal(err) } - ba = &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + ba = &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Equal(t, s.writeSeq, ba.Requests[0].GetGet().RequestHeader.Sequence) br := ba.CreateReply() br.Txn = ba.Txn @@ -338,10 +339,10 @@ func TestModifyReadSeqNum(t *testing.T) { t.Fatal(err) } s.readSeq = cursorSeqNum - ba = &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - ba.Add(&roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + ba = &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + ba.Add(&kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Equal(t, cursorSeqNum, ba.Requests[0].GetGet().RequestHeader.Sequence) br := ba.CreateReply() br.Txn = ba.Txn @@ -354,10 +355,10 @@ func TestModifyReadSeqNum(t *testing.T) { if err := s.stepLocked(ctx); err != nil { t.Fatal(err) } - ba = &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - ba.Add(&roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + ba = &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + ba.Add(&kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Equal(t, curReadSeq, ba.Requests[0].GetGet().RequestHeader.Sequence) br := ba.CreateReply() br.Txn = ba.Txn @@ -377,12 +378,12 @@ func TestSequenceNumberAllocationTxnRequests(t *testing.T) { txn := makeTxnProto() keyA := roachpb.Key("a") - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - ba.Add(&roachpb.HeartbeatTxnRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.EndTxnRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + ba.Add(&kvpb.HeartbeatTxnRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.EndTxnRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 2) require.Equal(t, enginepb.TxnSeq(0), ba.Requests[0].GetInner().Header().Sequence) require.Equal(t, enginepb.TxnSeq(1), ba.Requests[1].GetInner().Header().Sequence) @@ -409,13 +410,13 @@ func TestSequenceNumberAllocationAfterEpochBump(t *testing.T) { keyA, keyB := roachpb.Key("a"), roachpb.Key("b") // Perform a few writes to increase the sequence number counter. - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - ba.Add(&roachpb.ConditionalPutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.InitPutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + ba.Add(&kvpb.ConditionalPutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.InitPutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 3) require.Equal(t, enginepb.TxnSeq(1), ba.Requests[0].GetInner().Header().Sequence) require.Equal(t, enginepb.TxnSeq(1), ba.Requests[1].GetInner().Header().Sequence) @@ -436,12 +437,12 @@ func TestSequenceNumberAllocationAfterEpochBump(t *testing.T) { // Perform a few more writes. The sequence numbers assigned to requests // should have started back at zero again. ba.Requests = nil - ba.Add(&roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.ConditionalPutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.ScanRequest{RequestHeader: roachpb.RequestHeader{Key: keyA, EndKey: keyB}}) - ba.Add(&roachpb.InitPutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.ConditionalPutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyA, EndKey: keyB}}) + ba.Add(&kvpb.InitPutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 4) require.Equal(t, enginepb.TxnSeq(0), ba.Requests[0].GetInner().Header().Sequence) require.Equal(t, enginepb.TxnSeq(1), ba.Requests[1].GetInner().Header().Sequence) @@ -477,12 +478,12 @@ func TestSequenceNumberAllocationAfterLeafInitialization(t *testing.T) { // Perform a few reads and writes. The sequence numbers assigned should // start at the sequence number provided in the LeafTxnInputState. - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - ba.Add(&roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + ba.Add(&kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 2) require.Equal(t, enginepb.TxnSeq(4), ba.Requests[0].GetInner().Header().Sequence) require.Equal(t, enginepb.TxnSeq(4), ba.Requests[1].GetInner().Header().Sequence) @@ -508,12 +509,12 @@ func TestSequenceNumberAllocationSavepoint(t *testing.T) { keyA, keyB := roachpb.Key("a"), roachpb.Key("b") // Perform a few writes to increase the sequence number counter. - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyB}}) + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyB}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { br := ba.CreateReply() br.Txn = ba.Txn return br, nil diff --git a/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher.go b/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher.go index a676181d9e06..e8bd807a4568 100644 --- a/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher.go +++ b/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher.go @@ -13,6 +13,7 @@ package kvcoord import ( "context" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -138,8 +139,8 @@ type txnSpanRefresher struct { // SendLocked implements the lockedSender interface. func (sr *txnSpanRefresher) SendLocked( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, *kvpb.Error) { // Set the batch's CanForwardReadTimestamp flag. ba.CanForwardReadTimestamp = sr.canForwardReadTimestampWithoutRefresh(ba.Txn) @@ -164,11 +165,11 @@ func (sr *txnSpanRefresher) SendLocked( // Iterate over and aggregate refresh spans in the requests, qualified by // possible resume spans in the responses. if err := sr.assertRefreshSpansAtInvalidTimestamp(br.Txn.ReadTimestamp); err != nil { - return nil, roachpb.NewError(err) + return nil, kvpb.NewError(err) } if !sr.refreshInvalid { if err := sr.appendRefreshSpans(ctx, ba, br); err != nil { - return nil, roachpb.NewError(err) + return nil, kvpb.NewError(err) } // Check whether we should condense the refresh spans. sr.maybeCondenseRefreshSpans(ctx, br.Txn) @@ -214,8 +215,8 @@ func (sr *txnSpanRefresher) maybeCondenseRefreshSpans( // catches serializable errors and attempts to avoid them by refreshing the txn // at a larger timestamp. func (sr *txnSpanRefresher) sendLockedWithRefreshAttempts( - ctx context.Context, ba *roachpb.BatchRequest, maxRefreshAttempts int, -) (*roachpb.BatchResponse, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, maxRefreshAttempts int, +) (*kvpb.BatchResponse, *kvpb.Error) { if ba.Txn.WriteTooOld { // The WriteTooOld flag is not supposed to be set on requests. It's only set // by the server and it's terminated by this interceptor on the client. @@ -256,8 +257,8 @@ func (sr *txnSpanRefresher) sendLockedWithRefreshAttempts( // chances that the refresh fails. bumpedTxn := br.Txn.Clone() bumpedTxn.WriteTooOld = false - pErr = roachpb.NewErrorWithTxn( - roachpb.NewTransactionRetryError(roachpb.RETRY_WRITE_TOO_OLD, + pErr = kvpb.NewErrorWithTxn( + kvpb.NewTransactionRetryError(kvpb.RETRY_WRITE_TOO_OLD, "WriteTooOld flag converted to WriteTooOldError"), bumpedTxn) br = nil @@ -270,7 +271,7 @@ func (sr *txnSpanRefresher) sendLockedWithRefreshAttempts( } } if err := sr.forwardRefreshTimestampOnResponse(ba, br, pErr); err != nil { - return nil, roachpb.NewError(err) + return nil, kvpb.NewError(err) } return br, pErr } @@ -280,15 +281,15 @@ func (sr *txnSpanRefresher) sendLockedWithRefreshAttempts( // txn timestamp, it recurses into sendLockedWithRefreshAttempts and retries the // batch. If the refresh fails, the input pErr is returned. func (sr *txnSpanRefresher) maybeRefreshAndRetrySend( - ctx context.Context, ba *roachpb.BatchRequest, pErr *roachpb.Error, maxRefreshAttempts int, -) (*roachpb.BatchResponse, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, pErr *kvpb.Error, maxRefreshAttempts int, +) (*kvpb.BatchResponse, *kvpb.Error) { txn := pErr.GetTxn() if txn == nil || !sr.canForwardReadTimestamp(txn) { return nil, pErr } // Check for an error which can be refreshed away to avoid a client-side // transaction restart. - ok, refreshTS := roachpb.TransactionRefreshTimestamp(pErr) + ok, refreshTS := kvpb.TransactionRefreshTimestamp(pErr) if !ok { return nil, pErr } @@ -322,8 +323,8 @@ func (sr *txnSpanRefresher) maybeRefreshAndRetrySend( // of writes in the batch and then rejected wholesale when the EndTxn tries // to evaluate the pushed batch. When split, the writes will be pushed but // succeed, the transaction will be refreshed, and the EndTxn will succeed. - args, hasET := ba.GetArg(roachpb.EndTxn) - if len(ba.Requests) > 1 && hasET && !args.(*roachpb.EndTxnRequest).Require1PC { + args, hasET := ba.GetArg(kvpb.EndTxn) + if len(ba.Requests) > 1 && hasET && !args.(*kvpb.EndTxnRequest).Require1PC { log.Eventf(ctx, "sending EndTxn separately from rest of batch on retry") return sr.splitEndTxnAndRetrySend(ctx, ba) } @@ -343,8 +344,8 @@ func (sr *txnSpanRefresher) maybeRefreshAndRetrySend( // only the EndTxn request. It then issues the two partial batches in order, // stitching their results back together at the end. func (sr *txnSpanRefresher) splitEndTxnAndRetrySend( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, *kvpb.Error) { // NOTE: call back into SendLocked with each partial batch, not into // sendLockedWithRefreshAttempts. This ensures that we properly set // CanForwardReadTimestamp on each partial batch and that we provide @@ -370,9 +371,9 @@ func (sr *txnSpanRefresher) splitEndTxnAndRetrySend( // Combine the responses. br := brPrefix - br.Responses = append(br.Responses, roachpb.ResponseUnion{}) + br.Responses = append(br.Responses, kvpb.ResponseUnion{}) if err := br.Combine(brSuffix, []int{etIdx}); err != nil { - return nil, roachpb.NewError(err) + return nil, kvpb.NewError(err) } return br, nil } @@ -385,8 +386,8 @@ func (sr *txnSpanRefresher) splitEndTxnAndRetrySend( // If the force flag is true, the refresh will be attempted even if a refresh // is not inevitable. func (sr *txnSpanRefresher) maybeRefreshPreemptivelyLocked( - ctx context.Context, ba *roachpb.BatchRequest, force bool, -) (*roachpb.BatchRequest, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, force bool, +) (*kvpb.BatchRequest, *kvpb.Error) { // If we know that the transaction will need a refresh at some point because // its write timestamp has diverged from its read timestamp, consider doing // so preemptively. We perform a preemptive refresh if either a) doing so @@ -435,8 +436,8 @@ func (sr *txnSpanRefresher) maybeRefreshPreemptivelyLocked( refreshFree := ba.CanForwardReadTimestamp // If true, this batch is guaranteed to fail without a refresh. - args, hasET := ba.GetArg(roachpb.EndTxn) - refreshInevitable := hasET && args.(*roachpb.EndTxnRequest).Commit + args, hasET := ba.GetArg(kvpb.EndTxn) + refreshInevitable := hasET && args.(*kvpb.EndTxnRequest).Commit // If neither condition is true, defer the refresh. if !refreshFree && !refreshInevitable && !force { @@ -468,23 +469,23 @@ func (sr *txnSpanRefresher) maybeRefreshPreemptivelyLocked( } func newRetryErrorOnFailedPreemptiveRefresh( - txn *roachpb.Transaction, refreshErr *roachpb.Error, -) *roachpb.Error { - reason := roachpb.RETRY_SERIALIZABLE + txn *roachpb.Transaction, refreshErr *kvpb.Error, +) *kvpb.Error { + reason := kvpb.RETRY_SERIALIZABLE if txn.WriteTooOld { - reason = roachpb.RETRY_WRITE_TOO_OLD + reason = kvpb.RETRY_WRITE_TOO_OLD } msg := redact.StringBuilder{} msg.SafeString("failed preemptive refresh") if refreshErr != nil { - if refreshErr, ok := refreshErr.GetDetail().(*roachpb.RefreshFailedError); ok { + if refreshErr, ok := refreshErr.GetDetail().(*kvpb.RefreshFailedError); ok { msg.Printf(" due to a conflict: %s on key %s", refreshErr.FailureReason(), refreshErr.Key) } else { msg.Printf(" - unknown error: %s", refreshErr) } } - retryErr := roachpb.NewTransactionRetryError(reason, msg.RedactableString()) - return roachpb.NewErrorWithTxn(retryErr, txn) + retryErr := kvpb.NewTransactionRetryError(reason, msg.RedactableString()) + return kvpb.NewErrorWithTxn(retryErr, txn) } // tryRefreshTxnSpans sends Refresh and RefreshRange commands to all spans read @@ -496,7 +497,7 @@ func newRetryErrorOnFailedPreemptiveRefresh( // its ReadTimestamp adjusted by the Refresh() method. func (sr *txnSpanRefresher) tryRefreshTxnSpans( ctx context.Context, refreshFrom hlc.Timestamp, refreshToTxn *roachpb.Transaction, -) (err *roachpb.Error) { +) (err *kvpb.Error) { // Track the result of the refresh in metrics. defer func() { if err == nil { @@ -511,7 +512,7 @@ func (sr *txnSpanRefresher) tryRefreshTxnSpans( if sr.refreshInvalid { log.VEvent(ctx, 2, "can't refresh txn spans; not valid") - return roachpb.NewError(errors.AssertionFailedf("can't refresh txn spans; not valid")) + return kvpb.NewError(errors.AssertionFailedf("can't refresh txn spans; not valid")) } else if sr.refreshFootprint.empty() { log.VEvent(ctx, 2, "there are no txn spans to refresh") sr.forwardRefreshTimestampOnRefresh(refreshToTxn) @@ -520,7 +521,7 @@ func (sr *txnSpanRefresher) tryRefreshTxnSpans( // Refresh all spans (merge first). // TODO(nvanbenschoten): actually merge spans. - refreshSpanBa := &roachpb.BatchRequest{} + refreshSpanBa := &kvpb.BatchRequest{} refreshSpanBa.Txn = refreshToTxn addRefreshes := func(refreshes *condensableSpanSet) { // We're going to check writes between the previous refreshed timestamp, if @@ -535,15 +536,15 @@ func (sr *txnSpanRefresher) tryRefreshTxnSpans( // we simply used txn.OrigTimestamp here), could cause false-positives that // would fail the refresh. for _, u := range refreshes.asSlice() { - var req roachpb.Request + var req kvpb.Request if len(u.EndKey) == 0 { - req = &roachpb.RefreshRequest{ - RequestHeader: roachpb.RequestHeaderFromSpan(u), + req = &kvpb.RefreshRequest{ + RequestHeader: kvpb.RequestHeaderFromSpan(u), RefreshFrom: refreshFrom, } } else { - req = &roachpb.RefreshRangeRequest{ - RequestHeader: roachpb.RequestHeaderFromSpan(u), + req = &kvpb.RefreshRangeRequest{ + RequestHeader: kvpb.RequestHeaderFromSpan(u), RefreshFrom: refreshFrom, } } @@ -567,7 +568,7 @@ func (sr *txnSpanRefresher) tryRefreshTxnSpans( // appendRefreshSpans appends refresh spans from the supplied batch request, // qualified by the batch response where appropriate. func (sr *txnSpanRefresher) appendRefreshSpans( - ctx context.Context, ba *roachpb.BatchRequest, br *roachpb.BatchResponse, + ctx context.Context, ba *kvpb.BatchRequest, br *kvpb.BatchResponse, ) error { expLogEnabled := log.ExpensiveLogEnabled(ctx, 3) return ba.RefreshSpanIterate(br, func(span roachpb.Span) { @@ -616,7 +617,7 @@ func (sr *txnSpanRefresher) forwardRefreshTimestampOnRefresh(refreshToTxn *roach // refreshedTimestamp to stay in sync with "server-side refreshes", where the // transaction's read timestamp is updated during the evaluation of a batch. func (sr *txnSpanRefresher) forwardRefreshTimestampOnResponse( - ba *roachpb.BatchRequest, br *roachpb.BatchResponse, pErr *roachpb.Error, + ba *kvpb.BatchRequest, br *kvpb.BatchResponse, pErr *kvpb.Error, ) error { baTxn := ba.Txn var brTxn *roachpb.Transaction diff --git a/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher_test.go b/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher_test.go index 8437c0dea2b5..a7fe6f47c93b 100644 --- a/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher_test.go +++ b/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher_test.go @@ -16,6 +16,7 @@ import ( "strconv" "testing" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -55,19 +56,19 @@ func TestTxnSpanRefresherCollectsSpans(t *testing.T) { keyC, keyD := roachpb.Key("c"), roachpb.Key("d") // Basic case. - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - getArgs := roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}} - putArgs := roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}} - delRangeArgs := roachpb.DeleteRangeRequest{RequestHeader: roachpb.RequestHeader{Key: keyA, EndKey: keyB}} + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + getArgs := kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}} + putArgs := kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}} + delRangeArgs := kvpb.DeleteRangeRequest{RequestHeader: kvpb.RequestHeader{Key: keyA, EndKey: keyB}} ba.Add(&getArgs, &putArgs, &delRangeArgs) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 3) require.True(t, ba.CanForwardReadTimestamp) - require.IsType(t, &roachpb.GetRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[1].GetInner()) - require.IsType(t, &roachpb.DeleteRangeRequest{}, ba.Requests[2].GetInner()) + require.IsType(t, &kvpb.GetRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.DeleteRangeRequest{}, ba.Requests[2].GetInner()) br := ba.CreateReply() br.Txn = ba.Txn @@ -86,13 +87,13 @@ func TestTxnSpanRefresherCollectsSpans(t *testing.T) { // Scan with limit. Only the scanned keys are added to the refresh spans. ba.Requests = nil - scanArgs := roachpb.ScanRequest{RequestHeader: roachpb.RequestHeader{Key: keyB, EndKey: keyD}} + scanArgs := kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyB, EndKey: keyD}} ba.Add(&scanArgs) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) require.False(t, ba.CanForwardReadTimestamp) - require.IsType(t, &roachpb.ScanRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.ScanRequest{}, ba.Requests[0].GetInner()) br = ba.CreateReply() br.Txn = ba.Txn @@ -126,39 +127,39 @@ func TestTxnSpanRefresherRefreshesTransactions(t *testing.T) { name string // OnFirstSend, if set, is invoked to evaluate the batch. If not set, pErr() // will be used to provide an error. - onFirstSend func(*roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) - pErr func() *roachpb.Error + onFirstSend func(*kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) + pErr func() *kvpb.Error expRefresh bool expRefreshTS hlc.Timestamp }{ { - pErr: func() *roachpb.Error { - return roachpb.NewError( - &roachpb.TransactionRetryError{Reason: roachpb.RETRY_SERIALIZABLE}) + pErr: func() *kvpb.Error { + return kvpb.NewError( + &kvpb.TransactionRetryError{Reason: kvpb.RETRY_SERIALIZABLE}) }, expRefresh: true, expRefreshTS: txn.WriteTimestamp, }, { - pErr: func() *roachpb.Error { - return roachpb.NewError( - &roachpb.TransactionRetryError{Reason: roachpb.RETRY_WRITE_TOO_OLD}) + pErr: func() *kvpb.Error { + return kvpb.NewError( + &kvpb.TransactionRetryError{Reason: kvpb.RETRY_WRITE_TOO_OLD}) }, expRefresh: true, expRefreshTS: txn.WriteTimestamp, }, { - pErr: func() *roachpb.Error { - return roachpb.NewError( - &roachpb.WriteTooOldError{ActualTimestamp: txn.WriteTimestamp.Add(15, 0)}) + pErr: func() *kvpb.Error { + return kvpb.NewError( + &kvpb.WriteTooOldError{ActualTimestamp: txn.WriteTimestamp.Add(15, 0)}) }, expRefresh: true, expRefreshTS: txn.WriteTimestamp.Add(15, 0), }, { - pErr: func() *roachpb.Error { - return roachpb.NewError( - &roachpb.ReadWithinUncertaintyIntervalError{ + pErr: func() *kvpb.Error { + return kvpb.NewError( + &kvpb.ReadWithinUncertaintyIntervalError{ ValueTimestamp: txn.WriteTimestamp.Add(25, 0), }) }, @@ -166,9 +167,9 @@ func TestTxnSpanRefresherRefreshesTransactions(t *testing.T) { expRefreshTS: txn.WriteTimestamp.Add(25, 1), // see ExistingTimestamp }, { - pErr: func() *roachpb.Error { - return roachpb.NewError( - &roachpb.ReadWithinUncertaintyIntervalError{ + pErr: func() *kvpb.Error { + return kvpb.NewError( + &kvpb.ReadWithinUncertaintyIntervalError{ ValueTimestamp: txn.WriteTimestamp.Add(25, 0), LocalUncertaintyLimit: hlc.ClockTimestamp(txn.WriteTimestamp.Add(30, 0)), }) @@ -177,14 +178,14 @@ func TestTxnSpanRefresherRefreshesTransactions(t *testing.T) { expRefreshTS: txn.WriteTimestamp.Add(30, 0), // see LocalUncertaintyLimit }, { - pErr: func() *roachpb.Error { - return roachpb.NewErrorf("no refresh") + pErr: func() *kvpb.Error { + return kvpb.NewErrorf("no refresh") }, expRefresh: false, }, { name: "write_too_old flag", - onFirstSend: func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + onFirstSend: func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { br := ba.CreateReply() br.Txn = ba.Txn.Clone() br.Txn.WriteTooOld = true @@ -208,10 +209,10 @@ func TestTxnSpanRefresherRefreshesTransactions(t *testing.T) { tsr, mockSender := makeMockTxnSpanRefresher() // Collect some refresh spans. - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: txn.Clone()} // clone txn since it's shared between subtests - getArgs := roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}} - delRangeArgs := roachpb.DeleteRangeRequest{RequestHeader: roachpb.RequestHeader{Key: keyA, EndKey: keyB}} + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: txn.Clone()} // clone txn since it's shared between subtests + getArgs := kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}} + delRangeArgs := kvpb.DeleteRangeRequest{RequestHeader: kvpb.RequestHeader{Key: keyA, EndKey: keyB}} ba.Add(&getArgs, &delRangeArgs) br, pErr := tsr.SendLocked(ctx, ba) @@ -223,9 +224,9 @@ func TestTxnSpanRefresherRefreshesTransactions(t *testing.T) { require.Zero(t, tsr.refreshedTimestamp) // Hook up a chain of mocking functions. - onFirstSend := func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + onFirstSend := func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[0].GetInner()) // Return a transaction retry error. if tc.onFirstSend != nil { @@ -235,26 +236,26 @@ func TestTxnSpanRefresherRefreshesTransactions(t *testing.T) { pErr.SetTxn(ba.Txn) return nil, pErr } - onSecondSend := func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + onSecondSend := func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { // Should not be called if !expRefresh. require.True(t, tc.expRefresh) require.Len(t, ba.Requests, 1) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[0].GetInner()) // Don't return an error. br = ba.CreateReply() br.Txn = ba.Txn return br, nil } - onRefresh := func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + onRefresh := func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { // Should not be called if !expRefresh. require.True(t, tc.expRefresh) require.Len(t, ba.Requests, 2) require.Equal(t, tc.expRefreshTS, ba.Txn.ReadTimestamp) - require.IsType(t, &roachpb.RefreshRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.RefreshRangeRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.RefreshRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.RefreshRangeRequest{}, ba.Requests[1].GetInner()) refReq := ba.Requests[0].GetRefresh() require.Equal(t, getArgs.Span(), refReq.Span()) @@ -273,7 +274,7 @@ func TestTxnSpanRefresherRefreshesTransactions(t *testing.T) { // Send a request that will hit a retry error. Depending on the // error type, we may or may not perform a refresh. ba.Requests = nil - putArgs := roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}} + putArgs := kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}} ba.Add(&putArgs) br, pErr = tsr.SendLocked(ctx, ba) @@ -315,9 +316,9 @@ func TestTxnSpanRefresherMaxRefreshAttempts(t *testing.T) { tsr.knobs.MaxTxnRefreshAttempts = 2 // Collect some refresh spans. - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - scanArgs := roachpb.ScanRequest{RequestHeader: roachpb.RequestHeader{Key: keyA, EndKey: keyB}} + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + scanArgs := kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyA, EndKey: keyB}} ba.Add(&scanArgs) br, pErr := tsr.SendLocked(ctx, ba) @@ -328,20 +329,20 @@ func TestTxnSpanRefresherMaxRefreshAttempts(t *testing.T) { require.Zero(t, tsr.refreshedTimestamp) // Hook up a chain of mocking functions. - onPut := func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + onPut := func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[0].GetInner()) // Return a transaction retry error. - return nil, roachpb.NewErrorWithTxn( - roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE, ""), ba.Txn) + return nil, kvpb.NewErrorWithTxn( + kvpb.NewTransactionRetryError(kvpb.RETRY_SERIALIZABLE, ""), ba.Txn) } refreshes := 0 - onRefresh := func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + onRefresh := func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { refreshes++ require.Len(t, ba.Requests, 1) require.Equal(t, txn.WriteTimestamp, ba.Txn.ReadTimestamp) - require.IsType(t, &roachpb.RefreshRangeRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.RefreshRangeRequest{}, ba.Requests[0].GetInner()) refReq := ba.Requests[0].GetRefreshRange() require.Equal(t, scanArgs.Span(), refReq.Span()) @@ -351,7 +352,7 @@ func TestTxnSpanRefresherMaxRefreshAttempts(t *testing.T) { br.Txn = ba.Txn return br, nil } - unexpected := func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + unexpected := func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Fail(t, "unexpected") return nil, nil } @@ -361,13 +362,13 @@ func TestTxnSpanRefresherMaxRefreshAttempts(t *testing.T) { // but continue to hit a retry error each time it is attempted. Eventually, // the txnSpanRefresher should give up and propagate the error. ba.Requests = nil - putArgs := roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyB}} + putArgs := kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyB}} ba.Add(&putArgs) br, pErr = tsr.SendLocked(ctx, ba) require.Nil(t, br) require.NotNil(t, pErr) - exp := roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE, "") + exp := kvpb.NewTransactionRetryError(kvpb.RETRY_SERIALIZABLE, "") require.Equal(t, exp, pErr.GetDetail()) require.Equal(t, tsr.knobs.MaxTxnRefreshAttempts, refreshes) } @@ -393,15 +394,15 @@ func TestTxnSpanRefresherPreemptiveRefresh(t *testing.T) { // Send an EndTxn request that will need a refresh to succeed. Because // no refresh spans have been recorded, the preemptive refresh should be // free, so the txnSpanRefresher should do so. - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - etArgs := roachpb.EndTxnRequest{Commit: true} + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + etArgs := kvpb.EndTxnRequest{Commit: true} ba.Add(&etArgs) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) require.True(t, ba.CanForwardReadTimestamp) - require.IsType(t, &roachpb.EndTxnRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.EndTxnRequest{}, ba.Requests[0].GetInner()) // The transaction should be refreshed. require.NotEqual(t, origReadTs, ba.Txn.ReadTimestamp) @@ -429,13 +430,13 @@ func TestTxnSpanRefresherPreemptiveRefresh(t *testing.T) { // the txnSpanRefresher should do so. NOTE: This inhibits a server-side // refreshes when we issue EndTxn requests through the rest of this test. ba.Requests = nil - scanArgs := roachpb.ScanRequest{RequestHeader: roachpb.RequestHeader{Key: keyA, EndKey: keyB}} + scanArgs := kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyA, EndKey: keyB}} ba.Add(&scanArgs) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) require.True(t, ba.CanForwardReadTimestamp) - require.IsType(t, &roachpb.ScanRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.ScanRequest{}, ba.Requests[0].GetInner()) // The transaction should be refreshed. require.NotEqual(t, origReadTs, ba.Txn.ReadTimestamp) @@ -468,19 +469,19 @@ func TestTxnSpanRefresherPreemptiveRefresh(t *testing.T) { ba.Requests = nil ba.Add(&etArgs) - onRefresh := func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + onRefresh := func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) require.Equal(t, pushedWriteTs, ba.Txn.ReadTimestamp) - require.IsType(t, &roachpb.RefreshRangeRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.RefreshRangeRequest{}, ba.Requests[0].GetInner()) refReq := ba.Requests[0].GetRefreshRange() require.Equal(t, scanArgs.Span(), refReq.Span()) require.Equal(t, origReadTs, refReq.RefreshFrom) - return nil, roachpb.NewError(roachpb.NewRefreshFailedError( - roachpb.RefreshFailedError_REASON_COMMITTED_VALUE, roachpb.Key("a"), hlc.Timestamp{WallTime: 1})) + return nil, kvpb.NewError(kvpb.NewRefreshFailedError( + kvpb.RefreshFailedError_REASON_COMMITTED_VALUE, roachpb.Key("a"), hlc.Timestamp{WallTime: 1})) } - unexpected := func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + unexpected := func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Fail(t, "unexpected") return nil, nil } @@ -499,10 +500,10 @@ func TestTxnSpanRefresherPreemptiveRefresh(t *testing.T) { require.False(t, tsr.refreshInvalid) // Try again, but this time let the refresh succeed. - onRefresh = func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + onRefresh = func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) require.Equal(t, pushedWriteTs, ba.Txn.ReadTimestamp) - require.IsType(t, &roachpb.RefreshRangeRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.RefreshRangeRequest{}, ba.Requests[0].GetInner()) refReq := ba.Requests[0].GetRefreshRange() require.Equal(t, scanArgs.Span(), refReq.Span()) @@ -512,10 +513,10 @@ func TestTxnSpanRefresherPreemptiveRefresh(t *testing.T) { br.Txn = ba.Txn return br, nil } - onEndTxn := func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + onEndTxn := func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) require.False(t, ba.CanForwardReadTimestamp) - require.IsType(t, &roachpb.EndTxnRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.EndTxnRequest{}, ba.Requests[0].GetInner()) // The transaction should be refreshed. require.NotEqual(t, origReadTs, ba.Txn.ReadTimestamp) @@ -551,16 +552,16 @@ func TestTxnSpanRefresherSplitEndTxnOnAutoRetry(t *testing.T) { pushedTs2 := txn.ReadTimestamp.Add(2, 0) keyA, keyB := roachpb.Key("a"), roachpb.Key("b") - scanArgs := roachpb.ScanRequest{RequestHeader: roachpb.RequestHeader{Key: keyA, EndKey: keyB}} - putArgs := roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyB}} - etArgs := roachpb.EndTxnRequest{Commit: true} + scanArgs := kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyA, EndKey: keyB}} + putArgs := kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyB}} + etArgs := kvpb.EndTxnRequest{Commit: true} // Run the test with two slightly different configurations. When priorReads // is true, issue a {Put, EndTxn} batch after having previously accumulated // refresh spans due to a Scan. When priorReads is false, issue a {Scan, // Put, EndTxn} batch with no previously accumulated refresh spans. testutils.RunTrueAndFalse(t, "prior_reads", func(t *testing.T, priorReads bool) { - var mockFns []func(*roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) + var mockFns []func(*kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) if priorReads { // Hook up a chain of mocking functions. Expected order of requests: // 1. {Put, EndTxn} -> retry error with pushed timestamp @@ -568,21 +569,21 @@ func TestTxnSpanRefresherSplitEndTxnOnAutoRetry(t *testing.T) { // 3. {Put} -> successful with pushed timestamp // 4. {Refresh} -> successful // 5. {EndTxn} -> successful - onPutAndEndTxn := func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + onPutAndEndTxn := func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 2) require.False(t, ba.CanForwardReadTimestamp) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.EndTxnRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.EndTxnRequest{}, ba.Requests[1].GetInner()) pushedTxn := ba.Txn.Clone() pushedTxn.WriteTimestamp = pushedTs1 - return nil, roachpb.NewErrorWithTxn( - roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE, ""), pushedTxn) + return nil, kvpb.NewErrorWithTxn( + kvpb.NewTransactionRetryError(kvpb.RETRY_SERIALIZABLE, ""), pushedTxn) } - onRefresh1 := func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + onRefresh1 := func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) require.Equal(t, pushedTs1, ba.Txn.ReadTimestamp) - require.IsType(t, &roachpb.RefreshRangeRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.RefreshRangeRequest{}, ba.Requests[0].GetInner()) refReq := ba.Requests[0].GetRefreshRange() require.Equal(t, scanArgs.Span(), refReq.Span()) @@ -592,21 +593,21 @@ func TestTxnSpanRefresherSplitEndTxnOnAutoRetry(t *testing.T) { br.Txn = ba.Txn return br, nil } - onPut := func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + onPut := func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) require.False(t, ba.CanForwardReadTimestamp) require.Equal(t, pushedTs1, ba.Txn.ReadTimestamp) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[0].GetInner()) br := ba.CreateReply() br.Txn = ba.Txn.Clone() br.Txn.WriteTimestamp = pushedTs2 return br, nil } - onRefresh2 := func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + onRefresh2 := func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) require.Equal(t, pushedTs2, ba.Txn.ReadTimestamp) - require.IsType(t, &roachpb.RefreshRangeRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.RefreshRangeRequest{}, ba.Requests[0].GetInner()) refReq := ba.Requests[0].GetRefreshRange() require.Equal(t, scanArgs.Span(), refReq.Span()) @@ -616,11 +617,11 @@ func TestTxnSpanRefresherSplitEndTxnOnAutoRetry(t *testing.T) { br.Txn = ba.Txn return br, nil } - onEndTxn := func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + onEndTxn := func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) require.False(t, ba.CanForwardReadTimestamp) require.Equal(t, pushedTs2, ba.Txn.ReadTimestamp) - require.IsType(t, &roachpb.EndTxnRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.EndTxnRequest{}, ba.Requests[0].GetInner()) br := ba.CreateReply() br.Txn = ba.Txn.Clone() @@ -634,34 +635,34 @@ func TestTxnSpanRefresherSplitEndTxnOnAutoRetry(t *testing.T) { // 3. {Scan, Put} -> successful with pushed timestamp // 4. {Refresh} -> successful // 5. {EndTxn} -> successful - onScanPutAndEndTxn := func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + onScanPutAndEndTxn := func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 3) require.True(t, ba.CanForwardReadTimestamp) - require.IsType(t, &roachpb.ScanRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[1].GetInner()) - require.IsType(t, &roachpb.EndTxnRequest{}, ba.Requests[2].GetInner()) + require.IsType(t, &kvpb.ScanRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.EndTxnRequest{}, ba.Requests[2].GetInner()) pushedTxn := ba.Txn.Clone() pushedTxn.WriteTimestamp = pushedTs1 - return nil, roachpb.NewErrorWithTxn( - roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE, ""), pushedTxn) + return nil, kvpb.NewErrorWithTxn( + kvpb.NewTransactionRetryError(kvpb.RETRY_SERIALIZABLE, ""), pushedTxn) } - onScanAndPut := func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + onScanAndPut := func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 2) require.True(t, ba.CanForwardReadTimestamp) require.Equal(t, pushedTs1, ba.Txn.ReadTimestamp) - require.IsType(t, &roachpb.ScanRequest{}, ba.Requests[0].GetInner()) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[1].GetInner()) + require.IsType(t, &kvpb.ScanRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[1].GetInner()) br := ba.CreateReply() br.Txn = ba.Txn.Clone() br.Txn.WriteTimestamp = pushedTs2 return br, nil } - onRefresh := func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + onRefresh := func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) require.Equal(t, pushedTs2, ba.Txn.ReadTimestamp) - require.IsType(t, &roachpb.RefreshRangeRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.RefreshRangeRequest{}, ba.Requests[0].GetInner()) refReq := ba.Requests[0].GetRefreshRange() require.Equal(t, scanArgs.Span(), refReq.Span()) @@ -671,14 +672,14 @@ func TestTxnSpanRefresherSplitEndTxnOnAutoRetry(t *testing.T) { br.Txn = ba.Txn return br, nil } - onEndTxn := func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + onEndTxn := func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) // IMPORTANT! CanForwardReadTimestamp should no longer be set // for EndTxn batch, because the Scan in the earlier batch needs // to be refreshed if the read timestamp changes. require.False(t, ba.CanForwardReadTimestamp) require.Equal(t, pushedTs2, ba.Txn.ReadTimestamp) - require.IsType(t, &roachpb.EndTxnRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.EndTxnRequest{}, ba.Requests[0].GetInner()) br := ba.CreateReply() br.Txn = ba.Txn.Clone() @@ -700,10 +701,10 @@ func TestTxnSpanRefresherSplitEndTxnOnAutoRetry(t *testing.T) { ctx := context.Background() tsr, mockSender := makeMockTxnSpanRefresher() - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} if priorReads { // Collect some refresh spans first. - ba.Header = roachpb.Header{Txn: &txn} + ba.Header = kvpb.Header{Txn: &txn} ba.Add(&scanArgs) br, pErr := tsr.SendLocked(ctx, ba) @@ -719,23 +720,23 @@ func TestTxnSpanRefresherSplitEndTxnOnAutoRetry(t *testing.T) { // No refresh spans to begin with. require.Equal(t, []roachpb.Span(nil), tsr.refreshFootprint.asSlice()) - ba.Header = roachpb.Header{Txn: &txn} + ba.Header = kvpb.Header{Txn: &txn} ba.Add(&scanArgs, &putArgs, &etArgs) } // Construct the mock sender chain, injecting an error where // appropriate. Make a copy of mockFns to avoid sharing state // between subtests. - mockFnsCpy := append([]func(*roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error)(nil), mockFns...) + mockFnsCpy := append([]func(*kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error)(nil), mockFns...) if errIdx < len(mockFnsCpy) { errFn := mockFnsCpy[errIdx] - newErrFn := func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + newErrFn := func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { _, _ = errFn(ba) - return nil, roachpb.NewErrorf("error") + return nil, kvpb.NewErrorf("error") } mockFnsCpy[errIdx] = newErrFn } - unexpected := func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + unexpected := func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Fail(t, "unexpected") return nil, nil } @@ -750,8 +751,8 @@ func TestTxnSpanRefresherSplitEndTxnOnAutoRetry(t *testing.T) { require.Nil(t, pErr) require.NotNil(t, br) require.Len(t, br.Responses, 2) - require.IsType(t, &roachpb.PutResponse{}, br.Responses[0].GetInner()) - require.IsType(t, &roachpb.EndTxnResponse{}, br.Responses[1].GetInner()) + require.IsType(t, &kvpb.PutResponse{}, br.Responses[0].GetInner()) + require.IsType(t, &kvpb.EndTxnResponse{}, br.Responses[1].GetInner()) require.Equal(t, roachpb.COMMITTED, br.Txn.Status) require.Equal(t, pushedTs2, br.Txn.ReadTimestamp) require.Equal(t, pushedTs2, br.Txn.WriteTimestamp) @@ -786,9 +787,9 @@ func TestTxnSpanRefresherSplitEndTxnOnAutoRetry(t *testing.T) { require.Nil(t, pErr) require.NotNil(t, br) require.Len(t, br.Responses, 3) - require.IsType(t, &roachpb.ScanResponse{}, br.Responses[0].GetInner()) - require.IsType(t, &roachpb.PutResponse{}, br.Responses[1].GetInner()) - require.IsType(t, &roachpb.EndTxnResponse{}, br.Responses[2].GetInner()) + require.IsType(t, &kvpb.ScanResponse{}, br.Responses[0].GetInner()) + require.IsType(t, &kvpb.PutResponse{}, br.Responses[1].GetInner()) + require.IsType(t, &kvpb.EndTxnResponse{}, br.Responses[2].GetInner()) require.Equal(t, roachpb.COMMITTED, br.Txn.Status) require.Equal(t, pushedTs2, br.Txn.ReadTimestamp) require.Equal(t, pushedTs2, br.Txn.WriteTimestamp) @@ -866,9 +867,9 @@ func TestTxnSpanRefresherMaxTxnRefreshSpansBytes(t *testing.T) { MaxTxnRefreshSpansBytes.Override(ctx, &tsr.st.SV, 3+roachpb.SpanOverhead) // Send a batch below the limit. - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - scanArgs := roachpb.ScanRequest{RequestHeader: roachpb.RequestHeader{Key: keyA, EndKey: keyB}} + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + scanArgs := kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyA, EndKey: keyB}} ba.Add(&scanArgs) br, pErr := tsr.SendLocked(ctx, ba) @@ -883,7 +884,7 @@ func TestTxnSpanRefresherMaxTxnRefreshSpansBytes(t *testing.T) { // Send another batch that pushes us above the limit. The tracked spans are // adjacent so the spans will be merged, but not condensed. ba.Requests = nil - scanArgs2 := roachpb.ScanRequest{RequestHeader: roachpb.RequestHeader{Key: keyB, EndKey: keyC}} + scanArgs2 := kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyB, EndKey: keyC}} ba.Add(&scanArgs2) br, pErr = tsr.SendLocked(ctx, ba) @@ -900,7 +901,7 @@ func TestTxnSpanRefresherMaxTxnRefreshSpansBytes(t *testing.T) { // Exceed the limit again, this time with a non-adjacent span such that // condensing needs to occur. ba.Requests = nil - scanArgs3 := roachpb.ScanRequest{RequestHeader: roachpb.RequestHeader{Key: keyD, EndKey: keyE}} + scanArgs3 := kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyD, EndKey: keyE}} ba.Add(&scanArgs3) br, pErr = tsr.SendLocked(ctx, ba) @@ -916,13 +917,13 @@ func TestTxnSpanRefresherMaxTxnRefreshSpansBytes(t *testing.T) { // Return a transaction retry error and make sure the metric indicating that // we did not retry due to the refresh span bytes is incremented. - mockSender.MockSend(func(ba *roachpb.BatchRequest) (batchResponse *roachpb.BatchResponse, r *roachpb.Error) { - return nil, roachpb.NewErrorWithTxn( - roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE, ""), ba.Txn) + mockSender.MockSend(func(ba *kvpb.BatchRequest) (batchResponse *kvpb.BatchResponse, r *kvpb.Error) { + return nil, kvpb.NewErrorWithTxn( + kvpb.NewTransactionRetryError(kvpb.RETRY_SERIALIZABLE, ""), ba.Txn) }) br, pErr = tsr.SendLocked(ctx, ba) - exp := roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE, "") + exp := kvpb.NewTransactionRetryError(kvpb.RETRY_SERIALIZABLE, "") require.Equal(t, exp, pErr.GetDetail()) require.Nil(t, br) require.Equal(t, int64(1), tsr.refreshFailWithCondensedSpans.Count()) @@ -943,14 +944,14 @@ func TestTxnSpanRefresherAssignsCanForwardReadTimestamp(t *testing.T) { // Send a Put request. Should set CanForwardReadTimestamp flag. Should not // collect refresh spans. - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) require.True(t, ba.CanForwardReadTimestamp) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[0].GetInner()) br := ba.CreateReply() br.Txn = ba.Txn @@ -967,14 +968,14 @@ func TestTxnSpanRefresherAssignsCanForwardReadTimestamp(t *testing.T) { // Should NOT set CanForwardReadTimestamp flag. txnFixed := txn.Clone() txnFixed.CommitTimestampFixed = true - baFixed := &roachpb.BatchRequest{} - baFixed.Header = roachpb.Header{Txn: txnFixed} - baFixed.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}}) + baFixed := &kvpb.BatchRequest{} + baFixed.Header = kvpb.Header{Txn: txnFixed} + baFixed.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) require.False(t, ba.CanForwardReadTimestamp) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[0].GetInner()) br = ba.CreateReply() br.Txn = ba.Txn @@ -990,13 +991,13 @@ func TestTxnSpanRefresherAssignsCanForwardReadTimestamp(t *testing.T) { // Send a Scan request. Should set CanForwardReadTimestamp flag. Should // collect refresh spans. ba.Requests = nil - scanArgs := roachpb.ScanRequest{RequestHeader: roachpb.RequestHeader{Key: keyA, EndKey: keyB}} + scanArgs := kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyA, EndKey: keyB}} ba.Add(&scanArgs) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) require.True(t, ba.CanForwardReadTimestamp) - require.IsType(t, &roachpb.ScanRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.ScanRequest{}, ba.Requests[0].GetInner()) br = ba.CreateReply() br.Txn = ba.Txn @@ -1011,13 +1012,13 @@ func TestTxnSpanRefresherAssignsCanForwardReadTimestamp(t *testing.T) { // Send another Scan request. Should NOT set CanForwardReadTimestamp flag. ba.Requests = nil - scanArgs2 := roachpb.ScanRequest{RequestHeader: roachpb.RequestHeader{Key: keyC, EndKey: keyD}} + scanArgs2 := kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyC, EndKey: keyD}} ba.Add(&scanArgs2) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) require.False(t, ba.CanForwardReadTimestamp) - require.IsType(t, &roachpb.ScanRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.ScanRequest{}, ba.Requests[0].GetInner()) br = ba.CreateReply() br.Txn = ba.Txn @@ -1032,12 +1033,12 @@ func TestTxnSpanRefresherAssignsCanForwardReadTimestamp(t *testing.T) { // Send another Put request. Still should NOT set CanForwardReadTimestamp flag. ba.Requests = nil - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyB}}) + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyB}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) require.False(t, ba.CanForwardReadTimestamp) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[0].GetInner()) br = ba.CreateReply() br.Txn = ba.Txn @@ -1053,12 +1054,12 @@ func TestTxnSpanRefresherAssignsCanForwardReadTimestamp(t *testing.T) { // Increment the transaction's epoch and send another Put request. Should // set CanForwardReadTimestamp flag. ba.Requests = nil - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyB}}) + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyB}}) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) require.True(t, ba.CanForwardReadTimestamp) - require.IsType(t, &roachpb.PutRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.PutRequest{}, ba.Requests[0].GetInner()) br = ba.CreateReply() br.Txn = ba.Txn @@ -1091,9 +1092,9 @@ func TestTxnSpanRefresherEpochIncrement(t *testing.T) { MaxTxnRefreshSpansBytes.Override(ctx, &tsr.st.SV, 3+roachpb.SpanOverhead) // Send a batch below the limit. - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - scanArgs := roachpb.ScanRequest{RequestHeader: roachpb.RequestHeader{Key: keyA, EndKey: keyB}} + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + scanArgs := kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyA, EndKey: keyB}} ba.Add(&scanArgs) br, pErr := tsr.SendLocked(ctx, ba) @@ -1114,7 +1115,7 @@ func TestTxnSpanRefresherEpochIncrement(t *testing.T) { // Send a batch above the limit. ba.Requests = nil - scanArgs2 := roachpb.ScanRequest{RequestHeader: roachpb.RequestHeader{Key: keyC, EndKey: keyD}} + scanArgs2 := kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyC, EndKey: keyD}} ba.Add(&scanArgs, &scanArgs2) br, pErr = tsr.SendLocked(ctx, ba) @@ -1146,13 +1147,13 @@ func TestTxnSpanRefresherSavepoint(t *testing.T) { txn := makeTxnProto() read := func(key roachpb.Key) { - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} - getArgs := roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{Key: key}} + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} + getArgs := kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{Key: key}} ba.Add(&getArgs) - mockSender.MockSend(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { require.Len(t, ba.Requests, 1) - require.IsType(t, &roachpb.GetRequest{}, ba.Requests[0].GetInner()) + require.IsType(t, &kvpb.GetRequest{}, ba.Requests[0].GetInner()) br := ba.CreateReply() br.Txn = ba.Txn diff --git a/pkg/kv/kvclient/kvcoord/txn_lock_gatekeeper.go b/pkg/kv/kvclient/kvcoord/txn_lock_gatekeeper.go index 9948611d3adb..9ba4ed5fe045 100644 --- a/pkg/kv/kvclient/kvcoord/txn_lock_gatekeeper.go +++ b/pkg/kv/kvclient/kvcoord/txn_lock_gatekeeper.go @@ -15,7 +15,7 @@ import ( "sync" "github.com/cockroachdb/cockroach/pkg/kv" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/errors" ) @@ -30,7 +30,7 @@ type lockedSender interface { // WARNING: because the lock is released when calling this method and // re-acquired before it returned, callers cannot rely on a single mutual // exclusion zone mainted across the call. - SendLocked(context.Context, *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) + SendLocked(context.Context, *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) } // txnLockGatekeeper is a lockedSender that sits at the bottom of the @@ -54,8 +54,8 @@ type txnLockGatekeeper struct { // SendLocked implements the lockedSender interface. func (gs *txnLockGatekeeper) SendLocked( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, *kvpb.Error) { // If so configured, protect against concurrent use of the txn. Concurrent // requests don't work generally because of races between clients sending // requests and the TxnCoordSender restarting the transaction, and also @@ -66,7 +66,7 @@ func (gs *txnLockGatekeeper) SendLocked( // As a special case, allow for async heartbeats to be sent whenever. if !gs.allowConcurrentRequests && !ba.IsSingleHeartbeatTxnRequest() { if gs.requestInFlight { - return nil, roachpb.NewError( + return nil, kvpb.NewError( errors.AssertionFailedf("concurrent txn use detected. ba: %s", ba)) } gs.requestInFlight = true diff --git a/pkg/kv/kvclient/kvcoord/txn_test.go b/pkg/kv/kvclient/kvcoord/txn_test.go index 63e93f5f39bf..33c40847eb97 100644 --- a/pkg/kv/kvclient/kvcoord/txn_test.go +++ b/pkg/kv/kvclient/kvcoord/txn_test.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" @@ -205,12 +206,12 @@ func TestPriorityRatchetOnAbortOrPush(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) s := createTestDBWithKnobs(t, &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(_ context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + TestingRequestFilter: func(_ context.Context, ba *kvpb.BatchRequest) *kvpb.Error { // Reject transaction heartbeats, which can make the test flaky when they // detect an aborted transaction before the Get operation does. See #68584 // for an explanation. if ba.IsSingleHeartbeatTxnRequest() { - return roachpb.NewErrorf("rejected") + return kvpb.NewErrorf("rejected") } return nil }, @@ -638,21 +639,21 @@ func TestTxnCommitTimestampAdvancedByRefresh(t *testing.T) { var refreshTS hlc.Timestamp errKey := roachpb.Key("inject_err") s := createTestDBWithKnobs(t, &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(_ context.Context, ba *roachpb.BatchRequest) *roachpb.Error { - if g, ok := ba.GetArg(roachpb.Get); ok && g.(*roachpb.GetRequest).Key.Equal(errKey) { + TestingRequestFilter: func(_ context.Context, ba *kvpb.BatchRequest) *kvpb.Error { + if g, ok := ba.GetArg(kvpb.Get); ok && g.(*kvpb.GetRequest).Key.Equal(errKey) { if injected { return nil } injected = true txn := ba.Txn.Clone() refreshTS = txn.WriteTimestamp.Add(0, 1) - pErr := roachpb.NewReadWithinUncertaintyIntervalError( + pErr := kvpb.NewReadWithinUncertaintyIntervalError( txn.ReadTimestamp, hlc.ClockTimestamp{}, txn, refreshTS, hlc.ClockTimestamp{}) - return roachpb.NewErrorWithTxn(pErr, txn) + return kvpb.NewErrorWithTxn(pErr, txn) } return nil }, @@ -701,12 +702,12 @@ func TestTxnLeavesIntentBehindAfterWriteTooOldError(t *testing.T) { // Now we write and expect a WriteTooOld. intentVal := []byte("test") err = txn.Put(ctx, key, intentVal) - require.IsType(t, &roachpb.TransactionRetryWithProtoRefreshError{}, err) + require.IsType(t, &kvpb.TransactionRetryWithProtoRefreshError{}, err) require.Regexp(t, "WriteTooOld", err) // Check that the intent was left behind. b := kv.Batch{} - b.Header.ReadConsistency = roachpb.READ_UNCOMMITTED + b.Header.ReadConsistency = kvpb.READ_UNCOMMITTED b.Get(key) require.NoError(t, s.DB.Run(ctx, &b)) getResp := b.RawResponse().Responses[0].GetGet() @@ -736,7 +737,7 @@ func TestTxnContinueAfterCputError(t *testing.T) { // StrToCPutExistingValue() is not actually necessary. expVal := kvclientutils.StrToCPutExistingValue("dummy") err := txn.CPut(ctx, "a", "val", expVal) - require.IsType(t, &roachpb.ConditionFailedError{}, err) + require.IsType(t, &kvpb.ConditionFailedError{}, err) require.NoError(t, txn.Put(ctx, "a", "b'")) require.NoError(t, txn.Commit(ctx)) @@ -761,7 +762,7 @@ func TestTxnContinueAfterWriteIntentError(t *testing.T) { b.Header.WaitPolicy = lock.WaitPolicy_Error b.Put("a", "c") err := txn.Run(ctx, b) - require.IsType(t, &roachpb.WriteIntentError{}, err) + require.IsType(t, &kvpb.WriteIntentError{}, err) require.NoError(t, txn.Put(ctx, "a'", "c")) require.NoError(t, txn.Commit(ctx)) @@ -822,9 +823,9 @@ func TestTxnWaitPolicies(t *testing.T) { // Priority does not matter. err := <-errorC require.NotNil(t, err) - wiErr := new(roachpb.WriteIntentError) + wiErr := new(kvpb.WriteIntentError) require.True(t, errors.As(err, &wiErr)) - require.Equal(t, roachpb.WriteIntentError_REASON_WAIT_POLICY, wiErr.Reason) + require.Equal(t, kvpb.WriteIntentError_REASON_WAIT_POLICY, wiErr.Reason) // SkipLocked wait policy. type skipRes struct { @@ -874,9 +875,9 @@ func TestTxnLockTimeout(t *testing.T) { b.Get(key) err := s.DB.Run(ctx, &b) require.NotNil(t, err) - wiErr := new(roachpb.WriteIntentError) + wiErr := new(kvpb.WriteIntentError) require.True(t, errors.As(err, &wiErr)) - require.Equal(t, roachpb.WriteIntentError_REASON_LOCK_TIMEOUT, wiErr.Reason) + require.Equal(t, kvpb.WriteIntentError_REASON_LOCK_TIMEOUT, wiErr.Reason) } // TestTxnReturnsWriteTooOldErrorOnConflictingDeleteRange tests that if two @@ -1024,7 +1025,7 @@ func TestTxnRetryWithLatchesDroppedEarly(t *testing.T) { } // Ensure no rows were returned as part of the scan. - require.Equal(t, 0, len(b.RawResponse().Responses[0].GetInner().(*roachpb.ScanResponse).Rows)) + require.Equal(t, 0, len(b.RawResponse().Responses[0].GetInner().(*kvpb.ScanResponse).Rows)) return nil }) if err != nil { diff --git a/pkg/kv/kvclient/kvstreamer/BUILD.bazel b/pkg/kv/kvclient/kvstreamer/BUILD.bazel index 6a840ec3e258..38d31d481fcb 100644 --- a/pkg/kv/kvclient/kvstreamer/BUILD.bazel +++ b/pkg/kv/kvclient/kvstreamer/BUILD.bazel @@ -17,6 +17,7 @@ go_library( "//pkg/keys", "//pkg/kv", "//pkg/kv/kvclient/kvcoord", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/concurrency/lock", "//pkg/kv/kvserver/diskmap", "//pkg/roachpb", @@ -52,6 +53,7 @@ go_test( "//pkg/base", "//pkg/kv", "//pkg/kv/kvclient/kvcoord", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/concurrency/lock", "//pkg/roachpb", "//pkg/security/securityassets", diff --git a/pkg/kv/kvclient/kvstreamer/requests_provider.go b/pkg/kv/kvclient/kvstreamer/requests_provider.go index aee8a13368f0..e0a4ea462f24 100644 --- a/pkg/kv/kvclient/kvstreamer/requests_provider.go +++ b/pkg/kv/kvclient/kvstreamer/requests_provider.go @@ -14,6 +14,7 @@ import ( "sort" "sync" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/buildutil" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -24,7 +25,7 @@ import ( // been truncated to be within a single range. All requests within the // singleRangeBatch will be issued as a single BatchRequest. type singleRangeBatch struct { - reqs []roachpb.RequestUnion + reqs []kvpb.RequestUnion // reqsKeys stores the start key of the corresponding request in reqs. It is // only set prior to sorting reqs within this object (currently, this is // done only in the OutOfOrder mode for the original requests - resume @@ -78,7 +79,7 @@ type singleRangeBatch struct { // the memory usage of reqs, excluding the overhead. reqsReservedBytes int64 // overheadAccountedFor tracks the memory reservation against the budget for - // the overhead of the reqs slice (i.e. of roachpb.RequestUnion objects) as + // the overhead of the reqs slice (i.e. of kvpb.RequestUnion objects) as // well as the positions and the subRequestIdx slices. Since we reuse these // slices for the resume requests, this can be released only when the // BatchResponse doesn't have any resume spans. diff --git a/pkg/kv/kvclient/kvstreamer/results_buffer_test.go b/pkg/kv/kvclient/kvstreamer/results_buffer_test.go index 45e06daee2b3..32e09e1013a1 100644 --- a/pkg/kv/kvclient/kvstreamer/results_buffer_test.go +++ b/pkg/kv/kvclient/kvstreamer/results_buffer_test.go @@ -17,6 +17,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage" @@ -165,7 +166,7 @@ func TestInOrderResultsBuffer(t *testing.T) { func makeResultWithGetResp(rng *rand.Rand, empty bool) Result { var r Result - r.GetResp = &roachpb.GetResponse{} + r.GetResp = &kvpb.GetResponse{} if !empty { rawBytes := make([]byte, rng.Intn(20)+1) rng.Read(rawBytes) @@ -190,7 +191,7 @@ func makeResultWithScanResp(rng *rand.Rand) Result { rng.Read(batchResponse) batchResponses[i] = batchResponse } - r.ScanResp = &roachpb.ScanResponse{ + r.ScanResp = &kvpb.ScanResponse{ BatchResponses: batchResponses, } return r diff --git a/pkg/kv/kvclient/kvstreamer/size.go b/pkg/kv/kvclient/kvstreamer/size.go index 3bd80bc0f8d6..d08ff04f6ad7 100644 --- a/pkg/kv/kvclient/kvstreamer/size.go +++ b/pkg/kv/kvclient/kvstreamer/size.go @@ -13,6 +13,7 @@ package kvstreamer import ( "unsafe" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" ) @@ -21,17 +22,17 @@ const ( intSize = int64(unsafe.Sizeof(int(0))) int32SliceOverhead = int64(unsafe.Sizeof([]int32{})) int32Size = int64(unsafe.Sizeof(int32(0))) - requestUnionSliceOverhead = int64(unsafe.Sizeof([]roachpb.RequestUnion{})) - requestUnionOverhead = int64(unsafe.Sizeof(roachpb.RequestUnion{})) - requestOverhead = int64(unsafe.Sizeof(roachpb.RequestUnion_Get{}) + - unsafe.Sizeof(roachpb.GetRequest{})) + requestUnionSliceOverhead = int64(unsafe.Sizeof([]kvpb.RequestUnion{})) + requestUnionOverhead = int64(unsafe.Sizeof(kvpb.RequestUnion{})) + requestOverhead = int64(unsafe.Sizeof(kvpb.RequestUnion_Get{}) + + unsafe.Sizeof(kvpb.GetRequest{})) ) var zeroInt32Slice []int32 func init() { - scanRequestOverhead := int64(unsafe.Sizeof(roachpb.RequestUnion_Scan{}) + - unsafe.Sizeof(roachpb.ScanRequest{})) + scanRequestOverhead := int64(unsafe.Sizeof(kvpb.RequestUnion_Scan{}) + + unsafe.Sizeof(kvpb.ScanRequest{})) if requestOverhead != scanRequestOverhead { panic("GetRequest and ScanRequest have different overheads") } @@ -52,7 +53,7 @@ func requestSize(key, endKey roachpb.Key) int64 { return requestOverhead + int64(cap(key)) + int64(cap(endKey)) } -func requestsMemUsage(reqs []roachpb.RequestUnion) (memUsage int64) { +func requestsMemUsage(reqs []kvpb.RequestUnion) (memUsage int64) { for _, r := range reqs { h := r.GetInner().Header() memUsage += requestSize(h.Key, h.EndKey) @@ -62,7 +63,7 @@ func requestsMemUsage(reqs []roachpb.RequestUnion) (memUsage int64) { // getResponseSize calculates the size of the GetResponse similar to how it is // accounted for TargetBytes parameter by the KV layer. -func getResponseSize(get *roachpb.GetResponse) int64 { +func getResponseSize(get *kvpb.GetResponse) int64 { if get.Value == nil { return 0 } @@ -71,6 +72,6 @@ func getResponseSize(get *roachpb.GetResponse) int64 { // scanResponseSize calculates the size of the ScanResponse similar to how it is // accounted for TargetBytes parameter by the KV layer. -func scanResponseSize(scan *roachpb.ScanResponse) int64 { +func scanResponseSize(scan *kvpb.ScanResponse) int64 { return scan.NumBytes } diff --git a/pkg/kv/kvclient/kvstreamer/streamer.go b/pkg/kv/kvclient/kvstreamer/streamer.go index 005ba3848a10..051c7695e034 100644 --- a/pkg/kv/kvclient/kvstreamer/streamer.go +++ b/pkg/kv/kvclient/kvstreamer/streamer.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" @@ -76,7 +77,7 @@ type Result struct { // that the response is no longer needed. // // GetResp is guaranteed to have nil IntentValue. - GetResp *roachpb.GetResponse + GetResp *kvpb.GetResponse // ScanResp can contain a partial response to a ScanRequest (when // scanComplete is false). In that case, there will be a further result with // the continuation; that result will use the same Key. Notably, SQL rows @@ -84,7 +85,7 @@ type Result struct { // // The response is always using BATCH_RESPONSE format (meaning that Rows // field is always nil). IntentRows field is also nil. - ScanResp *roachpb.ScanResponse + ScanResp *kvpb.ScanResponse // Position tracks the ordinal among all originally enqueued requests that // this result satisfies. See singleRangeBatch.positions for more details. // TODO(yuzefovich): this might need to be []int when non-unique requests @@ -442,7 +443,7 @@ func (s *Streamer) Init( // Currently, enqueuing new requests while there are still requests in progress // from the previous invocation is prohibited. // TODO(yuzefovich): lift this restriction and introduce the pipelining. -func (s *Streamer) Enqueue(ctx context.Context, reqs []roachpb.RequestUnion) (retErr error) { +func (s *Streamer) Enqueue(ctx context.Context, reqs []kvpb.RequestUnion) (retErr error) { if !s.coordinatorStarted { var coordinatorCtx context.Context coordinatorCtx, s.coordinatorCtxCancel = s.stopper.WithCancelOnQuiesce(ctx) @@ -537,7 +538,7 @@ func (s *Streamer) Enqueue(ctx context.Context, reqs []roachpb.RequestUnion) (re return ri.Error() } // Find all requests that touch the current range. - var singleRangeReqs []roachpb.RequestUnion + var singleRangeReqs []kvpb.RequestUnion var positions []int if allRequestsAreWithinSingleRange { // All requests are within this range, so we can just use the @@ -563,7 +564,7 @@ func (s *Streamer) Enqueue(ctx context.Context, reqs []roachpb.RequestUnion) (re var subRequestIdxOverhead int64 if !s.hints.SingleRowLookup { for i, pos := range positions { - if _, isScan := reqs[pos].GetInner().(*roachpb.ScanRequest); isScan { + if _, isScan := reqs[pos].GetInner().(*kvpb.ScanRequest); isScan { if firstScanRequest { // We have some ScanRequests, and each might touch // multiple ranges, so we have to set up @@ -778,7 +779,7 @@ type workerCoordinator struct { asyncSem *quotapool.IntPool // For request and response admission control. - requestAdmissionHeader roachpb.AdmissionHeader + requestAdmissionHeader kvpb.AdmissionHeader responseAdmissionQ *admission.WorkQueue } @@ -1167,7 +1168,7 @@ func (w *workerCoordinator) performRequestAsync( }, func(ctx context.Context) { defer w.asyncRequestCleanup(false /* budgetMuAlreadyLocked */) - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Header.WaitPolicy = w.lockWaitPolicy ba.Header.TargetBytes = targetBytes ba.Header.AllowEmpty = !headOfLine @@ -1323,13 +1324,13 @@ func (fp singleRangeBatchResponseFootprint) hasIncomplete() bool { // calculateFootprint calculates the memory footprint of the batch response as // well as of the requests that will have to be created for the ResumeSpans. func calculateFootprint( - req singleRangeBatch, br *roachpb.BatchResponse, + req singleRangeBatch, br *kvpb.BatchResponse, ) (fp singleRangeBatchResponseFootprint, _ error) { for i, resp := range br.Responses { reply := resp.GetInner() switch req.reqs[i].GetInner().(type) { - case *roachpb.GetRequest: - get := reply.(*roachpb.GetResponse) + case *kvpb.GetRequest: + get := reply.(*kvpb.GetResponse) if get.IntentValue != nil { return fp, errors.AssertionFailedf( "unexpectedly got an IntentValue back from a SQL GetRequest %v", *get.IntentValue, @@ -1344,8 +1345,8 @@ func calculateFootprint( fp.memoryFootprintBytes += getResponseSize(get) fp.numGetResults++ } - case *roachpb.ScanRequest: - scan := reply.(*roachpb.ScanResponse) + case *kvpb.ScanRequest: + scan := reply.(*kvpb.ScanResponse) if len(scan.Rows) > 0 { return fp, errors.AssertionFailedf( "unexpectedly got a ScanResponse using KEY_VALUES response format", @@ -1384,7 +1385,7 @@ func processSingleRangeResponse( ctx context.Context, s *Streamer, req singleRangeBatch, - br *roachpb.BatchResponse, + br *kvpb.BatchResponse, fp singleRangeBatchResponseFootprint, ) { processSingleRangeResults(ctx, s, req, br, fp) @@ -1402,7 +1403,7 @@ func processSingleRangeResults( ctx context.Context, s *Streamer, req singleRangeBatch, - br *roachpb.BatchResponse, + br *kvpb.BatchResponse, fp singleRangeBatchResponseFootprint, ) { // If there are no results, this function has nothing to do. @@ -1458,7 +1459,7 @@ func processSingleRangeResults( } reply := resp.GetInner() switch response := reply.(type) { - case *roachpb.GetResponse: + case *kvpb.GetResponse: get := response if get.ResumeSpan != nil { // This Get wasn't completed. @@ -1483,7 +1484,7 @@ func processSingleRangeResults( } s.results.addLocked(result) - case *roachpb.ScanResponse: + case *kvpb.ScanResponse: scan := response if len(scan.BatchResponses) == 0 && scan.ResumeSpan != nil { // Only the first part of the conditional is true whenever we @@ -1551,10 +1552,7 @@ func processSingleRangeResults( // Note that it should only be called if the response has any incomplete // requests. func buildResumeSingleRangeBatch( - s *Streamer, - req singleRangeBatch, - br *roachpb.BatchResponse, - fp singleRangeBatchResponseFootprint, + s *Streamer, req singleRangeBatch, br *kvpb.BatchResponse, fp singleRangeBatchResponseFootprint, ) (resumeReq singleRangeBatch) { numIncompleteRequests := fp.numIncompleteGets + fp.numIncompleteScans // We have to allocate the new Get and Scan requests, but we can reuse the @@ -1571,19 +1569,19 @@ func buildResumeSingleRangeBatch( // requests are modified by txnSeqNumAllocator, even if they are not // evaluated due to TargetBytes limit). gets := make([]struct { - req roachpb.GetRequest - union roachpb.RequestUnion_Get + req kvpb.GetRequest + union kvpb.RequestUnion_Get }, fp.numIncompleteGets) scans := make([]struct { - req roachpb.ScanRequest - union roachpb.RequestUnion_Scan + req kvpb.ScanRequest + union kvpb.RequestUnion_Scan }, fp.numIncompleteScans) var resumeReqIdx int for i, resp := range br.Responses { position := req.positions[i] reply := resp.GetInner() switch response := reply.(type) { - case *roachpb.GetResponse: + case *kvpb.GetResponse: get := response if get.ResumeSpan == nil { continue @@ -1605,7 +1603,7 @@ func buildResumeSingleRangeBatch( } resumeReqIdx++ - case *roachpb.ScanResponse: + case *kvpb.ScanResponse: scan := response if scan.ResumeSpan == nil { continue @@ -1615,7 +1613,7 @@ func buildResumeSingleRangeBatch( newScan := scans[0] scans = scans[1:] newScan.req.SetSpan(*scan.ResumeSpan) - newScan.req.ScanFormat = roachpb.BATCH_RESPONSE + newScan.req.ScanFormat = kvpb.BATCH_RESPONSE newScan.req.KeyLocking = s.keyLocking newScan.union.Scan = &newScan.req resumeReq.reqs[resumeReqIdx].Value = &newScan.union @@ -1665,7 +1663,7 @@ func buildResumeSingleRangeBatch( // request. We don't have to do this if there aren't any incomplete requests // since req and resumeReq will be garbage collected on their own. for i := numIncompleteRequests; i < len(req.reqs); i++ { - req.reqs[i] = roachpb.RequestUnion{} + req.reqs[i] = kvpb.RequestUnion{} } atomic.AddInt64(&s.atomics.resumeBatchRequests, 1) atomic.AddInt64(&s.atomics.resumeSingleRangeRequests, int64(numIncompleteRequests)) diff --git a/pkg/kv/kvclient/kvstreamer/streamer_test.go b/pkg/kv/kvclient/kvstreamer/streamer_test.go index fef10aa78621..e2f1de7e1f3c 100644 --- a/pkg/kv/kvclient/kvstreamer/streamer_test.go +++ b/pkg/kv/kvclient/kvstreamer/streamer_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -76,10 +77,10 @@ func TestStreamerLimitations(t *testing.T) { streamer := getStreamer() defer streamer.Close(ctx) streamer.Init(OutOfOrder, Hints{UniqueRequests: true}, 1 /* maxKeysPerRow */, nil /* diskBuffer */) - get := roachpb.NewGet(roachpb.Key("key"), false /* forUpdate */) - reqs := []roachpb.RequestUnion{{ - Value: &roachpb.RequestUnion_Get{ - Get: get.(*roachpb.GetRequest), + get := kvpb.NewGet(roachpb.Key("key"), false /* forUpdate */) + reqs := []kvpb.RequestUnion{{ + Value: &kvpb.RequestUnion_Get{ + Get: get.(*kvpb.GetRequest), }, }} require.NoError(t, streamer.Enqueue(ctx, reqs)) @@ -137,10 +138,10 @@ func TestStreamerBudgetErrorInEnqueue(t *testing.T) { // makeGetRequest returns a valid GetRequest that wants to lookup a key with // value 'a' repeated keySize number of times in the primary index of table // foo. - makeGetRequest := func(keySize int) roachpb.RequestUnion { - var res roachpb.RequestUnion - var get roachpb.GetRequest - var union roachpb.RequestUnion_Get + makeGetRequest := func(keySize int) kvpb.RequestUnion { + var res kvpb.RequestUnion + var get kvpb.GetRequest + var union kvpb.RequestUnion_Get key := make([]byte, keySize+6) key[0] = tableID + 136 key[1] = 137 @@ -187,7 +188,7 @@ func TestStreamerBudgetErrorInEnqueue(t *testing.T) { defer streamer.Close(ctx) // A single request that exceeds the limit should be allowed. - reqs := make([]roachpb.RequestUnion, 1) + reqs := make([]kvpb.RequestUnion, 1) reqs[0] = makeGetRequest(limitBytes + 1) require.NoError(t, streamer.Enqueue(ctx, reqs)) }) @@ -198,7 +199,7 @@ func TestStreamerBudgetErrorInEnqueue(t *testing.T) { // A single request that exceeds the limit as well as the root SQL pool // should be denied. - reqs := make([]roachpb.RequestUnion, 1) + reqs := make([]kvpb.RequestUnion, 1) reqs[0] = makeGetRequest(rootPoolSize + 1) require.Error(t, streamer.Enqueue(ctx, reqs)) }) @@ -208,7 +209,7 @@ func TestStreamerBudgetErrorInEnqueue(t *testing.T) { defer streamer.Close(ctx) // Create two requests which exceed the limit when combined. - reqs := make([]roachpb.RequestUnion, 2) + reqs := make([]kvpb.RequestUnion, 2) reqs[0] = makeGetRequest(limitBytes/2 + 1) reqs[1] = makeGetRequest(limitBytes/2 + 1) require.Error(t, streamer.Enqueue(ctx, reqs)) @@ -397,10 +398,10 @@ func TestStreamerEmptyScans(t *testing.T) { _, err = db.Exec("SELECT count(*) from t") require.NoError(t, err) - makeScanRequest := func(start, end int) roachpb.RequestUnion { - var res roachpb.RequestUnion - var scan roachpb.ScanRequest - var union roachpb.RequestUnion_Scan + makeScanRequest := func(start, end int) kvpb.RequestUnion { + var res kvpb.RequestUnion + var scan kvpb.ScanRequest + var union kvpb.RequestUnion_Scan makeKey := func(pk int) []byte { // These numbers essentially make a key like '/t/primary/pk'. return []byte{tableID + 136, 137, byte(136 + pk)} @@ -424,7 +425,7 @@ func TestStreamerEmptyScans(t *testing.T) { defer streamer.Close(ctx) // Scan the row with pk=0. - reqs := make([]roachpb.RequestUnion, 1) + reqs := make([]kvpb.RequestUnion, 1) reqs[0] = makeScanRequest(0, 1) require.NoError(t, streamer.Enqueue(ctx, reqs)) results, err := streamer.GetResults(ctx) @@ -438,7 +439,7 @@ func TestStreamerEmptyScans(t *testing.T) { defer streamer.Close(ctx) // Scan the rows with pk in range [1, 4). - reqs := make([]roachpb.RequestUnion, 1) + reqs := make([]kvpb.RequestUnion, 1) reqs[0] = makeScanRequest(1, 4) require.NoError(t, streamer.Enqueue(ctx, reqs)) // We expect an empty response for each range. diff --git a/pkg/kv/kvclient/kvtenant/BUILD.bazel b/pkg/kv/kvclient/kvtenant/BUILD.bazel index addedfb971f4..0f7199aefc33 100644 --- a/pkg/kv/kvclient/kvtenant/BUILD.bazel +++ b/pkg/kv/kvclient/kvtenant/BUILD.bazel @@ -12,6 +12,7 @@ go_library( "//pkg/keys", "//pkg/kv/kvclient/kvcoord", "//pkg/kv/kvclient/rangecache", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/rpc", "//pkg/rpc/nodedialer", diff --git a/pkg/kv/kvclient/kvtenant/connector.go b/pkg/kv/kvclient/kvtenant/connector.go index ca310ad5d765..f44dfb60721a 100644 --- a/pkg/kv/kvclient/kvtenant/connector.go +++ b/pkg/kv/kvclient/kvtenant/connector.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangecache" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" @@ -110,8 +111,8 @@ type Connector interface { // token bucket. type TokenBucketProvider interface { TokenBucket( - ctx context.Context, in *roachpb.TokenBucketRequest, - ) (*roachpb.TokenBucketResponse, error) + ctx context.Context, in *kvpb.TokenBucketRequest, + ) (*kvpb.TokenBucketResponse, error) } // ConnectorConfig encompasses the configuration required to create a Connector. diff --git a/pkg/kv/kvclient/rangecache/BUILD.bazel b/pkg/kv/kvclient/rangecache/BUILD.bazel index 1c039f12e9c4..b84278b53d1f 100644 --- a/pkg/kv/kvclient/rangecache/BUILD.bazel +++ b/pkg/kv/kvclient/rangecache/BUILD.bazel @@ -8,6 +8,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/keys", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/settings/cluster", "//pkg/util", diff --git a/pkg/kv/kvclient/rangecache/range_cache.go b/pkg/kv/kvclient/rangecache/range_cache.go index 4d3f14f5465d..67b524cfb5e1 100644 --- a/pkg/kv/kvclient/rangecache/range_cache.go +++ b/pkg/kv/kvclient/rangecache/range_cache.go @@ -21,6 +21,7 @@ import ( "github.com/biogo/store/llrb" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/util" @@ -75,14 +76,14 @@ func (k *rangeCacheKey) Compare(o llrb.Comparable) int { // cases it is referenced, the only acceptable values are READ_UNCOMMITTED // and INCONSISTENT. The hope with this alias and the consts below // it increase code clarity and lead readers of the code here. -type RangeLookupConsistency = roachpb.ReadConsistencyType +type RangeLookupConsistency = kvpb.ReadConsistencyType const ( // ReadFromFollower is the RangeLookupConsistency used to read from a follower. - ReadFromFollower = roachpb.INCONSISTENT + ReadFromFollower = kvpb.INCONSISTENT // ReadFromLeaseholder is the RangeLookupConsistency used to read from the // leaseholder. - ReadFromLeaseholder = roachpb.READ_UNCOMMITTED + ReadFromLeaseholder = kvpb.READ_UNCOMMITTED ) // UnknownClosedTimestampPolicy is used to mark on a CacheEntry that the closed @@ -101,7 +102,7 @@ type RangeDescriptorDB interface { // // Note that the acceptable consistency values are the constants defined // in this package: ReadFromFollower and ReadFromLeaseholder. The - // RangeLookupConsistency type is aliased to roachpb.ReadConsistencyType + // RangeLookupConsistency type is aliased to kvpb.ReadConsistencyType // in order to permit implementations of this interface to import this // package. RangeLookup( diff --git a/pkg/kv/kvclient/rangecache/rangecachemock/BUILD.bazel b/pkg/kv/kvclient/rangecache/rangecachemock/BUILD.bazel index 40a5a611bc45..82420c28797e 100644 --- a/pkg/kv/kvclient/rangecache/rangecachemock/BUILD.bazel +++ b/pkg/kv/kvclient/rangecache/rangecachemock/BUILD.bazel @@ -25,6 +25,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangecache/rangecachemock", # keep visibility = ["//visibility:public"], deps = [ + "//pkg/kv/kvpb", # keep "//pkg/roachpb", # keep "@com_github_golang_mock//gomock", # keep ], diff --git a/pkg/kv/kvclient/rangecache/rangecachemock/mocks_generated.go b/pkg/kv/kvclient/rangecache/rangecachemock/mocks_generated.go index 3043a42479cc..c064c7754ef7 100644 --- a/pkg/kv/kvclient/rangecache/rangecachemock/mocks_generated.go +++ b/pkg/kv/kvclient/rangecache/rangecachemock/mocks_generated.go @@ -8,6 +8,7 @@ import ( context "context" reflect "reflect" + kvpb "github.com/cockroachdb/cockroach/pkg/kv/kvpb" roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" gomock "github.com/golang/mock/gomock" ) @@ -51,7 +52,7 @@ func (mr *MockRangeDescriptorDBMockRecorder) FirstRange() *gomock.Call { } // RangeLookup mocks base method. -func (m *MockRangeDescriptorDB) RangeLookup(arg0 context.Context, arg1 roachpb.RKey, arg2 roachpb.ReadConsistencyType, arg3 bool) ([]roachpb.RangeDescriptor, []roachpb.RangeDescriptor, error) { +func (m *MockRangeDescriptorDB) RangeLookup(arg0 context.Context, arg1 roachpb.RKey, arg2 kvpb.ReadConsistencyType, arg3 bool) ([]roachpb.RangeDescriptor, []roachpb.RangeDescriptor, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "RangeLookup", arg0, arg1, arg2, arg3) ret0, _ := ret[0].([]roachpb.RangeDescriptor) diff --git a/pkg/kv/kvclient/rangefeed/BUILD.bazel b/pkg/kv/kvclient/rangefeed/BUILD.bazel index 5cceb6d1c005..ab986c55cba7 100644 --- a/pkg/kv/kvclient/rangefeed/BUILD.bazel +++ b/pkg/kv/kvclient/rangefeed/BUILD.bazel @@ -18,6 +18,7 @@ go_library( "//pkg/keys", "//pkg/kv", "//pkg/kv/kvclient/kvcoord", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/settings", "//pkg/settings/cluster", @@ -68,6 +69,7 @@ go_test( "//pkg/keys", "//pkg/kv", "//pkg/kv/kvclient/kvcoord", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/roachpb", "//pkg/security/securityassets", diff --git a/pkg/kv/kvclient/rangefeed/config.go b/pkg/kv/kvclient/rangefeed/config.go index 2be820c90b49..38ef66c4c6e7 100644 --- a/pkg/kv/kvclient/rangefeed/config.go +++ b/pkg/kv/kvclient/rangefeed/config.go @@ -13,6 +13,7 @@ package rangefeed import ( "context" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/mon" @@ -150,7 +151,7 @@ func WithRetry(options retry.Options) Option { } // OnCheckpoint is called when a rangefeed checkpoint occurs. -type OnCheckpoint func(ctx context.Context, checkpoint *roachpb.RangeFeedCheckpoint) +type OnCheckpoint func(ctx context.Context, checkpoint *kvpb.RangeFeedCheckpoint) // WithOnCheckpoint sets up a callback that's invoked whenever a check point // event is emitted. @@ -179,7 +180,7 @@ func WithOnCheckpoint(f OnCheckpoint) Option { // callers to ensure this. type OnSSTable func( ctx context.Context, - sst *roachpb.RangeFeedSSTable, + sst *kvpb.RangeFeedSSTable, registeredSpan roachpb.Span, ) @@ -199,7 +200,7 @@ func WithOnSSTable(f OnSSTable) Option { // MVCC range tombstones are currently experimental, and requires the // MVCCRangeTombstones version gate. They are only expected during certain // operations like schema GC and IMPORT INTO (i.e. not across live tables). -type OnDeleteRange func(ctx context.Context, value *roachpb.RangeFeedDeleteRange) +type OnDeleteRange func(ctx context.Context, value *kvpb.RangeFeedDeleteRange) // WithOnDeleteRange sets up a callback that's invoked whenever an MVCC range // deletion tombstone is written. diff --git a/pkg/kv/kvclient/rangefeed/db_adapter.go b/pkg/kv/kvclient/rangefeed/db_adapter.go index 4d5c0534102d..0a73d507b058 100644 --- a/pkg/kv/kvclient/rangefeed/db_adapter.go +++ b/pkg/kv/kvclient/rangefeed/db_adapter.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -185,7 +186,7 @@ func (dbc *dbAdapter) scanSpan( admissionPri = admissionpb.NormalPri } return dbc.db.TxnWithAdmissionControl(ctx, - roachpb.AdmissionHeader_ROOT_KV, + kvpb.AdmissionHeader_ROOT_KV, admissionPri, kv.SteppingDisabled, func(ctx context.Context, txn *kv.Txn) error { diff --git a/pkg/kv/kvclient/rangefeed/db_adapter_external_test.go b/pkg/kv/kvclient/rangefeed/db_adapter_external_test.go index d3a7c2011a78..99e86a426993 100644 --- a/pkg/kv/kvclient/rangefeed/db_adapter_external_test.go +++ b/pkg/kv/kvclient/rangefeed/db_adapter_external_test.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils" @@ -197,7 +198,7 @@ func TestDBClientScan(t *testing.T) { retryScanErr := errors.New("retry scan") feed, err := f.RangeFeed(ctx, "foo-feed", []roachpb.Span{fooSpan}, db.Clock().Now(), - func(ctx context.Context, value *roachpb.RangeFeedValue) {}, + func(ctx context.Context, value *kvpb.RangeFeedValue) {}, rangefeed.WithScanRetryBehavior(rangefeed.ScanRetryRemaining), rangefeed.WithInitialScanParallelismFn(func() int { return parallelism }), diff --git a/pkg/kv/kvclient/rangefeed/rangefeed.go b/pkg/kv/kvclient/rangefeed/rangefeed.go index aaf2e75ecd76..2f8c88260c5e 100644 --- a/pkg/kv/kvclient/rangefeed/rangefeed.go +++ b/pkg/kv/kvclient/rangefeed/rangefeed.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" @@ -164,7 +165,7 @@ func (f *Factory) New( } // OnValue is called for each rangefeed value. -type OnValue func(ctx context.Context, value *roachpb.RangeFeedValue) +type OnValue func(ctx context.Context, value *kvpb.RangeFeedValue) // RangeFeed represents a running RangeFeed. type RangeFeed struct { @@ -309,8 +310,8 @@ func (f *RangeFeed) run(ctx context.Context, frontier *span.Frontier) { } err := ctxgroup.GoAndWait(ctx, rangeFeedTask, processEventsTask) - if errors.HasType(err, &roachpb.BatchTimestampBeforeGCError{}) || - errors.HasType(err, &roachpb.MVCCHistoryMutationError{}) { + if errors.HasType(err, &kvpb.BatchTimestampBeforeGCError{}) || + errors.HasType(err, &kvpb.MVCCHistoryMutationError{}) { if errCallback := f.onUnrecoverableError; errCallback != nil { errCallback(ctx, err) } diff --git a/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go b/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go index 2f0034ed8f2c..5e677b2cc4b1 100644 --- a/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go +++ b/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/spanconfig" @@ -90,10 +91,10 @@ func TestRangeFeedIntegration(t *testing.T) { f, err := rangefeed.NewFactory(srv0.Stopper(), db, srv0.ClusterSettings(), nil) require.NoError(t, err) - rows := make(chan *roachpb.RangeFeedValue) + rows := make(chan *kvpb.RangeFeedValue) initialScanDone := make(chan struct{}) r, err := f.RangeFeed(ctx, "test", []roachpb.Span{sp}, afterB, - func(ctx context.Context, value *roachpb.RangeFeedValue) { + func(ctx context.Context, value *kvpb.RangeFeedValue) { select { case rows <- value: case <-ctx.Done(): @@ -191,7 +192,7 @@ func TestWithOnFrontierAdvance(t *testing.T) { // these spans. We use the key we write to for the ranges below as keys for // this map. spanCheckpointTimestamps := make(map[string]hlc.Timestamp) - forwardCheckpointForKey := func(key string, checkpoint *roachpb.RangeFeedCheckpoint) { + forwardCheckpointForKey := func(key string, checkpoint *kvpb.RangeFeedCheckpoint) { ts := hlc.MinTimestamp if prevTS, found := spanCheckpointTimestamps[key]; found { ts = prevTS @@ -199,15 +200,15 @@ func TestWithOnFrontierAdvance(t *testing.T) { ts.Forward(checkpoint.ResolvedTS) spanCheckpointTimestamps[key] = ts } - rows := make(chan *roachpb.RangeFeedValue) + rows := make(chan *kvpb.RangeFeedValue) r, err := f.RangeFeed(ctx, "test", []roachpb.Span{sp}, db.Clock().Now(), - func(ctx context.Context, value *roachpb.RangeFeedValue) { + func(ctx context.Context, value *kvpb.RangeFeedValue) { select { case rows <- value: case <-ctx.Done(): } }, - rangefeed.WithOnCheckpoint(func(ctx context.Context, checkpoint *roachpb.RangeFeedCheckpoint) { + rangefeed.WithOnCheckpoint(func(ctx context.Context, checkpoint *kvpb.RangeFeedCheckpoint) { if checkpoint.Span.ContainsKey(mkKey("a")) { forwardCheckpointForKey("a", checkpoint) } @@ -298,7 +299,7 @@ func TestWithOnCheckpoint(t *testing.T) { var mu syncutil.RWMutex var afterWriteTS hlc.Timestamp - checkpoints := make(chan *roachpb.RangeFeedCheckpoint) + checkpoints := make(chan *kvpb.RangeFeedCheckpoint) // We need to start a goroutine that reads of the checkpoints channel, so to // not block the rangefeed itself. @@ -329,15 +330,15 @@ func TestWithOnCheckpoint(t *testing.T) { }) }() - rows := make(chan *roachpb.RangeFeedValue) + rows := make(chan *kvpb.RangeFeedValue) r, err := f.RangeFeed(ctx, "test", []roachpb.Span{sp}, db.Clock().Now(), - func(ctx context.Context, value *roachpb.RangeFeedValue) { + func(ctx context.Context, value *kvpb.RangeFeedValue) { select { case rows <- value: case <-ctx.Done(): } }, - rangefeed.WithOnCheckpoint(func(ctx context.Context, checkpoint *roachpb.RangeFeedCheckpoint) { + rangefeed.WithOnCheckpoint(func(ctx context.Context, checkpoint *kvpb.RangeFeedCheckpoint) { select { case checkpoints <- checkpoint: case <-ctx.Done(): @@ -395,9 +396,9 @@ func TestRangefeedValueTimestamps(t *testing.T) { f, err := rangefeed.NewFactory(srv0.Stopper(), db, srv0.ClusterSettings(), nil) require.NoError(t, err) - rows := make(chan *roachpb.RangeFeedValue) + rows := make(chan *kvpb.RangeFeedValue) r, err := f.RangeFeed(ctx, "test", []roachpb.Span{sp}, db.Clock().Now(), - func(ctx context.Context, value *roachpb.RangeFeedValue) { + func(ctx context.Context, value *kvpb.RangeFeedValue) { select { case rows <- value: case <-ctx.Done(): @@ -501,16 +502,16 @@ func TestWithOnSSTable(t *testing.T) { sstC := make(chan kvcoord.RangeFeedMessage) spans := []roachpb.Span{{Key: roachpb.Key("c"), EndKey: roachpb.Key("e")}} r, err := f.RangeFeed(ctx, "test", spans, db.Clock().Now(), - func(ctx context.Context, value *roachpb.RangeFeedValue) {}, - rangefeed.WithOnCheckpoint(func(ctx context.Context, checkpoint *roachpb.RangeFeedCheckpoint) { + func(ctx context.Context, value *kvpb.RangeFeedValue) {}, + rangefeed.WithOnCheckpoint(func(ctx context.Context, checkpoint *kvpb.RangeFeedCheckpoint) { once.Do(func() { close(checkpointC) }) }), - rangefeed.WithOnSSTable(func(ctx context.Context, sst *roachpb.RangeFeedSSTable, registeredSpan roachpb.Span) { + rangefeed.WithOnSSTable(func(ctx context.Context, sst *kvpb.RangeFeedSSTable, registeredSpan roachpb.Span) { select { case sstC <- kvcoord.RangeFeedMessage{ - RangeFeedEvent: &roachpb.RangeFeedEvent{ + RangeFeedEvent: &kvpb.RangeFeedEvent{ SST: sst, }, RegisteredSpan: registeredSpan, @@ -595,16 +596,16 @@ func TestWithOnSSTableCatchesUpIfNotSet(t *testing.T) { // a-f), to ensure only the restricted span is emitted by the catchup scan. var once sync.Once checkpointC := make(chan struct{}) - rowC := make(chan *roachpb.RangeFeedValue) + rowC := make(chan *kvpb.RangeFeedValue) spans := []roachpb.Span{{Key: roachpb.Key("c"), EndKey: roachpb.Key("e")}} r, err := f.RangeFeed(ctx, "test", spans, db.Clock().Now(), - func(ctx context.Context, value *roachpb.RangeFeedValue) { + func(ctx context.Context, value *kvpb.RangeFeedValue) { select { case rowC <- value: case <-ctx.Done(): } }, - rangefeed.WithOnCheckpoint(func(ctx context.Context, checkpoint *roachpb.RangeFeedCheckpoint) { + rangefeed.WithOnCheckpoint(func(ctx context.Context, checkpoint *kvpb.RangeFeedCheckpoint) { once.Do(func() { close(checkpointC) }) @@ -707,12 +708,12 @@ func TestWithOnDeleteRange(t *testing.T) { // to ensure the DeleteRange event is truncated to the registration span. var checkpointOnce sync.Once checkpointC := make(chan struct{}) - deleteRangeC := make(chan *roachpb.RangeFeedDeleteRange) - rowC := make(chan *roachpb.RangeFeedValue) + deleteRangeC := make(chan *kvpb.RangeFeedDeleteRange) + rowC := make(chan *kvpb.RangeFeedValue) spans := []roachpb.Span{{Key: roachpb.Key("c"), EndKey: roachpb.Key("g")}} r, err := f.RangeFeed(ctx, "test", spans, rangeFeedTS, - func(ctx context.Context, e *roachpb.RangeFeedValue) { + func(ctx context.Context, e *kvpb.RangeFeedValue) { select { case rowC <- e: case <-ctx.Done(): @@ -720,12 +721,12 @@ func TestWithOnDeleteRange(t *testing.T) { }, rangefeed.WithDiff(true), rangefeed.WithInitialScan(nil), - rangefeed.WithOnCheckpoint(func(ctx context.Context, checkpoint *roachpb.RangeFeedCheckpoint) { + rangefeed.WithOnCheckpoint(func(ctx context.Context, checkpoint *kvpb.RangeFeedCheckpoint) { checkpointOnce.Do(func() { close(checkpointC) }) }), - rangefeed.WithOnDeleteRange(func(ctx context.Context, e *roachpb.RangeFeedDeleteRange) { + rangefeed.WithOnDeleteRange(func(ctx context.Context, e *kvpb.RangeFeedDeleteRange) { select { case deleteRangeC <- e: case <-ctx.Done(): @@ -890,7 +891,7 @@ func TestUnrecoverableErrors(t *testing.T) { }) r, err := f.RangeFeed(ctx, "test", []roachpb.Span{sp}, preGCThresholdTS, - func(context.Context, *roachpb.RangeFeedValue) {}, + func(context.Context, *kvpb.RangeFeedValue) {}, rangefeed.WithDiff(true), rangefeed.WithOnInternalError(func(ctx context.Context, err error) { mu.Lock() @@ -906,7 +907,7 @@ func TestUnrecoverableErrors(t *testing.T) { mu.Lock() defer mu.Unlock() - if !errors.HasType(mu.internalErr, &roachpb.BatchTimestampBeforeGCError{}) { + if !errors.HasType(mu.internalErr, &kvpb.BatchTimestampBeforeGCError{}) { return errors.New("expected internal error") } return nil @@ -945,8 +946,8 @@ func TestMVCCHistoryMutationError(t *testing.T) { checkpointC := make(chan struct{}) errC := make(chan error) r, err := f.RangeFeed(ctx, "test", []roachpb.Span{sp}, srv0.Clock().Now(), - func(context.Context, *roachpb.RangeFeedValue) {}, - rangefeed.WithOnCheckpoint(func(ctx context.Context, checkpoint *roachpb.RangeFeedCheckpoint) { + func(context.Context, *kvpb.RangeFeedValue) {}, + rangefeed.WithOnCheckpoint(func(ctx context.Context, checkpoint *kvpb.RangeFeedCheckpoint) { once.Do(func() { close(checkpointC) }) @@ -971,8 +972,8 @@ func TestMVCCHistoryMutationError(t *testing.T) { } // Send a ClearRange request that mutates MVCC history. - _, pErr := kv.SendWrapped(ctx, db0.NonTransactionalSender(), &roachpb.ClearRangeRequest{ - RequestHeader: roachpb.RequestHeader{ + _, pErr := kv.SendWrapped(ctx, db0.NonTransactionalSender(), &kvpb.ClearRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: sp.Key, EndKey: sp.EndKey, }, @@ -982,9 +983,9 @@ func TestMVCCHistoryMutationError(t *testing.T) { // Wait for the MVCCHistoryMutationError. select { case err := <-errC: - var mvccErr *roachpb.MVCCHistoryMutationError + var mvccErr *kvpb.MVCCHistoryMutationError require.ErrorAs(t, err, &mvccErr) - require.Equal(t, &roachpb.MVCCHistoryMutationError{Span: sp}, err) + require.Equal(t, &kvpb.MVCCHistoryMutationError{Span: sp}, err) case <-time.After(3 * time.Second): require.Fail(t, "timed out waiting for error") } @@ -1068,7 +1069,7 @@ func TestRangefeedWithLabelsOption(t *testing.T) { keyDSeenCh := make(chan struct{}) r, err := f.RangeFeed(ctx, rangefeedName, []roachpb.Span{sp}, afterB, - func(ctx context.Context, value *roachpb.RangeFeedValue) { + func(ctx context.Context, value *kvpb.RangeFeedValue) { verifyLabels(ctx) if value.Key.Equal(keyD) { keyDSeen.Do(func() { close(keyDSeenCh) }) @@ -1131,9 +1132,9 @@ func TestRangeFeedStartTimeExclusive(t *testing.T) { f, err := rangefeed.NewFactory(srv0.Stopper(), db, srv0.ClusterSettings(), nil) require.NoError(t, err) - rows := make(chan *roachpb.RangeFeedValue) + rows := make(chan *kvpb.RangeFeedValue) r, err := f.RangeFeed(ctx, "test", []roachpb.Span{span}, ts2, - func(ctx context.Context, value *roachpb.RangeFeedValue) { + func(ctx context.Context, value *kvpb.RangeFeedValue) { select { case rows <- value: case <-ctx.Done(): diff --git a/pkg/kv/kvclient/rangefeed/rangefeed_mock_test.go b/pkg/kv/kvclient/rangefeed/rangefeed_mock_test.go index 67bfd19e51a8..c3b7d3aa8186 100644 --- a/pkg/kv/kvclient/rangefeed/rangefeed_mock_test.go +++ b/pkg/kv/kvclient/rangefeed/rangefeed_mock_test.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -112,9 +113,9 @@ func TestRangeFeedMock(t *testing.T) { } f := rangefeed.NewFactoryWithDB(stopper, &mc, nil /* knobs */) require.NotNil(t, f) - rows := make(chan *roachpb.RangeFeedValue) + rows := make(chan *kvpb.RangeFeedValue) - r, err := f.RangeFeed(ctx, "foo", []roachpb.Span{sp}, ts, func(ctx context.Context, value *roachpb.RangeFeedValue) { + r, err := f.RangeFeed(ctx, "foo", []roachpb.Span{sp}, ts, func(ctx context.Context, value *kvpb.RangeFeedValue) { rows <- value }, rangefeed.WithInitialScan(func(ctx context.Context) { close(rows) @@ -169,8 +170,8 @@ func TestRangeFeedMock(t *testing.T) { assert.False(t, withDiff) // it was not set sendEvent := func(ts hlc.Timestamp) { eventC <- kvcoord.RangeFeedMessage{ - RangeFeedEvent: &roachpb.RangeFeedEvent{ - Val: &roachpb.RangeFeedValue{ + RangeFeedEvent: &kvpb.RangeFeedEvent{ + Val: &kvpb.RangeFeedValue{ Key: sp.Key, }, }} @@ -183,8 +184,8 @@ func TestRangeFeedMock(t *testing.T) { return errors.New("boom") case iteration == firstPartialCheckpoint: assert.Equal(t, startFrom, initialTS) - eventC <- kvcoord.RangeFeedMessage{RangeFeedEvent: &roachpb.RangeFeedEvent{ - Checkpoint: &roachpb.RangeFeedCheckpoint{ + eventC <- kvcoord.RangeFeedMessage{RangeFeedEvent: &kvpb.RangeFeedEvent{ + Checkpoint: &kvpb.RangeFeedCheckpoint{ Span: roachpb.Span{ Key: sp.Key, EndKey: sp.Key.PrefixEnd(), @@ -197,8 +198,8 @@ func TestRangeFeedMock(t *testing.T) { case iteration == secondPartialCheckpoint: assert.Equal(t, startFrom, initialTS) eventC <- kvcoord.RangeFeedMessage{ - RangeFeedEvent: &roachpb.RangeFeedEvent{ - Checkpoint: &roachpb.RangeFeedCheckpoint{ + RangeFeedEvent: &kvpb.RangeFeedEvent{ + Checkpoint: &kvpb.RangeFeedCheckpoint{ Span: roachpb.Span{ Key: sp.Key.PrefixEnd(), EndKey: sp.EndKey, @@ -213,8 +214,8 @@ func TestRangeFeedMock(t *testing.T) { // nextTS. assert.Equal(t, startFrom, nextTS) eventC <- kvcoord.RangeFeedMessage{ - RangeFeedEvent: &roachpb.RangeFeedEvent{ - Checkpoint: &roachpb.RangeFeedCheckpoint{ + RangeFeedEvent: &kvpb.RangeFeedEvent{ + Checkpoint: &kvpb.RangeFeedCheckpoint{ Span: sp, ResolvedTS: lastTS, }, @@ -233,9 +234,9 @@ func TestRangeFeedMock(t *testing.T) { }, } f := rangefeed.NewFactoryWithDB(stopper, &mc, nil /* knobs */) - rows := make(chan *roachpb.RangeFeedValue) + rows := make(chan *kvpb.RangeFeedValue) r, err := f.RangeFeed(ctx, "foo", []roachpb.Span{sp}, initialTS, func( - ctx context.Context, value *roachpb.RangeFeedValue, + ctx context.Context, value *kvpb.RangeFeedValue, ) { rows <- value }, rangefeed.WithRetry(shortRetryOptions)) @@ -272,8 +273,8 @@ func TestRangeFeedMock(t *testing.T) { ) error { assert.True(t, withDiff) eventC <- kvcoord.RangeFeedMessage{ - RangeFeedEvent: &roachpb.RangeFeedEvent{ - Val: &roachpb.RangeFeedValue{ + RangeFeedEvent: &kvpb.RangeFeedEvent{ + Val: &kvpb.RangeFeedValue{ Key: sp.Key, }, }} @@ -282,9 +283,9 @@ func TestRangeFeedMock(t *testing.T) { }, } f := rangefeed.NewFactoryWithDB(stopper, &mc, nil /* knobs */) - rows := make(chan *roachpb.RangeFeedValue) + rows := make(chan *kvpb.RangeFeedValue) r, err := f.RangeFeed(ctx, "foo", []roachpb.Span{sp}, hlc.Timestamp{}, func( - ctx context.Context, value *roachpb.RangeFeedValue, + ctx context.Context, value *kvpb.RangeFeedValue, ) { rows <- value }, rangefeed.WithDiff(true)) @@ -302,7 +303,7 @@ func TestRangeFeedMock(t *testing.T) { stopper.Stop(ctx) f := rangefeed.NewFactoryWithDB(stopper, &mockClient{}, nil /* knobs */) r, err := f.RangeFeed(ctx, "foo", []roachpb.Span{sp}, hlc.Timestamp{}, func( - ctx context.Context, value *roachpb.RangeFeedValue, + ctx context.Context, value *kvpb.RangeFeedValue, ) { }) require.Nil(t, r) @@ -325,7 +326,7 @@ func TestRangeFeedMock(t *testing.T) { }, nil /* knobs */) done := make(chan struct{}) r, err := f.RangeFeed(ctx, "foo", []roachpb.Span{sp}, hlc.Timestamp{}, func( - ctx context.Context, value *roachpb.RangeFeedValue, + ctx context.Context, value *kvpb.RangeFeedValue, ) { }, rangefeed.WithInitialScan(nil), @@ -374,7 +375,7 @@ func TestBackoffOnRangefeedFailure(t *testing.T) { r, err := f.RangeFeed(ctx, "foo", []roachpb.Span{{Key: keys.MinKey, EndKey: keys.MaxKey}}, hlc.Timestamp{}, - func(ctx context.Context, value *roachpb.RangeFeedValue) {}, + func(ctx context.Context, value *kvpb.RangeFeedValue) {}, rangefeed.WithInitialScan(func(ctx context.Context) {}), rangefeed.WithRetry(retry.Options{InitialBackoff: time.Millisecond}), ) diff --git a/pkg/kv/kvclient/rangefeed/rangefeedbuffer/BUILD.bazel b/pkg/kv/kvclient/rangefeed/rangefeedbuffer/BUILD.bazel index 3fd4d1b8e089..7cfff958e3a1 100644 --- a/pkg/kv/kvclient/rangefeed/rangefeedbuffer/BUILD.bazel +++ b/pkg/kv/kvclient/rangefeed/rangefeedbuffer/BUILD.bazel @@ -10,6 +10,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedbuffer", visibility = ["//visibility:public"], deps = [ + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/util/hlc", "//pkg/util/log", @@ -28,6 +29,7 @@ go_test( embed = [":rangefeedbuffer"], deps = [ "//pkg/keys", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/util/encoding", "//pkg/util/hlc", diff --git a/pkg/kv/kvclient/rangefeed/rangefeedbuffer/kvs.go b/pkg/kv/kvclient/rangefeed/rangefeedbuffer/kvs.go index f0a575fa643d..b18ac18e270f 100644 --- a/pkg/kv/kvclient/rangefeed/rangefeedbuffer/kvs.go +++ b/pkg/kv/kvclient/rangefeed/rangefeedbuffer/kvs.go @@ -13,13 +13,14 @@ package rangefeedbuffer import ( "sort" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" ) // RangeFeedValueEventToKV is a function to type assert an Event into a -// *roachpb.RangeFeedValue and then convert it to a roachpb.KeyValue. +// *kvpb.RangeFeedValue and then convert it to a roachpb.KeyValue. func RangeFeedValueEventToKV(event Event) roachpb.KeyValue { - rfv := event.(*roachpb.RangeFeedValue) + rfv := event.(*kvpb.RangeFeedValue) return roachpb.KeyValue{Key: rfv.Key, Value: rfv.Value} } diff --git a/pkg/kv/kvclient/rangefeed/rangefeedbuffer/kvs_test.go b/pkg/kv/kvclient/rangefeed/rangefeedbuffer/kvs_test.go index 054904e1bd7f..db1af09e4ed9 100644 --- a/pkg/kv/kvclient/rangefeed/rangefeedbuffer/kvs_test.go +++ b/pkg/kv/kvclient/rangefeed/rangefeedbuffer/kvs_test.go @@ -16,6 +16,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -44,9 +45,9 @@ func TestMergeKVs(t *testing.T) { kv.Value.InitChecksum(kv.Key) return kv } - toRangeFeedEvent := func(r row) *roachpb.RangeFeedValue { + toRangeFeedEvent := func(r row) *kvpb.RangeFeedValue { kv := toKeyValue(r) - return &roachpb.RangeFeedValue{ + return &kvpb.RangeFeedValue{ Key: kv.Key, Value: kv.Value, } diff --git a/pkg/kv/kvclient/rangefeed/rangefeedcache/BUILD.bazel b/pkg/kv/kvclient/rangefeed/rangefeedcache/BUILD.bazel index d948a77cdbd7..443e776d9dc6 100644 --- a/pkg/kv/kvclient/rangefeed/rangefeedcache/BUILD.bazel +++ b/pkg/kv/kvclient/rangefeed/rangefeedcache/BUILD.bazel @@ -10,6 +10,7 @@ go_library( "//pkg/base", "//pkg/kv/kvclient/rangefeed", "//pkg/kv/kvclient/rangefeed/rangefeedbuffer", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/util/grpcutil", "//pkg/util/hlc", @@ -38,6 +39,7 @@ go_test( "//pkg/kv/kvclient/rangecache", # keep "//pkg/kv/kvclient/rangefeed", "//pkg/kv/kvclient/rangefeed/rangefeedbuffer", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/security/securityassets", "//pkg/security/securitytest", diff --git a/pkg/kv/kvclient/rangefeed/rangefeedcache/cache_impl_test.go b/pkg/kv/kvclient/rangefeed/rangefeedcache/cache_impl_test.go index eff63d770ce5..186f188046e3 100644 --- a/pkg/kv/kvclient/rangefeed/rangefeedcache/cache_impl_test.go +++ b/pkg/kv/kvclient/rangefeed/rangefeedcache/cache_impl_test.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedbuffer" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/stop" @@ -93,8 +94,6 @@ func (c *Cache) handleUpdate(ctx context.Context, update Update) { c.mu.timestamp = update.Timestamp } -func passThroughTranslation( - ctx context.Context, value *roachpb.RangeFeedValue, -) rangefeedbuffer.Event { +func passThroughTranslation(ctx context.Context, value *kvpb.RangeFeedValue) rangefeedbuffer.Event { return value } diff --git a/pkg/kv/kvclient/rangefeed/rangefeedcache/cache_test.go b/pkg/kv/kvclient/rangefeed/rangefeedcache/cache_test.go index d74b642a3fe7..95cd04ed93bf 100644 --- a/pkg/kv/kvclient/rangefeed/rangefeedcache/cache_test.go +++ b/pkg/kv/kvclient/rangefeed/rangefeedcache/cache_test.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedcache" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" @@ -66,13 +67,13 @@ func TestCache(t *testing.T) { readRowsAt := func(t *testing.T, ts hlc.Timestamp) []roachpb.KeyValue { txn := kvDB.NewTxn(ctx, "test") require.NoError(t, txn.SetFixedTimestamp(ctx, ts)) - ba := &roachpb.BatchRequest{} - ba.Add(&roachpb.ScanRequest{ - RequestHeader: roachpb.RequestHeader{ + ba := &kvpb.BatchRequest{} + ba.Add(&kvpb.ScanRequest{ + RequestHeader: kvpb.RequestHeader{ Key: scratch, EndKey: scratchSpan.EndKey, }, - ScanFormat: roachpb.KEY_VALUES, + ScanFormat: kvpb.KEY_VALUES, }) br, pErr := txn.Send(ctx, ba) require.NoError(t, pErr.GoError()) diff --git a/pkg/kv/kvclient/rangefeed/rangefeedcache/watcher.go b/pkg/kv/kvclient/rangefeed/rangefeedcache/watcher.go index 34182d8df0ce..9bfec6f0f3d3 100644 --- a/pkg/kv/kvclient/rangefeed/rangefeedcache/watcher.go +++ b/pkg/kv/kvclient/rangefeed/rangefeedcache/watcher.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedbuffer" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/grpcutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -103,7 +104,7 @@ func (u UpdateType) String() string { // TranslateEventFunc is used by the client to translate a low-level event // into an event for buffering. If nil is returned, the event is skipped. type TranslateEventFunc func( - context.Context, *roachpb.RangeFeedValue, + context.Context, *kvpb.RangeFeedValue, ) rangefeedbuffer.Event // OnUpdateFunc is used by the client to receive an Update, which is a batch @@ -256,7 +257,7 @@ func (s *Watcher) Run(ctx context.Context) error { mu.Unlock() }() - onValue := func(ctx context.Context, ev *roachpb.RangeFeedValue) { + onValue := func(ctx context.Context, ev *kvpb.RangeFeedValue) { bEv := s.translateEvent(ctx, ev) if bEv == nil { return diff --git a/pkg/kv/kvclient/rangefeed/scanner.go b/pkg/kv/kvclient/rangefeed/scanner.go index 93e87cc87ff2..a1cef452bc43 100644 --- a/pkg/kv/kvclient/rangefeed/scanner.go +++ b/pkg/kv/kvclient/rangefeed/scanner.go @@ -13,6 +13,7 @@ package rangefeed import ( "context" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -30,7 +31,7 @@ func (f *RangeFeed) runInitialScan( ctx context.Context, n *log.EveryN, r *retry.Retry, ) (canceled bool) { onValue := func(kv roachpb.KeyValue) { - v := roachpb.RangeFeedValue{ + v := kvpb.RangeFeedValue{ Key: kv.Key, Value: kv.Value, } diff --git a/pkg/kv/kvclient/rangestats/BUILD.bazel b/pkg/kv/kvclient/rangestats/BUILD.bazel index 7ea99deb0865..77353804cac9 100644 --- a/pkg/kv/kvclient/rangestats/BUILD.bazel +++ b/pkg/kv/kvclient/rangestats/BUILD.bazel @@ -8,6 +8,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/sql/sem/eval", ], diff --git a/pkg/kv/kvclient/rangestats/fetcher.go b/pkg/kv/kvclient/rangestats/fetcher.go index 7ac8dc89adbf..7078205132d6 100644 --- a/pkg/kv/kvclient/rangestats/fetcher.go +++ b/pkg/kv/kvclient/rangestats/fetcher.go @@ -15,6 +15,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" ) @@ -34,9 +35,9 @@ func NewFetcher(db *kv.DB) *Fetcher { // RangeStats is part of the eval.RangeStatsFetcher interface. func (f Fetcher) RangeStats( ctx context.Context, keys ...roachpb.Key, -) ([]*roachpb.RangeStatsResponse, error) { +) ([]*kvpb.RangeStatsResponse, error) { var ba kv.Batch - reqs := make([]roachpb.RangeStatsRequest, len(keys)) + reqs := make([]kvpb.RangeStatsRequest, len(keys)) for i, k := range keys { reqs[i].Key = k ba.AddRawRequest(&reqs[i]) @@ -44,9 +45,9 @@ func (f Fetcher) RangeStats( if err := f.db.Run(ctx, &ba); err != nil { return nil, err } - resps := make([]*roachpb.RangeStatsResponse, len(keys)) + resps := make([]*kvpb.RangeStatsResponse, len(keys)) for i, r := range ba.RawResponse().Responses { - resps[i] = r.GetInner().(*roachpb.RangeStatsResponse) + resps[i] = r.GetInner().(*kvpb.RangeStatsResponse) } return resps, nil } diff --git a/pkg/kv/kvclient/revision_reader.go b/pkg/kv/kvclient/revision_reader.go index 7f33c4c99d66..f4ed3896e82b 100644 --- a/pkg/kv/kvclient/revision_reader.go +++ b/pkg/kv/kvclient/revision_reader.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -34,11 +35,11 @@ func GetAllRevisions( ctx context.Context, db *kv.DB, startKey, endKey roachpb.Key, startTime, endTime hlc.Timestamp, ) ([]VersionedValues, error) { // TODO(dt): version check. - header := roachpb.Header{Timestamp: endTime} - req := &roachpb.ExportRequest{ - RequestHeader: roachpb.RequestHeader{Key: startKey, EndKey: endKey}, + header := kvpb.Header{Timestamp: endTime} + req := &kvpb.ExportRequest{ + RequestHeader: kvpb.RequestHeader{Key: startKey, EndKey: endKey}, StartTime: startTime, - MVCCFilter: roachpb.MVCCFilter_All, + MVCCFilter: kvpb.MVCCFilter_All, } resp, pErr := kv.SendWrappedWith(ctx, db.NonTransactionalSender(), header, req) if pErr != nil { @@ -46,7 +47,7 @@ func GetAllRevisions( } var res []VersionedValues - for _, file := range resp.(*roachpb.ExportResponse).Files { + for _, file := range resp.(*kvpb.ExportResponse).Files { iterOpts := storage.IterOptions{ KeyTypes: storage.IterKeyTypePointsOnly, LowerBound: file.Span.Key, diff --git a/pkg/kv/kvnemesis/BUILD.bazel b/pkg/kv/kvnemesis/BUILD.bazel index dc28f1d218f8..1cb1c8bb1858 100644 --- a/pkg/kv/kvnemesis/BUILD.bazel +++ b/pkg/kv/kvnemesis/BUILD.bazel @@ -26,6 +26,7 @@ go_library( "//pkg/kv", "//pkg/kv/kvclient/kvcoord", "//pkg/kv/kvnemesis/kvnemesisutil", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/kv/kvserver/liveness", "//pkg/roachpb", @@ -74,6 +75,7 @@ go_test( "//pkg/kv", "//pkg/kv/kvclient/kvcoord", "//pkg/kv/kvnemesis/kvnemesisutil", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/roachpb", "//pkg/security/securityassets", @@ -108,6 +110,7 @@ proto_library( strip_import_prefix = "/pkg", visibility = ["//visibility:public"], deps = [ + "//pkg/kv/kvpb:kvpb_proto", "//pkg/roachpb:roachpb_proto", "//pkg/util/hlc:hlc_proto", "@com_github_cockroachdb_errors//errorspb:errorspb_proto", @@ -122,6 +125,7 @@ go_proto_library( proto = ":kvnemesis_proto", visibility = ["//visibility:public"], deps = [ + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/util/hlc", "@com_github_cockroachdb_errors//errorspb", diff --git a/pkg/kv/kvnemesis/applier.go b/pkg/kv/kvnemesis/applier.go index 4bede89b0012..5cf894ddd88f 100644 --- a/pkg/kv/kvnemesis/applier.go +++ b/pkg/kv/kvnemesis/applier.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil" + kvpb "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -93,15 +94,15 @@ func exceptRollback(err error) bool { // true if intentional txn rollback } func exceptRetry(err error) bool { // true if retry error - return errors.HasInterface(err, (*roachpb.ClientVisibleRetryError)(nil)) + return errors.HasInterface(err, (*kvpb.ClientVisibleRetryError)(nil)) } func exceptUnhandledRetry(err error) bool { - return errors.HasType(err, (*roachpb.UnhandledRetryableError)(nil)) + return errors.HasType(err, (*kvpb.UnhandledRetryableError)(nil)) } func exceptAmbiguous(err error) bool { // true if ambiguous result - return errors.HasInterface(err, (*roachpb.ClientVisibleAmbiguousError)(nil)) + return errors.HasInterface(err, (*kvpb.ClientVisibleAmbiguousError)(nil)) } func exceptDelRangeUsingTombstoneStraddlesRangeBoundary(err error) bool { @@ -353,8 +354,8 @@ func applyClientOp(ctx context.Context, db clientI, op *Operation, inTxn bool) { // directly through to storage, including an MVCCValueHeader already // tagged with the sequence number. We therefore don't need to pass the // sequence number via the RequestHeader here. - b.AddRawRequest(&roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{ + b.AddRawRequest(&kvpb.AddSSTableRequest{ + RequestHeader: kvpb.RequestHeader{ Key: o.Span.Key, EndKey: o.Span.EndKey, }, @@ -501,7 +502,7 @@ func getRangeDesc(ctx context.Context, key roachpb.Key, dbs ...*kv.DB) roachpb.R var opts = retry.Options{} for r := retry.StartWithCtx(ctx, opts); r.Next(); dbIdx = (dbIdx + 1) % len(dbs) { sender := dbs[dbIdx].NonTransactionalSender() - descs, _, err := kv.RangeLookup(ctx, sender, key, roachpb.CONSISTENT, 0, false) + descs, _, err := kv.RangeLookup(ctx, sender, key, kvpb.CONSISTENT, 0, false) if err != nil { log.Infof(ctx, "looking up descriptor for %s: %+v", key, err) continue diff --git a/pkg/kv/kvnemesis/env.go b/pkg/kv/kvnemesis/env.go index 9f8e3cff737c..1f6112df1b2c 100644 --- a/pkg/kv/kvnemesis/env.go +++ b/pkg/kv/kvnemesis/env.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach-go/v2/crdb" "github.com/cockroachdb/cockroach/pkg/config/zonepb" + kvpb "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/errors" @@ -72,7 +73,7 @@ func (e *Env) CheckConsistency(ctx context.Context, span roachpb.Span) []error { // discrepancy in SysBytes. This hasn't been investigated, but it's not // critical so we ignore it for now. See: // https://github.com/cockroachdb/cockroach/issues/93896 - if status == roachpb.CheckConsistencyResponse_RANGE_CONSISTENT_STATS_INCORRECT.String() { + if status == kvpb.CheckConsistencyResponse_RANGE_CONSISTENT_STATS_INCORRECT.String() { m := regexp.MustCompile(`.*\ndelta \(stats-computed\): \{(.*)\}`).FindStringSubmatch(detail) if len(m) > 1 { delta := m[1] @@ -85,11 +86,11 @@ func (e *Env) CheckConsistency(ctx context.Context, span roachpb.Span) []error { } } switch status { - case roachpb.CheckConsistencyResponse_RANGE_INDETERMINATE.String(): + case kvpb.CheckConsistencyResponse_RANGE_INDETERMINATE.String(): // Can't do anything, so let it slide. - case roachpb.CheckConsistencyResponse_RANGE_CONSISTENT.String(): + case kvpb.CheckConsistencyResponse_RANGE_CONSISTENT.String(): // Good. - case roachpb.CheckConsistencyResponse_RANGE_CONSISTENT_STATS_ESTIMATED.String(): + case kvpb.CheckConsistencyResponse_RANGE_CONSISTENT_STATS_ESTIMATED.String(): // Ok. default: failures = append(failures, errors.Errorf("range %d (%s) %s:\n%s", rangeID, key, status, detail)) diff --git a/pkg/kv/kvnemesis/generator.go b/pkg/kv/kvnemesis/generator.go index d15cf53bf7be..e8a126aeb1d0 100644 --- a/pkg/kv/kvnemesis/generator.go +++ b/pkg/kv/kvnemesis/generator.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil" + kvpb "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog/bootstrap" @@ -722,7 +723,7 @@ func randMergeIsSplit(g *generator, rng *rand.Rand) Operation { func makeRemoveReplicaFn(key string, current []roachpb.ReplicationTarget) opGenFunc { return func(g *generator, rng *rand.Rand) Operation { - change := roachpb.ReplicationChange{ + change := kvpb.ReplicationChange{ ChangeType: roachpb.REMOVE_VOTER, Target: current[rng.Intn(len(current))], } @@ -745,12 +746,12 @@ func makeAddReplicaFn(key string, current []roachpb.ReplicationTarget, atomicSwa candidates = append(candidates, candidate) } candidate := candidates[rng.Intn(len(candidates))] - changes := []roachpb.ReplicationChange{{ + changes := []kvpb.ReplicationChange{{ ChangeType: roachpb.ADD_VOTER, Target: candidate, }} if atomicSwap { - changes = append(changes, roachpb.ReplicationChange{ + changes = append(changes, kvpb.ReplicationChange{ ChangeType: roachpb.REMOVE_VOTER, Target: current[rng.Intn(len(current))], }) @@ -1010,7 +1011,7 @@ func merge(key string) Operation { return Operation{Merge: &MergeOperation{Key: []byte(key)}} } -func changeReplicas(key string, changes ...roachpb.ReplicationChange) Operation { +func changeReplicas(key string, changes ...kvpb.ReplicationChange) Operation { return Operation{ChangeReplicas: &ChangeReplicasOperation{Key: []byte(key), Changes: changes}} } diff --git a/pkg/kv/kvnemesis/kvnemesis_test.go b/pkg/kv/kvnemesis/kvnemesis_test.go index f4335d3bdcfd..0ff00a0e8134 100644 --- a/pkg/kv/kvnemesis/kvnemesis_test.go +++ b/pkg/kv/kvnemesis/kvnemesis_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" @@ -63,13 +64,13 @@ func testClusterArgs(tr *SeqTracker) base.TestClusterArgs { // would add complexity to kvnemesis that isn't worth it. Instead, the operation // generator for the most part tries to avoid range-spanning requests, and the // ones that do end up happening get a hard error. - OnRangeSpanningNonTxnalBatch: func(ba *roachpb.BatchRequest) *roachpb.Error { + OnRangeSpanningNonTxnalBatch: func(ba *kvpb.BatchRequest) *kvpb.Error { for _, req := range ba.Requests { - if req.GetInner().Method() != roachpb.DeleteRange { + if req.GetInner().Method() != kvpb.DeleteRange { continue } if req.GetDeleteRange().UseRangeTombstone == true { - return roachpb.NewError(errDelRangeUsingTombstoneStraddlesRangeBoundary) + return kvpb.NewError(errDelRangeUsingTombstoneStraddlesRangeBoundary) } } return nil @@ -168,7 +169,7 @@ func testKVNemesisImpl(t *testing.T, cfg kvnemesisTestCfg) { skip.UnderRace(t) if !buildutil.CrdbTestBuild { - // `roachpb.RequestHeader` and `MVCCValueHeader` have a KVNemesisSeq field + // `kvpb.RequestHeader` and `MVCCValueHeader` have a KVNemesisSeq field // that is zero-sized outside test builds. We could revisit that should // a need arise to run kvnemesis against production binaries. skip.IgnoreLint(t, "kvnemesis must be run with the crdb_test build tag") diff --git a/pkg/kv/kvnemesis/operations.proto b/pkg/kv/kvnemesis/operations.proto index f16f32177ae1..9d74ffd5e8c6 100644 --- a/pkg/kv/kvnemesis/operations.proto +++ b/pkg/kv/kvnemesis/operations.proto @@ -14,7 +14,7 @@ option go_package = "kvnemesis"; import "errorspb/errors.proto"; import "gogoproto/gogo.proto"; -import "roachpb/api.proto"; +import "kv/kvpb/api.proto"; import "roachpb/data.proto"; import "util/hlc/timestamp.proto"; diff --git a/pkg/kv/kvnemesis/validator.go b/pkg/kv/kvnemesis/validator.go index 39895f175347..a0dfad837b51 100644 --- a/pkg/kv/kvnemesis/validator.go +++ b/pkg/kv/kvnemesis/validator.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil" + kvpb "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -563,7 +564,7 @@ func (v *validator) processOp(op Operation) { v.checkAtomic(`deleteRangeUsingTombstone`, t.Result) } case *AddSSTableOperation: - if resultHasErrorType(t.Result, &roachpb.RangeKeyMismatchError{}) { + if resultHasErrorType(t.Result, &kvpb.RangeKeyMismatchError{}) { // The AddSSTable may race with a range split. It's not possible to ingest // an SST spanning multiple ranges, but the generator will optimistically // try to fit the SST inside one of the current ranges, so we ignore the @@ -696,7 +697,7 @@ func (v *validator) processOp(op Operation) { } else if resultIsErrorStr(t.Result, `merge failed: unexpected value`) { // TODO(dan): If this error is going to remain a part of the kv API, we // should make it sniffable with errors.As. Currently this seems to be - // broken by wrapping it with `roachpb.NewErrorf("merge failed: %s", + // broken by wrapping it with `kvpb.NewErrorf("merge failed: %s", // err)`. // // However, I think the right thing to do is sniff this inside the @@ -1220,7 +1221,7 @@ func exceptLivenessCacheMiss(err error) bool { func resultIsAmbiguous(r Result) bool { resErr := errorFromResult(r) - hasClientVisibleAE := errors.HasInterface(resErr, (*roachpb.ClientVisibleAmbiguousError)(nil)) + hasClientVisibleAE := errors.HasInterface(resErr, (*kvpb.ClientVisibleAmbiguousError)(nil)) return hasClientVisibleAE } diff --git a/pkg/kv/kvnemesis/validator_test.go b/pkg/kv/kvnemesis/validator_test.go index fe080061255c..a5d3707d90cb 100644 --- a/pkg/kv/kvnemesis/validator_test.go +++ b/pkg/kv/kvnemesis/validator_test.go @@ -17,6 +17,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil" + kvpb "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage" @@ -33,7 +34,7 @@ import ( "github.com/stretchr/testify/require" ) -var retryableError = roachpb.NewTransactionRetryWithProtoRefreshError( +var retryableError = kvpb.NewTransactionRetryWithProtoRefreshError( ``, uuid.MakeV4(), roachpb.Transaction{}) func withTimestamp(op Operation, ts int) Operation { @@ -54,7 +55,7 @@ func withResult(op Operation) Operation { } func withAmbResult(op Operation) Operation { - err := roachpb.NewAmbiguousResultErrorf("boom") + err := kvpb.NewAmbiguousResultErrorf("boom") op = withResultErr(op, err) return op } @@ -113,7 +114,7 @@ func TestValidate(t *testing.T) { defer log.Scope(t).Close(t) if !buildutil.CrdbTestBuild { - // `roachpb.RequestHeader` and `MVCCValueHeader` have a KVNemesisSeq field + // `kvpb.RequestHeader` and `MVCCValueHeader` have a KVNemesisSeq field // that is zero-sized outside test builds. We could revisit that should // a need arise to run kvnemesis against production binaries. skip.IgnoreLint(t, "kvnemesis must be run with the crdb_test build tag") diff --git a/pkg/kv/kvnemesis/watcher.go b/pkg/kv/kvnemesis/watcher.go index 6737372b66ee..e4dd094bf964 100644 --- a/pkg/kv/kvnemesis/watcher.go +++ b/pkg/kv/kvnemesis/watcher.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil" + kvpb "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" @@ -161,21 +162,21 @@ func (w *Watcher) processEvents(ctx context.Context, eventC chan kvcoord.RangeFe return nil case event := <-eventC: switch e := event.GetValue().(type) { - case *roachpb.RangeFeedError: + case *kvpb.RangeFeedError: return e.Error.GoError() - case *roachpb.RangeFeedValue: + case *kvpb.RangeFeedValue: if err := w.handleValue(ctx, roachpb.Span{Key: e.Key}, e.Value, &e.PrevValue); err != nil { return err } - case *roachpb.RangeFeedDeleteRange: + case *kvpb.RangeFeedDeleteRange: if err := w.handleValue(ctx, e.Span, roachpb.Value{Timestamp: e.Timestamp}, nil /* prevV */); err != nil { return err } - case *roachpb.RangeFeedCheckpoint: + case *kvpb.RangeFeedCheckpoint: if err := w.handleCheckpoint(ctx, e.Span, e.ResolvedTS); err != nil { return err } - case *roachpb.RangeFeedSSTable: + case *kvpb.RangeFeedSSTable: if err := w.handleSSTable(ctx, e.Data); err != nil { return err } diff --git a/pkg/kv/kvpb/.gitattributes b/pkg/kv/kvpb/.gitattributes new file mode 100644 index 000000000000..f39d346a703f --- /dev/null +++ b/pkg/kv/kvpb/.gitattributes @@ -0,0 +1,2 @@ +batch_generated.go -diff +*_string.go -diff diff --git a/pkg/kv/kvpb/BUILD.bazel b/pkg/kv/kvpb/BUILD.bazel new file mode 100644 index 000000000000..082dae48c6a1 --- /dev/null +++ b/pkg/kv/kvpb/BUILD.bazel @@ -0,0 +1,153 @@ +load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data") +load("@rules_proto//proto:defs.bzl", "proto_library") +load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//build:STRINGER.bzl", "stringer") +load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") +load(":gen.bzl", "batch_gen") + +go_library( + name = "kvpb", + srcs = [ + "ambiguous_result_error.go", + "api.go", + "api_requestheader.go", + "batch.go", + "data.go", + "errors.go", + "method.go", + "replica_unavailable_error.go", + ":gen-batch-generated", # keep + ":gen-errordetailtype-stringer", # keep + ":gen-method-stringer", # keep + ], + embed = [":kvpb_go_proto"], + importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvpb", + visibility = ["//visibility:public"], + deps = [ + "//pkg/kv/kvnemesis/kvnemesisutil", + "//pkg/kv/kvserver/concurrency/lock", + "//pkg/roachpb", + "//pkg/storage/enginepb", + "//pkg/util/buildutil", + "//pkg/util/caller", + "//pkg/util/hlc", + "//pkg/util/humanizeutil", + "//pkg/util/log", + "//pkg/util/protoutil", + "//pkg/util/uuid", + "@com_github_cockroachdb_errors//:errors", + "@com_github_cockroachdb_errors//extgrpc", + "@com_github_cockroachdb_redact//:redact", + "@com_github_dustin_go_humanize//:go-humanize", + "@com_github_gogo_protobuf//proto", + "@com_github_golang_mock//gomock", # keep + "@org_golang_google_grpc//metadata", # keep + ], +) + +go_test( + name = "kvpb_test", + srcs = [ + "api_test.go", + "batch_test.go", + "errors_test.go", + "replica_unavailable_error_test.go", + "string_test.go", + ], + args = ["-test.timeout=295s"], + data = glob(["testdata/**"]), + embed = [":kvpb"], + deps = [ + "//pkg/keys", + "//pkg/kv/kvnemesis/kvnemesisutil", + "//pkg/kv/kvserver/concurrency/lock", + "//pkg/roachpb", + "//pkg/storage/enginepb", + "//pkg/testutils/echotest", + "//pkg/util/buildutil", + "//pkg/util/hlc", + "//pkg/util/protoutil", + "//pkg/util/uuid", + "@com_github_cockroachdb_errors//:errors", + "@com_github_cockroachdb_redact//:redact", + "@com_github_gogo_protobuf//proto", + "@com_github_golang_protobuf//proto:go_default_library", + "@com_github_kr_pretty//:pretty", + "@com_github_stretchr_testify//require", + "@org_golang_google_grpc//codes", + "@org_golang_google_grpc//status", + ], +) + +proto_library( + name = "kvpb_proto", + srcs = [ + "api.proto", + "errors.proto", + ], + strip_import_prefix = "/pkg", + visibility = ["//visibility:public"], + deps = [ + "//pkg/kv/kvserver/concurrency/lock:lock_proto", + "//pkg/kv/kvserver/readsummary/rspb:rspb_proto", + "//pkg/roachpb:roachpb_proto", + "//pkg/settings:settings_proto", + "//pkg/sql/catalog/fetchpb:fetchpb_proto", + "//pkg/storage/enginepb:enginepb_proto", + "//pkg/util/hlc:hlc_proto", + "//pkg/util/tracing/tracingpb:tracingpb_proto", + "@com_github_cockroachdb_errors//errorspb:errorspb_proto", + "@com_github_gogo_protobuf//gogoproto:gogo_proto", + "@com_google_protobuf//:duration_proto", + ], +) + +go_proto_library( + name = "kvpb_go_proto", + compilers = ["//pkg/cmd/protoc-gen-gogoroach:protoc-gen-gogoroach_grpc_compiler"], + importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvpb", + proto = ":kvpb_proto", + visibility = ["//visibility:public"], + deps = [ + "//pkg/kv/kvserver/concurrency/lock", + "//pkg/kv/kvserver/readsummary/rspb", + "//pkg/roachpb", + "//pkg/settings", + "//pkg/sql/catalog/fetchpb", + "//pkg/storage/enginepb", + "//pkg/util/hlc", + "//pkg/util/tracing/tracingpb", + "@com_github_cockroachdb_errors//errorspb", + "@com_github_gogo_protobuf//gogoproto", + ], +) + +stringer( + name = "gen-method-stringer", + src = "method.go", + typ = "Method", +) + +stringer( + name = "gen-errordetailtype-stringer", + src = "errors.go", + typ = "ErrorDetailType", +) + +batch_gen( + name = "gen-batch-generated", + src = ":kvpb_go_proto", + out = "batch_generated.go", + visibility = [ + ":__pkg__", + "//pkg/gen:__pkg__", + ], +) + +disallowed_imports_test( + "kvpb", + disallow_cdeps = True, +) + +get_x_data(name = "get_x_data") diff --git a/pkg/roachpb/ambiguous_result_error.go b/pkg/kv/kvpb/ambiguous_result_error.go similarity index 84% rename from pkg/roachpb/ambiguous_result_error.go rename to pkg/kv/kvpb/ambiguous_result_error.go index 6de969cb70f3..e347acc5459d 100644 --- a/pkg/roachpb/ambiguous_result_error.go +++ b/pkg/kv/kvpb/ambiguous_result_error.go @@ -8,12 +8,14 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package roachpb +package kvpb import ( "context" "fmt" + "reflect" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/errors" "github.com/gogo/protobuf/proto" ) @@ -34,7 +36,14 @@ func NewAmbiguousResultError(err error) *AmbiguousResultError { var _ errors.SafeFormatter = (*AmbiguousResultError)(nil) var _ fmt.Formatter = (*AmbiguousResultError)(nil) -var _ = func() errors.Wrapper { + +func init() { + // Register the migration of the error that used to be in the roachpb + // package and is now in the kv/kvpb package. + roachpbPath := reflect.TypeOf(roachpb.Key("")).PkgPath() + errors.RegisterTypeMigration(roachpbPath, "*roachpb.AmbiguousResultError", &AmbiguousResultError{}) + // Note that it is important that these wrapper methods are registered + // _after_ the type migration above. aErr := (*AmbiguousResultError)(nil) typeKey := errors.GetTypeKey(aErr) errors.RegisterWrapperEncoder(typeKey, func(ctx context.Context, err error) (msgPrefix string, safeDetails []string, payload proto.Message) { @@ -44,9 +53,7 @@ var _ = func() errors.Wrapper { errors.RegisterWrapperDecoder(typeKey, func(ctx context.Context, cause error, msgPrefix string, safeDetails []string, payload proto.Message) error { return payload.(*AmbiguousResultError) }) - - return aErr -}() +} // SafeFormatError implements errors.SafeFormatter. func (e *AmbiguousResultError) SafeFormatError(p errors.Printer) error { diff --git a/pkg/roachpb/api.go b/pkg/kv/kvpb/api.go similarity index 94% rename from pkg/roachpb/api.go rename to pkg/kv/kvpb/api.go index f84f1559505b..e5c2b2167912 100644 --- a/pkg/roachpb/api.go +++ b/pkg/kv/kvpb/api.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package roachpb +package kvpb import ( "context" @@ -16,6 +16,7 @@ import ( _ "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil" // see RequestHeader "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -25,36 +26,7 @@ import ( "github.com/dustin/go-humanize" ) -//go:generate mockgen -package=roachpbmock -destination=roachpbmock/mocks_generated.go . InternalClient,Internal_RangeFeedClient,Internal_MuxRangeFeedClient - -// UserPriority is a custom type for transaction's user priority. -type UserPriority float64 - -func (up UserPriority) String() string { - switch up { - case MinUserPriority: - return "low" - case UnspecifiedUserPriority, NormalUserPriority: - return "normal" - case MaxUserPriority: - return "high" - default: - return fmt.Sprintf("%g", float64(up)) - } -} - -const ( - // MinUserPriority is the minimum allowed user priority. - MinUserPriority UserPriority = 0.001 - // UnspecifiedUserPriority means NormalUserPriority. - UnspecifiedUserPriority UserPriority = 0 - // NormalUserPriority is set to 1, meaning ops run through the database - // are all given equal weight when a random priority is chosen. This can - // be set specifically via client.NewDBWithPriority(). - NormalUserPriority UserPriority = 1 - // MaxUserPriority is the maximum allowed user priority. - MaxUserPriority UserPriority = 1000 -) +//go:generate mockgen -package=kvpbmock -destination=kvpbmock/mocks_generated.go . InternalClient,Internal_RangeFeedClient,Internal_MuxRangeFeedClient // SupportsBatch determines whether the methods in the provided batch // are supported by the ReadConsistencyType, returning an error if not. @@ -568,19 +540,19 @@ func (rh *RequestHeader) SetHeader(other RequestHeader) { } // Span returns the key range that the Request operates over. -func (rh RequestHeader) Span() Span { - return Span{Key: rh.Key, EndKey: rh.EndKey} +func (rh RequestHeader) Span() roachpb.Span { + return roachpb.Span{Key: rh.Key, EndKey: rh.EndKey} } // SetSpan addresses the RequestHeader to the specified key span. -func (rh *RequestHeader) SetSpan(s Span) { +func (rh *RequestHeader) SetSpan(s roachpb.Span) { rh.Key = s.Key rh.EndKey = s.EndKey } // RequestHeaderFromSpan creates a RequestHeader addressed at the specified key // span. -func RequestHeaderFromSpan(s Span) RequestHeader { +func RequestHeaderFromSpan(s roachpb.Span) RequestHeader { return RequestHeader{Key: s.Key, EndKey: s.EndKey} } @@ -783,7 +755,9 @@ func (*SubsumeRequest) Method() Method { return Subsume } func (*RangeStatsRequest) Method() Method { return RangeStats } // Method implements the Request interface. -func (*AdminVerifyProtectedTimestampRequest) Method() Method { return AdminVerifyProtectedTimestamp } +func (*AdminVerifyProtectedTimestampRequest) Method() Method { + return AdminVerifyProtectedTimestamp +} // Method implements the Request interface. func (*QueryResolvedTimestampRequest) Method() Method { return QueryResolvedTimestamp } @@ -1085,7 +1059,7 @@ func (r *IsSpanEmptyRequest) ShallowCopy() Request { // NewGet returns a Request initialized to get the value at key. If // forUpdate is true, an unreplicated, exclusive lock is acquired on on // the key, if it exists. -func NewGet(key Key, forUpdate bool) Request { +func NewGet(key roachpb.Key, forUpdate bool) Request { return &GetRequest{ RequestHeader: RequestHeader{ Key: key, @@ -1096,7 +1070,7 @@ func NewGet(key Key, forUpdate bool) Request { // NewIncrement returns a Request initialized to increment the value at // key by increment. -func NewIncrement(key Key, increment int64) Request { +func NewIncrement(key roachpb.Key, increment int64) Request { return &IncrementRequest{ RequestHeader: RequestHeader{ Key: key, @@ -1106,7 +1080,7 @@ func NewIncrement(key Key, increment int64) Request { } // NewPut returns a Request initialized to put the value at key. -func NewPut(key Key, value Value) Request { +func NewPut(key roachpb.Key, value roachpb.Value) Request { value.InitChecksum(key) return &PutRequest{ RequestHeader: RequestHeader{ @@ -1118,7 +1092,7 @@ func NewPut(key Key, value Value) Request { // NewPutInline returns a Request initialized to put the value at key // using an inline value. -func NewPutInline(key Key, value Value) Request { +func NewPutInline(key roachpb.Key, value roachpb.Value) Request { value.InitChecksum(key) return &PutRequest{ RequestHeader: RequestHeader{ @@ -1135,7 +1109,9 @@ func NewPutInline(key Key, value Value) Request { // The callee takes ownership of value's underlying bytes and it will mutate // them. The caller retains ownership of expVal; NewConditionalPut will copy it // into the request. -func NewConditionalPut(key Key, value Value, expValue []byte, allowNotExist bool) Request { +func NewConditionalPut( + key roachpb.Key, value roachpb.Value, expValue []byte, allowNotExist bool, +) Request { value.InitChecksum(key) return &ConditionalPutRequest{ RequestHeader: RequestHeader{ @@ -1153,7 +1129,9 @@ func NewConditionalPut(key Key, value Value, expValue []byte, allowNotExist bool // The callee takes ownership of value's underlying bytes and it will mutate // them. The caller retains ownership of expVal; NewConditionalPut will copy it // into the request. -func NewConditionalPutInline(key Key, value Value, expValue []byte, allowNotExist bool) Request { +func NewConditionalPutInline( + key roachpb.Key, value roachpb.Value, expValue []byte, allowNotExist bool, +) Request { value.InitChecksum(key) return &ConditionalPutRequest{ RequestHeader: RequestHeader{ @@ -1171,7 +1149,7 @@ func NewConditionalPutInline(key Key, value Value, expValue []byte, allowNotExis // the existing value is different from value. If failOnTombstones is set to // true, tombstones count as mismatched values and will cause a // ConditionFailedError. -func NewInitPut(key Key, value Value, failOnTombstones bool) Request { +func NewInitPut(key roachpb.Key, value roachpb.Value, failOnTombstones bool) Request { value.InitChecksum(key) return &InitPutRequest{ RequestHeader: RequestHeader{ @@ -1183,7 +1161,7 @@ func NewInitPut(key Key, value Value, failOnTombstones bool) Request { } // NewDelete returns a Request initialized to delete the value at key. -func NewDelete(key Key) Request { +func NewDelete(key roachpb.Key) Request { return &DeleteRequest{ RequestHeader: RequestHeader{ Key: key, @@ -1193,7 +1171,7 @@ func NewDelete(key Key) Request { // NewDeleteRange returns a Request initialized to delete the values in // the given key range (excluding the endpoint). -func NewDeleteRange(startKey, endKey Key, returnKeys bool) Request { +func NewDeleteRange(startKey, endKey roachpb.Key, returnKeys bool) Request { return &DeleteRangeRequest{ RequestHeader: RequestHeader{ Key: startKey, @@ -1206,7 +1184,7 @@ func NewDeleteRange(startKey, endKey Key, returnKeys bool) Request { // NewScan returns a Request initialized to scan from start to end keys. // If forUpdate is true, unreplicated, exclusive locks are acquired on // each of the resulting keys. -func NewScan(key, endKey Key, forUpdate bool) Request { +func NewScan(key, endKey roachpb.Key, forUpdate bool) Request { return &ScanRequest{ RequestHeader: RequestHeader{ Key: key, @@ -1219,7 +1197,7 @@ func NewScan(key, endKey Key, forUpdate bool) Request { // NewReverseScan returns a Request initialized to reverse scan from end. // If forUpdate is true, unreplicated, exclusive locks are acquired on // each of the resulting keys. -func NewReverseScan(key, endKey Key, forUpdate bool) Request { +func NewReverseScan(key, endKey roachpb.Key, forUpdate bool) Request { return &ReverseScanRequest{ RequestHeader: RequestHeader{ Key: key, @@ -1524,7 +1502,7 @@ func (e *RangeFeedValue) Timestamp() hlc.Timestamp { // MakeReplicationChanges returns a slice of changes of the given type with an // item for each target. func MakeReplicationChanges( - changeType ReplicaChangeType, targets ...ReplicationTarget, + changeType roachpb.ReplicaChangeType, targets ...roachpb.ReplicationTarget, ) []ReplicationChange { chgs := make([]ReplicationChange, 0, len(targets)) for _, target := range targets { @@ -1538,17 +1516,17 @@ func MakeReplicationChanges( // ReplicationChangesForPromotion returns the replication changes that // correspond to the promotion of a non-voter to a voter. -func ReplicationChangesForPromotion(target ReplicationTarget) []ReplicationChange { +func ReplicationChangesForPromotion(target roachpb.ReplicationTarget) []ReplicationChange { return []ReplicationChange{ - {ChangeType: ADD_VOTER, Target: target}, {ChangeType: REMOVE_NON_VOTER, Target: target}, + {ChangeType: roachpb.ADD_VOTER, Target: target}, {ChangeType: roachpb.REMOVE_NON_VOTER, Target: target}, } } // ReplicationChangesForDemotion returns the replication changes that correspond // to the demotion of a voter to a non-voter. -func ReplicationChangesForDemotion(target ReplicationTarget) []ReplicationChange { +func ReplicationChangesForDemotion(target roachpb.ReplicationTarget) []ReplicationChange { return []ReplicationChange{ - {ChangeType: ADD_NON_VOTER, Target: target}, {ChangeType: REMOVE_VOTER, Target: target}, + {ChangeType: roachpb.ADD_NON_VOTER, Target: target}, {ChangeType: roachpb.REMOVE_VOTER, Target: target}, } } @@ -1566,8 +1544,8 @@ func (acrr *AdminChangeReplicasRequest) AddChanges(chgs ...ReplicationChange) { // ReplicationChanges is a slice of ReplicationChange. type ReplicationChanges []ReplicationChange -func (rc ReplicationChanges) byType(typ ReplicaChangeType) []ReplicationTarget { - var sl []ReplicationTarget +func (rc ReplicationChanges) byType(typ roachpb.ReplicaChangeType) []roachpb.ReplicationTarget { + var sl []roachpb.ReplicationTarget for _, chg := range rc { if chg.ChangeType == typ { sl = append(sl, chg.Target) @@ -1577,25 +1555,25 @@ func (rc ReplicationChanges) byType(typ ReplicaChangeType) []ReplicationTarget { } // VoterAdditions returns a slice of all contained replication changes that add replicas. -func (rc ReplicationChanges) VoterAdditions() []ReplicationTarget { - return rc.byType(ADD_VOTER) +func (rc ReplicationChanges) VoterAdditions() []roachpb.ReplicationTarget { + return rc.byType(roachpb.ADD_VOTER) } // VoterRemovals returns a slice of all contained replication changes that remove replicas. -func (rc ReplicationChanges) VoterRemovals() []ReplicationTarget { - return rc.byType(REMOVE_VOTER) +func (rc ReplicationChanges) VoterRemovals() []roachpb.ReplicationTarget { + return rc.byType(roachpb.REMOVE_VOTER) } // NonVoterAdditions returns a slice of all contained replication // changes that add non-voters. -func (rc ReplicationChanges) NonVoterAdditions() []ReplicationTarget { - return rc.byType(ADD_NON_VOTER) +func (rc ReplicationChanges) NonVoterAdditions() []roachpb.ReplicationTarget { + return rc.byType(roachpb.ADD_NON_VOTER) } // NonVoterRemovals returns a slice of all contained replication changes // that remove non-voters. -func (rc ReplicationChanges) NonVoterRemovals() []ReplicationTarget { - return rc.byType(REMOVE_NON_VOTER) +func (rc ReplicationChanges) NonVoterRemovals() []roachpb.ReplicationTarget { + return rc.byType(roachpb.REMOVE_NON_VOTER) } // Changes returns the changes requested by this AdminChangeReplicasRequest, taking @@ -1617,8 +1595,8 @@ func (acrr *AdminChangeReplicasRequest) Changes() []ReplicationChange { // AsLockUpdate creates a lock update message corresponding to the given resolve // intent request. -func (rir *ResolveIntentRequest) AsLockUpdate() LockUpdate { - return LockUpdate{ +func (rir *ResolveIntentRequest) AsLockUpdate() roachpb.LockUpdate { + return roachpb.LockUpdate{ Span: rir.Span(), Txn: rir.IntentTxn, Status: rir.Status, @@ -1629,8 +1607,8 @@ func (rir *ResolveIntentRequest) AsLockUpdate() LockUpdate { // AsLockUpdate creates a lock update message corresponding to the given resolve // intent range request. -func (rirr *ResolveIntentRangeRequest) AsLockUpdate() LockUpdate { - return LockUpdate{ +func (rirr *ResolveIntentRangeRequest) AsLockUpdate() roachpb.LockUpdate { + return roachpb.LockUpdate{ Span: rirr.Span(), Txn: rirr.IntentTxn, Status: rirr.Status, @@ -1642,14 +1620,14 @@ func (rirr *ResolveIntentRangeRequest) AsLockUpdate() LockUpdate { // CreateStoreIdent creates a store identifier out of the details captured // within the join node response (the join node RPC is used to allocate a store // ID for the client's first store). -func (r *JoinNodeResponse) CreateStoreIdent() (StoreIdent, error) { - nodeID, storeID := NodeID(r.NodeID), StoreID(r.StoreID) +func (r *JoinNodeResponse) CreateStoreIdent() (roachpb.StoreIdent, error) { + nodeID, storeID := roachpb.NodeID(r.NodeID), roachpb.StoreID(r.StoreID) clusterID, err := uuid.FromBytes(r.ClusterID) if err != nil { - return StoreIdent{}, err + return roachpb.StoreIdent{}, err } - sIdent := StoreIdent{ + sIdent := roachpb.StoreIdent{ ClusterID: clusterID, NodeID: nodeID, StoreID: storeID, diff --git a/pkg/roachpb/api.proto b/pkg/kv/kvpb/api.proto similarity index 98% rename from pkg/roachpb/api.proto rename to pkg/kv/kvpb/api.proto index 90b81624b30a..e3e0900a1618 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/kv/kvpb/api.proto @@ -10,13 +10,13 @@ syntax = "proto3"; package cockroach.roachpb; -option go_package = "roachpb"; +option go_package = "kvpb"; import "errorspb/errors.proto"; import "kv/kvserver/concurrency/lock/locking.proto"; import "kv/kvserver/readsummary/rspb/summary.proto"; import "roachpb/data.proto"; -import "roachpb/errors.proto"; +import "kv/kvpb/errors.proto"; import "roachpb/metadata.proto"; import "roachpb/span_config.proto"; import "settings/encoding.proto"; @@ -85,8 +85,8 @@ enum ResumeReason { // RequestHeaderPure is not to be used directly. It's generated only for use of // its marshaling methods by RequestHeader. See the comment there. message RequestHeaderPure { - bytes key = 3 [(gogoproto.casttype) = "Key"]; - bytes end_key = 4 [(gogoproto.casttype) = "Key"]; + bytes key = 3 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.Key"]; + bytes end_key = 4 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.Key"]; int32 sequence = 5 [ (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnSeq"]; } @@ -99,8 +99,8 @@ message RequestHeaderCrdbTest { (gogoproto.customname) = "KVNemesisSeq", (gogoproto.nullable) = false, (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil.Container"]; - bytes key = 3 [(gogoproto.casttype) = "Key"]; - bytes end_key = 4 [(gogoproto.casttype) = "Key"]; + bytes key = 3 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.Key"]; + bytes end_key = 4 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.Key"]; int32 sequence = 5 [ (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnSeq"]; } @@ -153,12 +153,12 @@ message RequestHeader { // The key for request. If the request operates on a range, this // represents the starting key for the range. - bytes key = 3 [(gogoproto.casttype) = "Key"]; + bytes key = 3 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.Key"]; // The end key is empty if the request spans only a single key. Otherwise, // it must order strictly after Key. In such a case, the header indicates // that the operation takes place on the key range from Key to EndKey, // including Key and excluding EndKey. - bytes end_key = 4 [(gogoproto.casttype) = "Key"]; + bytes end_key = 4 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.Key"]; // A zero-indexed transactional sequence number. int32 sequence = 5 [ (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnSeq"]; @@ -483,7 +483,7 @@ message DeleteRangePredicates { message DeleteRangeResponse { ResponseHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; // All the deleted keys if return_keys is set. - repeated bytes keys = 2 [(gogoproto.casttype) = "Key"]; + repeated bytes keys = 2 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.Key"]; } // A ClearRangeRequest is the argument to the ClearRange() method. It @@ -741,7 +741,8 @@ message CheckConsistencyResponse { } message Result { - int64 range_id = 1 [(gogoproto.customname) = "RangeID", (gogoproto.casttype) = "RangeID"]; + int64 range_id = 1 [(gogoproto.customname) = "RangeID", + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID"]; // start_key of the range corresponding to range_id (at the time of the // check). This is useful to send additional requests to only a subset of // ranges contained within a result later, as requests can only be routed by @@ -893,14 +894,14 @@ message EndTxnResponse { // automatic merging). message AdminSplitRequest { RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; - bytes split_key = 2 [(gogoproto.casttype) = "Key"]; + bytes split_key = 2 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.Key"]; reserved 3; util.hlc.Timestamp expiration_time = 4 [(gogoproto.nullable) = false]; // PredicateKeys specifies keys which if not contained within the range should // cause the split to be rejected. This can be used by a caller to effectively // send a "conditional split" request, i.e. a split if not already split. - repeated bytes predicate_keys = 5 [(gogoproto.casttype) = "Key"]; + repeated bytes predicate_keys = 5 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.Key"]; } // An AdminSplitResponse is the return value from the AdminSplit() @@ -951,7 +952,7 @@ message AdminMergeResponse { // replica of the range. message AdminTransferLeaseRequest { RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; - int32 target = 2 [(gogoproto.casttype) = "StoreID"]; + int32 target = 2 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.StoreID"]; // When set to true, bypass_safety_checks configures the lease transfer to // skip safety checks that ensure that the transfer target is known to be // (according to the outgoing leaseholder) alive and sufficiently caught up on @@ -1064,14 +1065,14 @@ message GCRequest { RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; message GCKey { - bytes key = 1 [(gogoproto.casttype) = "Key"]; + bytes key = 1 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.Key"]; util.hlc.Timestamp timestamp = 2 [(gogoproto.nullable) = false]; } repeated GCKey keys = 3 [(gogoproto.nullable) = false]; message GCRangeKey { - bytes start_key = 1 [(gogoproto.casttype) = "Key"]; - bytes end_key = 2 [(gogoproto.casttype) = "Key"]; + bytes start_key = 1 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.Key"]; + bytes end_key = 2 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.Key"]; util.hlc.Timestamp timestamp = 3 [(gogoproto.nullable) = false]; } repeated GCRangeKey range_keys = 6 [(gogoproto.nullable) = false]; @@ -1081,8 +1082,8 @@ message GCRequest { // GCClearRange contains a range for GC requests that would use // pebble range tombstones to delete data. message GCClearRange { - bytes start_key = 1 [(gogoproto.casttype) = "Key"]; - bytes end_key = 2 [(gogoproto.casttype) = "Key"]; + bytes start_key = 1 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.Key"]; + bytes end_key = 2 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.Key"]; util.hlc.Timestamp start_key_timestamp = 3 [(gogoproto.nullable) = false]; } // GCClearRange specifies a span that will be GCed using a Pebble range @@ -1424,7 +1425,8 @@ message TruncateLogRequest { // The header specifies a span, start and end keys, but not the range id // itself. The range may have changed from the one specified in the header // in the case of a merge. - int64 range_id = 3 [(gogoproto.customname) = "RangeID", (gogoproto.casttype) = "RangeID"]; + int64 range_id = 3 [(gogoproto.customname) = "RangeID", + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID"]; // ExpectedFirstIndex is the expected Index of the last TruncateLogRequest, // i.e., we expect that this request will typically be truncating entries @@ -1516,7 +1518,7 @@ message LeaseInfoResponse{ // possible that even if ReadConsistency=INCONSISTENT was used, the request is // still not evaluated by the node it was sent to if that node's replica is a // learner or the node doesn't have a replica at all. - int32 evaluated_by = 4 [(gogoproto.casttype) = "StoreID"]; + int32 evaluated_by = 4 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.StoreID"]; } // A RequestLeaseResponse is the response to a RequestLease() or TransferLease() @@ -1789,8 +1791,8 @@ message AdminVerifyProtectedTimestampResponse { repeated RangeDescriptor deprecated_failed_ranges = 3 [(gogoproto.nullable) = false]; message FailedRange { int64 range_id = 1 [(gogoproto.customname) = "RangeID"]; - bytes start_key = 2 [(gogoproto.casttype) = "RKey"]; - bytes end_key = 3 [(gogoproto.casttype) = "RKey"]; + bytes start_key = 2 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RKey"]; + bytes end_key = 3 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RKey"]; string reason = 4; } repeated FailedRange verification_failed_ranges = 4 [(gogoproto.nullable) = false]; @@ -1957,7 +1959,7 @@ message AddSSTableResponse { // not data before that key, or that that key actually exists; it may be // picked using cheaper, inconsistentent reads so it should be used only for // estimations of how ingested data and spans may overlap. - bytes following_likely_non_empty_span_start = 4 [(gogoproto.casttype) = "Key"]; + bytes following_likely_non_empty_span_start = 4 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.Key"]; } // RefreshRequest is arguments to the Refresh() method, which verifies that no @@ -2372,7 +2374,8 @@ message Header { // range_id specifies the ID of the Raft consensus group which the key // range belongs to. This is used by the receiving node to route the // request to the correct range. - int64 range_id = 3 [(gogoproto.customname) = "RangeID", (gogoproto.casttype) = "RangeID"]; + int64 range_id = 3 [(gogoproto.customname) = "RangeID", + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID"]; // user_priority allows any command's priority to be biased from the // default random priority. It specifies a multiple. If set to 0.5, // the chosen priority will be 1/2x as likely to beat any default @@ -2381,8 +2384,8 @@ message Header { // beat any default random priority, and so on. As a special case, 0 // priority is treated the same as 1. This value is ignored if txn // is specified. The min and max user priorities are set via - // MinUserPriority and MaxUserPriority in data.go. - double user_priority = 4 [(gogoproto.casttype) = "UserPriority"]; + // roachpb.MinUserPriority and roachpb.MaxUserPriority in roachpb/data.go. + double user_priority = 4 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.UserPriority"]; // txn is set non-nil if a transaction is underway. To start a txn, // the first request should set this field to non-nil with name and // isolation level set as desired. The response will contain the @@ -2592,7 +2595,8 @@ message Header { // gateway_node_id is the ID of the gateway node where the request originated. // For requests from tenants, this is set to the NodeID of the KV node handling // the BatchRequest. - int32 gateway_node_id = 11 [(gogoproto.customname) = "GatewayNodeID", (gogoproto.casttype) = "NodeID"]; + int32 gateway_node_id = 11 [(gogoproto.customname) = "GatewayNodeID", + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID"]; // client_range_info represents the kvclient's knowledge about the state of // the range (i.e. of the range descriptor and lease). The kvserver checks // whether the client's info is up to date and, if it isn't, it will return a @@ -2808,7 +2812,7 @@ message BatchResponse { // RangeLookupRequest is a request to proxy a RangeLookup through a Tenant // service. Its fields correspond to a subset of the args of kv.RangeLookup. message RangeLookupRequest { - bytes key = 1 [(gogoproto.casttype) = "RKey"]; + bytes key = 1 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RKey"]; ReadConsistencyType read_consistency = 2; int64 prefetch_num = 3; bool prefetch_reverse = 4; @@ -2850,7 +2854,7 @@ message RangeFeedRequest { // RangeFeedValue is a variant of RangeFeedEvent that represents an update to // the specified key with the provided value. message RangeFeedValue { - bytes key = 1 [(gogoproto.casttype) = "Key"]; + bytes key = 1 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.Key"]; Value value = 2 [(gogoproto.nullable) = false]; // prev_value is only populated if both: // 1. with_diff was passed in the corresponding RangeFeedRequest. @@ -2922,7 +2926,8 @@ message RangeFeedEvent { // the underlying RangeFeedEvent with the ID of the range that produced this event. message MuxRangeFeedEvent { RangeFeedEvent event = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; - int32 range_id = 2 [(gogoproto.customname) = "RangeID", (gogoproto.casttype) = "RangeID"]; + int32 range_id = 2 [(gogoproto.customname) = "RangeID", + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID"]; // Server echoes back stream_id set by the client. int64 stream_id = 3 [(gogoproto.customname) = "StreamID"]; } @@ -3194,7 +3199,7 @@ message ContentionEvent { option (gogoproto.goproto_stringer) = false; // Key is the key that this and the other transaction conflicted on. - bytes key = 1 [(gogoproto.casttype) = "Key"]; + bytes key = 1 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.Key"]; // TxnMeta is the transaction conflicted with, i.e. the transaction holding a // lock or lock reservation. cockroach.storage.enginepb.TxnMeta txn_meta = 2 [(gogoproto.nullable) = false]; diff --git a/pkg/roachpb/api_requestheader.go b/pkg/kv/kvpb/api_requestheader.go similarity index 99% rename from pkg/roachpb/api_requestheader.go rename to pkg/kv/kvpb/api_requestheader.go index 57892b4babb6..e12c56f96ff1 100644 --- a/pkg/roachpb/api_requestheader.go +++ b/pkg/kv/kvpb/api_requestheader.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package roachpb +package kvpb import "github.com/cockroachdb/cockroach/pkg/util/buildutil" diff --git a/pkg/roachpb/api_test.go b/pkg/kv/kvpb/api_test.go similarity index 78% rename from pkg/roachpb/api_test.go rename to pkg/kv/kvpb/api_test.go index 561fd12149ee..375f6627a50c 100644 --- a/pkg/roachpb/api_test.go +++ b/pkg/kv/kvpb/api_test.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package roachpb +package kvpb import ( "reflect" @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/buildutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -32,24 +33,24 @@ import ( func TestCombineResponses(t *testing.T) { t.Run("both combinable", func(t *testing.T) { left := &ScanResponse{ - Rows: []KeyValue{ - {Key: Key("A"), Value: MakeValueFromString("V")}, + Rows: []roachpb.KeyValue{ + {Key: roachpb.Key("A"), Value: roachpb.MakeValueFromString("V")}, }, - IntentRows: []KeyValue{ - {Key: Key("Ai"), Value: MakeValueFromString("X")}, + IntentRows: []roachpb.KeyValue{ + {Key: roachpb.Key("Ai"), Value: roachpb.MakeValueFromString("X")}, }, } right := &ScanResponse{ - Rows: []KeyValue{ - {Key: Key("B"), Value: MakeValueFromString("W")}, + Rows: []roachpb.KeyValue{ + {Key: roachpb.Key("B"), Value: roachpb.MakeValueFromString("W")}, }, - IntentRows: []KeyValue{ - {Key: Key("Bi"), Value: MakeValueFromString("Z")}, + IntentRows: []roachpb.KeyValue{ + {Key: roachpb.Key("Bi"), Value: roachpb.MakeValueFromString("Z")}, }, } expCombined := &ScanResponse{ - Rows: append(append([]KeyValue(nil), left.Rows...), right.Rows...), - IntentRows: append(append([]KeyValue(nil), left.IntentRows...), right.IntentRows...), + Rows: append(append([]roachpb.KeyValue(nil), left.Rows...), right.Rows...), + IntentRows: append(append([]roachpb.KeyValue(nil), left.IntentRows...), right.IntentRows...), } err := CombineResponses(left, right) @@ -59,10 +60,10 @@ func TestCombineResponses(t *testing.T) { t.Run("neither combinable", func(t *testing.T) { left := &GetResponse{ - Value: &Value{RawBytes: []byte("V")}, + Value: &roachpb.Value{RawBytes: []byte("V")}, } right := &GetResponse{ - Value: &Value{RawBytes: []byte("W")}, + Value: &roachpb.Value{RawBytes: []byte("W")}, } expCombined := &GetResponse{ Value: left.Value.ShallowClone(), @@ -75,15 +76,15 @@ func TestCombineResponses(t *testing.T) { t.Run("left combinable", func(t *testing.T) { left := &ScanResponse{ - Rows: []KeyValue{ - {Key: Key("A"), Value: MakeValueFromString("V")}, + Rows: []roachpb.KeyValue{ + {Key: roachpb.Key("A"), Value: roachpb.MakeValueFromString("V")}, }, - IntentRows: []KeyValue{ - {Key: Key("Ai"), Value: MakeValueFromString("X")}, + IntentRows: []roachpb.KeyValue{ + {Key: roachpb.Key("Ai"), Value: roachpb.MakeValueFromString("X")}, }, } right := &GetResponse{ - Value: &Value{RawBytes: []byte("W")}, + Value: &roachpb.Value{RawBytes: []byte("W")}, } err := CombineResponses(left, right) @@ -93,14 +94,14 @@ func TestCombineResponses(t *testing.T) { t.Run("right combinable", func(t *testing.T) { left := &GetResponse{ - Value: &Value{RawBytes: []byte("V")}, + Value: &roachpb.Value{RawBytes: []byte("V")}, } right := &ScanResponse{ - Rows: []KeyValue{ - {Key: Key("B"), Value: MakeValueFromString("W")}, + Rows: []roachpb.KeyValue{ + {Key: roachpb.Key("B"), Value: roachpb.MakeValueFromString("W")}, }, - IntentRows: []KeyValue{ - {Key: Key("Bi"), Value: MakeValueFromString("Z")}, + IntentRows: []roachpb.KeyValue{ + {Key: roachpb.Key("Bi"), Value: roachpb.MakeValueFromString("Z")}, }, } @@ -126,11 +127,11 @@ func TestCombinable(t *testing.T) { // Test that {Scan,DeleteRange}Response properly implement it. sr1 := &ScanResponse{ - Rows: []KeyValue{ - {Key: Key("A"), Value: MakeValueFromString("V")}, + Rows: []roachpb.KeyValue{ + {Key: roachpb.Key("A"), Value: roachpb.MakeValueFromString("V")}, }, - IntentRows: []KeyValue{ - {Key: Key("Ai"), Value: MakeValueFromString("X")}, + IntentRows: []roachpb.KeyValue{ + {Key: roachpb.Key("Ai"), Value: roachpb.MakeValueFromString("X")}, }, } @@ -139,17 +140,17 @@ func TestCombinable(t *testing.T) { } sr2 := &ScanResponse{ - Rows: []KeyValue{ - {Key: Key("B"), Value: MakeValueFromString("W")}, + Rows: []roachpb.KeyValue{ + {Key: roachpb.Key("B"), Value: roachpb.MakeValueFromString("W")}, }, - IntentRows: []KeyValue{ - {Key: Key("Bi"), Value: MakeValueFromString("Z")}, + IntentRows: []roachpb.KeyValue{ + {Key: roachpb.Key("Bi"), Value: roachpb.MakeValueFromString("Z")}, }, } wantedSR := &ScanResponse{ - Rows: append(append([]KeyValue(nil), sr1.Rows...), sr2.Rows...), - IntentRows: append(append([]KeyValue(nil), sr1.IntentRows...), sr2.IntentRows...), + Rows: append(append([]roachpb.KeyValue(nil), sr1.Rows...), sr2.Rows...), + IntentRows: append(append([]roachpb.KeyValue(nil), sr1.IntentRows...), sr2.IntentRows...), } if err := sr1.combine(sr2); err != nil { @@ -166,19 +167,19 @@ func TestCombinable(t *testing.T) { t.Run("DeleteRange", func(t *testing.T) { dr1 := &DeleteRangeResponse{ - Keys: []Key{[]byte("1")}, + Keys: []roachpb.Key{[]byte("1")}, } if _, ok := interface{}(dr1).(combinable); !ok { t.Fatalf("DeleteRangeResponse does not implement combinable") } dr2 := &DeleteRangeResponse{ - Keys: []Key{[]byte("2")}, + Keys: []roachpb.Key{[]byte("2")}, } dr3 := &DeleteRangeResponse{ Keys: nil, } wantedDR := &DeleteRangeResponse{ - Keys: []Key{[]byte("1"), []byte("2")}, + Keys: []roachpb.Key{[]byte("1"), []byte("2")}, } if err := dr2.combine(dr3); err != nil { t.Fatal(err) @@ -196,11 +197,11 @@ func TestCombinable(t *testing.T) { v1 := &AdminVerifyProtectedTimestampResponse{ ResponseHeader: ResponseHeader{}, Verified: false, - DeprecatedFailedRanges: []RangeDescriptor{ + DeprecatedFailedRanges: []roachpb.RangeDescriptor{ {RangeID: 1}, }, VerificationFailedRanges: []AdminVerifyProtectedTimestampResponse_FailedRange{ - {RangeID: 1, StartKey: RKeyMin, EndKey: RKeyMax, Reason: "foo"}, + {RangeID: 1, StartKey: roachpb.RKeyMin, EndKey: roachpb.RKeyMax, Reason: "foo"}, }, } @@ -215,24 +216,24 @@ func TestCombinable(t *testing.T) { v3 := &AdminVerifyProtectedTimestampResponse{ ResponseHeader: ResponseHeader{}, Verified: false, - DeprecatedFailedRanges: []RangeDescriptor{ + DeprecatedFailedRanges: []roachpb.RangeDescriptor{ {RangeID: 2}, }, VerificationFailedRanges: []AdminVerifyProtectedTimestampResponse_FailedRange{ - {RangeID: 2, StartKey: RKeyMin, EndKey: RKeyMax, Reason: "bar"}, + {RangeID: 2, StartKey: roachpb.RKeyMin, EndKey: roachpb.RKeyMax, Reason: "bar"}, }, } require.NoError(t, v1.combine(v2)) require.NoError(t, v1.combine(v3)) require.EqualValues(t, &AdminVerifyProtectedTimestampResponse{ Verified: false, - DeprecatedFailedRanges: []RangeDescriptor{ + DeprecatedFailedRanges: []roachpb.RangeDescriptor{ {RangeID: 1}, {RangeID: 2}, }, VerificationFailedRanges: []AdminVerifyProtectedTimestampResponse_FailedRange{ - {RangeID: 1, StartKey: RKeyMin, EndKey: RKeyMax, Reason: "foo"}, - {RangeID: 2, StartKey: RKeyMin, EndKey: RKeyMax, Reason: "bar"}, + {RangeID: 1, StartKey: roachpb.RKeyMin, EndKey: roachpb.RKeyMax, Reason: "foo"}, + {RangeID: 2, StartKey: roachpb.RKeyMin, EndKey: roachpb.RKeyMax, Reason: "bar"}, }, }, v1) @@ -242,7 +243,7 @@ func TestCombinable(t *testing.T) { // Test that AdminScatterResponse properly implement it. ar1 := &AdminScatterResponse{ - RangeInfos: []RangeInfo{{Desc: RangeDescriptor{ + RangeInfos: []roachpb.RangeInfo{{Desc: roachpb.RangeDescriptor{ RangeID: 1, }}}, MVCCStats: enginepb.MVCCStats{ @@ -258,7 +259,7 @@ func TestCombinable(t *testing.T) { } ar2 := &AdminScatterResponse{ - RangeInfos: []RangeInfo{{Desc: RangeDescriptor{ + RangeInfos: []roachpb.RangeInfo{{Desc: roachpb.RangeDescriptor{ RangeID: 2, }}}, MVCCStats: enginepb.MVCCStats{ @@ -270,7 +271,7 @@ func TestCombinable(t *testing.T) { } wantedAR := &AdminScatterResponse{ - RangeInfos: []RangeInfo{{Desc: RangeDescriptor{RangeID: 1}}, {Desc: RangeDescriptor{RangeID: 2}}}, + RangeInfos: []roachpb.RangeInfo{{Desc: roachpb.RangeDescriptor{RangeID: 1}}, {Desc: roachpb.RangeDescriptor{RangeID: 2}}}, MVCCStats: enginepb.MVCCStats{LiveBytes: 3, LiveCount: 3, KeyCount: 3}, ReplicasScatteredBytes: 84, } @@ -306,7 +307,7 @@ func TestMustSetInner(t *testing.T) { func TestContentionEvent_SafeFormat(t *testing.T) { ce := &ContentionEvent{ - Key: Key("foo"), + Key: roachpb.Key("foo"), TxnMeta: enginepb.TxnMeta{ID: uuid.FromStringOrNil("51b5ef6a-f18f-4e85-bc3f-c44e33f2bb27"), CoordinatorNodeID: 6}, } const exp = redact.RedactableString(`conflicted with ‹51b5ef6a-f18f-4e85-bc3f-c44e33f2bb27› on ‹"foo"› for 0.000s`) diff --git a/pkg/roachpb/batch.go b/pkg/kv/kvpb/batch.go similarity index 97% rename from pkg/roachpb/batch.go rename to pkg/kv/kvpb/batch.go index 7f9c13ff95d6..cf8414746923 100644 --- a/pkg/roachpb/batch.go +++ b/pkg/kv/kvpb/batch.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package roachpb +package kvpb import ( "bytes" @@ -17,6 +17,7 @@ import ( "strings" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/buildutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -140,7 +141,7 @@ func (ba BatchRequest) EarliestActiveTimestamp() hlc.Timestamp { // UpdateTxn updates the batch transaction from the supplied one in // a copy-on-write fashion, i.e. without mutating an existing // Transaction struct. -func (ba *BatchRequest) UpdateTxn(o *Transaction) { +func (ba *BatchRequest) UpdateTxn(o *roachpb.Transaction) { if o == nil { return } @@ -466,7 +467,7 @@ func (br *BatchResponse) String() string { // contained in the requests are used, but when a response contains a // ResumeSpan the ResumeSpan is subtracted from the request span to // provide a more minimal span of keys affected by the request. -func (ba *BatchRequest) LockSpanIterate(br *BatchResponse, fn func(Span, lock.Durability)) { +func (ba *BatchRequest) LockSpanIterate(br *BatchResponse, fn func(roachpb.Span, lock.Durability)) { for i, arg := range ba.Requests { req := arg.GetInner() if !IsLocking(req) { @@ -490,7 +491,7 @@ func (ba *BatchRequest) LockSpanIterate(br *BatchResponse, fn func(Span, lock.Du // ResumeSpan is subtracted from the request span to provide a more // minimal span of keys affected by the request. The supplied function // is called with each span. -func (ba *BatchRequest) RefreshSpanIterate(br *BatchResponse, fn func(Span)) error { +func (ba *BatchRequest) RefreshSpanIterate(br *BatchResponse, fn func(roachpb.Span)) error { for i, arg := range ba.Requests { req := arg.GetInner() if !NeedsRefresh(req) { @@ -516,8 +517,8 @@ func (ba *BatchRequest) RefreshSpanIterate(br *BatchResponse, fn func(Span)) err // to enforce serializable isolation across keys that were skipped by this // request, it does not need to validate that they have not changed if the // transaction ever needs to refresh. - if err := ResponseKeyIterate(req, resp, func(k Key) { - fn(Span{Key: k}) + if err := ResponseKeyIterate(req, resp, func(k roachpb.Key) { + fn(roachpb.Span{Key: k}) }); err != nil { return err } @@ -534,7 +535,7 @@ func (ba *BatchRequest) RefreshSpanIterate(br *BatchResponse, fn func(Span)) err // ActualSpan returns the actual request span which was operated on, // according to the existence of a resume span in the response. If // nothing was operated on, returns false. -func ActualSpan(req Request, resp Response) (Span, bool) { +func ActualSpan(req Request, resp Response) (roachpb.Span, bool) { h := req.Header() if resp != nil { resumeSpan := resp.Header().ResumeSpan @@ -543,12 +544,12 @@ func ActualSpan(req Request, resp Response) (Span, bool) { // Handle the reverse case first. if bytes.Equal(resumeSpan.Key, h.Key) { if bytes.Equal(resumeSpan.EndKey, h.EndKey) { - return Span{}, false + return roachpb.Span{}, false } - return Span{Key: resumeSpan.EndKey, EndKey: h.EndKey}, true + return roachpb.Span{Key: resumeSpan.EndKey, EndKey: h.EndKey}, true } // The forward case. - return Span{Key: h.Key, EndKey: resumeSpan.Key}, true + return roachpb.Span{Key: h.Key, EndKey: resumeSpan.Key}, true } } return h.Span(), true @@ -559,7 +560,7 @@ func ActualSpan(req Request, resp Response) (Span, bool) { // the function will not be called. // NOTE: it is assumed that req (if it is a Scan or a ReverseScan) didn't use // COL_BATCH_RESPONSE scan format. -func ResponseKeyIterate(req Request, resp Response, fn func(Key)) error { +func ResponseKeyIterate(req Request, resp Response, fn func(roachpb.Key)) error { if resp == nil { return nil } @@ -583,7 +584,7 @@ func ResponseKeyIterate(req Request, resp Response, fn func(Key)) error { // // This is not a concern for other scan formats because keys and values // are already separate heap allocations. - fn(Key(key).Clone()) + fn(roachpb.Key(key).Clone()) return nil }); err != nil { return err @@ -607,7 +608,7 @@ func ResponseKeyIterate(req Request, resp Response, fn func(Key)) error { // If ScanFormat == BATCH_RESPONSE. if err := enginepb.ScanDecodeKeyValues(v.BatchResponses, func(key []byte, _ hlc.Timestamp, _ []byte) error { // Same explanation as above. - fn(Key(key).Clone()) + fn(roachpb.Key(key).Clone()) return nil }); err != nil { return err diff --git a/pkg/roachpb/batch_generated.go b/pkg/kv/kvpb/batch_generated.go similarity index 99% rename from pkg/roachpb/batch_generated.go rename to pkg/kv/kvpb/batch_generated.go index de7080127570..061674053456 100644 --- a/pkg/roachpb/batch_generated.go +++ b/pkg/kv/kvpb/batch_generated.go @@ -1,7 +1,7 @@ // Code generated by genbatch/main.go; DO NOT EDIT. // GENERATED FILE DO NOT EDIT -package roachpb +package kvpb import ( "fmt" diff --git a/pkg/roachpb/batch_test.go b/pkg/kv/kvpb/batch_test.go similarity index 87% rename from pkg/roachpb/batch_test.go rename to pkg/kv/kvpb/batch_test.go index 48341042bc5f..c0cecedb897a 100644 --- a/pkg/roachpb/batch_test.go +++ b/pkg/kv/kvpb/batch_test.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package roachpb +package kvpb import ( "fmt" @@ -16,6 +16,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/kr/pretty" @@ -224,8 +225,8 @@ func TestLockSpanIterate(t *testing.T) { type testReq struct { req Request resp Response - span Span - resume Span + span roachpb.Span + resume roachpb.Span } testReqs := []testReq{ {&GetRequest{}, &GetResponse{}, sp("a", ""), sp("", "")}, @@ -254,14 +255,14 @@ func TestLockSpanIterate(t *testing.T) { br.Add(tr.resp) } - var spans [lock.MaxDurability + 1][]Span - fn := func(span Span, dur lock.Durability) { + var spans [lock.MaxDurability + 1][]roachpb.Span + fn := func(span roachpb.Span, dur lock.Durability) { spans[dur] = append(spans[dur], span) } ba.LockSpanIterate(&br, fn) - toExpSpans := func(trs ...testReq) []Span { - exp := make([]Span, len(trs)) + toExpSpans := func(trs ...testReq) []roachpb.Span { + exp := make([]roachpb.Span, len(trs)) for i, tr := range trs { exp[i] = tr.span if resume { @@ -284,15 +285,15 @@ func TestRefreshSpanIterate(t *testing.T) { testCases := []struct { req Request resp Response - span Span - resume Span + span roachpb.Span + resume roachpb.Span }{ - {&ConditionalPutRequest{}, &ConditionalPutResponse{}, sp("a", ""), Span{}}, - {&PutRequest{}, &PutResponse{}, sp("a-put", ""), Span{}}, - {&InitPutRequest{}, &InitPutResponse{}, sp("a-initput", ""), Span{}}, - {&IncrementRequest{}, &IncrementResponse{}, sp("a-inc", ""), Span{}}, + {&ConditionalPutRequest{}, &ConditionalPutResponse{}, sp("a", ""), roachpb.Span{}}, + {&PutRequest{}, &PutResponse{}, sp("a-put", ""), roachpb.Span{}}, + {&InitPutRequest{}, &InitPutResponse{}, sp("a-initput", ""), roachpb.Span{}}, + {&IncrementRequest{}, &IncrementResponse{}, sp("a-inc", ""), roachpb.Span{}}, {&ScanRequest{}, &ScanResponse{}, sp("a", "c"), sp("b", "c")}, - {&GetRequest{}, &GetResponse{}, sp("b", ""), Span{}}, + {&GetRequest{}, &GetResponse{}, sp("b", ""), roachpb.Span{}}, {&ReverseScanRequest{}, &ReverseScanResponse{}, sp("d", "f"), sp("d", "e")}, {&DeleteRangeRequest{}, &DeleteRangeResponse{}, sp("g", "i"), sp("h", "i")}, } @@ -306,13 +307,13 @@ func TestRefreshSpanIterate(t *testing.T) { br.Add(tc.resp) } - var readSpans []Span - fn := func(span Span) { + var readSpans []roachpb.Span + fn := func(span roachpb.Span) { readSpans = append(readSpans, span) } require.NoError(t, ba.RefreshSpanIterate(&br, fn)) // The conditional put and init put are not considered read spans. - expReadSpans := []Span{testCases[4].span, testCases[5].span, testCases[6].span, testCases[7].span} + expReadSpans := []roachpb.Span{testCases[4].span, testCases[5].span, testCases[6].span, testCases[7].span} require.Equal(t, expReadSpans, readSpans) // Batch responses with ResumeSpans. @@ -328,9 +329,9 @@ func TestRefreshSpanIterate(t *testing.T) { br.Add(tc.resp) } - readSpans = []Span{} + readSpans = []roachpb.Span{} require.NoError(t, ba.RefreshSpanIterate(&br, fn)) - expReadSpans = []Span{ + expReadSpans = []roachpb.Span{ sp("a", "b"), sp("b", ""), sp("e", "f"), @@ -341,16 +342,16 @@ func TestRefreshSpanIterate(t *testing.T) { func TestRefreshSpanIterateSkipLocked(t *testing.T) { ba := BatchRequest{} - ba.Add(NewGet(Key("a"), false)) - ba.Add(NewScan(Key("b"), Key("d"), false)) - ba.Add(NewReverseScan(Key("e"), Key("g"), false)) + ba.Add(NewGet(roachpb.Key("a"), false)) + ba.Add(NewScan(roachpb.Key("b"), roachpb.Key("d"), false)) + ba.Add(NewReverseScan(roachpb.Key("e"), roachpb.Key("g"), false)) br := ba.CreateReply() // Without a SkipLocked wait policy. - var readSpans []Span - fn := func(span Span) { readSpans = append(readSpans, span) } + var readSpans []roachpb.Span + fn := func(span roachpb.Span) { readSpans = append(readSpans, span) } require.NoError(t, ba.RefreshSpanIterate(br, fn)) - expReadSpans := []Span{ + expReadSpans := []roachpb.Span{ sp("a", ""), sp("b", "d"), sp("e", "g"), @@ -362,17 +363,17 @@ func TestRefreshSpanIterateSkipLocked(t *testing.T) { readSpans = nil require.NoError(t, ba.RefreshSpanIterate(br, fn)) - expReadSpans = []Span(nil) + expReadSpans = []roachpb.Span(nil) require.Equal(t, expReadSpans, readSpans) // With a SkipLocked wait policy and with some response keys. - br.Responses[0].GetGet().Value = &Value{} - br.Responses[1].GetScan().Rows = []KeyValue{{Key: Key("b")}, {Key: Key("c")}} - br.Responses[2].GetReverseScan().Rows = []KeyValue{{Key: Key("f")}} + br.Responses[0].GetGet().Value = &roachpb.Value{} + br.Responses[1].GetScan().Rows = []roachpb.KeyValue{{Key: roachpb.Key("b")}, {Key: roachpb.Key("c")}} + br.Responses[2].GetReverseScan().Rows = []roachpb.KeyValue{{Key: roachpb.Key("f")}} readSpans = nil require.NoError(t, ba.RefreshSpanIterate(br, fn)) - expReadSpans = []Span{ + expReadSpans = []roachpb.Span{ sp("a", ""), sp("b", ""), sp("c", ""), @@ -384,8 +385,8 @@ func TestRefreshSpanIterateSkipLocked(t *testing.T) { func TestBatchResponseCombine(t *testing.T) { br := &BatchResponse{} { - txn := MakeTransaction( - "test", nil /* baseKey */, NormalUserPriority, + txn := roachpb.MakeTransaction( + "test", nil /* baseKey */, roachpb.NormalUserPriority, hlc.Timestamp{WallTime: 123}, 0 /* baseKey */, 99, /* coordinatorNodeID */ ) brTxn := &BatchResponse{ @@ -406,11 +407,11 @@ func TestBatchResponseCombine(t *testing.T) { singleScanBR := func() *BatchResponse { var union ResponseUnion union.MustSetInner(&ScanResponse{ - Rows: []KeyValue{{ - Key: Key("bar"), + Rows: []roachpb.KeyValue{{ + Key: roachpb.Key("bar"), }}, - IntentRows: []KeyValue{{ - Key: Key("baz"), + IntentRows: []roachpb.KeyValue{{ + Key: roachpb.Key("baz"), }}, }) return &BatchResponse{ @@ -450,15 +451,15 @@ func TestBatchResponseCombine(t *testing.T) { t.Fatalf("expected %d intent rows, got %s", expRows, pretty.Sprint(scan)) } if err := br.Combine(singleScanBR(), []int{0}); err.Error() != - `can not combine *roachpb.PutResponse and *roachpb.ScanResponse` { + `can not combine *kvpb.PutResponse and *kvpb.ScanResponse` { t.Fatal(err) } } -func sp(start, end string) Span { - res := Span{Key: Key(start)} +func sp(start, end string) roachpb.Span { + res := roachpb.Span{Key: roachpb.Key(start)} if end != "" { - res.EndKey = Key(end) + res.EndKey = roachpb.Key(end) } return res } diff --git a/pkg/kv/kvpb/data.go b/pkg/kv/kvpb/data.go new file mode 100644 index 000000000000..940cccf5c077 --- /dev/null +++ b/pkg/kv/kvpb/data.go @@ -0,0 +1,145 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package kvpb + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/log" +) + +// PrepareTransactionForRetry returns a new Transaction to be used for retrying +// the original Transaction. Depending on the error, this might return an +// already-existing Transaction with an incremented epoch, or a completely new +// Transaction. +// +// The caller should generally check that the error was meant for this +// Transaction before calling this. +// +// pri is the priority that should be used when giving the restarted transaction +// the chance to get a higher priority. Not used when the transaction is being +// aborted. +// +// In case retryErr tells us that a new Transaction needs to be created, +// isolation and name help initialize this new transaction. +func PrepareTransactionForRetry( + ctx context.Context, pErr *Error, pri roachpb.UserPriority, clock *hlc.Clock, +) roachpb.Transaction { + if pErr.TransactionRestart() == TransactionRestart_NONE { + log.Fatalf(ctx, "invalid retryable err (%T): %s", pErr.GetDetail(), pErr) + } + + if pErr.GetTxn() == nil { + log.Fatalf(ctx, "missing txn for retryable error: %s", pErr) + } + + txn := *pErr.GetTxn() + aborted := false + switch tErr := pErr.GetDetail().(type) { + case *TransactionAbortedError: + // The txn coming with a TransactionAbortedError is not supposed to be used + // for the restart. Instead, a brand new transaction is created. + aborted = true + // TODO(andrei): Should we preserve the ObservedTimestamps across the + // restart? + errTxnPri := txn.Priority + // Start the new transaction at the current time from the local clock. + // The local hlc should have been advanced to at least the error's + // timestamp already. + now := clock.NowAsClockTimestamp() + txn = roachpb.MakeTransaction( + txn.Name, + nil, // baseKey + // We have errTxnPri, but this wants a roachpb.UserPriority. So + // we're going to overwrite the priority below. + roachpb.NormalUserPriority, + now.ToTimestamp(), + clock.MaxOffset().Nanoseconds(), + txn.CoordinatorNodeID, + ) + // Use the priority communicated back by the server. + txn.Priority = errTxnPri + case *ReadWithinUncertaintyIntervalError: + txn.WriteTimestamp.Forward(tErr.RetryTimestamp()) + case *TransactionPushError: + // Increase timestamp if applicable, ensuring that we're just ahead of + // the pushee. + txn.WriteTimestamp.Forward(tErr.PusheeTxn.WriteTimestamp) + txn.UpgradePriority(tErr.PusheeTxn.Priority - 1) + case *TransactionRetryError: + // Transaction.Timestamp has already been forwarded to be ahead of any + // timestamp cache entries or newer versions which caused the restart. + if tErr.Reason == RETRY_SERIALIZABLE { + // For RETRY_SERIALIZABLE case, we want to bump timestamp further than + // timestamp cache. + // This helps transactions that had their commit timestamp fixed (See + // roachpb.Transaction.CommitTimestampFixed for details on when it happens) + // or transactions that hit read-write contention and can't bump + // read timestamp because of later writes. + // Upon retry, we want those transactions to restart on now() instead of + // closed ts to give them some time to complete without a need to refresh + // read spans yet again and possibly fail. + // The tradeoff here is that transactions that failed because they were + // waiting on locks or were slowed down in their first epoch for any other + // reason (e.g. lease transfers, network congestion, node failure, etc.) + // would have a chance to retry and succeed, but transactions that are + // just slow would still retry indefinitely and delay transactions that + // try to write to the keys this transaction reads because reads are not + // in the past anymore. + now := clock.Now() + txn.WriteTimestamp.Forward(now) + } + case *WriteTooOldError: + // Increase the timestamp to the ts at which we've actually written. + txn.WriteTimestamp.Forward(tErr.RetryTimestamp()) + default: + log.Fatalf(ctx, "invalid retryable err (%T): %s", pErr.GetDetail(), pErr) + } + if !aborted { + if txn.Status.IsFinalized() { + log.Fatalf(ctx, "transaction unexpectedly finalized in (%T): %s", pErr.GetDetail(), pErr) + } + txn.Restart(pri, txn.Priority, txn.WriteTimestamp) + } + return txn +} + +// TransactionRefreshTimestamp returns whether the supplied error is a retry +// error that can be discarded if the transaction in the error is refreshed. If +// true, the function returns the timestamp that the Transaction object should +// be refreshed at in order to discard the error and avoid a restart. +func TransactionRefreshTimestamp(pErr *Error) (bool, hlc.Timestamp) { + txn := pErr.GetTxn() + if txn == nil { + return false, hlc.Timestamp{} + } + timestamp := txn.WriteTimestamp + switch err := pErr.GetDetail().(type) { + case *TransactionRetryError: + if err.Reason != RETRY_SERIALIZABLE && err.Reason != RETRY_WRITE_TOO_OLD { + return false, hlc.Timestamp{} + } + case *WriteTooOldError: + // TODO(andrei): Chances of success for on write-too-old conditions might be + // usually small: if our txn previously read the key that generated this + // error, obviously the refresh will fail. It might be worth trying to + // detect these cases and save the futile attempt; we'd need to have access + // to the key that generated the error. + timestamp.Forward(err.RetryTimestamp()) + case *ReadWithinUncertaintyIntervalError: + timestamp.Forward(err.RetryTimestamp()) + default: + return false, hlc.Timestamp{} + } + return true, timestamp +} diff --git a/pkg/roachpb/errordetailtype_string.go b/pkg/kv/kvpb/errordetailtype_string.go similarity index 99% rename from pkg/roachpb/errordetailtype_string.go rename to pkg/kv/kvpb/errordetailtype_string.go index 112a493b7fde..1e5d4925373d 100644 --- a/pkg/roachpb/errordetailtype_string.go +++ b/pkg/kv/kvpb/errordetailtype_string.go @@ -1,6 +1,6 @@ // Code generated by "stringer"; DO NOT EDIT. -package roachpb +package kvpb import "strconv" diff --git a/pkg/roachpb/errors.go b/pkg/kv/kvpb/errors.go similarity index 89% rename from pkg/roachpb/errors.go rename to pkg/kv/kvpb/errors.go index f63d96f8c0ec..8d17932e46ad 100644 --- a/pkg/roachpb/errors.go +++ b/pkg/kv/kvpb/errors.go @@ -8,12 +8,14 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package roachpb +package kvpb import ( "context" "fmt" + "reflect" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/caller" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" @@ -75,7 +77,7 @@ type transactionRestartError interface { } // ErrorUnexpectedlySet creates a string to panic with when a response (typically -// a roachpb.BatchResponse) unexpectedly has Error set in its response header. +// a kvpb.BatchResponse) unexpectedly has Error set in its response header. func ErrorUnexpectedlySet(culprit, response interface{}) error { return errors.AssertionFailedf("error is unexpectedly set, culprit is %T:\n%+v", culprit, response) } @@ -173,7 +175,7 @@ func NewError(err error) *Error { // NewErrorWithTxn creates an Error from the given error and a transaction. // // txn is cloned before being stored in Error. -func NewErrorWithTxn(err error, txn *Transaction) *Error { +func NewErrorWithTxn(err error, txn *roachpb.Transaction) *Error { e := NewError(err) e.SetTxn(txn) return e @@ -305,6 +307,47 @@ const ( NumErrors int = 45 ) +// Register the migration of all errors that used to be in the roachpb package +// and are now in the kv/kvpb package. +func init() { + roachpbPath := reflect.TypeOf(roachpb.Key("")).PkgPath() + errors.RegisterTypeMigration(roachpbPath, "*roachpb.UnhandledRetryableError", &UnhandledRetryableError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.internalError", &internalError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.NotLeaseHolderError", &NotLeaseHolderError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.RangeNotFoundError", &RangeNotFoundError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.RangeKeyMismatchError", &RangeKeyMismatchError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.ReadWithinUncertaintyIntervalError", &ReadWithinUncertaintyIntervalError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.TransactionAbortedError", &TransactionAbortedError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.TransactionPushError", &TransactionPushError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.TransactionRetryError", &TransactionRetryError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.TransactionStatusError", &TransactionStatusError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.WriteIntentError", &WriteIntentError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.WriteTooOldError", &WriteTooOldError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.OpRequiresTxnError", &OpRequiresTxnError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.ConditionFailedError", &ConditionFailedError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.LeaseRejectedError", &LeaseRejectedError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.NodeUnavailableError", &NodeUnavailableError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.RaftGroupDeletedError", &RaftGroupDeletedError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.ReplicaCorruptionError", &ReplicaCorruptionError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.ReplicaTooOldError", &ReplicaTooOldError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.StoreNotFoundError", &StoreNotFoundError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.TransactionRetryWithProtoRefreshError", &TransactionRetryWithProtoRefreshError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.IntegerOverflowError", &IntegerOverflowError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.UnsupportedRequestError", &UnsupportedRequestError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.BatchTimestampBeforeGCError", &BatchTimestampBeforeGCError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.TxnAlreadyEncounteredErrorError", &TxnAlreadyEncounteredErrorError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.IntentMissingError", &IntentMissingError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.MergeInProgressError", &MergeInProgressError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.RangeFeedRetryError", &RangeFeedRetryError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.IndeterminateCommitError", &IndeterminateCommitError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.InvalidLeaseError", &InvalidLeaseError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.OptimisticEvalConflictsError", &OptimisticEvalConflictsError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.MinTimestampBoundUnsatisfiableError", &MinTimestampBoundUnsatisfiableError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.RefreshFailedError", &RefreshFailedError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.MVCCHistoryMutationError", &MVCCHistoryMutationError{}) + errors.RegisterTypeMigration(roachpbPath, "*roachpb.InsufficientSpaceError", &InsufficientSpaceError{}) +} + // GoError returns a Go error converted from Error. If the error is a transaction // retry error, it returns the error itself wrapped in an UnhandledRetryableError. // Otherwise, if an error detail is present, is is returned (i.e. the result will @@ -352,14 +395,14 @@ func (e *Error) GetDetail() ErrorDetailInterface { // SetTxn sets the error transaction and resets the error message. // The argument is cloned before being stored in the Error. -func (e *Error) SetTxn(txn *Transaction) { +func (e *Error) SetTxn(txn *roachpb.Transaction) { e.UnexposedTxn = nil e.UpdateTxn(txn) } // UpdateTxn updates the error transaction and resets the error message. // The argument is cloned before being stored in the Error. -func (e *Error) UpdateTxn(o *Transaction) { +func (e *Error) UpdateTxn(o *roachpb.Transaction) { if o == nil { return } @@ -391,7 +434,7 @@ func (e *Error) checkTxnStatusValid() { } // GetTxn returns the txn. -func (e *Error) GetTxn() *Transaction { +func (e *Error) GetTxn() *roachpb.Transaction { if e == nil { return nil } @@ -435,7 +478,7 @@ func (e *NotLeaseHolderError) printError(s Printer) { s.Printf("; ") } s.Printf("r%d: ", e.RangeID) - if e.Replica != (ReplicaDescriptor{}) { + if e.Replica != (roachpb.ReplicaDescriptor{}) { s.Printf("replica %s not lease holder; ", e.Replica) } else { s.Printf("replica not lease holder; ") @@ -474,7 +517,7 @@ var _ ErrorDetailInterface = &LeaseRejectedError{} // NewRangeNotFoundError initializes a new RangeNotFoundError for the given RangeID and, optionally, // a StoreID. -func NewRangeNotFoundError(rangeID RangeID, storeID StoreID) *RangeNotFoundError { +func NewRangeNotFoundError(rangeID roachpb.RangeID, storeID roachpb.StoreID) *RangeNotFoundError { return &RangeNotFoundError{ RangeID: rangeID, StoreID: storeID, @@ -513,10 +556,10 @@ func IsRangeNotFoundError(err error) bool { // identical to NewRangeKeyMismatchError, with the given ClosedTimestampPolicy. func NewRangeKeyMismatchErrorWithCTPolicy( ctx context.Context, - start, end Key, - desc *RangeDescriptor, - lease *Lease, - ctPolicy RangeClosedTimestampPolicy, + start, end roachpb.Key, + desc *roachpb.RangeDescriptor, + lease *roachpb.Lease, + ctPolicy roachpb.RangeClosedTimestampPolicy, ) *RangeKeyMismatchError { if desc == nil { panic("NewRangeKeyMismatchError with nil descriptor") @@ -526,7 +569,7 @@ func NewRangeKeyMismatchErrorWithCTPolicy( // regressions of #6027. panic(fmt.Sprintf("descriptor is not initialized: %+v", desc)) } - var l Lease + var l roachpb.Lease if lease != nil { // We ignore leases that are not part of the descriptor. _, ok := desc.GetReplicaDescriptorByID(lease.Replica.ReplicaID) @@ -538,7 +581,7 @@ func NewRangeKeyMismatchErrorWithCTPolicy( RequestStartKey: start, RequestEndKey: end, } - ri := RangeInfo{ + ri := roachpb.RangeInfo{ Desc: *desc, Lease: l, ClosedTimestampPolicy: ctPolicy, @@ -558,14 +601,14 @@ func NewRangeKeyMismatchErrorWithCTPolicy( // Note that more range info is commonly added to the error after the error is // created. func NewRangeKeyMismatchError( - ctx context.Context, start, end Key, desc *RangeDescriptor, lease *Lease, + ctx context.Context, start, end roachpb.Key, desc *roachpb.RangeDescriptor, lease *roachpb.Lease, ) *RangeKeyMismatchError { return NewRangeKeyMismatchErrorWithCTPolicy(ctx, start, end, desc, lease, - LAG_BY_CLUSTER_SETTING, /* default closed timestsamp policy*/ + roachpb.LAG_BY_CLUSTER_SETTING, /* default closed timestsamp policy*/ ) } @@ -594,9 +637,9 @@ func (e *RangeKeyMismatchError) Type() ErrorDetailType { // MismatchedRange returns the range info for the range that the request was // erroneously routed to, or an error if the Ranges slice is empty. -func (e *RangeKeyMismatchError) MismatchedRange() (RangeInfo, error) { +func (e *RangeKeyMismatchError) MismatchedRange() (roachpb.RangeInfo, error) { if len(e.Ranges) == 0 { - return RangeInfo{}, errors.AssertionFailedf( + return roachpb.RangeInfo{}, errors.AssertionFailedf( "RangeKeyMismatchError (key range %s-%s) with empty RangeInfo slice", e.RequestStartKey, e.RequestEndKey, ) } @@ -608,7 +651,7 @@ func (e *RangeKeyMismatchError) MismatchedRange() (RangeInfo, error) { // // l can be empty. Otherwise, the leaseholder is asserted to be a replica in // desc. -func (e *RangeKeyMismatchError) AppendRangeInfo(ctx context.Context, ris ...RangeInfo) { +func (e *RangeKeyMismatchError) AppendRangeInfo(ctx context.Context, ris ...roachpb.RangeInfo) { for _, ri := range ris { if !ri.Lease.Empty() { if _, ok := ri.Desc.GetReplicaDescriptorByID(ri.Lease.Replica.ReplicaID); !ok { @@ -680,7 +723,7 @@ func (e *TransactionAbortedError) SafeFormatError(p errors.Printer) (next error) // to improve this: wrap `pErr.GoError()` with a barrier and then with the // TransactionRetryWithProtoRefreshError. func NewTransactionRetryWithProtoRefreshError( - msg redact.RedactableString, txnID uuid.UUID, txn Transaction, + msg redact.RedactableString, txnID uuid.UUID, txn roachpb.Transaction, ) *TransactionRetryWithProtoRefreshError { return &TransactionRetryWithProtoRefreshError{ Msg: msg.StripMarkers(), @@ -708,7 +751,7 @@ func (e *TransactionRetryWithProtoRefreshError) PrevTxnAborted() bool { } // NewTransactionPushError initializes a new TransactionPushError. -func NewTransactionPushError(pusheeTxn Transaction) *TransactionPushError { +func NewTransactionPushError(pusheeTxn roachpb.Transaction) *TransactionPushError { // Note: this error will cause a txn restart. The error that the client // receives contains a txn that might have a modified priority. return &TransactionPushError{PusheeTxn: pusheeTxn} @@ -820,7 +863,7 @@ func (e *WriteIntentError) printError(buf Printer) { // If we have a lot of intents, we only want to show the first and the last. const maxBegin = 5 const maxEnd = 5 - var begin, end []Intent + var begin, end []roachpb.Intent if len(e.Intents) <= maxBegin+maxEnd { begin = e.Intents } else { @@ -870,7 +913,7 @@ var _ ErrorDetailInterface = &WriteIntentError{} // immediately after the existing write which had a higher timestamp and which // caused the error. An optional Key parameter is accepted to denote one key // where this error was encountered. -func NewWriteTooOldError(operationTS, actualTS hlc.Timestamp, key Key) *WriteTooOldError { +func NewWriteTooOldError(operationTS, actualTS hlc.Timestamp, key roachpb.Key) *WriteTooOldError { if len(key) > 0 { oldKey := key key = make([]byte, len(oldKey)) @@ -922,12 +965,12 @@ var _ transactionRestartError = &WriteTooOldError{} func NewReadWithinUncertaintyIntervalError( readTS hlc.Timestamp, localUncertaintyLimit hlc.ClockTimestamp, - txn *Transaction, + txn *roachpb.Transaction, valueTS hlc.Timestamp, localTS hlc.ClockTimestamp, ) *ReadWithinUncertaintyIntervalError { var globalUncertaintyLimit hlc.Timestamp - var observedTSs []ObservedTimestamp + var observedTSs []roachpb.ObservedTimestamp if txn != nil { globalUncertaintyLimit = txn.GlobalUncertaintyLimit observedTSs = txn.ObservedTimestamps @@ -962,7 +1005,7 @@ func (e *ReadWithinUncertaintyIntervalError) printError(p Printer) { e.ReadTimestamp, e.ValueTimestamp, localTsStr, e.LocalUncertaintyLimit, e.GlobalUncertaintyLimit) p.Printf("[") - for i, ot := range observedTimestampSlice(e.ObservedTimestamps) { + for i, ot := range e.ObservedTimestamps { if i > 0 { p.Printf(" ") } @@ -1103,7 +1146,7 @@ func (e *ReplicaCorruptionError) Type() ErrorDetailType { var _ ErrorDetailInterface = &ReplicaCorruptionError{} // NewReplicaTooOldError initializes a new ReplicaTooOldError. -func NewReplicaTooOldError(replicaID ReplicaID) *ReplicaTooOldError { +func NewReplicaTooOldError(replicaID roachpb.ReplicaID) *ReplicaTooOldError { return &ReplicaTooOldError{ ReplicaID: replicaID, } @@ -1126,7 +1169,7 @@ func (e *ReplicaTooOldError) Type() ErrorDetailType { var _ ErrorDetailInterface = &ReplicaTooOldError{} // NewStoreNotFoundError initializes a new StoreNotFoundError. -func NewStoreNotFoundError(storeID StoreID) *StoreNotFoundError { +func NewStoreNotFoundError(storeID roachpb.StoreID) *StoreNotFoundError { return &StoreNotFoundError{ StoreID: storeID, } @@ -1234,7 +1277,7 @@ func (e *MVCCHistoryMutationError) Type() ErrorDetailType { var _ ErrorDetailInterface = &MVCCHistoryMutationError{} // NewIntentMissingError creates a new IntentMissingError. -func NewIntentMissingError(key Key, wrongIntent *Intent) *IntentMissingError { +func NewIntentMissingError(key roachpb.Key, wrongIntent *roachpb.Intent) *IntentMissingError { return &IntentMissingError{ Key: key, WrongIntent: wrongIntent, @@ -1305,7 +1348,7 @@ func (e *RangeFeedRetryError) Type() ErrorDetailType { var _ ErrorDetailInterface = &RangeFeedRetryError{} // NewIndeterminateCommitError initializes a new IndeterminateCommitError. -func NewIndeterminateCommitError(txn Transaction) *IndeterminateCommitError { +func NewIndeterminateCommitError(txn roachpb.Transaction) *IndeterminateCommitError { return &IndeterminateCommitError{StagingTxn: txn} } @@ -1396,7 +1439,7 @@ var _ ErrorDetailInterface = &MinTimestampBoundUnsatisfiableError{} // or 'intent' which caused the failed refresh, key is the key that we failed // refreshing, and ts is the timestamp of the committed value or intent that was written. func NewRefreshFailedError( - reason RefreshFailedError_Reason, key Key, ts hlc.Timestamp, + reason RefreshFailedError_Reason, key roachpb.Key, ts hlc.Timestamp, ) *RefreshFailedError { return &RefreshFailedError{ Reason: reason, @@ -1446,7 +1489,7 @@ func (e *InsufficientSpaceError) Error() string { // Note that this error can be generated on the Raft processing goroutine, so // its output should be completely determined by its parameters. func NewNotLeaseHolderError( - l Lease, proposerStoreID StoreID, rangeDesc *RangeDescriptor, msg string, + l roachpb.Lease, proposerStoreID roachpb.StoreID, rangeDesc *roachpb.RangeDescriptor, msg string, ) *NotLeaseHolderError { err := &NotLeaseHolderError{ RangeID: rangeDesc.RangeID, @@ -1465,7 +1508,7 @@ func NewNotLeaseHolderError( // could catch tests in a loop, presumably due to manual clocks). _, stillMember := rangeDesc.GetReplicaDescriptor(l.Replica.StoreID) if stillMember { - err.Lease = new(Lease) + err.Lease = new(roachpb.Lease) *err.Lease = l // TODO(arul): We only need to return this for the 22.1 <-> 22.2 mixed // version state, as v22.1 use this field to log NLHE messages. We can @@ -1482,9 +1525,12 @@ func NewNotLeaseHolderError( // current lease is not known, but the error is being created by guessing who // the leaseholder may be. func NewNotLeaseHolderErrorWithSpeculativeLease( - leaseHolder ReplicaDescriptor, proposerStoreID StoreID, rangeDesc *RangeDescriptor, msg string, + leaseHolder roachpb.ReplicaDescriptor, + proposerStoreID roachpb.StoreID, + rangeDesc *roachpb.RangeDescriptor, + msg string, ) *NotLeaseHolderError { - speculativeLease := Lease{ + speculativeLease := roachpb.Lease{ Replica: leaseHolder, } return NewNotLeaseHolderError(speculativeLease, proposerStoreID, rangeDesc, msg) diff --git a/pkg/roachpb/errors.proto b/pkg/kv/kvpb/errors.proto similarity index 95% rename from pkg/roachpb/errors.proto rename to pkg/kv/kvpb/errors.proto index ad8c593fc8ef..6d0302270ca2 100644 --- a/pkg/roachpb/errors.proto +++ b/pkg/kv/kvpb/errors.proto @@ -24,7 +24,7 @@ syntax = "proto2"; // correct name, the files can be moved into a newly created `kvpb` package // at leisure. package cockroach.kv.kvpb; -option go_package = "roachpb"; +option go_package = "kvpb"; import "errorspb/errors.proto"; import "roachpb/data.proto"; @@ -60,7 +60,8 @@ message NotLeaseHolderError { // set. optional roachpb.Lease lease = 4; optional int64 range_id = 3 [(gogoproto.nullable) = false, - (gogoproto.customname) = "RangeID", (gogoproto.casttype) = "RangeID"]; + (gogoproto.customname) = "RangeID", + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID"]; // The range descriptor from the replica the error originated from. // The generation of the descriptor is used by the DistSender's RangeCache to // determine whether the error was returned because the replica had a stale @@ -88,17 +89,19 @@ message UnsupportedRequestError { // which is not hosted on this store. message RangeNotFoundError { optional int64 range_id = 1 [(gogoproto.nullable) = false, - (gogoproto.customname) = "RangeID", (gogoproto.casttype) = "RangeID"]; + (gogoproto.customname) = "RangeID", + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID"]; // store_id is nonzero only if the error originated on a Store. optional int64 store_id = 2 [(gogoproto.nullable) = false, - (gogoproto.customname) = "StoreID", (gogoproto.casttype) = "StoreID"]; + (gogoproto.customname) = "StoreID", + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.StoreID"]; } // A RangeKeyMismatchError indicates that a command was sent to a // range which did not contain the key(s) specified by the command. message RangeKeyMismatchError { - optional bytes request_start_key = 1 [(gogoproto.casttype) = "Key"]; - optional bytes request_end_key = 2 [(gogoproto.casttype) = "Key"]; + optional bytes request_start_key = 1 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.Key"]; + optional bytes request_end_key = 2 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.Key"]; // Ranges contains information intended for the client's range cache. The // server populates it with info on the range with the ID addressed by the @@ -322,7 +325,7 @@ message WriteIntentError { // operating under. Used on the server to avoid adding discovered locks // which were discovered under old leases to the lock table. optional int64 lease_sequence = 3 [(gogoproto.nullable) = false, - (gogoproto.casttype) = "LeaseSequence"]; + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.LeaseSequence"]; enum Reason { // The reason for the WriteIntentError is unspecified. This will // always be the case for errors returned from MVCC. @@ -363,7 +366,7 @@ message WriteTooOldError { // to be used in any transaction logic. As a result, it's not even necessary // for this key to be at actual_timestamp; it could be at any timestamp in // between timestamp and actual_timestamp. - optional bytes key = 3 [(gogoproto.casttype) = "Key"]; + optional bytes key = 3 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.Key"]; } // An OpRequiresTxnError indicates that a command required to be @@ -424,14 +427,16 @@ message ReplicaCorruptionError { message ReplicaTooOldError { // replica_id is the ID of the replica that is too old. optional int32 replica_id = 1 [(gogoproto.nullable) = false, - (gogoproto.customname) = "ReplicaID", (gogoproto.casttype) = "ReplicaID"]; + (gogoproto.customname) = "ReplicaID", + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.ReplicaID"]; } // A StoreNotFoundError indicates that a command was sent to a store // which is not hosted on this node. message StoreNotFoundError { optional int64 store_id = 1 [(gogoproto.nullable) = false, - (gogoproto.customname) = "StoreID", (gogoproto.casttype) = "StoreID"]; + (gogoproto.customname) = "StoreID", + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.StoreID"]; } // UnhandledRetryableError tells the recipient that a KV request must be @@ -499,7 +504,7 @@ message TxnAlreadyEncounteredErrorError{ // An IntegerOverflowError indicates that an operation was aborted because // it would have caused an integer overflow. message IntegerOverflowError { - optional bytes key = 1 [(gogoproto.casttype) = "Key"]; + optional bytes key = 1 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.Key"]; optional int64 current_value = 2 [(gogoproto.nullable) = false]; optional int64 increment_value = 3 [(gogoproto.nullable) = false]; } @@ -528,7 +533,7 @@ message IntentMissingError { // The non-matching intent that was found at that key, if any. optional roachpb.Intent wrong_intent = 1; // The key where the intent was expected. - optional bytes key = 2 [(gogoproto.casttype) = "Key"]; + optional bytes key = 2 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.Key"]; } // A MergeInProgressError indicates that the request could not be completed @@ -607,7 +612,7 @@ message RefreshFailedError { REASON_INTENT = 1; } optional Reason reason = 1 [(gogoproto.nullable) = false]; - optional bytes key = 2 [(gogoproto.casttype) = "Key"]; + optional bytes key = 2 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.Key"]; // The timestamp the key was last updated. optional util.hlc.Timestamp timestamp = 3 [(gogoproto.nullable) = false]; @@ -698,7 +703,8 @@ message Error { optional roachpb.Transaction unexposed_txn = 4; // Node at which the error was generated (zero if does not apply). - optional int32 origin_node = 5 [(gogoproto.nullable) = false, (gogoproto.casttype) = "NodeID"]; + optional int32 origin_node = 5 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID"]; // encoded_error is the Go error that caused this Error. optional errorspb.EncodedError encoded_error = 9 [(gogoproto.nullable) = false]; @@ -720,7 +726,8 @@ message Error { message InsufficientSpaceError { // StoreID is the store that had insufficient space. optional int64 store_id = 1 [(gogoproto.nullable) = false, - (gogoproto.customname) = "StoreID", (gogoproto.casttype) = "StoreID"]; + (gogoproto.customname) = "StoreID", + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.StoreID"]; // Op is the operaton that was unable to be performed. optional string op = 2 [(gogoproto.nullable) = false]; diff --git a/pkg/roachpb/errors_test.go b/pkg/kv/kvpb/errors_test.go similarity index 92% rename from pkg/roachpb/errors_test.go rename to pkg/kv/kvpb/errors_test.go index 9aaa49c336ba..7c617f272384 100644 --- a/pkg/roachpb/errors_test.go +++ b/pkg/kv/kvpb/errors_test.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package roachpb +package kvpb import ( "bytes" @@ -17,6 +17,7 @@ import ( "strings" "testing" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -52,7 +53,7 @@ func TestNewErrorNil(t *testing.T) { // TestSetTxn verifies that SetTxn updates the error message. func TestSetTxn(t *testing.T) { e := NewError(NewTransactionAbortedError(ABORT_REASON_ABORTED_RECORD_FOUND)) - txn := MakeTransaction("test", Key("a"), 1, hlc.Timestamp{}, 0, 99) + txn := roachpb.MakeTransaction("test", roachpb.Key("a"), 1, hlc.Timestamp{}, 0, 99) e.SetTxn(&txn) if !strings.HasPrefix( e.String(), "TransactionAbortedError(ABORT_REASON_ABORTED_RECORD_FOUND): \"test\"") { @@ -74,12 +75,12 @@ func TestErrPriority(t *testing.T) { id1 := uuid.Must(uuid.NewV4()) require.Equal(t, ErrorScoreTxnRestart, ErrPriority(&TransactionRetryWithProtoRefreshError{ TxnID: id1, - Transaction: Transaction{TxnMeta: enginepb.TxnMeta{ID: id1}}, + Transaction: roachpb.Transaction{TxnMeta: enginepb.TxnMeta{ID: id1}}, })) id2 := uuid.Nil require.Equal(t, ErrorScoreTxnAbort, ErrPriority(&TransactionRetryWithProtoRefreshError{ TxnID: id1, - Transaction: Transaction{TxnMeta: enginepb.TxnMeta{ID: id2}}, + Transaction: roachpb.Transaction{TxnMeta: enginepb.TxnMeta{ID: id2}}, })) } require.Equal(t, ErrorScoreUnambiguousError, ErrPriority(&ConditionFailedError{})) @@ -94,7 +95,7 @@ func TestErrorTxn(t *testing.T) { } pErr = NewErrorf("foo") const name = "X" - pErr.SetTxn(&Transaction{Name: name}) + pErr.SetTxn(&roachpb.Transaction{Name: name}) if txn := pErr.GetTxn(); txn == nil || txn.Name != name { t.Fatalf("wanted name %s, unexpected: %+v", name, txn) } @@ -105,9 +106,9 @@ func TestReadWithinUncertaintyIntervalError(t *testing.T) { rwueNew := NewReadWithinUncertaintyIntervalError( hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{WallTime: 2, Logical: 2}, - &Transaction{ + &roachpb.Transaction{ GlobalUncertaintyLimit: hlc.Timestamp{WallTime: 3}, - ObservedTimestamps: []ObservedTimestamp{{NodeID: 12, Timestamp: hlc.ClockTimestamp{WallTime: 4}}}, + ObservedTimestamps: []roachpb.ObservedTimestamp{{NodeID: 12, Timestamp: hlc.ClockTimestamp{WallTime: 4}}}, }, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{WallTime: 1, Logical: 2}) @@ -164,14 +165,14 @@ func TestErrorRedaction(t *testing.T) { wrappedPErr := NewError(NewReadWithinUncertaintyIntervalError( hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{WallTime: 2, Logical: 2}, - &Transaction{ + &roachpb.Transaction{ GlobalUncertaintyLimit: hlc.Timestamp{WallTime: 3}, - ObservedTimestamps: []ObservedTimestamp{{NodeID: 12, Timestamp: hlc.ClockTimestamp{WallTime: 4}}}, + ObservedTimestamps: []roachpb.ObservedTimestamp{{NodeID: 12, Timestamp: hlc.ClockTimestamp{WallTime: 4}}}, }, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{WallTime: 1, Logical: 2}, )) - txn := MakeTransaction("foo", Key("bar"), 1, hlc.Timestamp{WallTime: 1}, 1, 99) + txn := roachpb.MakeTransaction("foo", roachpb.Key("bar"), 1, hlc.Timestamp{WallTime: 1}, 1, 99) txn.ID = uuid.Nil txn.Priority = 1234 wrappedPErr.UnexposedTxn = &txn @@ -353,15 +354,15 @@ func TestErrorGRPCStatus(t *testing.T) { } func TestRefreshSpanError(t *testing.T) { - e1 := NewRefreshFailedError(RefreshFailedError_REASON_COMMITTED_VALUE, Key("foo"), hlc.Timestamp{WallTime: 3}) + e1 := NewRefreshFailedError(RefreshFailedError_REASON_COMMITTED_VALUE, roachpb.Key("foo"), hlc.Timestamp{WallTime: 3}) require.Equal(t, "encountered recently written committed value \"foo\" @0.000000003,0", e1.Error()) - e2 := NewRefreshFailedError(RefreshFailedError_REASON_INTENT, Key("bar"), hlc.Timestamp{WallTime: 4}) + e2 := NewRefreshFailedError(RefreshFailedError_REASON_INTENT, roachpb.Key("bar"), hlc.Timestamp{WallTime: 4}) require.Equal(t, "encountered recently written intent \"bar\" @0.000000004,0", e2.Error()) } func TestNotLeaseholderError(t *testing.T) { - rd := &ReplicaDescriptor{ + rd := &roachpb.ReplicaDescriptor{ ReplicaID: 1, StoreID: 1, NodeID: 1, } for _, tc := range []struct { @@ -379,12 +380,12 @@ func TestNotLeaseholderError(t *testing.T) { exp: `[NotLeaseHolderError] r1: replica not lease holder; current lease is repl=(n1,s1):1 seq=2 start=0.000000001,0 epo=1`, err: &NotLeaseHolderError{ RangeID: 1, - Lease: &Lease{ + Lease: &roachpb.Lease{ Start: hlc.ClockTimestamp{WallTime: 1}, Replica: *rd, Epoch: 1, Sequence: 2, - AcquisitionType: LeaseAcquisitionType_Transfer, + AcquisitionType: roachpb.LeaseAcquisitionType_Transfer, }, }, }, diff --git a/pkg/roachpb/gen.bzl b/pkg/kv/kvpb/gen.bzl similarity index 90% rename from pkg/roachpb/gen.bzl rename to pkg/kv/kvpb/gen.bzl index b407c0cc1c9a..562bd9af5c54 100644 --- a/pkg/roachpb/gen.bzl +++ b/pkg/kv/kvpb/gen.bzl @@ -15,6 +15,6 @@ batch_gen = rule( attrs = { "out": attr.output(mandatory = True), "src": attr.label(providers = [GoSource]), - "_tool": attr.label(default = "//pkg/roachpb/gen", executable = True, cfg = "exec"), + "_tool": attr.label(default = "//pkg/kv/kvpb/gen", executable = True, cfg = "exec"), }, ) diff --git a/pkg/roachpb/gen/BUILD.bazel b/pkg/kv/kvpb/gen/BUILD.bazel similarity index 87% rename from pkg/roachpb/gen/BUILD.bazel rename to pkg/kv/kvpb/gen/BUILD.bazel index 7fc10e4c0b74..c83c25062641 100644 --- a/pkg/roachpb/gen/BUILD.bazel +++ b/pkg/kv/kvpb/gen/BUILD.bazel @@ -4,7 +4,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_binary", "go_library") go_library( name = "gen_lib", srcs = ["main.go"], - importpath = "github.com/cockroachdb/cockroach/pkg/roachpb/gen", + importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvpb/gen", visibility = ["//visibility:private"], deps = ["@org_golang_x_tools//go/ast/inspector"], ) diff --git a/pkg/roachpb/gen/main.go b/pkg/kv/kvpb/gen/main.go similarity index 99% rename from pkg/roachpb/gen/main.go rename to pkg/kv/kvpb/gen/main.go index 17b4143d4246..937c7036aa6d 100644 --- a/pkg/roachpb/gen/main.go +++ b/pkg/kv/kvpb/gen/main.go @@ -250,7 +250,7 @@ func main() { fmt.Fprint(f, `// Code generated by genbatch/main.go; DO NOT EDIT. // GENERATED FILE DO NOT EDIT -package roachpb +package kvpb import ( "fmt" diff --git a/pkg/roachpb/roachpbmock/BUILD.bazel b/pkg/kv/kvpb/kvpbmock/BUILD.bazel similarity index 69% rename from pkg/roachpb/roachpbmock/BUILD.bazel rename to pkg/kv/kvpb/kvpbmock/BUILD.bazel index 9c9d0d0f4f11..416e269bc814 100644 --- a/pkg/roachpb/roachpbmock/BUILD.bazel +++ b/pkg/kv/kvpb/kvpbmock/BUILD.bazel @@ -3,16 +3,16 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") load("@bazel_gomock//:gomock.bzl", "gomock") gomock( - name = "mock_roachpb", + name = "mock_kvpb", out = "mocks_generated.go", interfaces = [ "InternalClient", "Internal_RangeFeedClient", "Internal_MuxRangeFeedClient", ], - library = "//pkg/roachpb", - package = "roachpbmock", - self_package = "github.com/cockroachdb/cockroach/pkg/roachpb/roachpbmock", + library = "//pkg/kv/kvpb", + package = "kvpbmock", + self_package = "github.com/cockroachdb/cockroach/pkg/kv/kvpb/kvpbmock", visibility = [ ":__pkg__", "//pkg/gen:__pkg__", @@ -20,13 +20,14 @@ gomock( ) go_library( - name = "roachpbmock", + name = "kvpbmock", srcs = [ - ":mock_roachpb", # keep + ":mock_kvpb", # keep ], - importpath = "github.com/cockroachdb/cockroach/pkg/roachpb/roachpbmock", # keep + importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvpb/kvpbmock", # keep visibility = ["//visibility:public"], deps = [ + "//pkg/kv/kvpb", # keep "//pkg/roachpb", # keep "@com_github_golang_mock//gomock", # keep "@org_golang_google_grpc//:go_default_library", # keep diff --git a/pkg/roachpb/roachpbmock/mocks_generated.go b/pkg/kv/kvpb/kvpbmock/mocks_generated.go similarity index 90% rename from pkg/roachpb/roachpbmock/mocks_generated.go rename to pkg/kv/kvpb/kvpbmock/mocks_generated.go index 484f2db25bb0..2de51b2db533 100644 --- a/pkg/roachpb/roachpbmock/mocks_generated.go +++ b/pkg/kv/kvpb/kvpbmock/mocks_generated.go @@ -1,13 +1,14 @@ // Code generated by MockGen. DO NOT EDIT. -// Source: github.com/cockroachdb/cockroach/pkg/roachpb (interfaces: InternalClient,Internal_RangeFeedClient,Internal_MuxRangeFeedClient) +// Source: github.com/cockroachdb/cockroach/pkg/kv/kvpb (interfaces: InternalClient,Internal_RangeFeedClient,Internal_MuxRangeFeedClient) -// Package roachpbmock is a generated GoMock package. -package roachpbmock +// Package kvpbmock is a generated GoMock package. +package kvpbmock import ( context "context" reflect "reflect" + kvpb "github.com/cockroachdb/cockroach/pkg/kv/kvpb" roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" gomock "github.com/golang/mock/gomock" grpc "google.golang.org/grpc" @@ -38,14 +39,14 @@ func (m *MockInternalClient) EXPECT() *MockInternalClientMockRecorder { } // Batch mocks base method. -func (m *MockInternalClient) Batch(arg0 context.Context, arg1 *roachpb.BatchRequest, arg2 ...grpc.CallOption) (*roachpb.BatchResponse, error) { +func (m *MockInternalClient) Batch(arg0 context.Context, arg1 *kvpb.BatchRequest, arg2 ...grpc.CallOption) (*kvpb.BatchResponse, error) { m.ctrl.T.Helper() varargs := []interface{}{arg0, arg1} for _, a := range arg2 { varargs = append(varargs, a) } ret := m.ctrl.Call(m, "Batch", varargs...) - ret0, _ := ret[0].(*roachpb.BatchResponse) + ret0, _ := ret[0].(*kvpb.BatchResponse) ret1, _ := ret[1].(error) return ret0, ret1 } @@ -78,14 +79,14 @@ func (mr *MockInternalClientMockRecorder) GetAllSystemSpanConfigsThatApply(arg0, } // GetRangeDescriptors mocks base method. -func (m *MockInternalClient) GetRangeDescriptors(arg0 context.Context, arg1 *roachpb.GetRangeDescriptorsRequest, arg2 ...grpc.CallOption) (roachpb.Internal_GetRangeDescriptorsClient, error) { +func (m *MockInternalClient) GetRangeDescriptors(arg0 context.Context, arg1 *kvpb.GetRangeDescriptorsRequest, arg2 ...grpc.CallOption) (kvpb.Internal_GetRangeDescriptorsClient, error) { m.ctrl.T.Helper() varargs := []interface{}{arg0, arg1} for _, a := range arg2 { varargs = append(varargs, a) } ret := m.ctrl.Call(m, "GetRangeDescriptors", varargs...) - ret0, _ := ret[0].(roachpb.Internal_GetRangeDescriptorsClient) + ret0, _ := ret[0].(kvpb.Internal_GetRangeDescriptorsClient) ret1, _ := ret[1].(error) return ret0, ret1 } @@ -118,14 +119,14 @@ func (mr *MockInternalClientMockRecorder) GetSpanConfigs(arg0, arg1 interface{}, } // GossipSubscription mocks base method. -func (m *MockInternalClient) GossipSubscription(arg0 context.Context, arg1 *roachpb.GossipSubscriptionRequest, arg2 ...grpc.CallOption) (roachpb.Internal_GossipSubscriptionClient, error) { +func (m *MockInternalClient) GossipSubscription(arg0 context.Context, arg1 *kvpb.GossipSubscriptionRequest, arg2 ...grpc.CallOption) (kvpb.Internal_GossipSubscriptionClient, error) { m.ctrl.T.Helper() varargs := []interface{}{arg0, arg1} for _, a := range arg2 { varargs = append(varargs, a) } ret := m.ctrl.Call(m, "GossipSubscription", varargs...) - ret0, _ := ret[0].(roachpb.Internal_GossipSubscriptionClient) + ret0, _ := ret[0].(kvpb.Internal_GossipSubscriptionClient) ret1, _ := ret[1].(error) return ret0, ret1 } @@ -138,14 +139,14 @@ func (mr *MockInternalClientMockRecorder) GossipSubscription(arg0, arg1 interfac } // Join mocks base method. -func (m *MockInternalClient) Join(arg0 context.Context, arg1 *roachpb.JoinNodeRequest, arg2 ...grpc.CallOption) (*roachpb.JoinNodeResponse, error) { +func (m *MockInternalClient) Join(arg0 context.Context, arg1 *kvpb.JoinNodeRequest, arg2 ...grpc.CallOption) (*kvpb.JoinNodeResponse, error) { m.ctrl.T.Helper() varargs := []interface{}{arg0, arg1} for _, a := range arg2 { varargs = append(varargs, a) } ret := m.ctrl.Call(m, "Join", varargs...) - ret0, _ := ret[0].(*roachpb.JoinNodeResponse) + ret0, _ := ret[0].(*kvpb.JoinNodeResponse) ret1, _ := ret[1].(error) return ret0, ret1 } @@ -158,14 +159,14 @@ func (mr *MockInternalClientMockRecorder) Join(arg0, arg1 interface{}, arg2 ...i } // MuxRangeFeed mocks base method. -func (m *MockInternalClient) MuxRangeFeed(arg0 context.Context, arg1 ...grpc.CallOption) (roachpb.Internal_MuxRangeFeedClient, error) { +func (m *MockInternalClient) MuxRangeFeed(arg0 context.Context, arg1 ...grpc.CallOption) (kvpb.Internal_MuxRangeFeedClient, error) { m.ctrl.T.Helper() varargs := []interface{}{arg0} for _, a := range arg1 { varargs = append(varargs, a) } ret := m.ctrl.Call(m, "MuxRangeFeed", varargs...) - ret0, _ := ret[0].(roachpb.Internal_MuxRangeFeedClient) + ret0, _ := ret[0].(kvpb.Internal_MuxRangeFeedClient) ret1, _ := ret[1].(error) return ret0, ret1 } @@ -178,14 +179,14 @@ func (mr *MockInternalClientMockRecorder) MuxRangeFeed(arg0 interface{}, arg1 .. } // RangeFeed mocks base method. -func (m *MockInternalClient) RangeFeed(arg0 context.Context, arg1 *roachpb.RangeFeedRequest, arg2 ...grpc.CallOption) (roachpb.Internal_RangeFeedClient, error) { +func (m *MockInternalClient) RangeFeed(arg0 context.Context, arg1 *kvpb.RangeFeedRequest, arg2 ...grpc.CallOption) (kvpb.Internal_RangeFeedClient, error) { m.ctrl.T.Helper() varargs := []interface{}{arg0, arg1} for _, a := range arg2 { varargs = append(varargs, a) } ret := m.ctrl.Call(m, "RangeFeed", varargs...) - ret0, _ := ret[0].(roachpb.Internal_RangeFeedClient) + ret0, _ := ret[0].(kvpb.Internal_RangeFeedClient) ret1, _ := ret[1].(error) return ret0, ret1 } @@ -198,14 +199,14 @@ func (mr *MockInternalClientMockRecorder) RangeFeed(arg0, arg1 interface{}, arg2 } // RangeLookup mocks base method. -func (m *MockInternalClient) RangeLookup(arg0 context.Context, arg1 *roachpb.RangeLookupRequest, arg2 ...grpc.CallOption) (*roachpb.RangeLookupResponse, error) { +func (m *MockInternalClient) RangeLookup(arg0 context.Context, arg1 *kvpb.RangeLookupRequest, arg2 ...grpc.CallOption) (*kvpb.RangeLookupResponse, error) { m.ctrl.T.Helper() varargs := []interface{}{arg0, arg1} for _, a := range arg2 { varargs = append(varargs, a) } ret := m.ctrl.Call(m, "RangeLookup", varargs...) - ret0, _ := ret[0].(*roachpb.RangeLookupResponse) + ret0, _ := ret[0].(*kvpb.RangeLookupResponse) ret1, _ := ret[1].(error) return ret0, ret1 } @@ -218,14 +219,14 @@ func (mr *MockInternalClientMockRecorder) RangeLookup(arg0, arg1 interface{}, ar } // ResetQuorum mocks base method. -func (m *MockInternalClient) ResetQuorum(arg0 context.Context, arg1 *roachpb.ResetQuorumRequest, arg2 ...grpc.CallOption) (*roachpb.ResetQuorumResponse, error) { +func (m *MockInternalClient) ResetQuorum(arg0 context.Context, arg1 *kvpb.ResetQuorumRequest, arg2 ...grpc.CallOption) (*kvpb.ResetQuorumResponse, error) { m.ctrl.T.Helper() varargs := []interface{}{arg0, arg1} for _, a := range arg2 { varargs = append(varargs, a) } ret := m.ctrl.Call(m, "ResetQuorum", varargs...) - ret0, _ := ret[0].(*roachpb.ResetQuorumResponse) + ret0, _ := ret[0].(*kvpb.ResetQuorumResponse) ret1, _ := ret[1].(error) return ret0, ret1 } @@ -258,14 +259,14 @@ func (mr *MockInternalClientMockRecorder) SpanConfigConformance(arg0, arg1 inter } // TenantSettings mocks base method. -func (m *MockInternalClient) TenantSettings(arg0 context.Context, arg1 *roachpb.TenantSettingsRequest, arg2 ...grpc.CallOption) (roachpb.Internal_TenantSettingsClient, error) { +func (m *MockInternalClient) TenantSettings(arg0 context.Context, arg1 *kvpb.TenantSettingsRequest, arg2 ...grpc.CallOption) (kvpb.Internal_TenantSettingsClient, error) { m.ctrl.T.Helper() varargs := []interface{}{arg0, arg1} for _, a := range arg2 { varargs = append(varargs, a) } ret := m.ctrl.Call(m, "TenantSettings", varargs...) - ret0, _ := ret[0].(roachpb.Internal_TenantSettingsClient) + ret0, _ := ret[0].(kvpb.Internal_TenantSettingsClient) ret1, _ := ret[1].(error) return ret0, ret1 } @@ -278,14 +279,14 @@ func (mr *MockInternalClientMockRecorder) TenantSettings(arg0, arg1 interface{}, } // TokenBucket mocks base method. -func (m *MockInternalClient) TokenBucket(arg0 context.Context, arg1 *roachpb.TokenBucketRequest, arg2 ...grpc.CallOption) (*roachpb.TokenBucketResponse, error) { +func (m *MockInternalClient) TokenBucket(arg0 context.Context, arg1 *kvpb.TokenBucketRequest, arg2 ...grpc.CallOption) (*kvpb.TokenBucketResponse, error) { m.ctrl.T.Helper() varargs := []interface{}{arg0, arg1} for _, a := range arg2 { varargs = append(varargs, a) } ret := m.ctrl.Call(m, "TokenBucket", varargs...) - ret0, _ := ret[0].(*roachpb.TokenBucketResponse) + ret0, _ := ret[0].(*kvpb.TokenBucketResponse) ret1, _ := ret[1].(error) return ret0, ret1 } @@ -384,10 +385,10 @@ func (mr *MockInternal_RangeFeedClientMockRecorder) Header() *gomock.Call { } // Recv mocks base method. -func (m *MockInternal_RangeFeedClient) Recv() (*roachpb.RangeFeedEvent, error) { +func (m *MockInternal_RangeFeedClient) Recv() (*kvpb.RangeFeedEvent, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Recv") - ret0, _ := ret[0].(*roachpb.RangeFeedEvent) + ret0, _ := ret[0].(*kvpb.RangeFeedEvent) ret1, _ := ret[1].(error) return ret0, ret1 } @@ -507,10 +508,10 @@ func (mr *MockInternal_MuxRangeFeedClientMockRecorder) Header() *gomock.Call { } // Recv mocks base method. -func (m *MockInternal_MuxRangeFeedClient) Recv() (*roachpb.MuxRangeFeedEvent, error) { +func (m *MockInternal_MuxRangeFeedClient) Recv() (*kvpb.MuxRangeFeedEvent, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Recv") - ret0, _ := ret[0].(*roachpb.MuxRangeFeedEvent) + ret0, _ := ret[0].(*kvpb.MuxRangeFeedEvent) ret1, _ := ret[1].(error) return ret0, ret1 } @@ -536,7 +537,7 @@ func (mr *MockInternal_MuxRangeFeedClientMockRecorder) RecvMsg(arg0 interface{}) } // Send mocks base method. -func (m *MockInternal_MuxRangeFeedClient) Send(arg0 *roachpb.RangeFeedRequest) error { +func (m *MockInternal_MuxRangeFeedClient) Send(arg0 *kvpb.RangeFeedRequest) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Send", arg0) ret0, _ := ret[0].(error) diff --git a/pkg/roachpb/method.go b/pkg/kv/kvpb/method.go similarity index 99% rename from pkg/roachpb/method.go rename to pkg/kv/kvpb/method.go index 421aae2e9f21..223d0aba8072 100644 --- a/pkg/roachpb/method.go +++ b/pkg/kv/kvpb/method.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package roachpb +package kvpb // Method is the enumerated type for methods. type Method int diff --git a/pkg/roachpb/method_string.go b/pkg/kv/kvpb/method_string.go similarity index 99% rename from pkg/roachpb/method_string.go rename to pkg/kv/kvpb/method_string.go index 64947031722b..77c580c5cc58 100644 --- a/pkg/roachpb/method_string.go +++ b/pkg/kv/kvpb/method_string.go @@ -1,6 +1,6 @@ // Code generated by "stringer"; DO NOT EDIT. -package roachpb +package kvpb import "strconv" diff --git a/pkg/roachpb/replica_unavailable_error.go b/pkg/kv/kvpb/replica_unavailable_error.go similarity index 80% rename from pkg/roachpb/replica_unavailable_error.go rename to pkg/kv/kvpb/replica_unavailable_error.go index 75d1e7c2d492..248c4a49e75a 100644 --- a/pkg/roachpb/replica_unavailable_error.go +++ b/pkg/kv/kvpb/replica_unavailable_error.go @@ -8,12 +8,14 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package roachpb +package kvpb import ( context "context" "fmt" + "reflect" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/errors" "github.com/gogo/protobuf/proto" ) @@ -22,7 +24,7 @@ import ( // provided with the range descriptor known to the replica, and the relevant // replica descriptor within. func NewReplicaUnavailableError( - cause error, desc *RangeDescriptor, replDesc ReplicaDescriptor, + cause error, desc *roachpb.RangeDescriptor, replDesc roachpb.ReplicaDescriptor, ) error { return &ReplicaUnavailableError{ Desc: *desc, @@ -55,6 +57,12 @@ func (e *ReplicaUnavailableError) Unwrap() error { } func init() { + // Register the migration of the error that used to be in the roachpb + // package and is now in the kv/kvpb package. + roachpbPath := reflect.TypeOf(roachpb.Key("")).PkgPath() + errors.RegisterTypeMigration(roachpbPath, "*roachpb.ReplicaUnavailableError", &ReplicaUnavailableError{}) + // Note that it is important that these wrapper methods are registered + // _after_ the type migration above. encode := func(ctx context.Context, err error) (msgPrefix string, safeDetails []string, payload proto.Message) { errors.As(err, &payload) // payload = err.(proto.Message) return "", nil, payload diff --git a/pkg/roachpb/replica_unavailable_error_test.go b/pkg/kv/kvpb/replica_unavailable_error_test.go similarity index 96% rename from pkg/roachpb/replica_unavailable_error_test.go rename to pkg/kv/kvpb/replica_unavailable_error_test.go index 6f079dccb0bc..c1d91db10ecf 100644 --- a/pkg/roachpb/replica_unavailable_error_test.go +++ b/pkg/kv/kvpb/replica_unavailable_error_test.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package roachpb_test +package kvpb_test // See TestReplicaUnavailableError in string_test.go; it needed to be moved there // for obscure reasons explained in: diff --git a/pkg/kv/kvpb/string_test.go b/pkg/kv/kvpb/string_test.go new file mode 100644 index 000000000000..f51b45cdaa80 --- /dev/null +++ b/pkg/kv/kvpb/string_test.go @@ -0,0 +1,95 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package kvpb_test + +import ( + "context" + "fmt" + "path/filepath" + "testing" + + // Hook up the pretty printer. + _ "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/testutils/echotest" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/uuid" + "github.com/cockroachdb/errors" + "github.com/cockroachdb/redact" + "github.com/stretchr/testify/require" +) + +func TestBatchRequestString(t *testing.T) { + ba := kvpb.BatchRequest{} + txn := roachpb.MakeTransaction( + "test", + nil, // baseKey + roachpb.NormalUserPriority, + hlc.Timestamp{}, // now + 0, // maxOffsetNs + 99, // coordinatorNodeID + ) + txn.ID = uuid.NamespaceDNS + ba.Txn = &txn + ba.WaitPolicy = lock.WaitPolicy_Error + ba.CanForwardReadTimestamp = true + ba.BoundedStaleness = &kvpb.BoundedStalenessHeader{ + MinTimestampBound: hlc.Timestamp{WallTime: 1}, + MinTimestampBoundStrict: true, + MaxTimestampBound: hlc.Timestamp{WallTime: 2}, + } + for i := 0; i < 100; i++ { + var ru kvpb.RequestUnion + ru.MustSetInner(&kvpb.GetRequest{}) + ba.Requests = append(ba.Requests, ru) + } + var ru kvpb.RequestUnion + ru.MustSetInner(&kvpb.EndTxnRequest{}) + ba.Requests = append(ba.Requests, ru) + + { + exp := `Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min),... 76 skipped ..., Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), EndTxn(abort) [/Min], [txn: 6ba7b810], [wait-policy: Error], [can-forward-ts], [bounded-staleness, min_ts_bound: 0.000000001,0, min_ts_bound_strict, max_ts_bound: 0.000000002,0]` + act := ba.String() + require.Equal(t, exp, act) + } +} + +func TestReplicaUnavailableError(t *testing.T) { + ctx := context.Background() + rDesc := roachpb.ReplicaDescriptor{NodeID: 1, StoreID: 2, ReplicaID: 3} + var set roachpb.ReplicaSet + set.AddReplica(rDesc) + desc := roachpb.NewRangeDescriptor(123, roachpb.RKeyMin, roachpb.RKeyMax, set) + + errSlowProposal := errors.New("slow proposal") + var err = kvpb.NewReplicaUnavailableError(errSlowProposal, desc, rDesc) + err = errors.DecodeError(ctx, errors.EncodeError(ctx, err)) + // Sanity check that Unwrap() was implemented. + require.True(t, errors.Is(err, errSlowProposal), "%+v", err) + require.True(t, errors.HasType(err, (*kvpb.ReplicaUnavailableError)(nil)), "%+v", err) + + s := fmt.Sprintf("%s\n%s", err, redact.Sprint(err)) + echotest.Require(t, s, filepath.Join("testdata", "replica_unavailable_error.txt")) +} + +func TestAmbiguousResultError(t *testing.T) { + ctx := context.Background() + + wrapped := errors.Errorf("boom with a %s", redact.Unsafe("secret")) + var err error = kvpb.NewAmbiguousResultError(wrapped) + err = errors.DecodeError(ctx, errors.EncodeError(ctx, err)) + require.True(t, errors.Is(err, wrapped), "%+v", err) + + s := fmt.Sprintf("%s\n%s", err, redact.Sprint(err)) + echotest.Require(t, s, filepath.Join("testdata", "ambiguous_result_error.txt")) +} diff --git a/pkg/roachpb/testdata/ambiguous_result_error.txt b/pkg/kv/kvpb/testdata/ambiguous_result_error.txt similarity index 100% rename from pkg/roachpb/testdata/ambiguous_result_error.txt rename to pkg/kv/kvpb/testdata/ambiguous_result_error.txt diff --git a/pkg/roachpb/testdata/replica_unavailable_error.txt b/pkg/kv/kvpb/testdata/replica_unavailable_error.txt similarity index 100% rename from pkg/roachpb/testdata/replica_unavailable_error.txt rename to pkg/kv/kvpb/testdata/replica_unavailable_error.txt diff --git a/pkg/kv/kvprober/BUILD.bazel b/pkg/kv/kvprober/BUILD.bazel index 77da3e7ed2e2..d812f7a43cfd 100644 --- a/pkg/kv/kvprober/BUILD.bazel +++ b/pkg/kv/kvprober/BUILD.bazel @@ -47,6 +47,7 @@ go_test( "//pkg/base", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/roachpb", "//pkg/security/securityassets", diff --git a/pkg/kv/kvprober/kvprober_integration_test.go b/pkg/kv/kvprober/kvprober_integration_test.go index 7b56f26cabba..1b1bd78caa83 100644 --- a/pkg/kv/kvprober/kvprober_integration_test.go +++ b/pkg/kv/kvprober/kvprober_integration_test.go @@ -20,9 +20,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvprober" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" @@ -87,11 +87,11 @@ func TestProberDoesReadsAndWrites(t *testing.T) { t.Run("a single range is unavailable for all KV ops", func(t *testing.T) { s, _, p, cleanup := initTestProber(t, base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(i context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + TestingRequestFilter: func(i context.Context, ba *kvpb.BatchRequest) *kvpb.Error { for _, ru := range ba.Requests { key := ru.GetInner().Header().Key if bytes.HasPrefix(key, keys.TimeseriesPrefix) { - return roachpb.NewError(fmt.Errorf("boom")) + return kvpb.NewError(fmt.Errorf("boom")) } } return nil @@ -129,11 +129,11 @@ func TestProberDoesReadsAndWrites(t *testing.T) { s, _, p, cleanup := initTestProber(t, base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(i context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + TestingRequestFilter: func(i context.Context, ba *kvpb.BatchRequest) *kvpb.Error { if !dbIsAvailable.Get() { for _, ru := range ba.Requests { if ru.GetGet() != nil { - return roachpb.NewError(fmt.Errorf("boom")) + return kvpb.NewError(fmt.Errorf("boom")) } } return nil @@ -174,11 +174,11 @@ func TestProberDoesReadsAndWrites(t *testing.T) { s, _, p, cleanup := initTestProber(t, base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(i context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + TestingRequestFilter: func(i context.Context, ba *kvpb.BatchRequest) *kvpb.Error { if !dbIsAvailable.Get() { for _, ru := range ba.Requests { if ru.GetPut() != nil { - return roachpb.NewError(fmt.Errorf("boom")) + return kvpb.NewError(fmt.Errorf("boom")) } } return nil diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index c76cc5a2ad18..6b9edfeaab37 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -114,6 +114,7 @@ go_library( "//pkg/kv/kvbase", "//pkg/kv/kvclient/rangecache", "//pkg/kv/kvnemesis/kvnemesisutil", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/abortspan", "//pkg/kv/kvserver/allocator", "//pkg/kv/kvserver/allocator/allocatorimpl", @@ -345,6 +346,7 @@ go_test( "//pkg/kv/kvclient", "//pkg/kv/kvclient/kvcoord", "//pkg/kv/kvclient/rangefeed/rangefeedcache", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/abortspan", "//pkg/kv/kvserver/allocator", "//pkg/kv/kvserver/allocator/allocatorimpl", diff --git a/pkg/kv/kvserver/addressing_test.go b/pkg/kv/kvserver/addressing_test.go index de3fe2210ec6..0d410206d4ce 100644 --- a/pkg/kv/kvserver/addressing_test.go +++ b/pkg/kv/kvserver/addressing_test.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -172,7 +173,7 @@ func TestUpdateRangeAddressing(t *testing.T) { hlc.MaxTimestamp, storage.MVCCScanOptions{}) if err != nil { // Wait for the intent to be resolved. - if errors.HasType(err, (*roachpb.WriteIntentError)(nil)) { + if errors.HasType(err, (*kvpb.WriteIntentError)(nil)) { return err } t.Fatal(err) diff --git a/pkg/kv/kvserver/allocation_op.go b/pkg/kv/kvserver/allocation_op.go index 06f97131664b..f991ffab28ca 100644 --- a/pkg/kv/kvserver/allocation_op.go +++ b/pkg/kv/kvserver/allocation_op.go @@ -11,6 +11,7 @@ package kvserver import ( + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" @@ -70,7 +71,7 @@ func (o AllocationTransferLeaseOp) trackPlanningMetrics() { type AllocationChangeReplicasOp struct { usage allocator.RangeUsageInfo lhStore roachpb.StoreID - chgs roachpb.ReplicationChanges + chgs kvpb.ReplicationChanges priority kvserverpb.SnapshotRequest_Priority allocatorPriority float64 reason kvserverpb.RangeLogEventReason diff --git a/pkg/kv/kvserver/batcheval/BUILD.bazel b/pkg/kv/kvserver/batcheval/BUILD.bazel index 2878aee6cb8d..c10f21d60ca8 100644 --- a/pkg/kv/kvserver/batcheval/BUILD.bazel +++ b/pkg/kv/kvserver/batcheval/BUILD.bazel @@ -58,6 +58,7 @@ go_library( deps = [ "//pkg/clusterversion", "//pkg/keys", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/abortspan", "//pkg/kv/kvserver/batcheval/result", "//pkg/kv/kvserver/concurrency", @@ -131,6 +132,7 @@ go_test( "//pkg/clusterversion", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/kv/kvserver/abortspan", "//pkg/kv/kvserver/batcheval/result", diff --git a/pkg/kv/kvserver/batcheval/cmd_add_sstable.go b/pkg/kv/kvserver/batcheval/cmd_add_sstable.go index 394332636106..18ba9a417fd2 100644 --- a/pkg/kv/kvserver/batcheval/cmd_add_sstable.go +++ b/pkg/kv/kvserver/batcheval/cmd_add_sstable.go @@ -16,6 +16,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" @@ -37,17 +38,17 @@ func init() { // could instead iterate over the SST and take out point latches/locks, but // the cost is likely not worth it since AddSSTable is often used with // unpopulated spans. - RegisterReadWriteCommand(roachpb.AddSSTable, declareKeysAddSSTable, EvalAddSSTable) + RegisterReadWriteCommand(kvpb.AddSSTable, declareKeysAddSSTable, EvalAddSSTable) } func declareKeysAddSSTable( rs ImmutableRangeState, - header *roachpb.Header, - req roachpb.Request, + header *kvpb.Header, + req kvpb.Request, latchSpans, lockSpans *spanset.SpanSet, maxOffset time.Duration, ) { - args := req.(*roachpb.AddSSTableRequest) + args := req.(*kvpb.AddSSTableRequest) DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans, maxOffset) // We look up the range descriptor key to return its span. latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(rs.GetStartKey())}) @@ -123,9 +124,9 @@ var forceRewrite = util.ConstantWithMetamorphicTestBool("addsst-rewrite-forced", // EvalAddSSTable evaluates an AddSSTable command. For details, see doc comment // on AddSSTableRequest. func EvalAddSSTable( - ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.AddSSTableRequest) + args := cArgs.Args.(*kvpb.AddSSTableRequest) h := cArgs.Header ms := cArgs.Stats start, end := storage.MVCCKey{Key: args.Key}, storage.MVCCKey{Key: args.EndKey} @@ -144,7 +145,7 @@ func EvalAddSSTable( return result.Result{}, err } if remaining := float64(cap.Available) / float64(cap.Capacity); remaining < min { - return result.Result{}, &roachpb.InsufficientSpaceError{ + return result.Result{}, &kvpb.InsufficientSpaceError{ StoreID: cArgs.EvalCtx.StoreID(), Op: "ingest data", Available: cap.Available, @@ -243,7 +244,7 @@ func EvalAddSSTable( if err != nil { return result.Result{}, errors.Wrap(err, "scanning intents") } else if len(intents) > 0 { - return result.Result{}, &roachpb.WriteIntentError{Intents: intents} + return result.Result{}, &kvpb.WriteIntentError{Intents: intents} } } @@ -362,7 +363,7 @@ func EvalAddSSTable( } } - reply := resp.(*roachpb.AddSSTableResponse) + reply := resp.(*kvpb.AddSSTableResponse) reply.RangeSpan = cArgs.EvalCtx.Desc().KeySpan().AsRawSpanWithNoLocals() reply.AvailableBytes = cArgs.EvalCtx.GetMaxBytes() - cArgs.EvalCtx.GetMVCCStats().Total() - stats.Total() diff --git a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go index b76cddfdc671..c3c41cf63799 100644 --- a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" @@ -90,12 +91,12 @@ func TestEvalAddSSTable(t *testing.T) { "blind returns WriteIntentError on conflict": { data: kvs{pointKV("b", intentTS, "b0")}, sst: kvs{pointKV("b", 1, "sst")}, - expectErr: &roachpb.WriteIntentError{}, + expectErr: &kvpb.WriteIntentError{}, }, "blind returns WriteIntentError in span": { data: kvs{pointKV("b", intentTS, "b0")}, sst: kvs{pointKV("a", 1, "sst"), pointKV("c", 1, "sst")}, - expectErr: &roachpb.WriteIntentError{}, + expectErr: &kvpb.WriteIntentError{}, }, "blind ignores intent outside span": { data: kvs{pointKV("b", intentTS, "b0")}, @@ -214,7 +215,7 @@ func TestEvalAddSSTable(t *testing.T) { toReqTS: 1, data: kvs{pointKV("a", intentTS, "intent")}, sst: kvs{pointKV("a", 1, "a@1")}, - expectErr: &roachpb.WriteIntentError{}, + expectErr: &kvpb.WriteIntentError{}, }, "SSTTimestampToRequestTimestamp errors with DisallowConflicts below existing": { reqTS: 5, @@ -222,7 +223,7 @@ func TestEvalAddSSTable(t *testing.T) { noConflict: true, data: kvs{pointKV("a", 5, "a5"), pointKV("b", 7, "b7")}, sst: kvs{pointKV("a", 10, "sst"), pointKV("b", 10, "sst")}, - expectErr: &roachpb.WriteTooOldError{}, + expectErr: &kvpb.WriteTooOldError{}, }, "SSTTimestampToRequestTimestamp succeeds with DisallowConflicts above existing": { reqTS: 8, @@ -319,38 +320,38 @@ func TestEvalAddSSTable(t *testing.T) { noConflict: true, data: kvs{pointKV("a", 3, "a3")}, sst: kvs{pointKV("a", 2, "sst")}, - expectErr: &roachpb.WriteTooOldError{}, + expectErr: &kvpb.WriteTooOldError{}, }, "DisallowConflicts returns WriteTooOldError at existing": { noConflict: true, data: kvs{pointKV("a", 3, "a3")}, sst: kvs{pointKV("a", 3, "sst")}, - expectErr: &roachpb.WriteTooOldError{}, + expectErr: &kvpb.WriteTooOldError{}, }, "DisallowConflicts returns WriteTooOldError at existing tombstone": { noConflict: true, data: kvs{pointKV("a", 3, "")}, sst: kvs{pointKV("a", 3, "sst")}, - expectErr: &roachpb.WriteTooOldError{}, + expectErr: &kvpb.WriteTooOldError{}, }, "DisallowConflicts returns WriteTooOldError at existing range tombstone": { noConflict: true, data: kvs{rangeKV("a", "c", 3, "")}, sst: kvs{pointKV("b", 3, "sst")}, - expectErr: &roachpb.WriteTooOldError{}, + expectErr: &kvpb.WriteTooOldError{}, }, // Regression tests for https://github.com/cockroachdb/cockroach/issues/93968. "DisallowConflicts WriteTooOldError straddling at existing range tombstone": { noConflict: true, data: kvs{rangeKV("b", "z", 3, "")}, sst: kvs{pointKV("a", 3, "sst"), pointKV("c", 3, "sst")}, - expectErr: &roachpb.WriteTooOldError{}, + expectErr: &kvpb.WriteTooOldError{}, }, "DisallowConflicts WriteTooOldError straddling multiple existing range tombstones": { noConflict: true, data: kvs{rangeKV("b", "c", 3, ""), rangeKV("d", "f", 3, "")}, sst: kvs{pointKV("a", 3, "sst"), pointKV("e", 3, "sst")}, - expectErr: &roachpb.WriteTooOldError{}, + expectErr: &kvpb.WriteTooOldError{}, }, "DisallowConflicts WriteTooOldError with SSTTimestampToRequestTimestamp straddling range keys below and at timestamp": { toReqTS: 1, @@ -358,19 +359,19 @@ func TestEvalAddSSTable(t *testing.T) { noConflict: true, data: kvs{rangeKV("a", "c", 2, ""), rangeKV("e", "g", 3, "")}, sst: kvs{pointKV("b", 1, "sst"), pointKV("d", 1, "sst"), pointKV("f", 1, "sst")}, - expectErr: &roachpb.WriteTooOldError{}, + expectErr: &kvpb.WriteTooOldError{}, }, "DisallowConflicts returns WriteIntentError below intent": { noConflict: true, data: kvs{pointKV("a", intentTS, "intent")}, sst: kvs{pointKV("a", 3, "sst")}, - expectErr: &roachpb.WriteIntentError{}, + expectErr: &kvpb.WriteIntentError{}, }, "DisallowConflicts returns WriteIntentError below intent above range key": { noConflict: true, data: kvs{pointKV("b", intentTS, "intent"), rangeKV("a", "d", 2, ""), pointKV("b", 1, "b1")}, sst: kvs{pointKV("b", 3, "sst")}, - expectErr: &roachpb.WriteIntentError{}, + expectErr: &kvpb.WriteIntentError{}, }, "DisallowConflicts ignores intents in span": { // inconsistent with blind writes noConflict: true, @@ -382,7 +383,7 @@ func TestEvalAddSSTable(t *testing.T) { noConflict: true, data: kvs{pointKV("a", 3, "a3")}, sst: kvs{pointKV("a", 3, "a3")}, - expectErr: &roachpb.WriteTooOldError{}, + expectErr: &kvpb.WriteTooOldError{}, }, "DisallowConflicts allows new SST tombstones": { noConflict: true, @@ -450,13 +451,13 @@ func TestEvalAddSSTable(t *testing.T) { noShadow: true, data: kvs{pointKV("a", 3, "")}, sst: kvs{pointKV("a", 3, "sst")}, - expectErr: &roachpb.WriteTooOldError{}, + expectErr: &kvpb.WriteTooOldError{}, }, "DisallowShadowing returns WriteTooOldError below existing tombstone": { noShadow: true, data: kvs{pointKV("a", 3, "")}, sst: kvs{pointKV("a", 2, "sst")}, - expectErr: &roachpb.WriteTooOldError{}, + expectErr: &kvpb.WriteTooOldError{}, }, "DisallowShadowing allows above existing tombstone": { noShadow: true, @@ -468,13 +469,13 @@ func TestEvalAddSSTable(t *testing.T) { noShadow: true, data: kvs{pointKV("a", intentTS, "intent")}, sst: kvs{pointKV("a", 3, "sst")}, - expectErr: &roachpb.WriteIntentError{}, + expectErr: &kvpb.WriteIntentError{}, }, "DisallowShadowing returns WriteIntentError below intent above range key": { noShadow: true, data: kvs{pointKV("b", intentTS, "intent"), rangeKV("a", "d", 2, ""), pointKV("b", 1, "b1")}, sst: kvs{pointKV("b", 3, "sst")}, - expectErr: &roachpb.WriteIntentError{}, + expectErr: &kvpb.WriteIntentError{}, }, "DisallowShadowing ignores intents in span": { // inconsistent with blind writes noShadow: true, @@ -578,13 +579,13 @@ func TestEvalAddSSTable(t *testing.T) { noShadowBelow: 5, data: kvs{pointKV("a", 3, "")}, sst: kvs{pointKV("a", 3, "sst")}, - expectErr: &roachpb.WriteTooOldError{}, + expectErr: &kvpb.WriteTooOldError{}, }, "DisallowShadowingBelow returns WriteTooOldError below existing tombstone": { noShadowBelow: 5, data: kvs{pointKV("a", 3, "")}, sst: kvs{pointKV("a", 2, "sst")}, - expectErr: &roachpb.WriteTooOldError{}, + expectErr: &kvpb.WriteTooOldError{}, }, "DisallowShadowingBelow allows above existing tombstone": { noShadowBelow: 5, @@ -596,13 +597,13 @@ func TestEvalAddSSTable(t *testing.T) { noShadowBelow: 5, data: kvs{pointKV("a", intentTS, "intent")}, sst: kvs{pointKV("a", 3, "sst")}, - expectErr: &roachpb.WriteIntentError{}, + expectErr: &kvpb.WriteIntentError{}, }, "DisallowShadowingBelow returns WriteIntentError below intent above range key": { noShadowBelow: 5, data: kvs{pointKV("b", intentTS, "intent"), rangeKV("a", "d", 2, ""), pointKV("b", 1, "b1")}, sst: kvs{pointKV("b", 3, "sst")}, - expectErr: &roachpb.WriteIntentError{}, + expectErr: &kvpb.WriteIntentError{}, }, "DisallowShadowingBelow ignores intents in span": { // inconsistent with blind writes noShadowBelow: 5, @@ -620,7 +621,7 @@ func TestEvalAddSSTable(t *testing.T) { noShadowBelow: 5, data: kvs{pointKV("a", 3, "")}, sst: kvs{pointKV("a", 3, "")}, - expectErr: &roachpb.WriteTooOldError{}, + expectErr: &kvpb.WriteTooOldError{}, }, "DisallowShadowingBelow allows new SST tombstones": { noShadowBelow: 5, @@ -791,13 +792,13 @@ func TestEvalAddSSTable(t *testing.T) { noShadowBelow: 5, data: kvs{pointKV("a", 8, "a8")}, sst: kvs{pointKV("a", 7, "a8")}, - expectErr: &roachpb.WriteTooOldError{}, + expectErr: &kvpb.WriteTooOldError{}, }, "DisallowShadowingBelow above limit errors below tombstone": { noShadowBelow: 5, data: kvs{pointKV("a", 8, "")}, sst: kvs{pointKV("a", 7, "a8")}, - expectErr: &roachpb.WriteTooOldError{}, + expectErr: &kvpb.WriteTooOldError{}, }, // MVCC Range tombstone cases. "DisallowConflicts allows sst range keys": { @@ -822,25 +823,25 @@ func TestEvalAddSSTable(t *testing.T) { noConflict: true, data: kvs{pointKV("b", intentTS, "intent")}, sst: kvs{rangeKV("a", "c", intentTS+8, "")}, - expectErr: &roachpb.WriteIntentError{}, + expectErr: &kvpb.WriteIntentError{}, }, "DisallowConflicts disallows sst range keys below engine point key": { noConflict: true, data: kvs{pointKV("a", 6, "d")}, sst: kvs{pointKV("a", 7, "a8"), rangeKV("a", "b", 5, "")}, - expectErr: &roachpb.WriteTooOldError{}, + expectErr: &kvpb.WriteTooOldError{}, }, "DisallowConflicts disallows sst point keys below engine range key": { noConflict: true, data: kvs{rangeKV("a", "b", 8, ""), pointKV("a", 6, "b6")}, sst: kvs{pointKV("a", 7, "a8")}, - expectErr: &roachpb.WriteTooOldError{}, + expectErr: &kvpb.WriteTooOldError{}, }, "DisallowConflicts disallows sst range keys below engine range key": { noConflict: true, data: kvs{rangeKV("a", "b", 8, ""), pointKV("a", 6, "d")}, sst: kvs{pointKV("a", 9, "a8"), rangeKV("a", "b", 7, "")}, - expectErr: &roachpb.WriteTooOldError{}, + expectErr: &kvpb.WriteTooOldError{}, }, "DisallowConflicts allows sst range keys above engine range keys": { noConflict: true, @@ -894,7 +895,7 @@ func TestEvalAddSSTable(t *testing.T) { noConflict: true, data: kvs{pointKV("a", 6, "d"), rangeKV("b", "c", 9, ""), rangeKV("c", "d", 5, "")}, sst: kvs{pointKV("a", 7, "a8"), rangeKV("a", "e", 8, "")}, - expectErr: &roachpb.WriteTooOldError{}, + expectErr: &kvpb.WriteTooOldError{}, }, "DisallowConflicts allows sst range keys contained within engine range keys": { noConflict: true, @@ -1047,7 +1048,7 @@ func TestEvalAddSSTable(t *testing.T) { } } sst, start, end := storageutils.MakeSST(t, st, sstKvs) - resp := &roachpb.AddSSTableResponse{} + resp := &kvpb.AddSSTableResponse{} var mvccStats *enginepb.MVCCStats // In the no-overlap case i.e. approxDiskBytes == 0, force a regular // non-prefix Seek in the conflict check. Sending in nil stats @@ -1061,11 +1062,11 @@ func TestEvalAddSSTable(t *testing.T) { result, err := batcheval.EvalAddSSTable(ctx, engine, batcheval.CommandArgs{ EvalCtx: (&batcheval.MockEvalCtx{ClusterSettings: st, Desc: &roachpb.RangeDescriptor{}, ApproxDiskBytes: approxDiskBytes}).EvalContext(), Stats: stats, - Header: roachpb.Header{ + Header: kvpb.Header{ Timestamp: hlc.Timestamp{WallTime: tc.reqTS}, }, - Args: &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, + Args: &kvpb.AddSSTableRequest{ + RequestHeader: kvpb.RequestHeader{Key: start, EndKey: end}, Data: sst, MVCCStats: mvccStats, DisallowConflicts: tc.noConflict, @@ -1210,18 +1211,18 @@ func TestEvalAddSSTableRangefeed(t *testing.T) { sst, start, end := storageutils.MakeSST(t, st, tc.sst) result, err := batcheval.EvalAddSSTable(ctx, opLogger, batcheval.CommandArgs{ EvalCtx: (&batcheval.MockEvalCtx{ClusterSettings: st, Desc: &roachpb.RangeDescriptor{}}).EvalContext(), - Header: roachpb.Header{ + Header: kvpb.Header{ Timestamp: reqTS, }, Stats: &enginepb.MVCCStats{}, - Args: &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, + Args: &kvpb.AddSSTableRequest{ + RequestHeader: kvpb.RequestHeader{Key: start, EndKey: end}, Data: sst, MVCCStats: storageutils.SSTStats(t, sst, 0), SSTTimestampToRequestTimestamp: hlc.Timestamp{WallTime: tc.toReqTS}, IngestAsWrites: tc.asWrites, }, - }, &roachpb.AddSSTableResponse{}) + }, &kvpb.AddSSTableResponse{}) require.NoError(t, err) if tc.asWrites { @@ -1498,16 +1499,16 @@ func TestAddSSTableMVCCStats(t *testing.T) { cArgs := batcheval.CommandArgs{ EvalCtx: evalCtx.EvalContext(), - Header: roachpb.Header{ + Header: kvpb.Header{ Timestamp: ts, }, - Args: &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, + Args: &kvpb.AddSSTableRequest{ + RequestHeader: kvpb.RequestHeader{Key: start, EndKey: end}, Data: sst, }, Stats: &enginepb.MVCCStats{}, } - var resp roachpb.AddSSTableResponse + var resp kvpb.AddSSTableResponse _, err := batcheval.EvalAddSSTable(ctx, engine, cArgs, &resp) require.NoError(t, err) @@ -1530,14 +1531,14 @@ func TestAddSSTableMVCCStats(t *testing.T) { sst, start, end = storageutils.MakeSST(t, st, kvs{pointKV("zzzzzzz", int(ts.WallTime), "zzz")}) cArgs = batcheval.CommandArgs{ EvalCtx: evalCtx.EvalContext(), - Header: roachpb.Header{Timestamp: ts}, - Args: &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, + Header: kvpb.Header{Timestamp: ts}, + Args: &kvpb.AddSSTableRequest{ + RequestHeader: kvpb.RequestHeader{Key: start, EndKey: end}, Data: sst, }, Stats: &enginepb.MVCCStats{}, } - _, err = batcheval.EvalAddSSTable(ctx, engine, cArgs, &roachpb.AddSSTableResponse{}) + _, err = batcheval.EvalAddSSTable(ctx, engine, cArgs, &kvpb.AddSSTableResponse{}) require.NoError(t, err) require.Equal(t, enginepb.MVCCStats{ ContainsEstimates: 1, @@ -1600,18 +1601,18 @@ func TestAddSSTableMVCCStatsDisallowShadowing(t *testing.T) { cArgs := batcheval.CommandArgs{ EvalCtx: evalCtx, - Header: roachpb.Header{ + Header: kvpb.Header{ Timestamp: hlc.Timestamp{WallTime: 7}, }, - Args: &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, + Args: &kvpb.AddSSTableRequest{ + RequestHeader: kvpb.RequestHeader{Key: start, EndKey: end}, Data: sst, DisallowShadowing: true, MVCCStats: storageutils.SSTStats(t, sst, 0), }, Stats: &commandStats, } - _, err := batcheval.EvalAddSSTable(ctx, engine, cArgs, &roachpb.AddSSTableResponse{}) + _, err := batcheval.EvalAddSSTable(ctx, engine, cArgs, &kvpb.AddSSTableResponse{}) require.NoError(t, err) firstSSTStats := commandStats @@ -1629,13 +1630,13 @@ func TestAddSSTableMVCCStatsDisallowShadowing(t *testing.T) { pointKV("h", 6, "hh"), // key has the same timestamp and value as the one present in the existing data. }) - cArgs.Args = &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, + cArgs.Args = &kvpb.AddSSTableRequest{ + RequestHeader: kvpb.RequestHeader{Key: start, EndKey: end}, Data: sst, DisallowShadowing: true, MVCCStats: storageutils.SSTStats(t, sst, 0), } - _, err = batcheval.EvalAddSSTable(ctx, engine, cArgs, &roachpb.AddSSTableResponse{}) + _, err = batcheval.EvalAddSSTable(ctx, engine, cArgs, &kvpb.AddSSTableResponse{}) require.NoError(t, err) // Check that there has been no double counting of stats. All keys in second SST are shadowing. @@ -1654,13 +1655,13 @@ func TestAddSSTableMVCCStatsDisallowShadowing(t *testing.T) { pointKV("x", 7, ""), // new tombstone. }) - cArgs.Args = &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, + cArgs.Args = &kvpb.AddSSTableRequest{ + RequestHeader: kvpb.RequestHeader{Key: start, EndKey: end}, Data: sst, DisallowShadowing: true, MVCCStats: storageutils.SSTStats(t, sst, 0), } - _, err = batcheval.EvalAddSSTable(ctx, engine, cArgs, &roachpb.AddSSTableResponse{}) + _, err = batcheval.EvalAddSSTable(ctx, engine, cArgs, &kvpb.AddSSTableResponse{}) require.NoError(t, err) // This is the stats contribution of the KVs {"e", 2, "ee"} and {"x", 7, ""}. @@ -1705,11 +1706,11 @@ func TestAddSSTableIntentResolution(t *testing.T) { pointKV("b", 1, "2"), pointKV("c", 1, "3"), }) - ba := &roachpb.BatchRequest{ - Header: roachpb.Header{UserPriority: roachpb.MaxUserPriority}, + ba := &kvpb.BatchRequest{ + Header: kvpb.Header{UserPriority: roachpb.MaxUserPriority}, } - ba.Add(&roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, + ba.Add(&kvpb.AddSSTableRequest{ + RequestHeader: kvpb.RequestHeader{Key: start, EndKey: end}, Data: sst, MVCCStats: storageutils.SSTStats(t, sst, 0), DisallowShadowing: true, @@ -1743,14 +1744,14 @@ func TestAddSSTableSSTTimestampToRequestTimestampRespectsTSCache(t *testing.T) { // Add an SST writing below the previous write. sst, start, end := storageutils.MakeSST(t, s.ClusterSettings(), kvs{pointKV("key", 1, "sst")}) - sstReq := &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, + sstReq := &kvpb.AddSSTableRequest{ + RequestHeader: kvpb.RequestHeader{Key: start, EndKey: end}, Data: sst, MVCCStats: storageutils.SSTStats(t, sst, 0), SSTTimestampToRequestTimestamp: hlc.Timestamp{WallTime: 1}, } - ba := &roachpb.BatchRequest{ - Header: roachpb.Header{Timestamp: txnTS.Prev()}, + ba := &kvpb.BatchRequest{ + Header: kvpb.Header{Timestamp: txnTS.Prev()}, } ba.Add(sstReq) _, pErr := db.NonTransactionalSender().Send(ctx, ba) @@ -1764,8 +1765,8 @@ func TestAddSSTableSSTTimestampToRequestTimestampRespectsTSCache(t *testing.T) { // Adding the SST again and reading results in the new value, because the // tscache pushed the SST forward. - ba = &roachpb.BatchRequest{ - Header: roachpb.Header{Timestamp: txnTS.Prev()}, + ba = &kvpb.BatchRequest{ + Header: kvpb.Header{Timestamp: txnTS.Prev()}, } ba.Add(sstReq) _, pErr = db.NonTransactionalSender().Send(ctx, ba) @@ -1807,14 +1808,14 @@ func TestAddSSTableSSTTimestampToRequestTimestampRespectsClosedTS(t *testing.T) // Add an SST writing below the closed timestamp. It should get pushed above it. reqTS := closedTS.Prev() sst, start, end := storageutils.MakeSST(t, store.ClusterSettings(), kvs{pointKV("key", 1, "sst")}) - sstReq := &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, + sstReq := &kvpb.AddSSTableRequest{ + RequestHeader: kvpb.RequestHeader{Key: start, EndKey: end}, Data: sst, MVCCStats: storageutils.SSTStats(t, sst, 0), SSTTimestampToRequestTimestamp: hlc.Timestamp{WallTime: 1}, } - ba := &roachpb.BatchRequest{ - Header: roachpb.Header{Timestamp: reqTS}, + ba := &kvpb.BatchRequest{ + Header: kvpb.Header{Timestamp: reqTS}, } ba.Add(sstReq) result, pErr := db.NonTransactionalSender().Send(ctx, ba) diff --git a/pkg/kv/kvserver/batcheval/cmd_barrier.go b/pkg/kv/kvserver/batcheval/cmd_barrier.go index b5396de511b8..4b4b5e7b1e2c 100644 --- a/pkg/kv/kvserver/batcheval/cmd_barrier.go +++ b/pkg/kv/kvserver/batcheval/cmd_barrier.go @@ -14,20 +14,20 @@ import ( "context" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" ) func init() { - RegisterReadWriteCommand(roachpb.Barrier, declareKeysBarrier, Barrier) + RegisterReadWriteCommand(kvpb.Barrier, declareKeysBarrier, Barrier) } func declareKeysBarrier( _ ImmutableRangeState, - _ *roachpb.Header, - req roachpb.Request, + _ *kvpb.Header, + req kvpb.Request, latchSpans, _ *spanset.SpanSet, _ time.Duration, ) { @@ -47,9 +47,9 @@ func declareKeysBarrier( // Barrier evaluation is a no-op, as all the latch waiting happens in // the latch manager. func Barrier( - _ context.Context, _ storage.ReadWriter, cArgs CommandArgs, response roachpb.Response, + _ context.Context, _ storage.ReadWriter, cArgs CommandArgs, response kvpb.Response, ) (result.Result, error) { - resp := response.(*roachpb.BarrierResponse) + resp := response.(*kvpb.BarrierResponse) resp.Timestamp = cArgs.EvalCtx.Clock().Now() return result.Result{}, nil diff --git a/pkg/kv/kvserver/batcheval/cmd_clear_range.go b/pkg/kv/kvserver/batcheval/cmd_clear_range.go index 46341529681d..1c570ffb46c5 100644 --- a/pkg/kv/kvserver/batcheval/cmd_clear_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_clear_range.go @@ -15,6 +15,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" @@ -34,13 +35,13 @@ import ( const ClearRangeBytesThreshold = 512 << 10 // 512KiB func init() { - RegisterReadWriteCommand(roachpb.ClearRange, declareKeysClearRange, ClearRange) + RegisterReadWriteCommand(kvpb.ClearRange, declareKeysClearRange, ClearRange) } func declareKeysClearRange( rs ImmutableRangeState, - header *roachpb.Header, - req roachpb.Request, + header *kvpb.Header, + req kvpb.Request, latchSpans, lockSpans *spanset.SpanSet, maxOffset time.Duration, ) { @@ -59,7 +60,7 @@ func declareKeysClearRange( // Even if we obtain latches beyond the end range here, it won't cause // contention with the subsequent range because latches are enforced per // range. - args := req.(*roachpb.ClearRangeRequest) + args := req.(*kvpb.ClearRangeRequest) l, r := rangeTombstonePeekBounds(args.Key, args.EndKey, rs.GetStartKey().AsRawKey(), nil) latchSpans.AddMVCC(spanset.SpanReadOnly, roachpb.Span{Key: l, EndKey: r}, header.Timestamp) @@ -78,7 +79,7 @@ func declareKeysClearRange( // or queried any more, such as after a DROP or TRUNCATE table, or // DROP index. func ClearRange( - ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { if cArgs.Header.Txn != nil { return result.Result{}, errors.New("cannot execute ClearRange within a transaction") @@ -86,7 +87,7 @@ func ClearRange( log.VEventf(ctx, 2, "ClearRange %+v", cArgs.Args) // Encode MVCCKey values for start and end of clear span. - args := cArgs.Args.(*roachpb.ClearRangeRequest) + args := cArgs.Args.(*kvpb.ClearRangeRequest) from := args.Key to := args.EndKey @@ -114,7 +115,7 @@ func ClearRange( if err != nil { return result.Result{}, err } else if len(intents) > 0 { - return result.Result{}, &roachpb.WriteIntentError{Intents: intents} + return result.Result{}, &kvpb.WriteIntentError{Intents: intents} } // Before clearing, compute the delta in MVCCStats. diff --git a/pkg/kv/kvserver/batcheval/cmd_clear_range_test.go b/pkg/kv/kvserver/batcheval/cmd_clear_range_test.go index 0b8f56afae12..43a310a8ddf2 100644 --- a/pkg/kv/kvserver/batcheval/cmd_clear_range_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_clear_range_test.go @@ -17,6 +17,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -168,12 +169,12 @@ func TestCmdClearRange(t *testing.T) { Clock: hlc.NewClockForTesting(nil), Stats: stats, }).EvalContext(), - Header: roachpb.Header{ + Header: kvpb.Header{ RangeID: desc.RangeID, Timestamp: hlc.Timestamp{WallTime: nowNanos}, }, - Args: &roachpb.ClearRangeRequest{ - RequestHeader: roachpb.RequestHeader{ + Args: &kvpb.ClearRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: startKey, EndKey: endKey, }, @@ -191,7 +192,7 @@ func TestCmdClearRange(t *testing.T) { defer batch.Close() // Run the request. - result, err := ClearRange(ctx, batch, cArgs, &roachpb.ClearRangeResponse{}) + result, err := ClearRange(ctx, batch, cArgs, &kvpb.ClearRangeResponse{}) require.NoError(t, err) require.NotNil(t, result.Replicated.MVCCHistoryMutation) require.Equal(t, result.Replicated.MVCCHistoryMutation.Spans, []roachpb.Span{{Key: startKey, EndKey: endKey}}) @@ -247,12 +248,12 @@ func TestCmdClearRangeDeadline(t *testing.T) { clock := hlc.NewClockForTesting(timeutil.NewManualTime(timeutil.Unix(0, 123))) - args := roachpb.ClearRangeRequest{ - RequestHeader: roachpb.RequestHeader{Key: startKey, EndKey: endKey}, + args := kvpb.ClearRangeRequest{ + RequestHeader: kvpb.RequestHeader{Key: startKey, EndKey: endKey}, } cArgs := CommandArgs{ - Header: roachpb.Header{RangeID: desc.RangeID}, + Header: kvpb.Header{RangeID: desc.RangeID}, EvalCtx: (&MockEvalCtx{ ClusterSettings: cluster.MakeTestingClusterSettings(), Desc: &desc, @@ -268,26 +269,26 @@ func TestCmdClearRangeDeadline(t *testing.T) { // no deadline args.Deadline = hlc.Timestamp{} - if _, err := ClearRange(ctx, batch, cArgs, &roachpb.ClearRangeResponse{}); err != nil { + if _, err := ClearRange(ctx, batch, cArgs, &kvpb.ClearRangeResponse{}); err != nil { t.Fatal(err) } // before deadline args.Deadline = hlc.Timestamp{WallTime: 124} - if _, err := ClearRange(ctx, batch, cArgs, &roachpb.ClearRangeResponse{}); err != nil { + if _, err := ClearRange(ctx, batch, cArgs, &kvpb.ClearRangeResponse{}); err != nil { t.Fatal(err) } // at deadline. args.Deadline = hlc.Timestamp{WallTime: 123} - if _, err := ClearRange(ctx, batch, cArgs, &roachpb.ClearRangeResponse{}); err == nil { + if _, err := ClearRange(ctx, batch, cArgs, &kvpb.ClearRangeResponse{}); err == nil { t.Fatal("expected deadline error") } // after deadline args.Deadline = hlc.Timestamp{WallTime: 122} if _, err := ClearRange( - ctx, batch, cArgs, &roachpb.ClearRangeResponse{}, + ctx, batch, cArgs, &kvpb.ClearRangeResponse{}, ); !testutils.IsError(err, "ClearRange has deadline") { t.Fatal("expected deadline error") } diff --git a/pkg/kv/kvserver/batcheval/cmd_compute_checksum.go b/pkg/kv/kvserver/batcheval/cmd_compute_checksum.go index 48a1a7236259..131db73f7607 100644 --- a/pkg/kv/kvserver/batcheval/cmd_compute_checksum.go +++ b/pkg/kv/kvserver/batcheval/cmd_compute_checksum.go @@ -15,6 +15,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" @@ -24,13 +25,13 @@ import ( ) func init() { - RegisterReadOnlyCommand(roachpb.ComputeChecksum, declareKeysComputeChecksum, ComputeChecksum) + RegisterReadOnlyCommand(kvpb.ComputeChecksum, declareKeysComputeChecksum, ComputeChecksum) } func declareKeysComputeChecksum( rs ImmutableRangeState, - _ *roachpb.Header, - _ roachpb.Request, + _ *kvpb.Header, + _ kvpb.Request, latchSpans, _ *spanset.SpanSet, _ time.Duration, ) { @@ -54,11 +55,11 @@ const ReplicaChecksumVersion = 4 // a particular snapshot. The checksum is later verified through a // CollectChecksumRequest. func ComputeChecksum( - _ context.Context, _ storage.Reader, cArgs CommandArgs, resp roachpb.Response, + _ context.Context, _ storage.Reader, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.ComputeChecksumRequest) + args := cArgs.Args.(*kvpb.ComputeChecksumRequest) - reply := resp.(*roachpb.ComputeChecksumResponse) + reply := resp.(*kvpb.ComputeChecksumResponse) reply.ChecksumID = uuid.MakeV4() var pd result.Result diff --git a/pkg/kv/kvserver/batcheval/cmd_conditional_put.go b/pkg/kv/kvserver/batcheval/cmd_conditional_put.go index 7043d2fdff73..75bd4e1ec1bd 100644 --- a/pkg/kv/kvserver/batcheval/cmd_conditional_put.go +++ b/pkg/kv/kvserver/batcheval/cmd_conditional_put.go @@ -14,25 +14,25 @@ import ( "context" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/hlc" ) func init() { - RegisterReadWriteCommand(roachpb.ConditionalPut, declareKeysConditionalPut, ConditionalPut) + RegisterReadWriteCommand(kvpb.ConditionalPut, declareKeysConditionalPut, ConditionalPut) } func declareKeysConditionalPut( rs ImmutableRangeState, - header *roachpb.Header, - req roachpb.Request, + header *kvpb.Header, + req kvpb.Request, latchSpans, lockSpans *spanset.SpanSet, maxOffset time.Duration, ) { - args := req.(*roachpb.ConditionalPutRequest) + args := req.(*kvpb.ConditionalPutRequest) if args.Inline { DefaultDeclareKeys(rs, header, req, latchSpans, lockSpans, maxOffset) } else { @@ -44,9 +44,9 @@ func declareKeysConditionalPut( // the expected value matches. If not, the return value contains // the actual value. func ConditionalPut( - ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.ConditionalPutRequest) + args := cArgs.Args.(*kvpb.ConditionalPutRequest) h := cArgs.Header var ts hlc.Timestamp diff --git a/pkg/kv/kvserver/batcheval/cmd_delete.go b/pkg/kv/kvserver/batcheval/cmd_delete.go index 58fb5fd2cc7e..cbefdc473979 100644 --- a/pkg/kv/kvserver/batcheval/cmd_delete.go +++ b/pkg/kv/kvserver/batcheval/cmd_delete.go @@ -13,23 +13,23 @@ package batcheval import ( "context" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" ) func init() { - RegisterReadWriteCommand(roachpb.Delete, DefaultDeclareIsolatedKeys, Delete) + RegisterReadWriteCommand(kvpb.Delete, DefaultDeclareIsolatedKeys, Delete) } // Delete deletes the key and value specified by key. func Delete( - ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.DeleteRequest) + args := cArgs.Args.(*kvpb.DeleteRequest) h := cArgs.Header - reply := resp.(*roachpb.DeleteResponse) + reply := resp.(*kvpb.DeleteResponse) var err error reply.FoundKey, err = storage.MVCCDelete( diff --git a/pkg/kv/kvserver/batcheval/cmd_delete_range.go b/pkg/kv/kvserver/batcheval/cmd_delete_range.go index f5df0b69dec4..c3b9df8b50fe 100644 --- a/pkg/kv/kvserver/batcheval/cmd_delete_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_delete_range.go @@ -15,6 +15,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" @@ -26,17 +27,17 @@ import ( ) func init() { - RegisterReadWriteCommand(roachpb.DeleteRange, declareKeysDeleteRange, DeleteRange) + RegisterReadWriteCommand(kvpb.DeleteRange, declareKeysDeleteRange, DeleteRange) } func declareKeysDeleteRange( rs ImmutableRangeState, - header *roachpb.Header, - req roachpb.Request, + header *kvpb.Header, + req kvpb.Request, latchSpans, lockSpans *spanset.SpanSet, maxOffset time.Duration, ) { - args := req.(*roachpb.DeleteRangeRequest) + args := req.(*kvpb.DeleteRangeRequest) if args.Inline { DefaultDeclareKeys(rs, header, req, latchSpans, lockSpans, maxOffset) } else { @@ -81,13 +82,13 @@ const maxDeleteRangeBatchBytes = 32 << 20 // DeleteRange deletes the range of key/value pairs specified by // start and end keys. func DeleteRange( - ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.DeleteRangeRequest) + args := cArgs.Args.(*kvpb.DeleteRangeRequest) h := cArgs.Header - reply := resp.(*roachpb.DeleteRangeResponse) + reply := resp.(*kvpb.DeleteRangeResponse) - if args.Predicates != (roachpb.DeleteRangePredicates{}) && !args.UseRangeTombstone { + if args.Predicates != (kvpb.DeleteRangePredicates{}) && !args.UseRangeTombstone { // This ensures predicate based DeleteRange piggybacks on the version gate, // roachpb api flags, and latch declarations used by the UseRangeTombstone. return result.Result{}, errors.AssertionFailedf( @@ -151,7 +152,7 @@ func DeleteRange( // If no predicate parameters are passed, use the fast path. If we're // deleting the entire Raft range, use an even faster path that avoids a // point key scan to update MVCC stats. - if args.Predicates == (roachpb.DeleteRangePredicates{}) { + if args.Predicates == (kvpb.DeleteRangePredicates{}) { var statsCovered *enginepb.MVCCStats if args.Key.Equal(desc.StartKey.AsRawKey()) && args.EndKey.Equal(desc.EndKey.AsRawKey()) { // NB: We take the fast path even if stats are estimates, because the @@ -195,7 +196,7 @@ func DeleteRange( if resumeSpan != nil { reply.ResumeSpan = resumeSpan - reply.ResumeReason = roachpb.RESUME_KEY_LIMIT + reply.ResumeReason = kvpb.RESUME_KEY_LIMIT // Note: While MVCCPredicateDeleteRange _could_ return reply.NumKeys, as // the number of keys iterated through, doing so could lead to a @@ -230,7 +231,7 @@ func DeleteRange( reply.NumKeys = num if resumeSpan != nil { reply.ResumeSpan = resumeSpan - reply.ResumeReason = roachpb.RESUME_KEY_LIMIT + reply.ResumeReason = kvpb.RESUME_KEY_LIMIT } // If requested, replace point tombstones with range tombstones. diff --git a/pkg/kv/kvserver/batcheval/cmd_delete_range_gchint_test.go b/pkg/kv/kvserver/batcheval/cmd_delete_range_gchint_test.go index 1f415d615ff4..4f2dae75c489 100644 --- a/pkg/kv/kvserver/batcheval/cmd_delete_range_gchint_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_delete_range_gchint_test.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server" @@ -51,8 +52,8 @@ func TestDeleteRangeTombstoneSetsGCHint(t *testing.T) { gcHint := repl.GetGCHint() require.True(t, gcHint.LatestRangeDeleteTimestamp.IsEmpty(), "gc hint should be empty by default") - pArgs := &roachpb.PutRequest{ - RequestHeader: roachpb.RequestHeader{ + pArgs := &kvpb.PutRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, }, Value: roachpb.MakeValueFromBytes(content), @@ -64,10 +65,10 @@ func TestDeleteRangeTombstoneSetsGCHint(t *testing.T) { r, err := s.LookupRange(key) require.NoError(t, err, "failed to lookup range") - drArgs := &roachpb.DeleteRangeRequest{ + drArgs := &kvpb.DeleteRangeRequest{ UpdateRangeDeleteGCHint: true, UseRangeTombstone: true, - RequestHeader: roachpb.RequestHeader{ + RequestHeader: kvpb.RequestHeader{ Key: r.StartKey.AsRawKey(), EndKey: r.EndKey.AsRawKey(), }, diff --git a/pkg/kv/kvserver/batcheval/cmd_delete_range_test.go b/pkg/kv/kvserver/batcheval/cmd_delete_range_test.go index c5a7de6695fb..39743c6dbde8 100644 --- a/pkg/kv/kvserver/batcheval/cmd_delete_range_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_delete_range_test.go @@ -17,6 +17,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -146,19 +147,19 @@ func TestDeleteRangeTombstone(t *testing.T) { start: "i", end: "j", ts: 10e9, - expectErr: &roachpb.WriteIntentError{}, + expectErr: &kvpb.WriteIntentError{}, }, "below point errors with WriteTooOldError": { start: "a", end: "d", ts: 1e9, - expectErr: &roachpb.WriteTooOldError{}, + expectErr: &kvpb.WriteTooOldError{}, }, "below range tombstone errors with WriteTooOldError": { start: "f", end: "h", ts: 1e9, - expectErr: &roachpb.WriteTooOldError{}, + expectErr: &kvpb.WriteTooOldError{}, }, "predicate without UsingRangeTombstone error": { start: "a", @@ -210,29 +211,29 @@ func TestDeleteRangeTombstone(t *testing.T) { }, } - h := roachpb.Header{ + h := kvpb.Header{ Timestamp: rangeKey.Timestamp, } if tc.txn { txn := roachpb.MakeTransaction("txn", nil /* baseKey */, roachpb.NormalUserPriority, rangeKey.Timestamp, 0, 0) h.Txn = &txn } - var predicates roachpb.DeleteRangePredicates + var predicates kvpb.DeleteRangePredicates if runWithPredicates { - predicates = roachpb.DeleteRangePredicates{ + predicates = kvpb.DeleteRangePredicates{ StartTime: hlc.Timestamp{WallTime: 1}, } h.MaxSpanRequestKeys = math.MaxInt64 } if tc.predicateStartTime > 0 { - predicates = roachpb.DeleteRangePredicates{ + predicates = kvpb.DeleteRangePredicates{ StartTime: hlc.Timestamp{WallTime: tc.predicateStartTime}, } h.MaxSpanRequestKeys = tc.maxBatchSize } - req := &roachpb.DeleteRangeRequest{ - RequestHeader: roachpb.RequestHeader{ + req := &kvpb.DeleteRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: rangeKey.StartKey, EndKey: rangeKey.EndKey, }, @@ -255,7 +256,7 @@ func TestDeleteRangeTombstone(t *testing.T) { defer batch.Close() // Run the request. - resp := &roachpb.DeleteRangeResponse{} + resp := &kvpb.DeleteRangeResponse{} _, err := DeleteRange(ctx, batch, CommandArgs{ EvalCtx: evalCtx.EvalContext(), Stats: &ms, diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go index 766e8615cd41..e83b195a8413 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go @@ -18,6 +18,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/abortspan" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/gc" @@ -38,13 +39,13 @@ import ( ) func init() { - RegisterReadWriteCommand(roachpb.EndTxn, declareKeysEndTxn, EndTxn) + RegisterReadWriteCommand(kvpb.EndTxn, declareKeysEndTxn, EndTxn) } // declareKeysWriteTransaction is the shared portion of // declareKeys{End,Heartbeat}Transaction. func declareKeysWriteTransaction( - _ ImmutableRangeState, header *roachpb.Header, req roachpb.Request, latchSpans *spanset.SpanSet, + _ ImmutableRangeState, header *kvpb.Header, req kvpb.Request, latchSpans *spanset.SpanSet, ) { if header.Txn != nil { header.Txn.AssertInitialized(context.TODO()) @@ -56,12 +57,12 @@ func declareKeysWriteTransaction( func declareKeysEndTxn( rs ImmutableRangeState, - header *roachpb.Header, - req roachpb.Request, + header *kvpb.Header, + req kvpb.Request, latchSpans, _ *spanset.SpanSet, _ time.Duration, ) { - et := req.(*roachpb.EndTxnRequest) + et := req.(*kvpb.EndTxnRequest) declareKeysWriteTransaction(rs, header, req, latchSpans) var minTxnTS hlc.Timestamp if header.Txn != nil { @@ -211,12 +212,12 @@ func declareKeysEndTxn( // TODO(nvanbenschoten): rename this file to cmd_end_txn.go once some of andrei's // recent PRs have landed. func EndTxn( - ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.EndTxnRequest) + args := cArgs.Args.(*kvpb.EndTxnRequest) h := cArgs.Header ms := cArgs.Stats - reply := resp.(*roachpb.EndTxnResponse) + reply := resp.(*kvpb.EndTxnResponse) if err := VerifyTransaction(h, args, roachpb.PENDING, roachpb.STAGING, roachpb.ABORTED); err != nil { return result.Result{}, err @@ -272,8 +273,8 @@ func EndTxn( // meantime. The TransactionStatusError is going to be handled by the // txnCommitter interceptor. log.VEventf(ctx, 2, "transaction found to be already committed") - return result.Result{}, roachpb.NewTransactionStatusError( - roachpb.TransactionStatusError_REASON_TXN_COMMITTED, + return result.Result{}, kvpb.NewTransactionStatusError( + kvpb.TransactionStatusError_REASON_TXN_COMMITTED, "already committed") case roachpb.ABORTED: @@ -307,7 +308,7 @@ func EndTxn( // can go. reply.Txn.LockSpans = args.LockSpans return result.FromEndTxn(reply.Txn, true /* alwaysReturn */, args.Poison), - roachpb.NewTransactionAbortedError(roachpb.ABORT_REASON_ABORTED_RECORD_FOUND) + kvpb.NewTransactionAbortedError(kvpb.ABORT_REASON_ABORTED_RECORD_FOUND) case roachpb.PENDING: if h.Txn.Epoch < reply.Txn.Epoch { @@ -360,7 +361,7 @@ func EndTxn( // request is marking the commit as explicit, this check must succeed. We // assert this in txnCommitter.makeTxnCommitExplicitAsync. if retry, reason, extraMsg := IsEndTxnTriggeringRetryError(reply.Txn, args); retry { - return result.Result{}, roachpb.NewTransactionRetryError(reason, extraMsg) + return result.Result{}, kvpb.NewTransactionRetryError(reason, extraMsg) } // If the transaction needs to be staged as part of an implicit commit @@ -416,7 +417,7 @@ func EndTxn( // committed. Doing so is only possible if we can guarantee that under no // circumstances can an implicitly committed transaction be rolled back. if reply.Txn.Status == roachpb.STAGING { - err := roachpb.NewIndeterminateCommitError(*reply.Txn) + err := kvpb.NewIndeterminateCommitError(*reply.Txn) log.VEventf(ctx, 1, "%v", err) return result.Result{}, err } @@ -493,12 +494,12 @@ func IsEndTxnExceedingDeadline(commitTS hlc.Timestamp, deadline hlc.Timestamp) b // committed and needs to return a TransactionRetryError. It also returns the // reason and possibly an extra message to be used for the error. func IsEndTxnTriggeringRetryError( - txn *roachpb.Transaction, args *roachpb.EndTxnRequest, -) (retry bool, reason roachpb.TransactionRetryReason, extraMsg redact.RedactableString) { + txn *roachpb.Transaction, args *kvpb.EndTxnRequest, +) (retry bool, reason kvpb.TransactionRetryReason, extraMsg redact.RedactableString) { // If we saw any WriteTooOldErrors, we must restart to avoid lost // update anomalies. if txn.WriteTooOld { - retry, reason = true, roachpb.RETRY_WRITE_TOO_OLD + retry, reason = true, kvpb.RETRY_WRITE_TOO_OLD } else { readTimestamp := txn.ReadTimestamp isTxnPushed := txn.WriteTimestamp != readTimestamp @@ -506,7 +507,7 @@ func IsEndTxnTriggeringRetryError( // Return a transaction retry error if the commit timestamp isn't equal to // the txn timestamp. if isTxnPushed { - retry, reason = true, roachpb.RETRY_SERIALIZABLE + retry, reason = true, kvpb.RETRY_SERIALIZABLE } } @@ -516,7 +517,7 @@ func IsEndTxnTriggeringRetryError( extraMsg = redact.Sprintf( "txn timestamp pushed too much; deadline exceeded by %s (%s > %s)", exceededBy, txn.WriteTimestamp, args.Deadline) - retry, reason = true, roachpb.RETRY_COMMIT_DEADLINE_EXCEEDED + retry, reason = true, kvpb.RETRY_COMMIT_DEADLINE_EXCEEDED } return retry, reason, extraMsg } @@ -535,7 +536,7 @@ func resolveLocalLocks( desc *roachpb.RangeDescriptor, readWriter storage.ReadWriter, ms *enginepb.MVCCStats, - args *roachpb.EndTxnRequest, + args *kvpb.EndTxnRequest, txn *roachpb.Transaction, evalCtx EvalContext, ) (resolvedLocks []roachpb.LockUpdate, externalLocks []roachpb.Span, _ error) { @@ -652,7 +653,7 @@ func updateStagingTxn( readWriter storage.ReadWriter, ms *enginepb.MVCCStats, key []byte, - args *roachpb.EndTxnRequest, + args *kvpb.EndTxnRequest, txn *roachpb.Transaction, ) error { txn.LockSpans = args.LockSpans @@ -670,7 +671,7 @@ func updateFinalizedTxn( readWriter storage.ReadWriter, ms *enginepb.MVCCStats, key []byte, - args *roachpb.EndTxnRequest, + args *kvpb.EndTxnRequest, txn *roachpb.Transaction, recordAlreadyExisted bool, externalLocks []roachpb.Span, @@ -700,7 +701,7 @@ func RunCommitTrigger( rec EvalContext, batch storage.Batch, ms *enginepb.MVCCStats, - args *roachpb.EndTxnRequest, + args *kvpb.EndTxnRequest, txn *roachpb.Transaction, ) (result.Result, error) { ct := args.InternalCommitTrigger @@ -729,7 +730,7 @@ func RunCommitTrigger( ctx, rec, batch, *ms, ct.SplitTrigger, txn.WriteTimestamp, ) if err != nil { - return result.Result{}, roachpb.NewReplicaCorruptionError(err) + return result.Result{}, kvpb.NewReplicaCorruptionError(err) } *ms = newMS return res, nil @@ -737,7 +738,7 @@ func RunCommitTrigger( if mt := ct.GetMergeTrigger(); mt != nil { res, err := mergeTrigger(ctx, rec, batch, ms, mt, txn.WriteTimestamp) if err != nil { - return result.Result{}, roachpb.NewReplicaCorruptionError(err) + return result.Result{}, kvpb.NewReplicaCorruptionError(err) } return res, nil } diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go index 71e1ebac4389..51d7e2a4d307 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go @@ -16,6 +16,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/abortspan" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" @@ -1021,8 +1022,8 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { } // Issue an EndTxn request. - req := roachpb.EndTxnRequest{ - RequestHeader: roachpb.RequestHeader{Key: txn.Key}, + req := kvpb.EndTxnRequest{ + RequestHeader: kvpb.RequestHeader{Key: txn.Key}, Commit: c.commit, InFlightWrites: c.inFlightWrites, @@ -1031,23 +1032,23 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { if !c.noLockSpans { req.LockSpans = intents } - var resp roachpb.EndTxnResponse + var resp kvpb.EndTxnResponse _, err := EndTxn(ctx, batch, CommandArgs{ EvalCtx: (&MockEvalCtx{ Desc: &desc, AbortSpan: as, - CanCreateTxnRecordFn: func() (bool, roachpb.TransactionAbortedReason) { + CanCreateTxnRecordFn: func() (bool, kvpb.TransactionAbortedReason) { if c.canCreateTxn { return true, 0 } - return false, roachpb.ABORT_REASON_ABORTED_RECORD_FOUND + return false, kvpb.ABORT_REASON_ABORTED_RECORD_FOUND }, MinTxnCommitTSFn: func() hlc.Timestamp { return c.minTxnCommitTS }, }).EvalContext(), Args: &req, - Header: roachpb.Header{ + Header: kvpb.Header{ Timestamp: ts, Txn: c.headerTxn, }, @@ -1140,18 +1141,18 @@ func TestPartialRollbackOnEndTransaction(t *testing.T) { } // Issue the end txn command. - req := roachpb.EndTxnRequest{ - RequestHeader: roachpb.RequestHeader{Key: txn.Key}, + req := kvpb.EndTxnRequest{ + RequestHeader: kvpb.RequestHeader{Key: txn.Key}, Commit: true, LockSpans: intents, } - var resp roachpb.EndTxnResponse + var resp kvpb.EndTxnResponse if _, err := EndTxn(ctx, batch, CommandArgs{ EvalCtx: (&MockEvalCtx{ Desc: &desc, }).EvalContext(), Args: &req, - Header: roachpb.Header{ + Header: kvpb.Header{ Timestamp: ts, Txn: &txn, }, @@ -1247,8 +1248,8 @@ func TestCommitWaitBeforeIntentResolutionIfCommitTrigger(t *testing.T) { txn.WriteTimestamp = commitTS // Issue the end txn command. - req := roachpb.EndTxnRequest{ - RequestHeader: roachpb.RequestHeader{Key: txn.Key}, + req := kvpb.EndTxnRequest{ + RequestHeader: kvpb.RequestHeader{Key: txn.Key}, Commit: true, } if commitTrigger { @@ -1256,14 +1257,14 @@ func TestCommitWaitBeforeIntentResolutionIfCommitTrigger(t *testing.T) { ModifiedSpanTrigger: &roachpb.ModifiedSpanTrigger{NodeLivenessSpan: &roachpb.Span{}}, } } - var resp roachpb.EndTxnResponse + var resp kvpb.EndTxnResponse _, err := EndTxn(ctx, batch, CommandArgs{ EvalCtx: (&MockEvalCtx{ Desc: &desc, Clock: clock, }).EvalContext(), Args: &req, - Header: roachpb.Header{ + Header: kvpb.Header{ Timestamp: commitTS, Txn: &txn, }, diff --git a/pkg/kv/kvserver/batcheval/cmd_export.go b/pkg/kv/kvserver/batcheval/cmd_export.go index 7f1a9f706e12..8ff1e07f7e66 100644 --- a/pkg/kv/kvserver/batcheval/cmd_export.go +++ b/pkg/kv/kvserver/batcheval/cmd_export.go @@ -16,6 +16,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -62,13 +63,13 @@ var ExportRequestMaxAllowedFileSizeOverage = settings.RegisterByteSizeSetting( ).WithPublic() func init() { - RegisterReadOnlyCommand(roachpb.Export, declareKeysExport, evalExport) + RegisterReadOnlyCommand(kvpb.Export, declareKeysExport, evalExport) } func declareKeysExport( rs ImmutableRangeState, - header *roachpb.Header, - req roachpb.Request, + header *kvpb.Header, + req kvpb.Request, latchSpans, lockSpans *spanset.SpanSet, maxOffset time.Duration, ) { @@ -84,11 +85,11 @@ func declareKeysExport( // evalExport dumps the requested keys into files of non-overlapping key ranges // in a format suitable for bulk ingest. func evalExport( - ctx context.Context, reader storage.Reader, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, reader storage.Reader, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.ExportRequest) + args := cArgs.Args.(*kvpb.ExportRequest) h := cArgs.Header - reply := resp.(*roachpb.ExportResponse) + reply := resp.(*kvpb.ExportResponse) ctx, evalExportSpan := tracing.ChildSpan(ctx, "evalExport") defer evalExportSpan.Finish() @@ -116,7 +117,7 @@ func evalExport( // NOTE: Since export requests may not be holding latches during evaluation, // this `GetGCThreshold()` call is going to potentially return a higher GC // threshold than the pebble state we're evaluating over. This is copacetic. - if args.MVCCFilter == roachpb.MVCCFilter_All { + if args.MVCCFilter == kvpb.MVCCFilter_All { reply.StartTime = args.StartTime if args.StartTime.IsEmpty() { reply.StartTime = cArgs.EvalCtx.GetGCThreshold() @@ -125,9 +126,9 @@ func evalExport( var exportAllRevisions bool switch args.MVCCFilter { - case roachpb.MVCCFilter_Latest: + case kvpb.MVCCFilter_Latest: exportAllRevisions = false - case roachpb.MVCCFilter_All: + case kvpb.MVCCFilter_All: exportAllRevisions = true default: return result.Result{}, errors.Errorf("unknown MVCC filter: %s", args.MVCCFilter) @@ -182,7 +183,7 @@ func evalExport( MaxIntents: maxIntents, StopMidKey: args.SplitMidKey, } - var summary roachpb.BulkOpSummary + var summary kvpb.BulkOpSummary var resume storage.MVCCKey var fingerprint uint64 var err error @@ -241,20 +242,20 @@ func evalExport( span.EndKey = args.EndKey } - var exported roachpb.ExportResponse_File + var exported kvpb.ExportResponse_File if args.ExportFingerprint { // A fingerprinting ExportRequest does not need to return the // BulkOpSummary or the exported Span. This is because we do not expect // the sender of a fingerprint ExportRequest to use anything but the // `Fingerprint` for point-keys and the SST file that contains the // rangekeys we encountered during ExportRequest evaluation. - exported = roachpb.ExportResponse_File{ + exported = kvpb.ExportResponse_File{ EndKeyTS: resume.Timestamp, SST: data, Fingerprint: fingerprint, } } else { - exported = roachpb.ExportResponse_File{ + exported = kvpb.ExportResponse_File{ Span: span, EndKeyTS: resume.Timestamp, Exported: summary, @@ -299,7 +300,7 @@ func evalExport( Key: resume.Key, EndKey: args.EndKey, } - reply.ResumeReason = roachpb.RESUME_BYTE_LIMIT + reply.ResumeReason = kvpb.RESUME_BYTE_LIMIT } break } diff --git a/pkg/kv/kvserver/batcheval/cmd_export_test.go b/pkg/kv/kvserver/batcheval/cmd_export_test.go index 05999dea4668..f67a0743a7c8 100644 --- a/pkg/kv/kvserver/batcheval/cmd_export_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_export_test.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -50,22 +51,22 @@ func TestExportCmd(t *testing.T) { kvDB := tc.Server(0).DB() export := func( - t *testing.T, start hlc.Timestamp, mvccFilter roachpb.MVCCFilter, maxResponseSSTBytes int64, - ) (roachpb.Response, *roachpb.Error) { - req := &roachpb.ExportRequest{ - RequestHeader: roachpb.RequestHeader{Key: bootstrap.TestingUserTableDataMin(), EndKey: keys.MaxKey}, + t *testing.T, start hlc.Timestamp, mvccFilter kvpb.MVCCFilter, maxResponseSSTBytes int64, + ) (kvpb.Response, *kvpb.Error) { + req := &kvpb.ExportRequest{ + RequestHeader: kvpb.RequestHeader{Key: bootstrap.TestingUserTableDataMin(), EndKey: keys.MaxKey}, StartTime: start, MVCCFilter: mvccFilter, TargetFileSize: batcheval.ExportRequestTargetFileSize.Get(&tc.Server(0).ClusterSettings().SV), } - var h roachpb.Header + var h kvpb.Header h.TargetBytes = maxResponseSSTBytes return kv.SendWrappedWith(ctx, kvDB.NonTransactionalSender(), h, req) } exportAndSlurpOne := func( - t *testing.T, start hlc.Timestamp, mvccFilter roachpb.MVCCFilter, maxResponseSSTBytes int64, - ) ([]string, []storage.MVCCKeyValue, roachpb.ResponseHeader) { + t *testing.T, start hlc.Timestamp, mvccFilter kvpb.MVCCFilter, maxResponseSSTBytes int64, + ) ([]string, []storage.MVCCKeyValue, kvpb.ResponseHeader) { res, pErr := export(t, start, mvccFilter, maxResponseSSTBytes) if pErr != nil { t.Fatalf("%+v", pErr) @@ -73,7 +74,7 @@ func TestExportCmd(t *testing.T) { var paths []string var kvs []storage.MVCCKeyValue - for _, file := range res.(*roachpb.ExportResponse).Files { + for _, file := range res.(*kvpb.ExportResponse).Files { paths = append(paths, file.Path) iterOpts := storage.IterOptions{ KeyTypes: storage.IterKeyTypePointsOnly, @@ -105,7 +106,7 @@ func TestExportCmd(t *testing.T) { } } - return paths, kvs, res.(*roachpb.ExportResponse).Header() + return paths, kvs, res.(*kvpb.ExportResponse).Header() } type ExportAndSlurpResult struct { end hlc.Timestamp @@ -113,17 +114,17 @@ func TestExportCmd(t *testing.T) { mvccLatestKVs []storage.MVCCKeyValue mvccAllFiles []string mvccAllKVs []storage.MVCCKeyValue - mvccLatestResponseHeader roachpb.ResponseHeader - mvccAllResponseHeader roachpb.ResponseHeader + mvccLatestResponseHeader kvpb.ResponseHeader + mvccAllResponseHeader kvpb.ResponseHeader } exportAndSlurp := func(t *testing.T, start hlc.Timestamp, maxResponseSSTBytes int64) ExportAndSlurpResult { var ret ExportAndSlurpResult ret.end = hlc.NewClockForTesting(nil).Now() ret.mvccLatestFiles, ret.mvccLatestKVs, ret.mvccLatestResponseHeader = exportAndSlurpOne(t, - start, roachpb.MVCCFilter_Latest, maxResponseSSTBytes) + start, kvpb.MVCCFilter_Latest, maxResponseSSTBytes) ret.mvccAllFiles, ret.mvccAllKVs, ret.mvccAllResponseHeader = exportAndSlurpOne(t, start, - roachpb.MVCCFilter_All, maxResponseSSTBytes) + kvpb.MVCCFilter_All, maxResponseSSTBytes) return ret } @@ -139,8 +140,8 @@ func TestExportCmd(t *testing.T) { } expectResponseHeader := func( - t *testing.T, res ExportAndSlurpResult, mvccLatestResponseHeader roachpb.ResponseHeader, - mvccAllResponseHeader roachpb.ResponseHeader) { + t *testing.T, res ExportAndSlurpResult, mvccLatestResponseHeader kvpb.ResponseHeader, + mvccAllResponseHeader kvpb.ResponseHeader) { t.Helper() requireResumeSpan := func(expect, actual *roachpb.Span, msgAndArgs ...interface{}) { t.Helper() @@ -284,13 +285,13 @@ INTO defer resetMaxOverage(t) setMaxOverage(t, "'1b'") const expectedError = `export size \(11 bytes\) exceeds max size \(2 bytes\)` - _, pErr := export(t, res5.end, roachpb.MVCCFilter_Latest, noTargetBytes) + _, pErr := export(t, res5.end, kvpb.MVCCFilter_Latest, noTargetBytes) require.Regexp(t, expectedError, pErr) hints := errors.GetAllHints(pErr.GoError()) require.Equal(t, 1, len(hints)) const expectedHint = `consider increasing cluster setting "kv.bulk_sst.max_allowed_overage"` require.Regexp(t, expectedHint, hints[0]) - _, pErr = export(t, res5.end, roachpb.MVCCFilter_All, noTargetBytes) + _, pErr = export(t, res5.end, kvpb.MVCCFilter_All, noTargetBytes) require.Regexp(t, expectedError, pErr) // Disable the TargetSize and ensure that we don't get any errors @@ -317,10 +318,10 @@ INTO maxResponseSSTBytes = kvByteSize + 1 res7 = exportAndSlurp(t, res5.end, maxResponseSSTBytes) expect(t, res7, 2, 100, 2, 100) - latestRespHeader := roachpb.ResponseHeader{ + latestRespHeader := kvpb.ResponseHeader{ NumBytes: maxResponseSSTBytes, } - allRespHeader := roachpb.ResponseHeader{ + allRespHeader := kvpb.ResponseHeader{ NumBytes: maxResponseSSTBytes, } expectResponseHeader(t, res7, latestRespHeader, allRespHeader) @@ -332,20 +333,20 @@ INTO maxResponseSSTBytes = kvByteSize res7 = exportAndSlurp(t, res5.end, maxResponseSSTBytes) expect(t, res7, 1, 1, 1, 1) - latestRespHeader = roachpb.ResponseHeader{ + latestRespHeader = kvpb.ResponseHeader{ ResumeSpan: &roachpb.Span{ Key: []byte(fmt.Sprintf("/Table/%d/1/2", tableID)), EndKey: []byte("/Max"), }, - ResumeReason: roachpb.RESUME_BYTE_LIMIT, + ResumeReason: kvpb.RESUME_BYTE_LIMIT, NumBytes: maxResponseSSTBytes, } - allRespHeader = roachpb.ResponseHeader{ + allRespHeader = kvpb.ResponseHeader{ ResumeSpan: &roachpb.Span{ Key: []byte(fmt.Sprintf("/Table/%d/1/2", tableID)), EndKey: []byte("/Max"), }, - ResumeReason: roachpb.RESUME_BYTE_LIMIT, + ResumeReason: kvpb.RESUME_BYTE_LIMIT, NumBytes: maxResponseSSTBytes, } expectResponseHeader(t, res7, latestRespHeader, allRespHeader) @@ -356,20 +357,20 @@ INTO maxResponseSSTBytes = 2 * kvByteSize res7 = exportAndSlurp(t, res5.end, maxResponseSSTBytes) expect(t, res7, 2, 2, 2, 2) - latestRespHeader = roachpb.ResponseHeader{ + latestRespHeader = kvpb.ResponseHeader{ ResumeSpan: &roachpb.Span{ Key: []byte(fmt.Sprintf("/Table/%d/1/3/0", tableID)), EndKey: []byte("/Max"), }, - ResumeReason: roachpb.RESUME_BYTE_LIMIT, + ResumeReason: kvpb.RESUME_BYTE_LIMIT, NumBytes: maxResponseSSTBytes, } - allRespHeader = roachpb.ResponseHeader{ + allRespHeader = kvpb.ResponseHeader{ ResumeSpan: &roachpb.Span{ Key: []byte(fmt.Sprintf("/Table/%d/1/3/0", tableID)), EndKey: []byte("/Max"), }, - ResumeReason: roachpb.RESUME_BYTE_LIMIT, + ResumeReason: kvpb.RESUME_BYTE_LIMIT, NumBytes: maxResponseSSTBytes, } expectResponseHeader(t, res7, latestRespHeader, allRespHeader) @@ -379,20 +380,20 @@ INTO maxResponseSSTBytes = 99 * kvByteSize res7 = exportAndSlurp(t, res5.end, maxResponseSSTBytes) expect(t, res7, 99, 99, 99, 99) - latestRespHeader = roachpb.ResponseHeader{ + latestRespHeader = kvpb.ResponseHeader{ ResumeSpan: &roachpb.Span{ Key: []byte(fmt.Sprintf("/Table/%d/1/100/0", tableID)), EndKey: []byte("/Max"), }, - ResumeReason: roachpb.RESUME_BYTE_LIMIT, + ResumeReason: kvpb.RESUME_BYTE_LIMIT, NumBytes: maxResponseSSTBytes, } - allRespHeader = roachpb.ResponseHeader{ + allRespHeader = kvpb.ResponseHeader{ ResumeSpan: &roachpb.Span{ Key: []byte(fmt.Sprintf("/Table/%d/1/100/0", tableID)), EndKey: []byte("/Max"), }, - ResumeReason: roachpb.RESUME_BYTE_LIMIT, + ResumeReason: kvpb.RESUME_BYTE_LIMIT, NumBytes: maxResponseSSTBytes, } expectResponseHeader(t, res7, latestRespHeader, allRespHeader) @@ -404,10 +405,10 @@ INTO maxResponseSSTBytes = 101 * kvByteSize res7 = exportAndSlurp(t, res5.end, maxResponseSSTBytes) expect(t, res7, 100, 100, 100, 100) - latestRespHeader = roachpb.ResponseHeader{ + latestRespHeader = kvpb.ResponseHeader{ NumBytes: 100 * kvByteSize, } - allRespHeader = roachpb.ResponseHeader{ + allRespHeader = kvpb.ResponseHeader{ NumBytes: 100 * kvByteSize, } expectResponseHeader(t, res7, latestRespHeader, allRespHeader) @@ -422,8 +423,8 @@ func TestExportGCThreshold(t *testing.T) { defer tc.Stopper().Stop(ctx) kvDB := tc.Server(0).DB() - req := &roachpb.ExportRequest{ - RequestHeader: roachpb.RequestHeader{Key: bootstrap.TestingUserTableDataMin(), EndKey: keys.MaxKey}, + req := &kvpb.ExportRequest{ + RequestHeader: kvpb.RequestHeader{Key: bootstrap.TestingUserTableDataMin(), EndKey: keys.MaxKey}, StartTime: hlc.Timestamp{WallTime: -1}, } _, pErr := kv.SendWrapped(ctx, kvDB.NonTransactionalSender(), req) @@ -436,7 +437,7 @@ func TestExportGCThreshold(t *testing.T) { // as an oracle to check the correctness of pebbleExportToSst. func exportUsingGoIterator( ctx context.Context, - filter roachpb.MVCCFilter, + filter kvpb.MVCCFilter, startTime, endTime hlc.Timestamp, startKey, endKey roachpb.Key, reader storage.Reader, @@ -450,10 +451,10 @@ func exportUsingGoIterator( var skipTombstones bool var iterFn func(*storage.MVCCIncrementalIterator) switch filter { - case roachpb.MVCCFilter_Latest: + case kvpb.MVCCFilter_Latest: skipTombstones = true iterFn = (*storage.MVCCIncrementalIterator).NextKey - case roachpb.MVCCFilter_All: + case kvpb.MVCCFilter_All: skipTombstones = false iterFn = (*storage.MVCCIncrementalIterator).Next default: @@ -571,11 +572,11 @@ func assertEqualKVs( return func(t *testing.T) { t.Helper() - var filter roachpb.MVCCFilter + var filter kvpb.MVCCFilter if exportAllRevisions { - filter = roachpb.MVCCFilter_All + filter = kvpb.MVCCFilter_All } else { - filter = roachpb.MVCCFilter_Latest + filter = kvpb.MVCCFilter_Latest } // Run the oracle which is a legacy implementation of pebbleExportToSst @@ -590,7 +591,7 @@ func assertEqualKVs( start := storage.MVCCKey{Key: startKey} for start.Key != nil { var sst []byte - var summary roachpb.BulkOpSummary + var summary kvpb.BulkOpSummary maxSize := uint64(0) prevStart := start sstFile := &storage.MemFile{} diff --git a/pkg/kv/kvserver/batcheval/cmd_gc.go b/pkg/kv/kvserver/batcheval/cmd_gc.go index ed366bb1058f..2635ca53baaa 100644 --- a/pkg/kv/kvserver/batcheval/cmd_gc.go +++ b/pkg/kv/kvserver/batcheval/cmd_gc.go @@ -16,6 +16,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" @@ -26,17 +27,17 @@ import ( ) func init() { - RegisterReadWriteCommand(roachpb.GC, declareKeysGC, GC) + RegisterReadWriteCommand(kvpb.GC, declareKeysGC, GC) } func declareKeysGC( rs ImmutableRangeState, - header *roachpb.Header, - req roachpb.Request, + header *kvpb.Header, + req kvpb.Request, latchSpans, _ *spanset.SpanSet, _ time.Duration, ) { - gcr := req.(*roachpb.GCRequest) + gcr := req.(*kvpb.GCRequest) if gcr.RangeKeys != nil { // When GC-ing MVCC range key tombstones, we need to serialize with // range key writes that overlap the MVCC range tombstone, as well as @@ -135,9 +136,9 @@ func mergeAdjacentSpans(spans []roachpb.Span) []roachpb.Span { // listed key along with the expiration timestamp. The GC metadata // specified in the args is persisted after GC. func GC( - ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.GCRequest) + args := cArgs.Args.(*kvpb.GCRequest) h := cArgs.Header // We do not allow GC requests to bump the GC threshold at the same time that @@ -181,10 +182,10 @@ func GC( // safe because they can simply be re-collected later on the correct // replica. Discrepancies here can arise from race conditions during // range splitting. - globalKeys := make([]roachpb.GCRequest_GCKey, 0, len(args.Keys)) + globalKeys := make([]kvpb.GCRequest_GCKey, 0, len(args.Keys)) // Local keys are rarer, so don't pre-allocate slice. We separate the two // kinds of keys since it is a requirement when calling MVCCGarbageCollect. - var localKeys []roachpb.GCRequest_GCKey + var localKeys []kvpb.GCRequest_GCKey for _, k := range args.Keys { if cArgs.EvalCtx.ContainsKey(k.Key) { if keys.IsLocal(k.Key) { @@ -196,7 +197,7 @@ func GC( } // Garbage collect the specified keys by expiration timestamps. - for _, gcKeys := range [][]roachpb.GCRequest_GCKey{localKeys, globalKeys} { + for _, gcKeys := range [][]kvpb.GCRequest_GCKey{localKeys, globalKeys} { if err := storage.MVCCGarbageCollect( ctx, readWriter, cArgs.Stats, gcKeys, h.Timestamp, ); err != nil { @@ -296,7 +297,7 @@ func GC( // ensure merging of range tombstones could be performed and at the same time // no data is accessed outside of latches. func makeLookupBoundariesForGCRanges( - rangeStart, rangeEnd roachpb.Key, rangeKeys []roachpb.GCRequest_GCRangeKey, + rangeStart, rangeEnd roachpb.Key, rangeKeys []kvpb.GCRequest_GCRangeKey, ) []roachpb.Span { spans := make([]roachpb.Span, len(rangeKeys)) for i := range rangeKeys { @@ -313,7 +314,7 @@ func makeLookupBoundariesForGCRanges( // containing ranges to be removed as well as safe iteration boundaries. // See makeLookupBoundariesForGCRanges for why additional boundaries are used. func makeCollectableGCRangesFromGCRequests( - rangeStart, rangeEnd roachpb.Key, rangeKeys []roachpb.GCRequest_GCRangeKey, + rangeStart, rangeEnd roachpb.Key, rangeKeys []kvpb.GCRequest_GCRangeKey, ) []storage.CollectableGCRangeKey { latches := makeLookupBoundariesForGCRanges(rangeStart, rangeEnd, rangeKeys) collectableKeys := make([]storage.CollectableGCRangeKey, len(rangeKeys)) diff --git a/pkg/kv/kvserver/batcheval/cmd_get.go b/pkg/kv/kvserver/batcheval/cmd_get.go index 04c76685eb8e..56ea88f9a3de 100644 --- a/pkg/kv/kvserver/batcheval/cmd_get.go +++ b/pkg/kv/kvserver/batcheval/cmd_get.go @@ -13,6 +13,7 @@ package batcheval import ( "context" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -21,19 +22,19 @@ import ( ) func init() { - RegisterReadOnlyCommand(roachpb.Get, DefaultDeclareIsolatedKeys, Get) + RegisterReadOnlyCommand(kvpb.Get, DefaultDeclareIsolatedKeys, Get) } // Get returns the value for a specified key. func Get( - ctx context.Context, reader storage.Reader, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, reader storage.Reader, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.GetRequest) + args := cArgs.Args.(*kvpb.GetRequest) h := cArgs.Header - reply := resp.(*roachpb.GetResponse) + reply := resp.(*kvpb.GetResponse) getRes, err := storage.MVCCGet(ctx, reader, args.Key, h.Timestamp, storage.MVCCGetOptions{ - Inconsistent: h.ReadConsistency != roachpb.CONSISTENT, + Inconsistent: h.ReadConsistency != kvpb.CONSISTENT, SkipLocked: h.WaitPolicy == lock.WaitPolicy_SkipLocked, Txn: h.Txn, FailOnMoreRecent: args.KeyLocking != lock.None, @@ -62,7 +63,7 @@ func Get( } reply.Value = getRes.Value - if h.ReadConsistency == roachpb.READ_UNCOMMITTED { + if h.ReadConsistency == kvpb.READ_UNCOMMITTED { var intentVals []roachpb.KeyValue // NOTE: MVCCGet uses a Prefix iterator, so we want to use one in // CollectIntentRows as well so that we're guaranteed to use the same diff --git a/pkg/kv/kvserver/batcheval/cmd_get_test.go b/pkg/kv/kvserver/batcheval/cmd_get_test.go index 07d6f8ac6064..34bc0323f8ff 100644 --- a/pkg/kv/kvserver/batcheval/cmd_get_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_get_test.go @@ -15,6 +15,7 @@ import ( "fmt" "testing" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage" @@ -27,7 +28,7 @@ import ( func TestGetResumeSpan(t *testing.T) { defer leaktest.AfterTest(t)() ctx := context.Background() - resp := roachpb.GetResponse{} + resp := kvpb.GetResponse{} key := roachpb.Key([]byte{'a'}) value := roachpb.MakeValueFromString("woohoo") @@ -37,9 +38,9 @@ func TestGetResumeSpan(t *testing.T) { // This has a size of 11 bytes. _, err := Put(ctx, db, CommandArgs{ EvalCtx: (&MockEvalCtx{}).EvalContext(), - Header: roachpb.Header{TargetBytes: -1}, - Args: &roachpb.PutRequest{ - RequestHeader: roachpb.RequestHeader{ + Header: kvpb.Header{TargetBytes: -1}, + Args: &kvpb.PutRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, }, Value: value, @@ -52,24 +53,24 @@ func TestGetResumeSpan(t *testing.T) { targetBytes int64 allowEmpty bool expectResume bool - expectReason roachpb.ResumeReason + expectReason kvpb.ResumeReason expectNextBytes int64 }{ - {maxKeys: -1, expectResume: true, expectReason: roachpb.RESUME_KEY_LIMIT, expectNextBytes: 0}, + {maxKeys: -1, expectResume: true, expectReason: kvpb.RESUME_KEY_LIMIT, expectNextBytes: 0}, {maxKeys: 0, expectResume: false}, {maxKeys: 1, expectResume: false}, {maxKeys: 1, allowEmpty: true, expectResume: false}, - {targetBytes: -1, expectResume: true, expectReason: roachpb.RESUME_BYTE_LIMIT, expectNextBytes: 0}, + {targetBytes: -1, expectResume: true, expectReason: kvpb.RESUME_BYTE_LIMIT, expectNextBytes: 0}, {targetBytes: 0, expectResume: false}, {targetBytes: 1, expectResume: false}, {targetBytes: 11, expectResume: false}, {targetBytes: 12, expectResume: false}, - {targetBytes: 1, allowEmpty: true, expectResume: true, expectReason: roachpb.RESUME_BYTE_LIMIT, expectNextBytes: 11}, + {targetBytes: 1, allowEmpty: true, expectResume: true, expectReason: kvpb.RESUME_BYTE_LIMIT, expectNextBytes: 11}, {targetBytes: 11, allowEmpty: true, expectResume: false}, {targetBytes: 12, allowEmpty: true, expectResume: false}, - {maxKeys: -1, targetBytes: -1, expectResume: true, expectReason: roachpb.RESUME_KEY_LIMIT, expectNextBytes: 0}, + {maxKeys: -1, targetBytes: -1, expectResume: true, expectReason: kvpb.RESUME_KEY_LIMIT, expectNextBytes: 0}, {maxKeys: 10, targetBytes: 100, expectResume: false}, } for _, tc := range testCases { @@ -78,16 +79,16 @@ func TestGetResumeSpan(t *testing.T) { t.Run(name, func(t *testing.T) { settings := cluster.MakeTestingClusterSettings() - resp := roachpb.GetResponse{} + resp := kvpb.GetResponse{} _, err := Get(ctx, db, CommandArgs{ EvalCtx: (&MockEvalCtx{ClusterSettings: settings}).EvalContext(), - Header: roachpb.Header{ + Header: kvpb.Header{ MaxSpanRequestKeys: tc.maxKeys, TargetBytes: tc.targetBytes, AllowEmpty: tc.allowEmpty, }, - Args: &roachpb.GetRequest{ - RequestHeader: roachpb.RequestHeader{Key: key}, + Args: &kvpb.GetRequest{ + RequestHeader: kvpb.RequestHeader{Key: key}, }, }, &resp) require.NoError(t, err) diff --git a/pkg/kv/kvserver/batcheval/cmd_heartbeat_txn.go b/pkg/kv/kvserver/batcheval/cmd_heartbeat_txn.go index 434ea1233dec..c0f9dc8d9680 100644 --- a/pkg/kv/kvserver/batcheval/cmd_heartbeat_txn.go +++ b/pkg/kv/kvserver/batcheval/cmd_heartbeat_txn.go @@ -16,6 +16,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -24,13 +25,13 @@ import ( ) func init() { - RegisterReadWriteCommand(roachpb.HeartbeatTxn, declareKeysHeartbeatTransaction, HeartbeatTxn) + RegisterReadWriteCommand(kvpb.HeartbeatTxn, declareKeysHeartbeatTransaction, HeartbeatTxn) } func declareKeysHeartbeatTransaction( rs ImmutableRangeState, - header *roachpb.Header, - req roachpb.Request, + header *kvpb.Header, + req kvpb.Request, latchSpans, _ *spanset.SpanSet, _ time.Duration, ) { @@ -41,11 +42,11 @@ func declareKeysHeartbeatTransaction( // timestamp after receiving transaction heartbeat messages from // coordinator. Returns the updated transaction. func HeartbeatTxn( - ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.HeartbeatTxnRequest) + args := cArgs.Args.(*kvpb.HeartbeatTxnRequest) h := cArgs.Header - reply := resp.(*roachpb.HeartbeatTxnResponse) + reply := resp.(*kvpb.HeartbeatTxnResponse) if err := VerifyTransaction(h, args, roachpb.PENDING, roachpb.STAGING); err != nil { return result.Result{}, err diff --git a/pkg/kv/kvserver/batcheval/cmd_increment.go b/pkg/kv/kvserver/batcheval/cmd_increment.go index aac9e412ef09..210552c80362 100644 --- a/pkg/kv/kvserver/batcheval/cmd_increment.go +++ b/pkg/kv/kvserver/batcheval/cmd_increment.go @@ -13,24 +13,24 @@ package batcheval import ( "context" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" ) func init() { - RegisterReadWriteCommand(roachpb.Increment, DefaultDeclareIsolatedKeys, Increment) + RegisterReadWriteCommand(kvpb.Increment, DefaultDeclareIsolatedKeys, Increment) } // Increment increments the value (interpreted as varint64 encoded) and // returns the newly incremented value (encoded as varint64). If no value // exists for the key, zero is incremented. func Increment( - ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.IncrementRequest) + args := cArgs.Args.(*kvpb.IncrementRequest) h := cArgs.Header - reply := resp.(*roachpb.IncrementResponse) + reply := resp.(*kvpb.IncrementResponse) newVal, err := storage.MVCCIncrement( ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, cArgs.Now, h.Txn, args.Increment) diff --git a/pkg/kv/kvserver/batcheval/cmd_init_put.go b/pkg/kv/kvserver/batcheval/cmd_init_put.go index 30ebb1e2451a..64c0e7f9cbfd 100644 --- a/pkg/kv/kvserver/batcheval/cmd_init_put.go +++ b/pkg/kv/kvserver/batcheval/cmd_init_put.go @@ -13,13 +13,13 @@ package batcheval import ( "context" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" ) func init() { - RegisterReadWriteCommand(roachpb.InitPut, DefaultDeclareIsolatedKeys, InitPut) + RegisterReadWriteCommand(kvpb.InitPut, DefaultDeclareIsolatedKeys, InitPut) } // InitPut sets the value for a specified key only if it doesn't exist. It @@ -27,9 +27,9 @@ func init() { // is different from the value provided. If FailOnTombstone is set to true, // tombstones count as mismatched values and will cause a ConditionFailedError. func InitPut( - ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.InitPutRequest) + args := cArgs.Args.(*kvpb.InitPutRequest) h := cArgs.Header if args.FailOnTombstones && cArgs.EvalCtx.EvalKnobs().DisableInitPutFailOnTombstones { diff --git a/pkg/kv/kvserver/batcheval/cmd_is_span_empty.go b/pkg/kv/kvserver/batcheval/cmd_is_span_empty.go index 85a73a560bde..5a3a85173526 100644 --- a/pkg/kv/kvserver/batcheval/cmd_is_span_empty.go +++ b/pkg/kv/kvserver/batcheval/cmd_is_span_empty.go @@ -13,24 +13,24 @@ package batcheval import ( "context" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/errors" ) func init() { - RegisterReadOnlyCommand(roachpb.IsSpanEmpty, DefaultDeclareKeys, IsSpanEmpty) + RegisterReadOnlyCommand(kvpb.IsSpanEmpty, DefaultDeclareKeys, IsSpanEmpty) } // IsSpanEmpty determines whether there are any keys in the key span requested // at any time. If there are any keys, the response header will have a NumKeys // value of 1. func IsSpanEmpty( - ctx context.Context, reader storage.Reader, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, reader storage.Reader, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.IsSpanEmptyRequest) - reply := resp.(*roachpb.IsSpanEmptyResponse) + args := cArgs.Args.(*kvpb.IsSpanEmptyRequest) + reply := resp.(*kvpb.IsSpanEmptyResponse) isEmpty, err := storage.MVCCIsSpanEmpty(ctx, reader, storage.MVCCIsSpanEmptyOptions{ StartKey: args.Key, EndKey: args.EndKey, diff --git a/pkg/kv/kvserver/batcheval/cmd_is_span_empty_test.go b/pkg/kv/kvserver/batcheval/cmd_is_span_empty_test.go index e06c84764a8b..003bb2ac3e7a 100644 --- a/pkg/kv/kvserver/batcheval/cmd_is_span_empty_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_is_span_empty_test.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" @@ -33,8 +34,8 @@ func TestIsSpanEmpty(t *testing.T) { ServerArgs: base.TestServerArgs{ Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(ctx context.Context, request *roachpb.BatchRequest) *roachpb.Error { - if _, exists := request.GetArg(roachpb.IsSpanEmpty); exists { + TestingRequestFilter: func(ctx context.Context, request *kvpb.BatchRequest) *kvpb.Error { + if _, exists := request.GetArg(kvpb.IsSpanEmpty); exists { atomic.AddInt64(&sentIsSpanEmptyRequests, 1) } return nil @@ -54,8 +55,8 @@ func TestIsSpanEmpty(t *testing.T) { checkIsEmpty := func(t *testing.T, exp bool, from, to roachpb.Key) { var ba kv.Batch ba.Header.MaxSpanRequestKeys = 1 - ba.AddRawRequest(&roachpb.IsSpanEmptyRequest{ - RequestHeader: roachpb.RequestHeader{ + ba.AddRawRequest(&kvpb.IsSpanEmptyRequest{ + RequestHeader: kvpb.RequestHeader{ Key: from, EndKey: to, }, }) diff --git a/pkg/kv/kvserver/batcheval/cmd_lease.go b/pkg/kv/kvserver/batcheval/cmd_lease.go index c33bb178d311..5b50513dd32e 100644 --- a/pkg/kv/kvserver/batcheval/cmd_lease.go +++ b/pkg/kv/kvserver/batcheval/cmd_lease.go @@ -14,6 +14,7 @@ import ( "context" "fmt" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary" @@ -66,7 +67,7 @@ func evalNewLease( (lease.Type() == roachpb.LeaseEpoch && lease.Expiration != nil) { // This amounts to a bug. return newFailedLeaseTrigger(isTransfer), - &roachpb.LeaseRejectedError{ + &kvpb.LeaseRejectedError{ Existing: prevLease, Requested: lease, Message: fmt.Sprintf("illegal lease: epoch=%d, interval=[%s, %s)", @@ -78,7 +79,7 @@ func evalNewLease( desc := rec.Desc() if _, ok := desc.GetReplicaDescriptor(lease.Replica.StoreID); !ok { return newFailedLeaseTrigger(isTransfer), - &roachpb.LeaseRejectedError{ + &kvpb.LeaseRejectedError{ Existing: prevLease, Requested: lease, Message: "replica not found", @@ -90,7 +91,7 @@ func evalNewLease( // succeeding. if lease.Sequence != 0 { return newFailedLeaseTrigger(isTransfer), - &roachpb.LeaseRejectedError{ + &kvpb.LeaseRejectedError{ Existing: prevLease, Requested: lease, Message: "sequence number should not be set", diff --git a/pkg/kv/kvserver/batcheval/cmd_lease_info.go b/pkg/kv/kvserver/batcheval/cmd_lease_info.go index 08fe24495149..5829dad900e1 100644 --- a/pkg/kv/kvserver/batcheval/cmd_lease_info.go +++ b/pkg/kv/kvserver/batcheval/cmd_lease_info.go @@ -15,6 +15,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -22,13 +23,13 @@ import ( ) func init() { - RegisterReadOnlyCommand(roachpb.LeaseInfo, declareKeysLeaseInfo, LeaseInfo) + RegisterReadOnlyCommand(kvpb.LeaseInfo, declareKeysLeaseInfo, LeaseInfo) } func declareKeysLeaseInfo( rs ImmutableRangeState, - _ *roachpb.Header, - _ roachpb.Request, + _ *kvpb.Header, + _ kvpb.Request, latchSpans, _ *spanset.SpanSet, _ time.Duration, ) { @@ -37,9 +38,9 @@ func declareKeysLeaseInfo( // LeaseInfo returns information about the lease holder for the range. func LeaseInfo( - ctx context.Context, reader storage.Reader, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, reader storage.Reader, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - reply := resp.(*roachpb.LeaseInfoResponse) + reply := resp.(*kvpb.LeaseInfoResponse) lease, nextLease := cArgs.EvalCtx.GetLease() if nextLease != (roachpb.Lease{}) { // If there's a lease request in progress, speculatively return that future diff --git a/pkg/kv/kvserver/batcheval/cmd_lease_request.go b/pkg/kv/kvserver/batcheval/cmd_lease_request.go index 64a10019baba..f800ce98e2bb 100644 --- a/pkg/kv/kvserver/batcheval/cmd_lease_request.go +++ b/pkg/kv/kvserver/batcheval/cmd_lease_request.go @@ -15,6 +15,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" @@ -23,13 +24,13 @@ import ( ) func init() { - RegisterReadWriteCommand(roachpb.RequestLease, declareKeysRequestLease, RequestLease) + RegisterReadWriteCommand(kvpb.RequestLease, declareKeysRequestLease, RequestLease) } func declareKeysRequestLease( rs ImmutableRangeState, - _ *roachpb.Header, - _ roachpb.Request, + _ *kvpb.Header, + _ kvpb.Request, latchSpans, _ *spanset.SpanSet, _ time.Duration, ) { @@ -51,18 +52,18 @@ func declareKeysRequestLease( // lease, all duties required of the range lease holder are commenced, including // releasing all latches and clearing the timestamp cache. func RequestLease( - ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { // When returning an error from this method, must always return a // newFailedLeaseTrigger() to satisfy stats. - args := cArgs.Args.(*roachpb.RequestLeaseRequest) + args := cArgs.Args.(*kvpb.RequestLeaseRequest) // NOTE: we use the range's current lease as prevLease instead of // args.PrevLease so that we can detect lease requests that will // inevitably fail early and reject them with a detailed // LeaseRejectedError before going through Raft. prevLease, _ := cArgs.EvalCtx.GetLease() - rErr := &roachpb.LeaseRejectedError{ + rErr := &kvpb.LeaseRejectedError{ Existing: prevLease, Requested: args.Lease, } diff --git a/pkg/kv/kvserver/batcheval/cmd_lease_test.go b/pkg/kv/kvserver/batcheval/cmd_lease_test.go index 88ed034ed07d..7ea79362a660 100644 --- a/pkg/kv/kvserver/batcheval/cmd_lease_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_lease_test.go @@ -16,6 +16,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -111,7 +112,7 @@ func TestLeaseTransferWithPipelinedWrite(t *testing.T) { // We allow the transaction to run into an aborted error due to a lease // transfer when it attempts to create its transaction record. This it // outside of the focus of this test. - okErr := testutils.IsError(err, roachpb.ABORT_REASON_NEW_LEASE_PREVENTS_TXN.String()) + okErr := testutils.IsError(err, kvpb.ABORT_REASON_NEW_LEASE_PREVENTS_TXN.String()) if !okErr { t.Fatalf("worker failed: %+v", err) } @@ -140,7 +141,7 @@ func TestLeaseCommandLearnerReplica(t *testing.T) { Desc: &desc, Clock: clock, }).EvalContext(), - Args: &roachpb.TransferLeaseRequest{ + Args: &kvpb.TransferLeaseRequest{ Lease: roachpb.Lease{ Replica: replicas[1], }, @@ -152,14 +153,14 @@ func TestLeaseCommandLearnerReplica(t *testing.T) { _, err := TransferLease(ctx, nil, cArgs, nil) require.EqualError(t, err, `replica cannot hold lease`) - cArgs.Args = &roachpb.RequestLeaseRequest{} + cArgs.Args = &kvpb.RequestLeaseRequest{} _, err = RequestLease(ctx, nil, cArgs, nil) const expForUnknown = `cannot replace lease with : ` + `replica not found in RangeDescriptor` require.EqualError(t, err, expForUnknown) - cArgs.Args = &roachpb.RequestLeaseRequest{ + cArgs.Args = &kvpb.RequestLeaseRequest{ Lease: roachpb.Lease{ Replica: replicas[1], }, @@ -229,7 +230,7 @@ func TestLeaseTransferForwardsStartTime(t *testing.T) { } cArgs := CommandArgs{ EvalCtx: evalCtx.EvalContext(), - Args: &roachpb.TransferLeaseRequest{ + Args: &kvpb.TransferLeaseRequest{ Lease: nextLease, PrevLease: prevLease, }, diff --git a/pkg/kv/kvserver/batcheval/cmd_lease_transfer.go b/pkg/kv/kvserver/batcheval/cmd_lease_transfer.go index 25283563225d..8f4cccff2621 100644 --- a/pkg/kv/kvserver/batcheval/cmd_lease_transfer.go +++ b/pkg/kv/kvserver/batcheval/cmd_lease_transfer.go @@ -14,6 +14,7 @@ import ( "context" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" @@ -23,13 +24,13 @@ import ( ) func init() { - RegisterReadWriteCommand(roachpb.TransferLease, declareKeysTransferLease, TransferLease) + RegisterReadWriteCommand(kvpb.TransferLease, declareKeysTransferLease, TransferLease) } func declareKeysTransferLease( _ ImmutableRangeState, - _ *roachpb.Header, - _ roachpb.Request, + _ *kvpb.Header, + _ kvpb.Request, latchSpans, _ *spanset.SpanSet, _ time.Duration, ) { @@ -61,11 +62,11 @@ func declareKeysTransferLease( // ex-) lease holder which must have dropped all of its lease holder powers // before proposing. func TransferLease( - ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { // When returning an error from this method, must always return // a newFailedLeaseTrigger() to satisfy stats. - args := cArgs.Args.(*roachpb.TransferLeaseRequest) + args := cArgs.Args.(*kvpb.TransferLeaseRequest) // NOTE: we use the range's current lease as prevLease instead of // args.PrevLease so that we can detect lease transfers that will diff --git a/pkg/kv/kvserver/batcheval/cmd_merge.go b/pkg/kv/kvserver/batcheval/cmd_merge.go index b13a2c18a84b..5921ba3f80cb 100644 --- a/pkg/kv/kvserver/batcheval/cmd_merge.go +++ b/pkg/kv/kvserver/batcheval/cmd_merge.go @@ -13,13 +13,13 @@ package batcheval import ( "context" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" ) func init() { - RegisterReadWriteCommand(roachpb.Merge, DefaultDeclareKeys, Merge) + RegisterReadWriteCommand(kvpb.Merge, DefaultDeclareKeys, Merge) } // Merge is used to merge a value into an existing key. Merge is an @@ -29,9 +29,9 @@ func init() { // transactional, merges are not currently exposed directly to // clients. Merged values are explicitly not MVCC data. func Merge( - ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.MergeRequest) + args := cArgs.Args.(*kvpb.MergeRequest) h := cArgs.Header return result.Result{}, storage.MVCCMerge(ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, args.Value) diff --git a/pkg/kv/kvserver/batcheval/cmd_migrate.go b/pkg/kv/kvserver/batcheval/cmd_migrate.go index 1157c08e1ba8..979ef49be728 100644 --- a/pkg/kv/kvserver/batcheval/cmd_migrate.go +++ b/pkg/kv/kvserver/batcheval/cmd_migrate.go @@ -15,6 +15,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" @@ -24,13 +25,13 @@ import ( ) func init() { - RegisterReadWriteCommand(roachpb.Migrate, declareKeysMigrate, Migrate) + RegisterReadWriteCommand(kvpb.Migrate, declareKeysMigrate, Migrate) } func declareKeysMigrate( rs ImmutableRangeState, - _ *roachpb.Header, - _ roachpb.Request, + _ *kvpb.Header, + _ kvpb.Request, latchSpans, _ *spanset.SpanSet, _ time.Duration, ) { @@ -52,11 +53,11 @@ var migrationRegistry = make(map[roachpb.Version]migration) type migration func(context.Context, storage.ReadWriter, CommandArgs) (result.Result, error) // Migrate executes the below-raft migration corresponding to the given version. -// See roachpb.MigrateRequest for more details. +// See kvpb.MigrateRequest for more details. func Migrate( - ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, _ roachpb.Response, + ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, _ kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.MigrateRequest) + args := cArgs.Args.(*kvpb.MigrateRequest) migrationVersion := args.Version fn, ok := migrationRegistry[migrationVersion] diff --git a/pkg/kv/kvserver/batcheval/cmd_probe.go b/pkg/kv/kvserver/batcheval/cmd_probe.go index 5ac264d3fb03..739b59a05269 100644 --- a/pkg/kv/kvserver/batcheval/cmd_probe.go +++ b/pkg/kv/kvserver/batcheval/cmd_probe.go @@ -14,19 +14,15 @@ import ( "context" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" ) func declareKeysProbe( - _ ImmutableRangeState, - _ *roachpb.Header, - _ roachpb.Request, - _, _ *spanset.SpanSet, - _ time.Duration, + _ ImmutableRangeState, _ *kvpb.Header, _ kvpb.Request, _, _ *spanset.SpanSet, _ time.Duration, ) { // Declare no keys. This means that we're not even serializing with splits // (i.e. a probe could be directed at a key that will become the right-hand @@ -37,14 +33,14 @@ func declareKeysProbe( } func init() { - RegisterReadWriteCommand(roachpb.Probe, declareKeysProbe, Probe) + RegisterReadWriteCommand(kvpb.Probe, declareKeysProbe, Probe) } // Probe causes an effectless round-trip through the replication layer, // i.e. it is a write that does not change any kv pair. It declares a // write on the targeted key (but no lock). func Probe( - ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { return result.Result{ Replicated: kvserverpb.ReplicatedEvalResult{ diff --git a/pkg/kv/kvserver/batcheval/cmd_push_txn.go b/pkg/kv/kvserver/batcheval/cmd_push_txn.go index 3fec1fc7bca4..56d76aaf05d3 100644 --- a/pkg/kv/kvserver/batcheval/cmd_push_txn.go +++ b/pkg/kv/kvserver/batcheval/cmd_push_txn.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/txnwait" @@ -29,17 +30,17 @@ import ( ) func init() { - RegisterReadWriteCommand(roachpb.PushTxn, declareKeysPushTransaction, PushTxn) + RegisterReadWriteCommand(kvpb.PushTxn, declareKeysPushTransaction, PushTxn) } func declareKeysPushTransaction( rs ImmutableRangeState, - _ *roachpb.Header, - req roachpb.Request, + _ *kvpb.Header, + req kvpb.Request, latchSpans, _ *spanset.SpanSet, _ time.Duration, ) { - pr := req.(*roachpb.PushTxnRequest) + pr := req.(*kvpb.PushTxnRequest) latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.TransactionKey(pr.PusheeTxn.Key, pr.PusheeTxn.ID)}) latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.AbortSpanKey(rs.GetRangeID(), pr.PusheeTxn.ID)}) } @@ -109,11 +110,11 @@ func declareKeysPushTransaction( // purge records for which the transaction coordinator must have found out // via its heartbeats that the transaction has failed. func PushTxn( - ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.PushTxnRequest) + args := cArgs.Args.(*kvpb.PushTxnRequest) h := cArgs.Header - reply := resp.(*roachpb.PushTxnResponse) + reply := resp.(*kvpb.PushTxnResponse) if h.Txn != nil { return result.Result{}, ErrTransactionUnsupported @@ -193,7 +194,7 @@ func PushTxn( // If we're trying to move the timestamp forward, and it's already // far enough forward, return success. pushType := args.PushType - if pushType == roachpb.PUSH_TIMESTAMP && args.PushTo.LessEq(reply.PusheeTxn.WriteTimestamp) { + if pushType == kvpb.PUSH_TIMESTAMP && args.PushTo.LessEq(reply.PusheeTxn.WriteTimestamp) { // Trivial noop. return result.Result{}, nil } @@ -235,8 +236,8 @@ func PushTxn( // cases, the push acts the same as a short-circuited transaction // recovery process, because the transaction recovery procedure always // finalizes target transactions, even if initiated by a PUSH_TIMESTAMP. - if pushType == roachpb.PUSH_TIMESTAMP { - pushType = roachpb.PUSH_ABORT + if pushType == kvpb.PUSH_TIMESTAMP { + pushType = kvpb.PUSH_ABORT } } @@ -248,9 +249,9 @@ func PushTxn( reason = "pushee is expired" // When cleaning up, actually clean up (as opposed to simply pushing // the garbage in the path of future writers). - pushType = roachpb.PUSH_ABORT + pushType = kvpb.PUSH_ABORT pusherWins = true - case pushType == roachpb.PUSH_TOUCH: + case pushType == kvpb.PUSH_TOUCH: // If just attempting to cleanup old or already-committed txns, // pusher always fails. pusherWins = false @@ -280,13 +281,13 @@ func PushTxn( // attempting to finalize it. recoverOnFailedPush := cArgs.EvalCtx.EvalKnobs().RecoverIndeterminateCommitsOnFailedPushes if reply.PusheeTxn.Status == roachpb.STAGING && (pusherWins || recoverOnFailedPush) { - err := roachpb.NewIndeterminateCommitError(reply.PusheeTxn) + err := kvpb.NewIndeterminateCommitError(reply.PusheeTxn) log.VEventf(ctx, 1, "%v", err) return result.Result{}, err } if !pusherWins { - err := roachpb.NewTransactionPushError(reply.PusheeTxn) + err := kvpb.NewTransactionPushError(reply.PusheeTxn) log.VEventf(ctx, 1, "%v", err) return result.Result{}, err } @@ -296,7 +297,7 @@ func PushTxn( // Determine what to do with the pushee, based on the push type. switch pushType { - case roachpb.PUSH_ABORT: + case kvpb.PUSH_ABORT: // If aborting the transaction, set the new status. reply.PusheeTxn.Status = roachpb.ABORTED // Forward the timestamp to accommodate AbortSpan GC. See method comment for @@ -315,7 +316,7 @@ func PushTxn( return result.Result{}, err } } - case roachpb.PUSH_TIMESTAMP: + case kvpb.PUSH_TIMESTAMP: if existTxn.Status != roachpb.PENDING { return result.Result{}, errors.AssertionFailedf( "PUSH_TIMESTAMP succeeded against non-PENDING txn: %v", existTxn) diff --git a/pkg/kv/kvserver/batcheval/cmd_put.go b/pkg/kv/kvserver/batcheval/cmd_put.go index 283164a2d5ea..df0074119039 100644 --- a/pkg/kv/kvserver/batcheval/cmd_put.go +++ b/pkg/kv/kvserver/batcheval/cmd_put.go @@ -14,25 +14,25 @@ import ( "context" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/hlc" ) func init() { - RegisterReadWriteCommand(roachpb.Put, declareKeysPut, Put) + RegisterReadWriteCommand(kvpb.Put, declareKeysPut, Put) } func declareKeysPut( rs ImmutableRangeState, - header *roachpb.Header, - req roachpb.Request, + header *kvpb.Header, + req kvpb.Request, latchSpans, lockSpans *spanset.SpanSet, maxOffset time.Duration, ) { - args := req.(*roachpb.PutRequest) + args := req.(*kvpb.PutRequest) if args.Inline { DefaultDeclareKeys(rs, header, req, latchSpans, lockSpans, maxOffset) } else { @@ -42,9 +42,9 @@ func declareKeysPut( // Put sets the value for a specified key. func Put( - ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.PutRequest) + args := cArgs.Args.(*kvpb.PutRequest) h := cArgs.Header ms := cArgs.Stats diff --git a/pkg/kv/kvserver/batcheval/cmd_query_intent.go b/pkg/kv/kvserver/batcheval/cmd_query_intent.go index 26debfe3d7e5..f9ff04f82057 100644 --- a/pkg/kv/kvserver/batcheval/cmd_query_intent.go +++ b/pkg/kv/kvserver/batcheval/cmd_query_intent.go @@ -14,22 +14,22 @@ import ( "context" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) func init() { - RegisterReadOnlyCommand(roachpb.QueryIntent, declareKeysQueryIntent, QueryIntent) + RegisterReadOnlyCommand(kvpb.QueryIntent, declareKeysQueryIntent, QueryIntent) } func declareKeysQueryIntent( _ ImmutableRangeState, - _ *roachpb.Header, - req roachpb.Request, + _ *kvpb.Header, + req kvpb.Request, latchSpans, _ *spanset.SpanSet, _ time.Duration, ) { @@ -49,11 +49,11 @@ func declareKeysQueryIntent( // request is special-cased to return a SERIALIZABLE retry error if a transaction // queries its own intent and finds it has been pushed. func QueryIntent( - ctx context.Context, reader storage.Reader, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, reader storage.Reader, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.QueryIntentRequest) + args := cArgs.Args.(*kvpb.QueryIntentRequest) h := cArgs.Header - reply := resp.(*roachpb.QueryIntentResponse) + reply := resp.(*kvpb.QueryIntentResponse) ownTxn := false if h.Txn != nil { @@ -129,9 +129,9 @@ func QueryIntent( // TransactionRetryError immediately with an updated transaction proto. // This is an optimization that can help the txn use refresh spans more // effectively. - return result.Result{}, roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE, "intent pushed") + return result.Result{}, kvpb.NewTransactionRetryError(kvpb.RETRY_SERIALIZABLE, "intent pushed") } - return result.Result{}, roachpb.NewIntentMissingError(args.Key, intent) + return result.Result{}, kvpb.NewIntentMissingError(args.Key, intent) } return result.Result{}, nil } diff --git a/pkg/kv/kvserver/batcheval/cmd_query_intent_test.go b/pkg/kv/kvserver/batcheval/cmd_query_intent_test.go index cb6729a12d8a..534894fc3b8b 100644 --- a/pkg/kv/kvserver/batcheval/cmd_query_intent_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_query_intent_test.go @@ -14,6 +14,7 @@ import ( "context" "testing" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage" @@ -83,11 +84,11 @@ func TestQueryIntent(t *testing.T) { {"readAA", txAA, txAA, keyAA, true, success}, {"readB", txB, txB, keyB, true, success}, - {"readC", txA, txA, keyC, true, &roachpb.IntentMissingError{}}, + {"readC", txA, txA, keyC, true, &kvpb.IntentMissingError{}}, // This tries reading a different key than this tx was written with. The // returned error depends on the error flag setting. - {"wrongTxE", txA, txA, keyB, true, &roachpb.IntentMissingError{}}, + {"wrongTxE", txA, txA, keyB, true, &kvpb.IntentMissingError{}}, {"wrongTx", txA, txA, keyB, false, notFound}, // This sets a mismatch for transactions in the header and the body. An @@ -97,7 +98,7 @@ func TestQueryIntent(t *testing.T) { // This simulates pushed intents by moving the tx clock backwards in time. // An error is only returned if the error flag is set. - {"clockBackE", txABack, txABack, keyA, true, &roachpb.TransactionRetryError{}}, + {"clockBackE", txABack, txABack, keyA, true, &kvpb.TransactionRetryError{}}, {"clockBack", txABack, txABack, keyA, false, notFound}, // This simulates pushed transactions by moving the tx clock forward in time. @@ -116,15 +117,15 @@ func TestQueryIntent(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { cArgs := CommandArgs{ - Header: roachpb.Header{Timestamp: clock.Now(), Txn: &test.hTransaction}, - Args: &roachpb.QueryIntentRequest{ - RequestHeader: roachpb.RequestHeader{Key: test.key}, + Header: kvpb.Header{Timestamp: clock.Now(), Txn: &test.hTransaction}, + Args: &kvpb.QueryIntentRequest{ + RequestHeader: kvpb.RequestHeader{Key: test.key}, Txn: test.argTransaction.TxnMeta, ErrorIfMissing: test.errorFlag, }, } cArgs.EvalCtx = evalCtx.EvalContext() - var resp roachpb.QueryIntentResponse + var resp kvpb.QueryIntentResponse _, err := QueryIntent(ctx, db, cArgs, &resp) switch test.response { case success: diff --git a/pkg/kv/kvserver/batcheval/cmd_query_locks.go b/pkg/kv/kvserver/batcheval/cmd_query_locks.go index 1e76ee2573d3..312ec37f6f1e 100644 --- a/pkg/kv/kvserver/batcheval/cmd_query_locks.go +++ b/pkg/kv/kvserver/batcheval/cmd_query_locks.go @@ -15,6 +15,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" @@ -23,13 +24,13 @@ import ( ) func init() { - RegisterReadOnlyCommand(roachpb.QueryLocks, declareKeysQueryLocks, QueryLocks) + RegisterReadOnlyCommand(kvpb.QueryLocks, declareKeysQueryLocks, QueryLocks) } func declareKeysQueryLocks( rs ImmutableRangeState, - _ *roachpb.Header, - _ roachpb.Request, + _ *kvpb.Header, + _ kvpb.Request, latchSpans, _ *spanset.SpanSet, _ time.Duration, ) { @@ -45,11 +46,11 @@ func declareKeysQueryLocks( // response will include one result if at least one lock is found, ensuring // that we do not allow empty responses due to byte limits. func QueryLocks( - ctx context.Context, _ storage.Reader, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, _ storage.Reader, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.QueryLocksRequest) + args := cArgs.Args.(*kvpb.QueryLocksRequest) h := cArgs.Header - reply := resp.(*roachpb.QueryLocksResponse) + reply := resp.(*kvpb.QueryLocksResponse) concurrencyManager := cArgs.EvalCtx.GetConcurrencyManager() keyScope := spanset.SpanGlobal diff --git a/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp.go b/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp.go index 9be0af466496..c844b32feb0a 100644 --- a/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp.go +++ b/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp.go @@ -15,6 +15,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/gc" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -37,7 +38,7 @@ var QueryResolvedTimestampIntentCleanupAge = settings.RegisterDurationSetting( ) func init() { - RegisterReadOnlyCommand(roachpb.QueryResolvedTimestamp, DefaultDeclareKeys, QueryResolvedTimestamp) + RegisterReadOnlyCommand(kvpb.QueryResolvedTimestamp, DefaultDeclareKeys, QueryResolvedTimestamp) } // QueryResolvedTimestamp requests a resolved timestamp for the key span it is @@ -45,16 +46,16 @@ func init() { // which all future reads within the span are guaranteed to produce the same // results, i.e. at which MVCC history has become immutable. func QueryResolvedTimestamp( - ctx context.Context, reader storage.Reader, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, reader storage.Reader, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.QueryResolvedTimestampRequest) - reply := resp.(*roachpb.QueryResolvedTimestampResponse) + args := cArgs.Args.(*kvpb.QueryResolvedTimestampRequest) + reply := resp.(*kvpb.QueryResolvedTimestampResponse) // Grab the closed timestamp from the local replica. We do this before // iterating over intents to ensure that we observe any and all intents // written before the closed timestamp went into effect. This is important // because QueryResolvedTimestamp requests are often run without acquiring - // latches (see roachpb.INCONSISTENT) and often also on follower replicas, + // latches (see kvpb.INCONSISTENT) and often also on follower replicas, // so latches won't help them to synchronize with writes. closedTS := cArgs.EvalCtx.GetClosedTimestampOlderThanStorageSnapshot() diff --git a/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go b/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go index 145e67921c16..1063061aabc3 100644 --- a/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go @@ -15,6 +15,7 @@ import ( "testing" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/gc" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -193,15 +194,15 @@ func TestQueryResolvedTimestamp(t *testing.T) { } cArgs := CommandArgs{ EvalCtx: evalCtx.EvalContext(), - Args: &roachpb.QueryResolvedTimestampRequest{ - RequestHeader: roachpb.RequestHeader{ + Args: &kvpb.QueryResolvedTimestampRequest{ + RequestHeader: kvpb.RequestHeader{ Key: roachpb.Key(cfg.span[0]), EndKey: roachpb.Key(cfg.span[1]), }, }, } - var resp roachpb.QueryResolvedTimestampResponse + var resp kvpb.QueryResolvedTimestampResponse res, err := QueryResolvedTimestamp(ctx, db, cArgs, &resp) require.NoError(t, err) require.Equal(t, cfg.expResolvedTS, resp.ResolvedTS) @@ -243,14 +244,14 @@ func TestQueryResolvedTimestampErrors(t *testing.T) { } cArgs := CommandArgs{ EvalCtx: evalCtx.EvalContext(), - Args: &roachpb.QueryResolvedTimestampRequest{ - RequestHeader: roachpb.RequestHeader{ + Args: &kvpb.QueryResolvedTimestampRequest{ + RequestHeader: kvpb.RequestHeader{ Key: roachpb.Key("a"), EndKey: roachpb.Key("z"), }, }, } - var resp roachpb.QueryResolvedTimestampResponse + var resp kvpb.QueryResolvedTimestampResponse t.Run("LockTable entry without MVCC metadata", func(t *testing.T) { require.NoError(t, db.PutEngineKey(engineKey, buf)) _, err := QueryResolvedTimestamp(ctx, db, cArgs, &resp) diff --git a/pkg/kv/kvserver/batcheval/cmd_query_txn.go b/pkg/kv/kvserver/batcheval/cmd_query_txn.go index a96a08810ade..11db94c923d2 100644 --- a/pkg/kv/kvserver/batcheval/cmd_query_txn.go +++ b/pkg/kv/kvserver/batcheval/cmd_query_txn.go @@ -15,6 +15,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -24,17 +25,17 @@ import ( ) func init() { - RegisterReadOnlyCommand(roachpb.QueryTxn, declareKeysQueryTransaction, QueryTxn) + RegisterReadOnlyCommand(kvpb.QueryTxn, declareKeysQueryTransaction, QueryTxn) } func declareKeysQueryTransaction( _ ImmutableRangeState, - _ *roachpb.Header, - req roachpb.Request, + _ *kvpb.Header, + req kvpb.Request, latchSpans, _ *spanset.SpanSet, _ time.Duration, ) { - qr := req.(*roachpb.QueryTxnRequest) + qr := req.(*kvpb.QueryTxnRequest) latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.TransactionKey(qr.Txn.Key, qr.Txn.ID)}) } @@ -46,11 +47,11 @@ func declareKeysQueryTransaction( // other txns which are waiting on this transaction in order // to find dependency cycles. func QueryTxn( - ctx context.Context, reader storage.Reader, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, reader storage.Reader, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.QueryTxnRequest) + args := cArgs.Args.(*kvpb.QueryTxnRequest) h := cArgs.Header - reply := resp.(*roachpb.QueryTxnResponse) + reply := resp.(*kvpb.QueryTxnResponse) if h.Txn != nil { return result.Result{}, ErrTransactionUnsupported diff --git a/pkg/kv/kvserver/batcheval/cmd_range_stats.go b/pkg/kv/kvserver/batcheval/cmd_range_stats.go index 5d8563a3e8eb..bd28e6ac5081 100644 --- a/pkg/kv/kvserver/batcheval/cmd_range_stats.go +++ b/pkg/kv/kvserver/batcheval/cmd_range_stats.go @@ -15,6 +15,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -22,13 +23,13 @@ import ( ) func init() { - RegisterReadOnlyCommand(roachpb.RangeStats, declareKeysRangeStats, RangeStats) + RegisterReadOnlyCommand(kvpb.RangeStats, declareKeysRangeStats, RangeStats) } func declareKeysRangeStats( rs ImmutableRangeState, - header *roachpb.Header, - req roachpb.Request, + header *kvpb.Header, + req kvpb.Request, latchSpans, lockSpans *spanset.SpanSet, maxOffset time.Duration, ) { @@ -40,9 +41,9 @@ func declareKeysRangeStats( // RangeStats returns the MVCC statistics for a range. func RangeStats( - ctx context.Context, _ storage.Reader, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, _ storage.Reader, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - reply := resp.(*roachpb.RangeStatsResponse) + reply := resp.(*kvpb.RangeStatsResponse) reply.MVCCStats = cArgs.EvalCtx.GetMVCCStats() maxQPS, qpsOK := cArgs.EvalCtx.GetMaxSplitQPS(ctx) diff --git a/pkg/kv/kvserver/batcheval/cmd_recompute_stats.go b/pkg/kv/kvserver/batcheval/cmd_recompute_stats.go index 5146476b67ab..c9f16242665b 100644 --- a/pkg/kv/kvserver/batcheval/cmd_recompute_stats.go +++ b/pkg/kv/kvserver/batcheval/cmd_recompute_stats.go @@ -15,6 +15,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" @@ -26,13 +27,13 @@ import ( ) func init() { - RegisterReadOnlyCommand(roachpb.RecomputeStats, declareKeysRecomputeStats, RecomputeStats) + RegisterReadOnlyCommand(kvpb.RecomputeStats, declareKeysRecomputeStats, RecomputeStats) } func declareKeysRecomputeStats( rs ImmutableRangeState, - _ *roachpb.Header, - _ roachpb.Request, + _ *kvpb.Header, + _ kvpb.Request, latchSpans, _ *spanset.SpanSet, _ time.Duration, ) { @@ -60,10 +61,10 @@ func declareKeysRecomputeStats( // RecomputeStats recomputes the MVCCStats stored for this range and adjust them accordingly, // returning the MVCCStats delta obtained in the process. func RecomputeStats( - ctx context.Context, reader storage.Reader, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, reader storage.Reader, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { desc := cArgs.EvalCtx.Desc() - args := cArgs.Args.(*roachpb.RecomputeStatsRequest) + args := cArgs.Args.(*kvpb.RecomputeStatsRequest) if !desc.StartKey.AsRawKey().Equal(args.Key) { return result.Result{}, errors.New("descriptor mismatch; range likely merged") } @@ -93,6 +94,6 @@ func RecomputeStats( cArgs.Stats.Add(delta) } - resp.(*roachpb.RecomputeStatsResponse).AddedDelta = enginepb.MVCCStatsDelta(delta) + resp.(*kvpb.RecomputeStatsResponse).AddedDelta = enginepb.MVCCStatsDelta(delta) return result.Result{}, nil } diff --git a/pkg/kv/kvserver/batcheval/cmd_recover_txn.go b/pkg/kv/kvserver/batcheval/cmd_recover_txn.go index 15b980715090..e6900c4cbfe2 100644 --- a/pkg/kv/kvserver/batcheval/cmd_recover_txn.go +++ b/pkg/kv/kvserver/batcheval/cmd_recover_txn.go @@ -15,6 +15,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -24,17 +25,17 @@ import ( ) func init() { - RegisterReadWriteCommand(roachpb.RecoverTxn, declareKeysRecoverTransaction, RecoverTxn) + RegisterReadWriteCommand(kvpb.RecoverTxn, declareKeysRecoverTransaction, RecoverTxn) } func declareKeysRecoverTransaction( rs ImmutableRangeState, - _ *roachpb.Header, - req roachpb.Request, + _ *kvpb.Header, + req kvpb.Request, latchSpans, _ *spanset.SpanSet, _ time.Duration, ) { - rr := req.(*roachpb.RecoverTxnRequest) + rr := req.(*kvpb.RecoverTxnRequest) latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.TransactionKey(rr.Txn.Key, rr.Txn.ID)}) latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.AbortSpanKey(rs.GetRangeID(), rr.Txn.ID)}) } @@ -50,11 +51,11 @@ func declareKeysRecoverTransaction( // result of the recovery should be committing the abandoned transaction or // aborting it. func RecoverTxn( - ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.RecoverTxnRequest) + args := cArgs.Args.(*kvpb.RecoverTxnRequest) h := cArgs.Header - reply := resp.(*roachpb.RecoverTxnResponse) + reply := resp.(*kvpb.RecoverTxnResponse) if h.Txn != nil { return result.Result{}, ErrTransactionUnsupported diff --git a/pkg/kv/kvserver/batcheval/cmd_recover_txn_test.go b/pkg/kv/kvserver/batcheval/cmd_recover_txn_test.go index e955b83a83fb..76222776f57c 100644 --- a/pkg/kv/kvserver/batcheval/cmd_recover_txn_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_recover_txn_test.go @@ -15,6 +15,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -53,14 +54,14 @@ func TestRecoverTxn(t *testing.T) { } // Issue a RecoverTxn request. - var resp roachpb.RecoverTxnResponse + var resp kvpb.RecoverTxnResponse if _, err := RecoverTxn(ctx, db, CommandArgs{ - Args: &roachpb.RecoverTxnRequest{ - RequestHeader: roachpb.RequestHeader{Key: txn.Key}, + Args: &kvpb.RecoverTxnRequest{ + RequestHeader: kvpb.RequestHeader{Key: txn.Key}, Txn: txn.TxnMeta, ImplicitlyCommitted: !missingWrite, }, - Header: roachpb.Header{ + Header: kvpb.Header{ Timestamp: ts, }, }, &resp); err != nil { @@ -229,14 +230,14 @@ func TestRecoverTxnRecordChanged(t *testing.T) { } // Issue a RecoverTxn request. - var resp roachpb.RecoverTxnResponse + var resp kvpb.RecoverTxnResponse _, err := RecoverTxn(ctx, db, CommandArgs{ - Args: &roachpb.RecoverTxnRequest{ - RequestHeader: roachpb.RequestHeader{Key: txn.Key}, + Args: &kvpb.RecoverTxnRequest{ + RequestHeader: kvpb.RequestHeader{Key: txn.Key}, Txn: txn.TxnMeta, ImplicitlyCommitted: c.implicitlyCommitted, }, - Header: roachpb.Header{ + Header: kvpb.Header{ Timestamp: ts, }, }, &resp) diff --git a/pkg/kv/kvserver/batcheval/cmd_refresh.go b/pkg/kv/kvserver/batcheval/cmd_refresh.go index 548215a22ad2..dc29e469c068 100644 --- a/pkg/kv/kvserver/batcheval/cmd_refresh.go +++ b/pkg/kv/kvserver/batcheval/cmd_refresh.go @@ -13,23 +13,23 @@ package batcheval import ( "context" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) func init() { - RegisterReadOnlyCommand(roachpb.Refresh, DefaultDeclareKeys, Refresh) + RegisterReadOnlyCommand(kvpb.Refresh, DefaultDeclareKeys, Refresh) } // Refresh checks whether the key has any values written in the interval // (args.RefreshFrom, header.Timestamp]. func Refresh( - ctx context.Context, reader storage.Reader, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, reader storage.Reader, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.RefreshRequest) + args := cArgs.Args.(*kvpb.RefreshRequest) h := cArgs.Header if h.Txn == nil { @@ -64,14 +64,14 @@ func Refresh( } else if res.Value != nil { if ts := res.Value.Timestamp; refreshFrom.Less(ts) { return result.Result{}, - roachpb.NewRefreshFailedError(roachpb.RefreshFailedError_REASON_COMMITTED_VALUE, args.Key, ts) + kvpb.NewRefreshFailedError(kvpb.RefreshFailedError_REASON_COMMITTED_VALUE, args.Key, ts) } } // Check if an intent which is not owned by this transaction was written // at or beneath the refresh timestamp. if res.Intent != nil && res.Intent.Txn.ID != h.Txn.ID { - return result.Result{}, roachpb.NewRefreshFailedError(roachpb.RefreshFailedError_REASON_INTENT, + return result.Result{}, kvpb.NewRefreshFailedError(kvpb.RefreshFailedError_REASON_INTENT, res.Intent.Key, res.Intent.Txn.WriteTimestamp) } diff --git a/pkg/kv/kvserver/batcheval/cmd_refresh_range.go b/pkg/kv/kvserver/batcheval/cmd_refresh_range.go index 3ce05bdfad35..3708ddadf7af 100644 --- a/pkg/kv/kvserver/batcheval/cmd_refresh_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_refresh_range.go @@ -13,6 +13,7 @@ package batcheval import ( "context" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" @@ -25,15 +26,15 @@ import ( ) func init() { - RegisterReadOnlyCommand(roachpb.RefreshRange, DefaultDeclareKeys, RefreshRange) + RegisterReadOnlyCommand(kvpb.RefreshRange, DefaultDeclareKeys, RefreshRange) } // RefreshRange checks whether the key range specified has any values written in // the interval (args.RefreshFrom, header.Timestamp]. func RefreshRange( - ctx context.Context, reader storage.Reader, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, reader storage.Reader, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.RefreshRangeRequest) + args := cArgs.Args.(*kvpb.RefreshRangeRequest) h := cArgs.Header if h.Txn == nil { @@ -93,7 +94,7 @@ func refreshRange( key := iter.UnsafeKey().Clone() if _, hasRange := iter.HasPointAndRange(); hasRange { - return roachpb.NewRefreshFailedError(roachpb.RefreshFailedError_REASON_COMMITTED_VALUE, + return kvpb.NewRefreshFailedError(kvpb.RefreshFailedError_REASON_COMMITTED_VALUE, key.Key, iter.RangeKeys().Versions[0].Timestamp) } @@ -128,12 +129,12 @@ func refreshRange( } continue } - return roachpb.NewRefreshFailedError(roachpb.RefreshFailedError_REASON_INTENT, + return kvpb.NewRefreshFailedError(kvpb.RefreshFailedError_REASON_INTENT, key.Key, meta.Txn.WriteTimestamp) } // If a committed value is found, return an error. - return roachpb.NewRefreshFailedError(roachpb.RefreshFailedError_REASON_COMMITTED_VALUE, + return kvpb.NewRefreshFailedError(kvpb.RefreshFailedError_REASON_COMMITTED_VALUE, key.Key, key.Timestamp) } return nil diff --git a/pkg/kv/kvserver/batcheval/cmd_refresh_range_bench_test.go b/pkg/kv/kvserver/batcheval/cmd_refresh_range_bench_test.go index a3350b564586..1712aa96e351 100644 --- a/pkg/kv/kvserver/batcheval/cmd_refresh_range_bench_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_refresh_range_bench_test.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -127,17 +128,17 @@ func runRefreshRangeBenchmark(b *testing.B, emk engineMaker, opts benchOptions) b.ResetTimer() for i := 0; i < b.N; i++ { func() { - var resp roachpb.RefreshRangeResponse + var resp kvpb.RefreshRangeResponse _, err := batcheval.RefreshRange(ctx, eng, batcheval.CommandArgs{ EvalCtx: evalCtx, - Args: &roachpb.RefreshRangeRequest{ - RequestHeader: roachpb.RequestHeader{ + Args: &kvpb.RefreshRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: startKey, EndKey: endKey, }, RefreshFrom: opts.refreshFrom, }, - Header: roachpb.Header{ + Header: kvpb.Header{ Txn: &roachpb.Transaction{ TxnMeta: enginepb.TxnMeta{ WriteTimestamp: opts.refreshTo, diff --git a/pkg/kv/kvserver/batcheval/cmd_refresh_range_test.go b/pkg/kv/kvserver/batcheval/cmd_refresh_range_test.go index 13883b31c637..ff59fe5f9dd7 100644 --- a/pkg/kv/kvserver/batcheval/cmd_refresh_range_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_refresh_range_test.go @@ -15,6 +15,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage" @@ -53,23 +54,23 @@ func TestRefreshRange(t *testing.T) { "above all": {"a", "z", 8, 10, nil}, "between": {"a", "z", 4, 4, nil}, "beside": {"x", "z", 1, 10, nil}, - "point key": {"a", "z", 2, 4, &roachpb.RefreshFailedError{ - Reason: roachpb.RefreshFailedError_REASON_COMMITTED_VALUE, + "point key": {"a", "z", 2, 4, &kvpb.RefreshFailedError{ + Reason: kvpb.RefreshFailedError_REASON_COMMITTED_VALUE, Key: roachpb.Key("b"), Timestamp: hlc.Timestamp{WallTime: 3}, }}, - "point tombstone": {"a", "z", 4, 6, &roachpb.RefreshFailedError{ - Reason: roachpb.RefreshFailedError_REASON_COMMITTED_VALUE, + "point tombstone": {"a", "z", 4, 6, &kvpb.RefreshFailedError{ + Reason: kvpb.RefreshFailedError_REASON_COMMITTED_VALUE, Key: roachpb.Key("c"), Timestamp: hlc.Timestamp{WallTime: 5}, }}, - "range tombstone": {"a", "z", 6, 8, &roachpb.RefreshFailedError{ - Reason: roachpb.RefreshFailedError_REASON_COMMITTED_VALUE, + "range tombstone": {"a", "z", 6, 8, &kvpb.RefreshFailedError{ + Reason: kvpb.RefreshFailedError_REASON_COMMITTED_VALUE, Key: roachpb.Key("d"), Timestamp: hlc.Timestamp{WallTime: 7}, }}, - "to is inclusive": {"a", "z", 1, 3, &roachpb.RefreshFailedError{ - Reason: roachpb.RefreshFailedError_REASON_COMMITTED_VALUE, + "to is inclusive": {"a", "z", 1, 3, &kvpb.RefreshFailedError{ + Reason: kvpb.RefreshFailedError_REASON_COMMITTED_VALUE, Key: roachpb.Key("b"), Timestamp: hlc.Timestamp{WallTime: 3}, }}, @@ -81,14 +82,14 @@ func TestRefreshRange(t *testing.T) { EvalCtx: (&MockEvalCtx{ ClusterSettings: cluster.MakeTestingClusterSettings(), }).EvalContext(), - Args: &roachpb.RefreshRangeRequest{ - RequestHeader: roachpb.RequestHeader{ + Args: &kvpb.RefreshRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: roachpb.Key(tc.start), EndKey: roachpb.Key(tc.end), }, RefreshFrom: hlc.Timestamp{WallTime: tc.from}, }, - Header: roachpb.Header{ + Header: kvpb.Header{ Timestamp: hlc.Timestamp{WallTime: tc.to}, Txn: &roachpb.Transaction{ TxnMeta: enginepb.TxnMeta{ @@ -97,12 +98,12 @@ func TestRefreshRange(t *testing.T) { ReadTimestamp: hlc.Timestamp{WallTime: tc.to}, }, }, - }, &roachpb.RefreshRangeResponse{}) + }, &kvpb.RefreshRangeResponse{}) if tc.expectErr == nil { require.NoError(t, err) } else { - var refreshErr *roachpb.RefreshFailedError + var refreshErr *kvpb.RefreshFailedError require.Error(t, err) require.ErrorAs(t, err, &refreshErr) require.Equal(t, tc.expectErr, refreshErr) @@ -204,19 +205,19 @@ func TestRefreshRangeTimeBoundIterator(t *testing.T) { // time-bound iterator meant that we would see the first sstable but // not the second and incorrectly report the intent as pending, // resulting in an error from RefreshRange. - var resp roachpb.RefreshRangeResponse + var resp kvpb.RefreshRangeResponse _, err := RefreshRange(ctx, db, CommandArgs{ EvalCtx: (&MockEvalCtx{ ClusterSettings: cluster.MakeTestingClusterSettings(), }).EvalContext(), - Args: &roachpb.RefreshRangeRequest{ - RequestHeader: roachpb.RequestHeader{ + Args: &kvpb.RefreshRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: k, EndKey: keys.MaxKey, }, RefreshFrom: ts2, }, - Header: roachpb.Header{ + Header: kvpb.Header{ Txn: &roachpb.Transaction{ TxnMeta: enginepb.TxnMeta{ WriteTimestamp: ts3, @@ -279,19 +280,19 @@ func TestRefreshRangeError(t *testing.T) { // We are trying to refresh from time 1 to 3, but the key was written at time // 2, therefore the refresh should fail. - var resp roachpb.RefreshRangeResponse + var resp kvpb.RefreshRangeResponse _, err := RefreshRange(ctx, db, CommandArgs{ EvalCtx: (&MockEvalCtx{ ClusterSettings: cluster.MakeTestingClusterSettings(), }).EvalContext(), - Args: &roachpb.RefreshRangeRequest{ - RequestHeader: roachpb.RequestHeader{ + Args: &kvpb.RefreshRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: k, EndKey: keys.MaxKey, }, RefreshFrom: ts1, }, - Header: roachpb.Header{ + Header: kvpb.Header{ Txn: &roachpb.Transaction{ TxnMeta: enginepb.TxnMeta{ WriteTimestamp: ts3, @@ -301,7 +302,7 @@ func TestRefreshRangeError(t *testing.T) { Timestamp: ts3, }, }, &resp) - require.IsType(t, &roachpb.RefreshFailedError{}, err) + require.IsType(t, &kvpb.RefreshFailedError{}, err) if resolveIntent { require.Equal(t, "encountered recently written committed value \"resolved_key\" @0.000000002,0", err.Error()) diff --git a/pkg/kv/kvserver/batcheval/cmd_refresh_test.go b/pkg/kv/kvserver/batcheval/cmd_refresh_test.go index 89a9a664633b..808f43bf2856 100644 --- a/pkg/kv/kvserver/batcheval/cmd_refresh_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_refresh_test.go @@ -14,6 +14,7 @@ import ( "context" "testing" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" @@ -73,15 +74,15 @@ func TestRefreshError(t *testing.T) { // We are trying to refresh from time 1 to 3, but the key was written at // time 2, therefore the refresh should fail. - var resp roachpb.RefreshResponse + var resp kvpb.RefreshResponse _, err := Refresh(ctx, db, CommandArgs{ - Args: &roachpb.RefreshRequest{ - RequestHeader: roachpb.RequestHeader{ + Args: &kvpb.RefreshRequest{ + RequestHeader: kvpb.RequestHeader{ Key: k, }, RefreshFrom: ts1, }, - Header: roachpb.Header{ + Header: kvpb.Header{ Txn: &roachpb.Transaction{ TxnMeta: enginepb.TxnMeta{ WriteTimestamp: ts3, @@ -91,7 +92,7 @@ func TestRefreshError(t *testing.T) { Timestamp: ts3, }, }, &resp) - require.IsType(t, &roachpb.RefreshFailedError{}, err) + require.IsType(t, &kvpb.RefreshFailedError{}, err) if resolveIntent { require.Equal(t, "encountered recently written committed value \"resolved_key\" @0.000000002,0", err.Error()) @@ -133,15 +134,15 @@ func TestRefreshTimestampBounds(t *testing.T) { // RefreshTo is exclusive, so expect no error on collision. {ts2, ts3, false}, } { - var resp roachpb.RefreshResponse + var resp kvpb.RefreshResponse _, err := Refresh(ctx, db, CommandArgs{ - Args: &roachpb.RefreshRequest{ - RequestHeader: roachpb.RequestHeader{ + Args: &kvpb.RefreshRequest{ + RequestHeader: kvpb.RequestHeader{ Key: k, }, RefreshFrom: tc.from, }, - Header: roachpb.Header{ + Header: kvpb.Header{ Txn: &roachpb.Transaction{ TxnMeta: enginepb.TxnMeta{ WriteTimestamp: tc.to, diff --git a/pkg/kv/kvserver/batcheval/cmd_resolve_intent.go b/pkg/kv/kvserver/batcheval/cmd_resolve_intent.go index 268838d235f1..29353a86703e 100644 --- a/pkg/kv/kvserver/batcheval/cmd_resolve_intent.go +++ b/pkg/kv/kvserver/batcheval/cmd_resolve_intent.go @@ -15,6 +15,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -24,21 +25,21 @@ import ( ) func init() { - RegisterReadWriteCommand(roachpb.ResolveIntent, declareKeysResolveIntent, ResolveIntent) + RegisterReadWriteCommand(kvpb.ResolveIntent, declareKeysResolveIntent, ResolveIntent) } func declareKeysResolveIntentCombined( - rs ImmutableRangeState, req roachpb.Request, latchSpans *spanset.SpanSet, + rs ImmutableRangeState, req kvpb.Request, latchSpans *spanset.SpanSet, ) { var status roachpb.TransactionStatus var txnID uuid.UUID var minTxnTS hlc.Timestamp switch t := req.(type) { - case *roachpb.ResolveIntentRequest: + case *kvpb.ResolveIntentRequest: status = t.Status txnID = t.IntentTxn.ID minTxnTS = t.IntentTxn.MinTimestamp - case *roachpb.ResolveIntentRangeRequest: + case *kvpb.ResolveIntentRangeRequest: status = t.Status txnID = t.IntentTxn.ID minTxnTS = t.IntentTxn.MinTimestamp @@ -53,8 +54,8 @@ func declareKeysResolveIntentCombined( func declareKeysResolveIntent( rs ImmutableRangeState, - _ *roachpb.Header, - req roachpb.Request, + _ *kvpb.Header, + req kvpb.Request, latchSpans, _ *spanset.SpanSet, _ time.Duration, ) { @@ -78,9 +79,9 @@ func resolveToMetricType(status roachpb.TransactionStatus, poison bool) *result. // ResolveIntent resolves a write intent from the specified key // according to the status of the transaction which created it. func ResolveIntent( - ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.ResolveIntentRequest) + args := cArgs.Args.(*kvpb.ResolveIntentRequest) h := cArgs.Header ms := cArgs.Stats @@ -98,11 +99,11 @@ func ResolveIntent( if err != nil { return result.Result{}, err } - reply := resp.(*roachpb.ResolveIntentResponse) + reply := resp.(*kvpb.ResolveIntentResponse) reply.NumBytes = numBytes if resumeSpan != nil { reply.ResumeSpan = resumeSpan - reply.ResumeReason = roachpb.RESUME_BYTE_LIMIT + reply.ResumeReason = kvpb.RESUME_BYTE_LIMIT return result.Result{}, nil } diff --git a/pkg/kv/kvserver/batcheval/cmd_resolve_intent_range.go b/pkg/kv/kvserver/batcheval/cmd_resolve_intent_range.go index b64b24f06b2a..64cc8c73e442 100644 --- a/pkg/kv/kvserver/batcheval/cmd_resolve_intent_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_resolve_intent_range.go @@ -14,6 +14,7 @@ import ( "context" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -21,13 +22,13 @@ import ( ) func init() { - RegisterReadWriteCommand(roachpb.ResolveIntentRange, declareKeysResolveIntentRange, ResolveIntentRange) + RegisterReadWriteCommand(kvpb.ResolveIntentRange, declareKeysResolveIntentRange, ResolveIntentRange) } func declareKeysResolveIntentRange( rs ImmutableRangeState, - _ *roachpb.Header, - req roachpb.Request, + _ *kvpb.Header, + req kvpb.Request, latchSpans, _ *spanset.SpanSet, _ time.Duration, ) { @@ -37,9 +38,9 @@ func declareKeysResolveIntentRange( // ResolveIntentRange resolves write intents in the specified // key range according to the status of the transaction which created it. func ResolveIntentRange( - ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.ResolveIntentRangeRequest) + args := cArgs.Args.(*kvpb.ResolveIntentRangeRequest) h := cArgs.Header ms := cArgs.Stats @@ -57,7 +58,7 @@ func ResolveIntentRange( if err != nil { return result.Result{}, err } - reply := resp.(*roachpb.ResolveIntentRangeResponse) + reply := resp.(*kvpb.ResolveIntentRangeResponse) reply.NumKeys = numKeys reply.NumBytes = numBytes if resumeSpan != nil { diff --git a/pkg/kv/kvserver/batcheval/cmd_resolve_intent_test.go b/pkg/kv/kvserver/batcheval/cmd_resolve_intent_test.go index 822aae38ceb2..09984c9b6890 100644 --- a/pkg/kv/kvserver/batcheval/cmd_resolve_intent_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_resolve_intent_test.go @@ -17,6 +17,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/abortspan" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -83,13 +84,13 @@ func TestDeclareKeysResolveIntent(t *testing.T) { testutils.RunTrueAndFalse(t, "ranged", func(t *testing.T, ranged bool) { for _, test := range tests { t.Run("", func(t *testing.T) { - ri := roachpb.ResolveIntentRequest{ + ri := kvpb.ResolveIntentRequest{ IntentTxn: txnMeta, Status: test.status, Poison: test.poison, } ri.Key = roachpb.Key("b") - rir := roachpb.ResolveIntentRangeRequest{ + rir := kvpb.ResolveIntentRangeRequest{ IntentTxn: ri.IntentTxn, Status: ri.Status, Poison: ri.Poison, @@ -101,7 +102,7 @@ func TestDeclareKeysResolveIntent(t *testing.T) { var latchSpans, lockSpans spanset.SpanSet - var h roachpb.Header + var h kvpb.Header h.RangeID = desc.RangeID cArgs := CommandArgs{Header: h} @@ -112,7 +113,7 @@ func TestDeclareKeysResolveIntent(t *testing.T) { declareKeysResolveIntent(&desc, &h, &ri, &latchSpans, &lockSpans, 0) batch := spanset.NewBatch(engine.NewBatch(), &latchSpans) defer batch.Close() - if _, err := ResolveIntent(ctx, batch, cArgs, &roachpb.ResolveIntentResponse{}); err != nil { + if _, err := ResolveIntent(ctx, batch, cArgs, &kvpb.ResolveIntentResponse{}); err != nil { t.Fatal(err) } } else { @@ -120,7 +121,7 @@ func TestDeclareKeysResolveIntent(t *testing.T) { declareKeysResolveIntentRange(&desc, &h, &rir, &latchSpans, &lockSpans, 0) batch := spanset.NewBatch(engine.NewBatch(), &latchSpans) defer batch.Close() - if _, err := ResolveIntentRange(ctx, batch, cArgs, &roachpb.ResolveIntentRangeResponse{}); err != nil { + if _, err := ResolveIntentRange(ctx, batch, cArgs, &kvpb.ResolveIntentRangeResponse{}); err != nil { t.Fatal(err) } } @@ -183,7 +184,7 @@ func TestResolveIntentAfterPartialRollback(t *testing.T) { // Partially revert the 2nd store above. ignoredSeqNums := []enginepb.IgnoredSeqNumRange{{Start: 1, End: 1}} - h := roachpb.Header{ + h := kvpb.Header{ RangeID: desc.RangeID, Timestamp: ts, } @@ -196,7 +197,7 @@ func TestResolveIntentAfterPartialRollback(t *testing.T) { if !ranged { // Resolve a point intent. - ri := roachpb.ResolveIntentRequest{ + ri := kvpb.ResolveIntentRequest{ IntentTxn: txn.TxnMeta, Status: roachpb.COMMITTED, IgnoredSeqNums: ignoredSeqNums, @@ -213,13 +214,13 @@ func TestResolveIntentAfterPartialRollback(t *testing.T) { EvalCtx: (&MockEvalCtx{ClusterSettings: st}).EvalContext(), Args: &ri, }, - &roachpb.ResolveIntentResponse{}, + &kvpb.ResolveIntentResponse{}, ); err != nil { t.Fatal(err) } } else { // Resolve an intent range. - rir := roachpb.ResolveIntentRangeRequest{ + rir := kvpb.ResolveIntentRangeRequest{ IntentTxn: txn.TxnMeta, Status: roachpb.COMMITTED, IgnoredSeqNums: ignoredSeqNums, @@ -238,7 +239,7 @@ func TestResolveIntentAfterPartialRollback(t *testing.T) { EvalCtx: (&MockEvalCtx{ClusterSettings: st}).EvalContext(), Args: &rir, }, - &roachpb.ResolveIntentRangeResponse{}, + &kvpb.ResolveIntentRangeResponse{}, ); err != nil { t.Fatal(err) } @@ -310,7 +311,7 @@ func TestResolveIntentWithTargetBytes(t *testing.T) { if !ranged { // Resolve a point intent for testKeys[0]. - ri := roachpb.ResolveIntentRequest{ + ri := kvpb.ResolveIntentRequest{ IntentTxn: txn.TxnMeta, Status: roachpb.COMMITTED, } @@ -319,12 +320,12 @@ func TestResolveIntentWithTargetBytes(t *testing.T) { { // Case 1: TargetBytes = -1. In this case, we should not resolve any // intents. - resp := &roachpb.ResolveIntentResponse{} + resp := &kvpb.ResolveIntentResponse{} _, err := ResolveIntent(ctx, batch, CommandArgs{ EvalCtx: (&MockEvalCtx{ClusterSettings: st}).EvalContext(), Args: &ri, - Header: roachpb.Header{ + Header: kvpb.Header{ TargetBytes: -1, }, }, @@ -333,7 +334,7 @@ func TestResolveIntentWithTargetBytes(t *testing.T) { require.NoError(t, err) require.Equal(t, resp.NumBytes, int64(0)) require.Equal(t, resp.ResumeSpan.Key, testKeys[0]) - require.Equal(t, resp.ResumeReason, roachpb.RESUME_BYTE_LIMIT) + require.Equal(t, resp.ResumeReason, kvpb.RESUME_BYTE_LIMIT) require.NoError(t, err) numBytes := batch.Len() require.Equal(t, numBytes, initialBytes) @@ -345,12 +346,12 @@ func TestResolveIntentWithTargetBytes(t *testing.T) { { // Case 2: TargetBytes = 500. In this case, we should resolve the // intent for testKeys[0]. - resp := &roachpb.ResolveIntentResponse{} + resp := &kvpb.ResolveIntentResponse{} _, err := ResolveIntent(ctx, batch, CommandArgs{ EvalCtx: (&MockEvalCtx{ClusterSettings: st}).EvalContext(), Args: &ri, - Header: roachpb.Header{ + Header: kvpb.Header{ TargetBytes: 500, }, }, @@ -359,7 +360,7 @@ func TestResolveIntentWithTargetBytes(t *testing.T) { require.Greater(t, resp.NumBytes, int64(1000)) require.Less(t, resp.NumBytes, int64(1100)) require.Nil(t, resp.ResumeSpan) - require.Equal(t, resp.ResumeReason, roachpb.RESUME_UNKNOWN) + require.Equal(t, resp.ResumeReason, kvpb.RESUME_UNKNOWN) require.NoError(t, err) numBytes := batch.Len() require.Greater(t, numBytes, initialBytes+1000) @@ -373,7 +374,7 @@ func TestResolveIntentWithTargetBytes(t *testing.T) { } else { // Resolve an intent range for testKeys[0], testKeys[1], ..., // testKeys[4]. - rir := roachpb.ResolveIntentRangeRequest{ + rir := kvpb.ResolveIntentRangeRequest{ IntentTxn: txn.TxnMeta, Status: roachpb.COMMITTED, } @@ -383,12 +384,12 @@ func TestResolveIntentWithTargetBytes(t *testing.T) { { // Case 1: TargetBytes = -1. In this case, we should not resolve any // intents. - respr := &roachpb.ResolveIntentRangeResponse{} + respr := &kvpb.ResolveIntentRangeResponse{} _, err := ResolveIntentRange(ctx, batch, CommandArgs{ EvalCtx: (&MockEvalCtx{ClusterSettings: st}).EvalContext(), Args: &rir, - Header: roachpb.Header{ + Header: kvpb.Header{ TargetBytes: -1, }, }, @@ -399,7 +400,7 @@ func TestResolveIntentWithTargetBytes(t *testing.T) { require.Equal(t, respr.NumBytes, int64(0)) require.Equal(t, respr.ResumeSpan.Key, testKeys[0]) require.Equal(t, respr.ResumeSpan.EndKey, testKeys[nKeys-1].Next()) - require.Equal(t, respr.ResumeReason, roachpb.RESUME_BYTE_LIMIT) + require.Equal(t, respr.ResumeReason, kvpb.RESUME_BYTE_LIMIT) require.NoError(t, err) numBytes := batch.Len() require.Equal(t, numBytes, initialBytes) @@ -412,12 +413,12 @@ func TestResolveIntentWithTargetBytes(t *testing.T) { // Case 2: TargetBytes = 2900. In this case, we should resolve the // first 3 intents - testKey[0], testKeys[1], and testKeys[2] (since we // resolve intents until we exceed the TargetBytes limit). - respr := &roachpb.ResolveIntentRangeResponse{} + respr := &kvpb.ResolveIntentRangeResponse{} _, err := ResolveIntentRange(ctx, batch, CommandArgs{ EvalCtx: (&MockEvalCtx{ClusterSettings: st}).EvalContext(), Args: &rir, - Header: roachpb.Header{ + Header: kvpb.Header{ TargetBytes: 2900, }, }, @@ -428,7 +429,7 @@ func TestResolveIntentWithTargetBytes(t *testing.T) { require.Less(t, respr.NumBytes, int64(3300)) require.Equal(t, respr.ResumeSpan.Key, testKeys[2].Next()) require.Equal(t, respr.ResumeSpan.EndKey, testKeys[nKeys-1].Next()) - require.Equal(t, respr.ResumeReason, roachpb.RESUME_BYTE_LIMIT) + require.Equal(t, respr.ResumeReason, kvpb.RESUME_BYTE_LIMIT) require.NoError(t, err) numBytes := batch.Len() require.Greater(t, numBytes, initialBytes+3000) @@ -447,12 +448,12 @@ func TestResolveIntentWithTargetBytes(t *testing.T) { // testKeys[4]). In this case, we should resolve the remaining // intents - testKey[4] and testKeys[5] (since we resolve intents until // we exceed the TargetBytes limit). - respr := &roachpb.ResolveIntentRangeResponse{} + respr := &kvpb.ResolveIntentRangeResponse{} _, err := ResolveIntentRange(ctx, batch, CommandArgs{ EvalCtx: (&MockEvalCtx{ClusterSettings: st}).EvalContext(), Args: &rir, - Header: roachpb.Header{ + Header: kvpb.Header{ TargetBytes: 1100, }, }, @@ -462,7 +463,7 @@ func TestResolveIntentWithTargetBytes(t *testing.T) { require.Greater(t, respr.NumBytes, int64(2000)) require.Less(t, respr.NumBytes, int64(2200)) require.Nil(t, respr.ResumeSpan) - require.Equal(t, respr.ResumeReason, roachpb.RESUME_UNKNOWN) + require.Equal(t, respr.ResumeReason, kvpb.RESUME_UNKNOWN) require.NoError(t, err) numBytes := batch.Len() require.Greater(t, numBytes, initialBytes+5000) diff --git a/pkg/kv/kvserver/batcheval/cmd_reverse_scan.go b/pkg/kv/kvserver/batcheval/cmd_reverse_scan.go index f438f9299e52..98c3d6f0d09f 100644 --- a/pkg/kv/kvserver/batcheval/cmd_reverse_scan.go +++ b/pkg/kv/kvserver/batcheval/cmd_reverse_scan.go @@ -14,14 +14,14 @@ import ( "context" "fmt" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" ) func init() { - RegisterReadOnlyCommand(roachpb.ReverseScan, DefaultDeclareIsolatedKeys, ReverseScan) + RegisterReadOnlyCommand(kvpb.ReverseScan, DefaultDeclareIsolatedKeys, ReverseScan) } // ReverseScan scans the key range specified by start key through @@ -29,18 +29,18 @@ func init() { // maxKeys stores the number of scan results remaining for this batch // (MaxInt64 for no limit). func ReverseScan( - ctx context.Context, reader storage.Reader, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, reader storage.Reader, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.ReverseScanRequest) + args := cArgs.Args.(*kvpb.ReverseScanRequest) h := cArgs.Header - reply := resp.(*roachpb.ReverseScanResponse) + reply := resp.(*kvpb.ReverseScanResponse) var res result.Result var scanRes storage.MVCCScanResult var err error opts := storage.MVCCScanOptions{ - Inconsistent: h.ReadConsistency != roachpb.CONSISTENT, + Inconsistent: h.ReadConsistency != kvpb.CONSISTENT, SkipLocked: h.WaitPolicy == lock.WaitPolicy_SkipLocked, Txn: h.Txn, MaxKeys: h.MaxSpanRequestKeys, @@ -56,14 +56,14 @@ func ReverseScan( } switch args.ScanFormat { - case roachpb.BATCH_RESPONSE: + case kvpb.BATCH_RESPONSE: scanRes, err = storage.MVCCScanToBytes( ctx, reader, args.Key, args.EndKey, h.Timestamp, opts) if err != nil { return result.Result{}, err } reply.BatchResponses = scanRes.KVData - case roachpb.COL_BATCH_RESPONSE: + case kvpb.COL_BATCH_RESPONSE: scanRes, err = storage.MVCCScanToCols( ctx, reader, cArgs.Header.IndexFetchSpec, args.Key, args.EndKey, h.Timestamp, opts, cArgs.EvalCtx.ClusterSettings(), @@ -72,7 +72,7 @@ func ReverseScan( return result.Result{}, err } reply.BatchResponses = scanRes.KVData - case roachpb.KEY_VALUES: + case kvpb.KEY_VALUES: scanRes, err = storage.MVCCScan( ctx, reader, args.Key, args.EndKey, h.Timestamp, opts) if err != nil { @@ -92,7 +92,7 @@ func ReverseScan( reply.ResumeNextBytes = scanRes.ResumeNextBytes } - if h.ReadConsistency == roachpb.READ_UNCOMMITTED { + if h.ReadConsistency == kvpb.READ_UNCOMMITTED { // NOTE: MVCCScan doesn't use a Prefix iterator, so we don't want to use // one in CollectIntentRows either so that we're guaranteed to use the // same cached iterator and observe a consistent snapshot of the engine. diff --git a/pkg/kv/kvserver/batcheval/cmd_revert_range.go b/pkg/kv/kvserver/batcheval/cmd_revert_range.go index 0f8f52ccb7a6..9f1d09b26717 100644 --- a/pkg/kv/kvserver/batcheval/cmd_revert_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_revert_range.go @@ -15,6 +15,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" @@ -31,17 +32,17 @@ import ( const clearRangeThreshold = 64 func init() { - RegisterReadWriteCommand(roachpb.RevertRange, declareKeysRevertRange, RevertRange) + RegisterReadWriteCommand(kvpb.RevertRange, declareKeysRevertRange, RevertRange) } func declareKeysRevertRange( rs ImmutableRangeState, - header *roachpb.Header, - req roachpb.Request, + header *kvpb.Header, + req kvpb.Request, latchSpans, lockSpans *spanset.SpanSet, maxOffset time.Duration, ) { - args := req.(*roachpb.RevertRangeRequest) + args := req.(*kvpb.RevertRangeRequest) DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans, maxOffset) // We look up the range descriptor key to check whether the span // is equal to the entire range for fast stats updating. @@ -97,15 +98,15 @@ const maxRevertRangeBatchBytes = 32 << 20 // Note: this should only be used when there is no user traffic writing to the // target span at or above the target time. func RevertRange( - ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { if cArgs.Header.Txn != nil { return result.Result{}, ErrTransactionUnsupported } log.VEventf(ctx, 2, "RevertRange %+v", cArgs.Args) - args := cArgs.Args.(*roachpb.RevertRangeRequest) - reply := resp.(*roachpb.RevertRangeResponse) + args := cArgs.Args.(*kvpb.RevertRangeRequest) + reply := resp.(*kvpb.RevertRangeResponse) desc := cArgs.EvalCtx.Desc() pd := result.Result{ Replicated: kvserverpb.ReplicatedEvalResult{ @@ -152,7 +153,7 @@ func RevertRange( // there only be one. Thus we just set it to MaxKeys when, and only when, // we're returning a ResumeSpan. reply.NumKeys = cArgs.Header.MaxSpanRequestKeys - reply.ResumeReason = roachpb.RESUME_KEY_LIMIT + reply.ResumeReason = kvpb.RESUME_KEY_LIMIT } return pd, nil diff --git a/pkg/kv/kvserver/batcheval/cmd_revert_range_test.go b/pkg/kv/kvserver/batcheval/cmd_revert_range_test.go index fc67194ec1ad..13d808987a82 100644 --- a/pkg/kv/kvserver/batcheval/cmd_revert_range_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_revert_range_test.go @@ -18,6 +18,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -106,7 +107,7 @@ func TestCmdRevertRange(t *testing.T) { StartKey: roachpb.RKey(startKey), EndKey: roachpb.RKey(endKey), } - cArgs := batcheval.CommandArgs{Header: roachpb.Header{RangeID: desc.RangeID, Timestamp: tsReq, MaxSpanRequestKeys: 2}} + cArgs := batcheval.CommandArgs{Header: kvpb.Header{RangeID: desc.RangeID, Timestamp: tsReq, MaxSpanRequestKeys: 2}} evalCtx := &batcheval.MockEvalCtx{Desc: &desc, Clock: hlc.NewClockForTesting(nil), Stats: stats} cArgs.EvalCtx = evalCtx.EvalContext() afterStats, err := storage.ComputeStats(eng, keys.LocalMax, keys.MaxKey, 0) @@ -125,15 +126,15 @@ func TestCmdRevertRange(t *testing.T) { batch := &wrappedBatch{Batch: eng.NewBatch()} defer batch.Close() - req := roachpb.RevertRangeRequest{ - RequestHeader: roachpb.RequestHeader{Key: startKey, EndKey: endKey}, + req := kvpb.RevertRangeRequest{ + RequestHeader: kvpb.RequestHeader{Key: startKey, EndKey: endKey}, TargetTime: tc.ts, } cArgs.Stats = &enginepb.MVCCStats{} cArgs.Args = &req var resumes int for { - var reply roachpb.RevertRangeResponse + var reply kvpb.RevertRangeResponse result, err := batcheval.RevertRange(ctx, batch, cArgs, &reply) if err != nil { t.Fatal(err) @@ -203,15 +204,15 @@ func TestCmdRevertRange(t *testing.T) { batch := &wrappedBatch{Batch: eng.NewBatch()} defer batch.Close() cArgs.Stats = &enginepb.MVCCStats{} - req := roachpb.RevertRangeRequest{ - RequestHeader: roachpb.RequestHeader{Key: startKey, EndKey: endKey}, + req := kvpb.RevertRangeRequest{ + RequestHeader: kvpb.RequestHeader{Key: startKey, EndKey: endKey}, TargetTime: tc.ts, } cArgs.Args = &req var resumes int var err error for { - var reply roachpb.RevertRangeResponse + var reply kvpb.RevertRangeResponse var result result.Result result, err = batcheval.RevertRange(ctx, batch, cArgs, &reply) if err != nil || reply.ResumeSpan == nil { @@ -286,17 +287,17 @@ func TestCmdRevertRangeMVCCRangeTombstones(t *testing.T) { Clock: hlc.NewClockForTesting(nil), Stats: ms, }).EvalContext(), - Header: roachpb.Header{ + Header: kvpb.Header{ RangeID: desc.RangeID, Timestamp: wallTS(10e9), }, - Args: &roachpb.RevertRangeRequest{ - RequestHeader: roachpb.RequestHeader{Key: startKey, EndKey: endKey}, + Args: &kvpb.RevertRangeRequest{ + RequestHeader: kvpb.RequestHeader{Key: startKey, EndKey: endKey}, TargetTime: wallTS(1e9), }, Stats: &ms, } - _, err := batcheval.RevertRange(ctx, batch, cArgs, &roachpb.RevertRangeResponse{}) + _, err := batcheval.RevertRange(ctx, batch, cArgs, &kvpb.RevertRangeResponse{}) require.NoError(t, err) require.NoError(t, batch.Commit(false)) diff --git a/pkg/kv/kvserver/batcheval/cmd_scan.go b/pkg/kv/kvserver/batcheval/cmd_scan.go index d4817ec92823..317b1cff1265 100644 --- a/pkg/kv/kvserver/batcheval/cmd_scan.go +++ b/pkg/kv/kvserver/batcheval/cmd_scan.go @@ -14,14 +14,14 @@ import ( "context" "fmt" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" ) func init() { - RegisterReadOnlyCommand(roachpb.Scan, DefaultDeclareIsolatedKeys, Scan) + RegisterReadOnlyCommand(kvpb.Scan, DefaultDeclareIsolatedKeys, Scan) } // Scan scans the key range specified by start key through end key @@ -29,18 +29,18 @@ func init() { // stores the number of scan results remaining for this batch // (MaxInt64 for no limit). func Scan( - ctx context.Context, reader storage.Reader, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, reader storage.Reader, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.ScanRequest) + args := cArgs.Args.(*kvpb.ScanRequest) h := cArgs.Header - reply := resp.(*roachpb.ScanResponse) + reply := resp.(*kvpb.ScanResponse) var res result.Result var scanRes storage.MVCCScanResult var err error opts := storage.MVCCScanOptions{ - Inconsistent: h.ReadConsistency != roachpb.CONSISTENT, + Inconsistent: h.ReadConsistency != kvpb.CONSISTENT, SkipLocked: h.WaitPolicy == lock.WaitPolicy_SkipLocked, Txn: h.Txn, Uncertainty: cArgs.Uncertainty, @@ -57,14 +57,14 @@ func Scan( } switch args.ScanFormat { - case roachpb.BATCH_RESPONSE: + case kvpb.BATCH_RESPONSE: scanRes, err = storage.MVCCScanToBytes( ctx, reader, args.Key, args.EndKey, h.Timestamp, opts) if err != nil { return result.Result{}, err } reply.BatchResponses = scanRes.KVData - case roachpb.COL_BATCH_RESPONSE: + case kvpb.COL_BATCH_RESPONSE: scanRes, err = storage.MVCCScanToCols( ctx, reader, cArgs.Header.IndexFetchSpec, args.Key, args.EndKey, h.Timestamp, opts, cArgs.EvalCtx.ClusterSettings(), @@ -73,7 +73,7 @@ func Scan( return result.Result{}, err } reply.BatchResponses = scanRes.KVData - case roachpb.KEY_VALUES: + case kvpb.KEY_VALUES: scanRes, err = storage.MVCCScan( ctx, reader, args.Key, args.EndKey, h.Timestamp, opts) if err != nil { @@ -93,7 +93,7 @@ func Scan( reply.ResumeNextBytes = scanRes.ResumeNextBytes } - if h.ReadConsistency == roachpb.READ_UNCOMMITTED { + if h.ReadConsistency == kvpb.READ_UNCOMMITTED { // NOTE: MVCCScan doesn't use a Prefix iterator, so we don't want to use // one in CollectIntentRows either so that we're guaranteed to use the // same cached iterator and observe a consistent snapshot of the engine. diff --git a/pkg/kv/kvserver/batcheval/cmd_scan_test.go b/pkg/kv/kvserver/batcheval/cmd_scan_test.go index 0031d4aed50a..eb3c594fe8bd 100644 --- a/pkg/kv/kvserver/batcheval/cmd_scan_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_scan_test.go @@ -16,6 +16,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog" @@ -60,7 +61,7 @@ func TestScanReverseScanTargetBytes(t *testing.T) { } else if tb == tbMid { expN = 1 } - for _, sf := range []roachpb.ScanFormat{roachpb.KEY_VALUES, roachpb.BATCH_RESPONSE} { + for _, sf := range []kvpb.ScanFormat{kvpb.KEY_VALUES, kvpb.BATCH_RESPONSE} { t.Run(fmt.Sprintf("format=%s", sf), func(t *testing.T) { testScanReverseScanInner(t, tb, sf, reverse, allowEmpty, expN) }) @@ -73,7 +74,7 @@ func TestScanReverseScanTargetBytes(t *testing.T) { } func testScanReverseScanInner( - t *testing.T, tb int64, sf roachpb.ScanFormat, reverse bool, allowEmpty bool, expN int, + t *testing.T, tb int64, sf kvpb.ScanFormat, reverse bool, allowEmpty bool, expN int, ) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -91,22 +92,22 @@ func testScanReverseScanInner( require.NoError(t, err) } - var req roachpb.Request - var resp roachpb.Response + var req kvpb.Request + var resp kvpb.Response if !reverse { - req = &roachpb.ScanRequest{ScanFormat: sf} - resp = &roachpb.ScanResponse{} + req = &kvpb.ScanRequest{ScanFormat: sf} + resp = &kvpb.ScanResponse{} } else { - req = &roachpb.ReverseScanRequest{ScanFormat: sf} - resp = &roachpb.ReverseScanResponse{} + req = &kvpb.ReverseScanRequest{ScanFormat: sf} + resp = &kvpb.ReverseScanResponse{} } - req.SetHeader(roachpb.RequestHeader{Key: k1, EndKey: roachpb.KeyMax}) + req.SetHeader(kvpb.RequestHeader{Key: k1, EndKey: roachpb.KeyMax}) settings := cluster.MakeTestingClusterSettings() cArgs := CommandArgs{ Args: req, - Header: roachpb.Header{ + Header: kvpb.Header{ Timestamp: ts, TargetBytes: tb, AllowEmpty: allowEmpty, @@ -157,9 +158,9 @@ func testScanReverseScanInner( var rows []roachpb.KeyValue if !reverse { - rows = resp.(*roachpb.ScanResponse).Rows + rows = resp.(*kvpb.ScanResponse).Rows } else { - rows = resp.(*roachpb.ReverseScanResponse).Rows + rows = resp.(*kvpb.ReverseScanResponse).Rows } if rows != nil { @@ -191,23 +192,23 @@ func TestScanReverseScanWholeRows(t *testing.T) { } testutils.RunTrueAndFalse(t, "reverse", func(t *testing.T, reverse bool) { - var req roachpb.Request - var resp roachpb.Response + var req kvpb.Request + var resp kvpb.Response if !reverse { - req = &roachpb.ScanRequest{} - resp = &roachpb.ScanResponse{} + req = &kvpb.ScanRequest{} + resp = &kvpb.ScanResponse{} } else { - req = &roachpb.ReverseScanRequest{} - resp = &roachpb.ReverseScanResponse{} + req = &kvpb.ReverseScanRequest{} + resp = &kvpb.ReverseScanResponse{} } - req.SetHeader(roachpb.RequestHeader{Key: rowKeys[0], EndKey: roachpb.KeyMax}) + req.SetHeader(kvpb.RequestHeader{Key: rowKeys[0], EndKey: roachpb.KeyMax}) // Scan with limit of 5 keys. This should only return the first row (3 keys), // since they second row would yield 6 keys total. cArgs := CommandArgs{ EvalCtx: (&MockEvalCtx{ClusterSettings: cluster.MakeTestingClusterSettings()}).EvalContext(), Args: req, - Header: roachpb.Header{ + Header: kvpb.Header{ Timestamp: ts, MaxSpanRequestKeys: 5, WholeRowsOfSize: 3, diff --git a/pkg/kv/kvserver/batcheval/cmd_subsume.go b/pkg/kv/kvserver/batcheval/cmd_subsume.go index 093249b10706..05dcc8ddb02e 100644 --- a/pkg/kv/kvserver/batcheval/cmd_subsume.go +++ b/pkg/kv/kvserver/batcheval/cmd_subsume.go @@ -16,23 +16,23 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/errors" ) func init() { - RegisterReadWriteCommand(roachpb.Subsume, declareKeysSubsume, Subsume) + RegisterReadWriteCommand(kvpb.Subsume, declareKeysSubsume, Subsume) } func declareKeysSubsume( _ ImmutableRangeState, - _ *roachpb.Header, - _ roachpb.Request, + _ *kvpb.Header, + _ kvpb.Request, latchSpans, _ *spanset.SpanSet, _ time.Duration, ) { @@ -73,10 +73,10 @@ func declareKeysSubsume( // The period of time after intents have been placed but before the merge // transaction is complete is called the merge's "critical phase". func Subsume( - ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.SubsumeRequest) - reply := resp.(*roachpb.SubsumeResponse) + args := cArgs.Args.(*kvpb.SubsumeRequest) + reply := resp.(*kvpb.SubsumeResponse) // Verify that the Subsume request was sent to the correct range and that // the range's bounds have not changed during the merge transaction. diff --git a/pkg/kv/kvserver/batcheval/cmd_truncate_log.go b/pkg/kv/kvserver/batcheval/cmd_truncate_log.go index 6c17990f62c5..0777e6a54b19 100644 --- a/pkg/kv/kvserver/batcheval/cmd_truncate_log.go +++ b/pkg/kv/kvserver/batcheval/cmd_truncate_log.go @@ -15,6 +15,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" @@ -25,13 +26,13 @@ import ( ) func init() { - RegisterReadWriteCommand(roachpb.TruncateLog, declareKeysTruncateLog, TruncateLog) + RegisterReadWriteCommand(kvpb.TruncateLog, declareKeysTruncateLog, TruncateLog) } func declareKeysTruncateLog( rs ImmutableRangeState, - _ *roachpb.Header, - _ roachpb.Request, + _ *kvpb.Header, + _ kvpb.Request, latchSpans, _ *spanset.SpanSet, _ time.Duration, ) { @@ -43,9 +44,9 @@ func declareKeysTruncateLog( // has already been truncated has no effect. If this range is not the one // specified within the request body, the request will also be ignored. func TruncateLog( - ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp roachpb.Response, + ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.TruncateLogRequest) + args := cArgs.Args.(*kvpb.TruncateLogRequest) // After a merge, it's possible that this request was sent to the wrong // range based on the start key. This will cancel the request if this is not diff --git a/pkg/kv/kvserver/batcheval/cmd_truncate_log_test.go b/pkg/kv/kvserver/batcheval/cmd_truncate_log_test.go index 481d9af8c716..89bee9807284 100644 --- a/pkg/kv/kvserver/batcheval/cmd_truncate_log_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_truncate_log_test.go @@ -15,6 +15,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage" @@ -82,7 +83,7 @@ func TestTruncateLog(t *testing.T) { putTruncatedState(t, eng, rangeID, truncState) // Send a truncation request. - req := roachpb.TruncateLogRequest{ + req := kvpb.TruncateLogRequest{ RangeID: rangeID, Index: firstIndex + 7, } @@ -90,7 +91,7 @@ func TestTruncateLog(t *testing.T) { EvalCtx: evalCtx.EvalContext(), Args: &req, } - resp := &roachpb.TruncateLogResponse{} + resp := &kvpb.TruncateLogResponse{} res, err := TruncateLog(ctx, eng, cArgs, resp) if err != nil { t.Fatal(err) diff --git a/pkg/kv/kvserver/batcheval/command.go b/pkg/kv/kvserver/batcheval/command.go index 180b809e4380..205e83d167ea 100644 --- a/pkg/kv/kvserver/batcheval/command.go +++ b/pkg/kv/kvserver/batcheval/command.go @@ -14,6 +14,7 @@ import ( "context" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -26,8 +27,8 @@ import ( // isolation from conflicting transactions to the lockSpans set. type DeclareKeysFunc func( rs ImmutableRangeState, - header *roachpb.Header, - request roachpb.Request, + header *kvpb.Header, + request kvpb.Request, latchSpans, lockSpans *spanset.SpanSet, maxOffset time.Duration, ) @@ -65,22 +66,22 @@ type Command struct { // should treat the provided request as immutable. // // Only one of these is ever set at a time. - EvalRW func(context.Context, storage.ReadWriter, CommandArgs, roachpb.Response) (result.Result, error) - EvalRO func(context.Context, storage.Reader, CommandArgs, roachpb.Response) (result.Result, error) + EvalRW func(context.Context, storage.ReadWriter, CommandArgs, kvpb.Response) (result.Result, error) + EvalRO func(context.Context, storage.Reader, CommandArgs, kvpb.Response) (result.Result, error) } func (c Command) isEmpty() bool { return c.EvalRW == nil && c.EvalRO == nil } -var cmds [roachpb.NumMethods]Command +var cmds [kvpb.NumMethods]Command // RegisterReadWriteCommand makes a read-write command available for execution. // It must only be called before any evaluation takes place. func RegisterReadWriteCommand( - method roachpb.Method, + method kvpb.Method, declare DeclareKeysFunc, - impl func(context.Context, storage.ReadWriter, CommandArgs, roachpb.Response) (result.Result, error), + impl func(context.Context, storage.ReadWriter, CommandArgs, kvpb.Response) (result.Result, error), ) { register(method, Command{ DeclareKeys: declare, @@ -91,9 +92,9 @@ func RegisterReadWriteCommand( // RegisterReadOnlyCommand makes a read-only command available for execution. It // must only be called before any evaluation takes place. func RegisterReadOnlyCommand( - method roachpb.Method, + method kvpb.Method, declare DeclareKeysFunc, - impl func(context.Context, storage.Reader, CommandArgs, roachpb.Response) (result.Result, error), + impl func(context.Context, storage.Reader, CommandArgs, kvpb.Response) (result.Result, error), ) { register(method, Command{ DeclareKeys: declare, @@ -101,7 +102,7 @@ func RegisterReadOnlyCommand( }) } -func register(method roachpb.Method, command Command) { +func register(method kvpb.Method, command Command) { if !cmds[method].isEmpty() { log.Fatalf(context.TODO(), "cannot overwrite previously registered method %v", method) } @@ -110,13 +111,13 @@ func register(method roachpb.Method, command Command) { // UnregisterCommand is provided for testing and allows removing a command. // It is a no-op if the command is not registered. -func UnregisterCommand(method roachpb.Method) { +func UnregisterCommand(method kvpb.Method) { cmds[method] = Command{} } // LookupCommand returns the command for the given method, with the boolean // indicating success or failure. -func LookupCommand(method roachpb.Method) (Command, bool) { +func LookupCommand(method kvpb.Method) (Command, bool) { if int(method) >= len(cmds) { return Command{}, false } diff --git a/pkg/kv/kvserver/batcheval/declare.go b/pkg/kv/kvserver/batcheval/declare.go index 2e230d3b20a5..5b8271f9b9af 100644 --- a/pkg/kv/kvserver/batcheval/declare.go +++ b/pkg/kv/kvserver/batcheval/declare.go @@ -15,6 +15,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" @@ -27,13 +28,13 @@ import ( // DefaultDeclareKeys is the default implementation of Command.DeclareKeys. func DefaultDeclareKeys( _ ImmutableRangeState, - header *roachpb.Header, - req roachpb.Request, + header *kvpb.Header, + req kvpb.Request, latchSpans, _ *spanset.SpanSet, _ time.Duration, ) { access := spanset.SpanReadWrite - if roachpb.IsReadOnly(req) && !roachpb.IsLocking(req) { + if kvpb.IsReadOnly(req) && !kvpb.IsLocking(req) { access = spanset.SpanReadOnly } latchSpans.AddMVCC(access, req.Header().Span(), header.Timestamp) @@ -46,14 +47,14 @@ func DefaultDeclareKeys( // when it evaluated. func DefaultDeclareIsolatedKeys( _ ImmutableRangeState, - header *roachpb.Header, - req roachpb.Request, + header *kvpb.Header, + req kvpb.Request, latchSpans, lockSpans *spanset.SpanSet, maxOffset time.Duration, ) { access := spanset.SpanReadWrite timestamp := header.Timestamp - if roachpb.IsReadOnly(req) && !roachpb.IsLocking(req) { + if kvpb.IsReadOnly(req) && !kvpb.IsLocking(req) { access = spanset.SpanReadOnly // For non-locking reads, acquire read latches all the way up to the @@ -86,9 +87,7 @@ func DefaultDeclareIsolatedKeys( // DeclareKeysForBatch adds all keys that the batch with the provided header // touches to the given SpanSet. This does not include keys touched during the // processing of the batch's individual commands. -func DeclareKeysForBatch( - rs ImmutableRangeState, header *roachpb.Header, latchSpans *spanset.SpanSet, -) { +func DeclareKeysForBatch(rs ImmutableRangeState, header *kvpb.Header, latchSpans *spanset.SpanSet) { if header.Txn != nil { header.Txn.AssertInitialized(context.TODO()) latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{ @@ -117,8 +116,8 @@ func declareAllKeys(latchSpans *spanset.SpanSet) { // constraints). type CommandArgs struct { EvalCtx EvalContext - Header roachpb.Header - Args roachpb.Request + Header kvpb.Header + Args kvpb.Request Now hlc.ClockTimestamp // *Stats should be mutated to reflect any writes made by the command. Stats *enginepb.MVCCStats diff --git a/pkg/kv/kvserver/batcheval/declare_test.go b/pkg/kv/kvserver/batcheval/declare_test.go index 3b0a65cbd72f..0f2b5a1c24b1 100644 --- a/pkg/kv/kvserver/batcheval/declare_test.go +++ b/pkg/kv/kvserver/batcheval/declare_test.go @@ -13,6 +13,7 @@ package batcheval import ( "testing" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" @@ -35,8 +36,8 @@ func TestRequestsSerializeWithAllKeys(t *testing.T) { if command.isEmpty() { continue } - method := roachpb.Method(i) - if method == roachpb.Probe { + method := kvpb.Method(i) + if method == kvpb.Probe { // Probe is special since it's a no-op round-trip through the replication // layer. It does not declare any keys. continue @@ -59,16 +60,16 @@ func TestRequestsSerializeWithAllKeys(t *testing.T) { }, Name: "test txn", } - header := roachpb.Header{Txn: testTxn} - otherRequest := roachpb.CreateRequest(method) - if queryTxnReq, ok := otherRequest.(*roachpb.QueryTxnRequest); ok { + header := kvpb.Header{Txn: testTxn} + otherRequest := kvpb.CreateRequest(method) + if queryTxnReq, ok := otherRequest.(*kvpb.QueryTxnRequest); ok { // QueryTxnRequest declares read-only access over the txn record of the txn // it is supposed to query and not the txn that sent it. We fill this Txn // field in here to prevent it from being nil and leading to the txn key // falling outside our test range's keyspace. queryTxnReq.Txn = testTxn.TxnMeta } - otherRequest.SetHeader(roachpb.RequestHeader{ + otherRequest.SetHeader(kvpb.RequestHeader{ Key: startKey, EndKey: endKey, Sequence: 0, diff --git a/pkg/kv/kvserver/batcheval/eval_context.go b/pkg/kv/kvserver/batcheval/eval_context.go index 5c9aabacd3cb..4522de2d38ad 100644 --- a/pkg/kv/kvserver/batcheval/eval_context.go +++ b/pkg/kv/kvserver/batcheval/eval_context.go @@ -15,6 +15,7 @@ import ( "fmt" "math" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/abortspan" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" @@ -73,7 +74,7 @@ type EvalContext interface { // for details about its arguments, return values, and preconditions. CanCreateTxnRecord( ctx context.Context, txnID uuid.UUID, txnKey []byte, txnMinTS hlc.Timestamp, - ) (ok bool, reason roachpb.TransactionAbortedReason) + ) (ok bool, reason kvpb.TransactionAbortedReason) // MinTxnCommitTS determines the minimum timestamp at which a transaction with // the provided ID and key can commit. See Replica.MinTxnCommitTS for details @@ -172,7 +173,7 @@ type MockEvalCtx struct { AbortSpan *abortspan.AbortSpan GCThreshold hlc.Timestamp Term, FirstIndex uint64 - CanCreateTxnRecordFn func() (bool, roachpb.TransactionAbortedReason) + CanCreateTxnRecordFn func() (bool, kvpb.TransactionAbortedReason) MinTxnCommitTSFn func() hlc.Timestamp Lease roachpb.Lease CurrentReadSummary rspb.ReadSummary @@ -253,7 +254,7 @@ func (m *mockEvalCtxImpl) GetMaxSplitCPU(context.Context) (float64, bool) { } func (m *mockEvalCtxImpl) CanCreateTxnRecord( context.Context, uuid.UUID, []byte, hlc.Timestamp, -) (bool, roachpb.TransactionAbortedReason) { +) (bool, kvpb.TransactionAbortedReason) { if m.CanCreateTxnRecordFn == nil { return true, 0 } diff --git a/pkg/kv/kvserver/batcheval/intent.go b/pkg/kv/kvserver/batcheval/intent.go index 523904934c65..5ab3a9ce7850 100644 --- a/pkg/kv/kvserver/batcheval/intent.go +++ b/pkg/kv/kvserver/batcheval/intent.go @@ -13,6 +13,7 @@ package batcheval import ( "context" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -54,8 +55,8 @@ func CollectIntentRows( for i := range intents { kv, err := readProvisionalVal(ctx, reader, usePrefixIter, &intents[i]) if err != nil { - if errors.HasType(err, (*roachpb.WriteIntentError)(nil)) || - errors.HasType(err, (*roachpb.ReadWithinUncertaintyIntervalError)(nil)) { + if errors.HasType(err, (*kvpb.WriteIntentError)(nil)) || + errors.HasType(err, (*kvpb.ReadWithinUncertaintyIntervalError)(nil)) { log.Fatalf(ctx, "unexpected %T in CollectIntentRows: %+v", err, err) } return nil, err @@ -108,24 +109,24 @@ func readProvisionalVal( func acquireUnreplicatedLocksOnKeys( res *result.Result, txn *roachpb.Transaction, - scanFmt roachpb.ScanFormat, + scanFmt kvpb.ScanFormat, scanRes *storage.MVCCScanResult, ) error { res.Local.AcquiredLocks = make([]roachpb.LockAcquisition, scanRes.NumKeys) switch scanFmt { - case roachpb.BATCH_RESPONSE: + case kvpb.BATCH_RESPONSE: var i int return storage.MVCCScanDecodeKeyValues(scanRes.KVData, func(key storage.MVCCKey, _ []byte) error { res.Local.AcquiredLocks[i] = roachpb.MakeLockAcquisition(txn, copyKey(key.Key), lock.Unreplicated) i++ return nil }) - case roachpb.KEY_VALUES: + case kvpb.KEY_VALUES: for i, row := range scanRes.KVs { res.Local.AcquiredLocks[i] = roachpb.MakeLockAcquisition(txn, copyKey(row.Key), lock.Unreplicated) } return nil - case roachpb.COL_BATCH_RESPONSE: + case kvpb.COL_BATCH_RESPONSE: return errors.AssertionFailedf("unexpectedly acquiring unreplicated locks with COL_BATCH_RESPONSE scan format") default: panic("unexpected scanFormat") diff --git a/pkg/kv/kvserver/batcheval/intent_test.go b/pkg/kv/kvserver/batcheval/intent_test.go index 561940e783ef..6dd2c9128237 100644 --- a/pkg/kv/kvserver/batcheval/intent_test.go +++ b/pkg/kv/kvserver/batcheval/intent_test.go @@ -14,6 +14,7 @@ import ( "context" "testing" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage" @@ -53,9 +54,9 @@ func TestCollectIntentsUsesSameIterator(t *testing.T) { ctx := context.Background() key := roachpb.Key("key") ts := hlc.Timestamp{WallTime: 123} - header := roachpb.Header{ + header := kvpb.Header{ Timestamp: ts, - ReadConsistency: roachpb.READ_UNCOMMITTED, + ReadConsistency: kvpb.READ_UNCOMMITTED, } evalCtx := (&MockEvalCtx{ClusterSettings: cluster.MakeTestingClusterSettings()}).EvalContext() @@ -68,10 +69,10 @@ func TestCollectIntentsUsesSameIterator(t *testing.T) { { name: "get", run: func(t *testing.T, db storage.ReadWriter) ([]roachpb.KeyValue, error) { - req := &roachpb.GetRequest{ - RequestHeader: roachpb.RequestHeader{Key: key}, + req := &kvpb.GetRequest{ + RequestHeader: kvpb.RequestHeader{Key: key}, } - var resp roachpb.GetResponse + var resp kvpb.GetResponse if _, err := Get(ctx, db, CommandArgs{Args: req, Header: header, EvalCtx: evalCtx}, &resp); err != nil { return nil, err } @@ -86,10 +87,10 @@ func TestCollectIntentsUsesSameIterator(t *testing.T) { { name: "scan", run: func(t *testing.T, db storage.ReadWriter) ([]roachpb.KeyValue, error) { - req := &roachpb.ScanRequest{ - RequestHeader: roachpb.RequestHeader{Key: key, EndKey: key.Next()}, + req := &kvpb.ScanRequest{ + RequestHeader: kvpb.RequestHeader{Key: key, EndKey: key.Next()}, } - var resp roachpb.ScanResponse + var resp kvpb.ScanResponse if _, err := Scan(ctx, db, CommandArgs{Args: req, Header: header, EvalCtx: evalCtx}, &resp); err != nil { return nil, err } @@ -101,10 +102,10 @@ func TestCollectIntentsUsesSameIterator(t *testing.T) { { name: "reverse scan", run: func(t *testing.T, db storage.ReadWriter) ([]roachpb.KeyValue, error) { - req := &roachpb.ReverseScanRequest{ - RequestHeader: roachpb.RequestHeader{Key: key, EndKey: key.Next()}, + req := &kvpb.ReverseScanRequest{ + RequestHeader: kvpb.RequestHeader{Key: key, EndKey: key.Next()}, } - var resp roachpb.ReverseScanResponse + var resp kvpb.ReverseScanResponse if _, err := ReverseScan(ctx, db, CommandArgs{Args: req, Header: header, EvalCtx: evalCtx}, &resp); err != nil { return nil, err } diff --git a/pkg/kv/kvserver/batcheval/ranges_test.go b/pkg/kv/kvserver/batcheval/ranges_test.go index f74c97a75985..0d1ceac33d5f 100644 --- a/pkg/kv/kvserver/batcheval/ranges_test.go +++ b/pkg/kv/kvserver/batcheval/ranges_test.go @@ -13,6 +13,7 @@ package batcheval import ( "testing" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/stretchr/testify/require" @@ -21,8 +22,8 @@ import ( func TestMergeGCRangeBoundaries(t *testing.T) { defer leaktest.AfterTest(t)() - gcr := func(start, end roachpb.Key) roachpb.GCRequest_GCRangeKey { - return roachpb.GCRequest_GCRangeKey{ + gcr := func(start, end roachpb.Key) kvpb.GCRequest_GCRangeKey { + return kvpb.GCRequest_GCRangeKey{ StartKey: start, EndKey: end, } @@ -49,21 +50,21 @@ func TestMergeGCRangeBoundaries(t *testing.T) { name string rangeStart roachpb.Key rangeEnd roachpb.Key - rangeKeys []roachpb.GCRequest_GCRangeKey + rangeKeys []kvpb.GCRequest_GCRangeKey spans []roachpb.Span }{ { name: "empty", rangeStart: key("a"), rangeEnd: key("b"), - rangeKeys: []roachpb.GCRequest_GCRangeKey{}, + rangeKeys: []kvpb.GCRequest_GCRangeKey{}, spans: nil, }, { name: "full range", rangeStart: key("a"), rangeEnd: key("b"), - rangeKeys: []roachpb.GCRequest_GCRangeKey{ + rangeKeys: []kvpb.GCRequest_GCRangeKey{ gcr(key("a"), key("b")), }, spans: []roachpb.Span{ @@ -74,7 +75,7 @@ func TestMergeGCRangeBoundaries(t *testing.T) { name: "sub range", rangeStart: key("a"), rangeEnd: key("z"), - rangeKeys: []roachpb.GCRequest_GCRangeKey{ + rangeKeys: []kvpb.GCRequest_GCRangeKey{ gcr(key("c"), key("d")), }, spans: []roachpb.Span{ @@ -85,7 +86,7 @@ func TestMergeGCRangeBoundaries(t *testing.T) { name: "non adjacent", rangeStart: key("a"), rangeEnd: key("z"), - rangeKeys: []roachpb.GCRequest_GCRangeKey{ + rangeKeys: []kvpb.GCRequest_GCRangeKey{ gcr(key("c"), key("d")), gcr(key("e"), key("f")), }, @@ -98,7 +99,7 @@ func TestMergeGCRangeBoundaries(t *testing.T) { name: "merge adjacent", rangeStart: key("a"), rangeEnd: key("z"), - rangeKeys: []roachpb.GCRequest_GCRangeKey{ + rangeKeys: []kvpb.GCRequest_GCRangeKey{ gcr(key("a"), key("b")), gcr(key("b"), key("c")), gcr(key("c"), key("d")), diff --git a/pkg/kv/kvserver/batcheval/result/BUILD.bazel b/pkg/kv/kvserver/batcheval/result/BUILD.bazel index 90c3e574f2a8..0a27e7b5f795 100644 --- a/pkg/kv/kvserver/batcheval/result/BUILD.bazel +++ b/pkg/kv/kvserver/batcheval/result/BUILD.bazel @@ -11,6 +11,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result", visibility = ["//visibility:public"], deps = [ + "//pkg/kv/kvpb", "//pkg/kv/kvserver/concurrency/lock", "//pkg/kv/kvserver/kvserverpb", "//pkg/roachpb", diff --git a/pkg/kv/kvserver/batcheval/result/result.go b/pkg/kv/kvserver/batcheval/result/result.go index e4b5a6571f59..6a2136b03376 100644 --- a/pkg/kv/kvserver/batcheval/result/result.go +++ b/pkg/kv/kvserver/batcheval/result/result.go @@ -14,6 +14,7 @@ import ( "context" "fmt" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -26,7 +27,7 @@ import ( // the proposing node may die before the local results are processed, // so any side effects here are only best-effort. type LocalResult struct { - Reply *roachpb.BatchResponse + Reply *kvpb.BatchResponse // EncounteredIntents stores any intents from other transactions that the // request encountered but did not conflict with. They should be handed off diff --git a/pkg/kv/kvserver/batcheval/transaction.go b/pkg/kv/kvserver/batcheval/transaction.go index b300843165c0..d349e92ef5d5 100644 --- a/pkg/kv/kvserver/batcheval/transaction.go +++ b/pkg/kv/kvserver/batcheval/transaction.go @@ -14,6 +14,7 @@ import ( "bytes" "context" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" @@ -29,7 +30,7 @@ var ErrTransactionUnsupported = errors.AssertionFailedf("not supported within a // VerifyTransaction runs sanity checks verifying that the transaction in the // header and the request are compatible. func VerifyTransaction( - h roachpb.Header, args roachpb.Request, permittedStatuses ...roachpb.TransactionStatus, + h kvpb.Header, args kvpb.Request, permittedStatuses ...roachpb.TransactionStatus, ) error { if h.Txn == nil { return errors.AssertionFailedf("no transaction specified to %s", args.Method()) @@ -45,11 +46,11 @@ func VerifyTransaction( } } if !statusPermitted { - reason := roachpb.TransactionStatusError_REASON_UNKNOWN + reason := kvpb.TransactionStatusError_REASON_UNKNOWN if h.Txn.Status == roachpb.COMMITTED { - reason = roachpb.TransactionStatusError_REASON_TXN_COMMITTED + reason = kvpb.TransactionStatusError_REASON_TXN_COMMITTED } - return roachpb.NewTransactionStatusError(reason, + return kvpb.NewTransactionStatusError(reason, redact.Sprintf("cannot perform %s with txn status %v", args.Method(), h.Txn.Status)) } return nil @@ -126,7 +127,7 @@ func CanCreateTxnRecord(ctx context.Context, rec EvalContext, txn *roachpb.Trans ok, reason := rec.CanCreateTxnRecord(ctx, txn.ID, txn.Key, txn.MinTimestamp) if !ok { log.VEventf(ctx, 2, "txn tombstone present; transaction has been aborted") - return roachpb.NewTransactionAbortedError(reason) + return kvpb.NewTransactionAbortedError(reason) } return nil } diff --git a/pkg/kv/kvserver/batcheval/transaction_test.go b/pkg/kv/kvserver/batcheval/transaction_test.go index be32ce54636f..b0c32831acf8 100644 --- a/pkg/kv/kvserver/batcheval/transaction_test.go +++ b/pkg/kv/kvserver/batcheval/transaction_test.go @@ -15,6 +15,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/abortspan" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" @@ -102,15 +103,15 @@ func TestUpdateAbortSpan(t *testing.T) { // Request helpers. endTxn := func(b storage.ReadWriter, rec EvalContext, ms *enginepb.MVCCStats, commit bool, poison bool) error { - req := roachpb.EndTxnRequest{ - RequestHeader: roachpb.RequestHeader{Key: txnKey}, + req := kvpb.EndTxnRequest{ + RequestHeader: kvpb.RequestHeader{Key: txnKey}, Commit: commit, Poison: poison, LockSpans: []roachpb.Span{{Key: intentKey}}, } args := CommandArgs{ EvalCtx: rec, - Header: roachpb.Header{ + Header: kvpb.Header{ Timestamp: txn.ReadTimestamp, Txn: &txn, }, @@ -118,15 +119,15 @@ func TestUpdateAbortSpan(t *testing.T) { Stats: ms, } - var resp roachpb.EndTxnResponse + var resp kvpb.EndTxnResponse _, err := EndTxn(ctx, b, args, &resp) return err } resolveIntent := func( b storage.ReadWriter, rec EvalContext, ms *enginepb.MVCCStats, status roachpb.TransactionStatus, poison bool, ) error { - req := roachpb.ResolveIntentRequest{ - RequestHeader: roachpb.RequestHeader{Key: intentKey}, + req := kvpb.ResolveIntentRequest{ + RequestHeader: kvpb.RequestHeader{Key: intentKey}, IntentTxn: txn.TxnMeta, Status: status, Poison: poison, @@ -137,15 +138,15 @@ func TestUpdateAbortSpan(t *testing.T) { Stats: ms, } - var resp roachpb.ResolveIntentResponse + var resp kvpb.ResolveIntentResponse _, err := ResolveIntent(ctx, b, args, &resp) return err } resolveIntentRange := func( b storage.ReadWriter, rec EvalContext, ms *enginepb.MVCCStats, status roachpb.TransactionStatus, poison bool, ) error { - req := roachpb.ResolveIntentRangeRequest{ - RequestHeader: roachpb.RequestHeader{Key: startKey, EndKey: endKey}, + req := kvpb.ResolveIntentRangeRequest{ + RequestHeader: kvpb.RequestHeader{Key: startKey, EndKey: endKey}, IntentTxn: txn.TxnMeta, Status: status, Poison: poison, @@ -156,7 +157,7 @@ func TestUpdateAbortSpan(t *testing.T) { Stats: ms, } - var resp roachpb.ResolveIntentRangeResponse + var resp kvpb.ResolveIntentRangeResponse _, err := ResolveIntentRange(ctx, b, args, &resp) return err } diff --git a/pkg/kv/kvserver/client_atomic_membership_change_test.go b/pkg/kv/kvserver/client_atomic_membership_change_test.go index c331f5b3f310..4b6568bfafe6 100644 --- a/pkg/kv/kvserver/client_atomic_membership_change_test.go +++ b/pkg/kv/kvserver/client_atomic_membership_change_test.go @@ -16,6 +16,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -56,7 +57,7 @@ func TestAtomicReplicationChange(t *testing.T) { desc, err := tc.AddVoters(k, tc.Target(2)) require.NoError(t, err) - runChange := func(expDesc roachpb.RangeDescriptor, chgs []roachpb.ReplicationChange) roachpb.RangeDescriptor { + runChange := func(expDesc roachpb.RangeDescriptor, chgs []kvpb.ReplicationChange) roachpb.RangeDescriptor { t.Helper() desc, err := tc.Servers[0].DB().AdminChangeReplicas(ctx, k, expDesc, chgs) require.NoError(t, err) @@ -101,7 +102,7 @@ func TestAtomicReplicationChange(t *testing.T) { } // Run a fairly general change. - desc = runChange(desc, []roachpb.ReplicationChange{ + desc = runChange(desc, []kvpb.ReplicationChange{ {ChangeType: roachpb.ADD_VOTER, Target: tc.Target(3)}, {ChangeType: roachpb.ADD_VOTER, Target: tc.Target(5)}, {ChangeType: roachpb.REMOVE_VOTER, Target: tc.Target(2)}, @@ -115,7 +116,7 @@ func TestAtomicReplicationChange(t *testing.T) { require.NoError(t, tc.TransferRangeLease(desc, tc.Target(4))) // Rebalance back down all the way. - desc = runChange(desc, []roachpb.ReplicationChange{ + desc = runChange(desc, []kvpb.ReplicationChange{ {ChangeType: roachpb.REMOVE_VOTER, Target: tc.Target(0)}, {ChangeType: roachpb.REMOVE_VOTER, Target: tc.Target(1)}, {ChangeType: roachpb.REMOVE_VOTER, Target: tc.Target(3)}, diff --git a/pkg/kv/kvserver/client_lease_test.go b/pkg/kv/kvserver/client_lease_test.go index 90606f6cb6cc..8d35dc468b3d 100644 --- a/pkg/kv/kvserver/client_lease_test.go +++ b/pkg/kv/kvserver/client_lease_test.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" @@ -199,7 +200,7 @@ func TestCannotTransferLeaseToVoterDemoting(t *testing.T) { <-ch } } - knobs.Store.(*kvserver.StoreTestingKnobs).TestingProposalFilter = func(args kvserverbase.ProposalFilterArgs) *roachpb.Error { + knobs.Store.(*kvserver.StoreTestingKnobs).TestingProposalFilter = func(args kvserverbase.ProposalFilterArgs) *kvpb.Error { blockIfShould(args) return nil } @@ -233,7 +234,7 @@ func TestCannotTransferLeaseToVoterDemoting(t *testing.T) { go func() { defer wg.Done() _, err = tc.Server(0).DB().AdminChangeReplicas(ctx, - scratchStartKey, desc, []roachpb.ReplicationChange{ + scratchStartKey, desc, []kvpb.ReplicationChange{ {ChangeType: roachpb.REMOVE_VOTER, Target: tc.Target(2)}, }) require.NoError(t, err) @@ -299,7 +300,7 @@ func TestTransferLeaseToVoterDemotingFails(t *testing.T) { <-ch } } - knobs.Store.(*kvserver.StoreTestingKnobs).TestingProposalFilter = func(args kvserverbase.ProposalFilterArgs) *roachpb.Error { + knobs.Store.(*kvserver.StoreTestingKnobs).TestingProposalFilter = func(args kvserverbase.ProposalFilterArgs) *kvpb.Error { blockIfShould(args) return nil } @@ -333,7 +334,7 @@ func TestTransferLeaseToVoterDemotingFails(t *testing.T) { go func() { defer wg.Done() _, err = tc.Server(0).DB().AdminChangeReplicas(ctx, - scratchStartKey, desc, []roachpb.ReplicationChange{ + scratchStartKey, desc, []kvpb.ReplicationChange{ {ChangeType: roachpb.REMOVE_VOTER, Target: tc.Target(2)}, {ChangeType: roachpb.ADD_VOTER, Target: tc.Target(3)}, }) @@ -498,12 +499,12 @@ func internalTransferLeaseFailureDuringJointConfig(t *testing.T, isManual bool) args.Req.IsSingleTransferLeaseRequest() } const failureMsg = "injected lease transfer" - knobs.Store.(*kvserver.StoreTestingKnobs).TestingProposalFilter = func(args kvserverbase.ProposalFilterArgs) *roachpb.Error { + knobs.Store.(*kvserver.StoreTestingKnobs).TestingProposalFilter = func(args kvserverbase.ProposalFilterArgs) *kvpb.Error { if shouldFailProposal(args) { // The lease transfer should be configured to bypass safety checks. // See maybeTransferLeaseDuringLeaveJoint for an explanation. require.True(t, args.Req.Requests[0].GetTransferLease().BypassSafetyChecks) - return roachpb.NewErrorf(failureMsg) + return kvpb.NewErrorf(failureMsg) } return nil } @@ -524,7 +525,7 @@ func internalTransferLeaseFailureDuringJointConfig(t *testing.T, isManual bool) atomic.StoreInt64(&scratchRangeID, int64(desc.RangeID)) _, err = tc.Server(0).DB().AdminChangeReplicas(ctx, - scratchStartKey, desc, []roachpb.ReplicationChange{ + scratchStartKey, desc, []kvpb.ReplicationChange{ {ChangeType: roachpb.REMOVE_VOTER, Target: tc.Target(0)}, {ChangeType: roachpb.ADD_VOTER, Target: tc.Target(3)}, }) @@ -631,7 +632,7 @@ func TestStoreLeaseTransferTimestampCacheRead(t *testing.T) { // Read the key at readTS. // NB: don't use SendWrapped because we want access to br.Timestamp. - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Timestamp = readTS ba.Add(getArgs(key)) br, pErr := tc.Servers[0].DistSender().Send(ctx, ba) @@ -649,7 +650,7 @@ func TestStoreLeaseTransferTimestampCacheRead(t *testing.T) { // Attempt to write under the read on the new leaseholder. The batch // should get forwarded to a timestamp after the read. // NB: don't use SendWrapped because we want access to br.Timestamp. - ba = &roachpb.BatchRequest{} + ba = &kvpb.BatchRequest{} ba.Timestamp = readTS ba.Add(incrementArgs(key, 1)) br, pErr = tc.Servers[0].DistSender().Send(ctx, ba) @@ -1336,11 +1337,11 @@ func TestAcquireLeaseTimeout(t *testing.T) { // return the context error. var blockRangeID int32 - maybeBlockLeaseRequest := func(ctx context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + maybeBlockLeaseRequest := func(ctx context.Context, ba *kvpb.BatchRequest) *kvpb.Error { if ba.IsSingleRequestLeaseRequest() && int32(ba.RangeID) == atomic.LoadInt32(&blockRangeID) { t.Logf("blocked lease request for r%d", ba.RangeID) <-ctx.Done() - return roachpb.NewError(ctx.Err()) + return kvpb.NewError(ctx.Err()) } return nil } @@ -1407,10 +1408,10 @@ func TestAcquireLeaseTimeout(t *testing.T) { // Trying to acquire the lease should error with an empty NLHE, since the // range doesn't have quorum. - var nlhe *roachpb.NotLeaseHolderError + var nlhe *kvpb.NotLeaseHolderError _, err = repl.TestingAcquireLease(ctx) require.Error(t, err) - require.IsType(t, &roachpb.NotLeaseHolderError{}, err) // check exact type + require.IsType(t, &kvpb.NotLeaseHolderError{}, err) // check exact type require.ErrorAs(t, err, &nlhe) require.Empty(t, nlhe.Lease) @@ -1437,7 +1438,7 @@ func TestAcquireLeaseTimeout(t *testing.T) { for err := range errC { require.Error(t, err) - require.IsType(t, &roachpb.NotLeaseHolderError{}, err) // check exact type + require.IsType(t, &kvpb.NotLeaseHolderError{}, err) // check exact type require.ErrorAs(t, err, &nlhe) require.Empty(t, nlhe.Lease) } diff --git a/pkg/kv/kvserver/client_merge_test.go b/pkg/kv/kvserver/client_merge_test.go index 744ca637e14f..98314d91330f 100644 --- a/pkg/kv/kvserver/client_merge_test.go +++ b/pkg/kv/kvserver/client_merge_test.go @@ -32,6 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" @@ -66,9 +67,9 @@ import ( "go.etcd.io/raft/v3/raftpb" ) -func adminMergeArgs(key roachpb.Key) *roachpb.AdminMergeRequest { - return &roachpb.AdminMergeRequest{ - RequestHeader: roachpb.RequestHeader{ +func adminMergeArgs(key roachpb.Key) *kvpb.AdminMergeRequest { + return &kvpb.AdminMergeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, }, } @@ -207,7 +208,7 @@ func TestStoreRangeMergeMetadataCleanup(t *testing.T) { // Write some values right of the split key. pArgs = putArgs(scratchKey("ccc"), content) - if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ + if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{ RangeID: rhsDesc.RangeID, }, pArgs); pErr != nil { t.Fatal(pErr) @@ -270,12 +271,12 @@ func mergeWithData(t *testing.T, retries int64) { manualClock := hlc.NewHybridManualClock() var store *kvserver.Store // Maybe inject some retryable errors when the merge transaction commits. - testingRequestFilter := func(_ context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + testingRequestFilter := func(_ context.Context, ba *kvpb.BatchRequest) *kvpb.Error { for _, req := range ba.Requests { if et := req.GetEndTxn(); et != nil && et.InternalCommitTrigger.GetMergeTrigger() != nil { if atomic.AddInt64(&retries, -1) >= 0 { - return roachpb.NewError( - roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE, "filter err")) + return kvpb.NewError( + kvpb.NewTransactionRetryError(kvpb.RETRY_SERIALIZABLE, "filter err")) } } if req.GetSubsume() != nil { @@ -319,7 +320,7 @@ func mergeWithData(t *testing.T, retries int64) { put := func(key roachpb.Key, rangeID roachpb.RangeID, value []byte) { pArgs := putArgs(key, value) - if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ + if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{ RangeID: rangeID, }, pArgs); pErr != nil { t.Fatal(pErr) @@ -329,10 +330,10 @@ func mergeWithData(t *testing.T, retries int64) { verify := func(key roachpb.Key, rangeID roachpb.RangeID, value []byte) { // Confirm the values are there. gArgs := getArgs(key) - if reply, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ + if reply, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{ RangeID: rangeID, }, gArgs); pErr != nil { - } else if replyBytes, err := reply.(*roachpb.GetResponse).Value.GetBytes(); err != nil { + } else if replyBytes, err := reply.(*kvpb.GetResponse).Value.GetBytes(); err != nil { t.Fatal(err) } else if !bytes.Equal(replyBytes, value) { t.Fatalf("actual value %q did not match expected value %q", replyBytes, content) @@ -387,7 +388,7 @@ func mergeWithData(t *testing.T, retries int64) { verify(rhsDesc.StartKey.Next().AsRawKey(), rhsRepl.RangeID, newContent) gArgs := getArgs(lhsDesc.StartKey.Next().AsRawKey()) - if _, pErr := kv.SendWrappedWith(ctx, store, roachpb.Header{ + if _, pErr := kv.SendWrappedWith(ctx, store, kvpb.Header{ RangeID: rhsDesc.RangeID, }, gArgs); !testutils.IsPError( pErr, `was not found on s`, @@ -445,20 +446,20 @@ func mergeCheckingTimestampCaches( // leader-leaseholder state. blockHBAndGCs chan struct{} } - testingRequestFilter := func(_ context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + testingRequestFilter := func(_ context.Context, ba *kvpb.BatchRequest) *kvpb.Error { filterMu.Lock() mergeCommitFilterCopy := filterMu.mergeCommitFilter blockHBAndGCsCopy := filterMu.blockHBAndGCs filterMu.Unlock() for _, req := range ba.Requests { switch v := req.GetInner().(type) { - case *roachpb.EndTxnRequest: + case *kvpb.EndTxnRequest: if v.InternalCommitTrigger.GetMergeTrigger() != nil { if mergeCommitFilterCopy != nil { mergeCommitFilterCopy() } } - case *roachpb.HeartbeatTxnRequest, *roachpb.GCRequest: + case *kvpb.HeartbeatTxnRequest, *kvpb.GCRequest: if blockHBAndGCsCopy != nil { <-blockHBAndGCsCopy } @@ -551,7 +552,7 @@ func mergeCheckingTimestampCaches( // Write a key to the RHS. rhsKey := scratchKey("c") - if _, pErr := kv.SendWrappedWith(ctx, rhsStore, roachpb.Header{ + if _, pErr := kv.SendWrappedWith(ctx, rhsStore, kvpb.Header{ RangeID: rhsDesc.RangeID, }, incrementArgs(rhsKey, 1)); pErr != nil { t.Fatal(pErr) @@ -563,7 +564,7 @@ func mergeCheckingTimestampCaches( } // Simulate a read on the RHS from a node with a newer clock. - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Timestamp = readTS ba.RangeID = rhsDesc.RangeID ba.Add(getArgs(rhsKey)) @@ -582,10 +583,10 @@ func mergeCheckingTimestampCaches( // the timestamp cache to record the abort. pushee := roachpb.MakeTransaction("pushee", rhsKey, roachpb.MinUserPriority, readTS, 0, 0) pusher := roachpb.MakeTransaction("pusher", rhsKey, roachpb.MaxUserPriority, readTS, 0, 0) - ba = &roachpb.BatchRequest{} + ba = &kvpb.BatchRequest{} ba.Timestamp = readTS.Next() ba.RangeID = rhsDesc.RangeID - ba.Add(pushTxnArgs(&pusher, &pushee, roachpb.PUSH_ABORT)) + ba.Add(pushTxnArgs(&pusher, &pushee, kvpb.PUSH_ABORT)) if br, pErr := rhsStore.Send(ctx, ba); pErr != nil { t.Fatal(pErr) } else if txn := br.Responses[0].GetPushTxn().PusheeTxn; txn.Status != roachpb.ABORTED { @@ -677,8 +678,8 @@ func mergeCheckingTimestampCaches( // succeeded on the majority quorum. So we launch async goroutines to // perform the write and the log truncation and only wait for them to // complete after the partition heals. - incChan := make(chan *roachpb.Error, 1) - truncChan := make(chan *roachpb.Error, 1) + incChan := make(chan *kvpb.Error, 1) + truncChan := make(chan *kvpb.Error, 1) snapChan := make(chan kvserver.IncomingSnapshot, 1) filterMu.Lock() @@ -709,7 +710,7 @@ func mergeCheckingTimestampCaches( // be the only replica that does not apply the proposal. go func() { incArgs := incrementArgs(lhsKey, 4) - _, pErr := kv.SendWrappedWith(ctx, lhsStore, roachpb.Header{RangeID: lhsDesc.RangeID}, incArgs) + _, pErr := kv.SendWrappedWith(ctx, lhsStore, kvpb.Header{RangeID: lhsDesc.RangeID}, incArgs) incChan <- pErr }() // NB: the operation won't complete, so peek below Raft and wait for @@ -739,7 +740,7 @@ func mergeCheckingTimestampCaches( go func() { truncArgs := truncateLogArgs(truncIndex, lhsDesc.RangeID) truncArgs.Key = lhsKey - _, pErr := kv.SendWrappedWith(ctx, lhsStore, roachpb.Header{RangeID: lhsDesc.RangeID}, truncArgs) + _, pErr := kv.SendWrappedWith(ctx, lhsStore, kvpb.Header{RangeID: lhsDesc.RangeID}, truncArgs) truncChan <- pErr }() // NB: the operation won't complete, so peek below Raft and wait for @@ -772,7 +773,7 @@ func mergeCheckingTimestampCaches( filterMu.Unlock() // Merge the RHS back into the LHS. - mergeChan := make(chan *roachpb.Error, 1) + mergeChan := make(chan *kvpb.Error, 1) go func() { args := adminMergeArgs(lhsDesc.StartKey.AsRawKey()) _, pErr := kv.SendWrapped(ctx, lhsStore.TestSender(), args) @@ -825,14 +826,14 @@ func mergeCheckingTimestampCaches( after45s := time.After(45 * time.Second) for _, asyncRes := range []struct { name string - ch chan *roachpb.Error + ch chan *kvpb.Error }{ {"increment", incChan}, {"truncate", truncChan}, {"merge", mergeChan}, } { t.Logf("waiting for result of %s", asyncRes.name) - var err *roachpb.Error + var err *kvpb.Error select { case err = <-asyncRes.ch: case <-after45s: @@ -845,7 +846,7 @@ func mergeCheckingTimestampCaches( // After the merge, attempt to write under the read. The batch should get // forwarded to a timestamp after the read. - ba = &roachpb.BatchRequest{} + ba = &kvpb.BatchRequest{} ba.Timestamp = readTS ba.RangeID = lhsDesc.RangeID ba.Add(incrementArgs(rhsKey, 1)) @@ -864,19 +865,19 @@ func mergeCheckingTimestampCaches( // application or a Raft snapshot. Either way though, the transaction should // not be allowed to create its record. hb, hbH := heartbeatArgs(&pushee, tc.Servers[0].Clock().Now()) - ba = &roachpb.BatchRequest{} + ba = &kvpb.BatchRequest{} ba.Header = hbH ba.RangeID = lhsDesc.RangeID ba.Add(hb) - var expReason roachpb.TransactionAbortedReason + var expReason kvpb.TransactionAbortedReason if disjointLeaseholders || throughSnapshot { - expReason = roachpb.ABORT_REASON_TIMESTAMP_CACHE_REJECTED + expReason = kvpb.ABORT_REASON_TIMESTAMP_CACHE_REJECTED } else { - expReason = roachpb.ABORT_REASON_ABORTED_RECORD_FOUND + expReason = kvpb.ABORT_REASON_ABORTED_RECORD_FOUND } if _, pErr := lhsStore.Send(ctx, ba); pErr == nil { t.Fatalf("expected TransactionAbortedError(%s) but got %v", expReason, pErr) - } else if abortErr, ok := pErr.GetDetail().(*roachpb.TransactionAbortedError); !ok { + } else if abortErr, ok := pErr.GetDetail().(*kvpb.TransactionAbortedError); !ok { t.Fatalf("expected TransactionAbortedError(%s) but got %v", expReason, pErr) } else if abortErr.Reason != expReason { t.Fatalf("expected TransactionAbortedError(%s) but got %v", expReason, pErr) @@ -926,11 +927,11 @@ func TestStoreRangeMergeTimestampCacheCausality(t *testing.T) { var readTS hlc.Timestamp rhsKey := scratchKey("c") var tc *testcluster.TestCluster - testingRequestFilter := func(_ context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + testingRequestFilter := func(_ context.Context, ba *kvpb.BatchRequest) *kvpb.Error { if ba.IsSingleSubsumeRequest() { // Before we execute a Subsume request, execute a read on the same store // at a much higher timestamp. - gba := &roachpb.BatchRequest{} + gba := &kvpb.BatchRequest{} gba.RangeID = ba.RangeID gba.Timestamp = ba.Timestamp.Add(42 /* wallTime */, 0 /* logical */) gba.Add(getArgs(rhsKey)) @@ -992,7 +993,7 @@ func TestStoreRangeMergeTimestampCacheCausality(t *testing.T) { // Merge [a, b) and [b, Max). Our request filter above will intercept the // merge and execute a read with a large timestamp immediately before the // Subsume request executes. - if _, pErr := kv.SendWrappedWith(ctx, tc.GetFirstStoreFromServer(t, 2), roachpb.Header{ + if _, pErr := kv.SendWrappedWith(ctx, tc.GetFirstStoreFromServer(t, 2), kvpb.Header{ RangeID: lhsRangeDesc.RangeID, }, adminMergeArgs(scratchKey("a"))); pErr != nil { t.Fatal(pErr) @@ -1019,7 +1020,7 @@ func TestStoreRangeMergeTimestampCacheCausality(t *testing.T) { // Attempt to write at the same time as the read. The write's timestamp // should be forwarded to after the read. - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Timestamp = readTS ba.RangeID = lhsRangeDesc.RangeID ba.Add(incrementArgs(rhsKey, 1)) @@ -1067,14 +1068,14 @@ func TestStoreRangeMergeTxnFailure(t *testing.T) { // Install a store filter that maybe injects retryable errors into a merge // transaction before ultimately aborting the merge. var retriesBeforeFailure int64 - testingRequestFilter := func(_ context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + testingRequestFilter := func(_ context.Context, ba *kvpb.BatchRequest) *kvpb.Error { for _, req := range ba.Requests { if et := req.GetEndTxn(); et != nil && et.InternalCommitTrigger.GetMergeTrigger() != nil { if atomic.AddInt64(&retriesBeforeFailure, -1) >= 0 { - return roachpb.NewError( - roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE, "filter err")) + return kvpb.NewError( + kvpb.NewTransactionRetryError(kvpb.RETRY_SERIALIZABLE, "filter err")) } - return roachpb.NewError(errors.New("injected permafail")) + return kvpb.NewError(errors.New("injected permafail")) } } return nil @@ -1116,11 +1117,11 @@ func TestStoreRangeMergeTxnFailure(t *testing.T) { {lhsDesc.RangeID, scratchKey("aa")}, {rhsDesc.RangeID, scratchKey("cc")}, } { - if reply, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ + if reply, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{ RangeID: tc.rangeID, }, getArgs(tc.key)); pErr != nil { t.Fatal(pErr) - } else if replyBytes, err := reply.(*roachpb.GetResponse).Value.GetBytes(); err != nil { + } else if replyBytes, err := reply.(*kvpb.GetResponse).Value.GetBytes(); err != nil { t.Fatal(err) } else if !bytes.Equal(replyBytes, []byte("val")) { t.Fatalf("actual value %q did not match expected value %q", replyBytes, []byte("val")) @@ -1163,10 +1164,10 @@ func TestStoreRangeMergeTxnRefresh(t *testing.T) { var sawMergeRefresh int32 testingResponseFilter := func( - ctx context.Context, ba *roachpb.BatchRequest, br *roachpb.BatchResponse, - ) *roachpb.Error { + ctx context.Context, ba *kvpb.BatchRequest, br *kvpb.BatchResponse, + ) *kvpb.Error { switch v := ba.Requests[0].GetInner().(type) { - case *roachpb.ConditionalPutRequest: + case *kvpb.ConditionalPutRequest: // Detect the range merge's deletion of the local range descriptor // and use it as an opportunity to bump the merge transaction's // write timestamp. This will necessitate a refresh. @@ -1179,7 +1180,7 @@ func TestStoreRangeMergeTxnRefresh(t *testing.T) { Add(100*time.Millisecond.Nanoseconds(), 0). WithSynthetic(true) } - case *roachpb.RefreshRequest: + case *kvpb.RefreshRequest: if bytes.HasSuffix(v.Key, keys.LocalRangeDescriptorSuffix) { atomic.AddInt32(&sawMergeRefresh, 1) } @@ -1670,7 +1671,7 @@ func TestStoreRangeMergeSplitRace_SplitWins(t *testing.T) { var mergePreSplit atomic.Value var splitCommit atomic.Value var mergeEndTxnTimestamp atomic.Value - testingRequestFilter := func(_ context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + testingRequestFilter := func(_ context.Context, ba *kvpb.BatchRequest) *kvpb.Error { for _, req := range ba.Requests { if get := req.GetGet(); get != nil && get.KeyLocking != lock.None { if v := lhsDescKey.Load(); v != nil && v.(roachpb.Key).Equal(get.Key) { @@ -1739,13 +1740,13 @@ func TestStoreRangeMergeSplitRace_SplitWins(t *testing.T) { } } -func checkConsistencyArgs(desc *roachpb.RangeDescriptor) *roachpb.CheckConsistencyRequest { - return &roachpb.CheckConsistencyRequest{ - RequestHeader: roachpb.RequestHeader{ +func checkConsistencyArgs(desc *roachpb.RangeDescriptor) *kvpb.CheckConsistencyRequest { + return &kvpb.CheckConsistencyRequest{ + RequestHeader: kvpb.RequestHeader{ Key: desc.StartKey.AsRawKey(), EndKey: desc.EndKey.AsRawKey(), }, - Mode: roachpb.ChecksumMode_CHECK_FULL, + Mode: kvpb.ChecksumMode_CHECK_FULL, } } @@ -1766,7 +1767,7 @@ func TestStoreRangeMergeRHSLeaseExpiration(t *testing.T) { // Install a hook to control when the merge transaction commits. mergeEndTxnReceived := make(chan *roachpb.Transaction, 10) // headroom in case the merge transaction retries finishMerge := make(chan struct{}) - testingRequestFilter := func(_ context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + testingRequestFilter := func(_ context.Context, ba *kvpb.BatchRequest) *kvpb.Error { for _, r := range ba.Requests { if et := r.GetEndTxn(); et != nil && et.InternalCommitTrigger.GetMergeTrigger() != nil { mergeEndTxnReceived <- ba.Txn @@ -1781,12 +1782,12 @@ func TestStoreRangeMergeRHSLeaseExpiration(t *testing.T) { const reqConcurrency = 10 var rhsSentinel roachpb.Key reqWaitingOnMerge := make(chan struct{}, reqConcurrency) - testingConcurrencyRetryFilter := func(_ context.Context, ba *roachpb.BatchRequest, pErr *roachpb.Error) { - if _, ok := pErr.GetDetail().(*roachpb.MergeInProgressError); ok { + testingConcurrencyRetryFilter := func(_ context.Context, ba *kvpb.BatchRequest, pErr *kvpb.Error) { + if _, ok := pErr.GetDetail().(*kvpb.MergeInProgressError); ok { for _, r := range ba.Requests { req := r.GetInner() switch req.Method() { - case roachpb.Get, roachpb.Put: + case kvpb.Get, kvpb.Put: if req.Header().Key.Equal(rhsSentinel) { reqWaitingOnMerge <- struct{}{} } @@ -1884,7 +1885,7 @@ func TestStoreRangeMergeRHSLeaseExpiration(t *testing.T) { // // This race has since been fixed by installing the mergeComplete channel // before the new lease. - reqErrs := make(chan *roachpb.Error) // closed when all reqs done + reqErrs := make(chan *kvpb.Error) // closed when all reqs done var wg sync.WaitGroup wg.Add(reqConcurrency) go func() { @@ -1906,13 +1907,13 @@ func TestStoreRangeMergeRHSLeaseExpiration(t *testing.T) { // interleaving, and successfully trigger the race when run with the race // detector enabled about 50% of the time. log.Infof(ctx, "starting req %d", i) - var req roachpb.Request + var req kvpb.Request if i%2 == 0 { req = getArgs(rhsSentinel) } else { req = putArgs(rhsSentinel, []byte(fmt.Sprintf("val%d", i))) } - _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ + _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{ RangeID: rhsDesc.RangeID, }, req) reqErrs <- pErr @@ -1931,7 +1932,7 @@ func TestStoreRangeMergeRHSLeaseExpiration(t *testing.T) { case pErr := <-reqErrs: // Requests may never wait on the merge if s1 has not yet learned // s2's lease is expired. Instead, we'll see a NotLeaseholderError. - require.IsType(t, &roachpb.NotLeaseHolderError{}, pErr.GetDetail()) + require.IsType(t, &kvpb.NotLeaseHolderError{}, pErr.GetDetail()) } } time.Sleep(50 * time.Millisecond) @@ -1945,7 +1946,7 @@ func TestStoreRangeMergeRHSLeaseExpiration(t *testing.T) { // and put request. Anything else is a consistency error (or a bug in the // test). for pErr := range reqErrs { - require.IsType(t, &roachpb.RangeNotFoundError{}, pErr.GetDetail()) + require.IsType(t, &kvpb.RangeNotFoundError{}, pErr.GetDetail()) } } @@ -1962,7 +1963,7 @@ func TestStoreRangeMergeRHSLeaseTransfers(t *testing.T) { var once sync.Once subsumeReceived := make(chan struct{}) finishSubsume := make(chan struct{}) - testingRequestFilter := func(_ context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + testingRequestFilter := func(_ context.Context, ba *kvpb.BatchRequest) *kvpb.Error { if ba.IsSingleSubsumeRequest() { once.Do(func() { subsumeReceived <- struct{}{} @@ -2059,7 +2060,7 @@ func TestStoreRangeMergeLHSLeaseTransfersAfterFreezeTime(t *testing.T) { var once sync.Once subsumeReceived := make(chan struct{}) finishSubsume := make(chan struct{}) - testingResponseFilter := func(_ context.Context, ba *roachpb.BatchRequest, _ *roachpb.BatchResponse) *roachpb.Error { + testingResponseFilter := func(_ context.Context, ba *kvpb.BatchRequest, _ *kvpb.BatchResponse) *kvpb.Error { if ba.IsSingleSubsumeRequest() { once.Do(func() { subsumeReceived <- struct{}{} @@ -2135,7 +2136,7 @@ func TestStoreRangeMergeLHSLeaseTransfersAfterFreezeTime(t *testing.T) { // Attempt to write below the closed timestamp, to the subsumed keyspace. // The write's timestamp should be forwarded to after the closed timestamp. // If it is not, we have violated the closed timestamp's promise! - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Timestamp = lhsClosedTS.Prev() ba.RangeID = lhsDesc.RangeID ba.Add(incrementArgs(rhsDesc.StartKey.AsRawKey().Next(), 1)) @@ -2162,12 +2163,12 @@ func TestStoreRangeMergeCheckConsistencyAfterSubsumption(t *testing.T) { // Install a hook to control when the merge transaction aborts. mergeEndTxnReceived := make(chan *roachpb.Transaction, 10) // headroom in case the merge transaction retries abortMergeTxn := make(chan struct{}) - testingRequestFilter := func(_ context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + testingRequestFilter := func(_ context.Context, ba *kvpb.BatchRequest) *kvpb.Error { for _, r := range ba.Requests { if et := r.GetEndTxn(); et != nil && et.InternalCommitTrigger.GetMergeTrigger() != nil { mergeEndTxnReceived <- ba.Txn <-abortMergeTxn - return roachpb.NewError(errors.New("abort the merge for test")) + return kvpb.NewError(errors.New("abort the merge for test")) } } return nil @@ -2199,7 +2200,7 @@ func TestStoreRangeMergeCheckConsistencyAfterSubsumption(t *testing.T) { tc.TransferRangeLeaseOrFatal(t, *rhsDesc, tc.Target(1)) // Launch the merge. - mergeErr := make(chan *roachpb.Error) + mergeErr := make(chan *kvpb.Error) go func() { args := adminMergeArgs(lhsDesc.StartKey.AsRawKey()) _, pErr := kv.SendWrapped(ctx, store.TestSender(), args) @@ -2228,7 +2229,7 @@ func TestStoreRangeMergeCheckConsistencyAfterSubsumption(t *testing.T) { close(abortMergeTxn) pErr := <-mergeErr - require.IsType(t, (*roachpb.Error)(nil), pErr) + require.IsType(t, (*kvpb.Error)(nil), pErr) require.Regexp(t, "abort the merge for test", pErr.String()) testutils.SucceedsSoon(t, func() error { @@ -2248,8 +2249,8 @@ func TestStoreRangeMergeConcurrentRequests(t *testing.T) { var store *kvserver.Store manualClock := hlc.NewHybridManualClock() testingResponseFilter := func( - ctx context.Context, ba *roachpb.BatchRequest, _ *roachpb.BatchResponse, - ) *roachpb.Error { + ctx context.Context, ba *kvpb.BatchRequest, _ *kvpb.BatchResponse, + ) *kvpb.Error { cput := ba.Requests[0].GetConditionalPut() if cput != nil && !cput.Value.IsPresent() && bytes.HasSuffix(cput.Key, keys.LocalRangeDescriptorSuffix) && rand.Int()%4 == 0 { // After every few deletions of the local range descriptor, expire all @@ -2466,7 +2467,7 @@ func TestStoreReplicaGCAfterMerge(t *testing.T) { nil, /* grpcServer */ tc.Servers[0].Stopper(), ) - errChan := errorChannelTestHandler(make(chan *roachpb.Error, 1)) + errChan := errorChannelTestHandler(make(chan *kvpb.Error, 1)) transport.Listen(store0.StoreID(), errChan) transport.Listen(store1.StoreID(), errChan) @@ -2493,7 +2494,7 @@ func TestStoreReplicaGCAfterMerge(t *testing.T) { select { case pErr := <-errChan: switch pErr.GetDetail().(type) { - case *roachpb.RaftGroupDeletedError: + case *kvpb.RaftGroupDeletedError: return default: t.Fatalf("unexpected error type %T: %s", pErr.GetDetail(), pErr) @@ -2569,7 +2570,7 @@ func TestStoreRangeMergeAddReplicaRace(t *testing.T) { }() go func() { _, err := tc.Server(0).DB().AdminChangeReplicas( - ctx, scratchStartKey, beforeDesc, roachpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(1))) + ctx, scratchStartKey, beforeDesc, kvpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(1))) addErrCh <- err }() mergeErr := <-mergeErrCh @@ -2624,7 +2625,7 @@ func TestStoreRangeMergeResplitAddReplicaRace(t *testing.T) { assert.NotEqual(t, origDesc.Generation, resplitDesc.Generation) _, err := tc.Server(0).DB().AdminChangeReplicas( - ctx, scratchStartKey, origDesc, roachpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(1))) + ctx, scratchStartKey, origDesc, kvpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(1))) if !kvserver.IsRetriableReplicationChangeError(err) { t.Fatalf(`expected "descriptor changed" error got: %+v`, err) } @@ -3078,7 +3079,7 @@ func TestStoreRangeMergeDeadFollowerDuringTxn(t *testing.T) { ctx := context.Background() var tc *testcluster.TestCluster - testingRequestFilter := func(_ context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + testingRequestFilter := func(_ context.Context, ba *kvpb.BatchRequest) *kvpb.Error { if ba.IsSingleSubsumeRequest() { tc.StopServer(2) // This is safe to call multiple times, it will only stop once } @@ -3176,7 +3177,7 @@ func TestStoreRangeReadoptedLHSFollower(t *testing.T) { if _, err := store0.DB().AdminChangeReplicas( ctx, lhsDesc.StartKey.AsRawKey(), *lhsDesc, - roachpb.MakeReplicationChanges( + kvpb.MakeReplicationChanges( roachpb.ADD_VOTER, tc.Target(2), )); !testutils.IsError(err, "descriptor changed") { t.Fatal(err) @@ -3422,12 +3423,12 @@ func testMergeWatcher(t *testing.T, injectFailures bool) { // Maybe inject some retryable errors when the merge transaction commits. lhsExpectedKey := scratchRangeDescriptorKey() - testingRequestFilter := func(_ context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + testingRequestFilter := func(_ context.Context, ba *kvpb.BatchRequest) *kvpb.Error { for _, req := range ba.Requests { if et := req.GetEndTxn(); et != nil && et.InternalCommitTrigger.GetMergeTrigger() != nil { if atomic.AddInt64(&mergeTxnRetries, -1) >= 0 { - return roachpb.NewError( - roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE, "filter err")) + return kvpb.NewError( + kvpb.NewTransactionRetryError(kvpb.RETRY_SERIALIZABLE, "filter err")) } } // We can detect PushTxn requests generated by the watcher goroutine @@ -3436,12 +3437,12 @@ func testMergeWatcher(t *testing.T, injectFailures bool) { pt.PusherTxn.Name == "merge" && pt.Key.Equal(keys.RangeDescriptorKey(lhsExpectedKey)) { if atomic.AddInt64(&pushTxnRetries, -1) >= 0 { - return roachpb.NewErrorf("injected failure") + return kvpb.NewErrorf("injected failure") } } - if g := req.GetGet(); g != nil && ba.ReadConsistency == roachpb.READ_UNCOMMITTED { + if g := req.GetGet(); g != nil && ba.ReadConsistency == kvpb.READ_UNCOMMITTED { if atomic.AddInt64(&meta2GetRetries, -1) >= 0 { - return roachpb.NewErrorf("injected failure") + return kvpb.NewErrorf("injected failure") } } } @@ -3504,7 +3505,7 @@ func testMergeWatcher(t *testing.T, injectFailures bool) { // and will notice that the merge has committed before the LHS does. getErr := make(chan error) go func() { - _, pErr = kv.SendWrappedWith(ctx, store2.TestSender(), roachpb.Header{ + _, pErr = kv.SendWrappedWith(ctx, store2.TestSender(), kvpb.Header{ RangeID: rhsDesc.RangeID, }, getArgs(rhsDesc.StartKey.AsRawKey())) getErr <- pErr.GoError() @@ -3569,7 +3570,7 @@ func TestStoreRangeMergeSlowWatcher(t *testing.T) { } } - testingRequestFilter := func(_ context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + testingRequestFilter := func(_ context.Context, ba *kvpb.BatchRequest) *kvpb.Error { for _, req := range ba.Requests { // We can detect PushTxn requests generated by the watcher goroutine // because they use the minimum transaction priority. Note that we @@ -3595,8 +3596,8 @@ func TestStoreRangeMergeSlowWatcher(t *testing.T) { var sawMeta2Req int64 meta2CKey := keys.RangeMetaKey(cKey).AsRawKey() testingResponseFilter := func( - ctx context.Context, ba *roachpb.BatchRequest, br *roachpb.BatchResponse, - ) *roachpb.Error { + ctx context.Context, ba *kvpb.BatchRequest, br *kvpb.BatchResponse, + ) *kvpb.Error { for i, req := range ba.Requests { if g := req.GetGet(); g != nil && g.Key.Equal(meta2CKey) && br.Responses[i].GetGet().Value == nil { atomic.StoreInt64(&sawMeta2Req, 1) @@ -3661,7 +3662,7 @@ func TestStoreRangeMergeSlowWatcher(t *testing.T) { // Immediately after the merge completes, send a request to B. getErr := make(chan error) go func() { - _, pErr := kv.SendWrappedWith(ctx, store1.TestSender(), roachpb.Header{ + _, pErr := kv.SendWrappedWith(ctx, store1.TestSender(), kvpb.Header{ RangeID: bRepl.GetRangeID(), }, getArgs(bKey.AsRawKey())) getErr <- pErr.GoError() @@ -4014,8 +4015,8 @@ func TestStoreRangeMergeRaftSnapshot(t *testing.T) { index := func() uint64 { repl := store0.LookupReplica(roachpb.RKey(keyA)) index := repl.GetLastIndex() - truncArgs := &roachpb.TruncateLogRequest{ - RequestHeader: roachpb.RequestHeader{Key: keyA}, + truncArgs := &kvpb.TruncateLogRequest{ + RequestHeader: kvpb.RequestHeader{Key: keyA}, Index: index, RangeID: repl.RangeID, } @@ -4104,7 +4105,7 @@ func TestStoreRangeMergeDuringShutdown(t *testing.T) { rhsDesc *roachpb.RangeDescriptor stop, stopping bool } - testingPostApplyFilter := func(args kvserverbase.ApplyFilterArgs) (int, *roachpb.Error) { + testingPostApplyFilter := func(args kvserverbase.ApplyFilterArgs) (int, *kvpb.Error) { state.Lock() if state.stop && !state.stopping && args.RangeID == state.rhsDesc.RangeID && args.IsLeaseRequest { // Shut down the store. The lease acquisition will notice that a merge is @@ -4258,8 +4259,8 @@ func TestMergeQueue(t *testing.T) { } clearRange := func(t *testing.T, start, end roachpb.RKey) { - if _, pErr := kv.SendWrapped(ctx, store.DB().NonTransactionalSender(), &roachpb.ClearRangeRequest{ - RequestHeader: roachpb.RequestHeader{Key: start.AsRawKey(), EndKey: end.AsRawKey()}, + if _, pErr := kv.SendWrapped(ctx, store.DB().NonTransactionalSender(), &kvpb.ClearRangeRequest{ + RequestHeader: kvpb.RequestHeader{Key: start.AsRawKey(), EndKey: end.AsRawKey()}, }); pErr != nil { t.Fatal(pErr) } @@ -4521,8 +4522,8 @@ func TestMergeQueue(t *testing.T) { verifyUnmergedSoon(t, store, lhsStartKey, rhsStartKey) // Delete sticky bit and verify that merge occurs. - unsplitArgs := &roachpb.AdminUnsplitRequest{ - RequestHeader: roachpb.RequestHeader{ + unsplitArgs := &kvpb.AdminUnsplitRequest{ + RequestHeader: kvpb.RequestHeader{ Key: rhsStartKey.AsRawKey(), }, } @@ -4869,8 +4870,8 @@ func TestInvalidSubsumeRequest(t *testing.T) { t.Fatal(err) } - getSnapArgs := roachpb.SubsumeRequest{ - RequestHeader: roachpb.RequestHeader{Key: rhsDesc.StartKey.AsRawKey()}, + getSnapArgs := kvpb.SubsumeRequest{ + RequestHeader: kvpb.RequestHeader{Key: rhsDesc.StartKey.AsRawKey()}, LeftDesc: *lhsDesc, RightDesc: *rhsDesc, } @@ -4881,7 +4882,7 @@ func TestInvalidSubsumeRequest(t *testing.T) { badRHSDesc.EndKey = badRHSDesc.EndKey.Next() badArgs := getSnapArgs badArgs.RightDesc = badRHSDesc - _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ + _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{ RangeID: rhsDesc.RangeID, }, &badArgs) if exp := "RHS range bounds do not match"; !testutils.IsPError(pErr, exp) { @@ -4893,7 +4894,7 @@ func TestInvalidSubsumeRequest(t *testing.T) { { badArgs := getSnapArgs badArgs.LeftDesc.EndKey = badArgs.LeftDesc.EndKey.Next() - _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ + _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{ RangeID: rhsDesc.RangeID, }, &badArgs) if exp := "ranges are not adjacent"; !testutils.IsPError(pErr, exp) { @@ -4902,7 +4903,7 @@ func TestInvalidSubsumeRequest(t *testing.T) { } // Subsume without an intent on the local range descriptor should fail. - _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ + _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{ RangeID: rhsDesc.RangeID, }, &getSnapArgs) if exp := "range missing intent on its local descriptor"; !testutils.IsPError(pErr, exp) { @@ -4917,7 +4918,7 @@ func TestInvalidSubsumeRequest(t *testing.T) { } // NB: Subsume intentionally takes place outside of the txn so // that it sees an intent rather than the value the txn just wrote. - _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ + _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{ RangeID: rhsDesc.RangeID, }, &getSnapArgs) if exp := "non-deletion intent on local range descriptor"; !testutils.IsPError(pErr, exp) { @@ -4935,11 +4936,11 @@ func sendWithTxn( desc *roachpb.RangeDescriptor, ts hlc.Timestamp, maxOffset time.Duration, - args roachpb.Request, + args kvpb.Request, ) error { txn := roachpb.MakeTransaction("test txn", desc.StartKey.AsRawKey(), 0, ts, maxOffset.Nanoseconds(), 0) - _, pErr := kv.SendWrappedWith(context.Background(), store.TestSender(), roachpb.Header{Txn: &txn}, args) + _, pErr := kv.SendWrappedWith(context.Background(), store.TestSender(), kvpb.Header{Txn: &txn}, args) return pErr.GoError() } @@ -4973,7 +4974,7 @@ func sendWithTxn( // type testCase struct { // name string // queryTsFunc func(freezeStart hlc.Timestamp) hlc.Timestamp -// queryArgsFunc func(key roachpb.Key) roachpb.Request +// queryArgsFunc func(key roachpb.Key) kvpb.Request // shouldBlock bool // } @@ -4983,7 +4984,7 @@ func sendWithTxn( // { // name: "historical read", // queryTsFunc: preUncertaintyTs, -// queryArgsFunc: func(key roachpb.Key) roachpb.Request { +// queryArgsFunc: func(key roachpb.Key) kvpb.Request { // return getArgs(key) // }, // shouldBlock: false, @@ -4993,7 +4994,7 @@ func sendWithTxn( // { // name: "historical write", // queryTsFunc: preUncertaintyTs, -// queryArgsFunc: func(key roachpb.Key) roachpb.Request { +// queryArgsFunc: func(key roachpb.Key) kvpb.Request { // return putArgs(key, []byte(`test value`)) // }, // shouldBlock: true, @@ -5005,7 +5006,7 @@ func sendWithTxn( // queryTsFunc: func(freezeStart hlc.Timestamp) hlc.Timestamp { // return freezeStart.Prev() // }, -// queryArgsFunc: func(key roachpb.Key) roachpb.Request { +// queryArgsFunc: func(key roachpb.Key) kvpb.Request { // return getArgs(key) // }, // shouldBlock: true, @@ -5101,9 +5102,9 @@ func setupClusterWithSubsumedRange( MaxOffset: testMaxOffset, TestingRequestFilter: filter.SuspendMergeTrigger, TestingConcurrencyRetryFilter: func( - ctx context.Context, ba *roachpb.BatchRequest, pErr *roachpb.Error, + ctx context.Context, ba *kvpb.BatchRequest, pErr *kvpb.Error, ) { - if _, ok := pErr.GetDetail().(*roachpb.MergeInProgressError); ok { + if _, ok := pErr.GetDetail().(*kvpb.MergeInProgressError); ok { atomic.AddInt32(&blockedRequestCount, 1) } }, @@ -5299,10 +5300,10 @@ func TestStoreMergeGCHint(t *testing.T) { gcHint := repl.GetGCHint() require.True(t, gcHint.IsEmpty(), "GC hint is not empty by default") - drArgs := &roachpb.DeleteRangeRequest{ + drArgs := &kvpb.DeleteRangeRequest{ UpdateRangeDeleteGCHint: true, UseRangeTombstone: true, - RequestHeader: roachpb.RequestHeader{ + RequestHeader: kvpb.RequestHeader{ Key: repl.Desc().StartKey.AsRawKey(), EndKey: repl.Desc().EndKey.AsRawKey(), }, diff --git a/pkg/kv/kvserver/client_metrics_test.go b/pkg/kv/kvserver/client_metrics_test.go index f54a3a23f589..08a76f735932 100644 --- a/pkg/kv/kvserver/client_metrics_test.go +++ b/pkg/kv/kvserver/client_metrics_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -204,7 +205,7 @@ func TestStoreResolveMetrics(t *testing.T) { const resolveAbortCount = int64(800) const resolvePoisonCount = int64(2400) - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} { repl := store.LookupReplica(keys.MustAddr(span.Key)) var err error @@ -219,7 +220,7 @@ func TestStoreResolveMetrics(t *testing.T) { key := span.Key endKey := span.EndKey if i > n/2 { - req := &roachpb.ResolveIntentRangeRequest{ + req := &kvpb.ResolveIntentRangeRequest{ IntentTxn: txn.TxnMeta, Status: status, Poison: poison, @@ -228,7 +229,7 @@ func TestStoreResolveMetrics(t *testing.T) { ba.Add(req) continue } - req := &roachpb.ResolveIntentRequest{ + req := &kvpb.ResolveIntentRequest{ IntentTxn: txn.TxnMeta, Status: status, Poison: poison, @@ -358,7 +359,7 @@ func TestStoreMetrics(t *testing.T) { _, err := tc.GetFirstStoreFromServer(t, 0).GetReplica(desc.RangeID) if err == nil { return fmt.Errorf("replica still exists on dest 0") - } else if errors.HasType(err, (*roachpb.RangeNotFoundError)(nil)) { + } else if errors.HasType(err, (*kvpb.RangeNotFoundError)(nil)) { return nil } return err diff --git a/pkg/kv/kvserver/client_migration_test.go b/pkg/kv/kvserver/client_migration_test.go index 761f8516affc..340c9bbace8f 100644 --- a/pkg/kv/kvserver/client_migration_test.go +++ b/pkg/kv/kvserver/client_migration_test.go @@ -19,6 +19,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" @@ -254,7 +255,7 @@ func TestMigrateWaitsForApplication(t *testing.T) { DisableAutomaticVersionUpgrade: make(chan struct{}), }, Store: &kvserver.StoreTestingKnobs{ - TestingApplyCalledTwiceFilter: func(args kvserverbase.ApplyFilterArgs) (int, *roachpb.Error) { + TestingApplyCalledTwiceFilter: func(args kvserverbase.ApplyFilterArgs) (int, *kvpb.Error) { if args.StoreID == roachpb.StoreID(n3) && args.State != nil && args.State.Version != nil { <-blockApplicationCh } diff --git a/pkg/kv/kvserver/client_raft_helpers_test.go b/pkg/kv/kvserver/client_raft_helpers_test.go index f966f936e7df..1d306324d750 100644 --- a/pkg/kv/kvserver/client_raft_helpers_test.go +++ b/pkg/kv/kvserver/client_raft_helpers_test.go @@ -14,6 +14,7 @@ import ( "context" "fmt" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -63,7 +64,7 @@ func (h *unreliableRaftHandler) HandleRaftRequest( ctx context.Context, req *kvserverpb.RaftMessageRequest, respStream kvserver.RaftMessageResponseStream, -) *roachpb.Error { +) *kvpb.Error { if len(req.Heartbeats)+len(req.HeartbeatResps) > 0 { reqCpy := *req req = &reqCpy @@ -165,10 +166,10 @@ func (h *testClusterStoreRaftMessageHandler) HandleRaftRequest( ctx context.Context, req *kvserverpb.RaftMessageRequest, respStream kvserver.RaftMessageResponseStream, -) *roachpb.Error { +) *kvpb.Error { store, err := h.getStore() if err != nil { - return roachpb.NewError(err) + return kvpb.NewError(err) } return store.HandleRaftRequest(ctx, req, respStream) } diff --git a/pkg/kv/kvserver/client_raft_test.go b/pkg/kv/kvserver/client_raft_test.go index e2890d6025c8..302247999b6b 100644 --- a/pkg/kv/kvserver/client_raft_test.go +++ b/pkg/kv/kvserver/client_raft_test.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" @@ -115,11 +116,11 @@ func TestStoreRecoverFromEngine(t *testing.T) { get := func(store *kvserver.Store, key roachpb.Key) int64 { args := getArgs(key) - resp, err := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{}, args) + resp, err := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{}, args) if err != nil { t.Fatal(err) } - return mustGetInt(resp.(*roachpb.GetResponse).Value) + return mustGetInt(resp.(*kvpb.GetResponse).Value) } validate := func(store *kvserver.Store) { if val := get(store, key1); val != 13 { @@ -134,10 +135,10 @@ func TestStoreRecoverFromEngine(t *testing.T) { // that both predate and postdate the split. func() { store := tc.GetFirstStoreFromServer(t, 0) - increment := func(key roachpb.Key, value int64) (*roachpb.IncrementResponse, *roachpb.Error) { + increment := func(key roachpb.Key, value int64) (*kvpb.IncrementResponse, *kvpb.Error) { args := incrementArgs(key, value) - resp, err := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{}, args) - incResp, _ := resp.(*roachpb.IncrementResponse) + resp, err := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{}, args) + incResp, _ := resp.(*kvpb.IncrementResponse) return incResp, err } @@ -179,7 +180,7 @@ func TestStoreRecoverFromEngine(t *testing.T) { t.Fatal(err) } incArgs = incrementArgs(key2, 0) - if _, err := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{}, incArgs); err != nil { + if _, err := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{}, incArgs); err != nil { t.Fatal(err) } @@ -209,8 +210,8 @@ func TestStoreRecoverWithErrors(t *testing.T) { }, Store: &kvserver.StoreTestingKnobs{ EvalKnobs: kvserverbase.BatchEvalTestingKnobs{ - TestingEvalFilter: func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { - _, ok := filterArgs.Req.(*roachpb.IncrementRequest) + TestingEvalFilter: func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { + _, ok := filterArgs.Req.(*kvpb.IncrementRequest) if ok && filterArgs.Req.Header().Key.Equal(keyA) { numIncrements++ } @@ -318,11 +319,11 @@ func TestReplicateRange(t *testing.T) { // Verify that the same data is available on the replica. testutils.SucceedsSoon(t, func() error { getArgs := getArgs(keyA) - if reply, err := kv.SendWrappedWith(ctx, tc.GetFirstStoreFromServer(t, 1).TestSender(), roachpb.Header{ - ReadConsistency: roachpb.INCONSISTENT, + if reply, err := kv.SendWrappedWith(ctx, tc.GetFirstStoreFromServer(t, 1).TestSender(), kvpb.Header{ + ReadConsistency: kvpb.INCONSISTENT, }, getArgs); err != nil { return errors.Errorf("failed to read data: %s", err) - } else if e, v := int64(5), mustGetInt(reply.(*roachpb.GetResponse).Value); v != e { + } else if e, v := int64(5), mustGetInt(reply.(*kvpb.GetResponse).Value); v != e { return errors.Errorf("failed to read correct data: expected %d, got %d", e, v) } return nil @@ -388,7 +389,7 @@ func TestRestoreReplicas(t *testing.T) { for i := 0; i < len(tc.Servers); i++ { if _, pErr := kv.SendWrapped(ctx, tc.GetFirstStoreFromServer(t, i).TestSender(), incArgs); pErr != nil { failures++ - if _, ok := pErr.GetDetail().(*roachpb.NotLeaseHolderError); !ok { + if _, ok := pErr.GetDetail().(*kvpb.NotLeaseHolderError); !ok { t.Fatalf("expected not lease holder error; got %s", pErr) } followerStore = tc.GetFirstStoreFromServer(t, i) @@ -401,11 +402,11 @@ func TestRestoreReplicas(t *testing.T) { testutils.SucceedsSoon(t, func() error { getArgs := getArgs([]byte("a")) - if reply, err := kv.SendWrappedWith(ctx, followerStore.TestSender(), roachpb.Header{ - ReadConsistency: roachpb.INCONSISTENT, + if reply, err := kv.SendWrappedWith(ctx, followerStore.TestSender(), kvpb.Header{ + ReadConsistency: kvpb.INCONSISTENT, }, getArgs); err != nil { return errors.Errorf("failed to read data: %s", err) - } else if e, v := int64(28), mustGetInt(reply.(*roachpb.GetResponse).Value); v != e { + } else if e, v := int64(28), mustGetInt(reply.(*kvpb.GetResponse).Value); v != e { return errors.Errorf("failed to read correct data: expected %d, got %d", e, v) } return nil @@ -440,11 +441,11 @@ func TestFailedReplicaChange(t *testing.T) { var runFilter atomic.Value runFilter.Store(true) - testingEvalFilter := func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { + testingEvalFilter := func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { if runFilter.Load().(bool) { - if et, ok := filterArgs.Req.(*roachpb.EndTxnRequest); ok && et.Commit && + if et, ok := filterArgs.Req.(*kvpb.EndTxnRequest); ok && et.Commit && et.InternalCommitTrigger != nil && et.InternalCommitTrigger.ChangeReplicasTrigger != nil { - return roachpb.NewErrorWithTxn(errors.Errorf("boom"), filterArgs.Hdr.Txn) + return kvpb.NewErrorWithTxn(errors.Errorf("boom"), filterArgs.Hdr.Txn) } } return nil @@ -542,11 +543,11 @@ func TestReplicateAfterTruncation(t *testing.T) { // Once it catches up, the effects of both commands can be seen. testutils.SucceedsSoon(t, func() error { getArgs := getArgs([]byte("a")) - if reply, err := kv.SendWrappedWith(ctx, tc.GetFirstStoreFromServer(t, 1).TestSender(), roachpb.Header{ - ReadConsistency: roachpb.INCONSISTENT, + if reply, err := kv.SendWrappedWith(ctx, tc.GetFirstStoreFromServer(t, 1).TestSender(), kvpb.Header{ + ReadConsistency: kvpb.INCONSISTENT, }, getArgs); err != nil { return errors.Errorf("failed to read data: %s", err) - } else if e, v := int64(16), mustGetInt(reply.(*roachpb.GetResponse).Value); v != e { + } else if e, v := int64(16), mustGetInt(reply.(*kvpb.GetResponse).Value); v != e { return errors.Errorf("failed to read correct data: expected %d, got %d", e, v) } return nil @@ -570,11 +571,11 @@ func TestReplicateAfterTruncation(t *testing.T) { testutils.SucceedsSoon(t, func() error { getArgs := getArgs([]byte("a")) - if reply, err := kv.SendWrappedWith(ctx, tc.GetFirstStoreFromServer(t, 1).TestSender(), roachpb.Header{ - ReadConsistency: roachpb.INCONSISTENT, + if reply, err := kv.SendWrappedWith(ctx, tc.GetFirstStoreFromServer(t, 1).TestSender(), kvpb.Header{ + ReadConsistency: kvpb.INCONSISTENT, }, getArgs); err != nil { return errors.Errorf("failed to read data: %s", err) - } else if e, v := int64(39), mustGetInt(reply.(*roachpb.GetResponse).Value); v != e { + } else if e, v := int64(39), mustGetInt(reply.(*kvpb.GetResponse).Value); v != e { return errors.Errorf("failed to read correct data: expected %d, got %d", e, v) } return nil @@ -979,7 +980,7 @@ func TestSnapshotAfterTruncationWithUncommittedTail(t *testing.T) { i++ sender := nonPartitionedSenders[i%2] _, pErr := kv.SendWrapped(ctx, sender, incArgs) - if _, ok := pErr.GetDetail().(*roachpb.NotLeaseHolderError); ok { + if _, ok := pErr.GetDetail().(*kvpb.NotLeaseHolderError); ok { return pErr.GoError() } else if pErr != nil { t.Fatal(pErr) @@ -1005,8 +1006,8 @@ func TestSnapshotAfterTruncationWithUncommittedTail(t *testing.T) { truncArgs.Key = partRepl.Desc().StartKey.AsRawKey() testutils.SucceedsSoon(t, func() error { manualClock.Increment(store.GetStoreConfig().LeaseExpiration()) - _, pErr := kv.SendWrappedWith(ctx, newLeaderReplSender, roachpb.Header{RangeID: partRepl.RangeID}, truncArgs) - if _, ok := pErr.GetDetail().(*roachpb.NotLeaseHolderError); ok { + _, pErr := kv.SendWrappedWith(ctx, newLeaderReplSender, kvpb.Header{RangeID: partRepl.RangeID}, truncArgs) + if _, ok := pErr.GetDetail().(*kvpb.NotLeaseHolderError); ok { return pErr.GoError() } else if pErr != nil { t.Fatal(pErr) @@ -1228,8 +1229,8 @@ func TestRequestsOnLaggingReplica(t *testing.T) { // the increment). This means that the partitioned replica will need a // snapshot to catch up. log.Infof(ctx, "test: truncating log...") - truncArgs := &roachpb.TruncateLogRequest{ - RequestHeader: roachpb.RequestHeader{ + truncArgs := &kvpb.TruncateLogRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, }, Index: index, @@ -1280,7 +1281,7 @@ func TestRequestsOnLaggingReplica(t *testing.T) { getRequest := getArgs(key) _, pErr := kv.SendWrapped(timeoutCtx, partitionedStoreSender, getRequest) require.NotNil(t, pErr, "unexpected success") - nlhe := pErr.GetDetail().(*roachpb.NotLeaseHolderError) + nlhe := pErr.GetDetail().(*kvpb.NotLeaseHolderError) require.NotNil(t, nlhe, "expected NotLeaseholderError, got: %s", pErr) require.False(t, nlhe.Lease.Empty()) require.NotNil(t, nlhe.Lease.Replica, "expected NotLeaseholderError with a known leaseholder, got: %s", pErr) @@ -1304,12 +1305,12 @@ func TestRequestsOnFollowerWithNonLiveLeaseholder(t *testing.T) { ctx := context.Background() var installPartition int32 - partitionFilter := func(_ context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + partitionFilter := func(_ context.Context, ba *kvpb.BatchRequest) *kvpb.Error { if atomic.LoadInt32(&installPartition) == 0 { return nil } if ba.GatewayNodeID == 1 && ba.Replica.NodeID == 4 { - return roachpb.NewError(context.Canceled) + return kvpb.NewError(context.Canceled) } return nil } @@ -1637,7 +1638,7 @@ func TestReplicateAfterRemoveAndSplit(t *testing.T) { // returned to us. if _, err := tc.Servers[0].DB().AdminChangeReplicas( ctx, splitKey, tc.LookupRangeOrFatal(t, splitKey), - roachpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(3)), + kvpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(3)), ); !kvserver.IsRetriableReplicationChangeError(err) { t.Fatalf("unexpected error %v", err) } @@ -1771,7 +1772,7 @@ func TestLogGrowthWhenRefreshingPendingCommands(t *testing.T) { initLogSize, _ := leaderRepl.GetRaftLogSize() // While a majority nodes are down, write some data. - putRes := make(chan *roachpb.Error) + putRes := make(chan *kvpb.Error) go func() { putArgs := putArgs([]byte("b"), make([]byte, raftConfig.RaftMaxUncommittedEntriesSize/8)) _, err := kv.SendWrapped(ctx, propNode, putArgs) @@ -1923,7 +1924,7 @@ func TestChangeReplicasDescriptorInvariant(t *testing.T) { require.NotNil(t, repl) addReplica := func(storeNum int, desc *roachpb.RangeDescriptor) error { - chgs := roachpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(storeNum)) + chgs := kvpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(storeNum)) _, err := repl.ChangeReplicas(ctx, desc, kvserverpb.SnapshotRequest_REBALANCE, kvserverpb.ReasonRangeUnderReplicated, "", chgs) return err } @@ -2156,7 +2157,7 @@ func runReplicateRestartAfterTruncation(t *testing.T, removeBeforeTruncateAndReA testutils.SucceedsSoon(t, func() error { tc.GetFirstStoreFromServer(t, 1).MustForceReplicaGCScanAndProcess() _, err := tc.GetFirstStoreFromServer(t, 1).GetReplica(desc.RangeID) - if !errors.HasType(err, (*roachpb.RangeNotFoundError)(nil)) { + if !errors.HasType(err, (*kvpb.RangeNotFoundError)(nil)) { // NB: errors.Wrapf(nil, ...) returns nil. // nolint:errwrap return errors.Errorf("expected replica to be garbage collected, got %v %T", err, err) @@ -2366,7 +2367,7 @@ func TestQuotaPool(t *testing.T) { // We block the third replica effectively causing acquisition of quota // without subsequent release. raftLockReplica(followerRepl) - ch := make(chan *roachpb.Error, 1) + ch := make(chan *kvpb.Error, 1) func() { defer followerRepl.RaftUnlock() @@ -2384,7 +2385,7 @@ func TestQuotaPool(t *testing.T) { // to be the same as what we started with. keyToWrite := key.Next() value := bytes.Repeat([]byte("v"), (3*quota)/4) - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Add(putArgs(keyToWrite, value)) if err := ba.SetActiveTimestamp(tc.Servers[0].Clock()); err != nil { t.Fatal(err) @@ -2405,10 +2406,10 @@ func TestQuotaPool(t *testing.T) { }) go func() { - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Add(putArgs(keyToWrite, value)) if err := ba.SetActiveTimestamp(tc.Servers[0].Clock()); err != nil { - ch <- roachpb.NewError(err) + ch <- kvpb.NewError(err) return } _, pErr := leaderRepl.Send(ctx, ba) @@ -2495,7 +2496,7 @@ func TestWedgedReplicaDetection(t *testing.T) { // Send a request to the leader replica. followerRepl is locked so it will // not respond. value := []byte("value") - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Add(putArgs(key, value)) if err := ba.SetActiveTimestamp(tc.Servers[0].Clock()); err != nil { t.Fatal(err) @@ -2740,7 +2741,7 @@ func TestReplicateAfterSplit(t *testing.T) { // Issue an increment for later check. incArgs := incrementArgs(key, 11) if _, err := kv.SendWrappedWith(ctx, store0.TestSender(), - roachpb.Header{}, incArgs); err != nil { + kvpb.Header{}, incArgs); err != nil { t.Fatal(err) } // Now add the second replica. @@ -2753,11 +2754,11 @@ func TestReplicateAfterSplit(t *testing.T) { testutils.SucceedsSoon(t, func() error { getArgs := getArgs(key) // Reading on non-lease holder replica should use inconsistent read - if reply, err := kv.SendWrappedWith(ctx, tc.GetFirstStoreFromServer(t, 1).TestSender(), roachpb.Header{ - ReadConsistency: roachpb.INCONSISTENT, + if reply, err := kv.SendWrappedWith(ctx, tc.GetFirstStoreFromServer(t, 1).TestSender(), kvpb.Header{ + ReadConsistency: kvpb.INCONSISTENT, }, getArgs); err != nil { return errors.Errorf("failed to read data: %s", err) - } else if e, v := int64(11), mustGetInt(reply.(*roachpb.GetResponse).Value); v != e { + } else if e, v := int64(11), mustGetInt(reply.(*kvpb.GetResponse).Value); v != e { return errors.Errorf("failed to read correct data: expected %d, got %d", e, v) } return nil @@ -2805,7 +2806,7 @@ func TestReplicaRemovalCampaign(t *testing.T) { replica2 := store0.LookupReplica(roachpb.RKey(key2)) rg2 := func(s *kvserver.Store) kv.Sender { - return kv.Wrap(s, func(ba *roachpb.BatchRequest) *roachpb.BatchRequest { + return kv.Wrap(s, func(ba *kvpb.BatchRequest) *kvpb.BatchRequest { if ba.RangeID == 0 { ba.RangeID = replica2.RangeID } @@ -2886,7 +2887,7 @@ func TestRaftAfterRemoveRange(t *testing.T) { for i := range tc.Servers[1:] { store := tc.GetFirstStoreFromServer(t, i) _, err := store.GetReplica(desc.RangeID) - if !errors.HasType(err, (*roachpb.RangeNotFoundError)(nil)) { + if !errors.HasType(err, (*kvpb.RangeNotFoundError)(nil)) { return errors.Wrapf(err, "range %d not yet removed from %s", desc.RangeID, store) } } @@ -3000,7 +3001,7 @@ func TestRemovePlaceholderRace(t *testing.T) { for i := 0; i < 100; i++ { for _, action := range []roachpb.ReplicaChangeType{roachpb.REMOVE_VOTER, roachpb.ADD_VOTER} { for { - chgs := roachpb.MakeReplicationChanges(action, tc.Target(1)) + chgs := kvpb.MakeReplicationChanges(action, tc.Target(1)) if _, err := repl.ChangeReplicas(ctx, repl.Desc(), kvserverpb.SnapshotRequest_REBALANCE, kvserverpb.ReasonUnknown, "", chgs); err != nil { if kvserver.IsRetriableReplicationChangeError(err) || kvserver.IsReplicationChangeInProgressError(err) { @@ -3023,7 +3024,7 @@ func (ncc *noConfChangeTestHandler) HandleRaftRequest( ctx context.Context, req *kvserverpb.RaftMessageRequest, respStream kvserver.RaftMessageResponseStream, -) *roachpb.Error { +) *kvpb.Error { for i, e := range req.Message.Entries { if e.Type == raftpb.EntryConfChange { var cc raftpb.ConfChange @@ -3054,9 +3055,9 @@ func (ncc *noConfChangeTestHandler) HandleRaftResponse( ctx context.Context, resp *kvserverpb.RaftMessageResponse, ) error { switch val := resp.Union.GetValue().(type) { - case *roachpb.Error: + case *kvpb.Error: switch val.GetDetail().(type) { - case *roachpb.ReplicaTooOldError: + case *kvpb.ReplicaTooOldError: // We're going to manually GC the replica, so ignore these errors. return nil } @@ -3099,7 +3100,7 @@ func TestReplicaGCRace(t *testing.T) { // Add the victim replica. Note that it will receive a snapshot and raft log // replays, but will not process the configuration change containing the new // range descriptor, preventing it from learning of the new NextReplicaID. - chgs := roachpb.MakeReplicationChanges(roachpb.ADD_VOTER, roachpb.ReplicationTarget{ + chgs := kvpb.MakeReplicationChanges(roachpb.ADD_VOTER, roachpb.ReplicationTarget{ NodeID: toStore.Ident.NodeID, StoreID: toStore.Ident.StoreID, }) @@ -3179,7 +3180,7 @@ func TestReplicaGCRace(t *testing.T) { nil, /* grpcServer */ tc.Servers[0].Stopper(), ) - errChan := errorChannelTestHandler(make(chan *roachpb.Error, 1)) + errChan := errorChannelTestHandler(make(chan *kvpb.Error, 1)) fromTransport.Listen(fromStore.StoreID(), errChan) // Send the heartbeat. Boom. See #11591. @@ -3200,7 +3201,7 @@ func TestReplicaGCRace(t *testing.T) { select { case pErr := <-errChan: switch pErr.GetDetail().(type) { - case *roachpb.RaftGroupDeletedError: + case *kvpb.RaftGroupDeletedError: default: t.Fatalf("unexpected error type %T: %s", pErr.GetDetail(), pErr) } @@ -3481,16 +3482,16 @@ func TestReplicateRogueRemovedNode(t *testing.T) { _, pErr := kv.SendWrappedWith( context.Background(), tc.GetFirstStoreFromServer(t, 2).TestSender(), - roachpb.Header{ + kvpb.Header{ Replica: replicaDesc, Timestamp: tc.Servers[2].Clock().Now(), }, incArgs, ) detail := pErr.GetDetail() switch detail.(type) { - case *roachpb.RangeNotFoundError: + case *kvpb.RangeNotFoundError: // The typical case. - case *roachpb.NotLeaseHolderError: + case *kvpb.NotLeaseHolderError: // The atypical case - the lease may have expired and the // lease acquisition may be refused. default: @@ -3546,11 +3547,11 @@ func TestReplicateRogueRemovedNode(t *testing.T) { finishWG.Wait() } -type errorChannelTestHandler chan *roachpb.Error +type errorChannelTestHandler chan *kvpb.Error func (errorChannelTestHandler) HandleRaftRequest( _ context.Context, _ *kvserverpb.RaftMessageRequest, _ kvserver.RaftMessageResponseStream, -) *roachpb.Error { +) *kvpb.Error { panic("unimplemented") } @@ -3558,7 +3559,7 @@ func (d errorChannelTestHandler) HandleRaftResponse( ctx context.Context, resp *kvserverpb.RaftMessageResponse, ) error { switch val := resp.Union.GetValue().(type) { - case *roachpb.Error: + case *kvpb.Error: d <- val default: log.Fatalf(ctx, "unexpected response type %T", val) @@ -3676,7 +3677,7 @@ func TestReplicateRemovedNodeDisruptiveElection(t *testing.T) { nil, /* grpcServer */ tc.Servers[0].Stopper(), ) - errChan := errorChannelTestHandler(make(chan *roachpb.Error, 1)) + errChan := errorChannelTestHandler(make(chan *kvpb.Error, 1)) transport0.Listen(target0.StoreID, errChan) // Simulate the removed node asking to trigger an election. Try and try again @@ -3699,7 +3700,7 @@ func TestReplicateRemovedNodeDisruptiveElection(t *testing.T) { select { case pErr := <-errChan: switch pErr.GetDetail().(type) { - case *roachpb.ReplicaTooOldError: + case *kvpb.ReplicaTooOldError: default: t.Fatalf("unexpected error type %T: %s", pErr.GetDetail(), pErr) } @@ -3802,7 +3803,7 @@ func TestReplicaTooOldGC(t *testing.T) { testutils.SucceedsSoon(t, func() error { replica, err := tc.GetFirstStoreFromServer(t, 3).GetReplica(desc.RangeID) if err != nil { - if errors.HasType(err, (*roachpb.RangeNotFoundError)(nil)) { + if errors.HasType(err, (*kvpb.RangeNotFoundError)(nil)) { return nil } return err @@ -3981,7 +3982,7 @@ func TestLeaseHolderRemoveSelf(t *testing.T) { // Expect that we can still successfully do a get on the range. getArgs := getArgs(key) - _, pErr := kv.SendWrappedWith(ctx, leaseHolder.TestSender(), roachpb.Header{}, getArgs) + _, pErr := kv.SendWrappedWith(ctx, leaseHolder.TestSender(), kvpb.Header{}, getArgs) if pErr != nil { t.Fatal(pErr) } @@ -4030,13 +4031,13 @@ func TestRemovedReplicaError(t *testing.T) { // start seeing the RangeNotFoundError after a little bit of time has passed. getArgs := getArgs(key) testutils.SucceedsSoon(t, func() error { - _, pErr := kv.SendWrappedWith(ctx, store, roachpb.Header{}, getArgs) + _, pErr := kv.SendWrappedWith(ctx, store, kvpb.Header{}, getArgs) switch pErr.GetDetail().(type) { - case *roachpb.AmbiguousResultError: + case *kvpb.AmbiguousResultError: return pErr.GoError() - case *roachpb.NotLeaseHolderError: + case *kvpb.NotLeaseHolderError: return pErr.GoError() - case *roachpb.RangeNotFoundError: + case *kvpb.RangeNotFoundError: return nil default: } @@ -4091,7 +4092,7 @@ func TestTransferRaftLeadership(t *testing.T) { getArgs := getArgs(key) if _, pErr := kv.SendWrappedWith( - context.Background(), store0, roachpb.Header{RangeID: repl0.RangeID}, getArgs, + context.Background(), store0, kvpb.Header{RangeID: repl0.RangeID}, getArgs, ); pErr != nil { t.Fatalf("expect get nil, actual get %v ", pErr) } @@ -4105,7 +4106,7 @@ func TestTransferRaftLeadership(t *testing.T) { // Transfer the lease. We'll then check that the leadership follows // automatically. transferLeaseArgs := adminTransferLeaseArgs(key, store1.StoreID()) - _, pErr := kv.SendWrappedWith(ctx, store0, roachpb.Header{RangeID: repl0.RangeID}, transferLeaseArgs) + _, pErr := kv.SendWrappedWith(ctx, store0, kvpb.Header{RangeID: repl0.RangeID}, transferLeaseArgs) require.NoError(t, pErr.GoError()) // Verify leadership is transferred. @@ -4399,9 +4400,9 @@ func TestFailedConfChange(t *testing.T) { // Trigger errors at apply time so they happen on both leaders and // followers. var filterActive int32 - testingApplyFilter := func(filterArgs kvserverbase.ApplyFilterArgs) (int, *roachpb.Error) { + testingApplyFilter := func(filterArgs kvserverbase.ApplyFilterArgs) (int, *kvpb.Error) { if atomic.LoadInt32(&filterActive) == 1 && filterArgs.ChangeReplicas != nil { - return 0, roachpb.NewErrorf("boom") + return 0, kvpb.NewErrorf("boom") } return 0, nil } @@ -4487,15 +4488,15 @@ func TestStoreRangeWaitForApplication(t *testing.T) { var filterRangeIDAtomic int64 ctx := context.Background() - testingRequestFilter := func(_ context.Context, ba *roachpb.BatchRequest) (retErr *roachpb.Error) { + testingRequestFilter := func(_ context.Context, ba *kvpb.BatchRequest) (retErr *kvpb.Error) { if rangeID := roachpb.RangeID(atomic.LoadInt64(&filterRangeIDAtomic)); rangeID != ba.RangeID { return nil } - pErr := roachpb.NewErrorf("blocking %s in this test", ba.Summary()) + pErr := kvpb.NewErrorf("blocking %s in this test", ba.Summary()) if len(ba.Requests) != 1 { return pErr } - _, ok := ba.Requests[0].GetInner().(*roachpb.PutRequest) + _, ok := ba.Requests[0].GetInner().(*kvpb.PutRequest) if !ok { return pErr } @@ -4974,7 +4975,7 @@ func TestAckWriteBeforeApplication(t *testing.T) { var magicTS hlc.Timestamp blockPreApplication, blockPostApplication := make(chan struct{}), make(chan struct{}) applyFilterFn := func(ch chan struct{}) kvserverbase.ReplicaApplyFilter { - return func(filterArgs kvserverbase.ApplyFilterArgs) (int, *roachpb.Error) { + return func(filterArgs kvserverbase.ApplyFilterArgs) (int, *kvpb.Error) { if atomic.LoadInt32(&filterActive) == 1 && filterArgs.WriteTimestamp == magicTS { <-ch } @@ -5007,10 +5008,10 @@ func TestAckWriteBeforeApplication(t *testing.T) { // Begin peforming a write on the Range. magicTS = tc.Servers[0].Clock().Now() atomic.StoreInt32(&filterActive, 1) - ch := make(chan *roachpb.Error, 1) + ch := make(chan *kvpb.Error, 1) go func() { put := putArgs(key, []byte("val")) - _, pErr := kv.SendWrappedWith(ctx, tc.GetFirstStoreFromServer(t, 0).TestSender(), roachpb.Header{ + _, pErr := kv.SendWrappedWith(ctx, tc.GetFirstStoreFromServer(t, 0).TestSender(), kvpb.Header{ Timestamp: magicTS, }, put) ch <- pErr @@ -5128,12 +5129,12 @@ func TestProcessSplitAfterRightHandSideHasBeenRemoved(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - noopProposalFilter := kvserverbase.ReplicaProposalFilter(func(args kvserverbase.ProposalFilterArgs) *roachpb.Error { + noopProposalFilter := kvserverbase.ReplicaProposalFilter(func(args kvserverbase.ProposalFilterArgs) *kvpb.Error { return nil }) var proposalFilter atomic.Value proposalFilter.Store(noopProposalFilter) - testingProposalFilter := func(args kvserverbase.ProposalFilterArgs) *roachpb.Error { + testingProposalFilter := func(args kvserverbase.ProposalFilterArgs) *kvpb.Error { return proposalFilter.Load().(kvserverbase.ReplicaProposalFilter)(args) } @@ -5163,12 +5164,12 @@ func TestProcessSplitAfterRightHandSideHasBeenRemoved(t *testing.T) { // Set up a hook to partition the RHS range at its initial range ID // before proposing the split trigger. var setupOnce sync.Once - f := kvserverbase.ReplicaProposalFilter(func(args kvserverbase.ProposalFilterArgs) *roachpb.Error { - req, ok := args.Req.GetArg(roachpb.EndTxn) + f := kvserverbase.ReplicaProposalFilter(func(args kvserverbase.ProposalFilterArgs) *kvpb.Error { + req, ok := args.Req.GetArg(kvpb.EndTxn) if !ok { return nil } - endTxn := req.(*roachpb.EndTxnRequest) + endTxn := req.(*kvpb.EndTxnRequest) if endTxn.InternalCommitTrigger == nil || endTxn.InternalCommitTrigger.SplitTrigger == nil { return nil } @@ -5313,7 +5314,7 @@ func TestProcessSplitAfterRightHandSideHasBeenRemoved(t *testing.T) { // has been removed at the old replica ID. _, err = tc.Servers[0].DB().AdminChangeReplicas( ctx, keyB, tc.LookupRangeOrFatal(t, keyB), - roachpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(0)), + kvpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(0)), ) require.True(t, kvserver.IsRetriableReplicationChangeError(err), err) @@ -5366,7 +5367,7 @@ func TestProcessSplitAfterRightHandSideHasBeenRemoved(t *testing.T) { // has been removed at the old replica ID. _, err = tc.Servers[0].DB().AdminChangeReplicas( ctx, keyB, tc.LookupRangeOrFatal(t, keyB), - roachpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(0)), + kvpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(0)), ) require.True(t, kvserver.IsRetriableReplicationChangeError(err), err) @@ -5442,7 +5443,7 @@ func TestProcessSplitAfterRightHandSideHasBeenRemoved(t *testing.T) { // retriable-looking situation here that will persist. _, err = tc.Servers[0].DB().AdminChangeReplicas( ctx, keyB, tc.LookupRangeOrFatal(t, keyB), - roachpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(0)), + kvpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(0)), ) require.True(t, kvserver.IsRetriableReplicationChangeError(err), err) // Ensure that the replica exists with the higher replica ID. @@ -5506,7 +5507,7 @@ func TestProcessSplitAfterRightHandSideHasBeenRemoved(t *testing.T) { // would be retried internally. _, err = tc.Servers[0].DB().AdminChangeReplicas( ctx, keyB, tc.LookupRangeOrFatal(t, keyB), - roachpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(0)), + kvpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(0)), ) require.True(t, kvserver.IsRetriableReplicationChangeError(err), err) // Ensure that there's no tombstone. @@ -5582,9 +5583,9 @@ func TestReplicaRemovalClosesProposalQuota(t *testing.T) { ServerArgs: base.TestServerArgs{ Knobs: base.TestingKnobs{Store: &kvserver.StoreTestingKnobs{ DisableReplicaGCQueue: true, - TestingRequestFilter: func(_ context.Context, r *roachpb.BatchRequest) *roachpb.Error { + TestingRequestFilter: func(_ context.Context, r *kvpb.BatchRequest) *kvpb.Error { if r.RangeID == roachpb.RangeID(atomic.LoadInt64(&rangeID)) { - if _, isPut := r.GetArg(roachpb.Put); isPut { + if _, isPut := r.GetArg(kvpb.Put); isPut { atomic.AddInt64(&putRequestCount, 1) } } @@ -5634,7 +5635,7 @@ func TestReplicaRemovalClosesProposalQuota(t *testing.T) { go func(i int) { defer wg.Done() k := append(key[0:len(key):len(key)], strconv.Itoa(i)...) - _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ + _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{ RangeID: desc.RangeID, }, putArgs(k, bytes.Repeat([]byte{'a'}, 1000))) require.Regexp(t, @@ -5873,10 +5874,10 @@ func TestRaftSnapshotsWithMVCCRangeKeys(t *testing.T) { // Write a couple of overlapping MVCC range tombstones across [a-d) and [b-e), and // record their timestamps. ts1 := ts.Clock().Now() - _, pErr := kv.SendWrappedWith(ctx, sender, roachpb.Header{ + _, pErr := kv.SendWrappedWith(ctx, sender, kvpb.Header{ Timestamp: ts1, - }, &roachpb.DeleteRangeRequest{ - RequestHeader: roachpb.RequestHeader{ + }, &kvpb.DeleteRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: roachpb.Key("a"), EndKey: roachpb.Key("d"), }, @@ -5885,10 +5886,10 @@ func TestRaftSnapshotsWithMVCCRangeKeys(t *testing.T) { require.NoError(t, pErr.GoError()) ts2 := ts.Clock().Now() - _, pErr = kv.SendWrappedWith(ctx, sender, roachpb.Header{ + _, pErr = kv.SendWrappedWith(ctx, sender, kvpb.Header{ Timestamp: ts2, - }, &roachpb.DeleteRangeRequest{ - RequestHeader: roachpb.RequestHeader{ + }, &kvpb.DeleteRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: roachpb.Key("b"), EndKey: roachpb.Key("e"), }, @@ -5937,11 +5938,11 @@ func TestRaftSnapshotsWithMVCCRangeKeys(t *testing.T) { checkConsistency := func(desc roachpb.RangeDescriptor) { resp, pErr := kv.SendWrapped(ctx, sender, checkConsistencyArgs(&desc)) require.NoError(t, pErr.GoError()) - ccResp := resp.(*roachpb.CheckConsistencyResponse) + ccResp := resp.(*kvpb.CheckConsistencyResponse) require.Len(t, ccResp.Result, 1) result := ccResp.Result[0] require.Equal(t, desc.RangeID, result.RangeID) - require.Equal(t, roachpb.CheckConsistencyResponse_RANGE_CONSISTENT, result.Status, "%+v", result) + require.Equal(t, kvpb.CheckConsistencyResponse_RANGE_CONSISTENT, result.Status, "%+v", result) } checkConsistency(descA) @@ -5997,8 +5998,8 @@ func TestRaftSnapshotsWithMVCCRangeKeysEverywhere(t *testing.T) { // this so that we can run a consistency check, since the stats may or may not // pick up the range keys in different key ranges. for _, desc := range descs { - _, pErr := kv.SendWrapped(ctx, sender, &roachpb.RecomputeStatsRequest{ - RequestHeader: roachpb.RequestHeader{ + _, pErr := kv.SendWrapped(ctx, sender, &kvpb.RecomputeStatsRequest{ + RequestHeader: kvpb.RequestHeader{ Key: desc.StartKey.AsRawKey(), }, }) @@ -6050,10 +6051,10 @@ func TestRaftSnapshotsWithMVCCRangeKeysEverywhere(t *testing.T) { for _, desc := range descs { resp, pErr := kv.SendWrapped(ctx, sender, checkConsistencyArgs(&desc)) require.NoError(t, pErr.GoError()) - ccResp := resp.(*roachpb.CheckConsistencyResponse) + ccResp := resp.(*kvpb.CheckConsistencyResponse) require.Len(t, ccResp.Result, 1) result := ccResp.Result[0] require.Equal(t, desc.RangeID, result.RangeID) - require.Equal(t, roachpb.CheckConsistencyResponse_RANGE_CONSISTENT, result.Status, "%+v", result) + require.Equal(t, kvpb.CheckConsistencyResponse_RANGE_CONSISTENT, result.Status, "%+v", result) } } diff --git a/pkg/kv/kvserver/client_rangefeed_test.go b/pkg/kv/kvserver/client_rangefeed_test.go index 028fe9ec03da..e21eba7c8af2 100644 --- a/pkg/kv/kvserver/client_rangefeed_test.go +++ b/pkg/kv/kvserver/client_rangefeed_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" @@ -313,7 +314,7 @@ func TestRangefeedWorksOnLivenessRange(t *testing.T) { }() // Wait for a liveness update. - var livenessEvent roachpb.RangeFeedValue + var livenessEvent kvpb.RangeFeedValue timeoutC := time.After(10 * time.Second) for { select { diff --git a/pkg/kv/kvserver/client_relocate_range_test.go b/pkg/kv/kvserver/client_relocate_range_test.go index 56289ecff7ba..9b11cebd63b5 100644 --- a/pkg/kv/kvserver/client_relocate_range_test.go +++ b/pkg/kv/kvserver/client_relocate_range_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -142,7 +143,7 @@ func requireLeaseAt( }) } -func usesAtomicReplicationChange(ops []roachpb.ReplicationChange) bool { +func usesAtomicReplicationChange(ops []kvpb.ReplicationChange) bool { // There are 4 sets of operations that are executed atomically: // 1. Voter rebalances (ADD_VOTER, REMOVE_VOTER) // 2. Non-voter promoted to voter (ADD_VOTER, REMOVE_NON_VOTER) @@ -171,7 +172,7 @@ func TestAdminRelocateRange(t *testing.T) { ctx := context.Background() type intercept struct { - ops []roachpb.ReplicationChange + ops []kvpb.ReplicationChange leaseTarget *roachpb.ReplicationTarget } var intercepted []intercept @@ -202,7 +203,7 @@ func TestAdminRelocateRange(t *testing.T) { knobs := base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ - OnRelocatedOne: func(ops []roachpb.ReplicationChange, leaseTarget *roachpb.ReplicationTarget) { + OnRelocatedOne: func(ops []kvpb.ReplicationChange, leaseTarget *roachpb.ReplicationTarget) { intercepted = append(intercepted, intercept{ ops: ops, leaseTarget: leaseTarget, @@ -470,8 +471,8 @@ func TestReplicaRemovalDuringGet(t *testing.T) { resp, pErr := evalDuringReplicaRemoval(ctx, getArgs(key)) require.Nil(t, pErr) require.NotNil(t, resp) - require.NotNil(t, resp.(*roachpb.GetResponse).Value) - val, err := resp.(*roachpb.GetResponse).Value.GetBytes() + require.NotNil(t, resp.(*kvpb.GetResponse).Value) + val, err := resp.(*kvpb.GetResponse).Value.GetBytes() require.NoError(t, err) require.Equal(t, []byte("foo"), val) } @@ -500,7 +501,7 @@ func TestReplicaRemovalDuringCPut(t *testing.T) { req := cPutArgs(key, []byte("bar"), []byte("foo")) _, pErr = evalDuringReplicaRemoval(ctx, req) require.NotNil(t, pErr) - require.IsType(t, &roachpb.AmbiguousResultError{}, pErr.GetDetail()) + require.IsType(t, &kvpb.AmbiguousResultError{}, pErr.GetDetail()) } // setupReplicaRemovalTest sets up a test cluster that can be used to test @@ -513,7 +514,7 @@ func setupReplicaRemovalTest( ) ( *testcluster.TestCluster, roachpb.Key, - func(context.Context, roachpb.Request) (roachpb.Response, *roachpb.Error), + func(context.Context, kvpb.Request) (kvpb.Response, *kvpb.Error), ) { t.Helper() @@ -521,7 +522,7 @@ func setupReplicaRemovalTest( requestReadyC := make(chan struct{}) // signals main thread that request is teed up requestEvalC := make(chan struct{}) // signals cluster to evaluate the request - evalFilter := func(args kvserverbase.FilterArgs) *roachpb.Error { + evalFilter := func(args kvserverbase.FilterArgs) *kvpb.Error { if args.Ctx.Value(magicKey{}) != nil { requestReadyC <- struct{}{} <-requestEvalC @@ -555,11 +556,11 @@ func setupReplicaRemovalTest( // Return a function that can be used to evaluate a delayed request // during replica removal. - evalDuringReplicaRemoval := func(ctx context.Context, req roachpb.Request) (roachpb.Response, *roachpb.Error) { + evalDuringReplicaRemoval := func(ctx context.Context, req kvpb.Request) (kvpb.Response, *kvpb.Error) { // Submit request and wait for it to block. type result struct { - resp roachpb.Response - err *roachpb.Error + resp kvpb.Response + err *kvpb.Error } resultC := make(chan result) srv := tc.Servers[0] diff --git a/pkg/kv/kvserver/client_replica_backpressure_test.go b/pkg/kv/kvserver/client_replica_backpressure_test.go index d2780aae98cd..a0c3b1c45a97 100644 --- a/pkg/kv/kvserver/client_replica_backpressure_test.go +++ b/pkg/kv/kvserver/client_replica_backpressure_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -71,13 +72,13 @@ func TestBackpressureNotAppliedWhenReducingRangeSize(t *testing.T) { ServerArgs: base.TestServerArgs{ Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(ctx context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + TestingRequestFilter: func(ctx context.Context, ba *kvpb.BatchRequest) *kvpb.Error { if ba.Header.Txn != nil && ba.Header.Txn.Name == "split" && !allowSplits.Load().(bool) { rangesBlocked.Store(ba.Header.RangeID, true) defer rangesBlocked.Delete(ba.Header.RangeID) select { case <-unblockCh: - return roachpb.NewError(errors.Errorf("splits disabled")) + return kvpb.NewError(errors.Errorf("splits disabled")) case <-ctx.Done(): <-ctx.Done() } diff --git a/pkg/kv/kvserver/client_replica_circuit_breaker_test.go b/pkg/kv/kvserver/client_replica_circuit_breaker_test.go index c1cd5b7c8fcf..fa69d60451c7 100644 --- a/pkg/kv/kvserver/client_replica_circuit_breaker_test.go +++ b/pkg/kv/kvserver/client_replica_circuit_breaker_test.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" @@ -215,7 +216,7 @@ func TestReplicaCircuitBreaker_LeaselessTripped(t *testing.T) { resumeHeartbeats() testutils.SucceedsSoon(t, func() error { err := tc.Read(n1) - if errors.HasType(err, (*roachpb.NotLeaseHolderError)(nil)) { + if errors.HasType(err, (*kvpb.NotLeaseHolderError)(nil)) { // Retriable errors can occur when manipulating the liveness record in // preparation for requesting a lease, such as: // @@ -271,7 +272,7 @@ func TestReplicaCircuitBreaker_Leaseholder_QuorumLoss(t *testing.T) { tc.SetSlowThreshold(10 * time.Millisecond) { err := tc.Write(n1) - var ae *roachpb.AmbiguousResultError + var ae *kvpb.AmbiguousResultError require.True(t, errors.As(err, &ae), "%+v", err) t.Log(err) tc.RequireIsBreakerOpen(t, err) @@ -401,14 +402,14 @@ type dummyStream struct { Logf(string, ...interface{}) } ctx context.Context - recv chan *roachpb.RangeFeedEvent + recv chan *kvpb.RangeFeedEvent } func (s *dummyStream) Context() context.Context { return s.ctx } -func (s *dummyStream) Send(ev *roachpb.RangeFeedEvent) error { +func (s *dummyStream) Send(ev *kvpb.RangeFeedEvent) error { if ev.Val == nil && ev.Error == nil { s.t.Logf("%s: ignoring event: %v", s.name, ev) return nil @@ -434,13 +435,13 @@ func TestReplicaCircuitBreaker_RangeFeed(t *testing.T) { require.NoError(t, tc.Write(n1)) desc := tc.LookupRangeOrFatal(t, tc.ScratchRange(t)) - args := &roachpb.RangeFeedRequest{ + args := &kvpb.RangeFeedRequest{ Span: roachpb.Span{Key: desc.StartKey.AsRawKey(), EndKey: desc.EndKey.AsRawKey()}, } ctx, cancel := context.WithCancel(ctx) defer cancel() - stream1 := &dummyStream{t: t, ctx: ctx, name: "rangefeed1", recv: make(chan *roachpb.RangeFeedEvent)} + stream1 := &dummyStream{t: t, ctx: ctx, name: "rangefeed1", recv: make(chan *kvpb.RangeFeedEvent)} require.NoError(t, tc.Stopper().RunAsyncTask(ctx, "stream1", func(ctx context.Context) { err := tc.repls[0].RangeFeed(args, stream1, nil /* pacer */).GoError() if ctx.Err() != nil { @@ -494,7 +495,7 @@ func TestReplicaCircuitBreaker_RangeFeed(t *testing.T) { // Start another stream during the "outage" to make sure it isn't rejected by // the breaker. - stream2 := &dummyStream{t: t, ctx: ctx, name: "rangefeed2", recv: make(chan *roachpb.RangeFeedEvent)} + stream2 := &dummyStream{t: t, ctx: ctx, name: "rangefeed2", recv: make(chan *kvpb.RangeFeedEvent)} require.NoError(t, tc.Stopper().RunAsyncTask(ctx, "stream2", func(ctx context.Context) { err := tc.repls[0].RangeFeed(args, stream2, nil /* pacer */).GoError() if ctx.Err() != nil { @@ -512,7 +513,7 @@ func TestReplicaCircuitBreaker_RangeFeed(t *testing.T) { // return a circuit breaker error, but in theory it could also never have // tried to acquire a lease, in which case it might return a value as well. if err := readOneVal(ctx, stream2, testutils.DefaultSucceedsSoonDuration); err != nil { - if !tc.IsBreakerOpen(err) && !errors.HasType(err, (*roachpb.RangeFeedRetryError)(nil)) { + if !tc.IsBreakerOpen(err) && !errors.HasType(err, (*kvpb.RangeFeedRetryError)(nil)) { t.Fatalf("%+v", err) } } @@ -530,32 +531,32 @@ func TestReplicaCircuitBreaker_ExemptRequests(t *testing.T) { tc.SetProbeEnabled(n1, false) tc.TripBreaker(n1) - exemptRequests := []func() roachpb.Request{ - func() roachpb.Request { return &roachpb.ExportRequest{} }, - func() roachpb.Request { + exemptRequests := []func() kvpb.Request{ + func() kvpb.Request { return &kvpb.ExportRequest{} }, + func() kvpb.Request { sstFile := &storage.MemFile{} sst := storage.MakeIngestionSSTWriter(context.Background(), cluster.MakeTestingClusterSettings(), sstFile) defer sst.Close() require.NoError(t, sst.LogData([]byte("hello"))) require.NoError(t, sst.Finish()) - addReq := &roachpb.AddSSTableRequest{ + addReq := &kvpb.AddSSTableRequest{ Data: sstFile.Data(), IngestAsWrites: true, } return addReq }, - func() roachpb.Request { - return &roachpb.RevertRangeRequest{TargetTime: tc.Servers[0].Clock().Now()} + func() kvpb.Request { + return &kvpb.RevertRangeRequest{TargetTime: tc.Servers[0].Clock().Now()} }, - func() roachpb.Request { - return &roachpb.GCRequest{} + func() kvpb.Request { + return &kvpb.GCRequest{} }, - func() roachpb.Request { - return &roachpb.ClearRangeRequest{} + func() kvpb.Request { + return &kvpb.ClearRangeRequest{} }, - func() roachpb.Request { - return &roachpb.ProbeRequest{} + func() kvpb.Request { + return &kvpb.ProbeRequest{} }, } @@ -575,7 +576,7 @@ func TestReplicaCircuitBreaker_ExemptRequests(t *testing.T) { // tripped. That's why there is a retry loop here. testutils.SucceedsSoon(t, func() error { err := tc.Send(n1, req) - if errors.HasType(err, (*roachpb.NotLeaseHolderError)(nil)) { + if errors.HasType(err, (*kvpb.NotLeaseHolderError)(nil)) { return err } require.NoError(t, err) @@ -589,7 +590,7 @@ func TestReplicaCircuitBreaker_ExemptRequests(t *testing.T) { for _, reqFn := range exemptRequests { req := reqFn() tc.Run(t, fmt.Sprintf("with-unavailable-lease/%s", req.Method()), func(t *testing.T) { - if m := req.Method(); m == roachpb.Probe { + if m := req.Method(); m == kvpb.Probe { // Probe does not require the lease, and is the most-tested of the bunch // already. We don't have to test it again here, which would require undue // amounts of special-casing below. @@ -626,7 +627,7 @@ func TestReplicaCircuitBreaker_ExemptRequests(t *testing.T) { for _, reqFn := range exemptRequests { req := reqFn() tc.Run(t, fmt.Sprintf("with-poisoned-latch/%s", req.Method()), func(t *testing.T) { - if m := req.Method(); m == roachpb.GC { + if m := req.Method(); m == kvpb.GC { // GC without GCKeys acquires no latches and is a pure read. If we want // to put a key in there, we need to pick the right timestamp (since you // can't GC a live key); it's all rather annoying and not worth it. In @@ -691,7 +692,7 @@ func setupCircuitBreakerTest(t *testing.T) *circuitBreakerTest { slowThresh := &atomic.Value{} // supports .SetSlowThreshold(x) slowThresh.Store(time.Duration(0)) storeKnobs := &kvserver.StoreTestingKnobs{ - SlowReplicationThresholdOverride: func(ba *roachpb.BatchRequest) time.Duration { + SlowReplicationThresholdOverride: func(ba *kvpb.BatchRequest) time.Duration { t.Helper() if rid := roachpb.RangeID(atomic.LoadInt64(&rangeID)); rid == 0 || ba == nil || ba.RangeID != rid { return 0 @@ -709,7 +710,7 @@ func setupCircuitBreakerTest(t *testing.T) *circuitBreakerTest { // The TestingApplyCalledTwiceFilter prevents n2 from requesting a lease (or from the lease // being transferred to n2). The test seems to pass pretty reliably without this // but it can't hurt. - TestingApplyCalledTwiceFilter: func(args kvserverbase.ApplyFilterArgs) (int, *roachpb.Error) { + TestingApplyCalledTwiceFilter: func(args kvserverbase.ApplyFilterArgs) (int, *kvpb.Error) { if !args.IsLeaseRequest { return 0, nil } @@ -720,7 +721,7 @@ func setupCircuitBreakerTest(t *testing.T) *circuitBreakerTest { if lease.Replica.NodeID != 2 { return 0, nil } - pErr := roachpb.NewErrorf("test prevents lease acquisition by n2") + pErr := kvpb.NewErrorf("test prevents lease acquisition by n2") return 0, pErr }, } @@ -811,7 +812,7 @@ func (cbt *circuitBreakerTest) UntripsSoon(t *testing.T, method func(idx int) er // current lease is repl=(n1,s1):1 seq=1 start=0,0 epo=1 pro=[...] if err != nil && !errors.Is(err, circuit.ErrBreakerOpen) && - !errors.HasType(err, (*roachpb.NotLeaseHolderError)(nil)) { + !errors.HasType(err, (*kvpb.NotLeaseHolderError)(nil)) { t.Fatalf("saw unexpected error %+v", err) } @@ -865,17 +866,17 @@ func (cbt *circuitBreakerTest) ExpireAllLeasesAndN1LivenessRecord( } } -func (cbt *circuitBreakerTest) Send(idx int, req roachpb.Request) error { +func (cbt *circuitBreakerTest) Send(idx int, req kvpb.Request) error { cbt.t.Helper() return cbt.SendCtx(context.Background(), idx, req) } -func (cbt *circuitBreakerTest) SendCtx(ctx context.Context, idx int, req roachpb.Request) error { +func (cbt *circuitBreakerTest) SendCtx(ctx context.Context, idx int, req kvpb.Request) error { return cbt.SendCtxTS(ctx, idx, req, cbt.repls[idx].Clock().Now()) } func (cbt *circuitBreakerTest) SendCtxTS( - ctx context.Context, idx int, req roachpb.Request, ts hlc.Timestamp, + ctx context.Context, idx int, req kvpb.Request, ts hlc.Timestamp, ) error { cbt.t.Helper() ctx, finishAndGet := tracing.ContextWithRecordingSpan(ctx, cbt.repls[idx].Tracer, "SendCtx("+req.Method().String()+")") @@ -888,14 +889,14 @@ func (cbt *circuitBreakerTest) SendCtxTS( rec := finishAndGet() cbt.t.Logf("%s", rec) }() - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} repl := cbt.repls[idx] ba.RangeID = repl.Desc().RangeID ba.Timestamp = ts ba.Add(req) if h := req.Header(); len(h.Key) == 0 { h.Key = repl.Desc().StartKey.AsRawKey() - if roachpb.IsRange(req) { + if kvpb.IsRange(req) { h.EndKey = repl.Desc().EndKey.AsRawKey() } req.SetHeader(h) @@ -913,7 +914,7 @@ func (cbt *circuitBreakerTest) SendCtxTS( // absence of actual error. This makes sure we don't accidentally pass tests as // a result of our context cancellation. if err := ctx.Err(); err != nil && parCtx.Err() == nil { - pErr = roachpb.NewErrorf("timed out waiting for batch response: %v", pErr) + pErr = kvpb.NewErrorf("timed out waiting for batch response: %v", pErr) } return pErr.GoError() @@ -921,15 +922,13 @@ func (cbt *circuitBreakerTest) SendCtxTS( func (cbt *circuitBreakerTest) WriteDS(idx int) error { cbt.t.Helper() - put := roachpb.NewPut(cbt.repls[idx].Desc().StartKey.AsRawKey(), roachpb.MakeValueFromString("hello")) + put := kvpb.NewPut(cbt.repls[idx].Desc().StartKey.AsRawKey(), roachpb.MakeValueFromString("hello")) return cbt.sendViaDistSender(cbt.Servers[idx].DistSender(), put) } -func (cbt *circuitBreakerTest) sendViaDistSender( - ds *kvcoord.DistSender, req roachpb.Request, -) error { +func (cbt *circuitBreakerTest) sendViaDistSender(ds *kvcoord.DistSender, req kvpb.Request) error { cbt.t.Helper() - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Add(req) ctx, cancel := context.WithTimeout(context.Background(), testutils.DefaultSucceedsSoonDuration) defer cancel() @@ -938,7 +937,7 @@ func (cbt *circuitBreakerTest) sendViaDistSender( // absence of actual error. This makes sure we don't accidentally pass tests as // a result of our context cancellation. if err := ctx.Err(); err != nil { - pErr = roachpb.NewErrorf("timed out waiting for batch response: %v", pErr) + pErr = kvpb.NewErrorf("timed out waiting for batch response: %v", pErr) } return pErr.GoError() } @@ -950,7 +949,7 @@ func (*circuitBreakerTest) IsBreakerOpen(err error) bool { if !errors.Is(err, circuit.ErrBreakerOpen) { return false } - return errors.HasType(err, (*roachpb.ReplicaUnavailableError)(nil)) + return errors.HasType(err, (*kvpb.ReplicaUnavailableError)(nil)) } func (cbt *circuitBreakerTest) RequireIsBreakerOpen(t *testing.T, err error) { @@ -965,7 +964,7 @@ func (cbt *circuitBreakerTest) RequireIsBreakerOpen(t *testing.T, err error) { func (*circuitBreakerTest) RequireIsNotLeaseholderError(t *testing.T, err error) { t.Helper() t.Log(err) - ok := errors.HasType(err, (*roachpb.NotLeaseHolderError)(nil)) + ok := errors.HasType(err, (*kvpb.NotLeaseHolderError)(nil)) require.True(t, ok, "%+v", err) } @@ -980,7 +979,7 @@ func (cbt *circuitBreakerTest) Write(idx int) error { cbt.t.Helper() repl := cbt.repls[idx] cbt.seq++ - put := roachpb.NewPut( + put := kvpb.NewPut( repl.Desc().StartKey.AsRawKey(), roachpb.MakeValueFromString(fmt.Sprintf("hello-%d", cbt.seq)), ) return cbt.Send(idx, put) @@ -989,14 +988,14 @@ func (cbt *circuitBreakerTest) Write(idx int) error { func (cbt *circuitBreakerTest) Read(idx int) error { cbt.t.Helper() repl := cbt.repls[idx] - get := roachpb.NewGet(repl.Desc().StartKey.AsRawKey(), false /* forUpdate */) + get := kvpb.NewGet(repl.Desc().StartKey.AsRawKey(), false /* forUpdate */) return cbt.Send(idx, get) } func (cbt *circuitBreakerTest) FollowerRead(idx int) error { cbt.t.Helper() repl := cbt.repls[idx] - get := roachpb.NewGet(repl.Desc().StartKey.AsRawKey(), false /* forUpdate */) + get := kvpb.NewGet(repl.Desc().StartKey.AsRawKey(), false /* forUpdate */) ctx := context.Background() ts := repl.GetCurrentClosedTimestamp(ctx) return cbt.SendCtxTS(ctx, idx, get, ts) diff --git a/pkg/kv/kvserver/client_replica_gc_test.go b/pkg/kv/kvserver/client_replica_gc_test.go index 0cc59f2544c9..53c9e04e9d21 100644 --- a/pkg/kv/kvserver/client_replica_gc_test.go +++ b/pkg/kv/kvserver/client_replica_gc_test.go @@ -17,6 +17,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -48,8 +49,8 @@ func TestReplicaGCQueueDropReplicaDirect(t *testing.T) { // Node. We use the TestingEvalFilter to make sure that the second Node // waits for the first. testKnobs.EvalKnobs.TestingEvalFilter = - func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { - et, ok := filterArgs.Req.(*roachpb.EndTxnRequest) + func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { + et, ok := filterArgs.Req.(*kvpb.EndTxnRequest) if !ok || filterArgs.Sid != 2 { return nil } diff --git a/pkg/kv/kvserver/client_replica_test.go b/pkg/kv/kvserver/client_replica_test.go index 42085ff35837..20725a55e807 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedcache" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" @@ -120,13 +121,13 @@ func TestReplicaClockUpdates(t *testing.T) { // Pick a timestamp in the future of all nodes by less than the // MaxOffset. Set the synthetic flag according to the test case. reqTS := clocks[0].Now().Add(clocks[0].MaxOffset().Nanoseconds()/2, 0).WithSynthetic(synthetic) - h := roachpb.Header{Timestamp: reqTS} + h := kvpb.Header{Timestamp: reqTS} if !reqTS.Synthetic { h.Now = hlc.ClockTimestamp(reqTS) } // Execute the command. - var req roachpb.Request + var req kvpb.Request if write { req = incrementArgs(reqKey, 5) } else { @@ -186,7 +187,7 @@ func TestLeaseholdersRejectClockUpdateWithJump(t *testing.T) { clockOffset := s.Clock().MaxOffset() / numCmds for i := int64(1); i <= numCmds; i++ { ts := hlc.ClockTimestamp(ts1.Add(i*clockOffset.Nanoseconds(), 0)) - if _, err := kv.SendWrappedWith(context.Background(), store.TestSender(), roachpb.Header{Now: ts}, incArgs); err != nil { + if _, err := kv.SendWrappedWith(context.Background(), store.TestSender(), kvpb.Header{Now: ts}, incArgs); err != nil { t.Fatal(err) } } @@ -198,7 +199,7 @@ func TestLeaseholdersRejectClockUpdateWithJump(t *testing.T) { // Once the accumulated offset reaches MaxOffset, commands will be rejected. tsFuture := hlc.ClockTimestamp(ts1.Add(s.Clock().MaxOffset().Nanoseconds()+1, 0)) - _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{Now: tsFuture}, incArgs) + _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{Now: tsFuture}, incArgs) if !testutils.IsPError(pErr, "remote wall time is too far ahead") { t.Fatalf("unexpected error %v", pErr) } @@ -258,12 +259,12 @@ func TestTxnPutOutOfOrder(t *testing.T) { // Set up a filter to so that the get operation at Step 3 will return an error. var shouldFailGet atomic.Value - testingEvalFilter := func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { - if _, ok := filterArgs.Req.(*roachpb.GetRequest); ok && + testingEvalFilter := func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { + if _, ok := filterArgs.Req.(*kvpb.GetRequest); ok && filterArgs.Req.Header().Key.Equal(roachpb.Key(key)) && filterArgs.Hdr.Txn == nil { if shouldFail := shouldFailGet.Load(); shouldFail != nil && shouldFail.(bool) { - return roachpb.NewErrorWithTxn(errors.Errorf("Test"), filterArgs.Hdr.Txn) + return kvpb.NewErrorWithTxn(errors.Errorf("Test"), filterArgs.Hdr.Txn) } } return nil @@ -369,21 +370,21 @@ func TestTxnPutOutOfOrder(t *testing.T) { manual.Increment(100) priority := roachpb.UserPriority(-math.MaxInt32) - requestHeader := roachpb.RequestHeader{ + requestHeader := kvpb.RequestHeader{ Key: roachpb.Key(key), } - h := roachpb.Header{ + h := kvpb.Header{ Timestamp: s.Clock().Now(), UserPriority: priority, } if _, err := kv.SendWrappedWith( - context.Background(), store.TestSender(), h, &roachpb.GetRequest{RequestHeader: requestHeader}, + context.Background(), store.TestSender(), h, &kvpb.GetRequest{RequestHeader: requestHeader}, ); err != nil { t.Fatalf("failed to get: %+v", err) } // Write to the restart key so that the Writer's txn must restart. - putReq := &roachpb.PutRequest{ - RequestHeader: roachpb.RequestHeader{Key: roachpb.Key(restartKey)}, + putReq := &kvpb.PutRequest{ + RequestHeader: kvpb.RequestHeader{Key: roachpb.Key(restartKey)}, Value: roachpb.MakeValueFromBytes([]byte("restart-value")), } if _, err := kv.SendWrappedWith(context.Background(), store.TestSender(), h, putReq); err != nil { @@ -403,7 +404,7 @@ func TestTxnPutOutOfOrder(t *testing.T) { h.Timestamp = s.Clock().Now() shouldFailGet.Store(true) if _, err := kv.SendWrappedWith( - context.Background(), store.TestSender(), h, &roachpb.GetRequest{RequestHeader: requestHeader}, + context.Background(), store.TestSender(), h, &kvpb.GetRequest{RequestHeader: requestHeader}, ); err == nil { t.Fatal("unexpected success of get") } @@ -472,7 +473,7 @@ func TestTxnReadWithinUncertaintyInterval(t *testing.T) { // timestamp below the value, collect one now. if observedTS { get := getArgs(key) - resp, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{Txn: &txn}, get) + resp, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{Txn: &txn}, get) require.Nil(t, pErr) txn.Update(resp.Header().Txn) require.Len(t, txn.ObservedTimestamps, 1) @@ -487,12 +488,12 @@ func TestTxnReadWithinUncertaintyInterval(t *testing.T) { // had collected an observed timestamp, it may or may not observe the value // in its uncertainty interval and throw an error. get := getArgs(key) - _, pErr = kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{Txn: &txn}, get) + _, pErr = kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{Txn: &txn}, get) if observedTS { require.Nil(t, pErr) } else { require.NotNil(t, pErr) - require.IsType(t, &roachpb.ReadWithinUncertaintyIntervalError{}, pErr.GetDetail()) + require.IsType(t, &kvpb.ReadWithinUncertaintyIntervalError{}, pErr.GetDetail()) } }) } @@ -611,7 +612,7 @@ func testTxnReadWithinUncertaintyIntervalAfterIntentResolution( // Write to key A and key B in the writer transaction. for _, key := range []roachpb.Key{keyA, keyB} { put := putArgs(key, []byte("val")) - resp, pErr := kv.SendWrappedWith(ctx, tc.Servers[0].DistSender(), roachpb.Header{Txn: &writerTxn}, put) + resp, pErr := kv.SendWrappedWith(ctx, tc.Servers[0].DistSender(), kvpb.Header{Txn: &writerTxn}, put) require.Nil(t, pErr) writerTxn.Update(resp.Header().Txn) } @@ -664,25 +665,25 @@ func testTxnReadWithinUncertaintyIntervalAfterIntentResolution( // transactions are always an observed timestamp from their own gateway node. for i, key := range []roachpb.Key{keyB, keyA} { get := getArgs(key.Next()) - resp, pErr := kv.SendWrappedWith(ctx, tc.Servers[1].DistSender(), roachpb.Header{Txn: &readerTxn}, get) + resp, pErr := kv.SendWrappedWith(ctx, tc.Servers[1].DistSender(), kvpb.Header{Txn: &readerTxn}, get) require.Nil(t, pErr) - require.Nil(t, resp.(*roachpb.GetResponse).Value) + require.Nil(t, resp.(*kvpb.GetResponse).Value) readerTxn.Update(resp.Header().Txn) require.Len(t, readerTxn.ObservedTimestamps, i+1) } // Resolve the intent on key B zero, one, or two times. { - resolveIntentArgs := func(status roachpb.TransactionStatus) roachpb.Request { + resolveIntentArgs := func(status roachpb.TransactionStatus) kvpb.Request { if rangedResolution { - return &roachpb.ResolveIntentRangeRequest{ - RequestHeader: roachpb.RequestHeader{Key: keyB, EndKey: keyB.Next()}, + return &kvpb.ResolveIntentRangeRequest{ + RequestHeader: kvpb.RequestHeader{Key: keyB, EndKey: keyB.Next()}, IntentTxn: writerTxn.TxnMeta, Status: status, } } else { - return &roachpb.ResolveIntentRequest{ - RequestHeader: roachpb.RequestHeader{Key: keyB}, + return &kvpb.ResolveIntentRequest{ + RequestHeader: kvpb.RequestHeader{Key: keyB}, IntentTxn: writerTxn.TxnMeta, Status: status, } @@ -732,9 +733,9 @@ func testTxnReadWithinUncertaintyIntervalAfterIntentResolution( // ReadWithinUncertaintyIntervalErrors. for _, key := range []roachpb.Key{keyA, keyB} { get := getArgs(key) - _, pErr := kv.SendWrappedWith(ctx, tc.Servers[0].DistSender(), roachpb.Header{Txn: &readerTxn}, get) + _, pErr := kv.SendWrappedWith(ctx, tc.Servers[0].DistSender(), kvpb.Header{Txn: &readerTxn}, get) require.NotNil(t, pErr) - var rwuiErr *roachpb.ReadWithinUncertaintyIntervalError + var rwuiErr *kvpb.ReadWithinUncertaintyIntervalError require.True(t, errors.As(pErr.GetDetail(), &rwuiErr)) require.Equal(t, readerTxn.ReadTimestamp, rwuiErr.ReadTimestamp) require.Equal(t, readerTxn.GlobalUncertaintyLimit, rwuiErr.GlobalUncertaintyLimit) @@ -813,7 +814,7 @@ func TestTxnReadWithinUncertaintyIntervalAfterLeaseTransfer(t *testing.T) { // Collect an observed timestamp in that transaction from node 2. getB := getArgs(keyB) - resp, pErr := kv.SendWrappedWith(ctx, tc.Servers[1].DistSender(), roachpb.Header{Txn: &txn}, getB) + resp, pErr := kv.SendWrappedWith(ctx, tc.Servers[1].DistSender(), kvpb.Header{Txn: &txn}, getB) require.Nil(t, pErr) txn.Update(resp.Header().Txn) require.Len(t, txn.ObservedTimestamps, 1) @@ -836,7 +837,7 @@ func TestTxnReadWithinUncertaintyIntervalAfterLeaseTransfer(t *testing.T) { // flakiness. For now, we just re-order the operations and assert that we // receive an uncertainty error even though its absence would not be a true // stale read. - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Add(putArgs(keyA, []byte("val"))) br, pErr := tc.Servers[0].DistSender().Send(ctx, ba) require.Nil(t, pErr) @@ -860,9 +861,9 @@ func TestTxnReadWithinUncertaintyIntervalAfterLeaseTransfer(t *testing.T) { // avoid the uncertainty error. This is a good thing, as doing so would allow // for a stale read. getA := getArgs(keyA) - _, pErr = kv.SendWrappedWith(ctx, tc.Servers[1].DistSender(), roachpb.Header{Txn: &txn}, getA) + _, pErr = kv.SendWrappedWith(ctx, tc.Servers[1].DistSender(), kvpb.Header{Txn: &txn}, getA) require.NotNil(t, pErr) - require.IsType(t, &roachpb.ReadWithinUncertaintyIntervalError{}, pErr.GetDetail()) + require.IsType(t, &kvpb.ReadWithinUncertaintyIntervalError{}, pErr.GetDetail()) } // TestTxnReadWithinUncertaintyIntervalAfterRangeMerge verifies that on a merge of two @@ -998,7 +999,7 @@ func TestTxnReadWithinUncertaintyIntervalAfterRangeMerge(t *testing.T) { txn2 := roachpb.MakeTransaction("txn2", keyA, 1, now, maxOffset, instanceId) // Simulate a read which will cause the observed time to be set to now - resp, pErr := kv.SendWrappedWith(ctx, tc.Servers[1].DistSender(), roachpb.Header{Txn: &txn}, getArgs(keyA)) + resp, pErr := kv.SendWrappedWith(ctx, tc.Servers[1].DistSender(), kvpb.Header{Txn: &txn}, getArgs(keyA)) require.Nil(t, pErr) // The client needs to update its transaction to the returned transaction which has observed timestamps in it txn = *resp.Header().Txn @@ -1024,8 +1025,8 @@ func TestTxnReadWithinUncertaintyIntervalAfterRangeMerge(t *testing.T) { // Try and read the transaction from the context of a new transaction. This // will fail as expected as the observed timestamp will not be set. - _, pErr = kv.SendWrappedWith(ctx, tc.Servers[0].DistSender(), roachpb.Header{Txn: &txn2}, getArgs(keyC)) - require.IsType(t, &roachpb.ReadWithinUncertaintyIntervalError{}, pErr.GetDetail()) + _, pErr = kv.SendWrappedWith(ctx, tc.Servers[0].DistSender(), kvpb.Header{Txn: &txn2}, getArgs(keyC)) + require.IsType(t, &kvpb.ReadWithinUncertaintyIntervalError{}, pErr.GetDetail()) // Try and read the key from the existing transaction. This should fail the // same way. @@ -1034,8 +1035,8 @@ func TestTxnReadWithinUncertaintyIntervalAfterRangeMerge(t *testing.T) { // - Other error (Bad) - We expect an uncertainty error so the client can choose a new timestamp and retry. // - Not found (Bad) - Error because the data was written before us. // - Found (Bad) - The write HLC timestamp is after our timestamp. - _, pErr = kv.SendWrappedWith(ctx, tc.Servers[0].DistSender(), roachpb.Header{Txn: &txn}, getArgs(keyC)) - require.IsType(t, &roachpb.ReadWithinUncertaintyIntervalError{}, pErr.GetDetail()) + _, pErr = kv.SendWrappedWith(ctx, tc.Servers[0].DistSender(), kvpb.Header{Txn: &txn}, getArgs(keyC)) + require.IsType(t, &kvpb.ReadWithinUncertaintyIntervalError{}, pErr.GetDetail()) } testutils.RunTrueAndFalse(t, "alignLeaseholders", func(t *testing.T, alignLeaseholders bool) { @@ -1069,7 +1070,7 @@ func TestNonTxnReadWithinUncertaintyIntervalAfterLeaseTransfer(t *testing.T) { type nonTxnGetKey struct{} nonTxnOrigTsC := make(chan hlc.Timestamp, 1) nonTxnBlockerC := make(chan struct{}) - requestFilter := func(ctx context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + requestFilter := func(ctx context.Context, ba *kvpb.BatchRequest) *kvpb.Error { if ctx.Value(nonTxnGetKey{}) != nil { // Give the test the server-assigned timestamp. require.NotNil(t, ba.TimestampFromServerClock) @@ -1084,9 +1085,9 @@ func TestNonTxnReadWithinUncertaintyIntervalAfterLeaseTransfer(t *testing.T) { return nil } var uncertaintyErrs int32 - concurrencyRetryFilter := func(ctx context.Context, _ *roachpb.BatchRequest, pErr *roachpb.Error) { + concurrencyRetryFilter := func(ctx context.Context, _ *kvpb.BatchRequest, pErr *kvpb.Error) { if ctx.Value(nonTxnGetKey{}) != nil { - if _, ok := pErr.GetDetail().(*roachpb.ReadWithinUncertaintyIntervalError); ok { + if _, ok := pErr.GetDetail().(*kvpb.ReadWithinUncertaintyIntervalError); ok { atomic.AddInt32(&uncertaintyErrs, 1) } } @@ -1140,13 +1141,13 @@ func TestNonTxnReadWithinUncertaintyIntervalAfterLeaseTransfer(t *testing.T) { // consulted the lease. We'll transfer the lease to node 2 before the request // checks, so that it ends up evaluating on node 2. type resp struct { - *roachpb.BatchResponse - *roachpb.Error + *kvpb.BatchResponse + *kvpb.Error } nonTxnRespC := make(chan resp, 1) _ = tc.Stopper().RunAsyncTask(ctx, "non-txn get", func(ctx context.Context) { ctx = context.WithValue(ctx, nonTxnGetKey{}, "foo") - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.RangeID = desc.RangeID ba.Add(getArgs(key)) br, pErr := tc.GetFirstStoreFromServer(t, 1).Send(ctx, ba) @@ -1181,7 +1182,7 @@ func TestNonTxnReadWithinUncertaintyIntervalAfterLeaseTransfer(t *testing.T) { // possible that we could avoid flakiness. For now, we just re-order the // operations and assert that we observe an uncertainty error even though its // absence would not be a true stale read. - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Add(putArgs(key, []byte("val"))) br, pErr := tc.Servers[0].DistSender().Send(ctx, ba) require.Nil(t, pErr) @@ -1241,7 +1242,7 @@ func TestRangeLookupUseReverse(t *testing.T) { // Init test ranges: // ["","a"), ["a","c"), ["c","e"), ["e","g") and ["g","\xff\xff"). - splits := []*roachpb.AdminSplitRequest{ + splits := []*kvpb.AdminSplitRequest{ adminSplitArgs(roachpb.Key("g")), adminSplitArgs(roachpb.Key("e")), adminSplitArgs(roachpb.Key("c")), @@ -1261,8 +1262,8 @@ func TestRangeLookupUseReverse(t *testing.T) { } // Resolve the intents. - scanArgs := roachpb.ScanRequest{ - RequestHeader: roachpb.RequestHeader{ + scanArgs := kvpb.ScanRequest{ + RequestHeader: kvpb.RequestHeader{ Key: keys.RangeMetaKey(roachpb.RKeyMin.Next()).AsRawKey(), EndKey: keys.RangeMetaKey(roachpb.RKeyMax).AsRawKey(), }, @@ -1340,7 +1341,7 @@ func TestRangeLookupUseReverse(t *testing.T) { for _, test := range testCases { t.Run(fmt.Sprintf("key=%s", test.key), func(t *testing.T) { rs, preRs, err := kv.RangeLookup(context.Background(), store.TestSender(), - test.key.AsRawKey(), roachpb.READ_UNCOMMITTED, test.maxResults-1, true /* prefetchReverse */) + test.key.AsRawKey(), kvpb.READ_UNCOMMITTED, test.maxResults-1, true /* prefetchReverse */) if err != nil { t.Fatalf("LookupRange error: %+v", err) } @@ -1386,7 +1387,7 @@ func setupLeaseTransferTest(t *testing.T) *leaseTransferTest { manualClock: hlc.NewHybridManualClock(), } - testingEvalFilter := func(args kvserverbase.FilterArgs) *roachpb.Error { + testingEvalFilter := func(args kvserverbase.FilterArgs) *kvpb.Error { l.filterMu.Lock() filterCopy := l.evalFilter l.filterMu.Unlock() @@ -1396,7 +1397,7 @@ func setupLeaseTransferTest(t *testing.T) *leaseTransferTest { return nil } - testingProposalFilter := func(args kvserverbase.ProposalFilterArgs) *roachpb.Error { + testingProposalFilter := func(args kvserverbase.ProposalFilterArgs) *kvpb.Error { l.filterMu.Lock() filterCopy := l.propFilter l.filterMu.Unlock() @@ -1463,16 +1464,16 @@ func setupLeaseTransferTest(t *testing.T) *leaseTransferTest { return l } -func (l *leaseTransferTest) sendRead(t *testing.T, storeIdx int) *roachpb.Error { +func (l *leaseTransferTest) sendRead(t *testing.T, storeIdx int) *kvpb.Error { desc := l.tc.GetFirstStoreFromServer(t, storeIdx).LookupReplica(keys.MustAddr(l.leftKey)) replicaDesc, err := desc.GetReplicaDescriptor() if err != nil { - return roachpb.NewError(err) + return kvpb.NewError(err) } _, pErr := kv.SendWrappedWith( context.Background(), l.tc.GetFirstStoreFromServer(t, storeIdx).TestSender(), - roachpb.Header{RangeID: desc.RangeID, Replica: replicaDesc}, + kvpb.Header{RangeID: desc.RangeID, Replica: replicaDesc}, getArgs(l.leftKey), ) if pErr != nil { @@ -1503,11 +1504,11 @@ func (l *leaseTransferTest) setFilter(setTo bool, extensionSem chan struct{}) { l.evalFilter = nil return } - l.evalFilter = func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { + l.evalFilter = func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { if filterArgs.Sid != l.tc.Target(1).StoreID { return nil } - llReq, ok := filterArgs.Req.(*roachpb.RequestLeaseRequest) + llReq, ok := filterArgs.Req.(*kvpb.RequestLeaseRequest) if !ok { return nil } @@ -1540,7 +1541,7 @@ func (l *leaseTransferTest) forceLeaseExtension( // attempt fails because it's already been renewed. This used to work // before we compared the proposer's lease with the actual lease because // the renewed lease still encompassed the previous request. - if errors.HasType(err, (*roachpb.NotLeaseHolderError)(nil)) { + if errors.HasType(err, (*kvpb.NotLeaseHolderError)(nil)) { err = nil } return err @@ -1625,9 +1626,9 @@ func TestLeaseExpirationBasedRangeTransfer(t *testing.T) { // Check that replica0 doesn't serve reads any more. pErr := l.sendRead(t, 0) - nlhe, ok := pErr.GetDetail().(*roachpb.NotLeaseHolderError) + nlhe, ok := pErr.GetDetail().(*kvpb.NotLeaseHolderError) if !ok { - t.Fatalf("expected %T, got %s", &roachpb.NotLeaseHolderError{}, pErr) + t.Fatalf("expected %T, got %s", &kvpb.NotLeaseHolderError{}, pErr) } if !nlhe.Lease.Replica.Equal(&l.replica1Desc) { t.Fatalf("expected lease holder %+v, got %+v", @@ -1688,7 +1689,7 @@ func TestLeaseExpirationBasedRangeTransferWithExtension(t *testing.T) { // Transfer back from replica1 to replica0. err := l.replica1.AdminTransferLease(context.Background(), l.replica0Desc.StoreID, false /* bypassSafetyChecks */) // Ignore not leaseholder errors which can arise due to re-proposals. - if errors.HasType(err, (*roachpb.NotLeaseHolderError)(nil)) { + if errors.HasType(err, (*kvpb.NotLeaseHolderError)(nil)) { err = nil } transferErrCh <- err @@ -1725,9 +1726,9 @@ func TestLeaseExpirationBasedDrainTransfer(t *testing.T) { // Check that replica0 doesn't serve reads any more. pErr := l.sendRead(t, 0) - nlhe, ok := pErr.GetDetail().(*roachpb.NotLeaseHolderError) + nlhe, ok := pErr.GetDetail().(*kvpb.NotLeaseHolderError) if !ok { - t.Fatalf("expected %T, got %s", &roachpb.NotLeaseHolderError{}, pErr) + t.Fatalf("expected %T, got %s", &kvpb.NotLeaseHolderError{}, pErr) } if nlhe.Lease.Empty() || !nlhe.Lease.Replica.Equal(&l.replica1Desc) { t.Fatalf("expected lease holder %+v, got %+v", @@ -1812,12 +1813,12 @@ func TestLeaseExpirationBasedDrainTransferWithProscribed(t *testing.T) { l.propFilter = nil return } - l.propFilter = func(filterArgs kvserverbase.ProposalFilterArgs) *roachpb.Error { + l.propFilter = func(filterArgs kvserverbase.ProposalFilterArgs) *kvpb.Error { if filterArgs.Req.IsSingleTransferLeaseRequest() { target := filterArgs.Req.Requests[0].GetTransferLease().Lease.Replica if target == l.replica0Desc { failedOnce.Do(func() { close(failedCh) }) - return roachpb.NewError(kvserver.NewLeaseTransferRejectedBecauseTargetMayNeedSnapshotError( + return kvpb.NewError(kvserver.NewLeaseTransferRejectedBecauseTargetMayNeedSnapshotError( target, raftutil.ReplicaStateProbe)) } } @@ -1903,7 +1904,7 @@ func TestLeaseExpirationBelowFutureTimeRequest(t *testing.T) { // Issue a get with the request timestamp. args := getArgs(l.leftKey) - _, pErr := kv.SendWrappedWith(ctx, l.tc.GetFirstStoreFromServer(t, 1).TestSender(), roachpb.Header{ + _, pErr := kv.SendWrappedWith(ctx, l.tc.GetFirstStoreFromServer(t, 1).TestSender(), kvpb.Header{ RangeID: l.replica0.RangeID, Replica: l.replica1Desc, Timestamp: reqTime, }, args) @@ -2010,7 +2011,7 @@ func TestRangeLocalUncertaintyLimitAfterNewLease(t *testing.T) { // leaseholder. If the max timestamp were not being properly limited, // we would end up incorrectly reading nothing for keyA. Instead we // expect to see an uncertainty interval error. - h := roachpb.Header{Txn: &txn} + h := kvpb.Header{Txn: &txn} if _, pErr := kv.SendWrappedWith( ctx, tc.Servers[0].DistSender(), h, getArgs(keyA), ); !testutils.IsPError(pErr, "uncertainty") { @@ -2025,8 +2026,8 @@ func TestLeaseMetricsOnSplitAndTransfer(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) var injectLeaseTransferError atomic.Value - testingEvalFilter := func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { - if args, ok := filterArgs.Req.(*roachpb.TransferLeaseRequest); ok { + testingEvalFilter := func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { + if args, ok := filterArgs.Req.(*kvpb.TransferLeaseRequest); ok { if val := injectLeaseTransferError.Load(); val != nil && val.(bool) { // Note that we can't just return an error here as we only // end up counting failures in the metrics if the command @@ -2150,7 +2151,7 @@ func TestLeaseNotUsedAfterRestart(t *testing.T) { StickyEngineRegistry: stickyEngineRegistry, }, Store: &kvserver.StoreTestingKnobs{ - LeaseRequestEvent: func(ts hlc.Timestamp, _ roachpb.StoreID, _ roachpb.RangeID) *roachpb.Error { + LeaseRequestEvent: func(ts hlc.Timestamp, _ roachpb.StoreID, _ roachpb.RangeID) *kvpb.Error { val := leaseAcquisitionTrap.Load() if val == nil { return nil @@ -2223,7 +2224,7 @@ func TestLeaseExtensionNotBlockedByRead(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() readBlocked := make(chan struct{}) - cmdFilter := func(fArgs kvserverbase.FilterArgs) *roachpb.Error { + cmdFilter := func(fArgs kvserverbase.FilterArgs) *kvpb.Error { if fArgs.Hdr.UserPriority == 42 { // Signal that the read is blocked. readBlocked <- struct{}{} @@ -2254,13 +2255,13 @@ func TestLeaseExtensionNotBlockedByRead(t *testing.T) { key := roachpb.Key("a") errChan := make(chan error) go func() { - getReq := roachpb.GetRequest{ - RequestHeader: roachpb.RequestHeader{ + getReq := kvpb.GetRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, }, } if _, pErr := kv.SendWrappedWith(ctx, s.DB().NonTransactionalSender(), - roachpb.Header{UserPriority: 42}, + kvpb.Header{UserPriority: 42}, &getReq); pErr != nil { errChan <- pErr.GoError() } @@ -2284,8 +2285,8 @@ func TestLeaseExtensionNotBlockedByRead(t *testing.T) { t.Fatalf("replica descriptor for key %s not found", rKey) } - leaseReq := roachpb.RequestLeaseRequest{ - RequestHeader: roachpb.RequestHeader{ + leaseReq := kvpb.RequestLeaseRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, }, Lease: roachpb.Lease{ @@ -2303,11 +2304,11 @@ func TestLeaseExtensionNotBlockedByRead(t *testing.T) { leaseReq.PrevLease = leaseInfo.CurrentOrProspective() _, pErr := kv.SendWrapped(ctx, s.DB().NonTransactionalSender(), &leaseReq) - if _, ok := pErr.GetDetail().(*roachpb.AmbiguousResultError); ok { + if _, ok := pErr.GetDetail().(*kvpb.AmbiguousResultError); ok { log.Infof(ctx, "retrying lease after %s", pErr) continue } - if _, ok := pErr.GetDetail().(*roachpb.LeaseRejectedError); ok { + if _, ok := pErr.GetDetail().(*kvpb.LeaseRejectedError); ok { // Lease rejected? Try again. The extension should work because // extending is idempotent (assuming the PrevLease matches). log.Infof(ctx, "retrying lease after %s", pErr) @@ -2339,20 +2340,20 @@ func validateLeaseholderSoon( func getLeaseInfoOrFatal( t *testing.T, ctx context.Context, db *kv.DB, key roachpb.Key, -) *roachpb.LeaseInfoResponse { - header := roachpb.Header{ +) *kvpb.LeaseInfoResponse { + header := kvpb.Header{ // INCONSISTENT read with a NEAREST routing policy, since we want to make // sure that the node used to send this is the one that processes the // command, regardless of whether it is the leaseholder. - ReadConsistency: roachpb.INCONSISTENT, - RoutingPolicy: roachpb.RoutingPolicy_NEAREST, + ReadConsistency: kvpb.INCONSISTENT, + RoutingPolicy: kvpb.RoutingPolicy_NEAREST, } - leaseInfoReq := &roachpb.LeaseInfoRequest{RequestHeader: roachpb.RequestHeader{Key: key}} + leaseInfoReq := &kvpb.LeaseInfoRequest{RequestHeader: kvpb.RequestHeader{Key: key}} reply, pErr := kv.SendWrappedWith(ctx, db.NonTransactionalSender(), header, leaseInfoReq) if pErr != nil { t.Fatal(pErr) } - return reply.(*roachpb.LeaseInfoResponse) + return reply.(*kvpb.LeaseInfoResponse) } func TestRemoveLeaseholder(t *testing.T) { @@ -2446,20 +2447,20 @@ func TestLeaseInfoRequest(t *testing.T) { if err != nil { t.Fatal(err) } - leaseInfoReq := &roachpb.LeaseInfoRequest{ - RequestHeader: roachpb.RequestHeader{ + leaseInfoReq := &kvpb.LeaseInfoRequest{ + RequestHeader: kvpb.RequestHeader{ Key: rangeDesc.StartKey.AsRawKey(), }, } reply, pErr := kv.SendWrappedWith( - context.Background(), s, roachpb.Header{ + context.Background(), s, kvpb.Header{ RangeID: rangeDesc.RangeID, - ReadConsistency: roachpb.INCONSISTENT, + ReadConsistency: kvpb.INCONSISTENT, }, leaseInfoReq) if pErr != nil { t.Fatal(pErr) } - resp := *(reply.(*roachpb.LeaseInfoResponse)) + resp := *(reply.(*kvpb.LeaseInfoResponse)) if !resp.Lease.Replica.Equal(replicas[2]) { t.Fatalf("lease holder should be replica %s, but is: %s", replicas[2], resp.Lease.Replica) @@ -2479,9 +2480,9 @@ func TestLeaseInfoRequest(t *testing.T) { func TestErrorHandlingForNonKVCommand(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - cmdFilter := func(fArgs kvserverbase.FilterArgs) *roachpb.Error { + cmdFilter := func(fArgs kvserverbase.FilterArgs) *kvpb.Error { if fArgs.Hdr.UserPriority == 42 { - return roachpb.NewErrorf("injected error") + return kvpb.NewErrorf("injected error") } return nil } @@ -2500,15 +2501,15 @@ func TestErrorHandlingForNonKVCommand(t *testing.T) { // Send the lease request. key := roachpb.Key("a") - leaseReq := roachpb.LeaseInfoRequest{ - RequestHeader: roachpb.RequestHeader{ + leaseReq := kvpb.LeaseInfoRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, }, } _, pErr := kv.SendWrappedWith( context.Background(), s.DB().NonTransactionalSender(), - roachpb.Header{UserPriority: 42}, + kvpb.Header{UserPriority: 42}, &leaseReq, ) if !testutils.IsPError(pErr, "injected error") { @@ -2565,16 +2566,16 @@ func TestRangeInfoAfterSplit(t *testing.T) { } for _, tc := range tests { t.Run(tc.name, func(t *testing.T) { - ba := &roachpb.BatchRequest{ - Header: roachpb.Header{ + ba := &kvpb.BatchRequest{ + Header: kvpb.Header{ RangeID: tc.rangeID, ClientRangeInfo: roachpb.ClientRangeInfo{ DescriptorGeneration: preSplitDesc.Generation, }, }, } - gArgs := &roachpb.GetRequest{ - RequestHeader: roachpb.RequestHeader{ + gArgs := &kvpb.GetRequest{ + RequestHeader: kvpb.RequestHeader{ Key: tc.key.AsRawKey(), }, } @@ -2614,7 +2615,7 @@ func TestDrainRangeRejection(t *testing.T) { drainingIdx := 1 tc.GetFirstStoreFromServer(t, 1).SetDraining(true, nil /* reporter */, false /* verbose */) - chgs := roachpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(drainingIdx)) + chgs := kvpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(drainingIdx)) if _, err := repl.ChangeReplicas(context.Background(), repl.Desc(), kvserverpb.SnapshotRequest_REBALANCE, kvserverpb.ReasonRangeUnderReplicated, "", chgs); !testutils.IsError(err, "store is draining") { t.Fatalf("unexpected error: %+v", err) } @@ -2634,7 +2635,7 @@ func TestChangeReplicasGeneration(t *testing.T) { repl := tc.GetFirstStoreFromServer(t, 0).LookupReplica(roachpb.RKey(key)) oldGeneration := repl.Desc().Generation - chgs := roachpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(1)) + chgs := kvpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(1)) if _, err := repl.ChangeReplicas(context.Background(), repl.Desc(), kvserverpb.SnapshotRequest_REBALANCE, kvserverpb.ReasonRangeUnderReplicated, "", chgs); err != nil { t.Fatalf("unexpected error: %v", err) } @@ -2783,8 +2784,8 @@ func TestClearRange(t *testing.T) { clearRange := func(start, end roachpb.Key) { t.Helper() - if _, err := kv.SendWrapped(ctx, store.DB().NonTransactionalSender(), &roachpb.ClearRangeRequest{ - RequestHeader: roachpb.RequestHeader{ + if _, err := kv.SendWrapped(ctx, store.DB().NonTransactionalSender(), &kvpb.ClearRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: start, EndKey: end, }, @@ -2909,7 +2910,7 @@ func TestLeaseTransferInSnapshotUpdatesTimestampCache(t *testing.T) { // Read the key at readTS. // NB: don't use SendWrapped because we want access to br.Timestamp. - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Timestamp = readTS ba.Add(getArgs(keyA)) br, pErr := tc.Servers[0].DistSender().Send(ctx, ba) @@ -2974,7 +2975,7 @@ func TestLeaseTransferInSnapshotUpdatesTimestampCache(t *testing.T) { // snapshot had an empty timestamp cache and would simply let us write // under the previous read. // NB: don't use SendWrapped because we want access to br.Timestamp. - ba = &roachpb.BatchRequest{} + ba = &kvpb.BatchRequest{} ba.Timestamp = readTS ba.Add(incrementArgs(keyA, 1)) br, pErr = tc.Servers[0].DistSender().Send(ctx, ba) @@ -3013,7 +3014,7 @@ func TestLeaseTransferRejectedIfTargetNeedsSnapshot(t *testing.T) { ServerArgs: base.TestServerArgs{ Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(ctx context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + TestingRequestFilter: func(ctx context.Context, ba *kvpb.BatchRequest) *kvpb.Error { if rejectAfterRevoke && ba.IsSingleTransferLeaseRequest() { transferLeaseReqBlockOnce.Do(func() { close(transferLeaseReqBlockedC) @@ -3185,12 +3186,12 @@ func TestConcurrentAdminChangeReplicasRequests(t *testing.T) { wg.Add(2) go func() { res1, err1 = db.AdminChangeReplicas( - ctx, key, expects1, roachpb.MakeReplicationChanges(roachpb.ADD_VOTER, targets1...)) + ctx, key, expects1, kvpb.MakeReplicationChanges(roachpb.ADD_VOTER, targets1...)) wg.Done() }() go func() { res2, err2 = db.AdminChangeReplicas( - ctx, key, expects2, roachpb.MakeReplicationChanges(roachpb.ADD_VOTER, targets2...)) + ctx, key, expects2, kvpb.MakeReplicationChanges(roachpb.ADD_VOTER, targets2...)) wg.Done() }() wg.Wait() @@ -3273,7 +3274,7 @@ func TestRandomConcurrentAdminChangeReplicasRequests(t *testing.T) { if rand.Intn(2) == 0 { op = roachpb.ADD_NON_VOTER } - _, err := db.AdminChangeReplicas(ctx, key, rangeInfo.Desc, roachpb.MakeReplicationChanges(op, pickTargets()...)) + _, err := db.AdminChangeReplicas(ctx, key, rangeInfo.Desc, kvpb.MakeReplicationChanges(op, pickTargets()...)) return err } wg.Add(actors) @@ -3411,8 +3412,8 @@ func TestReplicaTombstone(t *testing.T) { sawTooOld := make(chan struct{}, 1) raftFuncs := noopRaftHandlerFuncs() raftFuncs.dropResp = func(resp *kvserverpb.RaftMessageResponse) bool { - if pErr, ok := resp.Union.GetValue().(*roachpb.Error); ok { - if _, isTooOld := pErr.GetDetail().(*roachpb.ReplicaTooOldError); isTooOld { + if pErr, ok := resp.Union.GetValue().(*kvpb.Error); ok { + if _, isTooOld := pErr.GetDetail().(*kvpb.ReplicaTooOldError); isTooOld { select { case sawTooOld <- struct{}{}: default: @@ -3620,7 +3621,7 @@ func TestReplicaTombstone(t *testing.T) { // Don't use tc.AddVoter; this would retry internally as we're faking // a snapshot error here (and these are all considered retriable). _, err = tc.Servers[0].DB().AdminChangeReplicas( - ctx, key, tc.LookupRangeOrFatal(t, key), roachpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(2)), + ctx, key, tc.LookupRangeOrFatal(t, key), kvpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(2)), ) require.Regexp(t, "boom", err) tombstone := waitForTombstone(t, store.TODOEngine(), rangeID) @@ -3638,7 +3639,7 @@ func TestReplicaTombstone(t *testing.T) { // also should be rare. Note this is not new with learner replicas. setMinHeartbeat(5) _, err = tc.Servers[0].DB().AdminChangeReplicas( - ctx, key, tc.LookupRangeOrFatal(t, key), roachpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(2)), + ctx, key, tc.LookupRangeOrFatal(t, key), kvpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(2)), ) require.Regexp(t, "boom", err) // We will start out reading the old tombstone so keep retrying. @@ -3657,7 +3658,7 @@ func TestReplicaTombstone(t *testing.T) { ctx := context.Background() var proposalFilter atomic.Value - noopProposalFilter := func(kvserverbase.ProposalFilterArgs) *roachpb.Error { + noopProposalFilter := func(kvserverbase.ProposalFilterArgs) *kvpb.Error { return nil } proposalFilter.Store(noopProposalFilter) @@ -3665,9 +3666,9 @@ func TestReplicaTombstone(t *testing.T) { ServerArgs: base.TestServerArgs{ Knobs: base.TestingKnobs{Store: &kvserver.StoreTestingKnobs{ DisableReplicaGCQueue: true, - TestingProposalFilter: func(args kvserverbase.ProposalFilterArgs) *roachpb.Error { + TestingProposalFilter: func(args kvserverbase.ProposalFilterArgs) *kvpb.Error { return proposalFilter. - Load().(func(kvserverbase.ProposalFilterArgs) *roachpb.Error)(args) + Load().(func(kvserverbase.ProposalFilterArgs) *kvpb.Error)(args) }, }}, }, @@ -3706,7 +3707,7 @@ func TestReplicaTombstone(t *testing.T) { unreliableRaftHandlerFuncs: raftFuncs, }, }) - proposalFilter.Store(func(args kvserverbase.ProposalFilterArgs) *roachpb.Error { + proposalFilter.Store(func(args kvserverbase.ProposalFilterArgs) *kvpb.Error { merge := args.Cmd.ReplicatedEvalResult.Merge if merge != nil && merge.LeftDesc.RangeID == lhsDesc.RangeID { partActive.Store(true) @@ -3760,9 +3761,9 @@ func TestAdminRelocateRangeSafety(t *testing.T) { var useSeenAdd atomic.Value useSeenAdd.Store(false) seenAdd := make(chan struct{}, 1) - responseFilter := func(ctx context.Context, ba *roachpb.BatchRequest, _ *roachpb.BatchResponse) *roachpb.Error { + responseFilter := func(ctx context.Context, ba *kvpb.BatchRequest, _ *kvpb.BatchResponse) *kvpb.Error { if ba.IsSingleRequest() { - changeReplicas, ok := ba.Requests[0].GetInner().(*roachpb.AdminChangeReplicasRequest) + changeReplicas, ok := ba.Requests[0].GetInner().(*kvpb.AdminChangeReplicasRequest) if ok && changeReplicas.Changes()[0].ChangeType == roachpb.ADD_VOTER && useSeenAdd.Load().(bool) { seenAdd <- struct{}{} } @@ -3827,7 +3828,7 @@ func TestAdminRelocateRangeSafety(t *testing.T) { var changedDesc *roachpb.RangeDescriptor // only populated if changeErr == nil change := func() { <-seenAdd - chgs := roachpb.MakeReplicationChanges(roachpb.REMOVE_VOTER, makeReplicationTargets(2)...) + chgs := kvpb.MakeReplicationChanges(roachpb.REMOVE_VOTER, makeReplicationTargets(2)...) changedDesc, changeErr = r1.ChangeReplicas(ctx, &expDescAfterAdd, kvserverpb.SnapshotRequest_REBALANCE, "replicate", "testing", chgs) } relocate := func() { @@ -3891,11 +3892,11 @@ func TestChangeReplicasLeaveAtomicRacesWithMerge(t *testing.T) { var rangeToBlockRangeDescriptorRead atomic.Value rangeToBlockRangeDescriptorRead.Store(roachpb.RangeID(0)) blockRangeDescriptorReadChan := make(chan struct{}, 1) - blockOnChangeReplicasRead := func(ctx context.Context, ba *roachpb.BatchRequest) *roachpb.Error { - if req, isGet := ba.GetArg(roachpb.Get); !isGet || + blockOnChangeReplicasRead := func(ctx context.Context, ba *kvpb.BatchRequest) *kvpb.Error { + if req, isGet := ba.GetArg(kvpb.Get); !isGet || ba.RangeID != rangeToBlockRangeDescriptorRead.Load().(roachpb.RangeID) || !ba.IsSingleRequest() || - !bytes.HasSuffix(req.(*roachpb.GetRequest).Key, + !bytes.HasSuffix(req.(*kvpb.GetRequest).Key, keys.LocalRangeDescriptorSuffix) { return nil } @@ -3951,7 +3952,7 @@ func TestChangeReplicasLeaveAtomicRacesWithMerge(t *testing.T) { // not yet support atomic replication changes. db := tc.Servers[0].DB() swapReplicas := func(key roachpb.Key, desc roachpb.RangeDescriptor, add, remove int) (*roachpb.RangeDescriptor, error) { - return db.AdminChangeReplicas(ctx, key, desc, []roachpb.ReplicationChange{ + return db.AdminChangeReplicas(ctx, key, desc, []kvpb.ReplicationChange{ {ChangeType: roachpb.ADD_VOTER, Target: tc.Target(add)}, {ChangeType: roachpb.REMOVE_VOTER, Target: tc.Target(remove)}, }) @@ -3983,7 +3984,7 @@ func TestChangeReplicasLeaveAtomicRacesWithMerge(t *testing.T) { go func() { defer wg.Done() _, err := db.AdminChangeReplicas( - ctx, rhs, *rhsDesc, roachpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(2)), + ctx, rhs, *rhsDesc, kvpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(2)), ) // We'll ultimately fail because we're going to race with the work below. msg := `descriptor changed:` @@ -4064,12 +4065,12 @@ func TestTransferLeaseBlocksWrites(t *testing.T) { tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ Knobs: base.TestingKnobs{Store: &kvserver.StoreTestingKnobs{ - TestingProposalFilter: func(args kvserverbase.ProposalFilterArgs) *roachpb.Error { + TestingProposalFilter: func(args kvserverbase.ProposalFilterArgs) *kvpb.Error { if args.Req.RangeID != scratchRangeID.Load().(roachpb.RangeID) { return nil } // Block increment requests on blockInc. - if _, isInc := args.Req.GetArg(roachpb.Increment); isInc { + if _, isInc := args.Req.GetArg(kvpb.Increment); isInc { unblock := make(chan struct{}) blockInc <- unblock <-unblock @@ -4492,15 +4493,15 @@ func TestProposalOverhead(t *testing.T) { var overhead uint32 var key atomic.Value key.Store(roachpb.Key{}) - filter := func(args kvserverbase.ProposalFilterArgs) *roachpb.Error { + filter := func(args kvserverbase.ProposalFilterArgs) *kvpb.Error { if len(args.Req.Requests) != 1 { return nil } - req, ok := args.Req.GetArg(roachpb.Put) + req, ok := args.Req.GetArg(kvpb.Put) if !ok { return nil } - put := req.(*roachpb.PutRequest) + put := req.(*kvpb.PutRequest) if !bytes.Equal(put.Key, key.Load().(roachpb.Key)) { return nil } @@ -4589,7 +4590,7 @@ func TestDiscoverIntentAcrossLeaseTransferAwayAndBack(t *testing.T) { // Detect when txn4 discovers txn3's intent and begins to push. var txn4ID atomic.Value txn4PushingC := make(chan struct{}, 1) - requestFilter := func(_ context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + requestFilter := func(_ context.Context, ba *kvpb.BatchRequest) *kvpb.Error { if !ba.IsSinglePushTxnRequest() { return nil } @@ -4606,10 +4607,10 @@ func TestDiscoverIntentAcrossLeaseTransferAwayAndBack(t *testing.T) { ServerArgs: base.TestServerArgs{Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ TestingRequestFilter: requestFilter, - TestingConcurrencyRetryFilter: func(ctx context.Context, ba *roachpb.BatchRequest, pErr *roachpb.Error) { + TestingConcurrencyRetryFilter: func(ctx context.Context, ba *kvpb.BatchRequest, pErr *kvpb.Error) { if txn := ba.Txn; txn != nil && txn.ID == txn2ID.Load() { txn2BBlockOnce.Do(func() { - if !errors.HasType(pErr.GoError(), (*roachpb.WriteIntentError)(nil)) { + if !errors.HasType(pErr.GoError(), (*kvpb.WriteIntentError)(nil)) { t.Errorf("expected WriteIntentError; got %v", pErr) } @@ -4713,8 +4714,8 @@ func getRangeInfo( ) (ri *roachpb.RangeInfo, err error) { err = db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { b := txn.NewBatch() - b.AddRawRequest(&roachpb.RangeStatsRequest{ - RequestHeader: roachpb.RequestHeader{Key: key}, + b.AddRawRequest(&kvpb.RangeStatsRequest{ + RequestHeader: kvpb.RequestHeader{Key: key}, }) if err = db.Run(ctx, b); err != nil { return err @@ -5006,7 +5007,7 @@ func TestRangeMigration(t *testing.T) { kvDB := tc.Servers[0].DB() req := migrateArgs(desc.StartKey.AsRawKey(), desc.EndKey.AsRawKey(), endV) - if _, pErr := kv.SendWrappedWith(ctx, kvDB.GetFactory().NonTransactionalSender(), roachpb.Header{RangeID: desc.RangeID}, req); pErr != nil { + if _, pErr := kv.SendWrappedWith(ctx, kvDB.GetFactory().NonTransactionalSender(), kvpb.Header{RangeID: desc.RangeID}, req); pErr != nil { t.Fatal(pErr) } diff --git a/pkg/kv/kvserver/client_split_burst_test.go b/pkg/kv/kvserver/client_split_burst_test.go index 89a010853cfd..5200dc2792c0 100644 --- a/pkg/kv/kvserver/client_split_burst_test.go +++ b/pkg/kv/kvserver/client_split_burst_test.go @@ -19,6 +19,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -74,7 +75,7 @@ func setupSplitBurstTest(t *testing.T, delay time.Duration) *splitBurstTest { numSplitsSeenOnSlowFollower := new(int32) // atomic var quiesceCh <-chan struct{} knobs := base.TestingKnobs{Store: &kvserver.StoreTestingKnobs{ - TestingApplyCalledTwiceFilter: func(args kvserverbase.ApplyFilterArgs) (int, *roachpb.Error) { + TestingApplyCalledTwiceFilter: func(args kvserverbase.ApplyFilterArgs) (int, *kvpb.Error) { if args.Split == nil || delay == 0 { return 0, nil } diff --git a/pkg/kv/kvserver/client_split_test.go b/pkg/kv/kvserver/client_split_test.go index c47396b35c8c..1110d441ceb2 100644 --- a/pkg/kv/kvserver/client_split_test.go +++ b/pkg/kv/kvserver/client_split_test.go @@ -31,6 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keyvisualizer" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/abortspan" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" @@ -70,9 +71,9 @@ import ( ) // adminSplitArgs creates an AdminSplitRequest for the provided split key. -func adminSplitArgs(splitKey roachpb.Key) *roachpb.AdminSplitRequest { - return &roachpb.AdminSplitRequest{ - RequestHeader: roachpb.RequestHeader{ +func adminSplitArgs(splitKey roachpb.Key) *kvpb.AdminSplitRequest { + return &kvpb.AdminSplitRequest{ + RequestHeader: kvpb.RequestHeader{ Key: splitKey, }, SplitKey: splitKey, @@ -140,12 +141,12 @@ func TestStoreSplitAbortSpan(t *testing.T) { var expAll []roachpb.AbortSpanEntry - populateAbortSpan := func(key roachpb.Key, ts hlc.Timestamp) *roachpb.ResolveIntentRequest { + populateAbortSpan := func(key roachpb.Key, ts hlc.Timestamp) *kvpb.ResolveIntentRequest { pushee := txn(key, ts) // First write an intent on the key... incArgs := incrementArgs(key, 1) - _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{Txn: pushee}, incArgs) + _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{Txn: pushee}, incArgs) if pErr != nil { t.Fatalf("while sending +%v: %s", incArgs, pErr) } @@ -157,8 +158,8 @@ func TestStoreSplitAbortSpan(t *testing.T) { Key: key, Timestamp: ts, }) - return &roachpb.ResolveIntentRequest{ - RequestHeader: roachpb.RequestHeader{ + return &kvpb.ResolveIntentRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, }, IntentTxn: pushee.TxnMeta, @@ -185,7 +186,7 @@ func TestStoreSplitAbortSpan(t *testing.T) { tsFresh := hlc.Timestamp{WallTime: s.Clock().Now().WallTime} tsStale := hlc.Timestamp{WallTime: s.Clock().Now().WallTime - thresh - 1} - args := []roachpb.Request{ + args := []kvpb.Request{ populateAbortSpan(key(left, 1), tsFresh), populateAbortSpan(key(left, 2), tsStale), populateAbortSpan(key(middle, 1), tsFresh), @@ -483,8 +484,8 @@ func TestQueryLocksAcrossRanges(t *testing.T) { // held by txn2, both with txn3 currently waiting to read. b := kv.Batch{} b.Header.MaxSpanRequestKeys = 2 - queryLocksRequest := &roachpb.QueryLocksRequest{ - RequestHeader: roachpb.RequestHeader{ + queryLocksRequest := &kvpb.QueryLocksRequest{ + RequestHeader: kvpb.RequestHeader{ Key: roachpb.Key("a"), EndKey: roachpb.Key("z"), }, @@ -496,7 +497,7 @@ func TestQueryLocksAcrossRanges(t *testing.T) { res := b.RawResponse().Responses[0].GetQueryLocks() require.Len(t, res.Locks, 2) - require.Equal(t, roachpb.RESUME_KEY_LIMIT, res.ResumeReason, + require.Equal(t, kvpb.RESUME_KEY_LIMIT, res.ResumeReason, "observed locks on %v and %v", res.Locks[0].Key, res.Locks[1].Key, ) require.Equal(t, roachpb.Key("x"), res.ResumeSpan.Key) @@ -554,7 +555,7 @@ func TestStoreRangeSplitAtRangeBounds(t *testing.T) { // Split range 1 at an arbitrary key. key := roachpb.Key("a") rngID := store.LookupReplica(roachpb.RKey(key)).RangeID - h := roachpb.Header{RangeID: rngID} + h := kvpb.Header{RangeID: rngID} args := adminSplitArgs(key) if _, pErr := kv.SendWrappedWith(ctx, store, h, args); pErr != nil { t.Fatal(pErr) @@ -632,7 +633,7 @@ func TestStoreRangeSplitIdempotency(t *testing.T) { lTxn := txn lTxn.Sequence++ lIncArgs.Sequence = lTxn.Sequence - if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ + if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{ Txn: &lTxn, }, lIncArgs); pErr != nil { t.Fatal(pErr) @@ -641,7 +642,7 @@ func TestStoreRangeSplitIdempotency(t *testing.T) { rTxn := txn rTxn.Sequence++ rIncArgs.Sequence = rTxn.Sequence - if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ + if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{ Txn: &rTxn, }, rIncArgs); pErr != nil { t.Fatal(pErr) @@ -686,17 +687,17 @@ func TestStoreRangeSplitIdempotency(t *testing.T) { gArgs := getArgs([]byte("c")) if reply, pErr := kv.SendWrapped(ctx, store.TestSender(), gArgs); pErr != nil { t.Fatal(pErr) - } else if replyBytes, pErr := reply.(*roachpb.GetResponse).Value.GetBytes(); pErr != nil { + } else if replyBytes, pErr := reply.(*kvpb.GetResponse).Value.GetBytes(); pErr != nil { t.Fatal(pErr) } else if !bytes.Equal(replyBytes, content) { t.Fatalf("actual value %q did not match expected value %q", replyBytes, content) } gArgs = getArgs([]byte("x")) - if reply, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ + if reply, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{ RangeID: newRng.RangeID, }, gArgs); pErr != nil { t.Fatal(pErr) - } else if replyBytes, err := reply.(*roachpb.GetResponse).Value.GetBytes(); err != nil { + } else if replyBytes, err := reply.(*kvpb.GetResponse).Value.GetBytes(); err != nil { t.Fatal(err) } else if !bytes.Equal(replyBytes, content) { t.Fatalf("actual value %q did not match expected value %q", replyBytes, content) @@ -704,7 +705,7 @@ func TestStoreRangeSplitIdempotency(t *testing.T) { // Send out an increment request copied from above (same txn/sequence) // which remains in the old range. - _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ + _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{ Txn: &lTxn, }, lIncArgs) if pErr != nil { @@ -713,7 +714,7 @@ func TestStoreRangeSplitIdempotency(t *testing.T) { // Send out the same increment copied from above (same txn/sequence), but // now to the newly created range (which should hold that key). - _, pErr = kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ + _, pErr = kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{ RangeID: newRng.RangeID, Txn: &rTxn, }, rIncArgs) @@ -998,7 +999,7 @@ func TestStoreRangeSplitStatsWithMerges(t *testing.T) { // Split the range at approximate halfway point. args = adminSplitArgs(midKey) - _, pErr = kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ + _, pErr = kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{ RangeID: repl.RangeID, }, args) require.NoError(t, pErr.GoError()) @@ -1047,12 +1048,12 @@ func fillRange( } val := randutil.RandBytes(src, int(src.Int31n(1<<8))) pArgs := putArgs(key, val) - _, pErr := kv.SendWrappedWith(context.Background(), store, roachpb.Header{ + _, pErr := kv.SendWrappedWith(context.Background(), store, kvpb.Header{ RangeID: rangeID, }, pArgs) // When the split occurs in the background, our writes may start failing. // We know we can stop writing when this happens. - if _, ok := pErr.GetDetail().(*roachpb.RangeKeyMismatchError); ok { + if _, ok := pErr.GetDetail().(*kvpb.RangeKeyMismatchError); ok { return } else if pErr != nil { t.Fatal(pErr) @@ -1230,15 +1231,15 @@ func TestStoreRangeSplitBackpressureWrites(t *testing.T) { zoneConfig.RangeMaxBytes = proto.Int64(maxBytes) testingRequestFilter := - func(_ context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + func(_ context.Context, ba *kvpb.BatchRequest) *kvpb.Error { for _, req := range ba.Requests { - if cPut, ok := req.GetInner().(*roachpb.ConditionalPutRequest); ok { + if cPut, ok := req.GetInner().(*kvpb.ConditionalPutRequest); ok { if cPut.Key.Equal(keys.RangeDescriptorKey(splitKey)) { if atomic.CompareAndSwapInt32(&activateSplitFilter, 1, 0) { splitPending <- struct{}{} <-blockSplits if tc.splitErr { - return roachpb.NewErrorf("boom") + return kvpb.NewErrorf("boom") } } } @@ -1270,7 +1271,7 @@ func TestStoreRangeSplitBackpressureWrites(t *testing.T) { // Split at the split key. sArgs := adminSplitArgs(splitKey.AsRawKey()) repl := store.LookupReplica(splitKey) - if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ + if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{ RangeID: repl.RangeID, }, sArgs); pErr != nil { t.Fatal(pErr) @@ -1509,8 +1510,8 @@ func runSetupSplitSnapshotRace( // Scan the meta ranges to resolve all intents if _, pErr := kv.SendWrapped(context.Background(), tc.Servers[0].DistSender(), - &roachpb.ScanRequest{ - RequestHeader: roachpb.RequestHeader{ + &kvpb.ScanRequest{ + RequestHeader: kvpb.RequestHeader{ Key: keys.MetaMin, EndKey: keys.MetaMax, }, @@ -1639,8 +1640,8 @@ func TestStoreSplitTimestampCacheDifferentLeaseHolder(t *testing.T) { // This filter is better understood when reading the meat of the test // below first. var noLeaseForDesc atomic.Value - filter := func(args kvserverbase.FilterArgs) *roachpb.Error { - leaseReq, argOK := args.Req.(*roachpb.RequestLeaseRequest) + filter := func(args kvserverbase.FilterArgs) *kvpb.Error { + leaseReq, argOK := args.Req.(*kvpb.RequestLeaseRequest) forbiddenDesc, descOK := noLeaseForDesc.Load().(*roachpb.ReplicaDescriptor) if !argOK || !descOK || !bytes.Equal(leaseReq.Key, splitKey) { return nil @@ -1653,7 +1654,7 @@ func TestStoreSplitTimestampCacheDifferentLeaseHolder(t *testing.T) { log.Infof(ctx, "refusing lease request (%s, %s) because %+v held lease for LHS of split", leaseReq.Span(), leaseReq.Lease, forbiddenDesc) - return roachpb.NewError(&roachpb.NotLeaseHolderError{RangeID: args.Hdr.RangeID}) + return kvpb.NewError(&kvpb.NotLeaseHolderError{RangeID: args.Hdr.RangeID}) } var args base.TestClusterArgs @@ -1798,14 +1799,14 @@ func TestStoreSplitOnRemovedReplica(t *testing.T) { inFilter := make(chan struct{}, 1) beginBlockingSplit := make(chan struct{}) finishBlockingSplit := make(chan struct{}) - filter := func(_ context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + filter := func(_ context.Context, ba *kvpb.BatchRequest) *kvpb.Error { // Block replica 1's attempt to perform the AdminSplit. We detect the // split's range descriptor update and block until the rest of the test // is ready. We then return a ConditionFailedError, simulating a // descriptor update race. if ba.Replica.NodeID == 1 { for _, req := range ba.Requests { - if cput, ok := req.GetInner().(*roachpb.ConditionalPutRequest); ok { + if cput, ok := req.GetInner().(*kvpb.ConditionalPutRequest); ok { leftDescKey := keys.RangeDescriptorKey(roachpb.RKey(leftKey)) if cput.Key.Equal(leftDescKey) { var desc roachpb.RangeDescriptor @@ -1827,7 +1828,7 @@ func TestStoreSplitOnRemovedReplica(t *testing.T) { if err := val.SetProto(&newDesc); err != nil { panic(err) } - return roachpb.NewError(&roachpb.ConditionFailedError{ + return kvpb.NewError(&kvpb.ConditionFailedError{ ActualValue: &val, }) default: @@ -1930,8 +1931,8 @@ func TestStoreSplitGCThreshold(t *testing.T) { specifiedGCThreshold := hlc.Timestamp{ WallTime: 2e9, } - gcArgs := &roachpb.GCRequest{ - RequestHeader: roachpb.RequestHeader{ + gcArgs := &kvpb.GCRequest{ + RequestHeader: kvpb.RequestHeader{ Key: leftKey, EndKey: rightKey, }, @@ -1995,10 +1996,10 @@ func TestStoreSplitGCHint(t *testing.T) { gcHint := repl.GetGCHint() require.True(t, gcHint.IsEmpty(), "GC hint is not empty by default") - drArgs := &roachpb.DeleteRangeRequest{ + drArgs := &kvpb.DeleteRangeRequest{ UpdateRangeDeleteGCHint: true, UseRangeTombstone: true, - RequestHeader: roachpb.RequestHeader{ + RequestHeader: kvpb.RequestHeader{ Key: r.StartKey.AsRawKey(), EndKey: r.EndKey.AsRawKey(), }, @@ -2034,8 +2035,8 @@ func TestStoreRangeSplitRaceUninitializedRHS(t *testing.T) { } seen.sids = make(map[kvserverbase.CmdIDKey][2]bool) - testingEvalFilter := func(args kvserverbase.FilterArgs) *roachpb.Error { - et, ok := args.Req.(*roachpb.EndTxnRequest) + testingEvalFilter := func(args kvserverbase.FilterArgs) *kvpb.Error { + et, ok := args.Req.(*kvpb.EndTxnRequest) if !ok || et.InternalCommitTrigger == nil { return nil } @@ -2060,8 +2061,8 @@ func TestStoreRangeSplitRaceUninitializedRHS(t *testing.T) { case currentTrigger <- trigger: default: } - return roachpb.NewError( - roachpb.NewReadWithinUncertaintyIntervalError( + return kvpb.NewError( + kvpb.NewReadWithinUncertaintyIntervalError( args.Hdr.Timestamp, hlc.ClockTimestamp{}, nil, args.Hdr.Timestamp, hlc.ClockTimestamp{}, )) } @@ -2101,7 +2102,7 @@ func TestStoreRangeSplitRaceUninitializedRHS(t *testing.T) { tc.AddVotersOrFatal(t, leftRange.Desc().StartKey.AsRawKey(), tc.Target(1)) for i := 0; i < 10; i++ { - errChan := make(chan *roachpb.Error) + errChan := make(chan *kvpb.Error) // Closed when the split goroutine is done. splitDone := make(chan struct{}) @@ -2282,13 +2283,13 @@ func writeRandomTimeSeriesDataToRange( if err := value.SetProto(&idata); err != nil { t.Fatal(err) } - mArgs := roachpb.MergeRequest{ - RequestHeader: roachpb.RequestHeader{ + mArgs := kvpb.MergeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: encoding.EncodeVarintAscending(keyPrefix, idata.StartTimestampNanos), }, Value: value, } - if _, pErr := kv.SendWrappedWith(context.Background(), store.TestSender(), roachpb.Header{ + if _, pErr := kv.SendWrappedWith(context.Background(), store.TestSender(), kvpb.Header{ RangeID: rangeID, }, &mArgs); pErr != nil { t.Fatal(pErr) @@ -2361,12 +2362,12 @@ func TestStoreRangeGossipOnSplits(t *testing.T) { // Pull the first gossiped range count. lastRangeCount := <-rangeCountCh - splitFunc := func(i int) *roachpb.Error { + splitFunc := func(i int) *kvpb.Error { splitKey := roachpb.Key(fmt.Sprintf("%02d", i)) _, pErr := store.LookupReplica(roachpb.RKey(splitKey)).AdminSplit( context.Background(), - roachpb.AdminSplitRequest{ - RequestHeader: roachpb.RequestHeader{ + kvpb.AdminSplitRequest{ + RequestHeader: kvpb.RequestHeader{ Key: splitKey, }, SplitKey: splitKey, @@ -2466,7 +2467,7 @@ func TestDistributedTxnCleanup(t *testing.T) { // Split at "b". rhsKey := roachpb.Key("b") args = adminSplitArgs(rhsKey) - if _, pErr := kv.SendWrappedWith(context.Background(), store, roachpb.Header{ + if _, pErr := kv.SendWrappedWith(context.Background(), store, kvpb.Header{ RangeID: lhs.RangeID, }, args); pErr != nil { t.Fatalf("split at %q: %s", rhsKey, pErr) @@ -2496,15 +2497,15 @@ func TestDistributedTxnCleanup(t *testing.T) { // This simulates txn deadlock or a max priority txn aborting a // normal or min priority txn. if force { - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Timestamp = store.Clock().Now() ba.RangeID = lhs.RangeID - ba.Add(&roachpb.PushTxnRequest{ - RequestHeader: roachpb.RequestHeader{ + ba.Add(&kvpb.PushTxnRequest{ + RequestHeader: kvpb.RequestHeader{ Key: proto.Key, }, PusheeTxn: proto.TxnMeta, - PushType: roachpb.PUSH_ABORT, + PushType: kvpb.PUSH_ABORT, Force: true, }) _, pErr := store.Send(ctx, ba) @@ -2660,8 +2661,8 @@ func TestTxnWaitQueueDependencyCycleWithRangeSplit(t *testing.T) { firstPush := make(chan struct{}) testingEvalFilter := - func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { - if _, ok := filterArgs.Req.(*roachpb.PushTxnRequest); ok { + func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { + if _, ok := filterArgs.Req.(*kvpb.PushTxnRequest); ok { if atomic.AddInt32(&pushCount, 1) == 1 { close(firstPush) } @@ -2754,7 +2755,7 @@ func TestTxnWaitQueueDependencyCycleWithRangeSplit(t *testing.T) { // Split at "b". args = adminSplitArgs(rhsKey) - if _, pErr := kv.SendWrappedWith(ctx, store, roachpb.Header{ + if _, pErr := kv.SendWrappedWith(ctx, store, kvpb.Header{ RangeID: lhs.RangeID, }, args); pErr != nil { t.Fatalf("split at %q: %s", rhsKey, pErr) @@ -2844,7 +2845,7 @@ func TestStoreCapacityAfterSplit(t *testing.T) { getArgs := getArgs(key) if reply, err := kv.SendWrapped(ctx, s.TestSender(), getArgs); err != nil { return errors.Errorf("failed to read data: %s", err) - } else if e, v := int64(10), mustGetInt(reply.(*roachpb.GetResponse).Value); v != e { + } else if e, v := int64(10), mustGetInt(reply.(*kvpb.GetResponse).Value); v != e { return errors.Errorf("failed to read correct data: expected %d, got %d", e, v) } return nil @@ -2971,17 +2972,17 @@ func TestRangeLookupAfterMeta2Split(t *testing.T) { // perform a continuation lookup. scanStart := keys.SystemSQLCodec.TablePrefix(tableID - 2) // 49 scanEnd := scanStart.PrefixEnd() // 50 - header := roachpb.RequestHeader{ + header := kvpb.RequestHeader{ Key: scanStart, EndKey: scanEnd, } - var lookupReq roachpb.Request + var lookupReq kvpb.Request if rev { // A ReverseScanRequest will trigger a reverse RangeLookup scan. - lookupReq = &roachpb.ReverseScanRequest{RequestHeader: header} + lookupReq = &kvpb.ReverseScanRequest{RequestHeader: header} } else { - lookupReq = &roachpb.ScanRequest{RequestHeader: header} + lookupReq = &kvpb.ScanRequest{RequestHeader: header} } if _, err := kv.SendWrapped(ctx, s.DB().NonTransactionalSender(), lookupReq); err != nil { t.Fatalf("%T %v", err.GoError(), err) @@ -3042,7 +3043,7 @@ func TestStoreSplitRangeLookupRace(t *testing.T) { blockedRangeLookups := int32(0) rangeLookupIsBlocked := make(chan struct{}, 1) unblockRangeLookups := make(chan struct{}) - respFilter := func(ctx context.Context, ba *roachpb.BatchRequest, _ *roachpb.BatchResponse) *roachpb.Error { + respFilter := func(ctx context.Context, ba *kvpb.BatchRequest, _ *kvpb.BatchResponse) *kvpb.Error { select { case <-blockRangeLookups: if kv.TestingIsRangeLookup(ba) && @@ -3084,7 +3085,7 @@ func TestStoreSplitRangeLookupRace(t *testing.T) { // Don't use s.DistSender() so that we don't disturb the RangeDescriptorCache. rangeID := store.LookupReplica(roachpb.RKey(splitKey)).RangeID - _, pErr := kv.SendWrappedWith(context.Background(), store, roachpb.Header{ + _, pErr := kv.SendWrappedWith(context.Background(), store, kvpb.Header{ RangeID: rangeID, }, args) if pErr != nil { @@ -3146,9 +3147,9 @@ func TestRangeLookupAsyncResolveIntent(t *testing.T) { defer close(blockPushTxn) testingProposalFilter := - func(args kvserverbase.ProposalFilterArgs) *roachpb.Error { + func(args kvserverbase.ProposalFilterArgs) *kvpb.Error { for _, union := range args.Req.Requests { - if union.GetInner().Method() == roachpb.PushTxn { + if union.GetInner().Method() == kvpb.PushTxn { <-blockPushTxn break } @@ -3184,7 +3185,7 @@ func TestRangeLookupAsyncResolveIntent(t *testing.T) { } // Get original meta2 descriptor. - rs, _, err := kv.RangeLookup(ctx, store.TestSender(), key, roachpb.READ_UNCOMMITTED, 0, false) + rs, _, err := kv.RangeLookup(ctx, store.TestSender(), key, kvpb.READ_UNCOMMITTED, 0, false) if err != nil { t.Fatal(err) } @@ -3213,7 +3214,7 @@ func TestRangeLookupAsyncResolveIntent(t *testing.T) { pArgs := putArgs(keys.RangeMetaKey(roachpb.RKey(key2)).AsRawKey(), data) txn.Sequence++ pArgs.Sequence = txn.Sequence - if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{Txn: &txn}, pArgs); pErr != nil { + if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{Txn: &txn}, pArgs); pErr != nil { t.Fatal(pErr) } @@ -3361,7 +3362,7 @@ func TestSplitTriggerMeetsUnexpectedReplicaID(t *testing.T) { g := ctxgroup.WithContext(ctx) g.GoCtx(func(ctx context.Context) error { _, err := tc.Servers[0].DB().AdminChangeReplicas( - ctx, k, tc.LookupRangeOrFatal(t, k), roachpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(1)), + ctx, k, tc.LookupRangeOrFatal(t, k), kvpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(1)), ) return err }) @@ -3383,7 +3384,7 @@ func TestSplitTriggerMeetsUnexpectedReplicaID(t *testing.T) { // We avoid sending a snapshot because that snapshot would include the // split trigger and we want that to be processed via the log. d, err := tc.Servers[0].DB().AdminChangeReplicas( - ctx, descLHS.StartKey.AsRawKey(), descLHS, roachpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(1)), + ctx, descLHS.StartKey.AsRawKey(), descLHS, kvpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(1)), ) require.NoError(t, err) descLHS = *d @@ -3398,7 +3399,7 @@ func TestSplitTriggerMeetsUnexpectedReplicaID(t *testing.T) { // different replicaID than the split trigger expects. add := func() { _, err := tc.Servers[0].DB().AdminChangeReplicas( - ctx, kRHS, tc.LookupRangeOrFatal(t, kRHS), roachpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(1)), + ctx, kRHS, tc.LookupRangeOrFatal(t, kRHS), kvpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(1)), ) // The "snapshot intersects existing range" error is expected if the store // has not heard a raft message addressed to a later replica ID while the @@ -3470,9 +3471,9 @@ func TestSplitBlocksReadsToRHS(t *testing.T) { keyLHS, keySplit, keyRHS := roachpb.Key("a"), roachpb.Key("b"), roachpb.Key("c") splitBlocked := make(chan struct{}) - propFilter := func(args kvserverbase.ProposalFilterArgs) *roachpb.Error { - if req, ok := args.Req.GetArg(roachpb.EndTxn); ok { - et := req.(*roachpb.EndTxnRequest) + propFilter := func(args kvserverbase.ProposalFilterArgs) *kvpb.Error { + if req, ok := args.Req.GetArg(kvpb.EndTxn); ok { + et := req.(*kvpb.EndTxnRequest) if tr := et.InternalCommitTrigger.GetSplitTrigger(); tr != nil { if tr.RightDesc.StartKey.Equal(keySplit) { // Signal that the split is blocked. @@ -3525,7 +3526,7 @@ func TestSplitBlocksReadsToRHS(t *testing.T) { {keyRHS, rhsDone}, } { for _, ts := range []hlc.Timestamp{tsBefore, tsAfter} { - h := roachpb.Header{Timestamp: ts, RangeID: repl.RangeID} + h := kvpb.Header{Timestamp: ts, RangeID: repl.RangeID} args := getArgs(keyAndChan.key) errCh := keyAndChan.errCh g.GoCtx(func(ctx context.Context) error { @@ -3575,9 +3576,9 @@ func TestStoreRangeSplitAndMergeWithGlobalReads(t *testing.T) { // necessary, see maybeCommitWaitBeforeCommitTrigger. var clock atomic.Value var splitsWithSyntheticTS, mergesWithSyntheticTS int64 - respFilter := func(ctx context.Context, ba *roachpb.BatchRequest, br *roachpb.BatchResponse) *roachpb.Error { - if req, ok := ba.GetArg(roachpb.EndTxn); ok { - endTxn := req.(*roachpb.EndTxnRequest) + respFilter := func(ctx context.Context, ba *kvpb.BatchRequest, br *kvpb.BatchResponse) *kvpb.Error { + if req, ok := ba.GetArg(kvpb.EndTxn); ok { + endTxn := req.(*kvpb.EndTxnRequest) if br.Txn.Status == roachpb.COMMITTED && br.Txn.WriteTimestamp.Synthetic { if ct := endTxn.InternalCommitTrigger; ct != nil { // The server-side commit-wait sleep should ensure that the commit diff --git a/pkg/kv/kvserver/client_tenant_test.go b/pkg/kv/kvserver/client_tenant_test.go index 6ddb55d22da7..b504b7183ebb 100644 --- a/pkg/kv/kvserver/client_tenant_test.go +++ b/pkg/kv/kvserver/client_tenant_test.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" _ "github.com/cockroachdb/cockroach/pkg/ccl" // for tenant functionality "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/tenantrate" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -303,7 +304,7 @@ func TestTenantCtx(t *testing.T) { s, _, _ := serverutils.StartServer(t, base.TestServerArgs{ Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(ctx context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + TestingRequestFilter: func(ctx context.Context, ba *kvpb.BatchRequest) *kvpb.Error { // We'll recognize a GetRequest and a PushRequest, check that the // context looks as expected, and signal their channels. @@ -313,13 +314,13 @@ func TestTenantCtx(t *testing.T) { return nil } - var getReq *roachpb.GetRequest - var pushReq *roachpb.PushTxnRequest - if isSingleGet := ba.IsSingleRequest() && ba.Requests[0].GetInner().Method() == roachpb.Get; isSingleGet { - getReq = ba.Requests[0].GetInner().(*roachpb.GetRequest) + var getReq *kvpb.GetRequest + var pushReq *kvpb.PushTxnRequest + if isSingleGet := ba.IsSingleRequest() && ba.Requests[0].GetInner().Method() == kvpb.Get; isSingleGet { + getReq = ba.Requests[0].GetInner().(*kvpb.GetRequest) } - if isSinglePushTxn := ba.IsSingleRequest() && ba.Requests[0].GetInner().Method() == roachpb.PushTxn; isSinglePushTxn { - pushReq = ba.Requests[0].GetInner().(*roachpb.PushTxnRequest) + if isSinglePushTxn := ba.IsSingleRequest() && ba.Requests[0].GetInner().Method() == kvpb.PushTxn; isSinglePushTxn { + pushReq = ba.Requests[0].GetInner().(*kvpb.PushTxnRequest) } switch { diff --git a/pkg/kv/kvserver/client_test.go b/pkg/kv/kvserver/client_test.go index e8cbdfbd5397..7b1194106e46 100644 --- a/pkg/kv/kvserver/client_test.go +++ b/pkg/kv/kvserver/client_test.go @@ -26,6 +26,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -38,32 +39,32 @@ import ( // getArgs returns a GetRequest and GetResponse pair addressed to // the default replica for the specified key. -func getArgs(key roachpb.Key) *roachpb.GetRequest { - return &roachpb.GetRequest{ - RequestHeader: roachpb.RequestHeader{ +func getArgs(key roachpb.Key) *kvpb.GetRequest { + return &kvpb.GetRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, }, } } // putArgs returns a PutRequest for the specified key / value. -func putArgs(key roachpb.Key, value []byte) *roachpb.PutRequest { - return &roachpb.PutRequest{ - RequestHeader: roachpb.RequestHeader{Key: key}, +func putArgs(key roachpb.Key, value []byte) *kvpb.PutRequest { + return &kvpb.PutRequest{ + RequestHeader: kvpb.RequestHeader{Key: key}, Value: roachpb.MakeValueFromBytes(value), } } // cPutArgs returns a ConditionPutRequest to the default replica // for the specified key and value, with the given expected value. -func cPutArgs(key roachpb.Key, value, expValue []byte) *roachpb.ConditionalPutRequest { +func cPutArgs(key roachpb.Key, value, expValue []byte) *kvpb.ConditionalPutRequest { var expBytes []byte if expValue != nil { expBytes = roachpb.MakeValueFromBytes(expValue).TagAndDataBytes() } - return &roachpb.ConditionalPutRequest{ - RequestHeader: roachpb.RequestHeader{ + return &kvpb.ConditionalPutRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, }, Value: roachpb.MakeValueFromBytes(value), @@ -73,17 +74,17 @@ func cPutArgs(key roachpb.Key, value, expValue []byte) *roachpb.ConditionalPutRe // incrementArgs returns an IncrementRequest addressed to the default replica // for the specified key. -func incrementArgs(key roachpb.Key, inc int64) *roachpb.IncrementRequest { - return &roachpb.IncrementRequest{ - RequestHeader: roachpb.RequestHeader{ +func incrementArgs(key roachpb.Key, inc int64) *kvpb.IncrementRequest { + return &kvpb.IncrementRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, }, Increment: inc, } } -func truncateLogArgs(index uint64, rangeID roachpb.RangeID) *roachpb.TruncateLogRequest { - return &roachpb.TruncateLogRequest{ +func truncateLogArgs(index uint64, rangeID roachpb.RangeID) *kvpb.TruncateLogRequest { + return &kvpb.TruncateLogRequest{ Index: index, RangeID: rangeID, } @@ -91,29 +92,29 @@ func truncateLogArgs(index uint64, rangeID roachpb.RangeID) *roachpb.TruncateLog func heartbeatArgs( txn *roachpb.Transaction, now hlc.Timestamp, -) (*roachpb.HeartbeatTxnRequest, roachpb.Header) { - return &roachpb.HeartbeatTxnRequest{ - RequestHeader: roachpb.RequestHeader{ +) (*kvpb.HeartbeatTxnRequest, kvpb.Header) { + return &kvpb.HeartbeatTxnRequest{ + RequestHeader: kvpb.RequestHeader{ Key: txn.Key, }, Now: now, - }, roachpb.Header{Txn: txn} + }, kvpb.Header{Txn: txn} } -func endTxnArgs(txn *roachpb.Transaction, commit bool) (*roachpb.EndTxnRequest, roachpb.Header) { - return &roachpb.EndTxnRequest{ - RequestHeader: roachpb.RequestHeader{ +func endTxnArgs(txn *roachpb.Transaction, commit bool) (*kvpb.EndTxnRequest, kvpb.Header) { + return &kvpb.EndTxnRequest{ + RequestHeader: kvpb.RequestHeader{ Key: txn.Key, // not allowed when going through TxnCoordSender, but we're not }, Commit: commit, - }, roachpb.Header{Txn: txn} + }, kvpb.Header{Txn: txn} } func pushTxnArgs( - pusher, pushee *roachpb.Transaction, pushType roachpb.PushTxnType, -) *roachpb.PushTxnRequest { - return &roachpb.PushTxnRequest{ - RequestHeader: roachpb.RequestHeader{ + pusher, pushee *roachpb.Transaction, pushType kvpb.PushTxnType, +) *kvpb.PushTxnRequest { + return &kvpb.PushTxnRequest{ + RequestHeader: kvpb.RequestHeader{ Key: pushee.Key, }, PushTo: pusher.WriteTimestamp.Next(), @@ -123,9 +124,9 @@ func pushTxnArgs( } } -func migrateArgs(start, end roachpb.Key, version roachpb.Version) *roachpb.MigrateRequest { - return &roachpb.MigrateRequest{ - RequestHeader: roachpb.RequestHeader{ +func migrateArgs(start, end roachpb.Key, version roachpb.Version) *kvpb.MigrateRequest { + return &kvpb.MigrateRequest{ + RequestHeader: kvpb.RequestHeader{ Key: start, EndKey: end, }, @@ -133,9 +134,9 @@ func migrateArgs(start, end roachpb.Key, version roachpb.Version) *roachpb.Migra } } -func adminTransferLeaseArgs(key roachpb.Key, target roachpb.StoreID) roachpb.Request { - return &roachpb.AdminTransferLeaseRequest{ - RequestHeader: roachpb.RequestHeader{ +func adminTransferLeaseArgs(key roachpb.Key, target roachpb.StoreID) kvpb.Request { + return &kvpb.AdminTransferLeaseRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, }, Target: target, diff --git a/pkg/kv/kvserver/closed_timestamp_test.go b/pkg/kv/kvserver/closed_timestamp_test.go index e8cc1350a1d3..788bb17201b7 100644 --- a/pkg/kv/kvserver/closed_timestamp_test.go +++ b/pkg/kv/kvserver/closed_timestamp_test.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/txnwait" @@ -102,8 +103,8 @@ func TestClosedTimestampCanServe(t *testing.T) { // We just served a follower read. As a sanity check, make sure that we can't write at // that same timestamp. { - baWrite := &roachpb.BatchRequest{} - r := &roachpb.DeleteRequest{} + baWrite := &kvpb.BatchRequest{} + r := &kvpb.DeleteRequest{} r.Key = desc.StartKey.AsRawKey() txn := roachpb.MakeTransaction("testwrite", r.Key, roachpb.NormalUserPriority, ts, 100, int32(tc.Server(0).SQLInstanceID())) baWrite.Txn = &txn @@ -116,7 +117,7 @@ func TestClosedTimestampCanServe(t *testing.T) { var found bool for _, repl := range repls { resp, pErr := repl.Send(ctx, baWrite) - if errors.HasType(pErr.GoError(), (*roachpb.NotLeaseHolderError)(nil)) { + if errors.HasType(pErr.GoError(), (*kvpb.NotLeaseHolderError)(nil)) { continue } else if pErr != nil { t.Fatal(pErr) @@ -232,7 +233,7 @@ func TestClosedTimestampCanServeThroughoutLeaseTransfer(t *testing.T) { if pErr != nil { return errors.Wrapf(pErr.GoError(), "on %s", r) } - rows := resp.Responses[0].GetInner().(*roachpb.ScanResponse).Rows + rows := resp.Responses[0].GetInner().(*kvpb.ScanResponse).Rows // Should see the write. if len(rows) != 1 { return fmt.Errorf("expected one row, but got %d", len(rows)) @@ -278,7 +279,7 @@ func TestClosedTimestampCantServeWithConflictingIntent(t *testing.T) { key := append(txnKey, []byte(strconv.Itoa(i))...) keys = append(keys, key) put := putArgs(key, []byte("val")) - resp, err := kv.SendWrappedWith(ctx, ds, roachpb.Header{Txn: &txn}, put) + resp, err := kv.SendWrappedWith(ctx, ds, kvpb.Header{Txn: &txn}, put) require.Nil(t, err) txn.Update(resp.Header().Txn) } @@ -326,7 +327,7 @@ func TestClosedTimestampCantServeWithConflictingIntent(t *testing.T) { for i := 0; i < len(repls)-1; i++ { err := <-respCh2 require.Error(t, err) - var lErr *roachpb.NotLeaseHolderError + var lErr *kvpb.NotLeaseHolderError require.True(t, errors.As(err, &lErr)) } select { @@ -336,12 +337,12 @@ func TestClosedTimestampCantServeWithConflictingIntent(t *testing.T) { } // Abort the transaction. All intents should be rolled back. - endTxn := &roachpb.EndTxnRequest{ - RequestHeader: roachpb.RequestHeader{Key: txn.Key}, + endTxn := &kvpb.EndTxnRequest{ + RequestHeader: kvpb.RequestHeader{Key: txn.Key}, Commit: false, LockSpans: []roachpb.Span{desc.KeySpan().AsRawSpanWithNoLocals()}, } - _, err := kv.SendWrappedWith(ctx, ds, roachpb.Header{Txn: &txn}, endTxn) + _, err := kv.SendWrappedWith(ctx, ds, kvpb.Header{Txn: &txn}, endTxn) require.Nil(t, err) // The blocked read on the leaseholder should succeed. @@ -546,9 +547,9 @@ func TestClosedTimestampCantServeForNonTransactionalReadRequest(t *testing.T) { }) // Create a "nontransactional" read-only batch. - baQueryTxn := &roachpb.BatchRequest{} + baQueryTxn := &kvpb.BatchRequest{} baQueryTxn.Header.RangeID = desc.RangeID - r := &roachpb.QueryTxnRequest{} + r := &kvpb.QueryTxnRequest{} r.Key = desc.StartKey.AsRawKey() r.Txn.Key = r.Key r.Txn.MinTimestamp = ts @@ -812,10 +813,10 @@ SET CLUSTER SETTING kv.closed_timestamp.follower_reads_enabled = true; require.NoError(t, err) writeTime := rhsLeaseStart.Prev() require.True(t, mergedLeaseholder.GetCurrentClosedTimestamp(ctx).Less(writeTime)) - baWrite := &roachpb.BatchRequest{} + baWrite := &kvpb.BatchRequest{} baWrite.Header.RangeID = leftDesc.RangeID baWrite.Header.Timestamp = writeTime - put := &roachpb.PutRequest{} + put := &kvpb.PutRequest{} put.Key = rightDesc.StartKey.AsRawKey() baWrite.Add(put) resp, pErr := mergedLeaseholder.Send(ctx, baWrite) @@ -935,8 +936,8 @@ func (filter *mergeFilter) resetBlocker() (*mergeBlocker, bool) { // Communication with actors interested in blocked merges is done through // BlockNextMerge(). func (filter *mergeFilter) SuspendMergeTrigger( - ctx context.Context, ba *roachpb.BatchRequest, -) *roachpb.Error { + ctx context.Context, ba *kvpb.BatchRequest, +) *kvpb.Error { for _, req := range ba.Requests { if et := req.GetEndTxn(); et != nil && et.Commit && et.InternalCommitTrigger.GetMergeTrigger() != nil { @@ -1064,7 +1065,7 @@ func getTargetStore( } func verifyNotLeaseHolderErrors( - t *testing.T, ba *roachpb.BatchRequest, repls []*kvserver.Replica, expectedNLEs int, + t *testing.T, ba *kvpb.BatchRequest, repls []*kvserver.Replica, expectedNLEs int, ) { t.Helper() notLeaseholderErrs, err := countNotLeaseHolderErrors(ba, repls) @@ -1076,14 +1077,14 @@ func verifyNotLeaseHolderErrors( } } -func countNotLeaseHolderErrors(ba *roachpb.BatchRequest, repls []*kvserver.Replica) (int64, error) { +func countNotLeaseHolderErrors(ba *kvpb.BatchRequest, repls []*kvserver.Replica) (int64, error) { g, ctx := errgroup.WithContext(context.Background()) var notLeaseholderErrs int64 for i := range repls { repl := repls[i] g.Go(func() (err error) { if _, pErr := repl.Send(ctx, ba); pErr != nil { - if _, ok := pErr.GetDetail().(*roachpb.NotLeaseHolderError); ok { + if _, ok := pErr.GetDetail().(*kvpb.NotLeaseHolderError); ok { atomic.AddInt64(¬LeaseholderErrs, 1) return nil } @@ -1250,12 +1251,12 @@ RESET statement_timeout; return tc, desc } -type respFunc func(*roachpb.BatchResponse, *roachpb.Error) (shouldRetry bool, err error) +type respFunc func(*kvpb.BatchResponse, *kvpb.Error) (shouldRetry bool, err error) // respFuncs returns a respFunc which is passes its arguments to each passed // func until one returns shouldRetry or a non-nil error. func respFuncs(funcs ...respFunc) respFunc { - return func(resp *roachpb.BatchResponse, pErr *roachpb.Error) (shouldRetry bool, err error) { + return func(resp *kvpb.BatchResponse, pErr *kvpb.Error) (shouldRetry bool, err error) { for _, f := range funcs { shouldRetry, err = f(resp, pErr) if err != nil || shouldRetry { @@ -1266,8 +1267,8 @@ func respFuncs(funcs ...respFunc) respFunc { } } -func retryOnError(f func(*roachpb.Error) bool) respFunc { - return func(resp *roachpb.BatchResponse, pErr *roachpb.Error) (shouldRetry bool, err error) { +func retryOnError(f func(*kvpb.Error) bool) respFunc { + return func(resp *kvpb.BatchResponse, pErr *kvpb.Error) (shouldRetry bool, err error) { if pErr != nil && f(pErr) { return true, nil } @@ -1275,22 +1276,22 @@ func retryOnError(f func(*roachpb.Error) bool) respFunc { } } -var retryOnRangeKeyMismatch = retryOnError(func(pErr *roachpb.Error) bool { - _, isRangeKeyMismatch := pErr.GetDetail().(*roachpb.RangeKeyMismatchError) +var retryOnRangeKeyMismatch = retryOnError(func(pErr *kvpb.Error) bool { + _, isRangeKeyMismatch := pErr.GetDetail().(*kvpb.RangeKeyMismatchError) return isRangeKeyMismatch }) -var retryOnRangeNotFound = retryOnError(func(pErr *roachpb.Error) bool { - _, isRangeNotFound := pErr.GetDetail().(*roachpb.RangeNotFoundError) +var retryOnRangeNotFound = retryOnError(func(pErr *kvpb.Error) bool { + _, isRangeNotFound := pErr.GetDetail().(*kvpb.RangeNotFoundError) return isRangeNotFound }) func expectRows(expectedRows int) respFunc { - return func(resp *roachpb.BatchResponse, pErr *roachpb.Error) (shouldRetry bool, err error) { + return func(resp *kvpb.BatchResponse, pErr *kvpb.Error) (shouldRetry bool, err error) { if pErr != nil { return false, pErr.GoError() } - rows := resp.Responses[0].GetInner().(*roachpb.ScanResponse).Rows + rows := resp.Responses[0].GetInner().(*kvpb.ScanResponse).Rows // Should see the write. if len(rows) != expectedRows { return false, fmt.Errorf("expected %d rows, but got %d", expectedRows, len(rows)) @@ -1302,7 +1303,7 @@ func expectRows(expectedRows int) respFunc { func verifyCanReadFromAllRepls( ctx context.Context, t *testing.T, - baRead *roachpb.BatchRequest, + baRead *kvpb.BatchRequest, repls []*kvserver.Replica, f respFunc, ) error { @@ -1330,14 +1331,14 @@ func verifyCanReadFromAllRepls( return g.Wait() } -func makeTxnReadBatchForDesc(desc roachpb.RangeDescriptor, ts hlc.Timestamp) *roachpb.BatchRequest { +func makeTxnReadBatchForDesc(desc roachpb.RangeDescriptor, ts hlc.Timestamp) *kvpb.BatchRequest { txn := roachpb.MakeTransaction("txn", nil, 0, ts, 0, 0) - baRead := &roachpb.BatchRequest{} + baRead := &kvpb.BatchRequest{} baRead.Header.RangeID = desc.RangeID baRead.Header.Timestamp = ts baRead.Header.Txn = &txn - r := &roachpb.ScanRequest{} + r := &kvpb.ScanRequest{} r.Key = desc.StartKey.AsRawKey() r.EndKey = desc.EndKey.AsRawKey() baRead.Add(r) diff --git a/pkg/kv/kvserver/concurrency/BUILD.bazel b/pkg/kv/kvserver/concurrency/BUILD.bazel index 8a47a32bdbe5..b101b0b5a600 100644 --- a/pkg/kv/kvserver/concurrency/BUILD.bazel +++ b/pkg/kv/kvserver/concurrency/BUILD.bazel @@ -18,6 +18,7 @@ go_library( deps = [ "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/concurrency/lock", "//pkg/kv/kvserver/concurrency/poison", "//pkg/kv/kvserver/intentresolver", @@ -59,6 +60,7 @@ go_test( shard_count = 16, deps = [ "//pkg/keys", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/batcheval", "//pkg/kv/kvserver/concurrency/lock", "//pkg/kv/kvserver/concurrency/poison", diff --git a/pkg/kv/kvserver/concurrency/concurrency_control.go b/pkg/kv/kvserver/concurrency/concurrency_control.go index 6d7342da0d1d..f65627fc8d54 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_control.go +++ b/pkg/kv/kvserver/concurrency/concurrency_control.go @@ -17,6 +17,7 @@ import ( "context" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/poison" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" @@ -228,7 +229,7 @@ type ContentionHandler interface { // lock table and observes the lock on key K, so it enters the lock's // wait-queue and waits for it to be resolved. HandleWriterIntentError( - context.Context, *Guard, roachpb.LeaseSequence, *roachpb.WriteIntentError, + context.Context, *Guard, roachpb.LeaseSequence, *kvpb.WriteIntentError, ) (*Guard, *Error) // HandleTransactionPushError consumes a TransactionPushError thrown by a @@ -247,7 +248,7 @@ type ContentionHandler interface { // before txn A retries its push. During the retry, txn A finds that txn B // is being tracked in the txn wait-queue so it waits there for txn B to // finish. - HandleTransactionPushError(context.Context, *Guard, *roachpb.TransactionPushError) *Guard + HandleTransactionPushError(context.Context, *Guard, *kvpb.TransactionPushError) *Guard } // LockManager is concerned with tracking locks that are stored on the manager's @@ -383,7 +384,7 @@ type Request struct { NonTxnPriority roachpb.UserPriority // The consistency level of the request. Only set if Txn is nil. - ReadConsistency roachpb.ReadConsistencyType + ReadConsistency kvpb.ReadConsistencyType // The wait policy of the request. Signifies how the request should // behave if it encounters conflicting locks held by other active @@ -407,7 +408,7 @@ type Request struct { PoisonPolicy poison.Policy // The individual requests in the batch. - Requests []roachpb.RequestUnion + Requests []kvpb.RequestUnion // The maximal set of spans that the request will access. Latches // will be acquired for these spans. @@ -449,10 +450,10 @@ type Guard struct { // Response is a slice of responses to requests in a batch. This type is used // when the concurrency manager is able to respond to a request directly during // sequencing. -type Response = []roachpb.ResponseUnion +type Response = []kvpb.ResponseUnion -// Error is an alias for a roachpb.Error. -type Error = roachpb.Error +// Error is an alias for a kvpb.Error. +type Error = kvpb.Error // QueryLockTableOptions bundles the options for the QueryLockTableState function. type QueryLockTableOptions struct { @@ -466,7 +467,7 @@ type QueryLockTableOptions struct { // results from the QueryLockTableState function. type QueryLockTableResumeState struct { ResumeSpan *roachpb.Span - ResumeReason roachpb.ResumeReason + ResumeReason kvpb.ResumeReason // ResumeNextBytes represents the size (in bytes) of the next // roachpb.LockStateInfo object that would have been returned by @@ -981,14 +982,14 @@ type txnWaitQueue interface { // // If the transaction is successfully pushed while this method is waiting, // the first return value is a non-nil PushTxnResponse object. - MaybeWaitForPush(context.Context, *roachpb.PushTxnRequest) (*roachpb.PushTxnResponse, *Error) + MaybeWaitForPush(context.Context, *kvpb.PushTxnRequest) (*kvpb.PushTxnResponse, *Error) // MaybeWaitForQuery checks whether there is a queue already established for // transaction being queried. If not, or if the QueryTxn request hasn't // specified WaitForUpdate, the method returns immediately. If there is a // queue, the method enqueues this request as a waiter and waits for any // updates to the target transaction. - MaybeWaitForQuery(context.Context, *roachpb.QueryTxnRequest) *Error + MaybeWaitForQuery(context.Context, *kvpb.QueryTxnRequest) *Error // OnRangeDescUpdated informs the Queue that its range's descriptor has been // updated. diff --git a/pkg/kv/kvserver/concurrency/concurrency_manager.go b/pkg/kv/kvserver/concurrency/concurrency_manager.go index 8dab281f4941..c8d60ed0cbcb 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_manager.go +++ b/pkg/kv/kvserver/concurrency/concurrency_manager.go @@ -17,6 +17,7 @@ import ( "sync" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanlatch" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" @@ -341,7 +342,7 @@ func (m *managerImpl) sequenceReqWithGuard( // into queues and optionally update its internal state based on the requests. func (m *managerImpl) maybeInterceptReq(ctx context.Context, req Request) (Response, *Error) { switch { - case req.isSingle(roachpb.PushTxn): + case req.isSingle(kvpb.PushTxn): // If necessary, wait in the txnWaitQueue for the pushee transaction to // expire or to move to a finalized state. t := req.Requests[0].GetPushTxn() @@ -351,7 +352,7 @@ func (m *managerImpl) maybeInterceptReq(ctx context.Context, req Request) (Respo } else if resp != nil { return makeSingleResponse(resp), nil } - case req.isSingle(roachpb.QueryTxn): + case req.isSingle(kvpb.QueryTxn): // If necessary, wait in the txnWaitQueue for a transaction state update // or for a dependent transaction to change. t := req.Requests[0].GetQueryTxn() @@ -361,9 +362,9 @@ func (m *managerImpl) maybeInterceptReq(ctx context.Context, req Request) (Respo // table to allow contending transactions to proceed. // for _, arg := range req.Requests { // switch t := arg.GetInner().(type) { - // case *roachpb.ResolveIntentRequest: + // case *kvpb.ResolveIntentRequest: // _ = t - // case *roachpb.ResolveIntentRangeRequest: + // case *kvpb.ResolveIntentRangeRequest: // _ = t // } // } @@ -379,10 +380,10 @@ func (m *managerImpl) maybeInterceptReq(ctx context.Context, req Request) (Respo // they could wait on them, even if they don't acquire latches. func shouldIgnoreLatches(req Request) bool { switch { - case req.ReadConsistency != roachpb.CONSISTENT: + case req.ReadConsistency != kvpb.CONSISTENT: // Only acquire latches for consistent operations. return true - case req.isSingle(roachpb.RequestLease): + case req.isSingle(kvpb.RequestLease): // Ignore latches for lease requests. These requests are run on replicas // that do not hold the lease, so acquiring latches wouldn't help // synchronize with other requests. @@ -394,7 +395,7 @@ func shouldIgnoreLatches(req Request) bool { // shouldWaitOnLatchesWithoutAcquiring determines if this is a request that // only waits on existing latches without acquiring any new ones. func shouldWaitOnLatchesWithoutAcquiring(req Request) bool { - return req.isSingle(roachpb.Barrier) + return req.isSingle(kvpb.Barrier) } // PoisonReq implements the RequestSequencer interface. @@ -435,7 +436,7 @@ func (m *managerImpl) FinishReq(g *Guard) { // HandleWriterIntentError implements the ContentionHandler interface. func (m *managerImpl) HandleWriterIntentError( - ctx context.Context, g *Guard, seq roachpb.LeaseSequence, t *roachpb.WriteIntentError, + ctx context.Context, g *Guard, seq roachpb.LeaseSequence, t *kvpb.WriteIntentError, ) (*Guard, *Error) { if g.ltg == nil { log.Fatalf(ctx, "cannot handle WriteIntentError %v for request without "+ @@ -497,7 +498,7 @@ func (m *managerImpl) HandleWriterIntentError( // HandleTransactionPushError implements the ContentionHandler interface. func (m *managerImpl) HandleTransactionPushError( - ctx context.Context, g *Guard, t *roachpb.TransactionPushError, + ctx context.Context, g *Guard, t *kvpb.TransactionPushError, ) *Guard { m.twq.EnqueueTxn(&t.PusheeTxn) @@ -630,7 +631,7 @@ func (r *Request) txnMeta() *enginepb.TxnMeta { return &r.Txn.TxnMeta } -func (r *Request) isSingle(m roachpb.Method) bool { +func (r *Request) isSingle(m kvpb.Method) bool { if len(r.Requests) != 1 { return false } @@ -777,7 +778,7 @@ func (g *Guard) moveLockTableGuard() lockTableGuard { return ltg } -func makeSingleResponse(r roachpb.Response) Response { +func makeSingleResponse(r kvpb.Response) Response { ru := make(Response, 1) ru[0].MustSetInner(r) return ru diff --git a/pkg/kv/kvserver/concurrency/concurrency_manager_test.go b/pkg/kv/kvserver/concurrency/concurrency_manager_test.go index 2efde9ef80ed..e866ab6646fb 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_manager_test.go +++ b/pkg/kv/kvserver/concurrency/concurrency_manager_test.go @@ -25,6 +25,7 @@ import ( "testing" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" @@ -159,9 +160,9 @@ func TestConcurrencyManagerBasic(t *testing.T) { txn.WriteTimestamp = ts } - readConsistency := roachpb.CONSISTENT + readConsistency := kvpb.CONSISTENT if d.HasArg("inconsistent") { - readConsistency = roachpb.INCONSISTENT + readConsistency = kvpb.INCONSISTENT } priority := scanUserPriority(t, d) @@ -183,7 +184,7 @@ func TestConcurrencyManagerBasic(t *testing.T) { pp := scanPoisonPolicy(t, d) - // Each roachpb.Request is provided on an indented line. + // Each kvpb.Request is provided on an indented line. reqs, reqUnions := scanRequests(t, d, c) latchSpans, lockSpans := c.collectSpans(t, txn, ts, reqs) @@ -327,7 +328,7 @@ func TestConcurrencyManagerBasic(t *testing.T) { opName := fmt.Sprintf("handle write intent error %s", reqName) mon.runAsync(opName, func(ctx context.Context) { seq := roachpb.LeaseSequence(leaseSeq) - wiErr := &roachpb.WriteIntentError{Intents: intents} + wiErr := &kvpb.WriteIntentError{Intents: intents} guard, err := m.HandleWriterIntentError(ctx, prev, seq, wiErr) if err != nil { log.Eventf(ctx, "handled %v, returned error: %v", wiErr, err) @@ -401,7 +402,7 @@ func TestConcurrencyManagerBasic(t *testing.T) { for _, ru := range guard.Req.Requests { req := ru.GetInner() keySpan := roachpb.Span{Key: roachpb.Key(key)} - if roachpb.IsLocking(req) && + if kvpb.IsLocking(req) && req.Header().Span().Contains(keySpan) && req.Header().Sequence == seqNum { found = true @@ -599,15 +600,15 @@ func TestConcurrencyManagerBasic(t *testing.T) { func scanRequests( t *testing.T, d *datadriven.TestData, c *cluster, -) ([]roachpb.Request, []roachpb.RequestUnion) { - // Each roachpb.Request is provided on an indented line. - var reqs []roachpb.Request +) ([]kvpb.Request, []kvpb.RequestUnion) { + // Each kvpb.Request is provided on an indented line. + var reqs []kvpb.Request singleReqLines := strings.Split(d.Input, "\n") for _, line := range singleReqLines { req := scanSingleRequest(t, d, line, c.txnsByName) reqs = append(reqs, req) } - reqUnions := make([]roachpb.RequestUnion, len(reqs)) + reqUnions := make([]kvpb.RequestUnion, len(reqs)) for i, req := range reqs { reqUnions[i].MustSetInner(req) } @@ -683,23 +684,23 @@ func (c *cluster) makeConfig() concurrency.Config { // PushTransaction implements the concurrency.IntentResolver interface. func (c *cluster) PushTransaction( - ctx context.Context, pushee *enginepb.TxnMeta, h roachpb.Header, pushType roachpb.PushTxnType, -) (*roachpb.Transaction, *roachpb.Error) { + ctx context.Context, pushee *enginepb.TxnMeta, h kvpb.Header, pushType kvpb.PushTxnType, +) (*roachpb.Transaction, *kvpb.Error) { pusheeRecord, err := c.getTxnRecord(pushee.ID) if err != nil { - return nil, roachpb.NewError(err) + return nil, kvpb.NewError(err) } var pusherRecord *txnRecord if h.Txn != nil { pusherID := h.Txn.ID pusherRecord, err = c.getTxnRecord(pusherID) if err != nil { - return nil, roachpb.NewError(err) + return nil, kvpb.NewError(err) } push, err := c.registerPush(ctx, pusherID, pushee.ID) if err != nil { - return nil, roachpb.NewError(err) + return nil, kvpb.NewError(err) } defer c.unregisterPush(push) } @@ -719,10 +720,10 @@ func (c *cluster) PushTransaction( case pusheeTxn.Status.IsFinalized(): // Already finalized. return pusheeTxn, nil - case pushType == roachpb.PUSH_TIMESTAMP && pushTo.LessEq(pusheeTxn.WriteTimestamp): + case pushType == kvpb.PUSH_TIMESTAMP && pushTo.LessEq(pusheeTxn.WriteTimestamp): // Already pushed. return pusheeTxn, nil - case pushType == roachpb.PUSH_TOUCH: + case pushType == kvpb.PUSH_TOUCH: pusherWins = false case txnwait.CanPushWithPriority(pusherPriority, pusheeTxn.Priority): pusherWins = true @@ -731,26 +732,26 @@ func (c *cluster) PushTransaction( } if pusherWins { switch pushType { - case roachpb.PUSH_ABORT: + case kvpb.PUSH_ABORT: log.Eventf(ctx, "pusher aborted pushee") err = c.updateTxnRecord(pusheeTxn.ID, roachpb.ABORTED, pusheeTxn.WriteTimestamp) - case roachpb.PUSH_TIMESTAMP: + case kvpb.PUSH_TIMESTAMP: log.Eventf(ctx, "pusher pushed pushee to %s", pushTo) err = c.updateTxnRecord(pusheeTxn.ID, pusheeTxn.Status, pushTo) default: err = errors.Errorf("unexpected push type: %s", pushType) } if err != nil { - return nil, roachpb.NewError(err) + return nil, kvpb.NewError(err) } pusheeTxn, _ = pusheeRecord.asTxn() return pusheeTxn, nil } // If PUSH_TOUCH, return error instead of waiting. - if pushType == roachpb.PUSH_TOUCH { + if pushType == kvpb.PUSH_TOUCH { log.Eventf(ctx, "pushee not abandoned") - err := roachpb.NewTransactionPushError(*pusheeTxn) - return nil, roachpb.NewError(err) + err := kvpb.NewTransactionPushError(*pusheeTxn) + return nil, kvpb.NewError(err) } // Or the pusher aborted? var pusherRecordSig chan struct{} @@ -759,8 +760,8 @@ func (c *cluster) PushTransaction( pusherTxn, pusherRecordSig = pusherRecord.asTxn() if pusherTxn.Status == roachpb.ABORTED { log.Eventf(ctx, "detected pusher aborted") - err := roachpb.NewTransactionAbortedError(roachpb.ABORT_REASON_PUSHER_ABORTED) - return nil, roachpb.NewError(err) + err := kvpb.NewTransactionAbortedError(kvpb.ABORT_REASON_PUSHER_ABORTED) + return nil, kvpb.NewError(err) } } // Wait until either record is updated. @@ -768,7 +769,7 @@ func (c *cluster) PushTransaction( case <-pusheeRecordSig: case <-pusherRecordSig: case <-ctx.Done(): - return nil, roachpb.NewError(ctx.Err()) + return nil, kvpb.NewError(ctx.Err()) } } } @@ -776,7 +777,7 @@ func (c *cluster) PushTransaction( // ResolveIntent implements the concurrency.IntentResolver interface. func (c *cluster) ResolveIntent( ctx context.Context, intent roachpb.LockUpdate, _ intentresolver.ResolveOptions, -) *roachpb.Error { +) *kvpb.Error { var obsStr string if obs := intent.ClockWhilePending; obs != (roachpb.ObservedTimestamp{}) { obsStr = fmt.Sprintf(" and clock observation {%d %v}", obs.NodeID, obs.Timestamp) @@ -790,7 +791,7 @@ func (c *cluster) ResolveIntent( // ResolveIntents implements the concurrency.IntentResolver interface. func (c *cluster) ResolveIntents( ctx context.Context, intents []roachpb.LockUpdate, opts intentresolver.ResolveOptions, -) *roachpb.Error { +) *kvpb.Error { log.Eventf(ctx, "resolving a batch of %d intent(s)", len(intents)) for _, intent := range intents { if err := c.ResolveIntent(ctx, intent, opts); err != nil { @@ -996,10 +997,10 @@ func (c *cluster) resetNamespace() { // collectSpans collects the declared spans for a set of requests. // Its logic mirrors that in Replica.collectSpans. func (c *cluster) collectSpans( - t *testing.T, txn *roachpb.Transaction, ts hlc.Timestamp, reqs []roachpb.Request, + t *testing.T, txn *roachpb.Transaction, ts hlc.Timestamp, reqs []kvpb.Request, ) (latchSpans, lockSpans *spanset.SpanSet) { latchSpans, lockSpans = &spanset.SpanSet{}, &spanset.SpanSet{} - h := roachpb.Header{Txn: txn, Timestamp: ts} + h := kvpb.Header{Txn: txn, Timestamp: ts} for _, req := range reqs { if cmd, ok := batcheval.LookupCommand(req.Method()); ok { cmd.DeclareKeys(c.rangeDesc, &h, req, latchSpans, lockSpans, 0) diff --git a/pkg/kv/kvserver/concurrency/datadriven_util_test.go b/pkg/kv/kvserver/concurrency/datadriven_util_test.go index ce1433cd6c52..e647eb94afe1 100644 --- a/pkg/kv/kvserver/concurrency/datadriven_util_test.go +++ b/pkg/kv/kvserver/concurrency/datadriven_util_test.go @@ -14,6 +14,7 @@ import ( "strconv" "testing" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/poison" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -133,7 +134,7 @@ func scanPoisonPolicy(t *testing.T, d *datadriven.TestData) poison.Policy { func scanSingleRequest( t *testing.T, d *datadriven.TestData, line string, txns map[string]*roachpb.Transaction, -) roachpb.Request { +) kvpb.Request { cmd, cmdArgs, err := datadriven.ParseLine(line) if err != nil { d.Fatalf(t, "error parsing single request: %v", err) @@ -169,13 +170,13 @@ func scanSingleRequest( switch cmd { case "get": - var r roachpb.GetRequest + var r kvpb.GetRequest r.Sequence = maybeGetSeq() r.Key = roachpb.Key(mustGetField("key")) return &r case "scan": - var r roachpb.ScanRequest + var r kvpb.ScanRequest r.Sequence = maybeGetSeq() r.Key = roachpb.Key(mustGetField("key")) if v, ok := fields["endkey"]; ok { @@ -184,25 +185,25 @@ func scanSingleRequest( return &r case "put": - var r roachpb.PutRequest + var r kvpb.PutRequest r.Sequence = maybeGetSeq() r.Key = roachpb.Key(mustGetField("key")) r.Value.SetString(mustGetField("value")) return &r case "resolve-intent": - var r roachpb.ResolveIntentRequest + var r kvpb.ResolveIntentRequest r.IntentTxn = txns[mustGetField("txn")].TxnMeta r.Key = roachpb.Key(mustGetField("key")) r.Status = parseTxnStatus(t, d, mustGetField("status")) return &r case "request-lease": - var r roachpb.RequestLeaseRequest + var r kvpb.RequestLeaseRequest return &r case "barrier": - var r roachpb.BarrierRequest + var r kvpb.BarrierRequest r.Key = roachpb.Key(mustGetField("key")) if v, ok := fields["endkey"]; ok { r.EndKey = roachpb.Key(v) diff --git a/pkg/kv/kvserver/concurrency/latch_manager.go b/pkg/kv/kvserver/concurrency/latch_manager.go index b0f07057344b..d40c249eae32 100644 --- a/pkg/kv/kvserver/concurrency/latch_manager.go +++ b/pkg/kv/kvserver/concurrency/latch_manager.go @@ -13,10 +13,10 @@ package concurrency import ( "context" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/poison" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanlatch" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" - "github.com/cockroachdb/cockroach/pkg/roachpb" ) // latchManagerImpl implements the latchManager interface. @@ -27,7 +27,7 @@ type latchManagerImpl struct { func (m *latchManagerImpl) Acquire(ctx context.Context, req Request) (latchGuard, *Error) { lg, err := m.m.Acquire(ctx, req.LatchSpans, req.PoisonPolicy) if err != nil { - return nil, roachpb.NewError(err) + return nil, kvpb.NewError(err) } return lg, nil } @@ -46,7 +46,7 @@ func (m *latchManagerImpl) WaitUntilAcquired( ) (latchGuard, *Error) { lg, err := m.m.WaitUntilAcquired(ctx, lg.(*spanlatch.Guard)) if err != nil { - return nil, roachpb.NewError(err) + return nil, kvpb.NewError(err) } return lg, nil } @@ -56,7 +56,7 @@ func (m *latchManagerImpl) WaitFor( ) *Error { err := m.m.WaitFor(ctx, ss, pp) if err != nil { - return roachpb.NewError(err) + return kvpb.NewError(err) } return nil } diff --git a/pkg/kv/kvserver/concurrency/lock_table.go b/pkg/kv/kvserver/concurrency/lock_table.go index fb3f3e50eb9b..a6a5d870681f 100644 --- a/pkg/kv/kvserver/concurrency/lock_table.go +++ b/pkg/kv/kvserver/concurrency/lock_table.go @@ -19,6 +19,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -2980,10 +2981,10 @@ func (t *lockTableImpl) QueryLockTableState( // Check if adding the lock would exceed our byte or count limits, // though we must ensure we return at least one lock. if len(lockTableState) > 0 && opts.TargetBytes > 0 && (numBytes+nextByteSize) > opts.TargetBytes { - resumeState.ResumeReason = roachpb.RESUME_BYTE_LIMIT + resumeState.ResumeReason = kvpb.RESUME_BYTE_LIMIT break } else if len(lockTableState) > 0 && opts.MaxLocks > 0 && numLocks >= opts.MaxLocks { - resumeState.ResumeReason = roachpb.RESUME_KEY_LIMIT + resumeState.ResumeReason = kvpb.RESUME_KEY_LIMIT break } diff --git a/pkg/kv/kvserver/concurrency/lock_table_waiter.go b/pkg/kv/kvserver/concurrency/lock_table_waiter.go index 2c4ab296e9d9..c520f419752e 100644 --- a/pkg/kv/kvserver/concurrency/lock_table_waiter.go +++ b/pkg/kv/kvserver/concurrency/lock_table_waiter.go @@ -15,6 +15,7 @@ import ( "math" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/intentresolver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" @@ -121,7 +122,7 @@ type IntentResolver interface { // block until the pushee transaction is finalized or eventually can be // pushed successfully. PushTransaction( - context.Context, *enginepb.TxnMeta, roachpb.Header, roachpb.PushTxnType, + context.Context, *enginepb.TxnMeta, kvpb.Header, kvpb.PushTxnType, ) (*roachpb.Transaction, *Error) // ResolveIntent synchronously resolves the provided intent. @@ -433,10 +434,10 @@ func (w *lockTableWaiterImpl) WaitOn( } case <-ctxDoneC: - return roachpb.NewError(ctx.Err()) + return kvpb.NewError(ctx.Err()) case <-shouldQuiesceC: - return roachpb.NewError(&roachpb.NodeUnavailableError{}) + return kvpb.NewError(&kvpb.NodeUnavailableError{}) } } } @@ -465,7 +466,7 @@ func (w *lockTableWaiterImpl) pushLockTxn( // Construct the request header and determine which form of push to use. h := w.pushHeader(req) - var pushType roachpb.PushTxnType + var pushType kvpb.PushTxnType var beforePushObs roachpb.ObservedTimestamp switch req.WaitPolicy { case lock.WaitPolicy_Block: @@ -477,7 +478,7 @@ func (w *lockTableWaiterImpl) pushLockTxn( // with its own lock. switch ws.guardAccess { case spanset.SpanReadOnly: - pushType = roachpb.PUSH_TIMESTAMP + pushType = kvpb.PUSH_TIMESTAMP beforePushObs = roachpb.ObservedTimestamp{ NodeID: w.nodeDesc.NodeID, Timestamp: w.clock.NowAsClockTimestamp(), @@ -499,7 +500,7 @@ func (w *lockTableWaiterImpl) pushLockTxn( log.VEventf(ctx, 2, "pushing timestamp of txn %s above %s", ws.txn.ID.Short(), h.Timestamp) case spanset.SpanReadWrite: - pushType = roachpb.PUSH_ABORT + pushType = kvpb.PUSH_ABORT log.VEventf(ctx, 2, "pushing txn %s to abort", ws.txn.ID.Short()) } @@ -509,7 +510,7 @@ func (w *lockTableWaiterImpl) pushLockTxn( // holder to ensure that it is active and that this isn't an abandoned // lock, but we push using a PUSH_TOUCH to immediately return an error // if the lock hold is still active. - pushType = roachpb.PUSH_TOUCH + pushType = kvpb.PUSH_TOUCH log.VEventf(ctx, 2, "pushing txn %s to check if abandoned", ws.txn.ID.Short()) default: @@ -520,7 +521,7 @@ func (w *lockTableWaiterImpl) pushLockTxn( if err != nil { // If pushing with an Error WaitPolicy and the push fails, then the lock // holder is still active. Transform the error into a WriteIntentError. - if _, ok := err.GetDetail().(*roachpb.TransactionPushError); ok && req.WaitPolicy == lock.WaitPolicy_Error { + if _, ok := err.GetDetail().(*kvpb.TransactionPushError); ok && req.WaitPolicy == lock.WaitPolicy_Error { err = newWriteIntentErr(req, ws, reasonWaitPolicy) } return err @@ -665,7 +666,7 @@ func (w *lockTableWaiterImpl) pushLockTxnAfterTimeout( ) *Error { req.WaitPolicy = lock.WaitPolicy_Error err := w.pushLockTxn(ctx, req, ws) - if _, ok := err.GetDetail().(*roachpb.WriteIntentError); ok { + if _, ok := err.GetDetail().(*kvpb.WriteIntentError); ok { err = newWriteIntentErr(req, ws, reasonLockTimeout) } return err @@ -696,7 +697,7 @@ func (w *lockTableWaiterImpl) pushRequestTxn( // aborted due to a deadlock or b) the request exits the lock wait-queue and // the caller of this function cancels the push. h := w.pushHeader(req) - pushType := roachpb.PUSH_ABORT + pushType := kvpb.PUSH_ABORT log.VEventf(ctx, 3, "pushing txn %s to detect request deadlock", ws.txn.ID.Short()) _, err := w.ir.PushTransaction(ctx, ws.txn, h, pushType) @@ -758,8 +759,8 @@ func (w *lockTableWaiterImpl) pushRequestTxn( // pushHeader returns a BatchRequest header to be used for pushing other // transactions on behalf of the provided request. -func (w *lockTableWaiterImpl) pushHeader(req Request) roachpb.Header { - h := roachpb.Header{ +func (w *lockTableWaiterImpl) pushHeader(req Request) kvpb.Header { + h := kvpb.Header{ Timestamp: req.Timestamp, UserPriority: req.NonTxnPriority, } @@ -984,7 +985,7 @@ const tagWaited = "wait" // tracing span. type contentionEventTracer struct { sp *tracing.Span - onEvent func(event *roachpb.ContentionEvent) // may be nil + onEvent func(event *kvpb.ContentionEvent) // may be nil tag contentionTag } @@ -1065,7 +1066,7 @@ func newContentionEventTracer(sp *tracing.Span, clock *hlc.Clock) *contentionEve // SetOnContentionEvent registers a callback to be called before each event is // emitted. The callback may modify the event. -func (h *contentionEventTracer) SetOnContentionEvent(f func(ev *roachpb.ContentionEvent)) { +func (h *contentionEventTracer) SetOnContentionEvent(f func(ev *kvpb.ContentionEvent)) { h.onEvent = f } @@ -1090,7 +1091,7 @@ func (h *contentionEventTracer) notify(ctx context.Context, s waitingState) { // emit records a ContentionEvent to the tracing span corresponding to the // current wait state (if any). -func (h *contentionEventTracer) emit(event *roachpb.ContentionEvent) { +func (h *contentionEventTracer) emit(event *kvpb.ContentionEvent) { if event == nil { return } @@ -1102,12 +1103,12 @@ func (h *contentionEventTracer) emit(event *roachpb.ContentionEvent) { h.sp.RecordStructured(event) } -func (tag *contentionTag) generateEventLocked() *roachpb.ContentionEvent { +func (tag *contentionTag) generateEventLocked() *kvpb.ContentionEvent { if !tag.mu.waiting { return nil } - return &roachpb.ContentionEvent{ + return &kvpb.ContentionEvent{ Key: tag.mu.curState.key, TxnMeta: *tag.mu.curState.txn, Duration: tag.clock.PhysicalTime().Sub(tag.mu.curState.lockWaitStart), @@ -1115,7 +1116,7 @@ func (tag *contentionTag) generateEventLocked() *roachpb.ContentionEvent { } // See contentionEventTracer.notify. -func (tag *contentionTag) notify(ctx context.Context, s waitingState) *roachpb.ContentionEvent { +func (tag *contentionTag) notify(ctx context.Context, s waitingState) *kvpb.ContentionEvent { tag.mu.Lock() defer tag.mu.Unlock() @@ -1135,7 +1136,7 @@ func (tag *contentionTag) notify(ctx context.Context, s waitingState) *roachpb.C curLockHolder, curKey := tag.mu.curState.txn.ID, tag.mu.curState.key differentLock = !curLockHolder.Equal(s.txn.ID) || !curKey.Equal(s.key) } - var res *roachpb.ContentionEvent + var res *kvpb.ContentionEvent if differentLock { res = tag.generateEventLocked() } @@ -1205,15 +1206,13 @@ func (tag *contentionTag) Render() []attribute.KeyValue { } const ( - reasonWaitPolicy = roachpb.WriteIntentError_REASON_WAIT_POLICY - reasonLockTimeout = roachpb.WriteIntentError_REASON_LOCK_TIMEOUT - reasonWaitQueueMaxLengthExceeded = roachpb.WriteIntentError_REASON_LOCK_WAIT_QUEUE_MAX_LENGTH_EXCEEDED + reasonWaitPolicy = kvpb.WriteIntentError_REASON_WAIT_POLICY + reasonLockTimeout = kvpb.WriteIntentError_REASON_LOCK_TIMEOUT + reasonWaitQueueMaxLengthExceeded = kvpb.WriteIntentError_REASON_LOCK_WAIT_QUEUE_MAX_LENGTH_EXCEEDED ) -func newWriteIntentErr( - req Request, ws waitingState, reason roachpb.WriteIntentError_Reason, -) *Error { - err := roachpb.NewError(&roachpb.WriteIntentError{ +func newWriteIntentErr(req Request, ws waitingState, reason kvpb.WriteIntentError_Reason) *Error { + err := kvpb.NewError(&kvpb.WriteIntentError{ Intents: []roachpb.Intent{roachpb.MakeIntent(ws.txn, ws.key)}, Reason: reason, }) diff --git a/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go b/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go index 5ca921c98f9f..14282c1b3429 100644 --- a/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go +++ b/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go @@ -18,6 +18,7 @@ import ( "testing" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/intentresolver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" @@ -37,14 +38,14 @@ import ( ) type mockIntentResolver struct { - pushTxn func(context.Context, *enginepb.TxnMeta, roachpb.Header, roachpb.PushTxnType) (*roachpb.Transaction, *Error) + pushTxn func(context.Context, *enginepb.TxnMeta, kvpb.Header, kvpb.PushTxnType) (*roachpb.Transaction, *Error) resolveIntent func(context.Context, roachpb.LockUpdate) *Error resolveIntents func(context.Context, []roachpb.LockUpdate) *Error } // mockIntentResolver implements the IntentResolver interface. func (m *mockIntentResolver) PushTransaction( - ctx context.Context, txn *enginepb.TxnMeta, h roachpb.Header, pushType roachpb.PushTxnType, + ctx context.Context, txn *enginepb.TxnMeta, h kvpb.Header, pushType kvpb.PushTxnType, ) (*roachpb.Transaction, *Error) { return m.pushTxn(ctx, txn, h, pushType) } @@ -226,7 +227,7 @@ func TestLockTableWaiterWithTxn(t *testing.T) { err := w.WaitOn(ctx, makeReq(), g) require.NotNil(t, err) - require.IsType(t, &roachpb.NodeUnavailableError{}, err.GetDetail()) + require.IsType(t, &kvpb.NodeUnavailableError{}, err.GetDetail()) }) }) } @@ -302,7 +303,7 @@ func TestLockTableWaiterWithNonTxn(t *testing.T) { err := w.WaitOn(ctx, makeReq(), g) require.NotNil(t, err) - require.IsType(t, &roachpb.NodeUnavailableError{}, err.GetDetail()) + require.IsType(t, &kvpb.NodeUnavailableError{}, err.GetDetail()) }) } @@ -348,16 +349,16 @@ func testWaitPush(t *testing.T, k waitKind, makeReq func() Request, expPushTS hl ir.pushTxn = func( _ context.Context, pusheeArg *enginepb.TxnMeta, - h roachpb.Header, - pushType roachpb.PushTxnType, + h kvpb.Header, + pushType kvpb.PushTxnType, ) (*roachpb.Transaction, *Error) { require.Equal(t, &pusheeTxn.TxnMeta, pusheeArg) require.Equal(t, req.Txn, h.Txn) require.Equal(t, expPushTS, h.Timestamp) if waitAsWrite || !lockHeld { - require.Equal(t, roachpb.PUSH_ABORT, pushType) + require.Equal(t, kvpb.PUSH_ABORT, pushType) } else { - require.Equal(t, roachpb.PUSH_TIMESTAMP, pushType) + require.Equal(t, kvpb.PUSH_TIMESTAMP, pushType) } resp := &roachpb.Transaction{TxnMeta: *pusheeArg, Status: roachpb.ABORTED} @@ -488,7 +489,7 @@ func testErrorWaitPush( k waitKind, makeReq func() Request, expPushTS hlc.Timestamp, - errReason roachpb.WriteIntentError_Reason, + errReason kvpb.WriteIntentError_Reason, ) { ctx := context.Background() keyA := roachpb.Key("keyA") @@ -523,7 +524,7 @@ func testErrorWaitPush( require.Nil(t, err) } else { require.NotNil(t, err) - wiErr := new(roachpb.WriteIntentError) + wiErr := new(kvpb.WriteIntentError) require.True(t, errors.As(err.GoError(), &wiErr)) require.Equal(t, errReason, wiErr.Reason) } @@ -533,17 +534,17 @@ func testErrorWaitPush( ir.pushTxn = func( _ context.Context, pusheeArg *enginepb.TxnMeta, - h roachpb.Header, - pushType roachpb.PushTxnType, + h kvpb.Header, + pushType kvpb.PushTxnType, ) (*roachpb.Transaction, *Error) { require.Equal(t, &pusheeTxn.TxnMeta, pusheeArg) require.Equal(t, req.Txn, h.Txn) require.Equal(t, expPushTS, h.Timestamp) - require.Equal(t, roachpb.PUSH_TOUCH, pushType) + require.Equal(t, kvpb.PUSH_TOUCH, pushType) resp := &roachpb.Transaction{TxnMeta: *pusheeArg, Status: roachpb.PENDING} if pusheeActive { - return nil, roachpb.NewError(&roachpb.TransactionPushError{ + return nil, kvpb.NewError(&kvpb.TransactionPushError{ PusheeTxn: *resp, }) } @@ -570,7 +571,7 @@ func testErrorWaitPush( err := w.WaitOn(ctx, req, g) if pusheeActive { require.NotNil(t, err) - wiErr := new(roachpb.WriteIntentError) + wiErr := new(kvpb.WriteIntentError) require.True(t, errors.As(err.GoError(), &wiErr)) require.Equal(t, errReason, wiErr.Reason) } else { @@ -693,7 +694,7 @@ func testWaitPushWithTimeout(t *testing.T, k waitKind, makeReq func() Request) { if !lockHeld && timeoutBeforePush { err := w.WaitOn(ctx, req, g) require.NotNil(t, err) - wiErr := new(roachpb.WriteIntentError) + wiErr := new(kvpb.WriteIntentError) require.True(t, errors.As(err.GoError(), &wiErr)) require.Equal(t, reasonLockTimeout, wiErr.Reason) return @@ -705,14 +706,14 @@ func testWaitPushWithTimeout(t *testing.T, k waitKind, makeReq func() Request) { ir.pushTxn = func( ctx context.Context, pusheeArg *enginepb.TxnMeta, - h roachpb.Header, - pushType roachpb.PushTxnType, + h kvpb.Header, + pushType kvpb.PushTxnType, ) (*roachpb.Transaction, *Error) { require.Equal(t, &pusheeTxn.TxnMeta, pusheeArg) require.Equal(t, req.Txn, h.Txn) if expBlockingPush { - require.Equal(t, roachpb.PUSH_ABORT, pushType) + require.Equal(t, kvpb.PUSH_ABORT, pushType) _, hasDeadline := ctx.Deadline() require.True(t, hasDeadline) sawBlockingPush = true @@ -720,17 +721,17 @@ func testWaitPushWithTimeout(t *testing.T, k waitKind, makeReq func() Request) { // Wait for the context to hit its timeout. <-ctx.Done() - return nil, roachpb.NewError(ctx.Err()) + return nil, kvpb.NewError(ctx.Err()) } - require.Equal(t, roachpb.PUSH_TOUCH, pushType) + require.Equal(t, kvpb.PUSH_TOUCH, pushType) _, hasDeadline := ctx.Deadline() require.False(t, hasDeadline) sawNonBlockingPush = true resp := &roachpb.Transaction{TxnMeta: *pusheeArg, Status: roachpb.PENDING} if pusheeActive { - return nil, roachpb.NewError(&roachpb.TransactionPushError{ + return nil, kvpb.NewError(&kvpb.TransactionPushError{ PusheeTxn: *resp, }) } @@ -757,7 +758,7 @@ func testWaitPushWithTimeout(t *testing.T, k waitKind, makeReq func() Request) { err := w.WaitOn(ctx, req, g) if pusheeActive { require.NotNil(t, err) - wiErr := new(roachpb.WriteIntentError) + wiErr := new(kvpb.WriteIntentError) require.True(t, errors.As(err.GoError(), &wiErr)) require.Equal(t, reasonLockTimeout, wiErr.Reason) } else { @@ -780,8 +781,8 @@ func TestLockTableWaiterIntentResolverError(t *testing.T) { w, ir, g, _ := setupLockTableWaiterTest() defer w.stopper.Stop(ctx) - err1 := roachpb.NewErrorf("error1") - err2 := roachpb.NewErrorf("error2") + err1 := kvpb.NewErrorf("error1") + err2 := kvpb.NewErrorf("error2") txn := makeTxnProto("request") req := Request{ @@ -806,7 +807,7 @@ func TestLockTableWaiterIntentResolverError(t *testing.T) { // Errors are propagated when observed while pushing transactions. g.notify() ir.pushTxn = func( - _ context.Context, _ *enginepb.TxnMeta, _ roachpb.Header, _ roachpb.PushTxnType, + _ context.Context, _ *enginepb.TxnMeta, _ kvpb.Header, _ kvpb.PushTxnType, ) (*roachpb.Transaction, *Error) { return nil, err1 } @@ -817,7 +818,7 @@ func TestLockTableWaiterIntentResolverError(t *testing.T) { // Errors are propagated when observed while resolving intents. g.notify() ir.pushTxn = func( - _ context.Context, _ *enginepb.TxnMeta, _ roachpb.Header, _ roachpb.PushTxnType, + _ context.Context, _ *enginepb.TxnMeta, _ kvpb.Header, _ kvpb.PushTxnType, ) (*roachpb.Transaction, *Error) { return &pusheeTxn, nil } @@ -860,7 +861,7 @@ func TestLockTableWaiterDeferredIntentResolverError(t *testing.T) { g.notify() // Errors are propagated when observed while resolving batches of intents. - err1 := roachpb.NewErrorf("error1") + err1 := kvpb.NewErrorf("error1") ir.resolveIntents = func(_ context.Context, intents []roachpb.LockUpdate) *Error { require.Len(t, intents, 1) require.Equal(t, keyA, intents[0].Key) @@ -943,10 +944,10 @@ func TestContentionEventTracer(t *testing.T) { defer sp.Finish() clock := hlc.NewClockForTesting(nil) - var events []*roachpb.ContentionEvent + var events []*kvpb.ContentionEvent h := newContentionEventTracer(sp, clock) - h.SetOnContentionEvent(func(ev *roachpb.ContentionEvent) { + h.SetOnContentionEvent(func(ev *kvpb.ContentionEvent) { events = append(events, ev) }) txn := makeTxnProto("foo") diff --git a/pkg/kv/kvserver/consistency_queue.go b/pkg/kv/kvserver/consistency_queue.go index 4bb14c274aec..662e7276b854 100644 --- a/pkg/kv/kvserver/consistency_queue.go +++ b/pkg/kv/kvserver/consistency_queue.go @@ -14,6 +14,7 @@ import ( "context" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/spanconfig" @@ -168,7 +169,7 @@ func (q *consistencyQueue) process( log.VErrEventf(ctx, 2, "failed to update last processed time: %v", err) } - req := roachpb.CheckConsistencyRequest{ + req := kvpb.CheckConsistencyRequest{ // Tell CheckConsistency that the caller is the queue. This triggers // code to handle inconsistencies by recomputing with a diff and // instructing the nodes in the minority to terminate with a fatal @@ -176,7 +177,7 @@ func (q *consistencyQueue) process( // inconsistency but the persisted stats are found to disagree with // those reflected in the data. All of this really ought to be lifted // into the queue in the future. - Mode: roachpb.ChecksumMode_CHECK_VIA_QUEUE, + Mode: kvpb.ChecksumMode_CHECK_VIA_QUEUE, } resp, pErr := repl.CheckConsistency(ctx, req) if pErr != nil { diff --git a/pkg/kv/kvserver/consistency_queue_test.go b/pkg/kv/kvserver/consistency_queue_test.go index 784177996636..877ca26ab01a 100644 --- a/pkg/kv/kvserver/consistency_queue_test.go +++ b/pkg/kv/kvserver/consistency_queue_test.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvstorage" @@ -140,13 +141,13 @@ func TestCheckConsistencyMultiStore(t *testing.T) { // Run consistency check on r1 and ScratchRange. for _, k := range []roachpb.Key{keys.LocalMax, k2} { t.Run(k.String(), func(t *testing.T) { - checkArgs := roachpb.CheckConsistencyRequest{ - RequestHeader: roachpb.RequestHeader{ + checkArgs := kvpb.CheckConsistencyRequest{ + RequestHeader: kvpb.RequestHeader{ Key: keys.LocalMax, EndKey: keys.LocalMax.Next(), }, } - _, pErr := kv.SendWrappedWith(context.Background(), store.DB().NonTransactionalSender(), roachpb.Header{ + _, pErr := kv.SendWrappedWith(context.Background(), store.DB().NonTransactionalSender(), kvpb.Header{ Timestamp: store.Clock().Now(), }, &checkArgs) require.NoError(t, pErr.GoError()) @@ -177,7 +178,7 @@ func TestCheckConsistencyReplay(t *testing.T) { } // Arrange to count the number of times each checksum command applies to each // store. - testKnobs.TestingPostApplyFilter = func(args kvserverbase.ApplyFilterArgs) (int, *roachpb.Error) { + testKnobs.TestingPostApplyFilter = func(args kvserverbase.ApplyFilterArgs) (int, *kvpb.Error) { state.Lock() defer state.Unlock() if ccr := args.ComputeChecksum; ccr != nil { @@ -188,14 +189,14 @@ func TestCheckConsistencyReplay(t *testing.T) { // Arrange to trigger a retry when a ComputeChecksum request arrives. testKnobs.TestingResponseFilter = func( - ctx context.Context, ba *roachpb.BatchRequest, br *roachpb.BatchResponse, - ) *roachpb.Error { + ctx context.Context, ba *kvpb.BatchRequest, br *kvpb.BatchResponse, + ) *kvpb.Error { state.Lock() defer state.Unlock() if ba.IsSingleComputeChecksumRequest() && !state.forcedRetry { state.forcedRetry = true // We need to return a retryable error from the perspective of the sender - return roachpb.NewError(&roachpb.NotLeaseHolderError{}) + return kvpb.NewError(&kvpb.NotLeaseHolderError{}) } return nil } @@ -214,8 +215,8 @@ func TestCheckConsistencyReplay(t *testing.T) { defer tc.Stopper().Stop(context.Background()) store := tc.GetFirstStoreFromServer(t, 0) - checkArgs := roachpb.CheckConsistencyRequest{ - RequestHeader: roachpb.RequestHeader{ + checkArgs := kvpb.CheckConsistencyRequest{ + RequestHeader: kvpb.RequestHeader{ Key: []byte("a"), EndKey: []byte("b"), }, @@ -301,17 +302,17 @@ func TestCheckConsistencyInconsistent(t *testing.T) { require.NoError(t, err.GoError()) } - runConsistencyCheck := func() *roachpb.CheckConsistencyResponse { - req := roachpb.CheckConsistencyRequest{ - RequestHeader: roachpb.RequestHeader{ // keys span that includes "a" & "c" + runConsistencyCheck := func() *kvpb.CheckConsistencyResponse { + req := kvpb.CheckConsistencyRequest{ + RequestHeader: kvpb.RequestHeader{ // keys span that includes "a" & "c" Key: []byte("a"), EndKey: []byte("z"), }, - Mode: roachpb.ChecksumMode_CHECK_VIA_QUEUE, + Mode: kvpb.ChecksumMode_CHECK_VIA_QUEUE, } resp, err := kv.SendWrapped(context.Background(), store.DB().NonTransactionalSender(), &req) require.NoError(t, err.GoError()) - return resp.(*roachpb.CheckConsistencyResponse) + return resp.(*kvpb.CheckConsistencyResponse) } onDiskCheckpointPaths := func(nodeIdx int) []string { @@ -331,7 +332,7 @@ func TestCheckConsistencyInconsistent(t *testing.T) { // Run the check the first time, it shouldn't find anything. resp := runConsistencyCheck() require.Len(t, resp.Result, 1) - assert.Equal(t, roachpb.CheckConsistencyResponse_RANGE_CONSISTENT, resp.Result[0].Status) + assert.Equal(t, kvpb.CheckConsistencyResponse_RANGE_CONSISTENT, resp.Result[0].Status) select { case <-notifyFatal: t.Fatal("unexpected panic") @@ -363,7 +364,7 @@ func TestCheckConsistencyInconsistent(t *testing.T) { } require.Len(t, resp.Result, 1) - assert.Equal(t, roachpb.CheckConsistencyResponse_RANGE_INCONSISTENT, resp.Result[0].Status) + assert.Equal(t, kvpb.CheckConsistencyResponse_RANGE_INCONSISTENT, resp.Result[0].Status) assert.Contains(t, resp.Result[0].Detail, `[minority]`) assert.Contains(t, resp.Result[0].Detail, `stats`) @@ -402,7 +403,7 @@ func TestCheckConsistencyInconsistent(t *testing.T) { // Compute a checksum over the content of the problematic range. rd, err := kvserver.CalcReplicaDigest(context.Background(), *desc, cpEng, - roachpb.ChecksumMode_CHECK_FULL, quotapool.NewRateLimiter("test", quotapool.Inf(), 0)) + kvpb.ChecksumMode_CHECK_FULL, quotapool.NewRateLimiter("test", quotapool.Inf(), 0)) require.NoError(t, err) hashes[i] = rd.SHA512[:] } @@ -451,10 +452,10 @@ func testConsistencyQueueRecomputeStatsImpl(t *testing.T, hadEstimates bool) { ScanMaxIdleTime: 100 * time.Millisecond, } - ccCh := make(chan roachpb.CheckConsistencyResponse, 1) + ccCh := make(chan kvpb.CheckConsistencyResponse, 1) knobs := &kvserver.StoreTestingKnobs{} - knobs.ConsistencyTestingKnobs.ConsistencyQueueResultHook = func(resp roachpb.CheckConsistencyResponse) { - if len(resp.Result) == 0 || resp.Result[0].Status != roachpb.CheckConsistencyResponse_RANGE_CONSISTENT_STATS_INCORRECT { + knobs.ConsistencyTestingKnobs.ConsistencyQueueResultHook = func(resp kvpb.CheckConsistencyResponse) { + if len(resp.Result) == 0 || resp.Result[0].Status != kvpb.CheckConsistencyResponse_RANGE_CONSISTENT_STATS_INCORRECT { // Ignore recomputations triggered by the time series ranges. return } @@ -481,12 +482,12 @@ func testConsistencyQueueRecomputeStatsImpl(t *testing.T, hadEstimates bool) { computeDelta := func(db *kv.DB) enginepb.MVCCStats { var b kv.Batch - b.AddRawRequest(&roachpb.RecomputeStatsRequest{ - RequestHeader: roachpb.RequestHeader{Key: key}, + b.AddRawRequest(&kvpb.RecomputeStatsRequest{ + RequestHeader: kvpb.RequestHeader{Key: key}, DryRun: true, }) require.NoError(t, db.Run(ctx, &b)) - resp := b.RawResponse().Responses[0].GetInner().(*roachpb.RecomputeStatsResponse) + resp := b.RawResponse().Responses[0].GetInner().(*kvpb.RecomputeStatsResponse) delta := enginepb.MVCCStats(resp.AddedDelta) delta.AgeTo(0) return delta @@ -620,7 +621,7 @@ func testConsistencyQueueRecomputeStatsImpl(t *testing.T, hadEstimates bool) { select { case resp := <-ccCh: assert.Contains(t, resp.Result[0].Detail, `KeyBytes`) // contains printed stats - assert.Equal(t, roachpb.CheckConsistencyResponse_RANGE_CONSISTENT_STATS_INCORRECT, resp.Result[0].Status) + assert.Equal(t, kvpb.CheckConsistencyResponse_RANGE_CONSISTENT_STATS_INCORRECT, resp.Result[0].Status) assert.False(t, hadEstimates) default: assert.True(t, hadEstimates) diff --git a/pkg/kv/kvserver/gc/BUILD.bazel b/pkg/kv/kvserver/gc/BUILD.bazel index 25ac6661cecb..62a3a03c3ae9 100644 --- a/pkg/kv/kvserver/gc/BUILD.bazel +++ b/pkg/kv/kvserver/gc/BUILD.bazel @@ -12,6 +12,7 @@ go_library( deps = [ "//pkg/base", "//pkg/keys", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/abortspan", "//pkg/kv/kvserver/rditer", "//pkg/roachpb", @@ -48,6 +49,7 @@ go_test( "//pkg/base", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/kv/kvserver/rditer", "//pkg/roachpb", diff --git a/pkg/kv/kvserver/gc/gc.go b/pkg/kv/kvserver/gc/gc.go index 40182aad27df..9768e4719ae7 100644 --- a/pkg/kv/kvserver/gc/gc.go +++ b/pkg/kv/kvserver/gc/gc.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/abortspan" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -176,8 +177,8 @@ type Thresholder interface { // PureGCer is part of the GCer interface. type PureGCer interface { - GC(context.Context, []roachpb.GCRequest_GCKey, []roachpb.GCRequest_GCRangeKey, - *roachpb.GCRequest_GCClearRange, + GC(context.Context, []kvpb.GCRequest_GCKey, []kvpb.GCRequest_GCRangeKey, + *kvpb.GCRequest_GCClearRange, ) error } @@ -198,9 +199,9 @@ func (NoopGCer) SetGCThreshold(context.Context, Threshold) error { return nil } // GC implements storage.GCer. func (NoopGCer) GC( context.Context, - []roachpb.GCRequest_GCKey, - []roachpb.GCRequest_GCRangeKey, - *roachpb.GCRequest_GCClearRange, + []kvpb.GCRequest_GCKey, + []kvpb.GCRequest_GCRangeKey, + *kvpb.GCRequest_GCClearRange, ) error { return nil } @@ -439,7 +440,7 @@ func processReplicatedKeyRange( end := desc.EndKey.AsRawKey() if coveredByRangeTombstone, err := storage.CanGCEntireRange(ctx, snap, start, end, threshold); err == nil && coveredByRangeTombstone { - if err = gcer.GC(ctx, nil, nil, &roachpb.GCRequest_GCClearRange{ + if err = gcer.GC(ctx, nil, nil, &kvpb.GCRequest_GCClearRange{ StartKey: start, EndKey: end, }); err == nil { @@ -599,7 +600,7 @@ type gcKeyBatcherThresholds struct { type pointsBatch struct { gcBatchCounters - batchGCKeys []roachpb.GCRequest_GCKey + batchGCKeys []kvpb.GCRequest_GCKey alloc bufalloc.ByteAllocator } @@ -763,7 +764,7 @@ func (b *gcKeyBatcher) foundGarbage( if b.prevWasNewest || len(b.pointsBatches[i].batchGCKeys) == 0 { b.pointsBatches[i].alloc, key = b.pointsBatches[i].alloc.Copy(cur.key.Key, 0) b.pointsBatches[i].batchGCKeys = append(b.pointsBatches[i].batchGCKeys, - roachpb.GCRequest_GCKey{Key: key, Timestamp: cur.key.Timestamp}) + kvpb.GCRequest_GCKey{Key: key, Timestamp: cur.key.Timestamp}) keyMemUsed := len(key) + hlcTimestampSize b.pointsBatches[i].memUsed += keyMemUsed b.totalMemUsed += keyMemUsed @@ -874,7 +875,7 @@ func (b *gcKeyBatcher) maybeFlushPendingBatches(ctx context.Context) (err error) if b.clearRangeCounters.keysAffected == 1 { endRange = b.clearRangeStartKey.Key.Next() } - if err := b.gcer.GC(ctx, nil, nil, &roachpb.GCRequest_GCClearRange{ + if err := b.gcer.GC(ctx, nil, nil, &kvpb.GCRequest_GCClearRange{ StartKey: b.clearRangeStartKey.Key, StartKeyTimestamp: b.clearRangeStartKey.Timestamp, EndKey: endRange, @@ -1308,9 +1309,9 @@ func (b *rangeKeyBatcher) addRangeKey(unsafeRk storage.MVCCRangeKey) { func (b *rangeKeyBatcher) flushPendingFragments(ctx context.Context) error { if pendingCount := len(b.pending); pendingCount > 0 { - toSend := make([]roachpb.GCRequest_GCRangeKey, pendingCount) + toSend := make([]kvpb.GCRequest_GCRangeKey, pendingCount) for i, rk := range b.pending { - toSend[i] = roachpb.GCRequest_GCRangeKey{ + toSend[i] = kvpb.GCRequest_GCRangeKey{ StartKey: rk.StartKey, EndKey: rk.EndKey, Timestamp: rk.Timestamp, @@ -1384,7 +1385,7 @@ type batchingInlineGCer struct { size int max int - gcKeys []roachpb.GCRequest_GCKey + gcKeys []kvpb.GCRequest_GCKey } func makeBatchingInlineGCer(gcer PureGCer, onErr func(error)) batchingInlineGCer { @@ -1392,7 +1393,7 @@ func makeBatchingInlineGCer(gcer PureGCer, onErr func(error)) batchingInlineGCer } func (b *batchingInlineGCer) FlushingAdd(ctx context.Context, key roachpb.Key) { - b.gcKeys = append(b.gcKeys, roachpb.GCRequest_GCKey{Key: key}) + b.gcKeys = append(b.gcKeys, kvpb.GCRequest_GCKey{Key: key}) b.size += len(key) if b.size < b.max { return diff --git a/pkg/kv/kvserver/gc/gc_old_test.go b/pkg/kv/kvserver/gc/gc_old_test.go index fa16af692978..f87d1afabd65 100644 --- a/pkg/kv/kvserver/gc/gc_old_test.go +++ b/pkg/kv/kvserver/gc/gc_old_test.go @@ -16,6 +16,7 @@ import ( "testing" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" @@ -70,7 +71,7 @@ func runGCOld( return Info{}, errors.Wrap(err, "failed to set GC thresholds") } - var batchGCKeys []roachpb.GCRequest_GCKey + var batchGCKeys []kvpb.GCRequest_GCKey var batchGCKeysBytes int64 var expBaseKey roachpb.Key var keys []storage.MVCCKey @@ -148,7 +149,7 @@ func runGCOld( // size, add the current timestamp to finish the current // chunk and start a new one. if batchGCKeysBytes >= KeyVersionChunkBytes { - batchGCKeys = append(batchGCKeys, roachpb.GCRequest_GCKey{Key: expBaseKey, Timestamp: keys[i].Timestamp}) + batchGCKeys = append(batchGCKeys, kvpb.GCRequest_GCKey{Key: expBaseKey, Timestamp: keys[i].Timestamp}) err := gcer.GC(ctx, batchGCKeys, nil, nil) @@ -167,7 +168,7 @@ func runGCOld( } // Add the key to the batch at the GC timestamp, unless it was already added. if batchGCKeysBytes != 0 { - batchGCKeys = append(batchGCKeys, roachpb.GCRequest_GCKey{Key: expBaseKey, Timestamp: gcTS}) + batchGCKeys = append(batchGCKeys, kvpb.GCRequest_GCKey{Key: expBaseKey, Timestamp: gcTS}) } info.NumKeysAffected++ } diff --git a/pkg/kv/kvserver/gc/gc_random_test.go b/pkg/kv/kvserver/gc/gc_random_test.go index c87bfdf183c3..6259b3ea1623 100644 --- a/pkg/kv/kvserver/gc/gc_random_test.go +++ b/pkg/kv/kvserver/gc/gc_random_test.go @@ -19,6 +19,7 @@ import ( "testing" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" @@ -756,13 +757,13 @@ func mergeRanges(fragments [][]storage.MVCCRangeKeyValue) []storage.MVCCRangeKey } type fakeGCer struct { - gcKeys map[string]roachpb.GCRequest_GCKey - gcPointsBatches [][]roachpb.GCRequest_GCKey + gcKeys map[string]kvpb.GCRequest_GCKey + gcPointsBatches [][]kvpb.GCRequest_GCKey // fake GCer stores range key batches as it since we need to be able to // feed them into MVCCGarbageCollectRangeKeys and ranges argument should be // non-overlapping. - gcRangeKeyBatches [][]roachpb.GCRequest_GCRangeKey - gcClearRanges []roachpb.GCRequest_GCClearRange + gcRangeKeyBatches [][]kvpb.GCRequest_GCRangeKey + gcClearRanges []kvpb.GCRequest_GCClearRange threshold Threshold intents []roachpb.Intent batches [][]roachpb.Intent @@ -771,7 +772,7 @@ type fakeGCer struct { func makeFakeGCer() fakeGCer { return fakeGCer{ - gcKeys: make(map[string]roachpb.GCRequest_GCKey), + gcKeys: make(map[string]kvpb.GCRequest_GCKey), } } @@ -784,9 +785,9 @@ func (f *fakeGCer) SetGCThreshold(ctx context.Context, t Threshold) error { func (f *fakeGCer) GC( ctx context.Context, - keys []roachpb.GCRequest_GCKey, - rangeKeys []roachpb.GCRequest_GCRangeKey, - clearRange *roachpb.GCRequest_GCClearRange, + keys []kvpb.GCRequest_GCKey, + rangeKeys []kvpb.GCRequest_GCRangeKey, + clearRange *kvpb.GCRequest_GCClearRange, ) error { for _, k := range keys { f.gcKeys[k.Key.String()] = k @@ -798,7 +799,7 @@ func (f *fakeGCer) GC( f.gcRangeKeyBatches = append(f.gcRangeKeyBatches, rangeKeys) } if clearRange != nil { - f.gcClearRanges = append(f.gcClearRanges, roachpb.GCRequest_GCClearRange{ + f.gcClearRanges = append(f.gcClearRanges, kvpb.GCRequest_GCClearRange{ StartKey: clearRange.StartKey.Clone(), StartKeyTimestamp: clearRange.StartKeyTimestamp, EndKey: clearRange.EndKey.Clone(), @@ -837,27 +838,27 @@ func (f *fakeGCer) normalize() { f.gcPointsBatches = nil } -func (f *fakeGCer) pointKeys() []roachpb.GCRequest_GCKey { - var reqs []roachpb.GCRequest_GCKey +func (f *fakeGCer) pointKeys() []kvpb.GCRequest_GCKey { + var reqs []kvpb.GCRequest_GCKey for _, r := range f.gcKeys { reqs = append(reqs, r) } return reqs } -func (f *fakeGCer) rangeKeyBatches() [][]roachpb.GCRequest_GCRangeKey { +func (f *fakeGCer) rangeKeyBatches() [][]kvpb.GCRequest_GCRangeKey { return f.gcRangeKeyBatches } -func (f *fakeGCer) rangeKeys() []roachpb.GCRequest_GCRangeKey { - var reqs []roachpb.GCRequest_GCRangeKey +func (f *fakeGCer) rangeKeys() []kvpb.GCRequest_GCRangeKey { + var reqs []kvpb.GCRequest_GCRangeKey for _, r := range f.gcRangeKeyBatches { reqs = append(reqs, r...) } return reqs } -func (f *fakeGCer) clearRanges() []roachpb.GCRequest_GCClearRange { +func (f *fakeGCer) clearRanges() []kvpb.GCRequest_GCClearRange { return f.gcClearRanges } @@ -881,7 +882,7 @@ type txnIntents struct { // makeCollectableGCRangesFromGCRequests mirrors // MakeCollectableGCRangesFromGCRequests to break cyclic dependecies. func makeCollectableGCRangesFromGCRequests( - rangeStart, rangeEnd roachpb.Key, rangeKeys []roachpb.GCRequest_GCRangeKey, + rangeStart, rangeEnd roachpb.Key, rangeKeys []kvpb.GCRequest_GCRangeKey, ) []storage.CollectableGCRangeKey { collectableKeys := make([]storage.CollectableGCRangeKey, len(rangeKeys)) for i, rk := range rangeKeys { diff --git a/pkg/kv/kvserver/gc/gc_test.go b/pkg/kv/kvserver/gc/gc_test.go index 665f912891cd..45a63253245a 100644 --- a/pkg/kv/kvserver/gc/gc_test.go +++ b/pkg/kv/kvserver/gc/gc_test.go @@ -24,6 +24,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" @@ -54,14 +55,14 @@ func TestCalculateThreshold(t *testing.T) { } type collectingGCer struct { - keys [][]roachpb.GCRequest_GCKey + keys [][]kvpb.GCRequest_GCKey } func (c *collectingGCer) GC( _ context.Context, - keys []roachpb.GCRequest_GCKey, - _ []roachpb.GCRequest_GCRangeKey, - _ *roachpb.GCRequest_GCClearRange, + keys []kvpb.GCRequest_GCKey, + _ []kvpb.GCRequest_GCRangeKey, + _ *kvpb.GCRequest_GCClearRange, ) error { c.keys = append(c.keys, keys) return nil @@ -78,10 +79,10 @@ func TestBatchingInlineGCer(t *testing.T) { } m.max = 10 // something reasonable for this unit test - long := roachpb.GCRequest_GCKey{ + long := kvpb.GCRequest_GCKey{ Key: bytes.Repeat([]byte("x"), m.max-1), } - short := roachpb.GCRequest_GCKey{ + short := kvpb.GCRequest_GCKey{ Key: roachpb.Key("q"), } @@ -744,7 +745,7 @@ func TestGC(t *testing.T) { } } -type gCR roachpb.GCRequest_GCClearRange +type gCR kvpb.GCRequest_GCClearRange // Format implements the fmt.Formatter interface. func (k gCR) Format(f fmt.State, r rune) { @@ -753,7 +754,7 @@ func (k gCR) Format(f fmt.State, r rune) { k.EndKey.Format(f, r) } -type gCRS []roachpb.GCRequest_GCClearRange +type gCRS []kvpb.GCRequest_GCClearRange func (k gCRS) toTestData() (spans []gCR) { if len(k) == 0 { @@ -770,7 +771,7 @@ func (k gCRS) toTestData() (spans []gCR) { return spans } -type clearPointsKey roachpb.GCRequest_GCKey +type clearPointsKey kvpb.GCRequest_GCKey // Format implements the fmt.Formatter interface. func (k clearPointsKey) Format(f fmt.State, c rune) { @@ -780,7 +781,7 @@ func (k clearPointsKey) Format(f fmt.State, c rune) { }.Format(f, c) } -type gcPointsBatches [][]roachpb.GCRequest_GCKey +type gcPointsBatches [][]kvpb.GCRequest_GCKey // toTestData converts batches to test data structs that are easier to create // and assert. This method also ignores nil convention that allows tests to @@ -1714,8 +1715,8 @@ func TestRangeKeyBatching(t *testing.T) { return rangeKeys } - mkGCr := func(start, end string, ts int) roachpb.GCRequest_GCRangeKey { - return roachpb.GCRequest_GCRangeKey{ + mkGCr := func(start, end string, ts int) kvpb.GCRequest_GCRangeKey { + return kvpb.GCRequest_GCRangeKey{ StartKey: mkKey(start), EndKey: mkKey(end), Timestamp: hlc.Timestamp{ @@ -1728,7 +1729,7 @@ func TestRangeKeyBatching(t *testing.T) { name string data []storage.MVCCRangeKeyStack batchSize int64 - expect []roachpb.GCRequest_GCRangeKey + expect []kvpb.GCRequest_GCRangeKey }{ { name: "single batch", @@ -1737,7 +1738,7 @@ func TestRangeKeyBatching(t *testing.T) { mkKvs("c", "d", 5, 2, 1), }, batchSize: 99999, - expect: []roachpb.GCRequest_GCRangeKey{ + expect: []kvpb.GCRequest_GCRangeKey{ mkGCr("a", "b", 5), mkGCr("c", "d", 5), }, @@ -1750,7 +1751,7 @@ func TestRangeKeyBatching(t *testing.T) { mkKvs("c", "d", 3, 2), }, batchSize: 99999, - expect: []roachpb.GCRequest_GCRangeKey{ + expect: []kvpb.GCRequest_GCRangeKey{ mkGCr("a", "c", 5), mkGCr("c", "d", 3), }, @@ -1763,7 +1764,7 @@ func TestRangeKeyBatching(t *testing.T) { mkKvs("c", "d", 3, 2), }, batchSize: 40, // We could only fit 2 keys in a batch. - expect: []roachpb.GCRequest_GCRangeKey{ + expect: []kvpb.GCRequest_GCRangeKey{ mkGCr("a", "b", 3), mkGCr("a", "b", 5), mkGCr("b", "c", 2), @@ -1780,7 +1781,7 @@ func TestRangeKeyBatching(t *testing.T) { mkKvs("c", "d", 3, 2), }, batchSize: 50, // We could only fit 3 keys in a batch. - expect: []roachpb.GCRequest_GCRangeKey{ + expect: []kvpb.GCRequest_GCRangeKey{ mkGCr("a", "b", 5), mkGCr("b", "c", 5), mkGCr("c", "d", 3), @@ -1794,7 +1795,7 @@ func TestRangeKeyBatching(t *testing.T) { mkKvs("c", "d", 5, 1), }, batchSize: 85, // We could only fit 5 keys in a batch. - expect: []roachpb.GCRequest_GCRangeKey{ + expect: []kvpb.GCRequest_GCRangeKey{ mkGCr("a", "c", 5), mkGCr("c", "d", 1), mkGCr("c", "d", 5), @@ -1840,9 +1841,9 @@ type capturingGCer struct { func (c *capturingGCer) GC( _ context.Context, - k []roachpb.GCRequest_GCKey, - _ []roachpb.GCRequest_GCRangeKey, - cr *roachpb.GCRequest_GCClearRange, + k []kvpb.GCRequest_GCKey, + _ []kvpb.GCRequest_GCRangeKey, + cr *kvpb.GCRequest_GCClearRange, ) error { if len(k) > 0 { kk := make([]gCR, len(k)) diff --git a/pkg/kv/kvserver/helpers_test.go b/pkg/kv/kvserver/helpers_test.go index 392c68050214..243c6d9d773c 100644 --- a/pkg/kv/kvserver/helpers_test.go +++ b/pkg/kv/kvserver/helpers_test.go @@ -23,6 +23,7 @@ import ( circuit "github.com/cockroachdb/circuitbreaker" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" @@ -418,10 +419,10 @@ func MakeSSTable( } func ProposeAddSSTable(ctx context.Context, key, val string, ts hlc.Timestamp, store *Store) error { - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.RangeID = store.LookupReplica(roachpb.RKey(key)).RangeID - var addReq roachpb.AddSSTableRequest + var addReq kvpb.AddSSTableRequest addReq.Data, _ = MakeSSTable(ctx, key, val, ts) addReq.Key = roachpb.Key(key) addReq.EndKey = addReq.Key.Next() @@ -435,15 +436,15 @@ func ProposeAddSSTable(ctx context.Context, key, val string, ts hlc.Timestamp, s } func SetMockAddSSTable() (undo func()) { - prev, _ := batcheval.LookupCommand(roachpb.AddSSTable) + prev, _ := batcheval.LookupCommand(kvpb.AddSSTable) // TODO(tschottdorf): this already does nontrivial work. Worth open-sourcing the relevant // subparts of the real evalAddSSTable to make this test less likely to rot. evalAddSSTable := func( - ctx context.Context, _ storage.ReadWriter, cArgs batcheval.CommandArgs, _ roachpb.Response, + ctx context.Context, _ storage.ReadWriter, cArgs batcheval.CommandArgs, _ kvpb.Response, ) (result.Result, error) { log.Event(ctx, "evaluated testing-only AddSSTable mock") - args := cArgs.Args.(*roachpb.AddSSTableRequest) + args := cArgs.Args.(*kvpb.AddSSTableRequest) return result.Result{ Replicated: kvserverpb.ReplicatedEvalResult{ @@ -455,11 +456,11 @@ func SetMockAddSSTable() (undo func()) { }, nil } - batcheval.UnregisterCommand(roachpb.AddSSTable) - batcheval.RegisterReadWriteCommand(roachpb.AddSSTable, batcheval.DefaultDeclareKeys, evalAddSSTable) + batcheval.UnregisterCommand(kvpb.AddSSTable) + batcheval.RegisterReadWriteCommand(kvpb.AddSSTable, batcheval.DefaultDeclareKeys, evalAddSSTable) return func() { - batcheval.UnregisterCommand(roachpb.AddSSTable) - batcheval.RegisterReadWriteCommand(roachpb.AddSSTable, prev.DeclareKeys, prev.EvalRW) + batcheval.UnregisterCommand(kvpb.AddSSTable) + batcheval.RegisterReadWriteCommand(kvpb.AddSSTable, prev.DeclareKeys, prev.EvalRW) } } @@ -510,7 +511,7 @@ func WriteRandomDataToRange( ctx := context.Background() src, _ := randutil.NewTestRand() for i := 0; i < 1000; i++ { - var req roachpb.Request + var req kvpb.Request if src.Float64() < 0.05 { // Write some occasional range tombstones. startKey := append(keyPrefix.Clone(), randutil.RandBytes(src, int(src.Int31n(1<<4)))...) @@ -518,8 +519,8 @@ func WriteRandomDataToRange( for startKey.Compare(endKey) >= 0 { endKey = append(keyPrefix.Clone(), randutil.RandBytes(src, int(src.Int31n(1<<4)))...) } - req = &roachpb.DeleteRangeRequest{ - RequestHeader: roachpb.RequestHeader{ + req = &kvpb.DeleteRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: startKey, EndKey: endKey, }, @@ -532,7 +533,7 @@ func WriteRandomDataToRange( pArgs := putArgs(key, val) req = &pArgs } - _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{RangeID: rangeID}, req) + _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{RangeID: rangeID}, req) require.NoError(t, pErr.GoError()) } // Return a random non-empty split key. diff --git a/pkg/kv/kvserver/intent_resolver_integration_test.go b/pkg/kv/kvserver/intent_resolver_integration_test.go index 9a4bd1f3f368..03de0fcf4373 100644 --- a/pkg/kv/kvserver/intent_resolver_integration_test.go +++ b/pkg/kv/kvserver/intent_resolver_integration_test.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" @@ -48,8 +49,8 @@ func beginTransaction( return txn } - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: txn} + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: txn} put := putArgs(key, []byte("value")) ba.Add(&put) assignSeqNumsForReqs(txn, &put) @@ -97,7 +98,7 @@ func TestContendedIntentWithDependencyCycle(t *testing.T) { go func() { put := putArgs(keyA, []byte("value")) assignSeqNumsForReqs(txn1, &put) - if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{Txn: txn1}, &put); pErr != nil { + if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{Txn: txn1}, &put); pErr != nil { txnCh1 <- pErr.GoError() return } @@ -125,7 +126,7 @@ func TestContendedIntentWithDependencyCycle(t *testing.T) { go func() { put := putArgs(keyB, []byte("value")) assignSeqNumsForReqs(txn2, &put) - repl, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{Txn: txn2}, &put) + repl, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{Txn: txn2}, &put) if pErr != nil { txnCh2 <- pErr.GoError() return @@ -159,13 +160,13 @@ func TestContendedIntentWithDependencyCycle(t *testing.T) { go func() { put := putArgs(keyB, []byte("value")) assignSeqNumsForReqs(txn3, &put) - _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{Txn: txn3}, &put) + _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{Txn: txn3}, &put) txnCh3 <- pErr.GoError() }() // The third transaction will always be aborted. err := <-txnCh3 - if !errors.HasType(err, (*roachpb.UnhandledRetryableError)(nil)) { + if !errors.HasType(err, (*kvpb.UnhandledRetryableError)(nil)) { t.Fatalf("expected transaction aborted error; got %T", err) } if err := <-txnCh1; err != nil { @@ -240,24 +241,24 @@ func TestReliableIntentCleanup(t *testing.T) { return readyC } - requestFilter := func(ctx context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + requestFilter := func(ctx context.Context, ba *kvpb.BatchRequest) *kvpb.Error { // If we receive a heartbeat from a txn in abortHeartbeats, // close the aborted channel and return an error response. - if _, ok := ba.GetArg(roachpb.HeartbeatTxn); ok && ba.Txn != nil { + if _, ok := ba.GetArg(kvpb.HeartbeatTxn); ok && ba.Txn != nil { if abortedC, ok := abortHeartbeats.LoadAndDelete(string(ba.Txn.Key)); ok { close(abortedC.(chan struct{})) - return roachpb.NewError(roachpb.NewTransactionAbortedError( - roachpb.ABORT_REASON_NEW_LEASE_PREVENTS_TXN)) + return kvpb.NewError(kvpb.NewTransactionAbortedError( + kvpb.ABORT_REASON_NEW_LEASE_PREVENTS_TXN)) } } return nil } - evalFilter := func(args kvserverbase.FilterArgs) *roachpb.Error { + evalFilter := func(args kvserverbase.FilterArgs) *kvpb.Error { // If we receive a Put request from a txn in blockPutEvals, signal // the caller that the Put is ready to block by passing it an // unblock channel, and wait for it to close. - if put, ok := args.Req.(*roachpb.PutRequest); ok && args.Hdr.Txn != nil { + if put, ok := args.Req.(*kvpb.PutRequest); ok && args.Hdr.Txn != nil { if bytes.HasPrefix(put.Key, prefix) { if ch, ok := blockPutEvals.LoadAndDelete(string(args.Hdr.Txn.Key)); ok { readyC := ch.(chan chan<- struct{}) @@ -271,12 +272,12 @@ func TestReliableIntentCleanup(t *testing.T) { return nil } - responseFilter := func(ctx context.Context, ba *roachpb.BatchRequest, br *roachpb.BatchResponse) *roachpb.Error { + responseFilter := func(ctx context.Context, ba *kvpb.BatchRequest, br *kvpb.BatchResponse) *kvpb.Error { // If we receive a Put request from a txn in blockPuts, signal // the caller that the Put is ready to block by passing it an // unblock channel, and wait for it to close. - if arg, ok := ba.GetArg(roachpb.Put); ok && ba.Txn != nil { - if bytes.HasPrefix(arg.(*roachpb.PutRequest).Key, prefix) { + if arg, ok := ba.GetArg(kvpb.Put); ok && ba.Txn != nil { + if bytes.HasPrefix(arg.(*kvpb.PutRequest).Key, prefix) { if ch, ok := blockPuts.LoadAndDelete(string(ba.Txn.Key)); ok { readyC := ch.(chan chan<- struct{}) unblockC := make(chan struct{}) @@ -402,8 +403,8 @@ func TestReliableIntentCleanup(t *testing.T) { removeKeys := func(t *testing.T) { t.Helper() batch := &kv.Batch{} - batch.AddRawRequest(&roachpb.ClearRangeRequest{ - RequestHeader: roachpb.RequestHeader{ + batch.AddRawRequest(&kvpb.ClearRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: prefix, EndKey: prefix.PrefixEnd(), }, @@ -560,7 +561,7 @@ func TestReliableIntentCleanup(t *testing.T) { for attempt := 1; ; attempt++ { txnKey := genKey(spec.singleRange) txns[txn.ID()] = txnKey // before testTxnExecute, id may change on errors - retryErr := &roachpb.TransactionRetryWithProtoRefreshError{} + retryErr := &kvpb.TransactionRetryWithProtoRefreshError{} err := testTxnExecute(t, ctx, spec, txn, txnKey) if err == nil { break diff --git a/pkg/kv/kvserver/intentresolver/BUILD.bazel b/pkg/kv/kvserver/intentresolver/BUILD.bazel index f70fb403dca2..9acdc246e938 100644 --- a/pkg/kv/kvserver/intentresolver/BUILD.bazel +++ b/pkg/kv/kvserver/intentresolver/BUILD.bazel @@ -14,6 +14,7 @@ go_library( "//pkg/keys", "//pkg/kv", "//pkg/kv/kvclient/rangecache", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/batcheval/result", "//pkg/kv/kvserver/kvserverbase", "//pkg/kv/kvserver/txnwait", @@ -39,6 +40,7 @@ go_test( embed = [":intentresolver"], deps = [ "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/batcheval/result", "//pkg/kv/kvserver/kvserverbase", "//pkg/roachpb", diff --git a/pkg/kv/kvserver/intentresolver/intent_resolver.go b/pkg/kv/kvserver/intentresolver/intent_resolver.go index f7032e486158..940965efb610 100644 --- a/pkg/kv/kvserver/intentresolver/intent_resolver.go +++ b/pkg/kv/kvserver/intentresolver/intent_resolver.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangecache" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/txnwait" @@ -247,7 +248,7 @@ func New(c Config) *IntentResolver { return ir } -func getPusherTxn(h roachpb.Header) roachpb.Transaction { +func getPusherTxn(h kvpb.Header) roachpb.Transaction { // If the txn is nil, we communicate a priority by sending an empty // txn with only the priority set. This is official usage of PushTxn. txn := h.Txn @@ -293,8 +294,8 @@ func updateIntentTxnStatus( // push type and request header. It returns the transaction proto corresponding // to the pushed transaction. func (ir *IntentResolver) PushTransaction( - ctx context.Context, pushTxn *enginepb.TxnMeta, h roachpb.Header, pushType roachpb.PushTxnType, -) (*roachpb.Transaction, *roachpb.Error) { + ctx context.Context, pushTxn *enginepb.TxnMeta, h kvpb.Header, pushType kvpb.PushTxnType, +) (*roachpb.Transaction, *kvpb.Error) { pushTxns := make(map[uuid.UUID]*enginepb.TxnMeta, 1) pushTxns[pushTxn.ID] = pushTxn pushedTxns, pErr := ir.MaybePushTransactions(ctx, pushTxns, h, pushType, false /* skipIfInFlight */) @@ -336,10 +337,10 @@ func (ir *IntentResolver) PushTransaction( func (ir *IntentResolver) MaybePushTransactions( ctx context.Context, pushTxns map[uuid.UUID]*enginepb.TxnMeta, - h roachpb.Header, - pushType roachpb.PushTxnType, + h kvpb.Header, + pushType kvpb.PushTxnType, skipIfInFlight bool, -) (map[uuid.UUID]*roachpb.Transaction, *roachpb.Error) { +) (map[uuid.UUID]*roachpb.Transaction, *kvpb.Error) { // Decide which transactions to push and which to ignore because // of other in-flight requests. For those transactions that we // will be pushing, increment their ref count in the in-flight @@ -382,8 +383,8 @@ func (ir *IntentResolver) MaybePushTransactions( b.Header.Timestamp = ir.clock.Now() b.Header.Timestamp.Forward(pushTo) for _, pushTxn := range pushTxns { - b.AddRawRequest(&roachpb.PushTxnRequest{ - RequestHeader: roachpb.RequestHeader{ + b.AddRawRequest(&kvpb.PushTxnRequest{ + RequestHeader: kvpb.RequestHeader{ Key: pushTxn.Key, }, PusherTxn: pusherTxn, @@ -405,7 +406,7 @@ func (ir *IntentResolver) MaybePushTransactions( br := b.RawResponse() pushedTxns := make(map[uuid.UUID]*roachpb.Transaction, len(br.Responses)) for _, resp := range br.Responses { - txn := &resp.GetInner().(*roachpb.PushTxnResponse).PusheeTxn + txn := &resp.GetInner().(*kvpb.PushTxnResponse).PusheeTxn if _, ok := pushedTxns[txn.ID]; ok { log.Fatalf(ctx, "have two PushTxn responses for %s", txn.ID) } @@ -468,7 +469,7 @@ func (ir *IntentResolver) CleanupIntentsAsync( return ir.runAsyncTask(ctx, allowSyncProcessing, func(ctx context.Context) { err := contextutil.RunWithTimeout(ctx, "async intent resolution", asyncIntentResolutionTimeout, func(ctx context.Context) error { - _, err := ir.CleanupIntents(ctx, intents, now, roachpb.PUSH_TOUCH) + _, err := ir.CleanupIntents(ctx, intents, now, kvpb.PUSH_TOUCH) return err }) if err != nil && ir.every.ShouldLog() { @@ -484,9 +485,9 @@ func (ir *IntentResolver) CleanupIntentsAsync( // subset of the intents may have been resolved, but zero will be // returned. func (ir *IntentResolver) CleanupIntents( - ctx context.Context, intents []roachpb.Intent, now hlc.Timestamp, pushType roachpb.PushTxnType, + ctx context.Context, intents []roachpb.Intent, now hlc.Timestamp, pushType kvpb.PushTxnType, ) (int, error) { - h := roachpb.Header{Timestamp: now} + h := kvpb.Header{Timestamp: now} // All transactions in MaybePushTransactions will be sent in a single batch. // In order to ensure that progress is made, we want to ensure that this @@ -667,13 +668,13 @@ func (ir *IntentResolver) CleanupTxnIntentsOnGCAsync( } b := &kv.Batch{} b.Header.Timestamp = now - b.AddRawRequest(&roachpb.PushTxnRequest{ - RequestHeader: roachpb.RequestHeader{Key: txn.Key}, + b.AddRawRequest(&kvpb.PushTxnRequest{ + RequestHeader: kvpb.RequestHeader{Key: txn.Key}, PusherTxn: roachpb.Transaction{ TxnMeta: enginepb.TxnMeta{Priority: enginepb.MaxTxnPriority}, }, PusheeTxn: txn.TxnMeta, - PushType: roachpb.PUSH_ABORT, + PushType: kvpb.PUSH_ABORT, }) pushed = true if err := ir.db.Run(ctx, b); err != nil { @@ -682,7 +683,7 @@ func (ir *IntentResolver) CleanupTxnIntentsOnGCAsync( } // Update the txn with the result of the push, such that the intents we're about // to resolve get a final status. - finalizedTxn := &b.RawResponse().Responses[0].GetInner().(*roachpb.PushTxnResponse).PusheeTxn + finalizedTxn := &b.RawResponse().Responses[0].GetInner().(*kvpb.PushTxnResponse).PusheeTxn txn = txn.Clone() txn.Update(finalizedTxn) } @@ -725,7 +726,7 @@ func (ir *IntentResolver) gcTxnRecord( // // #7880 will address this by making GCRequest less special and // thus obviating the need to cook up an artificial range here. - var gcArgs roachpb.GCRequest + var gcArgs kvpb.GCRequest { key := keys.MustAddr(txn.Key) if localMax := keys.MustAddr(keys.LocalMax); key.Less(localMax) { @@ -733,12 +734,12 @@ func (ir *IntentResolver) gcTxnRecord( } endKey := key.Next() - gcArgs.RequestHeader = roachpb.RequestHeader{ + gcArgs.RequestHeader = kvpb.RequestHeader{ Key: key.AsRawKey(), EndKey: endKey.AsRawKey(), } } - gcArgs.Keys = append(gcArgs.Keys, roachpb.GCRequest_GCKey{ + gcArgs.Keys = append(gcArgs.Keys, kvpb.GCRequest_GCKey{ Key: txnKey, }) // Although the IntentResolver has a RangeDescriptorCache it could consult to @@ -877,14 +878,14 @@ func (s *sliceLockUpdates) Index(i int) roachpb.LockUpdate { // ResolveIntent synchronously resolves an intent according to opts. func (ir *IntentResolver) ResolveIntent( ctx context.Context, intent roachpb.LockUpdate, opts ResolveOptions, -) *roachpb.Error { +) *kvpb.Error { return ir.ResolveIntents(ctx, []roachpb.LockUpdate{intent}, opts) } // ResolveIntents synchronously resolves intents according to opts. func (ir *IntentResolver) ResolveIntents( ctx context.Context, intents []roachpb.LockUpdate, opts ResolveOptions, -) (pErr *roachpb.Error) { +) (pErr *kvpb.Error) { return ir.resolveIntents(ctx, (*sliceLockUpdates)(&intents), opts) } @@ -894,7 +895,7 @@ func (ir *IntentResolver) ResolveIntents( // LockUpdates as they are accessed in this method. func (ir *IntentResolver) resolveIntents( ctx context.Context, intents lockUpdates, opts ResolveOptions, -) (pErr *roachpb.Error) { +) (pErr *kvpb.Error) { if intents.Len() == 0 { return nil } @@ -906,7 +907,7 @@ func (ir *IntentResolver) resolveIntents( // Avoid doing any work on behalf of expired contexts. See // https://github.com/cockroachdb/cockroach/issues/15997. if err := ctx.Err(); err != nil { - return roachpb.NewError(err) + return kvpb.NewError(err) } log.Eventf(ctx, "resolving intents") ctx, cancel := context.WithCancel(ctx) @@ -916,11 +917,11 @@ func (ir *IntentResolver) resolveIntents( for i := 0; i < intents.Len(); i++ { intent := intents.Index(i) rangeID := ir.lookupRangeID(ctx, intent.Key) - var req roachpb.Request + var req kvpb.Request var batcher *requestbatcher.RequestBatcher if len(intent.EndKey) == 0 { - req = &roachpb.ResolveIntentRequest{ - RequestHeader: roachpb.RequestHeaderFromSpan(intent.Span), + req = &kvpb.ResolveIntentRequest{ + RequestHeader: kvpb.RequestHeaderFromSpan(intent.Span), IntentTxn: intent.Txn, Status: intent.Status, Poison: opts.Poison, @@ -929,8 +930,8 @@ func (ir *IntentResolver) resolveIntents( } batcher = ir.irBatcher } else { - req = &roachpb.ResolveIntentRangeRequest{ - RequestHeader: roachpb.RequestHeaderFromSpan(intent.Span), + req = &kvpb.ResolveIntentRangeRequest{ + RequestHeader: kvpb.RequestHeaderFromSpan(intent.Span), IntentTxn: intent.Txn, Status: intent.Status, Poison: opts.Poison, @@ -941,20 +942,20 @@ func (ir *IntentResolver) resolveIntents( batcher = ir.irRangeBatcher } if err := batcher.SendWithChan(ctx, respChan, rangeID, req); err != nil { - return roachpb.NewError(err) + return kvpb.NewError(err) } } for seen := 0; seen < intents.Len(); seen++ { select { case resp := <-respChan: if resp.Err != nil { - return roachpb.NewError(resp.Err) + return kvpb.NewError(resp.Err) } _ = resp.Resp // ignore the response case <-ctx.Done(): - return roachpb.NewError(ctx.Err()) + return kvpb.NewError(ctx.Err()) case <-ir.stopper.ShouldQuiesce(): - return roachpb.NewErrorf("stopping") + return kvpb.NewErrorf("stopping") } } return nil diff --git a/pkg/kv/kvserver/intentresolver/intent_resolver_test.go b/pkg/kv/kvserver/intentresolver/intent_resolver_test.go index f398af826fa9..65a1da575e61 100644 --- a/pkg/kv/kvserver/intentresolver/intent_resolver_test.go +++ b/pkg/kv/kvserver/intentresolver/intent_resolver_test.go @@ -21,6 +21,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -365,16 +366,16 @@ func TestCleanupMultipleIntentsAsync(t *testing.T) { pushed []string resolved []string } - pushOrResolveFunc := func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + pushOrResolveFunc := func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { switch ba.Requests[0].GetInner().Method() { - case roachpb.PushTxn: + case kvpb.PushTxn: for _, ru := range ba.Requests { reqs.Lock() reqs.pushed = append(reqs.pushed, string(ru.GetPushTxn().Key)) reqs.Unlock() } return pushTxnSendFunc(t, len(ba.Requests))(ba) - case roachpb.ResolveIntent: + case kvpb.ResolveIntent: for _, ru := range ba.Requests { reqs.Lock() reqs.resolved = append(reqs.resolved, string(ru.GetResolveIntent().Key)) @@ -382,7 +383,7 @@ func TestCleanupMultipleIntentsAsync(t *testing.T) { } return resolveIntentsSendFunc(t)(ba) default: - return nil, roachpb.NewErrorf("unexpected") + return nil, kvpb.NewErrorf("unexpected") } } sf := newSendFuncs(t, repeat(pushOrResolveFunc, 5)...) @@ -469,14 +470,14 @@ func TestCleanupTxnIntentsAsyncWithPartialRollback(t *testing.T) { txn.IgnoredSeqNums = []enginepb.IgnoredSeqNumRange{{Start: 1, End: 1}} var gotResolveIntent, gotResolveIntentRange int32 - check := func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + check := func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { for _, r := range ba.Requests { - if ri, ok := r.GetInner().(*roachpb.ResolveIntentRequest); ok { + if ri, ok := r.GetInner().(*kvpb.ResolveIntentRequest); ok { atomic.StoreInt32(&gotResolveIntent, 1) if !reflect.DeepEqual(ri.IgnoredSeqNums, txn.IgnoredSeqNums) { t.Errorf("expected ignored list %v, got %v", txn.IgnoredSeqNums, ri.IgnoredSeqNums) } - } else if rir, ok := r.GetInner().(*roachpb.ResolveIntentRangeRequest); ok { + } else if rir, ok := r.GetInner().(*kvpb.ResolveIntentRangeRequest); ok { atomic.StoreInt32(&gotResolveIntentRange, 1) if !reflect.DeepEqual(rir.IgnoredSeqNums, txn.IgnoredSeqNums) { t.Errorf("expected ignored list %v, got %v", txn.IgnoredSeqNums, rir.IgnoredSeqNums) @@ -627,31 +628,31 @@ func TestCleanupMultipleTxnIntentsAsync(t *testing.T) { resolved []string gced []string } - resolveOrGCFunc := func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + resolveOrGCFunc := func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { if len(ba.Requests) != 1 { - return nil, roachpb.NewErrorf("unexpected") + return nil, kvpb.NewErrorf("unexpected") } ru := ba.Requests[0] switch ru.GetInner().Method() { - case roachpb.ResolveIntent: + case kvpb.ResolveIntent: reqs.Lock() reqs.resolved = append(reqs.resolved, string(ru.GetResolveIntent().Key)) reqs.Unlock() return resolveIntentsSendFunc(t)(ba) - case roachpb.ResolveIntentRange: + case kvpb.ResolveIntentRange: reqs.Lock() req := ru.GetResolveIntentRange() reqs.resolved = append(reqs.resolved, fmt.Sprintf("%s-%s", string(req.Key), string(req.EndKey))) reqs.Unlock() return resolveIntentsSendFunc(t)(ba) - case roachpb.GC: + case kvpb.GC: reqs.Lock() reqs.gced = append(reqs.gced, string(ru.GetGc().Key)) reqs.Unlock() return gcSendFunc(t)(ba) default: - return nil, roachpb.NewErrorf("unexpected") + return nil, kvpb.NewErrorf("unexpected") } } sf := newSendFuncs(t, repeat(resolveOrGCFunc, 8)...) @@ -748,7 +749,7 @@ func TestCleanupIntents(t *testing.T) { c.cfg.Stopper = stopper c.cfg.Clock = clock ir := newIntentResolverWithSendFuncs(c.cfg, c.sendFuncs, stopper) - num, err := ir.CleanupIntents(context.Background(), c.intents, clock.Now(), roachpb.PUSH_ABORT) + num, err := ir.CleanupIntents(context.Background(), c.intents, clock.Now(), kvpb.PUSH_ABORT) assert.Equal(t, num, c.expectedNum, "number of resolved intents") assert.Equal(t, err != nil, c.expectedErr, "error during CleanupIntents: %v", err) }) @@ -783,13 +784,13 @@ func makeTxnIntents(t *testing.T, clock *hlc.Clock, numIntents int) []roachpb.In // the IntentResolver tries to send. They are used in conjunction with the below // function to create an IntentResolver with a slice of sendFuncs. // A library of useful sendFuncs are defined below. -type sendFunc func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) +type sendFunc func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) func newIntentResolverWithSendFuncs( c Config, sf *sendFuncs, stopper *stop.Stopper, ) *IntentResolver { txnSenderFactory := kv.NonTransactionalFactoryFunc( - func(_ context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + func(_ context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { sf.mu.Lock() defer sf.mu.Unlock() f := sf.popLocked() @@ -805,7 +806,7 @@ func newIntentResolverWithSendFuncs( func pushTxnSendFuncs(sf *sendFuncs, N int) sendFunc { toPush := int64(N) var f sendFunc - f = func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + f = func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { if remaining := atomic.LoadInt64(&toPush); len(ba.Requests) > int(remaining) { sf.t.Errorf("expected at most %d PushTxnRequests in batch, got %d", remaining, len(ba.Requests)) @@ -820,7 +821,7 @@ func pushTxnSendFuncs(sf *sendFuncs, N int) sendFunc { } func pushTxnSendFunc(t *testing.T, numPushes int) sendFunc { - return func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + return func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { if len(ba.Requests) != numPushes { t.Errorf("expected %d PushTxnRequests in batch, got %d", numPushes, len(ba.Requests)) @@ -856,7 +857,7 @@ func resolveIntentsSendFuncsEx( toResolve := int64(numIntents) reqsSeen := int64(0) var f sendFunc - f = func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + f = func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { if remaining := atomic.LoadInt64(&toResolve); len(ba.Requests) > int(remaining) { sf.t.Errorf("expected at most %d ResolveIntentRequests in batch, got %d", remaining, len(ba.Requests)) @@ -875,7 +876,7 @@ func resolveIntentsSendFuncsEx( } func resolveIntentsSendFuncEx(t *testing.T, checkTxnStatusOpt checkTxnStatusOpt) sendFunc { - return func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + return func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { return respForResolveIntentBatch(t, ba, checkTxnStatusOpt), nil } } @@ -892,33 +893,33 @@ func resolveIntentsSendFunc(t *testing.T) sendFunc { return resolveIntentsSendFuncEx(t, dontCheckTxnStatus) } -func failSendFunc(*roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { - return nil, roachpb.NewError(fmt.Errorf("boom")) +func failSendFunc(*kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { + return nil, kvpb.NewError(fmt.Errorf("boom")) } func gcSendFunc(t *testing.T) sendFunc { - return func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { - resp := &roachpb.BatchResponse{} + return func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { + resp := &kvpb.BatchResponse{} for _, r := range ba.Requests { - if _, ok := r.GetInner().(*roachpb.GCRequest); !ok { + if _, ok := r.GetInner().(*kvpb.GCRequest); !ok { t.Errorf("Unexpected request type %T, expected GCRequest", r.GetInner()) } - resp.Add(&roachpb.GCResponse{}) + resp.Add(&kvpb.GCResponse{}) } return resp, nil } } -func respForPushTxnBatch(t *testing.T, ba *roachpb.BatchRequest) *roachpb.BatchResponse { - resp := &roachpb.BatchResponse{} +func respForPushTxnBatch(t *testing.T, ba *kvpb.BatchRequest) *kvpb.BatchResponse { + resp := &kvpb.BatchResponse{} for _, r := range ba.Requests { var txn enginepb.TxnMeta - if req, ok := r.GetInner().(*roachpb.PushTxnRequest); ok { + if req, ok := r.GetInner().(*kvpb.PushTxnRequest); ok { txn = req.PusheeTxn } else { t.Errorf("Unexpected request type %T, expected PushTxnRequest", r.GetInner()) } - resp.Add(&roachpb.PushTxnResponse{ + resp.Add(&kvpb.PushTxnResponse{ PusheeTxn: roachpb.Transaction{ Status: roachpb.ABORTED, TxnMeta: txn, @@ -929,17 +930,17 @@ func respForPushTxnBatch(t *testing.T, ba *roachpb.BatchRequest) *roachpb.BatchR } func respForResolveIntentBatch( - t *testing.T, ba *roachpb.BatchRequest, checkTxnStatusOpt checkTxnStatusOpt, -) *roachpb.BatchResponse { - resp := &roachpb.BatchResponse{} + t *testing.T, ba *kvpb.BatchRequest, checkTxnStatusOpt checkTxnStatusOpt, +) *kvpb.BatchResponse { + resp := &kvpb.BatchResponse{} var status roachpb.TransactionStatus for _, r := range ba.Requests { - if rir, ok := r.GetInner().(*roachpb.ResolveIntentRequest); ok { + if rir, ok := r.GetInner().(*kvpb.ResolveIntentRequest); ok { status = rir.AsLockUpdate().Status - resp.Add(&roachpb.ResolveIntentResponse{}) - } else if rirr, ok := r.GetInner().(*roachpb.ResolveIntentRangeRequest); ok { + resp.Add(&kvpb.ResolveIntentResponse{}) + } else if rirr, ok := r.GetInner().(*kvpb.ResolveIntentRangeRequest); ok { status = rirr.AsLockUpdate().Status - resp.Add(&roachpb.ResolveIntentRangeResponse{}) + resp.Add(&kvpb.ResolveIntentRangeResponse{}) } else { t.Errorf("Unexpected request in batch for intent resolution: %T", r.GetInner()) } diff --git a/pkg/kv/kvserver/kvadmission/BUILD.bazel b/pkg/kv/kvserver/kvadmission/BUILD.bazel index 3c8953754572..189dfd08f6cf 100644 --- a/pkg/kv/kvserver/kvadmission/BUILD.bazel +++ b/pkg/kv/kvserver/kvadmission/BUILD.bazel @@ -7,6 +7,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvadmission", visibility = ["//visibility:public"], deps = [ + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/settings", "//pkg/settings/cluster", diff --git a/pkg/kv/kvserver/kvadmission/kvadmission.go b/pkg/kv/kvserver/kvadmission/kvadmission.go index 92fb01f8a832..d67e6b7d509a 100644 --- a/pkg/kv/kvserver/kvadmission/kvadmission.go +++ b/pkg/kv/kvserver/kvadmission/kvadmission.go @@ -18,6 +18,7 @@ import ( "sync" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -97,7 +98,7 @@ type Controller interface { // populated for admission to work correctly. If err is non-nil, the // returned handle can be ignored. If err is nil, AdmittedKVWorkDone must be // called after the KV work is done executing. - AdmitKVWork(context.Context, roachpb.TenantID, *roachpb.BatchRequest) (Handle, error) + AdmitKVWork(context.Context, roachpb.TenantID, *kvpb.BatchRequest) (Handle, error) // AdmittedKVWorkDone is called after the admitted KV work is done // executing. AdmittedKVWorkDone(Handle, *StoreWriteBytes) @@ -105,7 +106,7 @@ type Controller interface { // If enabled, it returns a non-nil Pacer that's to be used within rangefeed // catchup scans (typically CPU-intensive and affecting scheduling // latencies). - AdmitRangefeedRequest(roachpb.TenantID, *roachpb.RangeFeedRequest) *admission.Pacer + AdmitRangefeedRequest(roachpb.TenantID, *kvpb.RangeFeedRequest) *admission.Pacer // SetTenantWeightProvider is used to set the provider that will be // periodically polled for weights. The stopper should be used to terminate // the periodic polling. @@ -192,7 +193,7 @@ func MakeController( // TODO(irfansharif): There's a fair bit happening here and there's no test // coverage. Fix that. func (n *controllerImpl) AdmitKVWork( - ctx context.Context, tenantID roachpb.TenantID, ba *roachpb.BatchRequest, + ctx context.Context, tenantID roachpb.TenantID, ba *kvpb.BatchRequest, ) (handle Handle, retErr error) { ah := Handle{tenantID: tenantID} if n.kvAdmissionQ == nil { @@ -204,9 +205,9 @@ func (n *controllerImpl) AdmitKVWork( if !roachpb.IsSystemTenantID(tenantID.ToUint64()) { // Request is from a SQL node. bypassAdmission = false - source = roachpb.AdmissionHeader_FROM_SQL + source = kvpb.AdmissionHeader_FROM_SQL } - if source == roachpb.AdmissionHeader_OTHER { + if source == kvpb.AdmissionHeader_OTHER { bypassAdmission = true } // TODO(abaptist): Revisit and deprecate this setting in v23.1. @@ -319,7 +320,7 @@ func (n *controllerImpl) AdmittedKVWorkDone(ah Handle, writeBytes *StoreWriteByt // AdmitRangefeedRequest implements the Controller interface. func (n *controllerImpl) AdmitRangefeedRequest( - tenantID roachpb.TenantID, request *roachpb.RangeFeedRequest, + tenantID roachpb.TenantID, request *kvpb.RangeFeedRequest, ) *admission.Pacer { if !rangefeedCatchupScanElasticControlEnabled.Get(&n.settings.SV) { return nil diff --git a/pkg/kv/kvserver/kvserverbase/BUILD.bazel b/pkg/kv/kvserver/kvserverbase/BUILD.bazel index 81b1e123fc90..f4a7f74ddcc7 100644 --- a/pkg/kv/kvserver/kvserverbase/BUILD.bazel +++ b/pkg/kv/kvserver/kvserverbase/BUILD.bazel @@ -16,6 +16,7 @@ go_library( deps = [ "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/kvserverpb", "//pkg/roachpb", "//pkg/settings", diff --git a/pkg/kv/kvserver/kvserverbase/base.go b/pkg/kv/kvserver/kvserverbase/base.go index 2db8d3a843ed..8dfd824b0ccd 100644 --- a/pkg/kv/kvserver/kvserverbase/base.go +++ b/pkg/kv/kvserver/kvserverbase/base.go @@ -17,6 +17,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" @@ -55,8 +56,8 @@ type FilterArgs struct { CmdID CmdIDKey Index int Sid roachpb.StoreID - Req roachpb.Request - Hdr roachpb.Header + Req kvpb.Request + Hdr kvpb.Header Version roachpb.Version Err error // only used for TestingPostEvalFilter } @@ -67,7 +68,7 @@ type ProposalFilterArgs struct { Cmd *kvserverpb.RaftCommand QuotaAlloc *quotapool.IntAlloc CmdID CmdIDKey - Req roachpb.BatchRequest + Req kvpb.BatchRequest } // ApplyFilterArgs groups the arguments to a ReplicaApplyFilter. @@ -76,8 +77,8 @@ type ApplyFilterArgs struct { CmdID CmdIDKey RangeID roachpb.RangeID StoreID roachpb.StoreID - Req *roachpb.BatchRequest // only set on the leaseholder - ForcedError *roachpb.Error + Req *kvpb.BatchRequest // only set on the leaseholder + ForcedError *kvpb.Error } // InRaftCmd returns true if the filter is running in the context of a Raft @@ -89,36 +90,36 @@ func (f *FilterArgs) InRaftCmd() bool { // ReplicaRequestFilter can be used in testing to influence the error returned // from a request before it is evaluated. Return nil to continue with regular // processing or non-nil to terminate processing with the returned error. -type ReplicaRequestFilter func(context.Context, *roachpb.BatchRequest) *roachpb.Error +type ReplicaRequestFilter func(context.Context, *kvpb.BatchRequest) *kvpb.Error // ReplicaConcurrencyRetryFilter can be used to examine a concurrency retry // error before it is handled and its batch is re-evaluated. -type ReplicaConcurrencyRetryFilter func(context.Context, *roachpb.BatchRequest, *roachpb.Error) +type ReplicaConcurrencyRetryFilter func(context.Context, *kvpb.BatchRequest, *kvpb.Error) // ReplicaCommandFilter may be used in tests through the StoreTestingKnobs to // intercept the handling of commands and artificially generate errors. Return // nil to continue with regular processing or non-nil to terminate processing // with the returned error. -type ReplicaCommandFilter func(args FilterArgs) *roachpb.Error +type ReplicaCommandFilter func(args FilterArgs) *kvpb.Error // ReplicaProposalFilter can be used in testing to influence the error returned // from proposals after a request is evaluated but before it is proposed. -type ReplicaProposalFilter func(args ProposalFilterArgs) *roachpb.Error +type ReplicaProposalFilter func(args ProposalFilterArgs) *kvpb.Error // A ReplicaApplyFilter is a testing hook into raft command application. // See StoreTestingKnobs. -type ReplicaApplyFilter func(args ApplyFilterArgs) (int, *roachpb.Error) +type ReplicaApplyFilter func(args ApplyFilterArgs) (int, *kvpb.Error) // ReplicaResponseFilter is used in unittests to modify the outbound // response returned to a waiting client after a replica command has // been processed. This filter is invoked only by the command proposer. -type ReplicaResponseFilter func(context.Context, *roachpb.BatchRequest, *roachpb.BatchResponse) *roachpb.Error +type ReplicaResponseFilter func(context.Context, *kvpb.BatchRequest, *kvpb.BatchResponse) *kvpb.Error // ReplicaRangefeedFilter is used in unit tests to modify the request, inject // responses, or return errors from rangefeeds. type ReplicaRangefeedFilter func( - args *roachpb.RangeFeedRequest, stream roachpb.RangeFeedEventSink, -) *roachpb.Error + args *kvpb.RangeFeedRequest, stream kvpb.RangeFeedEventSink, +) *kvpb.Error // ContainsKey returns whether this range contains the specified key. func ContainsKey(desc *roachpb.RangeDescriptor, key roachpb.Key) bool { diff --git a/pkg/kv/kvserver/kvserverbase/bulk_adder.go b/pkg/kv/kvserver/kvserverbase/bulk_adder.go index 6378666f10bf..62323091a093 100644 --- a/pkg/kv/kvserver/kvserverbase/bulk_adder.go +++ b/pkg/kv/kvserver/kvserverbase/bulk_adder.go @@ -15,6 +15,7 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" ) @@ -43,7 +44,7 @@ type BulkAdderOptions struct { // DisallowShadowingBelow controls whether shadowing of existing keys is // permitted when the SSTables produced by this adder are ingested. See the - // comment on roachpb.AddSSTableRequest for more details. Note that if this is + // comment on kvpb.AddSSTableRequest for more details. Note that if this is // set to a non-empty timestamp, the older flag DisallowShadowing will be set // on all requests as well, so callers should expect older nodes to handle any // requests accordingly or check the MVCCAddSSTable version gate. @@ -57,7 +58,7 @@ type BulkAdderOptions struct { // WriteAtBatchTimestamp will rewrite the SST to use the batch timestamp, even // if it gets pushed to a different timestamp on the server side. All SST MVCC // timestamps must equal BatchTimestamp. See - // roachpb.AddSSTableRequest.SSTTimestampToRequestTimestamp. + // kvpb.AddSSTableRequest.SSTTimestampToRequestTimestamp. WriteAtBatchTimestamp bool // InitialSplitsIfUnordered specifies a number of splits to make before the @@ -86,11 +87,11 @@ type BulkAdder interface { // CurrentBufferFill returns how full the configured buffer is. CurrentBufferFill() float32 // GetSummary returns a summary of rows/bytes/etc written by this batcher. - GetSummary() roachpb.BulkOpSummary + GetSummary() kvpb.BulkOpSummary // Close closes the underlying buffers/writers. Close(ctx context.Context) // SetOnFlush sets a callback function called after flushing the buffer. - SetOnFlush(func(summary roachpb.BulkOpSummary)) + SetOnFlush(func(summary kvpb.BulkOpSummary)) } // DuplicateKeyError represents a failed attempt to ingest the same key twice diff --git a/pkg/kv/kvserver/kvserverbase/forced_error.go b/pkg/kv/kvserver/kvserverbase/forced_error.go index 7b6ae0e8b970..4a4adbbccb0d 100644 --- a/pkg/kv/kvserver/kvserverbase/forced_error.go +++ b/pkg/kv/kvserver/kvserverbase/forced_error.go @@ -15,6 +15,7 @@ import ( "context" "fmt" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -36,17 +37,17 @@ const ( // noopOnEmptyRaftCommandErr is returned from CheckForcedErr when an empty raft // command is received. See the comment near its use. -var noopOnEmptyRaftCommandErr = roachpb.NewErrorf("no-op on empty Raft entry") +var noopOnEmptyRaftCommandErr = kvpb.NewErrorf("no-op on empty Raft entry") // NoopOnProbeCommandErr is returned from CheckForcedErr when a raft command // corresponding to a ProbeRequest is handled. -var NoopOnProbeCommandErr = roachpb.NewErrorf("no-op on ProbeRequest") +var NoopOnProbeCommandErr = kvpb.NewErrorf("no-op on ProbeRequest") // ForcedErrResult is the output from CheckForcedErr. type ForcedErrResult struct { LeaseIndex uint64 Rejection ProposalRejectionType - ForcedError *roachpb.Error + ForcedError *kvpb.Error } // CheckForcedErr determines whether or not a command should be applied to the @@ -179,7 +180,7 @@ func CheckForcedErr( return ForcedErrResult{ LeaseIndex: leaseIndex, Rejection: ProposalRejectionPermanent, - ForcedError: roachpb.NewError(&roachpb.LeaseRejectedError{ + ForcedError: kvpb.NewError(&kvpb.LeaseRejectedError{ Existing: *replicaState.Lease, Requested: requestedLease, Message: "proposed under invalid lease", @@ -189,7 +190,7 @@ func CheckForcedErr( // We return a NotLeaseHolderError so that the DistSender retries. // NB: we set proposerStoreID to 0 because we don't know who proposed the // Raft command. This is ok, as this is only used for debug information. - nlhe := roachpb.NewNotLeaseHolderError( + nlhe := kvpb.NewNotLeaseHolderError( *replicaState.Lease, 0 /* proposerStoreID */, replicaState.Desc, fmt.Sprintf( "stale proposal: command was proposed under lease #%d but is being applied "+ @@ -197,7 +198,7 @@ func CheckForcedErr( return ForcedErrResult{ LeaseIndex: leaseIndex, Rejection: ProposalRejectionPermanent, - ForcedError: roachpb.NewError(nlhe), + ForcedError: kvpb.NewError(nlhe), } } @@ -213,7 +214,7 @@ func CheckForcedErr( return ForcedErrResult{ LeaseIndex: leaseIndex, Rejection: ProposalRejectionPermanent, - ForcedError: roachpb.NewError(&roachpb.LeaseRejectedError{ + ForcedError: kvpb.NewError(&kvpb.LeaseRejectedError{ Existing: *replicaState.Lease, Requested: requestedLease, Message: "replica not part of range", @@ -248,7 +249,7 @@ func CheckForcedErr( return ForcedErrResult{ LeaseIndex: leaseIndex, Rejection: retry, - ForcedError: roachpb.NewErrorf( + ForcedError: kvpb.NewErrorf( "command observed at lease index %d, but required < %d", leaseIndex, raftCmd.MaxLeaseIndex, )} } @@ -267,7 +268,7 @@ func CheckForcedErr( return ForcedErrResult{ LeaseIndex: leaseIndex, Rejection: ProposalRejectionPermanent, - ForcedError: roachpb.NewError(&roachpb.BatchTimestampBeforeGCError{ + ForcedError: kvpb.NewError(&kvpb.BatchTimestampBeforeGCError{ Timestamp: wts, Threshold: *replicaState.GCThreshold, }), diff --git a/pkg/kv/kvserver/kvserverpb/BUILD.bazel b/pkg/kv/kvserver/kvserverpb/BUILD.bazel index f2bf110a9e7c..0a1ca39a65ba 100644 --- a/pkg/kv/kvserver/kvserverpb/BUILD.bazel +++ b/pkg/kv/kvserver/kvserverpb/BUILD.bazel @@ -34,6 +34,7 @@ proto_library( strip_import_prefix = "/pkg", visibility = ["//visibility:public"], deps = [ + "//pkg/kv/kvpb:kvpb_proto", "//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb:kvflowcontrolpb_proto", "//pkg/kv/kvserver/liveness/livenesspb:livenesspb_proto", "//pkg/kv/kvserver/readsummary/rspb:rspb_proto", @@ -55,6 +56,7 @@ go_proto_library( proto = ":kvserverpb_proto", visibility = ["//visibility:public"], deps = [ + "//pkg/kv/kvpb", "//pkg/kv/kvserver/closedts/ctpb", # keep "//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb", "//pkg/kv/kvserver/liveness/livenesspb", diff --git a/pkg/kv/kvserver/kvserverpb/proposer_kv.proto b/pkg/kv/kvserver/kvserverpb/proposer_kv.proto index 89acafa75709..c276d54abd68 100644 --- a/pkg/kv/kvserver/kvserverpb/proposer_kv.proto +++ b/pkg/kv/kvserver/kvserverpb/proposer_kv.proto @@ -12,7 +12,7 @@ syntax = "proto3"; package cockroach.kv.kvserver.storagepb; option go_package = "kvserverpb"; -import "roachpb/api.proto"; +import "kv/kvpb/api.proto"; import "roachpb/data.proto"; import "roachpb/metadata.proto"; import "storage/enginepb/mvcc.proto"; diff --git a/pkg/kv/kvserver/kvserverpb/raft.proto b/pkg/kv/kvserver/kvserverpb/raft.proto index 878f0290a133..4144c65d5171 100644 --- a/pkg/kv/kvserver/kvserverpb/raft.proto +++ b/pkg/kv/kvserver/kvserverpb/raft.proto @@ -13,7 +13,7 @@ package cockroach.kv.kvserver.kvserverpb; option go_package = "kvserverpb"; import "errorspb/errors.proto"; -import "roachpb/errors.proto"; +import "kv/kvpb/errors.proto"; import "roachpb/metadata.proto"; import "kv/kvserver/liveness/livenesspb/liveness.proto"; import "kv/kvserver/kvserverpb/state.proto"; diff --git a/pkg/kv/kvserver/kvstorage/BUILD.bazel b/pkg/kv/kvserver/kvstorage/BUILD.bazel index 4e971abe6ddf..31e6d409cad7 100644 --- a/pkg/kv/kvserver/kvstorage/BUILD.bazel +++ b/pkg/kv/kvserver/kvstorage/BUILD.bazel @@ -15,6 +15,7 @@ go_library( deps = [ "//pkg/clusterversion", "//pkg/keys", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/kvserverpb", "//pkg/kv/kvserver/logstore", "//pkg/kv/kvserver/rditer", diff --git a/pkg/kv/kvserver/kvstorage/replica_state.go b/pkg/kv/kvserver/kvstorage/replica_state.go index 975f64006445..ee397a5dcb5e 100644 --- a/pkg/kv/kvserver/kvstorage/replica_state.go +++ b/pkg/kv/kvserver/kvstorage/replica_state.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -103,7 +104,7 @@ func (r LoadedReplicaState) check(storeID roachpb.StoreID) error { } // CreateUninitializedReplica creates an uninitialized replica in storage. -// Returns roachpb.RaftGroupDeletedError if this replica can not be created +// Returns kvpb.RaftGroupDeletedError if this replica can not be created // because it has been deleted. func CreateUninitializedReplica( ctx context.Context, @@ -121,7 +122,7 @@ func CreateUninitializedReplica( ); err != nil { return err } else if ok && replicaID < tombstone.NextReplicaID { - return &roachpb.RaftGroupDeletedError{} + return &kvpb.RaftGroupDeletedError{} } // Write the RaftReplicaID for this replica. This is the only place in the diff --git a/pkg/kv/kvserver/liveness/BUILD.bazel b/pkg/kv/kvserver/liveness/BUILD.bazel index 61054b7b5fd7..9f438a44ef72 100644 --- a/pkg/kv/kvserver/liveness/BUILD.bazel +++ b/pkg/kv/kvserver/liveness/BUILD.bazel @@ -11,6 +11,7 @@ go_library( "//pkg/gossip", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/liveness/livenesspb", "//pkg/roachpb", "//pkg/server/telemetry", diff --git a/pkg/kv/kvserver/liveness/liveness.go b/pkg/kv/kvserver/liveness/liveness.go index a11f3abc4114..150748ecc2d0 100644 --- a/pkg/kv/kvserver/liveness/liveness.go +++ b/pkg/kv/kvserver/liveness/liveness.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/telemetry" @@ -80,13 +81,13 @@ func (e *errRetryLiveness) Error() string { } func isErrRetryLiveness(ctx context.Context, err error) bool { - if errors.HasType(err, (*roachpb.AmbiguousResultError)(nil)) { + if errors.HasType(err, (*kvpb.AmbiguousResultError)(nil)) { // We generally want to retry ambiguous errors immediately, except if the // ctx is canceled - in which case the ambiguous error is probably caused // by the cancellation (and in any case it's pointless to retry with a // canceled ctx). return ctx.Err() == nil - } else if errors.HasType(err, (*roachpb.TransactionStatusError)(nil)) { + } else if errors.HasType(err, (*kvpb.TransactionStatusError)(nil)) { // 21.2 nodes can return a TransactionStatusError when they should have // returned an AmbiguousResultError. // TODO(andrei): Remove this in 22.2. @@ -585,7 +586,7 @@ func (nl *NodeLiveness) CreateLivenessRecord(ctx context.Context, nodeID roachpb // We don't bother adding a gossip trigger, that'll happen with the // first heartbeat. We still keep it as a 1PC commit to avoid leaving // write intents. - b.AddRawRequest(&roachpb.EndTxnRequest{ + b.AddRawRequest(&kvpb.EndTxnRequest{ Commit: true, Require1PC: true, }) @@ -1354,7 +1355,7 @@ func (nl *NodeLiveness) updateLivenessAttempt( // Use a trigger on EndTxn to indicate that node liveness should be // re-gossiped. Further, require that this transaction complete as a one // phase commit to eliminate the possibility of leaving write intents. - b.AddRawRequest(&roachpb.EndTxnRequest{ + b.AddRawRequest(&kvpb.EndTxnRequest{ Commit: true, Require1PC: true, InternalCommitTrigger: &roachpb.InternalCommitTrigger{ @@ -1368,7 +1369,7 @@ func (nl *NodeLiveness) updateLivenessAttempt( }) return txn.Run(ctx, b) }); err != nil { - if tErr := (*roachpb.ConditionFailedError)(nil); errors.As(err, &tErr) { + if tErr := (*kvpb.ConditionFailedError)(nil); errors.As(err, &tErr) { if tErr.ActualValue == nil { return Record{}, handleCondFailed(Record{}) } diff --git a/pkg/kv/kvserver/loqrecovery/BUILD.bazel b/pkg/kv/kvserver/loqrecovery/BUILD.bazel index 99b8818e25be..1cb3e649fe05 100644 --- a/pkg/kv/kvserver/loqrecovery/BUILD.bazel +++ b/pkg/kv/kvserver/loqrecovery/BUILD.bazel @@ -68,6 +68,7 @@ go_test( deps = [ "//pkg/base", "//pkg/keys", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/kv/kvserver/kvserverbase", "//pkg/kv/kvserver/kvserverpb", diff --git a/pkg/kv/kvserver/loqrecovery/collect_raft_log_test.go b/pkg/kv/kvserver/loqrecovery/collect_raft_log_test.go index 1478edfda641..4fb4c2936548 100644 --- a/pkg/kv/kvserver/loqrecovery/collect_raft_log_test.go +++ b/pkg/kv/kvserver/loqrecovery/collect_raft_log_test.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/loqrecovery" @@ -145,7 +146,7 @@ func checkRaftLog( makeSnapshot := make(chan storage.Engine, 2) snapshots := make(chan storage.Reader, 2) - raftFilter := func(args kvserverbase.ApplyFilterArgs) (int, *roachpb.Error) { + raftFilter := func(args kvserverbase.ApplyFilterArgs) (int, *kvpb.Error) { t.Helper() select { case store := <-makeSnapshot: diff --git a/pkg/kv/kvserver/merge_queue.go b/pkg/kv/kvserver/merge_queue.go index 6d3d29bd9ea6..1f5097d998f7 100644 --- a/pkg/kv/kvserver/merge_queue.go +++ b/pkg/kv/kvserver/merge_queue.go @@ -17,6 +17,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" @@ -186,16 +187,16 @@ func (mq *mergeQueue) requestRangeStats( ctx context.Context, key roachpb.Key, ) (desc *roachpb.RangeDescriptor, stats enginepb.MVCCStats, ls loadSplitStat, err error) { - ba := &roachpb.BatchRequest{} - ba.Add(&roachpb.RangeStatsRequest{ - RequestHeader: roachpb.RequestHeader{Key: key}, + ba := &kvpb.BatchRequest{} + ba.Add(&kvpb.RangeStatsRequest{ + RequestHeader: kvpb.RequestHeader{Key: key}, }) br, pErr := mq.db.NonTransactionalSender().Send(ctx, ba) if pErr != nil { return nil, enginepb.MVCCStats{}, loadSplitStat{}, pErr.GoError() } - res := br.Responses[0].GetInner().(*roachpb.RangeStatsResponse) + res := br.Responses[0].GetInner().(*kvpb.RangeStatsResponse) desc = &res.RangeInfo.Desc stats = res.MVCCStats @@ -385,10 +386,10 @@ func (mq *mergeQueue) process( humanizeutil.IBytes(minBytes), loadMergeReason, ) - _, pErr := lhsRepl.AdminMerge(ctx, roachpb.AdminMergeRequest{ - RequestHeader: roachpb.RequestHeader{Key: lhsRepl.Desc().StartKey.AsRawKey()}, + _, pErr := lhsRepl.AdminMerge(ctx, kvpb.AdminMergeRequest{ + RequestHeader: kvpb.RequestHeader{Key: lhsRepl.Desc().StartKey.AsRawKey()}, }, reason) - if err := pErr.GoError(); errors.HasType(err, (*roachpb.ConditionFailedError)(nil)) { + if err := pErr.GoError(); errors.HasType(err, (*kvpb.ConditionFailedError)(nil)) { // ConditionFailedErrors are an expected outcome for range merge // attempts because merges can race with other descriptor modifications. // On seeing a ConditionFailedError, don't return an error and enqueue diff --git a/pkg/kv/kvserver/mvcc_gc_queue.go b/pkg/kv/kvserver/mvcc_gc_queue.go index 20c307c95d08..2d8af90ab5bf 100644 --- a/pkg/kv/kvserver/mvcc_gc_queue.go +++ b/pkg/kv/kvserver/mvcc_gc_queue.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/gc" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/intentresolver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvadmission" @@ -546,20 +547,20 @@ type replicaGCer struct { var _ gc.GCer = &replicaGCer{} -func (r *replicaGCer) template() roachpb.GCRequest { +func (r *replicaGCer) template() kvpb.GCRequest { desc := r.repl.Desc() - var template roachpb.GCRequest + var template kvpb.GCRequest template.Key = desc.StartKey.AsRawKey() template.EndKey = desc.EndKey.AsRawKey() return template } -func (r *replicaGCer) send(ctx context.Context, req roachpb.GCRequest) error { +func (r *replicaGCer) send(ctx context.Context, req kvpb.GCRequest) error { n := atomic.AddInt32(&r.count, 1) log.Eventf(ctx, "sending batch %d (%d keys)", n, len(req.Keys)) - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} // Technically not needed since we're talking directly to the Replica. ba.RangeID = r.repl.Desc().RangeID ba.Timestamp = r.repl.Clock().Now() @@ -569,7 +570,7 @@ func (r *replicaGCer) send(ctx context.Context, req roachpb.GCRequest) error { var admissionHandle kvadmission.Handle if r.admissionController != nil { pri := admissionpb.WorkPriority(gc.AdmissionPriority.Get(&r.repl.ClusterSettings().SV)) - ba.AdmissionHeader = roachpb.AdmissionHeader{ + ba.AdmissionHeader = kvpb.AdmissionHeader{ // TODO(irfansharif): GC could be expected to be BulkNormalPri, so // that it does not impact user-facing traffic when resources (e.g. // CPU, write capacity of the store) are scarce. However long delays @@ -595,7 +596,7 @@ func (r *replicaGCer) send(ctx context.Context, req roachpb.GCRequest) error { // it'll be lessened overall. Priority: int32(pri), CreateTime: timeutil.Now().UnixNano(), - Source: roachpb.AdmissionHeader_ROOT_KV, + Source: kvpb.AdmissionHeader_ROOT_KV, NoMemoryReservedAtSource: true, } ba.Replica.StoreID = r.storeID @@ -625,9 +626,9 @@ func (r *replicaGCer) SetGCThreshold(ctx context.Context, thresh gc.Threshold) e func (r *replicaGCer) GC( ctx context.Context, - keys []roachpb.GCRequest_GCKey, - rangeKeys []roachpb.GCRequest_GCRangeKey, - clearRange *roachpb.GCRequest_GCClearRange, + keys []kvpb.GCRequest_GCKey, + rangeKeys []kvpb.GCRequest_GCRangeKey, + clearRange *kvpb.GCRequest_GCClearRange, ) error { if len(keys) == 0 && len(rangeKeys) == 0 && clearRange == nil { return nil @@ -739,7 +740,7 @@ func (mgcq *mvccGCQueue) process( }, func(ctx context.Context, intents []roachpb.Intent) error { intentCount, err := repl.store.intentResolver. - CleanupIntents(ctx, intents, gcTimestamp, roachpb.PUSH_TOUCH) + CleanupIntents(ctx, intents, gcTimestamp, kvpb.PUSH_TOUCH) if err == nil { mgcq.store.metrics.GCResolveSuccess.Inc(int64(intentCount)) } else { @@ -792,8 +793,8 @@ func (mgcq *mvccGCQueue) process( log.Infof(ctx, "GC still needed following GC, recomputing MVCC stats") log.Infof(ctx, "old score %s", r) log.Infof(ctx, "new score %s", scoreAfter) - req := roachpb.RecomputeStatsRequest{ - RequestHeader: roachpb.RequestHeader{Key: desc.StartKey.AsRawKey()}, + req := kvpb.RecomputeStatsRequest{ + RequestHeader: kvpb.RequestHeader{Key: desc.StartKey.AsRawKey()}, } var b kv.Batch b.AddRawRequest(&req) diff --git a/pkg/kv/kvserver/mvcc_gc_queue_test.go b/pkg/kv/kvserver/mvcc_gc_queue_test.go index e6f1f530da7e..1516185c4f6a 100644 --- a/pkg/kv/kvserver/mvcc_gc_queue_test.go +++ b/pkg/kv/kvserver/mvcc_gc_queue_test.go @@ -22,6 +22,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/gc" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -782,7 +783,7 @@ func TestMVCCGCQueueProcess(t *testing.T) { if len(datum.endKey) > 0 { drArgs := deleteRangeArgs(datum.key, datum.endKey) drArgs.UseRangeTombstone = true - if _, err := tc.SendWrappedWith(roachpb.Header{ + if _, err := tc.SendWrappedWith(kvpb.Header{ Timestamp: datum.ts, }, &drArgs); err != nil { t.Fatalf("%d: could not delete data: %+v", i, err) @@ -800,7 +801,7 @@ func TestMVCCGCQueueProcess(t *testing.T) { txn.MinTimestamp = datum.ts assignSeqNumsForReqs(txn, &dArgs) } - if _, err := tc.SendWrappedWith(roachpb.Header{ + if _, err := tc.SendWrappedWith(kvpb.Header{ Timestamp: datum.ts, Txn: txn, }, &dArgs); err != nil { @@ -818,7 +819,7 @@ func TestMVCCGCQueueProcess(t *testing.T) { txn.MinTimestamp = datum.ts assignSeqNumsForReqs(txn, &pArgs) } - if _, err := tc.SendWrappedWith(roachpb.Header{ + if _, err := tc.SendWrappedWith(kvpb.Header{ Timestamp: datum.ts, Txn: txn, }, &pArgs); err != nil { @@ -1080,8 +1081,8 @@ func TestMVCCGCQueueTransactionTable(t *testing.T) { // intent resolution. tsc.TestingKnobs.IntentResolverKnobs.MaxIntentResolutionBatchSize = 1 tsc.TestingKnobs.EvalKnobs.TestingEvalFilter = - func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { - if resArgs, ok := filterArgs.Req.(*roachpb.ResolveIntentRequest); ok { + func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { + if resArgs, ok := filterArgs.Req.(*kvpb.ResolveIntentRequest); ok { id := string(resArgs.IntentTxn.Key) // Only count finalizing intent resolution attempts in `resolved`. if resArgs.Status != roachpb.PENDING { @@ -1099,7 +1100,7 @@ func TestMVCCGCQueueTransactionTable(t *testing.T) { // We've special cased one test case. Note that the intent is still // counted in `resolved`. if testCases[id].failResolve { - return roachpb.NewErrorWithTxn(errors.Errorf("boom"), filterArgs.Hdr.Txn) + return kvpb.NewErrorWithTxn(errors.Errorf("boom"), filterArgs.Hdr.Txn) } } return nil @@ -1263,7 +1264,7 @@ func TestMVCCGCQueueIntentResolution(t *testing.T) { for j := 0; j < 5; j++ { pArgs := putArgs(roachpb.Key(fmt.Sprintf("%d-%d", i, j)), []byte("value")) assignSeqNumsForReqs(txns[i], &pArgs) - if _, err := tc.SendWrappedWith(roachpb.Header{ + if _, err := tc.SendWrappedWith(kvpb.Header{ Txn: txns[i], }, &pArgs); err != nil { t.Fatalf("%d: could not put data: %+v", i, err) @@ -1376,8 +1377,8 @@ func TestMVCCGCQueueChunkRequests(t *testing.T) { manual := timeutil.NewManualTime(timeutil.Unix(0, 123)) tsc := TestStoreConfig(hlc.NewClockForTesting(manual)) tsc.TestingKnobs.EvalKnobs.TestingEvalFilter = - func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { - if _, ok := filterArgs.Req.(*roachpb.GCRequest); ok { + func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { + if _, ok := filterArgs.Req.(*kvpb.GCRequest); ok { atomic.AddInt32(&gcRequests, 1) return nil } @@ -1400,7 +1401,7 @@ func TestMVCCGCQueueChunkRequests(t *testing.T) { fmtStr := fmt.Sprintf("%%0%dd", keySize) // First write 2 * gcKeyVersionChunkBytes different keys (each with two versions). - ba1, ba2 := &roachpb.BatchRequest{}, &roachpb.BatchRequest{} + ba1, ba2 := &kvpb.BatchRequest{}, &kvpb.BatchRequest{} for i := 0; i < 2*keyCount; i++ { // Create keys which are key := roachpb.Key(fmt.Sprintf(fmtStr, i)) @@ -1409,11 +1410,11 @@ func TestMVCCGCQueueChunkRequests(t *testing.T) { pArgs = putArgs(key, []byte("value2")) ba2.Add(&pArgs) } - ba1.Header = roachpb.Header{Timestamp: tc.Clock().Now()} + ba1.Header = kvpb.Header{Timestamp: tc.Clock().Now()} if _, pErr := tc.Sender().Send(ctx, ba1); pErr != nil { t.Fatal(pErr) } - ba2.Header = roachpb.Header{Timestamp: tc.Clock().Now()} + ba2.Header = kvpb.Header{Timestamp: tc.Clock().Now()} if _, pErr := tc.Sender().Send(ctx, ba2); pErr != nil { t.Fatal(pErr) } @@ -1423,7 +1424,7 @@ func TestMVCCGCQueueChunkRequests(t *testing.T) { key1 := roachpb.Key(fmt.Sprintf(fmtStr, 2*keyCount)) key2 := roachpb.Key(fmt.Sprintf(fmtStr, 2*keyCount+1)) for i := 0; i < 2*keyCount+1; i++ { - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} // Only write keyCount+1 versions of key1. if i < keyCount+1 { pArgs1 := putArgs(key1, []byte(fmt.Sprintf("value%04d", i))) @@ -1433,7 +1434,7 @@ func TestMVCCGCQueueChunkRequests(t *testing.T) { // tackle key2 in two separate batches. pArgs2 := putArgs(key2, []byte(fmt.Sprintf("value%04d", i))) ba.Add(&pArgs2) - ba.Header = roachpb.Header{Timestamp: tc.Clock().Now()} + ba.Header = kvpb.Header{Timestamp: tc.Clock().Now()} if _, pErr := tc.Sender().Send(ctx, ba); pErr != nil { t.Fatal(pErr) } diff --git a/pkg/kv/kvserver/node_liveness_test.go b/pkg/kv/kvserver/node_liveness_test.go index f98f3f84a850..123ec7487d9b 100644 --- a/pkg/kv/kvserver/node_liveness_test.go +++ b/pkg/kv/kvserver/node_liveness_test.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" @@ -1014,14 +1015,14 @@ func TestNodeLivenessRetryAmbiguousResultError(t *testing.T) { var injectedErrorCount int32 injectError.Store(true) - testingEvalFilter := func(args kvserverbase.FilterArgs) *roachpb.Error { - if _, ok := args.Req.(*roachpb.ConditionalPutRequest); !ok { + testingEvalFilter := func(args kvserverbase.FilterArgs) *kvpb.Error { + if _, ok := args.Req.(*kvpb.ConditionalPutRequest); !ok { return nil } if val := injectError.Load(); val != nil && val.(bool) { atomic.AddInt32(&injectedErrorCount, 1) injectError.Store(false) - return roachpb.NewError(roachpb.NewAmbiguousResultErrorf("test")) + return kvpb.NewError(kvpb.NewAmbiguousResultErrorf("test")) } return nil } @@ -1064,8 +1065,8 @@ func TestNodeLivenessRetryAmbiguousResultOnCreateError(t *testing.T) { defer log.Scope(t).Close(t) errorsToTest := []error{ - roachpb.NewAmbiguousResultErrorf("test"), - roachpb.NewTransactionStatusError(roachpb.TransactionStatusError_REASON_UNKNOWN, "foo"), + kvpb.NewAmbiguousResultErrorf("test"), + kvpb.NewTransactionStatusError(kvpb.TransactionStatusError_REASON_UNKNOWN, "foo"), kv.OnePCNotAllowedError{}, } @@ -1081,8 +1082,8 @@ func TestNodeLivenessRetryAmbiguousResultOnCreateError(t *testing.T) { roachpb.NodeID(2): {true, 0}, roachpb.NodeID(3): {true, 0}, }) - testingEvalFilter := func(args kvserverbase.FilterArgs) *roachpb.Error { - if req, ok := args.Req.(*roachpb.ConditionalPutRequest); ok { + testingEvalFilter := func(args kvserverbase.FilterArgs) *kvpb.Error { + if req, ok := args.Req.(*kvpb.ConditionalPutRequest); ok { if val := injectError.Load(); val != nil { var liveness livenesspb.Liveness if err := req.Value.GetProto(&liveness); err != nil { @@ -1097,7 +1098,7 @@ func TestNodeLivenessRetryAmbiguousResultOnCreateError(t *testing.T) { } injectErrorMap[liveness.NodeID] = injectErrorData{false, injectErrorMap[liveness.NodeID].count + 1} injectError.Store(injectErrorMap) - return roachpb.NewError(errorToTest) + return kvpb.NewError(errorToTest) } } } @@ -1144,9 +1145,9 @@ func TestNodeLivenessNoRetryOnAmbiguousResultCausedByCancellation(t *testing.T) ctx := context.Background() var sem chan struct{} - testingEvalFilter := func(args kvserverbase.FilterArgs) *roachpb.Error { + testingEvalFilter := func(args kvserverbase.FilterArgs) *kvpb.Error { // Maybe trap a liveness heartbeat. - _, ok := args.Req.(*roachpb.ConditionalPutRequest) + _, ok := args.Req.(*kvpb.ConditionalPutRequest) if !ok { return nil } @@ -1208,7 +1209,7 @@ func TestNodeLivenessNoRetryOnAmbiguousResultCausedByCancellation(t *testing.T) // Check that Heartbeat() returned an ambiguous error, and take that as proof // that the heartbeat wasn't retried. - require.True(t, errors.HasType(err, (*roachpb.AmbiguousResultError)(nil)), "%+v", err) + require.True(t, errors.HasType(err, (*kvpb.AmbiguousResultError)(nil)), "%+v", err) } func verifyNodeIsDecommissioning(t *testing.T, tc *testcluster.TestCluster, nodeID roachpb.NodeID) { diff --git a/pkg/kv/kvserver/protectedts/ptcache/BUILD.bazel b/pkg/kv/kvserver/protectedts/ptcache/BUILD.bazel index 95854f43873d..5d7e11b71fc0 100644 --- a/pkg/kv/kvserver/protectedts/ptcache/BUILD.bazel +++ b/pkg/kv/kvserver/protectedts/ptcache/BUILD.bazel @@ -36,6 +36,7 @@ go_test( "//pkg/base", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/kv/kvserver/protectedts", "//pkg/kv/kvserver/protectedts/ptpb", diff --git a/pkg/kv/kvserver/protectedts/ptcache/cache_test.go b/pkg/kv/kvserver/protectedts/ptcache/cache_test.go index cfe9fdb25f3f..84d6aadb12bf 100644 --- a/pkg/kv/kvserver/protectedts/ptcache/cache_test.go +++ b/pkg/kv/kvserver/protectedts/ptcache/cache_test.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptcache" @@ -193,9 +194,9 @@ func TestRefresh(t *testing.T) { withCancel, cancel := context.WithCancel(ctx) defer cancel() done := make(chan struct{}) - st.setFilter(func(ba *roachpb.BatchRequest) *roachpb.Error { - if scanReq, ok := ba.GetArg(roachpb.Scan); ok { - scan := scanReq.(*roachpb.ScanRequest) + st.setFilter(func(ba *kvpb.BatchRequest) *kvpb.Error { + if scanReq, ok := ba.GetArg(kvpb.Scan); ok { + scan := scanReq.(*kvpb.ScanRequest) if scan.Span().Overlaps(metaTableSpan) { <-done } @@ -208,11 +209,11 @@ func TestRefresh(t *testing.T) { close(done) }) t.Run("error propagates while fetching metadata", func(t *testing.T) { - st.setFilter(func(ba *roachpb.BatchRequest) *roachpb.Error { - if scanReq, ok := ba.GetArg(roachpb.Scan); ok { - scan := scanReq.(*roachpb.ScanRequest) + st.setFilter(func(ba *kvpb.BatchRequest) *kvpb.Error { + if scanReq, ok := ba.GetArg(kvpb.Scan); ok { + scan := scanReq.(*kvpb.ScanRequest) if scan.Span().Overlaps(metaTableSpan) { - return roachpb.NewError(errors.New("boom")) + return kvpb.NewError(errors.New("boom")) } } return nil @@ -222,11 +223,11 @@ func TestRefresh(t *testing.T) { }) t.Run("error propagates while fetching records", func(t *testing.T) { protect(t, p, s.Clock().Now(), metaTableSpan) - st.setFilter(func(ba *roachpb.BatchRequest) *roachpb.Error { - if scanReq, ok := ba.GetArg(roachpb.Scan); ok { - scan := scanReq.(*roachpb.ScanRequest) + st.setFilter(func(ba *kvpb.BatchRequest) *kvpb.Error { + if scanReq, ok := ba.GetArg(kvpb.Scan); ok { + scan := scanReq.(*kvpb.ScanRequest) if scan.Span().Overlaps(recordsTableSpan) { - return roachpb.NewError(errors.New("boom")) + return kvpb.NewError(errors.New("boom")) } } return nil @@ -609,7 +610,7 @@ type scanTracker struct { mu syncutil.Mutex metaTableScans int recordsTableScans int - filterFunc func(ba *roachpb.BatchRequest) *roachpb.Error + filterFunc func(ba *kvpb.BatchRequest) *kvpb.Error } func (st *scanTracker) resetCounters() { @@ -626,17 +627,17 @@ func (st *scanTracker) verifyCounters(t *testing.T, expMeta, expRecords int) { require.Equal(t, expRecords, st.recordsTableScans) } -func (st *scanTracker) setFilter(f func(*roachpb.BatchRequest) *roachpb.Error) { +func (st *scanTracker) setFilter(f func(*kvpb.BatchRequest) *kvpb.Error) { st.mu.Lock() defer st.mu.Unlock() st.filterFunc = f } -func (st *scanTracker) requestFilter(_ context.Context, ba *roachpb.BatchRequest) *roachpb.Error { +func (st *scanTracker) requestFilter(_ context.Context, ba *kvpb.BatchRequest) *kvpb.Error { st.mu.Lock() defer st.mu.Unlock() - if scanReq, ok := ba.GetArg(roachpb.Scan); ok { - scan := scanReq.(*roachpb.ScanRequest) + if scanReq, ok := ba.GetArg(kvpb.Scan); ok { + scan := scanReq.(*kvpb.ScanRequest) if scan.Span().Overlaps(metaTableSpan) { st.metaTableScans++ } else if scan.Span().Overlaps(recordsTableSpan) { diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index 758e97059946..70b5cf2a9dbe 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -17,6 +17,7 @@ import ( "sync/atomic" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" @@ -260,7 +261,7 @@ type replicaInQueue interface { IsDestroyed() (DestroyReason, error) Desc() *roachpb.RangeDescriptor maybeInitializeRaftGroup(context.Context) - redirectOnOrAcquireLease(context.Context) (kvserverpb.LeaseStatus, *roachpb.Error) + redirectOnOrAcquireLease(context.Context) (kvserverpb.LeaseStatus, *kvpb.Error) LeaseStatusAt(context.Context, hlc.ClockTimestamp) kvserverpb.LeaseStatus } @@ -979,7 +980,7 @@ func (bq *baseQueue) processReplica(ctx context.Context, repl replicaInQueue) er if bq.needsLease { if _, pErr := repl.redirectOnOrAcquireLease(ctx); pErr != nil { switch v := pErr.GetDetail().(type) { - case *roachpb.NotLeaseHolderError, *roachpb.RangeNotFoundError: + case *kvpb.NotLeaseHolderError, *kvpb.RangeNotFoundError: log.VEventf(ctx, 3, "%s; skipping", v) return nil default: diff --git a/pkg/kv/kvserver/queue_concurrency_test.go b/pkg/kv/kvserver/queue_concurrency_test.go index 93183a35188b..a5d5ffe3f424 100644 --- a/pkg/kv/kvserver/queue_concurrency_test.go +++ b/pkg/kv/kvserver/queue_concurrency_test.go @@ -17,6 +17,7 @@ import ( "testing" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -181,7 +182,7 @@ func (fr *fakeReplica) Desc() *roachpb.RangeDescriptor { func (fr *fakeReplica) maybeInitializeRaftGroup(context.Context) {} func (fr *fakeReplica) redirectOnOrAcquireLease( context.Context, -) (kvserverpb.LeaseStatus, *roachpb.Error) { +) (kvserverpb.LeaseStatus, *kvpb.Error) { // baseQueue only checks that the returned error is nil. return kvserverpb.LeaseStatus{}, nil } diff --git a/pkg/kv/kvserver/raft_log_queue.go b/pkg/kv/kvserver/raft_log_queue.go index 10cc9cd56759..ea16cfd68ac9 100644 --- a/pkg/kv/kvserver/raft_log_queue.go +++ b/pkg/kv/kvserver/raft_log_queue.go @@ -17,6 +17,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -724,8 +725,8 @@ func (rlq *raftLogQueue) process( log.VEventf(ctx, 1, "%v", redact.Safe(decision.String())) } b := &kv.Batch{} - truncRequest := &roachpb.TruncateLogRequest{ - RequestHeader: roachpb.RequestHeader{Key: r.Desc().StartKey.AsRawKey()}, + truncRequest := &kvpb.TruncateLogRequest{ + RequestHeader: kvpb.RequestHeader{Key: r.Desc().StartKey.AsRawKey()}, Index: decision.NewFirstIndex, RangeID: r.RangeID, } diff --git a/pkg/kv/kvserver/raft_log_queue_test.go b/pkg/kv/kvserver/raft_log_queue_test.go index b1f0f82d8d2d..e8384a5c471b 100644 --- a/pkg/kv/kvserver/raft_log_queue_test.go +++ b/pkg/kv/kvserver/raft_log_queue_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" @@ -729,7 +730,7 @@ func TestTruncateLog(t *testing.T) { // Discard the first half of the log. truncateArgs := truncateLogArgs(indexes[5], rangeID) - if _, pErr := tc.SendWrappedWith(roachpb.Header{RangeID: 1}, &truncateArgs); pErr != nil { + if _, pErr := tc.SendWrappedWith(kvpb.Header{RangeID: 1}, &truncateArgs); pErr != nil { t.Fatal(pErr) } @@ -871,8 +872,8 @@ func TestTruncateLogRecompute(t *testing.T) { put := func() { var v roachpb.Value v.SetBytes(bytes.Repeat([]byte("x"), RaftLogQueueStaleSize*5)) - put := roachpb.NewPut(key, v) - ba := &roachpb.BatchRequest{} + put := kvpb.NewPut(key, v) + ba := &kvpb.BatchRequest{} ba.Add(put) ba.RangeID = repl.RangeID diff --git a/pkg/kv/kvserver/raft_transport.go b/pkg/kv/kvserver/raft_transport.go index 4d299d90b8d0..265ea7ceea69 100644 --- a/pkg/kv/kvserver/raft_transport.go +++ b/pkg/kv/kvserver/raft_transport.go @@ -18,6 +18,7 @@ import ( "time" "unsafe" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -106,7 +107,7 @@ type RaftMessageHandler interface { // If an error is encountered during asynchronous processing, it will be // streamed back to the sender of the message as a RaftMessageResponse. HandleRaftRequest(ctx context.Context, req *kvserverpb.RaftMessageRequest, - respStream RaftMessageResponseStream) *roachpb.Error + respStream RaftMessageResponseStream) *kvpb.Error // HandleRaftResponse is called for each raft response. Note that // not all messages receive a response. An error is returned if and only if @@ -233,12 +234,12 @@ func (t *RaftTransport) getHandler(storeID roachpb.StoreID) (RaftMessageHandler, // handleRaftRequest proxies a request to the listening server interface. func (t *RaftTransport) handleRaftRequest( ctx context.Context, req *kvserverpb.RaftMessageRequest, respStream RaftMessageResponseStream, -) *roachpb.Error { +) *kvpb.Error { handler, ok := t.getHandler(req.ToReplica.StoreID) if !ok { log.Warningf(ctx, "unable to accept Raft message from %+v: no handler registered for %+v", req.FromReplica, req.ToReplica) - return roachpb.NewError(roachpb.NewStoreNotFoundError(req.ToReplica.StoreID)) + return kvpb.NewError(kvpb.NewStoreNotFoundError(req.ToReplica.StoreID)) } return handler.HandleRaftRequest(ctx, req, respStream) @@ -247,7 +248,7 @@ func (t *RaftTransport) handleRaftRequest( // newRaftMessageResponse constructs a RaftMessageResponse from the // given request and error. func newRaftMessageResponse( - req *kvserverpb.RaftMessageRequest, pErr *roachpb.Error, + req *kvserverpb.RaftMessageRequest, pErr *kvpb.Error, ) *kvserverpb.RaftMessageResponse { resp := &kvserverpb.RaftMessageResponse{ RangeID: req.RangeID, @@ -376,7 +377,7 @@ func (t *RaftTransport) RaftSnapshot(stream MultiRaft_RaftSnapshotServer) error if !ok { log.Warningf(ctx, "unable to accept Raft message from %+v: no handler registered for %+v", rmr.FromReplica, rmr.ToReplica) - return roachpb.NewStoreNotFoundError(rmr.ToReplica.StoreID) + return kvpb.NewStoreNotFoundError(rmr.ToReplica.StoreID) } return handler.HandleSnapshot(ctx, req.Header, stream) } diff --git a/pkg/kv/kvserver/raft_transport_test.go b/pkg/kv/kvserver/raft_transport_test.go index d8fc2ce50578..46d0dc09b0a8 100644 --- a/pkg/kv/kvserver/raft_transport_test.go +++ b/pkg/kv/kvserver/raft_transport_test.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -60,7 +61,7 @@ func newChannelServer(bufSize int, maxSleep time.Duration) channelServer { func (s channelServer) HandleRaftRequest( ctx context.Context, req *kvserverpb.RaftMessageRequest, _ kvserver.RaftMessageResponseStream, -) *roachpb.Error { +) *kvpb.Error { if s.maxSleep != 0 { // maxSleep simulates goroutine scheduling delays that could // result in messages being processed out of order (in previous @@ -68,7 +69,7 @@ func (s channelServer) HandleRaftRequest( time.Sleep(time.Duration(rand.Int63n(int64(s.maxSleep)))) } if s.brokenRange != 0 && s.brokenRange == req.RangeID { - return roachpb.NewErrorf(channelServerBrokenRangeMessage) + return kvpb.NewErrorf(channelServerBrokenRangeMessage) } s.ch <- req return nil @@ -79,8 +80,8 @@ func (s channelServer) HandleRaftResponse( ) error { // Mimic the logic in (*Store).HandleRaftResponse without requiring an // entire Store object to be pulled into these tests. - if val, ok := resp.Union.GetValue().(*roachpb.Error); ok { - if err, ok := val.GetDetail().(*roachpb.StoreNotFoundError); ok { + if val, ok := resp.Union.GetValue().(*kvpb.Error); ok { + if err, ok := val.GetDetail().(*kvpb.StoreNotFoundError); ok { return err } } diff --git a/pkg/kv/kvserver/raftlog/encoding.go b/pkg/kv/kvserver/raftlog/encoding.go index e0427b51ec9c..25859cacefd2 100644 --- a/pkg/kv/kvserver/raftlog/encoding.go +++ b/pkg/kv/kvserver/raftlog/encoding.go @@ -40,7 +40,7 @@ const ( // below-raft admission control (Admission{Priority,CreateTime,OriginNode}). EntryEncodingStandardWithAC // EntryEncodingSideloadedWithAC indicates a proposal representing the - // result of a roachpb.AddSSTableRequest for which the payload (the SST) is + // result of a kvpb.AddSSTableRequest for which the payload (the SST) is // stored outside the storage engine to improve storage performance. // // This is a raftpb.Entry of type EntryNormal whose data slice is either diff --git a/pkg/kv/kvserver/rangefeed/BUILD.bazel b/pkg/kv/kvserver/rangefeed/BUILD.bazel index 9c566a82403c..dbd2972fe946 100644 --- a/pkg/kv/kvserver/rangefeed/BUILD.bazel +++ b/pkg/kv/kvserver/rangefeed/BUILD.bazel @@ -17,6 +17,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/keys", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/settings", "//pkg/storage", @@ -57,6 +58,7 @@ go_test( "//pkg/base", "//pkg/clusterversion", "//pkg/keys", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/settings/cluster", "//pkg/storage", diff --git a/pkg/kv/kvserver/rangefeed/catchup_scan.go b/pkg/kv/kvserver/rangefeed/catchup_scan.go index 3b5457d17a54..1432aeaddb5d 100644 --- a/pkg/kv/kvserver/rangefeed/catchup_scan.go +++ b/pkg/kv/kvserver/rangefeed/catchup_scan.go @@ -15,6 +15,7 @@ import ( "context" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" @@ -116,7 +117,7 @@ func (i *CatchUpIterator) Close() { // TODO(ssd): Clarify memory ownership. Currently, the memory backing // the RangeFeedEvents isn't modified by the caller after this // returns. However, we may revist this in #69596. -type outputEventFn func(e *roachpb.RangeFeedEvent) error +type outputEventFn func(e *kvpb.RangeFeedEvent) error // CatchUpScan iterates over all changes in the configured key/time span, and // emits them as RangeFeedEvents via outputFn in chronological order. @@ -138,7 +139,7 @@ func (i *CatchUpIterator) CatchUpScan( // events for the same key until a different key is encountered, then output // the encountered values in reverse. This also allows us to buffer events // as we fill in previous values. - reorderBuf := make([]roachpb.RangeFeedEvent, 0, 5) + reorderBuf := make([]kvpb.RangeFeedEvent, 0, 5) outputEvents := func() error { for i := len(reorderBuf) - 1; i >= 0; i-- { @@ -146,7 +147,7 @@ func (i *CatchUpIterator) CatchUpScan( if err := outputFn(&e); err != nil { return err } - reorderBuf[i] = roachpb.RangeFeedEvent{} // Drop references to values to allow GC + reorderBuf[i] = kvpb.RangeFeedEvent{} // Drop references to values to allow GC } reorderBuf = reorderBuf[:0] return nil @@ -191,8 +192,8 @@ func (i *CatchUpIterator) CatchUpScan( a, span.Key = a.Copy(rangeKeys.Bounds.Key, 0) a, span.EndKey = a.Copy(rangeKeys.Bounds.EndKey, 0) ts := rangeKeys.Versions[j].Timestamp - err := outputFn(&roachpb.RangeFeedEvent{ - DeleteRange: &roachpb.RangeFeedDeleteRange{ + err := outputFn(&kvpb.RangeFeedEvent{ + DeleteRange: &kvpb.RangeFeedDeleteRange{ Span: span, Timestamp: ts, }, @@ -328,8 +329,8 @@ func (i *CatchUpIterator) CatchUpScan( if !ignore { // Add value to reorderBuf to be output. - var event roachpb.RangeFeedEvent - event.MustSetValue(&roachpb.RangeFeedValue{ + var event kvpb.RangeFeedEvent + event.MustSetValue(&kvpb.RangeFeedValue{ Key: key, Value: roachpb.Value{ RawBytes: val, diff --git a/pkg/kv/kvserver/rangefeed/catchup_scan_bench_test.go b/pkg/kv/kvserver/rangefeed/catchup_scan_bench_test.go index f1be451858b9..c1a76ce35221 100644 --- a/pkg/kv/kvserver/rangefeed/catchup_scan_bench_test.go +++ b/pkg/kv/kvserver/rangefeed/catchup_scan_bench_test.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rangefeed" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -52,7 +53,7 @@ func runCatchUpBenchmark(b *testing.B, emk engineMaker, opts benchOptions) (numE iter := rangefeed.NewCatchUpIterator(eng, span, opts.ts, nil, nil) defer iter.Close() counter := 0 - err := iter.CatchUpScan(ctx, func(*roachpb.RangeFeedEvent) error { + err := iter.CatchUpScan(ctx, func(*kvpb.RangeFeedEvent) error { counter++ return nil }, opts.withDiff) diff --git a/pkg/kv/kvserver/rangefeed/catchup_scan_test.go b/pkg/kv/kvserver/rangefeed/catchup_scan_test.go index f4ce29a1473f..dfe610273590 100644 --- a/pkg/kv/kvserver/rangefeed/catchup_scan_test.go +++ b/pkg/kv/kvserver/rangefeed/catchup_scan_test.go @@ -15,6 +15,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" @@ -113,15 +114,15 @@ func TestCatchupScan(t *testing.T) { span := roachpb.Span{Key: testKey1, EndKey: roachpb.KeyMax} iter := NewCatchUpIterator(eng, span, ts1, nil, nil) defer iter.Close() - var events []roachpb.RangeFeedValue + var events []kvpb.RangeFeedValue // ts1 here is exclusive, so we do not want the versions at ts1. - require.NoError(t, iter.CatchUpScan(ctx, func(e *roachpb.RangeFeedEvent) error { + require.NoError(t, iter.CatchUpScan(ctx, func(e *kvpb.RangeFeedEvent) error { events = append(events, *e.Val) return nil }, withDiff)) require.Equal(t, 4, len(events)) checkEquality := func( - kv storage.MVCCKeyValue, prevKV storage.MVCCKeyValue, event roachpb.RangeFeedValue) { + kv storage.MVCCKeyValue, prevKV storage.MVCCKeyValue, event kvpb.RangeFeedValue) { require.Equal(t, string(kv.Key.Key), string(event.Key)) require.Equal(t, kv.Key.Timestamp, event.Value.Timestamp) require.Equal(t, string(kv.Value), string(event.Value.RawBytes)) @@ -198,7 +199,7 @@ func TestCatchupScanSeesOldIntent(t *testing.T) { defer iter.Close() keys := map[string]struct{}{} - require.NoError(t, iter.CatchUpScan(ctx, func(e *roachpb.RangeFeedEvent) error { + require.NoError(t, iter.CatchUpScan(ctx, func(e *kvpb.RangeFeedEvent) error { keys[string(e.Val.Key)] = struct{}{} return nil }, true /* withDiff */)) diff --git a/pkg/kv/kvserver/rangefeed/processor.go b/pkg/kv/kvserver/rangefeed/processor.go index 8daff9b90ca7..8eb02f02eca8 100644 --- a/pkg/kv/kvserver/rangefeed/processor.go +++ b/pkg/kv/kvserver/rangefeed/processor.go @@ -16,6 +16,7 @@ import ( "sync" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -40,9 +41,9 @@ const ( // newErrBufferCapacityExceeded creates an error that is returned to subscribers // if the rangefeed processor is not able to keep up with the flow of incoming // events and is forced to drop events in order to not block. -func newErrBufferCapacityExceeded() *roachpb.Error { - return roachpb.NewError( - roachpb.NewRangeFeedRetryError(roachpb.RangeFeedRetryError_REASON_SLOW_CONSUMER), +func newErrBufferCapacityExceeded() *kvpb.Error { + return kvpb.NewError( + kvpb.NewRangeFeedRetryError(kvpb.RangeFeedRetryError_REASON_SLOW_CONSUMER), ) } @@ -123,7 +124,7 @@ type Processor struct { filterResC chan *Filter eventC chan *event spanErrC chan spanErr - stopC chan *roachpb.Error + stopC chan *kvpb.Error stoppedC chan struct{} } @@ -185,7 +186,7 @@ type syncEvent struct { // registrations. type spanErr struct { span roachpb.Span - pErr *roachpb.Error + pErr *kvpb.Error } // NewProcessor creates a new rangefeed Processor. The corresponding goroutine @@ -206,7 +207,7 @@ func NewProcessor(cfg Config) *Processor { filterResC: make(chan *Filter), eventC: make(chan *event, cfg.EventChanCap), spanErrC: make(chan spanErr), - stopC: make(chan *roachpb.Error, 1), + stopC: make(chan *kvpb.Error, 1), stoppedC: make(chan struct{}), } return p @@ -237,7 +238,7 @@ func (p *Processor) Start(stopper *stop.Stopper, rtsIterFunc IntentScannerConstr if err := stopper.RunAsyncTask(ctx, "rangefeed.Processor", func(ctx context.Context) { p.run(ctx, p.RangeID, rtsIterFunc, stopper) }); err != nil { - p.reg.DisconnectWithErr(all, roachpb.NewError(err)) + p.reg.DisconnectWithErr(all, kvpb.NewError(err)) close(p.stoppedC) return err } @@ -322,7 +323,7 @@ func (p *Processor) run( } } if err := stopper.RunAsyncTask(ctx, "rangefeed: output loop", runOutputLoop); err != nil { - r.disconnect(roachpb.NewError(err)) + r.disconnect(kvpb.NewError(err)) p.reg.Unregister(ctx, &r) } @@ -399,7 +400,7 @@ func (p *Processor) run( // Exit on stopper. case <-stopper.ShouldQuiesce(): - pErr := roachpb.NewError(&roachpb.NodeUnavailableError{}) + pErr := kvpb.NewError(&kvpb.NodeUnavailableError{}) p.reg.DisconnectWithErr(all, pErr) return } @@ -416,7 +417,7 @@ func (p *Processor) Stop() { // StopWithErr shuts down the processor and closes all registrations with the // specified error. Safe to call on nil Processor. It is not valid to restart a // processor after it has been stopped. -func (p *Processor) StopWithErr(pErr *roachpb.Error) { +func (p *Processor) StopWithErr(pErr *kvpb.Error) { if p == nil { return } @@ -428,7 +429,7 @@ func (p *Processor) StopWithErr(pErr *roachpb.Error) { // DisconnectSpanWithErr disconnects all rangefeed registrations that overlap // the given span with the given error. -func (p *Processor) DisconnectSpanWithErr(span roachpb.Span, pErr *roachpb.Error) { +func (p *Processor) DisconnectSpanWithErr(span roachpb.Span, pErr *kvpb.Error) { if p == nil { return } @@ -439,7 +440,7 @@ func (p *Processor) DisconnectSpanWithErr(span roachpb.Span, pErr *roachpb.Error } } -func (p *Processor) sendStop(pErr *roachpb.Error) { +func (p *Processor) sendStop(pErr *kvpb.Error) { select { case p.stopC <- pErr: // stopC has non-zero capacity so this should not block unless @@ -473,7 +474,7 @@ func (p *Processor) Register( catchUpIterConstructor CatchUpIteratorConstructor, withDiff bool, stream Stream, - errC chan<- *roachpb.Error, + errC chan<- *kvpb.Error, ) (bool, *Filter) { // Synchronize the event channel so that this registration doesn't see any // events that were consumed before this registration was called. Instead, @@ -797,8 +798,8 @@ func (p *Processor) publishValue( if prevValue != nil { prevVal.RawBytes = prevValue } - var event roachpb.RangeFeedEvent - event.MustSetValue(&roachpb.RangeFeedValue{ + var event kvpb.RangeFeedEvent + event.MustSetValue(&kvpb.RangeFeedValue{ Key: key, Value: roachpb.Value{ RawBytes: value, @@ -820,8 +821,8 @@ func (p *Processor) publishDeleteRange( log.Fatalf(ctx, "span %s not in Processor's key range %v", span, p.Span) } - var event roachpb.RangeFeedEvent - event.MustSetValue(&roachpb.RangeFeedDeleteRange{ + var event kvpb.RangeFeedEvent + event.MustSetValue(&kvpb.RangeFeedDeleteRange{ Span: span, Timestamp: timestamp, }) @@ -841,8 +842,8 @@ func (p *Processor) publishSSTable( if sstWTS.IsEmpty() { panic(errors.AssertionFailedf("received SSTable without write timestamp")) } - p.reg.PublishToOverlapping(ctx, sstSpan, &roachpb.RangeFeedEvent{ - SST: &roachpb.RangeFeedSSTable{ + p.reg.PublishToOverlapping(ctx, sstSpan, &kvpb.RangeFeedEvent{ + SST: &kvpb.RangeFeedSSTable{ Data: sst, Span: sstSpan, WriteTS: sstWTS, @@ -858,12 +859,12 @@ func (p *Processor) publishCheckpoint(ctx context.Context) { p.reg.PublishToOverlapping(ctx, all, event, nil) } -func (p *Processor) newCheckpointEvent() *roachpb.RangeFeedEvent { +func (p *Processor) newCheckpointEvent() *kvpb.RangeFeedEvent { // Create a RangeFeedCheckpoint over the Processor's entire span. Each // individual registration will trim this down to just the key span that // it is listening on in registration.maybeStripEvent before publishing. - var event roachpb.RangeFeedEvent - event.MustSetValue(&roachpb.RangeFeedCheckpoint{ + var event kvpb.RangeFeedEvent + event.MustSetValue(&kvpb.RangeFeedCheckpoint{ Span: p.Span.AsRawSpanWithNoLocals(), ResolvedTS: p.rts.Get(), }) diff --git a/pkg/kv/kvserver/rangefeed/processor_test.go b/pkg/kv/kvserver/rangefeed/processor_test.go index d1508f54723a..944c3f70ccc9 100644 --- a/pkg/kv/kvserver/rangefeed/processor_test.go +++ b/pkg/kv/kvserver/rangefeed/processor_test.go @@ -23,6 +23,7 @@ import ( "time" "unsafe" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage" @@ -110,26 +111,26 @@ func abortTxnOp(txnID uuid.UUID) enginepb.MVCCLogicalOp { }) } -func makeRangeFeedEvent(val interface{}) *roachpb.RangeFeedEvent { - var event roachpb.RangeFeedEvent +func makeRangeFeedEvent(val interface{}) *kvpb.RangeFeedEvent { + var event kvpb.RangeFeedEvent event.MustSetValue(val) return &event } -func rangeFeedValueWithPrev(key roachpb.Key, val, prev roachpb.Value) *roachpb.RangeFeedEvent { - return makeRangeFeedEvent(&roachpb.RangeFeedValue{ +func rangeFeedValueWithPrev(key roachpb.Key, val, prev roachpb.Value) *kvpb.RangeFeedEvent { + return makeRangeFeedEvent(&kvpb.RangeFeedValue{ Key: key, Value: val, PrevValue: prev, }) } -func rangeFeedValue(key roachpb.Key, val roachpb.Value) *roachpb.RangeFeedEvent { +func rangeFeedValue(key roachpb.Key, val roachpb.Value) *kvpb.RangeFeedEvent { return rangeFeedValueWithPrev(key, val, roachpb.Value{}) } -func rangeFeedCheckpoint(span roachpb.Span, ts hlc.Timestamp) *roachpb.RangeFeedEvent { - return makeRangeFeedEvent(&roachpb.RangeFeedCheckpoint{ +func rangeFeedCheckpoint(span roachpb.Span, ts hlc.Timestamp) *kvpb.RangeFeedEvent { + return makeRangeFeedEvent(&kvpb.RangeFeedCheckpoint{ Span: span, ResolvedTS: ts, }) @@ -203,7 +204,7 @@ func TestProcessorBasic(t *testing.T) { // Add a registration. r1Stream := newTestStream() - r1ErrC := make(chan *roachpb.Error, 1) + r1ErrC := make(chan *kvpb.Error, 1) r1OK, r1Filter := p.Register( roachpb.RSpan{Key: roachpb.RKey("a"), EndKey: roachpb.RKey("m")}, hlc.Timestamp{WallTime: 1}, @@ -216,7 +217,7 @@ func TestProcessorBasic(t *testing.T) { p.syncEventAndRegistrations() require.Equal(t, 1, p.Len()) require.Equal(t, - []*roachpb.RangeFeedEvent{rangeFeedCheckpoint( + []*kvpb.RangeFeedEvent{rangeFeedCheckpoint( roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("m")}, hlc.Timestamp{WallTime: 1}, )}, @@ -235,7 +236,7 @@ func TestProcessorBasic(t *testing.T) { p.ForwardClosedTS(ctx, hlc.Timestamp{WallTime: 5}) p.syncEventAndRegistrations() require.Equal(t, - []*roachpb.RangeFeedEvent{rangeFeedCheckpoint( + []*kvpb.RangeFeedEvent{rangeFeedCheckpoint( roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("m")}, hlc.Timestamp{WallTime: 5}, )}, @@ -247,7 +248,7 @@ func TestProcessorBasic(t *testing.T) { writeValueOpWithKV(roachpb.Key("c"), hlc.Timestamp{WallTime: 6}, []byte("val"))) p.syncEventAndRegistrations() require.Equal(t, - []*roachpb.RangeFeedEvent{rangeFeedValue( + []*kvpb.RangeFeedEvent{rangeFeedValue( roachpb.Key("c"), roachpb.Value{ RawBytes: []byte("val"), @@ -261,18 +262,18 @@ func TestProcessorBasic(t *testing.T) { p.ConsumeLogicalOps(ctx, writeValueOpWithKV(roachpb.Key("s"), hlc.Timestamp{WallTime: 6}, []byte("val"))) p.syncEventAndRegistrations() - require.Equal(t, []*roachpb.RangeFeedEvent(nil), r1Stream.Events()) + require.Equal(t, []*kvpb.RangeFeedEvent(nil), r1Stream.Events()) // Test intent that is aborted with one registration. txn1 := uuid.MakeV4() // Write intent. p.ConsumeLogicalOps(ctx, writeIntentOp(txn1, hlc.Timestamp{WallTime: 6})) p.syncEventAndRegistrations() - require.Equal(t, []*roachpb.RangeFeedEvent(nil), r1Stream.Events()) + require.Equal(t, []*kvpb.RangeFeedEvent(nil), r1Stream.Events()) // Abort. p.ConsumeLogicalOps(ctx, abortIntentOp(txn1)) p.syncEventC() - require.Equal(t, []*roachpb.RangeFeedEvent(nil), r1Stream.Events()) + require.Equal(t, []*kvpb.RangeFeedEvent(nil), r1Stream.Events()) require.Equal(t, 0, p.rts.intentQ.Len()) // Test intent that is committed with one registration. @@ -280,12 +281,12 @@ func TestProcessorBasic(t *testing.T) { // Write intent. p.ConsumeLogicalOps(ctx, writeIntentOp(txn2, hlc.Timestamp{WallTime: 10})) p.syncEventAndRegistrations() - require.Equal(t, []*roachpb.RangeFeedEvent(nil), r1Stream.Events()) + require.Equal(t, []*kvpb.RangeFeedEvent(nil), r1Stream.Events()) // Forward closed timestamp. Should now be stuck on intent. p.ForwardClosedTS(ctx, hlc.Timestamp{WallTime: 15}) p.syncEventAndRegistrations() require.Equal(t, - []*roachpb.RangeFeedEvent{rangeFeedCheckpoint( + []*kvpb.RangeFeedEvent{rangeFeedCheckpoint( roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("m")}, hlc.Timestamp{WallTime: 9}, )}, @@ -295,7 +296,7 @@ func TestProcessorBasic(t *testing.T) { p.ConsumeLogicalOps(ctx, updateIntentOp(txn2, hlc.Timestamp{WallTime: 12})) p.syncEventAndRegistrations() require.Equal(t, - []*roachpb.RangeFeedEvent{rangeFeedCheckpoint( + []*kvpb.RangeFeedEvent{rangeFeedCheckpoint( roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("m")}, hlc.Timestamp{WallTime: 11}, )}, @@ -306,7 +307,7 @@ func TestProcessorBasic(t *testing.T) { commitIntentOpWithKV(txn2, roachpb.Key("e"), hlc.Timestamp{WallTime: 13}, []byte("ival"))) p.syncEventAndRegistrations() require.Equal(t, - []*roachpb.RangeFeedEvent{ + []*kvpb.RangeFeedEvent{ rangeFeedValue( roachpb.Key("e"), roachpb.Value{ @@ -324,7 +325,7 @@ func TestProcessorBasic(t *testing.T) { // Add another registration with withDiff = true. r2Stream := newTestStream() - r2ErrC := make(chan *roachpb.Error, 1) + r2ErrC := make(chan *kvpb.Error, 1) r2OK, r1And2Filter := p.Register( roachpb.RSpan{Key: roachpb.RKey("c"), EndKey: roachpb.RKey("z")}, hlc.Timestamp{WallTime: 1}, @@ -337,7 +338,7 @@ func TestProcessorBasic(t *testing.T) { p.syncEventAndRegistrations() require.Equal(t, 2, p.Len()) require.Equal(t, - []*roachpb.RangeFeedEvent{rangeFeedCheckpoint( + []*kvpb.RangeFeedEvent{rangeFeedCheckpoint( roachpb.Span{Key: roachpb.Key("c"), EndKey: roachpb.Key("z")}, hlc.Timestamp{WallTime: 15}, )}, @@ -357,12 +358,12 @@ func TestProcessorBasic(t *testing.T) { // Both registrations should see checkpoint. p.ForwardClosedTS(ctx, hlc.Timestamp{WallTime: 20}) p.syncEventAndRegistrations() - chEventAM := []*roachpb.RangeFeedEvent{rangeFeedCheckpoint( + chEventAM := []*kvpb.RangeFeedEvent{rangeFeedCheckpoint( roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("m")}, hlc.Timestamp{WallTime: 20}, )} require.Equal(t, chEventAM, r1Stream.Events()) - chEventCZ := []*roachpb.RangeFeedEvent{rangeFeedCheckpoint( + chEventCZ := []*kvpb.RangeFeedEvent{rangeFeedCheckpoint( roachpb.Span{Key: roachpb.Key("c"), EndKey: roachpb.Key("z")}, hlc.Timestamp{WallTime: 20}, )} @@ -372,7 +373,7 @@ func TestProcessorBasic(t *testing.T) { p.ConsumeLogicalOps(ctx, writeValueOpWithKV(roachpb.Key("k"), hlc.Timestamp{WallTime: 22}, []byte("val2"))) p.syncEventAndRegistrations() - valEvent := []*roachpb.RangeFeedEvent{rangeFeedValue( + valEvent := []*kvpb.RangeFeedEvent{rangeFeedValue( roachpb.Key("k"), roachpb.Value{ RawBytes: []byte("val2"), @@ -386,14 +387,14 @@ func TestProcessorBasic(t *testing.T) { p.ConsumeLogicalOps(ctx, writeValueOpWithKV(roachpb.Key("v"), hlc.Timestamp{WallTime: 23}, []byte("val3"))) p.syncEventAndRegistrations() - valEvent2 := []*roachpb.RangeFeedEvent{rangeFeedValue( + valEvent2 := []*kvpb.RangeFeedEvent{rangeFeedValue( roachpb.Key("v"), roachpb.Value{ RawBytes: []byte("val3"), Timestamp: hlc.Timestamp{WallTime: 23}, }, )} - require.Equal(t, []*roachpb.RangeFeedEvent(nil), r1Stream.Events()) + require.Equal(t, []*kvpb.RangeFeedEvent(nil), r1Stream.Events()) require.Equal(t, valEvent2, r2Stream.Events()) // Cancel the first registration. @@ -401,13 +402,13 @@ func TestProcessorBasic(t *testing.T) { require.NotNil(t, <-r1ErrC) // Stop the processor with an error. - pErr := roachpb.NewErrorf("stop err") + pErr := kvpb.NewErrorf("stop err") p.StopWithErr(pErr) require.NotNil(t, <-r2ErrC) // Adding another registration should fail. r3Stream := newTestStream() - r3ErrC := make(chan *roachpb.Error, 1) + r3ErrC := make(chan *kvpb.Error, 1) r3OK, _ := p.Register( roachpb.RSpan{Key: roachpb.RKey("c"), EndKey: roachpb.RKey("z")}, hlc.Timestamp{WallTime: 1}, @@ -449,7 +450,7 @@ func TestProcessorSlowConsumer(t *testing.T) { // Add a registration. r1Stream := newTestStream() - r1ErrC := make(chan *roachpb.Error, 1) + r1ErrC := make(chan *kvpb.Error, 1) p.Register( roachpb.RSpan{Key: roachpb.RKey("a"), EndKey: roachpb.RKey("m")}, hlc.Timestamp{WallTime: 1}, @@ -459,7 +460,7 @@ func TestProcessorSlowConsumer(t *testing.T) { r1ErrC, ) r2Stream := newTestStream() - r2ErrC := make(chan *roachpb.Error, 1) + r2ErrC := make(chan *kvpb.Error, 1) p.Register( roachpb.RSpan{Key: roachpb.RKey("a"), EndKey: roachpb.RKey("z")}, hlc.Timestamp{WallTime: 1}, @@ -471,14 +472,14 @@ func TestProcessorSlowConsumer(t *testing.T) { p.syncEventAndRegistrations() require.Equal(t, 2, p.Len()) require.Equal(t, - []*roachpb.RangeFeedEvent{rangeFeedCheckpoint( + []*kvpb.RangeFeedEvent{rangeFeedCheckpoint( roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("m")}, hlc.Timestamp{WallTime: 0}, )}, r1Stream.Events(), ) require.Equal(t, - []*roachpb.RangeFeedEvent{rangeFeedCheckpoint( + []*kvpb.RangeFeedEvent{rangeFeedCheckpoint( roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("z")}, hlc.Timestamp{WallTime: 0}, )}, @@ -561,7 +562,7 @@ func TestProcessorMemoryBudgetExceeded(t *testing.T) { // Add a registration. r1Stream := newTestStream() - r1ErrC := make(chan *roachpb.Error, 1) + r1ErrC := make(chan *kvpb.Error, 1) p.Register( roachpb.RSpan{Key: roachpb.RKey("a"), EndKey: roachpb.RKey("m")}, hlc.Timestamp{WallTime: 1}, @@ -630,7 +631,7 @@ func TestProcessorMemoryBudgetReleased(t *testing.T) { // Add a registration. r1Stream := newTestStream() - r1ErrC := make(chan *roachpb.Error, 1) + r1ErrC := make(chan *kvpb.Error, 1) p.Register( roachpb.RSpan{Key: roachpb.RKey("a"), EndKey: roachpb.RKey("m")}, hlc.Timestamp{WallTime: 1}, @@ -710,14 +711,14 @@ func TestProcessorInitializeResolvedTimestamp(t *testing.T) { nil, /* catchUpIter */ false, /* withDiff */ r1Stream, - make(chan *roachpb.Error, 1), + make(chan *kvpb.Error, 1), ) p.syncEventAndRegistrations() require.Equal(t, 1, p.Len()) // The registration should be provided a checkpoint immediately with an // empty resolved timestamp because it did not perform a catch-up scan. - chEvent := []*roachpb.RangeFeedEvent{rangeFeedCheckpoint( + chEvent := []*kvpb.RangeFeedEvent{rangeFeedCheckpoint( roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("m")}, hlc.Timestamp{}, )} @@ -748,7 +749,7 @@ func TestProcessorInitializeResolvedTimestamp(t *testing.T) { require.Equal(t, hlc.Timestamp{WallTime: 18}, p.rts.Get()) // The registration should have been informed of the new resolved timestamp. - chEvent = []*roachpb.RangeFeedEvent{rangeFeedCheckpoint( + chEvent = []*kvpb.RangeFeedEvent{rangeFeedCheckpoint( roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("m")}, hlc.Timestamp{WallTime: 18}, )} @@ -951,7 +952,7 @@ func TestProcessorConcurrentStop(t *testing.T) { defer wg.Done() runtime.Gosched() s := newTestStream() - errC := make(chan<- *roachpb.Error, 1) + errC := make(chan<- *kvpb.Error, 1) p.Register(p.Span, hlc.Timestamp{}, nil, false, s, errC) }() go func() { @@ -1020,7 +1021,7 @@ func TestProcessorRegistrationObservesOnlyNewEvents(t *testing.T) { // operation is should see is firstIdx. s := newTestStream() regs[s] = firstIdx - errC := make(chan *roachpb.Error, 1) + errC := make(chan *kvpb.Error, 1) p.Register(p.Span, hlc.Timestamp{}, nil, false, s, errC) regDone <- struct{}{} } @@ -1032,10 +1033,10 @@ func TestProcessorRegistrationObservesOnlyNewEvents(t *testing.T) { // from before they registered. for s, expFirstIdx := range regs { events := s.Events() - require.IsType(t, &roachpb.RangeFeedCheckpoint{}, events[0].GetValue()) - require.IsType(t, &roachpb.RangeFeedValue{}, events[1].GetValue()) + require.IsType(t, &kvpb.RangeFeedCheckpoint{}, events[0].GetValue()) + require.IsType(t, &kvpb.RangeFeedValue{}, events[1].GetValue()) - firstVal := events[1].GetValue().(*roachpb.RangeFeedValue) + firstVal := events[1].GetValue().(*kvpb.RangeFeedValue) firstIdx := firstVal.Value.Timestamp.WallTime require.Equal(t, expFirstIdx, firstIdx) } @@ -1105,7 +1106,7 @@ func TestBudgetReleaseOnProcessorStop(t *testing.T) { // Add a registration. rStream := newConsumer(50) defer func() { rStream.Resume() }() - rErrC := make(chan *roachpb.Error, 1) + rErrC := make(chan *kvpb.Error, 1) p.Register( roachpb.RSpan{Key: roachpb.RKey("a"), EndKey: roachpb.RKey("m")}, hlc.Timestamp{WallTime: 1}, @@ -1193,7 +1194,7 @@ func TestBudgetReleaseOnLastStreamError(t *testing.T) { // Add a registration. rStream := newConsumer(90) defer func() { rStream.Resume() }() - rErrC := make(chan *roachpb.Error, 1) + rErrC := make(chan *kvpb.Error, 1) p.Register( roachpb.RSpan{Key: roachpb.RKey("a"), EndKey: roachpb.RKey("m")}, hlc.Timestamp{WallTime: 1}, @@ -1270,7 +1271,7 @@ func TestBudgetReleaseOnOneStreamError(t *testing.T) { // Add a registration. r1Stream := newConsumer(50) defer func() { r1Stream.Resume() }() - r1ErrC := make(chan *roachpb.Error, 1) + r1ErrC := make(chan *kvpb.Error, 1) p.Register( roachpb.RSpan{Key: roachpb.RKey("a"), EndKey: roachpb.RKey("m")}, hlc.Timestamp{WallTime: 1}, @@ -1281,7 +1282,7 @@ func TestBudgetReleaseOnOneStreamError(t *testing.T) { ) // Non-blocking registration that would consume all events. r2Stream := newConsumer(0) - r2ErrC := make(chan *roachpb.Error, 1) + r2ErrC := make(chan *kvpb.Error, 1) p.Register( roachpb.RSpan{Key: roachpb.RKey("a"), EndKey: roachpb.RKey("m")}, hlc.Timestamp{WallTime: 1}, @@ -1357,7 +1358,7 @@ func newConsumer(blockAfter int) *consumer { } } -func (c *consumer) Send(e *roachpb.RangeFeedEvent) error { +func (c *consumer) Send(e *kvpb.RangeFeedEvent) error { //fmt.Printf("Stream received event %v\n", e) if e.Val != nil { v := int(atomic.AddInt32(&c.sentValues, 1)) @@ -1438,7 +1439,7 @@ func BenchmarkProcessorWithBudget(b *testing.B) { // Add a registration. r1Stream := newTestStream() - r1ErrC := make(chan *roachpb.Error, 1) + r1ErrC := make(chan *kvpb.Error, 1) p.Register( roachpb.RSpan{Key: roachpb.RKey("a"), EndKey: roachpb.RKey("m")}, hlc.Timestamp{WallTime: 1}, diff --git a/pkg/kv/kvserver/rangefeed/registry.go b/pkg/kv/kvserver/rangefeed/registry.go index db255c91f3ea..ae2adf261c32 100644 --- a/pkg/kv/kvserver/rangefeed/registry.go +++ b/pkg/kv/kvserver/rangefeed/registry.go @@ -16,6 +16,7 @@ import ( "sync" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/interval" @@ -32,7 +33,7 @@ type Stream interface { Context() context.Context // Send blocks until it sends m, the stream is done, or the stream breaks. // Send must be safe to call on the same stream in different goroutines. - Send(*roachpb.RangeFeedEvent) error + Send(*kvpb.RangeFeedEvent) error } // Shared event is an entry stored in registration channel. Each entry is @@ -40,7 +41,7 @@ type Stream interface { // to track memory budgets. event itself could either be shared or not in case // we optimized unused fields in it based on registration options. type sharedEvent struct { - event *roachpb.RangeFeedEvent + event *kvpb.RangeFeedEvent alloc *SharedBudgetAllocation } @@ -90,7 +91,7 @@ type registration struct { // Output. stream Stream - errC chan<- *roachpb.Error + errC chan<- *kvpb.Error // Internal. id int64 @@ -125,7 +126,7 @@ func newRegistration( bufferSz int, metrics *Metrics, stream Stream, - errC chan<- *roachpb.Error, + errC chan<- *kvpb.Error, ) registration { r := registration{ span: span, @@ -148,7 +149,7 @@ func newRegistration( // If overflowed is already set, events are ignored and not written to the // buffer. func (r *registration) publish( - ctx context.Context, event *roachpb.RangeFeedEvent, alloc *SharedBudgetAllocation, + ctx context.Context, event *kvpb.RangeFeedEvent, alloc *SharedBudgetAllocation, ) { r.validateEvent(event) e := getPooledSharedEvent(sharedEvent{event: r.maybeStripEvent(event), alloc: alloc}) @@ -172,9 +173,9 @@ func (r *registration) publish( // validateEvent checks that the event contains enough information for the // registation. -func (r *registration) validateEvent(event *roachpb.RangeFeedEvent) { +func (r *registration) validateEvent(event *kvpb.RangeFeedEvent) { switch t := event.GetValue().(type) { - case *roachpb.RangeFeedValue: + case *kvpb.RangeFeedValue: if t.Key == nil { panic(fmt.Sprintf("unexpected empty RangeFeedValue.Key: %v", t)) } @@ -184,11 +185,11 @@ func (r *registration) validateEvent(event *roachpb.RangeFeedEvent) { if t.Value.Timestamp.IsEmpty() { panic(fmt.Sprintf("unexpected empty RangeFeedValue.Value.Timestamp: %v", t)) } - case *roachpb.RangeFeedCheckpoint: + case *kvpb.RangeFeedCheckpoint: if t.Span.Key == nil { panic(fmt.Sprintf("unexpected empty RangeFeedCheckpoint.Span.Key: %v", t)) } - case *roachpb.RangeFeedSSTable: + case *kvpb.RangeFeedSSTable: if len(t.Data) == 0 { panic(fmt.Sprintf("unexpected empty RangeFeedSSTable.Data: %v", t)) } @@ -198,7 +199,7 @@ func (r *registration) validateEvent(event *roachpb.RangeFeedEvent) { if t.WriteTS.IsEmpty() { panic(fmt.Sprintf("unexpected empty RangeFeedSSTable.Timestamp: %v", t)) } - case *roachpb.RangeFeedDeleteRange: + case *kvpb.RangeFeedDeleteRange: if len(t.Span.Key) == 0 || len(t.Span.EndKey) == 0 { panic(fmt.Sprintf("unexpected empty key in RangeFeedDeleteRange.Span: %v", t)) } @@ -214,7 +215,7 @@ func (r *registration) validateEvent(event *roachpb.RangeFeedEvent) { // applicable to the current registration. If so, it makes a copy of the event // and strips the incompatible information to match only what the registration // requested. -func (r *registration) maybeStripEvent(event *roachpb.RangeFeedEvent) *roachpb.RangeFeedEvent { +func (r *registration) maybeStripEvent(event *kvpb.RangeFeedEvent) *kvpb.RangeFeedEvent { ret := event copyOnWrite := func() interface{} { if ret == event { @@ -224,7 +225,7 @@ func (r *registration) maybeStripEvent(event *roachpb.RangeFeedEvent) *roachpb.R } switch t := ret.GetValue().(type) { - case *roachpb.RangeFeedValue: + case *kvpb.RangeFeedValue: if t.PrevValue.IsPresent() && !r.withDiff { // If no registrations for the current Range are requesting previous // values, then we won't even retrieve them on the Raft goroutine. @@ -232,10 +233,10 @@ func (r *registration) maybeStripEvent(event *roachpb.RangeFeedEvent) *roachpb.R // previous value on the corresponding events will be populated. // If we're in this case and any other registrations don't want // previous values then we'll need to strip them. - t = copyOnWrite().(*roachpb.RangeFeedValue) + t = copyOnWrite().(*kvpb.RangeFeedValue) t.PrevValue = roachpb.Value{} } - case *roachpb.RangeFeedCheckpoint: + case *kvpb.RangeFeedCheckpoint: if !t.Span.EqualValue(r.span) { // Checkpoint events are always created spanning the entire Range. // However, a registration might not be listening on updates over @@ -248,16 +249,16 @@ func (r *registration) maybeStripEvent(event *roachpb.RangeFeedEvent) *roachpb.R if !t.Span.Contains(r.span) { panic(fmt.Sprintf("registration span %v larger than checkpoint span %v", r.span, t.Span)) } - t = copyOnWrite().(*roachpb.RangeFeedCheckpoint) + t = copyOnWrite().(*kvpb.RangeFeedCheckpoint) t.Span = r.span } - case *roachpb.RangeFeedDeleteRange: + case *kvpb.RangeFeedDeleteRange: // Truncate the range tombstone to the registration bounds. if i := t.Span.Intersect(r.span); !i.Equal(t.Span) { - t = copyOnWrite().(*roachpb.RangeFeedDeleteRange) + t = copyOnWrite().(*kvpb.RangeFeedDeleteRange) t.Span = i.Clone() } - case *roachpb.RangeFeedSSTable: + case *kvpb.RangeFeedSSTable: // SSTs are always sent in their entirety, it is up to the caller to // filter out irrelevant entries. default: @@ -270,7 +271,7 @@ func (r *registration) maybeStripEvent(event *roachpb.RangeFeedEvent) *roachpb.R // error to the output error stream for the registration. This also sets the // disconnected flag on the registration, preventing it from being disconnected // again. -func (r *registration) disconnect(pErr *roachpb.Error) { +func (r *registration) disconnect(pErr *kvpb.Error) { r.mu.Lock() defer r.mu.Unlock() if !r.mu.disconnected { @@ -345,7 +346,7 @@ func (r *registration) runOutputLoop(ctx context.Context, _forStacks roachpb.Ran ctx, r.mu.outputLoopCancelFn = context.WithCancel(ctx) r.mu.Unlock() err := r.outputLoop(ctx) - r.disconnect(roachpb.NewError(err)) + r.disconnect(kvpb.NewError(err)) } // drainAllocations should be done after registration is disconnected from @@ -440,22 +441,19 @@ func (reg *registry) nextID() int64 { // PublishToOverlapping publishes the provided event to all registrations whose // range overlaps the specified span. func (reg *registry) PublishToOverlapping( - ctx context.Context, - span roachpb.Span, - event *roachpb.RangeFeedEvent, - alloc *SharedBudgetAllocation, + ctx context.Context, span roachpb.Span, event *kvpb.RangeFeedEvent, alloc *SharedBudgetAllocation, ) { // Determine the earliest starting timestamp that a registration // can have while still needing to hear about this event. var minTS hlc.Timestamp switch t := event.GetValue().(type) { - case *roachpb.RangeFeedValue: + case *kvpb.RangeFeedValue: minTS = t.Value.Timestamp - case *roachpb.RangeFeedSSTable: + case *kvpb.RangeFeedSSTable: minTS = t.WriteTS - case *roachpb.RangeFeedDeleteRange: + case *kvpb.RangeFeedDeleteRange: minTS = t.Timestamp - case *roachpb.RangeFeedCheckpoint: + case *kvpb.RangeFeedCheckpoint: // Always publish checkpoint notifications, regardless of a registration's // starting timestamp. // @@ -466,7 +464,7 @@ func (reg *registry) PublishToOverlapping( panic(fmt.Sprintf("unexpected RangeFeedEvent variant: %v", t)) } - reg.forOverlappingRegs(span, func(r *registration) (bool, *roachpb.Error) { + reg.forOverlappingRegs(span, func(r *registration) (bool, *kvpb.Error) { // Don't publish events if they are equal to or less // than the registration's starting timestamp. if r.catchUpTimestamp.Less(minTS) { @@ -497,8 +495,8 @@ func (reg *registry) Disconnect(span roachpb.Span) { // DisconnectWithErr disconnects all registrations that overlap the specified // span with the provided error. -func (reg *registry) DisconnectWithErr(span roachpb.Span, pErr *roachpb.Error) { - reg.forOverlappingRegs(span, func(_ *registration) (bool, *roachpb.Error) { +func (reg *registry) DisconnectWithErr(span roachpb.Span, pErr *kvpb.Error) { + reg.forOverlappingRegs(span, func(_ *registration) (bool, *kvpb.Error) { return true, pErr }) } @@ -511,7 +509,7 @@ var all = roachpb.Span{Key: roachpb.KeyMin, EndKey: roachpb.KeyMax} // then that registration is unregistered and the error returned by the // function is send on its corresponding error channel. func (reg *registry) forOverlappingRegs( - span roachpb.Span, fn func(*registration) (disconnect bool, pErr *roachpb.Error), + span roachpb.Span, fn func(*registration) (disconnect bool, pErr *kvpb.Error), ) { var toDelete []interval.Interface matchFn := func(i interval.Interface) (done bool) { @@ -592,7 +590,7 @@ func (r *registration) detachCatchUpIter() *CatchUpIterator { // completely process their internal buffers. func (reg *registry) waitForCaughtUp(span roachpb.Span) error { var outerErr error - reg.forOverlappingRegs(span, func(r *registration) (bool, *roachpb.Error) { + reg.forOverlappingRegs(span, func(r *registration) (bool, *kvpb.Error) { if outerErr == nil { outerErr = r.waitForCaughtUp() } diff --git a/pkg/kv/kvserver/rangefeed/registry_test.go b/pkg/kv/kvserver/rangefeed/registry_test.go index 43a148e41eca..8821f402cb8f 100644 --- a/pkg/kv/kvserver/rangefeed/registry_test.go +++ b/pkg/kv/kvserver/rangefeed/registry_test.go @@ -16,6 +16,7 @@ import ( "testing" _ "github.com/cockroachdb/cockroach/pkg/keys" // hook up pretty printer + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -43,7 +44,7 @@ type testStream struct { mu struct { syncutil.Mutex sendErr error - events []*roachpb.RangeFeedEvent + events []*kvpb.RangeFeedEvent } } @@ -60,7 +61,7 @@ func (s *testStream) Cancel() { s.ctxDone() } -func (s *testStream) Send(e *roachpb.RangeFeedEvent) error { +func (s *testStream) Send(e *kvpb.RangeFeedEvent) error { s.mu.Lock() defer s.mu.Unlock() if s.mu.sendErr != nil { @@ -76,7 +77,7 @@ func (s *testStream) SetSendErr(err error) { s.mu.sendErr = err } -func (s *testStream) Events() []*roachpb.RangeFeedEvent { +func (s *testStream) Events() []*kvpb.RangeFeedEvent { s.mu.Lock() defer s.mu.Unlock() es := s.mu.events @@ -92,7 +93,7 @@ func (s *testStream) BlockSend() func() { type testRegistration struct { registration stream *testStream - errC <-chan *roachpb.Error + errC <-chan *kvpb.Error } func makeCatchUpIteratorConstructor(iter storage.SimpleMVCCIterator) CatchUpIteratorConstructor { @@ -112,7 +113,7 @@ func newTestRegistration( span roachpb.Span, ts hlc.Timestamp, catchup storage.SimpleMVCCIterator, withDiff bool, ) *testRegistration { s := newTestStream() - errC := make(chan *roachpb.Error, 1) + errC := make(chan *kvpb.Error, 1) r := newRegistration( span, @@ -132,11 +133,11 @@ func newTestRegistration( } } -func (r *testRegistration) Events() []*roachpb.RangeFeedEvent { +func (r *testRegistration) Events() []*kvpb.RangeFeedEvent { return r.stream.Events() } -func (r *testRegistration) Err() *roachpb.Error { +func (r *testRegistration) Err() *kvpb.Error { select { case pErr := <-r.errC: return pErr @@ -150,9 +151,9 @@ func TestRegistrationBasic(t *testing.T) { ctx := context.Background() val := roachpb.Value{RawBytes: []byte("val"), Timestamp: hlc.Timestamp{WallTime: 1}} - ev1, ev2 := new(roachpb.RangeFeedEvent), new(roachpb.RangeFeedEvent) - ev1.MustSetValue(&roachpb.RangeFeedValue{Key: keyA, Value: val}) - ev2.MustSetValue(&roachpb.RangeFeedValue{Key: keyB, Value: val}) + ev1, ev2 := new(kvpb.RangeFeedEvent), new(kvpb.RangeFeedEvent) + ev1.MustSetValue(&kvpb.RangeFeedValue{Key: keyA, Value: val}) + ev2.MustSetValue(&kvpb.RangeFeedValue{Key: keyB, Value: val}) // Registration with no catchup scan specified. noCatchupReg := newTestRegistration(spAB, hlc.Timestamp{}, nil, false) @@ -161,7 +162,7 @@ func TestRegistrationBasic(t *testing.T) { require.Equal(t, len(noCatchupReg.buf), 2) go noCatchupReg.runOutputLoop(context.Background(), 0) require.NoError(t, noCatchupReg.waitForCaughtUp()) - require.Equal(t, []*roachpb.RangeFeedEvent{ev1, ev2}, noCatchupReg.stream.Events()) + require.Equal(t, []*kvpb.RangeFeedEvent{ev1, ev2}, noCatchupReg.stream.Events()) noCatchupReg.disconnect(nil) <-noCatchupReg.errC @@ -179,7 +180,7 @@ func TestRegistrationBasic(t *testing.T) { require.NoError(t, catchupReg.waitForCaughtUp()) events := catchupReg.stream.Events() require.Equal(t, 5, len(events)) - require.Equal(t, []*roachpb.RangeFeedEvent{ev1, ev2}, events[3:]) + require.Equal(t, []*kvpb.RangeFeedEvent{ev1, ev2}, events[3:]) catchupReg.disconnect(nil) <-catchupReg.errC @@ -190,7 +191,7 @@ func TestRegistrationBasic(t *testing.T) { disconnectReg.publish(ctx, ev2, nil /* alloc */) go disconnectReg.runOutputLoop(context.Background(), 0) require.NoError(t, disconnectReg.waitForCaughtUp()) - discErr := roachpb.NewError(fmt.Errorf("disconnection error")) + discErr := kvpb.NewError(fmt.Errorf("disconnection error")) disconnectReg.disconnect(discErr) err := <-disconnectReg.errC require.Equal(t, discErr, err) @@ -283,7 +284,7 @@ func TestRegistrationCatchUpScan(t *testing.T) { require.NotZero(t, r.metrics.RangeFeedCatchUpScanNanos.Count()) // Compare the events sent on the registration's Stream to the expected events. - expEvents := []*roachpb.RangeFeedEvent{ + expEvents := []*kvpb.RangeFeedEvent{ rangeFeedValueWithPrev( roachpb.Key("d"), makeValWithTs("valD3", 16), @@ -336,16 +337,16 @@ func TestRegistryBasic(t *testing.T) { ctx := context.Background() val := roachpb.Value{RawBytes: []byte("val"), Timestamp: hlc.Timestamp{WallTime: 1}} - ev1, ev2 := new(roachpb.RangeFeedEvent), new(roachpb.RangeFeedEvent) - ev3, ev4 := new(roachpb.RangeFeedEvent), new(roachpb.RangeFeedEvent) - ev1.MustSetValue(&roachpb.RangeFeedValue{Key: keyA, Value: val, PrevValue: val}) - ev2.MustSetValue(&roachpb.RangeFeedValue{Key: keyB, Value: val, PrevValue: val}) - ev3.MustSetValue(&roachpb.RangeFeedValue{Key: keyC, Value: val, PrevValue: val}) - ev4.MustSetValue(&roachpb.RangeFeedValue{Key: keyD, Value: val, PrevValue: val}) - err1 := roachpb.NewErrorf("error1") - noPrev := func(ev *roachpb.RangeFeedEvent) *roachpb.RangeFeedEvent { + ev1, ev2 := new(kvpb.RangeFeedEvent), new(kvpb.RangeFeedEvent) + ev3, ev4 := new(kvpb.RangeFeedEvent), new(kvpb.RangeFeedEvent) + ev1.MustSetValue(&kvpb.RangeFeedValue{Key: keyA, Value: val, PrevValue: val}) + ev2.MustSetValue(&kvpb.RangeFeedValue{Key: keyB, Value: val, PrevValue: val}) + ev3.MustSetValue(&kvpb.RangeFeedValue{Key: keyC, Value: val, PrevValue: val}) + ev4.MustSetValue(&kvpb.RangeFeedValue{Key: keyD, Value: val, PrevValue: val}) + err1 := kvpb.NewErrorf("error1") + noPrev := func(ev *kvpb.RangeFeedEvent) *kvpb.RangeFeedEvent { ev = ev.ShallowCopy() - ev.GetValue().(*roachpb.RangeFeedValue).PrevValue = roachpb.Value{} + ev.GetValue().(*kvpb.RangeFeedValue).PrevValue = roachpb.Value{} return ev } @@ -384,10 +385,10 @@ func TestRegistryBasic(t *testing.T) { reg.PublishToOverlapping(ctx, spCD, ev3, nil /* alloc */) reg.PublishToOverlapping(ctx, spAC, ev4, nil /* alloc */) require.NoError(t, reg.waitForCaughtUp(all)) - require.Equal(t, []*roachpb.RangeFeedEvent{noPrev(ev1), noPrev(ev4)}, rAB.Events()) - require.Equal(t, []*roachpb.RangeFeedEvent{ev2, ev4}, rBC.Events()) - require.Equal(t, []*roachpb.RangeFeedEvent{ev3}, rCD.Events()) - require.Equal(t, []*roachpb.RangeFeedEvent{noPrev(ev1), noPrev(ev2), noPrev(ev4)}, rAC.Events()) + require.Equal(t, []*kvpb.RangeFeedEvent{noPrev(ev1), noPrev(ev4)}, rAB.Events()) + require.Equal(t, []*kvpb.RangeFeedEvent{ev2, ev4}, rBC.Events()) + require.Equal(t, []*kvpb.RangeFeedEvent{ev3}, rCD.Events()) + require.Equal(t, []*kvpb.RangeFeedEvent{noPrev(ev1), noPrev(ev2), noPrev(ev4)}, rAC.Events()) require.Nil(t, rAB.Err()) require.Nil(t, rBC.Err()) require.Nil(t, rCD.Err()) @@ -427,7 +428,7 @@ func TestRegistryBasic(t *testing.T) { reg.PublishToOverlapping(ctx, spCD, ev2, nil /* alloc */) reg.PublishToOverlapping(ctx, spAC, ev1, nil /* alloc */) require.NoError(t, reg.waitForCaughtUp(all)) - require.Equal(t, []*roachpb.RangeFeedEvent{noPrev(ev4), noPrev(ev1)}, rAB.Events()) + require.Equal(t, []*kvpb.RangeFeedEvent{noPrev(ev4), noPrev(ev1)}, rAB.Events()) // Disconnect from rAB without error. reg.Disconnect(spAB) @@ -479,10 +480,10 @@ func TestRegistryPublishAssertsPopulatedInformation(t *testing.T) { key := roachpb.Key("a") val := roachpb.Value{RawBytes: []byte("val"), Timestamp: hlc.Timestamp{WallTime: 1}} noVal := roachpb.Value{Timestamp: hlc.Timestamp{WallTime: 1}} - ev := new(roachpb.RangeFeedEvent) + ev := new(kvpb.RangeFeedEvent) // Both registrations require RangeFeedValue events to have a Key. - ev.MustSetValue(&roachpb.RangeFeedValue{ + ev.MustSetValue(&kvpb.RangeFeedValue{ Key: nil, Value: val, PrevValue: val, @@ -492,7 +493,7 @@ func TestRegistryPublishAssertsPopulatedInformation(t *testing.T) { require.NoError(t, reg.waitForCaughtUp(all)) // Both registrations require RangeFeedValue events to have a Value. - ev.MustSetValue(&roachpb.RangeFeedValue{ + ev.MustSetValue(&kvpb.RangeFeedValue{ Key: key, Value: noVal, PrevValue: val, @@ -503,7 +504,7 @@ func TestRegistryPublishAssertsPopulatedInformation(t *testing.T) { // Neither registrations require RangeFeedValue events to have a PrevValue. // Even when they are requested, the previous value can always be nil. - ev.MustSetValue(&roachpb.RangeFeedValue{ + ev.MustSetValue(&kvpb.RangeFeedValue{ Key: key, Value: val, PrevValue: roachpb.Value{}, @@ -527,8 +528,8 @@ func TestRegistryPublishBeneathStartTimestamp(t *testing.T) { // Publish a value with a timestamp beneath the registration's start // timestamp. Should be ignored. - ev := new(roachpb.RangeFeedEvent) - ev.MustSetValue(&roachpb.RangeFeedValue{ + ev := new(kvpb.RangeFeedEvent) + ev.MustSetValue(&kvpb.RangeFeedValue{ Value: roachpb.Value{Timestamp: hlc.Timestamp{WallTime: 5}}, }) reg.PublishToOverlapping(ctx, spAB, ev, nil /* alloc */) @@ -537,7 +538,7 @@ func TestRegistryPublishBeneathStartTimestamp(t *testing.T) { // Publish a value with a timestamp equal to the registration's start // timestamp. Should be ignored. - ev.MustSetValue(&roachpb.RangeFeedValue{ + ev.MustSetValue(&kvpb.RangeFeedValue{ Value: roachpb.Value{Timestamp: hlc.Timestamp{WallTime: 10}}, }) reg.PublishToOverlapping(ctx, spAB, ev, nil /* alloc */) @@ -546,12 +547,12 @@ func TestRegistryPublishBeneathStartTimestamp(t *testing.T) { // Publish a checkpoint with a timestamp beneath the registration's. Should // be delivered. - ev.MustSetValue(&roachpb.RangeFeedCheckpoint{ + ev.MustSetValue(&kvpb.RangeFeedCheckpoint{ Span: spAB, ResolvedTS: hlc.Timestamp{WallTime: 5}, }) reg.PublishToOverlapping(ctx, spAB, ev, nil /* alloc */) require.NoError(t, reg.waitForCaughtUp(all)) - require.Equal(t, []*roachpb.RangeFeedEvent{ev}, r.Events()) + require.Equal(t, []*kvpb.RangeFeedEvent{ev}, r.Events()) r.disconnect(nil) <-r.errC diff --git a/pkg/kv/kvserver/rangefeed/task.go b/pkg/kv/kvserver/rangefeed/task.go index 0a1ea4f827c7..2b0fad18a659 100644 --- a/pkg/kv/kvserver/rangefeed/task.go +++ b/pkg/kv/kvserver/rangefeed/task.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" @@ -51,7 +52,7 @@ func (s *initResolvedTSScan) Run(ctx context.Context) { if err := s.iterateAndConsume(ctx); err != nil { err = errors.Wrap(err, "initial resolved timestamp scan failed") log.Errorf(ctx, "%v", err) - s.p.StopWithErr(roachpb.NewError(err)) + s.p.StopWithErr(kvpb.NewError(err)) } else { // Inform the processor that its resolved timestamp can be initialized. s.p.setResolvedTSInitialized(ctx) diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 8b56287695ca..a523d1985f59 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/docs" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/abortspan" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" @@ -1237,7 +1238,7 @@ func (r *Replica) getReplicaDescriptorRLocked() (roachpb.ReplicaDescriptor, erro if ok { return repDesc, nil } - return roachpb.ReplicaDescriptor{}, roachpb.NewRangeNotFoundError(r.RangeID, r.store.StoreID()) + return roachpb.ReplicaDescriptor{}, kvpb.NewRangeNotFoundError(r.RangeID, r.store.StoreID()) } func (r *Replica) getMergeCompleteCh() chan struct{} { @@ -1580,7 +1581,7 @@ func (r *Replica) assertStateRaftMuLockedReplicaMuRLocked( // might be ok with this if they know that they will end up checking for a // pending merge at some later time. func (r *Replica) checkExecutionCanProceedBeforeStorageSnapshot( - ctx context.Context, ba *roachpb.BatchRequest, g *concurrency.Guard, + ctx context.Context, ba *kvpb.BatchRequest, g *concurrency.Guard, ) (kvserverpb.LeaseStatus, error) { rSpan, err := keys.Range(ba.Requests) if err != nil { @@ -1658,7 +1659,7 @@ func (r *Replica) checkExecutionCanProceedBeforeStorageSnapshot( // iterator. An error indicates that the request's timestamp is below the // Replica's GC threshold. func (r *Replica) checkExecutionCanProceedAfterStorageSnapshot( - ctx context.Context, ba *roachpb.BatchRequest, st kvserverpb.LeaseStatus, + ctx context.Context, ba *kvpb.BatchRequest, st kvserverpb.LeaseStatus, ) error { rSpan, err := keys.Range(ba.Requests) if err != nil { @@ -1697,7 +1698,7 @@ func (r *Replica) checkExecutionCanProceedAfterStorageSnapshot( // checkExecutionCanProceedRWOrAdmin returns an error if a batch request going // through the RW or admin paths cannot be executed by the Replica. func (r *Replica) checkExecutionCanProceedRWOrAdmin( - ctx context.Context, ba *roachpb.BatchRequest, g *concurrency.Guard, + ctx context.Context, ba *kvpb.BatchRequest, g *concurrency.Guard, ) (kvserverpb.LeaseStatus, error) { st, err := r.checkExecutionCanProceedBeforeStorageSnapshot(ctx, ba, g) if err != nil { @@ -1716,7 +1717,7 @@ func (r *Replica) checkExecutionCanProceedRWOrAdmin( // returned bool indicates whether the lease should be extended (only on nil // error). func (r *Replica) checkLeaseRLocked( - ctx context.Context, ba *roachpb.BatchRequest, + ctx context.Context, ba *kvpb.BatchRequest, ) (kvserverpb.LeaseStatus, bool, error) { now := r.Clock().NowAsClockTimestamp() // If the request is a write or a consistent read, it requires the @@ -1738,7 +1739,7 @@ func (r *Replica) checkLeaseRLocked( // // If the request is an INCONSISTENT request (and thus a read), it similarly // doesn't check the lease. - if !ba.IsSingleSkipsLeaseCheckRequest() && ba.ReadConsistency != roachpb.INCONSISTENT { + if !ba.IsSingleSkipsLeaseCheckRequest() && ba.ReadConsistency != kvpb.INCONSISTENT { // Check the lease. var err error shouldExtend, err = r.leaseGoodToGoForStatusRLocked(ctx, now, reqTS, st) @@ -1791,7 +1792,7 @@ func (r *Replica) checkSpanInRangeRLocked(ctx context.Context, rspan roachpb.RSp if desc.ContainsKeyRange(rspan.Key, rspan.EndKey) { return nil } - return roachpb.NewRangeKeyMismatchErrorWithCTPolicy( + return kvpb.NewRangeKeyMismatchErrorWithCTPolicy( ctx, rspan.Key.AsRawKey(), rspan.EndKey.AsRawKey(), desc, r.mu.state.Lease, r.closedTimestampPolicyRLocked()) } @@ -1804,7 +1805,7 @@ func (r *Replica) checkTSAboveGCThresholdRLocked( if threshold.Less(ts) { return nil } - return &roachpb.BatchTimestampBeforeGCError{ + return &kvpb.BatchTimestampBeforeGCError{ Timestamp: ts, Threshold: threshold, DataExcludedFromBackup: r.excludeReplicaFromBackupRLocked(), @@ -1816,7 +1817,7 @@ func (r *Replica) checkTSAboveGCThresholdRLocked( // If not, an error is returned to prevent the request from proceeding until the // merge completes. func (r *Replica) shouldWaitForPendingMergeRLocked( - ctx context.Context, ba *roachpb.BatchRequest, + ctx context.Context, ba *kvpb.BatchRequest, ) error { if !r.mergeInProgressRLocked() { log.Fatal(ctx, "programming error: shouldWaitForPendingMergeRLocked should"+ @@ -1916,7 +1917,7 @@ func (r *Replica) shouldWaitForPendingMergeRLocked( // merge. Instead, we release the latches we acquired above and return a // MergeInProgressError. The store will catch that error and resubmit the // request after mergeCompleteCh closes. See #27442 for the full context. - return &roachpb.MergeInProgressError{} + return &kvpb.MergeInProgressError{} } // isNewerThanSplit is a helper used in split(Pre|Post)Apply to @@ -2032,23 +2033,23 @@ func (r *Replica) maybeWatchForMergeLocked(ctx context.Context) (bool, error) { taskCtx := r.AnnotateCtx(context.Background()) err = r.store.stopper.RunAsyncTask(taskCtx, "wait-for-merge", func(ctx context.Context) { - var pushTxnRes *roachpb.PushTxnResponse + var pushTxnRes *kvpb.PushTxnResponse for retry := retry.Start(base.DefaultRetryOptions()); retry.Next(); { // Wait for the merge transaction to complete by attempting to push it. We // don't want to accidentally abort the merge transaction, so we use the // minimum transaction priority. Note that a push type of - // roachpb.PUSH_TOUCH, though it might appear more semantically correct, + // kvpb.PUSH_TOUCH, though it might appear more semantically correct, // returns immediately and causes us to spin hot, whereas - // roachpb.PUSH_ABORT efficiently blocks until the transaction completes. + // kvpb.PUSH_ABORT efficiently blocks until the transaction completes. b := &kv.Batch{} b.Header.Timestamp = r.Clock().Now() - b.AddRawRequest(&roachpb.PushTxnRequest{ - RequestHeader: roachpb.RequestHeader{Key: intentRes.Intent.Txn.Key}, + b.AddRawRequest(&kvpb.PushTxnRequest{ + RequestHeader: kvpb.RequestHeader{Key: intentRes.Intent.Txn.Key}, PusherTxn: roachpb.Transaction{ TxnMeta: enginepb.TxnMeta{Priority: enginepb.MinTxnPriority}, }, PusheeTxn: intentRes.Intent.Txn, - PushType: roachpb.PUSH_ABORT, + PushType: kvpb.PUSH_ABORT, }) if err := r.store.DB().Run(ctx, b); err != nil { select { @@ -2063,7 +2064,7 @@ func (r *Replica) maybeWatchForMergeLocked(ctx context.Context) (bool, error) { continue } } - pushTxnRes = b.RawResponse().Responses[0].GetInner().(*roachpb.PushTxnResponse) + pushTxnRes = b.RawResponse().Responses[0].GetInner().(*kvpb.PushTxnResponse) break } @@ -2082,17 +2083,17 @@ func (r *Replica) maybeWatchForMergeLocked(ctx context.Context) (bool, error) { // transaction completed, resolved its intents, and GC'd its transaction // record before our PushTxn arrived. To figure out what happened, we // need to look in meta2. - var getRes *roachpb.GetResponse + var getRes *kvpb.GetResponse for retry := retry.Start(base.DefaultRetryOptions()); retry.Next(); { metaKey := keys.RangeMetaKey(desc.EndKey) - res, pErr := kv.SendWrappedWith(ctx, r.store.DB().NonTransactionalSender(), roachpb.Header{ + res, pErr := kv.SendWrappedWith(ctx, r.store.DB().NonTransactionalSender(), kvpb.Header{ // Use READ_UNCOMMITTED to avoid trying to resolve intents, since // resolving those intents might involve sending requests to this // range, and that could deadlock. See the comment on // TestStoreRangeMergeConcurrentSplit for details. - ReadConsistency: roachpb.READ_UNCOMMITTED, - }, &roachpb.GetRequest{ - RequestHeader: roachpb.RequestHeader{Key: metaKey.AsRawKey()}, + ReadConsistency: kvpb.READ_UNCOMMITTED, + }, &kvpb.GetRequest{ + RequestHeader: kvpb.RequestHeader{Key: metaKey.AsRawKey()}, }) if pErr != nil { select { @@ -2107,7 +2108,7 @@ func (r *Replica) maybeWatchForMergeLocked(ctx context.Context) (bool, error) { continue } } - getRes = res.(*roachpb.GetResponse) + getRes = res.(*kvpb.GetResponse) break } if getRes.Value == nil { @@ -2134,7 +2135,7 @@ func (r *Replica) maybeWatchForMergeLocked(ctx context.Context) (bool, error) { // The merge committed but the left-hand replica on this store hasn't // subsumed this replica yet. Mark this replica as destroyed so it // doesn't serve requests when we close the mergeCompleteCh below. - r.mu.destroyStatus.Set(roachpb.NewRangeNotFoundError(r.RangeID, r.store.StoreID()), destroyReasonMergePending) + r.mu.destroyStatus.Set(kvpb.NewRangeNotFoundError(r.RangeID, r.store.StoreID()), destroyReasonMergePending) } // Unblock pending requests. If the merge committed, the requests will // notice that the replica has been destroyed and return an appropriate @@ -2211,11 +2212,11 @@ func (r *Replica) getReplicaDescriptorByIDRLocked( // transaction abort error. func checkIfTxnAborted( ctx context.Context, rec batcheval.EvalContext, reader storage.Reader, txn roachpb.Transaction, -) *roachpb.Error { +) *kvpb.Error { var entry roachpb.AbortSpanEntry aborted, err := rec.AbortSpan().Get(ctx, reader, txn.ID, &entry) if err != nil { - return roachpb.NewError(roachpb.NewReplicaCorruptionError( + return kvpb.NewError(kvpb.NewReplicaCorruptionError( errors.Wrap(err, "could not read from AbortSpan"))) } if aborted { @@ -2227,8 +2228,8 @@ func checkIfTxnAborted( newTxn.Priority = entry.Priority } newTxn.Status = roachpb.ABORTED - return roachpb.NewErrorWithTxn( - roachpb.NewTransactionAbortedError(roachpb.ABORT_REASON_ABORT_SPAN), newTxn) + return kvpb.NewErrorWithTxn( + kvpb.NewTransactionAbortedError(kvpb.ABORT_REASON_ABORT_SPAN), newTxn) } return nil } diff --git a/pkg/kv/kvserver/replica_app_batch.go b/pkg/kv/kvserver/replica_app_batch.go index 6116a6dce9e2..77923343b014 100644 --- a/pkg/kv/kvserver/replica_app_batch.go +++ b/pkg/kv/kvserver/replica_app_batch.go @@ -14,6 +14,7 @@ import ( "context" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/apply" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvadmission" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" @@ -199,7 +200,7 @@ func (b *replicaAppBatch) runPreAddTriggersReplicaOnly( // in Replica application. if p, filter := b.r.getRangefeedProcessorAndFilter(); p != nil { if err := populatePrevValsInLogicalOpLog(ctx, filter, ops, b.batch); err != nil { - b.r.disconnectRangefeedWithErr(p, roachpb.NewError(err)) + b.r.disconnectRangefeedWithErr(p, kvpb.NewError(err)) } } } @@ -267,7 +268,7 @@ func (b *replicaAppBatch) runPostAddTriggersReplicaOnly( // for this on appBatch. if res.MVCCHistoryMutation != nil { for _, span := range res.MVCCHistoryMutation.Spans { - b.r.disconnectRangefeedSpanWithErr(span, roachpb.NewError(&roachpb.MVCCHistoryMutationError{ + b.r.disconnectRangefeedSpanWithErr(span, kvpb.NewError(&kvpb.MVCCHistoryMutationError{ Span: span, })) } @@ -302,7 +303,7 @@ func (b *replicaAppBatch) runPostAddTriggersReplicaOnly( // that overlap with the new range of the split and keep registrations that // are only interested in keys that are still on the original range running. b.r.disconnectRangefeedWithReason( - roachpb.RangeFeedRetryError_REASON_RANGE_SPLIT, + kvpb.RangeFeedRetryError_REASON_RANGE_SPLIT, ) } @@ -325,7 +326,7 @@ func (b *replicaAppBatch) runPostAddTriggersReplicaOnly( rhsRepl.readOnlyCmdMu.Lock() rhsRepl.mu.Lock() rhsRepl.mu.destroyStatus.Set( - roachpb.NewRangeNotFoundError(rhsRepl.RangeID, rhsRepl.store.StoreID()), + kvpb.NewRangeNotFoundError(rhsRepl.RangeID, rhsRepl.store.StoreID()), destroyReasonRemoved) rhsRepl.mu.Unlock() rhsRepl.readOnlyCmdMu.Unlock() @@ -359,10 +360,10 @@ func (b *replicaAppBatch) runPostAddTriggersReplicaOnly( // rangefeeds with REASON_REPLICA_REMOVED. That's ok because we will have // already disconnected the rangefeed here. b.r.disconnectRangefeedWithReason( - roachpb.RangeFeedRetryError_REASON_RANGE_MERGED, + kvpb.RangeFeedRetryError_REASON_RANGE_MERGED, ) rhsRepl.disconnectRangefeedWithReason( - roachpb.RangeFeedRetryError_REASON_RANGE_MERGED, + kvpb.RangeFeedRetryError_REASON_RANGE_MERGED, ) } @@ -464,7 +465,7 @@ func (b *replicaAppBatch) runPostAddTriggersReplicaOnly( b.r.readOnlyCmdMu.Lock() b.r.mu.Lock() b.r.mu.destroyStatus.Set( - roachpb.NewRangeNotFoundError(b.r.RangeID, b.r.store.StoreID()), + kvpb.NewRangeNotFoundError(b.r.RangeID, b.r.store.StoreID()), destroyReasonRemoved) span := b.r.descRLocked().RSpan() b.r.mu.Unlock() diff --git a/pkg/kv/kvserver/replica_application_cmd.go b/pkg/kv/kvserver/replica_application_cmd.go index 1018e5ea6d84..d82eddffdc86 100644 --- a/pkg/kv/kvserver/replica_application_cmd.go +++ b/pkg/kv/kvserver/replica_application_cmd.go @@ -13,9 +13,9 @@ package kvserver import ( "context" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftlog" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/tracing" ) @@ -79,7 +79,7 @@ func (c *replicatedCmd) Ctx() context.Context { // AckErrAndFinish implements apply.Command. func (c *replicatedCmd) AckErrAndFinish(ctx context.Context, err error) error { if c.IsLocal() { - c.response.Err = roachpb.NewError(roachpb.NewAmbiguousResultError(err)) + c.response.Err = kvpb.NewError(kvpb.NewAmbiguousResultError(err)) } return c.AckOutcomeAndFinish(ctx) } @@ -110,7 +110,7 @@ func (c *replicatedCmd) CanAckBeforeApplication() bool { if !req.IsIntentWrite() || req.AsyncConsensus { return false } - if et, ok := req.GetArg(roachpb.EndTxn); ok && et.(*roachpb.EndTxnRequest).InternalCommitTrigger != nil { + if et, ok := req.GetArg(kvpb.EndTxn); ok && et.(*kvpb.EndTxnRequest).InternalCommitTrigger != nil { // Don't early-ack for commit triggers, just to keep things simple - the // caller is reasonably expecting to be able to run another replication // change right away, and some code paths pull the descriptor out of memory @@ -132,7 +132,7 @@ func (c *replicatedCmd) AckSuccess(ctx context.Context) error { // is finished. var resp proposalResult reply := *c.proposal.Local.Reply - reply.Responses = append([]roachpb.ResponseUnion(nil), reply.Responses...) + reply.Responses = append([]kvpb.ResponseUnion(nil), reply.Responses...) resp.Reply = &reply resp.EncounteredIntents = c.proposal.Local.DetachEncounteredIntents() resp.EndTxns = c.proposal.Local.DetachEndTxns(false /* alwaysOnly */) diff --git a/pkg/kv/kvserver/replica_application_result.go b/pkg/kv/kvserver/replica_application_result.go index 593dcc73536f..ef91c9a9bb66 100644 --- a/pkg/kv/kvserver/replica_application_result.go +++ b/pkg/kv/kvserver/replica_application_result.go @@ -13,6 +13,7 @@ package kvserver import ( "context" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb" @@ -74,7 +75,7 @@ func (r *Replica) prepareLocalResult(ctx context.Context, cmd *replicatedCmd) { return } - var pErr *roachpb.Error + var pErr *kvpb.Error if filter := r.store.cfg.TestingKnobs.TestingPostApplyFilter; filter != nil { var newPropRetry int newPropRetry, pErr = filter(kvserverbase.ApplyFilterArgs{ @@ -162,8 +163,8 @@ func (r *Replica) prepareLocalResult(ctx context.Context, cmd *replicatedCmd) { // reproposer is used by tryReproposeWithNewLeaseIndex. type reproposer interface { trackEvaluatingRequest(context.Context, hlc.Timestamp) (hlc.Timestamp, TrackedRequestToken) - propose(context.Context, *ProposalData, TrackedRequestToken) *roachpb.Error - newNotLeaseHolderError(string) *roachpb.NotLeaseHolderError + propose(context.Context, *ProposalData, TrackedRequestToken) *kvpb.Error + newNotLeaseHolderError(string) *kvpb.NotLeaseHolderError } type replicaReproposer Replica @@ -179,14 +180,14 @@ func (r *replicaReproposer) trackEvaluatingRequest( func (r *replicaReproposer) propose( ctx context.Context, p *ProposalData, tok TrackedRequestToken, -) *roachpb.Error { +) *kvpb.Error { return (*Replica)(r).propose(ctx, p, tok) } -func (r *replicaReproposer) newNotLeaseHolderError(msg string) *roachpb.NotLeaseHolderError { +func (r *replicaReproposer) newNotLeaseHolderError(msg string) *kvpb.NotLeaseHolderError { r.mu.RLock() defer r.mu.RUnlock() - return roachpb.NewNotLeaseHolderError( + return kvpb.NewNotLeaseHolderError( *r.mu.state.Lease, r.store.StoreID(), r.mu.state.Desc, @@ -209,7 +210,7 @@ func (r *replicaReproposer) newNotLeaseHolderError(msg string) *roachpb.NotLease // timestamps across subsequent calls. func tryReproposeWithNewLeaseIndex( ctx context.Context, cmd *replicatedCmd, r reproposer, -) *roachpb.Error { +) *kvpb.Error { // Note that we don't need to validate anything about the proposal's // lease here - if we got this far, we know that everything but the // index is valid at this point in the log. @@ -246,7 +247,7 @@ func tryReproposeWithNewLeaseIndex( // The tracker wants us to forward the request timestamp, but we can't // do that without re-evaluating, so give up. The error returned here // will go back to DistSender, so send something it can digest. - return roachpb.NewError(r.newNotLeaseHolderError("reproposal failed due to closed timestamp")) + return kvpb.NewError(r.newNotLeaseHolderError("reproposal failed due to closed timestamp")) } // Some tests check for this log message in the trace. log.VEventf(ctx, 2, "retry: proposalIllegalLeaseIndex") diff --git a/pkg/kv/kvserver/replica_application_state_machine.go b/pkg/kv/kvserver/replica_application_state_machine.go index b5ebd1637962..baf87b49268f 100644 --- a/pkg/kv/kvserver/replica_application_state_machine.go +++ b/pkg/kv/kvserver/replica_application_state_machine.go @@ -13,6 +13,7 @@ package kvserver import ( "context" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/apply" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvadmission" @@ -446,7 +447,7 @@ type closedTimestampSetterInfo struct { // NOTE: We only keep track of lease requests because keeping track of all // requests would be too expensive: cloning the request is expensive and also // requests can be large in memory. - leaseReq *roachpb.RequestLeaseRequest + leaseReq *kvpb.RequestLeaseRequest // split and merge are set if the request was an EndTxn with the respective // commit trigger set. split, merge bool @@ -462,9 +463,9 @@ func (s *closedTimestampSetterInfo) record(cmd *replicatedCmd, lease *roachpb.Le return } req := cmd.proposal.Request - et, ok := req.GetArg(roachpb.EndTxn) + et, ok := req.GetArg(kvpb.EndTxn) if ok { - endTxn := et.(*roachpb.EndTxnRequest) + endTxn := et.(*kvpb.EndTxnRequest) if trig := endTxn.InternalCommitTrigger; trig != nil { if trig.SplitTrigger != nil { s.split = true @@ -475,7 +476,7 @@ func (s *closedTimestampSetterInfo) record(cmd *replicatedCmd, lease *roachpb.Le } else if req.IsSingleRequestLeaseRequest() { // Make a deep copy since we're not allowed to hold on to request // memory. - lr, _ := req.GetArg(roachpb.RequestLease) - s.leaseReq = protoutil.Clone(lr).(*roachpb.RequestLeaseRequest) + lr, _ := req.GetArg(kvpb.RequestLease) + s.leaseReq = protoutil.Clone(lr).(*kvpb.RequestLeaseRequest) } } diff --git a/pkg/kv/kvserver/replica_application_state_machine_test.go b/pkg/kv/kvserver/replica_application_state_machine_test.go index 29f722057a45..bffcb6311131 100644 --- a/pkg/kv/kvserver/replica_application_state_machine_test.go +++ b/pkg/kv/kvserver/replica_application_state_machine_test.go @@ -15,6 +15,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/apply" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftlog" @@ -145,7 +146,7 @@ func TestReplicaStateMachineChangeReplicas(t *testing.T) { require.NoError(t, err) } else { require.Error(t, err) - require.IsType(t, &roachpb.RangeNotFoundError{}, err) + require.IsType(t, &kvpb.RangeNotFoundError{}, err) } // Set a destroyStatus to make sure there won't be any raft processing once // we release raftMu. We applied a command but not one from the raft log, so @@ -425,7 +426,7 @@ func TestReplicaStateMachineEphemeralAppBatchRejection(t *testing.T) { raftAppliedIndex := r.mu.state.RaftAppliedIndex r.mu.Unlock() - descWriteRepr := func(v string) (roachpb.Request, []byte) { + descWriteRepr := func(v string) (kvpb.Request, []byte) { b := tc.store.TODOEngine().NewBatch() defer b.Close() key := keys.LocalMax @@ -436,7 +437,7 @@ func TestReplicaStateMachineEphemeralAppBatchRejection(t *testing.T) { }, storage.MVCCValue{ Value: val, })) - return roachpb.NewPut(key, val), b.Repr() + return kvpb.NewPut(key, val), b.Repr() } // Make two commands that have the same MaxLeaseIndex. They'll go @@ -457,7 +458,7 @@ func TestReplicaStateMachineEphemeralAppBatchRejection(t *testing.T) { WriteBatch: &kvserverpb.WriteBatch{Data: repr}, }, } - var ba roachpb.BatchRequest + var ba kvpb.BatchRequest ba.Add(req) cmd := &replicatedCmd{ ctx: ctx, diff --git a/pkg/kv/kvserver/replica_backpressure.go b/pkg/kv/kvserver/replica_backpressure.go index 09a082a7c515..2fa57104e018 100644 --- a/pkg/kv/kvserver/replica_backpressure.go +++ b/pkg/kv/kvserver/replica_backpressure.go @@ -15,6 +15,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -84,7 +85,7 @@ var backpressurableSpans = []roachpb.Span{ // canBackpressureBatch returns whether the provided BatchRequest is eligible // for backpressure. -func canBackpressureBatch(ba *roachpb.BatchRequest) bool { +func canBackpressureBatch(ba *kvpb.BatchRequest) bool { // Don't backpressure splits themselves. if ba.Txn != nil && ba.Txn.Name == splitTxnName { return false @@ -94,7 +95,7 @@ func canBackpressureBatch(ba *roachpb.BatchRequest) bool { // method that is within a "backpressurable" key span. for _, ru := range ba.Requests { req := ru.GetInner() - if !roachpb.CanBackpressure(req) { + if !kvpb.CanBackpressure(req) { continue } @@ -112,10 +113,10 @@ func canBackpressureBatch(ba *roachpb.BatchRequest) bool { // poison.Policy_Wait, in which case it's a neverTripSignaller. In particular, // `(signaller).C() == nil` signals that the request bypasses the circuit // breakers. -func (r *Replica) signallerForBatch(ba *roachpb.BatchRequest) signaller { +func (r *Replica) signallerForBatch(ba *kvpb.BatchRequest) signaller { for _, ru := range ba.Requests { req := ru.GetInner() - if roachpb.BypassesReplicaCircuitBreaker(req) { + if kvpb.BypassesReplicaCircuitBreaker(req) { return neverTripSignaller{} } } @@ -149,7 +150,7 @@ func (r *Replica) shouldBackpressureWrites() bool { // maybeBackpressureBatch blocks to apply backpressure if the replica deems // that backpressure is necessary. -func (r *Replica) maybeBackpressureBatch(ctx context.Context, ba *roachpb.BatchRequest) error { +func (r *Replica) maybeBackpressureBatch(ctx context.Context, ba *kvpb.BatchRequest) error { if !canBackpressureBatch(ba) { return nil } diff --git a/pkg/kv/kvserver/replica_batch_updates.go b/pkg/kv/kvserver/replica_batch_updates.go index b6dda09e0efe..edab85439388 100644 --- a/pkg/kv/kvserver/replica_batch_updates.go +++ b/pkg/kv/kvserver/replica_batch_updates.go @@ -13,6 +13,7 @@ package kvserver import ( "context" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -38,13 +39,13 @@ import ( // entirely. This is possible if the function removes all of the in-flight // writes from an EndTxn request that was committing in parallel with writes // which all happened to be on the same range as the transaction record. -func maybeStripInFlightWrites(ba *roachpb.BatchRequest) (*roachpb.BatchRequest, error) { - args, hasET := ba.GetArg(roachpb.EndTxn) +func maybeStripInFlightWrites(ba *kvpb.BatchRequest) (*kvpb.BatchRequest, error) { + args, hasET := ba.GetArg(kvpb.EndTxn) if !hasET { return ba, nil } - et := args.(*roachpb.EndTxnRequest) + et := args.(*kvpb.EndTxnRequest) otherReqs := ba.Requests[:len(ba.Requests)-1] if !et.IsParallelCommit() || len(otherReqs) == 0 { return ba, nil @@ -56,8 +57,8 @@ func maybeStripInFlightWrites(ba *roachpb.BatchRequest) (*roachpb.BatchRequest, // any elements remain in it. origET := et etAlloc := new(struct { - et roachpb.EndTxnRequest - union roachpb.RequestUnion_EndTxn + et kvpb.EndTxnRequest + union kvpb.RequestUnion_EndTxn }) etAlloc.et = *origET // shallow copy etAlloc.union.EndTxn = &etAlloc.et @@ -65,7 +66,7 @@ func maybeStripInFlightWrites(ba *roachpb.BatchRequest) (*roachpb.BatchRequest, et.InFlightWrites = nil et.LockSpans = et.LockSpans[:len(et.LockSpans):len(et.LockSpans)] // immutable ba = ba.ShallowCopy() - ba.Requests = append([]roachpb.RequestUnion(nil), ba.Requests...) + ba.Requests = append([]kvpb.RequestUnion(nil), ba.Requests...) ba.Requests[len(ba.Requests)-1].Value = &etAlloc.union // Fast-path: If we know that this batch contains all of the transaction's @@ -77,10 +78,10 @@ func maybeStripInFlightWrites(ba *roachpb.BatchRequest) (*roachpb.BatchRequest, for _, ru := range otherReqs { req := ru.GetInner() switch { - case roachpb.IsIntentWrite(req) && !roachpb.IsRange(req): + case kvpb.IsIntentWrite(req) && !kvpb.IsRange(req): // Concurrent point write. writes++ - case req.Method() == roachpb.QueryIntent: + case req.Method() == kvpb.QueryIntent: // Earlier pipelined point write that hasn't been proven yet. writes++ default: @@ -108,9 +109,9 @@ func maybeStripInFlightWrites(ba *roachpb.BatchRequest) (*roachpb.BatchRequest, req := ru.GetInner() seq := req.Header().Sequence switch { - case roachpb.IsIntentWrite(req) && !roachpb.IsRange(req): + case kvpb.IsIntentWrite(req) && !kvpb.IsRange(req): // Concurrent point write. - case req.Method() == roachpb.QueryIntent: + case req.Method() == kvpb.QueryIntent: // Earlier pipelined point write that hasn't been proven yet. We // could remove from the in-flight writes set when we see these, // but doing so would prevent us from using the optimization we @@ -184,7 +185,7 @@ func maybeStripInFlightWrites(ba *roachpb.BatchRequest) (*roachpb.BatchRequest, // works for batches that exclusively contain writes; reads cannot be bumped // like this because they've already acquired timestamp-aware latches. func maybeBumpReadTimestampToWriteTimestamp( - ctx context.Context, ba *roachpb.BatchRequest, g *concurrency.Guard, + ctx context.Context, ba *kvpb.BatchRequest, g *concurrency.Guard, ) bool { if ba.Txn == nil { return false @@ -195,11 +196,11 @@ func maybeBumpReadTimestampToWriteTimestamp( if ba.Txn.ReadTimestamp == ba.Txn.WriteTimestamp { return false } - arg, ok := ba.GetArg(roachpb.EndTxn) + arg, ok := ba.GetArg(kvpb.EndTxn) if !ok { return false } - et := arg.(*roachpb.EndTxnRequest) + et := arg.(*kvpb.EndTxnRequest) if batcheval.IsEndTxnExceedingDeadline(ba.Txn.WriteTimestamp, et.Deadline) { return false } @@ -219,7 +220,7 @@ func maybeBumpReadTimestampToWriteTimestamp( // Returns true if the timestamp was bumped. Returns false if the timestamp could // not be bumped. func tryBumpBatchTimestamp( - ctx context.Context, ba *roachpb.BatchRequest, g *concurrency.Guard, ts hlc.Timestamp, + ctx context.Context, ba *kvpb.BatchRequest, g *concurrency.Guard, ts hlc.Timestamp, ) bool { if g != nil && !g.IsolatedAtLaterTimestamps() { return false diff --git a/pkg/kv/kvserver/replica_batch_updates_test.go b/pkg/kv/kvserver/replica_batch_updates_test.go index 8b92e3ba977e..5e330e1a8123 100644 --- a/pkg/kv/kvserver/replica_batch_updates_test.go +++ b/pkg/kv/kvserver/replica_batch_updates_test.go @@ -15,6 +15,7 @@ import ( "reflect" "testing" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -29,70 +30,70 @@ func TestMaybeStripInFlightWrites(t *testing.T) { defer log.Scope(t).Close(t) keyA, keyB, keyC := roachpb.Key("a"), roachpb.Key("b"), roachpb.Key("c") - qi1 := &roachpb.QueryIntentRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}} + qi1 := &kvpb.QueryIntentRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}} qi1.Txn.Sequence = 1 - put2 := &roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyB}} + put2 := &kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyB}} put2.Sequence = 2 - put3 := &roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: keyC}} + put3 := &kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyC}} put3.Sequence = 3 - delRng3 := &roachpb.DeleteRangeRequest{RequestHeader: roachpb.RequestHeader{Key: keyC}} + delRng3 := &kvpb.DeleteRangeRequest{RequestHeader: kvpb.RequestHeader{Key: keyC}} delRng3.Sequence = 3 - scan3 := &roachpb.ScanRequest{RequestHeader: roachpb.RequestHeader{Key: keyC}} + scan3 := &kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyC}} scan3.Sequence = 3 - et := &roachpb.EndTxnRequest{RequestHeader: roachpb.RequestHeader{Key: keyA}, Commit: true} + et := &kvpb.EndTxnRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}, Commit: true} et.Sequence = 4 et.LockSpans = []roachpb.Span{{Key: keyC}} et.InFlightWrites = []roachpb.SequencedWrite{{Key: keyA, Sequence: 1}, {Key: keyB, Sequence: 2}} testCases := []struct { - reqs []roachpb.Request + reqs []kvpb.Request expIFW []roachpb.SequencedWrite expLockSpans []roachpb.Span expErr string }{ { - reqs: []roachpb.Request{et}, + reqs: []kvpb.Request{et}, expIFW: []roachpb.SequencedWrite{{Key: keyA, Sequence: 1}, {Key: keyB, Sequence: 2}}, expLockSpans: []roachpb.Span{{Key: keyC}}, }, // QueryIntents aren't stripped from the in-flight writes set on the // slow-path of maybeStripInFlightWrites. This is intentional. { - reqs: []roachpb.Request{qi1, et}, + reqs: []kvpb.Request{qi1, et}, expIFW: []roachpb.SequencedWrite{{Key: keyA, Sequence: 1}, {Key: keyB, Sequence: 2}}, expLockSpans: []roachpb.Span{{Key: keyC}}, }, { - reqs: []roachpb.Request{put2, et}, + reqs: []kvpb.Request{put2, et}, expIFW: []roachpb.SequencedWrite{{Key: keyA, Sequence: 1}}, expLockSpans: []roachpb.Span{{Key: keyB}, {Key: keyC}}, }, { - reqs: []roachpb.Request{put3, et}, + reqs: []kvpb.Request{put3, et}, expErr: "write in batch with EndTxn missing from in-flight writes", }, { - reqs: []roachpb.Request{qi1, put2, et}, + reqs: []kvpb.Request{qi1, put2, et}, expIFW: nil, expLockSpans: []roachpb.Span{{Key: keyA}, {Key: keyB}, {Key: keyC}}, }, { - reqs: []roachpb.Request{qi1, put2, delRng3, et}, + reqs: []kvpb.Request{qi1, put2, delRng3, et}, expIFW: nil, expLockSpans: []roachpb.Span{{Key: keyA}, {Key: keyB}, {Key: keyC}}, }, { - reqs: []roachpb.Request{qi1, put2, scan3, et}, + reqs: []kvpb.Request{qi1, put2, scan3, et}, expIFW: nil, expLockSpans: []roachpb.Span{{Key: keyA}, {Key: keyB}, {Key: keyC}}, }, { - reqs: []roachpb.Request{qi1, put2, delRng3, scan3, et}, + reqs: []kvpb.Request{qi1, put2, delRng3, scan3, et}, expIFW: nil, expLockSpans: []roachpb.Span{{Key: keyA}, {Key: keyB}, {Key: keyC}}, }, } for _, c := range testCases { - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Add(c.reqs...) t.Run(fmt.Sprint(ba), func(t *testing.T) { resBa, err := maybeStripInFlightWrites(ba) @@ -100,8 +101,8 @@ func TestMaybeStripInFlightWrites(t *testing.T) { if err != nil { t.Errorf("expected no error, got %v", err) } - resArgs, _ := resBa.GetArg(roachpb.EndTxn) - resEt := resArgs.(*roachpb.EndTxnRequest) + resArgs, _ := resBa.GetArg(kvpb.EndTxn) + resEt := resArgs.(*kvpb.EndTxnRequest) if !reflect.DeepEqual(resEt.InFlightWrites, c.expIFW) { t.Errorf("expected in-flight writes %v, got %v", c.expIFW, resEt.InFlightWrites) } diff --git a/pkg/kv/kvserver/replica_circuit_breaker.go b/pkg/kv/kvserver/replica_circuit_breaker.go index 22ade252ad7c..6a6f60fae043 100644 --- a/pkg/kv/kvserver/replica_circuit_breaker.go +++ b/pkg/kv/kvserver/replica_circuit_breaker.go @@ -15,6 +15,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/telemetry" @@ -35,8 +36,8 @@ import ( type replicaInCircuitBreaker interface { Clock() *hlc.Clock Desc() *roachpb.RangeDescriptor - Send(context.Context, *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) - slowReplicationThreshold(ba *roachpb.BatchRequest) (time.Duration, bool) + Send(context.Context, *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) + slowReplicationThreshold(ba *kvpb.BatchRequest) (time.Duration, bool) replicaUnavailableError(err error) error poisonInflightLatches(err error) } @@ -213,10 +214,10 @@ func sendProbe(ctx context.Context, r replicaInCircuitBreaker) error { if !desc.IsInitialized() { return nil } - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Timestamp = r.Clock().Now() ba.RangeID = r.Desc().RangeID - probeReq := &roachpb.ProbeRequest{} + probeReq := &kvpb.ProbeRequest{} probeReq.Key = desc.StartKey.AsRawKey() ba.Add(probeReq) thresh, ok := r.slowReplicationThreshold(ba) @@ -273,7 +274,7 @@ func replicaUnavailableError( redact.Safe(rs), /* raft status contains no PII */ ) - return roachpb.NewReplicaUnavailableError(errors.Wrapf(err, "%s", buf), desc, replDesc) + return kvpb.NewReplicaUnavailableError(errors.Wrapf(err, "%s", buf), desc, replDesc) } func (r *Replica) replicaUnavailableError(err error) error { diff --git a/pkg/kv/kvserver/replica_closedts_internal_test.go b/pkg/kv/kvserver/replica_closedts_internal_test.go index 2b1bf67567e4..6d170b0d8633 100644 --- a/pkg/kv/kvserver/replica_closedts_internal_test.go +++ b/pkg/kv/kvserver/replica_closedts_internal_test.go @@ -19,6 +19,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" @@ -669,7 +670,7 @@ func TestQueryResolvedTimestamp(t *testing.T) { { pArgs := putArgs(intentKey, []byte("val")) assignSeqNumsForReqs(&txn, &pArgs) - _, pErr := kv.SendWrappedWith(ctx, tc.Sender(), roachpb.Header{Txn: &txn}, &pArgs) + _, pErr := kv.SendWrappedWith(ctx, tc.Sender(), kvpb.Header{Txn: &txn}, &pArgs) require.Nil(t, pErr) } @@ -713,22 +714,22 @@ func TestQueryResolvedTimestampResolvesAbandonedIntents(t *testing.T) { txn := roachpb.MakeTransaction("test", key, 0, ts10, 0, 0) pArgs := putArgs(key, []byte("val")) assignSeqNumsForReqs(&txn, &pArgs) - _, pErr := kv.SendWrappedWith(ctx, tc.Sender(), roachpb.Header{Txn: &txn}, &pArgs) + _, pErr := kv.SendWrappedWith(ctx, tc.Sender(), kvpb.Header{Txn: &txn}, &pArgs) require.Nil(t, pErr) intentExists := func() bool { t.Helper() gArgs := getArgs(key) assignSeqNumsForReqs(&txn, &gArgs) - resp, pErr := kv.SendWrappedWith(ctx, tc.Sender(), roachpb.Header{Txn: &txn}, &gArgs) + resp, pErr := kv.SendWrappedWith(ctx, tc.Sender(), kvpb.Header{Txn: &txn}, &gArgs) - abortErr, ok := pErr.GetDetail().(*roachpb.TransactionAbortedError) - if ok && abortErr.Reason == roachpb.ABORT_REASON_ABORT_SPAN { + abortErr, ok := pErr.GetDetail().(*kvpb.TransactionAbortedError) + if ok && abortErr.Reason == kvpb.ABORT_REASON_ABORT_SPAN { // When the intent is resolved, it will be replaced by an abort span entry. return false } require.Nil(t, pErr) - require.NotNil(t, resp.(*roachpb.GetResponse).Value) + require.NotNil(t, resp.(*kvpb.GetResponse).Value) return true } require.True(t, intentExists()) @@ -800,7 +801,7 @@ func TestServerSideBoundedStalenessNegotiation(t *testing.T) { for _, test := range []struct { name string - reqs []roachpb.Request + reqs []kvpb.Request minTSBound hlc.Timestamp maxTSBound hlc.Timestamp withTS bool // error case @@ -812,19 +813,19 @@ func TestServerSideBoundedStalenessNegotiation(t *testing.T) { }{ { name: "empty key, min bound below closed ts", - reqs: []roachpb.Request{&getEmptyKey}, + reqs: []kvpb.Request{&getEmptyKey}, minTSBound: ts20, expRespTS: ts30, }, { name: "empty key, min bound equal to closed ts", - reqs: []roachpb.Request{&getEmptyKey}, + reqs: []kvpb.Request{&getEmptyKey}, minTSBound: ts30, expRespTS: ts30, }, { name: "empty key, min bound above closed ts", - reqs: []roachpb.Request{&getEmptyKey}, + reqs: []kvpb.Request{&getEmptyKey}, minTSBound: ts40, expRespTS: ts40, // for !strict case expErr: ifStrict( @@ -834,13 +835,13 @@ func TestServerSideBoundedStalenessNegotiation(t *testing.T) { }, { name: "intent key, min bound below intent ts, min bound below closed ts", - reqs: []roachpb.Request{&getIntentKey}, + reqs: []kvpb.Request{&getIntentKey}, minTSBound: ts10, expRespTS: ts20.Prev(), }, { name: "intent key, min bound equal to intent ts, min bound below closed ts", - reqs: []roachpb.Request{&getIntentKey}, + reqs: []kvpb.Request{&getIntentKey}, minTSBound: ts20, expErr: ifStrict( "bounded staleness read .* could not be satisfied", @@ -849,7 +850,7 @@ func TestServerSideBoundedStalenessNegotiation(t *testing.T) { }, { name: "intent key, min bound above intent ts, min bound equal to closed ts", - reqs: []roachpb.Request{&getIntentKey}, + reqs: []kvpb.Request{&getIntentKey}, minTSBound: ts30, expErr: ifStrict( "bounded staleness read .* could not be satisfied", @@ -858,7 +859,7 @@ func TestServerSideBoundedStalenessNegotiation(t *testing.T) { }, { name: "intent key, min bound above intent ts, min bound above closed ts", - reqs: []roachpb.Request{&getIntentKey}, + reqs: []kvpb.Request{&getIntentKey}, minTSBound: ts40, expErr: ifStrict( "bounded staleness read .* could not be satisfied", @@ -867,13 +868,13 @@ func TestServerSideBoundedStalenessNegotiation(t *testing.T) { }, { name: "empty and intent key, min bound below intent ts, min bound below closed ts", - reqs: []roachpb.Request{&getEmptyKey, &getIntentKey}, + reqs: []kvpb.Request{&getEmptyKey, &getIntentKey}, minTSBound: ts10, expRespTS: ts20.Prev(), }, { name: "empty and intent key, min bound equal to intent ts, min bound below closed ts", - reqs: []roachpb.Request{&getEmptyKey, &getIntentKey}, + reqs: []kvpb.Request{&getEmptyKey, &getIntentKey}, minTSBound: ts20, expErr: ifStrict( "bounded staleness read .* could not be satisfied", @@ -882,7 +883,7 @@ func TestServerSideBoundedStalenessNegotiation(t *testing.T) { }, { name: "empty and intent key, min bound above intent ts, min bound equal to closed ts", - reqs: []roachpb.Request{&getEmptyKey, &getIntentKey}, + reqs: []kvpb.Request{&getEmptyKey, &getIntentKey}, minTSBound: ts30, expErr: ifStrict( "bounded staleness read .* could not be satisfied", @@ -891,7 +892,7 @@ func TestServerSideBoundedStalenessNegotiation(t *testing.T) { }, { name: "empty and intent key, min bound above intent ts, min bound above closed ts", - reqs: []roachpb.Request{&getEmptyKey, &getIntentKey}, + reqs: []kvpb.Request{&getEmptyKey, &getIntentKey}, minTSBound: ts40, expErr: ifStrict( "bounded staleness read .* could not be satisfied", @@ -900,61 +901,61 @@ func TestServerSideBoundedStalenessNegotiation(t *testing.T) { }, { name: "empty key, min and max bound below closed ts", - reqs: []roachpb.Request{&getEmptyKey}, + reqs: []kvpb.Request{&getEmptyKey}, minTSBound: ts10, maxTSBound: ts20.Next(), expRespTS: ts20, }, { name: "intent key, min and max bound below intent ts, min and max bound below closed ts", - reqs: []roachpb.Request{&getIntentKey}, + reqs: []kvpb.Request{&getIntentKey}, minTSBound: ts10, maxTSBound: ts10.Next(), expRespTS: ts10, }, { name: "empty and intent key, min and max bound below intent ts, min and max bound below closed ts", - reqs: []roachpb.Request{&getEmptyKey, &getIntentKey}, + reqs: []kvpb.Request{&getEmptyKey, &getIntentKey}, minTSBound: ts10, maxTSBound: ts10.Next(), expRespTS: ts10, }, { name: "req without min_timestamp_bound", - reqs: []roachpb.Request{&getEmptyKey}, + reqs: []kvpb.Request{&getEmptyKey}, expErr: "MinTimestampBound must be set in batch", }, { name: "req with equal min_timestamp_bound and max_timestamp_bound", - reqs: []roachpb.Request{&getEmptyKey}, + reqs: []kvpb.Request{&getEmptyKey}, minTSBound: ts10, maxTSBound: ts10, expErr: "MaxTimestampBound, if set in batch, must be greater than MinTimestampBound", }, { name: "req with inverted min_timestamp_bound and max_timestamp_bound", - reqs: []roachpb.Request{&getEmptyKey}, + reqs: []kvpb.Request{&getEmptyKey}, minTSBound: ts20, maxTSBound: ts10, expErr: "MaxTimestampBound, if set in batch, must be greater than MinTimestampBound", }, { name: "req with timestamp", - reqs: []roachpb.Request{&getEmptyKey}, + reqs: []kvpb.Request{&getEmptyKey}, minTSBound: ts20, withTS: true, expErr: "MinTimestampBound and Timestamp cannot both be set in batch", }, { name: "req with transaction", - reqs: []roachpb.Request{&getEmptyKey}, + reqs: []kvpb.Request{&getEmptyKey}, minTSBound: ts20, withTxn: true, expErr: "MinTimestampBound and Txn cannot both be set in batch", }, { name: "req with wrong range", - reqs: []roachpb.Request{&getEmptyKey}, + reqs: []kvpb.Request{&getEmptyKey}, minTSBound: ts20, withWrongRange: true, expErr: "r2 was not found on s1", @@ -976,7 +977,7 @@ func TestServerSideBoundedStalenessNegotiation(t *testing.T) { txn := roachpb.MakeTransaction("test", intentKey, 0, intentTS, 0, 0) pArgs := putArgs(intentKey, []byte("val")) assignSeqNumsForReqs(&txn, &pArgs) - _, pErr := kv.SendWrappedWith(ctx, tc.Sender(), roachpb.Header{Txn: &txn}, &pArgs) + _, pErr := kv.SendWrappedWith(ctx, tc.Sender(), kvpb.Header{Txn: &txn}, &pArgs) require.Nil(t, pErr) // Inject a closed timestamp. @@ -985,9 +986,9 @@ func TestServerSideBoundedStalenessNegotiation(t *testing.T) { tc.repl.mu.Unlock() // Construct and issue the request. - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.RangeID = tc.rangeID - ba.BoundedStaleness = &roachpb.BoundedStalenessHeader{ + ba.BoundedStaleness = &kvpb.BoundedStalenessHeader{ MinTimestampBound: test.minTSBound, MinTimestampBoundStrict: strict, MaxTimestampBound: test.maxTSBound, @@ -1050,19 +1051,19 @@ func TestServerSideBoundedStalenessNegotiationWithResumeSpan(t *testing.T) { setup := func(t *testing.T, tc *testContext) hlc.Timestamp { // Write values and intents. val := []byte("val") - send := func(h roachpb.Header, args roachpb.Request) { + send := func(h kvpb.Header, args kvpb.Request) { _, pErr := kv.SendWrappedWith(ctx, tc.Sender(), h, args) require.Nil(t, pErr) } writeValue := func(k string, ts int64) { pArgs := putArgs(roachpb.Key(k), val) - send(roachpb.Header{Timestamp: makeTS(ts)}, &pArgs) + send(kvpb.Header{Timestamp: makeTS(ts)}, &pArgs) } writeIntent := func(k string, ts int64) { txn := roachpb.MakeTransaction("test", roachpb.Key(k), 0, makeTS(ts), 0, 0) pArgs := putArgs(roachpb.Key(k), val) assignSeqNumsForReqs(&txn, &pArgs) - send(roachpb.Header{Txn: &txn}, &pArgs) + send(kvpb.Header{Txn: &txn}, &pArgs) } writeValue("a", 9) writeValue("b", 20) @@ -1089,9 +1090,9 @@ func TestServerSideBoundedStalenessNegotiationWithResumeSpan(t *testing.T) { // get: [g] // get: [h] // - makeReq := func(maxKeys int) *roachpb.BatchRequest { - ba := &roachpb.BatchRequest{} - ba.BoundedStaleness = &roachpb.BoundedStalenessHeader{ + makeReq := func(maxKeys int) *kvpb.BatchRequest { + ba := &kvpb.BatchRequest{} + ba.BoundedStaleness = &kvpb.BoundedStalenessHeader{ MinTimestampBound: makeTS(5), } ba.WaitPolicy = lock.WaitPolicy_Error @@ -1178,11 +1179,11 @@ func TestServerSideBoundedStalenessNegotiationWithResumeSpan(t *testing.T) { for i, ru := range br.Responses { req := ru.GetInner() switch v := req.(type) { - case *roachpb.ScanResponse: + case *kvpb.ScanResponse: for _, kv := range v.Rows { respKeys = append(respKeys, string(kv.Key)) } - case *roachpb.GetResponse: + case *kvpb.GetResponse: if v.Value.IsPresent() { respKeys = append(respKeys, string(ba.Requests[i].GetGet().Key)) } diff --git a/pkg/kv/kvserver/replica_closedts_test.go b/pkg/kv/kvserver/replica_closedts_test.go index 81a7caefcd47..293346ce28b9 100644 --- a/pkg/kv/kvserver/replica_closedts_test.go +++ b/pkg/kv/kvserver/replica_closedts_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" @@ -112,7 +113,7 @@ func TestBumpSideTransportClosed(t *testing.T) { exp: false, knobs: func() (*kvserver.StoreTestingKnobs, chan chan struct{}) { mergeC := make(chan chan struct{}) - testingResponseFilter := func(ctx context.Context, ba *roachpb.BatchRequest, br *roachpb.BatchResponse) *roachpb.Error { + testingResponseFilter := func(ctx context.Context, ba *kvpb.BatchRequest, br *kvpb.BatchResponse) *kvpb.Error { if ba.IsSingleSubsumeRequest() { unblockC := make(chan struct{}) mergeC <- unblockC @@ -139,7 +140,7 @@ func TestBumpSideTransportClosed(t *testing.T) { knobs: func() (*kvserver.StoreTestingKnobs, chan chan struct{}) { applyC := make(chan chan struct{}) var once sync.Once // ignore reproposals - testingApplyFilter := func(filterArgs kvserverbase.ApplyFilterArgs) (int, *roachpb.Error) { + testingApplyFilter := func(filterArgs kvserverbase.ApplyFilterArgs) (int, *kvpb.Error) { if filterArgs.Req != nil && filterArgs.Req.IsSingleRequest() { put := filterArgs.Req.Requests[0].GetPut() if put != nil && put.Key.Equal(roachpb.Key("key_filter")) { @@ -169,7 +170,7 @@ func TestBumpSideTransportClosed(t *testing.T) { exp: false, knobs: func() (*kvserver.StoreTestingKnobs, chan chan struct{}) { proposeC := make(chan chan struct{}) - testingProposalFilter := func(args kvserverbase.ProposalFilterArgs) *roachpb.Error { + testingProposalFilter := func(args kvserverbase.ProposalFilterArgs) *kvpb.Error { if args.Req.IsSingleRequest() { put := args.Req.Requests[0].GetPut() if put != nil && put.Key.Equal(roachpb.Key("key_filter")) { @@ -189,7 +190,7 @@ func TestBumpSideTransportClosed(t *testing.T) { ts := a.target.Add(-1, 0) putArgs := putArgs(roachpb.Key("key_filter"), []byte("val")) sender := a.tc.Server(0).DB().NonTransactionalSender() - _, pErr := kv.SendWrappedWith(ctx, sender, roachpb.Header{Timestamp: ts}, putArgs) + _, pErr := kv.SendWrappedWith(ctx, sender, kvpb.Header{Timestamp: ts}, putArgs) errC <- pErr.GoError() }) unblockFilterC := <-a.filterC @@ -201,7 +202,7 @@ func TestBumpSideTransportClosed(t *testing.T) { exp: false, knobs: func() (*kvserver.StoreTestingKnobs, chan chan struct{}) { proposeC := make(chan chan struct{}) - testingProposalFilter := func(args kvserverbase.ProposalFilterArgs) *roachpb.Error { + testingProposalFilter := func(args kvserverbase.ProposalFilterArgs) *kvpb.Error { if args.Req.IsSingleRequest() { put := args.Req.Requests[0].GetPut() if put != nil && put.Key.Equal(roachpb.Key("key_filter")) { @@ -221,7 +222,7 @@ func TestBumpSideTransportClosed(t *testing.T) { ts := a.target putArgs := putArgs(roachpb.Key("key_filter"), []byte("val")) sender := a.tc.Server(0).DB().NonTransactionalSender() - _, pErr := kv.SendWrappedWith(ctx, sender, roachpb.Header{Timestamp: ts}, putArgs) + _, pErr := kv.SendWrappedWith(ctx, sender, kvpb.Header{Timestamp: ts}, putArgs) errC <- pErr.GoError() }) unblockFilterC := <-a.filterC @@ -233,7 +234,7 @@ func TestBumpSideTransportClosed(t *testing.T) { exp: true, knobs: func() (*kvserver.StoreTestingKnobs, chan chan struct{}) { proposeC := make(chan chan struct{}) - testingProposalFilter := func(args kvserverbase.ProposalFilterArgs) *roachpb.Error { + testingProposalFilter := func(args kvserverbase.ProposalFilterArgs) *kvpb.Error { if args.Req.IsSingleRequest() { put := args.Req.Requests[0].GetPut() if put != nil && put.Key.Equal(roachpb.Key("key_filter")) { @@ -253,7 +254,7 @@ func TestBumpSideTransportClosed(t *testing.T) { ts := a.target.Add(1, 0) putArgs := putArgs(roachpb.Key("key_filter"), []byte("val")) sender := a.tc.Server(0).DB().NonTransactionalSender() - _, pErr := kv.SendWrappedWith(ctx, sender, roachpb.Header{Timestamp: ts}, putArgs) + _, pErr := kv.SendWrappedWith(ctx, sender, kvpb.Header{Timestamp: ts}, putArgs) errC <- pErr.GoError() }) unblockFilterC := <-a.filterC @@ -493,7 +494,7 @@ func TestRejectedLeaseDoesntDictateClosedTimestamp(t *testing.T) { blockLeaseAcquisition := func(args kvserverbase.FilterArgs) { blockedRID := roachpb.RangeID(atomic.LoadInt64(&blockedRangeID)) - leaseReq, ok := args.Req.(*roachpb.RequestLeaseRequest) + leaseReq, ok := args.Req.(*kvpb.RequestLeaseRequest) if !ok || args.Hdr.RangeID != blockedRID || leaseReq.Lease.Replica.NodeID != 2 { return } @@ -506,7 +507,7 @@ func TestRejectedLeaseDoesntDictateClosedTimestamp(t *testing.T) { blockWrites := func(args kvserverbase.FilterArgs) { wk1 := writeKey1.Load().(roachpb.Key) wk2 := writeKey2.Load().(roachpb.Key) - if put, ok := args.Req.(*roachpb.PutRequest); ok && (put.Key.Equal(wk1) || put.Key.Equal(wk2)) { + if put, ok := args.Req.(*kvpb.PutRequest); ok && (put.Key.Equal(wk1) || put.Key.Equal(wk2)) { writeCh <- struct{}{} <-unblockWritesCh } @@ -518,7 +519,7 @@ func TestRejectedLeaseDoesntDictateClosedTimestamp(t *testing.T) { if ba.RangeID != blockedRID { return } - _, ok := p.Request.GetArg(roachpb.TransferLease) + _, ok := p.Request.GetArg(kvpb.TransferLease) if !ok { return } @@ -547,7 +548,7 @@ func TestRejectedLeaseDoesntDictateClosedTimestamp(t *testing.T) { Store: &kvserver.StoreTestingKnobs{ DisableConsistencyQueue: true, EvalKnobs: kvserverbase.BatchEvalTestingKnobs{ - TestingPostEvalFilter: func(args kvserverbase.FilterArgs) *roachpb.Error { + TestingPostEvalFilter: func(args kvserverbase.FilterArgs) *kvpb.Error { blockWrites(args) blockLeaseAcquisition(args) return nil @@ -646,7 +647,7 @@ func TestRejectedLeaseDoesntDictateClosedTimestamp(t *testing.T) { writeKey1.Store(key) sender := n2.DB().NonTransactionalSender() pArgs := putArgs(key, []byte("test val")) - _, pErr := kv.SendWrappedWith(ctx, sender, roachpb.Header{Timestamp: lease.Start.ToTimestamp()}, pArgs) + _, pErr := kv.SendWrappedWith(ctx, sender, kvpb.Header{Timestamp: lease.Start.ToTimestamp()}, pArgs) err1 <- pErr.GoError() }() go func() { @@ -654,7 +655,7 @@ func TestRejectedLeaseDoesntDictateClosedTimestamp(t *testing.T) { writeKey2.Store(k) sender := n2.DB().NonTransactionalSender() pArgs := putArgs(k, []byte("test val2")) - _, pErr := kv.SendWrappedWith(ctx, sender, roachpb.Header{Timestamp: lease.Start.ToTimestamp()}, pArgs) + _, pErr := kv.SendWrappedWith(ctx, sender, kvpb.Header{Timestamp: lease.Start.ToTimestamp()}, pArgs) err2 <- pErr.GoError() }() // Wait for the writes to evaluate and block before proposal. @@ -747,12 +748,12 @@ func TestNonBlockingReadsAtResolvedTimestamp(t *testing.T) { var lastResTS hlc.Timestamp return func(ctx context.Context) error { // Query the key span's resolved timestamp. - queryResTS := roachpb.QueryResolvedTimestampRequest{ - RequestHeader: roachpb.RequestHeaderFromSpan(keySpan), + queryResTS := kvpb.QueryResolvedTimestampRequest{ + RequestHeader: kvpb.RequestHeaderFromSpan(keySpan), } - queryResTSHeader := roachpb.Header{ + queryResTSHeader := kvpb.Header{ RangeID: rangeID, - ReadConsistency: roachpb.INCONSISTENT, + ReadConsistency: kvpb.INCONSISTENT, } resp, pErr := kv.SendWrappedWith(ctx, store, queryResTSHeader, &queryResTS) if pErr != nil { @@ -760,7 +761,7 @@ func TestNonBlockingReadsAtResolvedTimestamp(t *testing.T) { } // Validate that the resolved timestamp increases monotonically. - resTS := resp.(*roachpb.QueryResolvedTimestampResponse).ResolvedTS + resTS := resp.(*kvpb.QueryResolvedTimestampResponse).ResolvedTS if resTS.IsEmpty() { return errors.Errorf("empty resolved timestamp") } @@ -775,13 +776,13 @@ func TestNonBlockingReadsAtResolvedTimestamp(t *testing.T) { // block on an intent. Send to a specific store instead of through a // DistSender so that we'll hear an error (NotLeaseholderError) if the // request would otherwise be redirected to the leaseholder. - scan := roachpb.ScanRequest{ - RequestHeader: roachpb.RequestHeaderFromSpan(keySpan), + scan := kvpb.ScanRequest{ + RequestHeader: kvpb.RequestHeaderFromSpan(keySpan), } txn := roachpb.MakeTransaction("test", keySpan.Key, 0, resTS, 0, 0) - scanHeader := roachpb.Header{ + scanHeader := kvpb.Header{ RangeID: rangeID, - ReadConsistency: roachpb.CONSISTENT, + ReadConsistency: kvpb.CONSISTENT, Txn: &txn, WaitPolicy: lock.WaitPolicy_Error, } @@ -817,15 +818,15 @@ func TestNonBlockingReadsWithServerSideBoundedStalenessNegotiation(t *testing.T) // to block on an intent. Send to a specific store instead of through // a DistSender so that we'll hear an error (NotLeaseholderError) if // the request would otherwise be redirected to the leaseholder. - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.RangeID = rangeID - ba.BoundedStaleness = &roachpb.BoundedStalenessHeader{ + ba.BoundedStaleness = &kvpb.BoundedStalenessHeader{ MinTimestampBound: minTSBound, MinTimestampBoundStrict: true, } ba.WaitPolicy = lock.WaitPolicy_Error - ba.Add(&roachpb.ScanRequest{ - RequestHeader: roachpb.RequestHeaderFromSpan(keySpan), + ba.Add(&kvpb.ScanRequest{ + RequestHeader: kvpb.RequestHeaderFromSpan(keySpan), }) br, pErr := store.Send(ctx, ba) if pErr != nil { diff --git a/pkg/kv/kvserver/replica_command.go b/pkg/kv/kvserver/replica_command.go index b46a3fb737c6..8c37f5b08739 100644 --- a/pkg/kv/kvserver/replica_command.go +++ b/pkg/kv/kvserver/replica_command.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/allocatorimpl" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" @@ -68,10 +69,10 @@ var sendSnapshotTimeout = envutil.EnvOrDefaultDuration( // AdminSplit divides the range into into two ranges using args.SplitKey. func (r *Replica) AdminSplit( - ctx context.Context, args roachpb.AdminSplitRequest, reason string, -) (reply roachpb.AdminSplitResponse, _ *roachpb.Error) { + ctx context.Context, args kvpb.AdminSplitRequest, reason string, +) (reply kvpb.AdminSplitResponse, _ *kvpb.Error) { if len(args.SplitKey) == 0 { - return roachpb.AdminSplitResponse{}, roachpb.NewErrorf("cannot split range with no key provided") + return kvpb.AdminSplitResponse{}, kvpb.NewErrorf("cannot split range with no key provided") } err := r.executeAdminCommandWithDescriptor(ctx, func(desc *roachpb.RangeDescriptor) error { @@ -85,7 +86,7 @@ func (r *Replica) AdminSplit( func maybeDescriptorChangedError( desc *roachpb.RangeDescriptor, err error, ) (ok bool, expectedDesc *roachpb.RangeDescriptor) { - if detail := (*roachpb.ConditionFailedError)(nil); errors.As(err, &detail) { + if detail := (*kvpb.ConditionFailedError)(nil); errors.As(err, &detail) { // Provide a better message in the common case that the range being changed // was already changed by a concurrent transaction. var actualDesc roachpb.RangeDescriptor @@ -227,7 +228,7 @@ func splitTxnAttempt( // End the transaction manually, instead of letting RunTransaction // loop do it, in order to provide a split trigger. - b.AddRawRequest(&roachpb.EndTxnRequest{ + b.AddRawRequest(&kvpb.EndTxnRequest{ Commit: true, InternalCommitTrigger: &roachpb.InternalCommitTrigger{ SplitTrigger: &roachpb.SplitTrigger{ @@ -263,7 +264,7 @@ func splitTxnStickyUpdateAttempt( } // End the transaction manually, instead of letting RunTransaction loop // do it, in order to provide a sticky bit trigger. - b.AddRawRequest(&roachpb.EndTxnRequest{ + b.AddRawRequest(&kvpb.EndTxnRequest{ Commit: true, InternalCommitTrigger: &roachpb.InternalCommitTrigger{ StickyBitTrigger: &roachpb.StickyBitTrigger{ @@ -294,13 +295,13 @@ func splitTxnStickyUpdateAttempt( // See the comment on splitTrigger for details on the complexities. func (r *Replica) adminSplitWithDescriptor( ctx context.Context, - args roachpb.AdminSplitRequest, + args kvpb.AdminSplitRequest, desc *roachpb.RangeDescriptor, delayable bool, reason string, -) (roachpb.AdminSplitResponse, error) { +) (kvpb.AdminSplitResponse, error) { var err error - var reply roachpb.AdminSplitResponse + var reply kvpb.AdminSplitResponse // The split queue doesn't care about the set of replicas, so if we somehow // are being handed one that's in a joint state, finalize that before @@ -336,7 +337,7 @@ func (r *Replica) adminSplitWithDescriptor( // correct range. if !kvserverbase.ContainsKey(desc, args.Key) { ri := r.GetRangeInfo(ctx) - return reply, roachpb.NewRangeKeyMismatchErrorWithCTPolicy(ctx, args.Key, args.Key, desc, &ri.Lease, ri.ClosedTimestampPolicy) + return reply, kvpb.NewRangeKeyMismatchErrorWithCTPolicy(ctx, args.Key, args.Key, desc, &ri.Lease, ri.ClosedTimestampPolicy) } foundSplitKey = args.SplitKey } @@ -430,9 +431,9 @@ func (r *Replica) adminSplitWithDescriptor( // AdminUnsplit removes the sticky bit of the range specified by the // args.Key. func (r *Replica) AdminUnsplit( - ctx context.Context, args roachpb.AdminUnsplitRequest, reason string, -) (roachpb.AdminUnsplitResponse, *roachpb.Error) { - var reply roachpb.AdminUnsplitResponse + ctx context.Context, args kvpb.AdminUnsplitRequest, reason string, +) (kvpb.AdminUnsplitResponse, *kvpb.Error) { + var reply kvpb.AdminUnsplitResponse err := r.executeAdminCommandWithDescriptor(ctx, func(desc *roachpb.RangeDescriptor) error { var err error reply, err = r.adminUnsplitWithDescriptor(ctx, args, desc, reason) @@ -442,12 +443,9 @@ func (r *Replica) AdminUnsplit( } func (r *Replica) adminUnsplitWithDescriptor( - ctx context.Context, - args roachpb.AdminUnsplitRequest, - desc *roachpb.RangeDescriptor, - reason string, -) (roachpb.AdminUnsplitResponse, error) { - var reply roachpb.AdminUnsplitResponse + ctx context.Context, args kvpb.AdminUnsplitRequest, desc *roachpb.RangeDescriptor, reason string, +) (kvpb.AdminUnsplitResponse, error) { + var reply kvpb.AdminUnsplitResponse if !bytes.Equal(desc.StartKey.AsRawKey(), args.Header().Key) { return reply, errors.Errorf("key %s is not the start of a range", args.Header().Key) } @@ -480,7 +478,7 @@ func (r *Replica) adminUnsplitWithDescriptor( return err } // End the transaction manually in order to provide a sticky bit trigger. - b.AddRawRequest(&roachpb.EndTxnRequest{ + b.AddRawRequest(&kvpb.EndTxnRequest{ Commit: true, InternalCommitTrigger: &roachpb.InternalCommitTrigger{ StickyBitTrigger: &roachpb.StickyBitTrigger{ @@ -510,7 +508,7 @@ func (r *Replica) adminUnsplitWithDescriptor( // retry loop. func (r *Replica) executeAdminCommandWithDescriptor( ctx context.Context, updateDesc func(*roachpb.RangeDescriptor) error, -) *roachpb.Error { +) *kvpb.Error { // Retry forever as long as we see errors we know will resolve. retryOpts := base.DefaultRetryOptions() // Randomize quite a lot just in case someone else also interferes with us @@ -527,7 +525,7 @@ func (r *Replica) executeAdminCommandWithDescriptor( // (i.e., the lease we have in r.mu.state.Lease) can remain valid // indefinitely. if _, err := r.IsDestroyed(); err != nil { - return roachpb.NewError(err) + return kvpb.NewError(err) } // Admin commands always require the range lease to begin (see @@ -543,14 +541,14 @@ func (r *Replica) executeAdminCommandWithDescriptor( // On seeing a retryable replication change or an AmbiguousResultError, // retry the command with the updated descriptor. if !IsRetriableReplicationChangeError(lastErr) && - !errors.HasType(lastErr, (*roachpb.AmbiguousResultError)(nil)) { + !errors.HasType(lastErr, (*kvpb.AmbiguousResultError)(nil)) { break } if splitRetryLogLimiter.ShouldLog() { log.Warningf(ctx, "retrying split after err: %v", lastErr) } } - return roachpb.NewError(lastErr) + return kvpb.NewError(lastErr) } // AdminMerge extends this range to subsume the range that comes next @@ -567,9 +565,9 @@ func (r *Replica) executeAdminCommandWithDescriptor( // The supplied RangeDescriptor is used as a form of optimistic lock. See the // comment of "AdminSplit" for more information on this pattern. func (r *Replica) AdminMerge( - ctx context.Context, args roachpb.AdminMergeRequest, reason string, -) (roachpb.AdminMergeResponse, *roachpb.Error) { - var reply roachpb.AdminMergeResponse + ctx context.Context, args kvpb.AdminMergeRequest, reason string, +) (kvpb.AdminMergeResponse, *kvpb.Error) { + var reply kvpb.AdminMergeResponse runMergeTxn := func(txn *kv.Txn) error { log.Event(ctx, "merge txn begins") @@ -736,15 +734,15 @@ func (r *Replica) AdminMerge( // commits, we'll write this data to the left-hand range in the merge // trigger. br, pErr := kv.SendWrapped(ctx, r.store.DB().NonTransactionalSender(), - &roachpb.SubsumeRequest{ - RequestHeader: roachpb.RequestHeader{Key: rightDesc.StartKey.AsRawKey()}, + &kvpb.SubsumeRequest{ + RequestHeader: kvpb.RequestHeader{Key: rightDesc.StartKey.AsRawKey()}, LeftDesc: *origLeftDesc, RightDesc: rightDesc, }) if pErr != nil { return pErr.GoError() } - rhsSnapshotRes := br.(*roachpb.SubsumeResponse) + rhsSnapshotRes := br.(*kvpb.SubsumeResponse) err = contextutil.RunWithTimeout(ctx, "waiting for merge application", mergeApplicationTimeout, func(ctx context.Context) error { @@ -762,7 +760,7 @@ func (r *Replica) AdminMerge( // not serve another request unless this transaction aborts. End the // transaction manually in order to provide a merge trigger. b = txn.NewBatch() - b.AddRawRequest(&roachpb.EndTxnRequest{ + b.AddRawRequest(&kvpb.EndTxnRequest{ Commit: true, InternalCommitTrigger: &roachpb.InternalCommitTrigger{ MergeTrigger: &roachpb.MergeTrigger{ @@ -804,9 +802,9 @@ func (r *Replica) AdminMerge( log.VEventf(ctx, 2, "merge txn rollback failed: %s", rollbackErr) } } - if !errors.HasType(err, (*roachpb.TransactionRetryWithProtoRefreshError)(nil)) { + if !errors.HasType(err, (*kvpb.TransactionRetryWithProtoRefreshError)(nil)) { if err != nil { - return reply, roachpb.NewErrorf("merge failed: %s", err) + return reply, kvpb.NewErrorf("merge failed: %s", err) } return reply, nil } @@ -961,7 +959,7 @@ func (r *Replica) ChangeReplicas( priority kvserverpb.SnapshotRequest_Priority, reason kvserverpb.RangeLogEventReason, details string, - chgs roachpb.ReplicationChanges, + chgs kvpb.ReplicationChanges, ) (updatedDesc *roachpb.RangeDescriptor, _ error) { if desc == nil { // TODO(tbg): is this check just FUD? @@ -993,7 +991,7 @@ func (r *Replica) changeReplicasImpl( senderQueuePriority float64, reason kvserverpb.RangeLogEventReason, details string, - chgs roachpb.ReplicationChanges, + chgs kvpb.ReplicationChanges, ) (updatedDesc *roachpb.RangeDescriptor, _ error) { var err error // If in a joint config, clean up. The assumption here is that the caller @@ -1156,7 +1154,7 @@ type targetsForReplicationChanges struct { // demotions of voters into non-voters. The rest of the changes are handled // distinctly and are thus segregated in the return result. func synthesizeTargetsByChangeType( - chgs roachpb.ReplicationChanges, + chgs kvpb.ReplicationChanges, ) (result targetsForReplicationChanges) { // Isolate the promotions to voters and the demotions to non-voters from the // rest of the changes, since we want to handle these together and execute @@ -1608,9 +1606,7 @@ func validateOneReplicaPerNode(desc *roachpb.RangeDescriptor, chgsByNodeID chang // 5. We're not removing a replica that doesn't exist. // 6. Additions to stores that already contain a replica are strictly the ones // that correspond to a voter demotion and/or a non-voter promotion -func validateReplicationChanges( - desc *roachpb.RangeDescriptor, chgs roachpb.ReplicationChanges, -) error { +func validateReplicationChanges(desc *roachpb.RangeDescriptor, chgs kvpb.ReplicationChanges) error { chgsByStoreID := getChangesByStoreID(chgs) chgsByNodeID := getChangesByNodeID(chgs) @@ -1632,28 +1628,28 @@ func validateReplicationChanges( // changesByStoreID represents a map from StoreID to a slice of replication // changes on that store. -type changesByStoreID map[roachpb.StoreID][]roachpb.ReplicationChange +type changesByStoreID map[roachpb.StoreID][]kvpb.ReplicationChange // changesByNodeID represents a map from NodeID to a slice of replication // changes on that node. -type changesByNodeID map[roachpb.NodeID][]roachpb.ReplicationChange +type changesByNodeID map[roachpb.NodeID][]kvpb.ReplicationChange -func getChangesByStoreID(chgs roachpb.ReplicationChanges) changesByStoreID { - chgsByStoreID := make(map[roachpb.StoreID][]roachpb.ReplicationChange, len(chgs)) +func getChangesByStoreID(chgs kvpb.ReplicationChanges) changesByStoreID { + chgsByStoreID := make(map[roachpb.StoreID][]kvpb.ReplicationChange, len(chgs)) for _, chg := range chgs { if _, ok := chgsByStoreID[chg.Target.StoreID]; !ok { - chgsByStoreID[chg.Target.StoreID] = make([]roachpb.ReplicationChange, 0, 2) + chgsByStoreID[chg.Target.StoreID] = make([]kvpb.ReplicationChange, 0, 2) } chgsByStoreID[chg.Target.StoreID] = append(chgsByStoreID[chg.Target.StoreID], chg) } return chgsByStoreID } -func getChangesByNodeID(chgs roachpb.ReplicationChanges) changesByNodeID { - chgsByNodeID := make(map[roachpb.NodeID][]roachpb.ReplicationChange, len(chgs)) +func getChangesByNodeID(chgs kvpb.ReplicationChanges) changesByNodeID { + chgsByNodeID := make(map[roachpb.NodeID][]kvpb.ReplicationChange, len(chgs)) for _, chg := range chgs { if _, ok := chgsByNodeID[chg.Target.NodeID]; !ok { - chgsByNodeID[chg.Target.NodeID] = make([]roachpb.ReplicationChange, 0, 2) + chgsByNodeID[chg.Target.NodeID] = make([]kvpb.ReplicationChange, 0, 2) } chgsByNodeID[chg.Target.NodeID] = append(chgsByNodeID[chg.Target.NodeID], chg) } @@ -2414,7 +2410,7 @@ func execChangeReplicasTxn( return err } - b.AddRawRequest(&roachpb.EndTxnRequest{ + b.AddRawRequest(&kvpb.EndTxnRequest{ Commit: true, InternalCommitTrigger: &roachpb.InternalCommitTrigger{ ChangeReplicasTrigger: crt, @@ -3237,7 +3233,7 @@ func conditionalGetDescValueFromDB( return existingDesc, existingDescKV.Value.TagAndDataBytes(), true /* skip */, nil } if !matched { - return nil, nil, false /* skip */, &roachpb.ConditionFailedError{ActualValue: existingDescKV.Value} + return nil, nil, false /* skip */, &kvpb.ConditionFailedError{ActualValue: existingDescKV.Value} } return existingDesc, existingDescKV.Value.TagAndDataBytes(), false /* skip */, nil } @@ -3398,7 +3394,7 @@ func (r *Replica) relocateReplicas( return rangeDesc, ctx.Err() } - opss := [][]roachpb.ReplicationChange{ops} + opss := [][]kvpb.ReplicationChange{ops} success := true for _, ops := range opss { newDesc, err := r.store.DB().AdminChangeReplicas(ctx, startKey, rangeDesc, ops) @@ -3515,7 +3511,7 @@ func (roo *replicaRelocateOneOptions) Leaseholder( ctx context.Context, startKey roachpb.RKey, ) (roachpb.ReplicaDescriptor, error) { var b kv.Batch - liReq := &roachpb.LeaseInfoRequest{} + liReq := &kvpb.LeaseInfoRequest{} liReq.Key = startKey.AsRawKey() b.AddRawRequest(liReq) if err := roo.store.DB().Run(ctx, &b); err != nil { @@ -3535,7 +3531,7 @@ func RelocateOne( voterTargets, nonVoterTargets []roachpb.ReplicationTarget, transferLeaseToFirstVoter bool, options RelocateOneOptions, -) ([]roachpb.ReplicationChange, *roachpb.ReplicationTarget, error) { +) ([]kvpb.ReplicationChange, *roachpb.ReplicationTarget, error) { if repls := desc.Replicas(); len(repls.VoterFullAndNonVoterDescriptors()) != len(repls.Descriptors()) { // The caller removed all the learners and left the joint config, so there // shouldn't be anything but voters and non_voters. @@ -3740,7 +3736,7 @@ func RelocateOne( } } - var ops []roachpb.ReplicationChange + var ops []kvpb.ReplicationChange if shouldAdd && shouldRemove { ops, _, err = replicationChangesForRebalance( ctx, desc, len(existingVoters), additionTarget, removalTarget, args.targetType, @@ -3750,9 +3746,9 @@ func RelocateOne( } } else if shouldAdd { if canPromoteNonVoter { - ops = roachpb.ReplicationChangesForPromotion(additionTarget) + ops = kvpb.ReplicationChangesForPromotion(additionTarget) } else { - ops = roachpb.MakeReplicationChanges(args.targetType.AddChangeType(), additionTarget) + ops = kvpb.MakeReplicationChanges(args.targetType.AddChangeType(), additionTarget) } } else if shouldRemove { // Carry out the removal only if there was no lease problem above. If there @@ -3760,9 +3756,9 @@ func RelocateOne( // (Note that !shouldRemove implies that we're trying to remove the last // replica left in the descriptor which is illegal). if canDemoteVoter { - ops = roachpb.ReplicationChangesForDemotion(removalTarget) + ops = kvpb.ReplicationChangesForDemotion(removalTarget) } else { - ops = roachpb.MakeReplicationChanges(args.targetType.RemoveChangeType(), removalTarget) + ops = kvpb.MakeReplicationChanges(args.targetType.RemoveChangeType(), removalTarget) } } @@ -3867,8 +3863,8 @@ func intersectTargets( // adminScatter moves replicas and leaseholders for a selection of ranges. func (r *Replica) adminScatter( - ctx context.Context, args roachpb.AdminScatterRequest, -) (roachpb.AdminScatterResponse, error) { + ctx context.Context, args kvpb.AdminScatterRequest, +) (kvpb.AdminScatterResponse, error) { rq := r.store.replicateQueue retryOpts := retry.Options{ InitialBackoff: 50 * time.Millisecond, @@ -3891,7 +3887,7 @@ func (r *Replica) adminScatter( if args.MaxSize > 0 { if existing, limit := r.GetMVCCStats().Total(), args.MaxSize; existing > limit { - return roachpb.AdminScatterResponse{}, errors.Errorf("existing range size %d exceeds specified limit %d", existing, limit) + return kvpb.AdminScatterResponse{}, errors.Errorf("existing range size %d exceeds specified limit %d", existing, limit) } } @@ -3964,7 +3960,7 @@ func (r *Replica) adminScatter( ri := r.GetRangeInfo(ctx) stats := r.GetMVCCStats() - return roachpb.AdminScatterResponse{ + return kvpb.AdminScatterResponse{ RangeInfos: []roachpb.RangeInfo{ri}, MVCCStats: stats, // Note, we use this replica's MVCCStats to estimate the size of the replicas @@ -3977,8 +3973,8 @@ func (r *Replica) adminScatter( // TODO(arul): AdminVerifyProtectedTimestampRequest can entirely go away in // 22.2. func (r *Replica) adminVerifyProtectedTimestamp( - ctx context.Context, _ roachpb.AdminVerifyProtectedTimestampRequest, -) (resp roachpb.AdminVerifyProtectedTimestampResponse, err error) { + ctx context.Context, _ kvpb.AdminVerifyProtectedTimestampRequest, +) (resp kvpb.AdminVerifyProtectedTimestampResponse, err error) { // AdminVerifyProtectedTimestampRequest is not supported starting from the // 22.1 release. We expect nodes running a 22.1 binary to still service this // request in a {21.2, 22.1} mixed version cluster. This can happen if the diff --git a/pkg/kv/kvserver/replica_command_test.go b/pkg/kv/kvserver/replica_command_test.go index 770e381a05f8..fc6ce2cd3c78 100644 --- a/pkg/kv/kvserver/replica_command_test.go +++ b/pkg/kv/kvserver/replica_command_test.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -132,7 +133,7 @@ func TestValidateReplicationChanges(t *testing.T) { type testCase struct { name string rangeDesc *roachpb.RangeDescriptor - changes roachpb.ReplicationChanges + changes kvpb.ReplicationChanges shouldFail bool expErrorRegex string } @@ -141,21 +142,21 @@ func TestValidateReplicationChanges(t *testing.T) { { name: "add a new voter to another node", rangeDesc: twoVotersAndALearner, - changes: roachpb.ReplicationChanges{ + changes: kvpb.ReplicationChanges{ {ChangeType: roachpb.ADD_VOTER, Target: roachpb.ReplicationTarget{NodeID: 2, StoreID: 2}}, }, }, { name: "remove a voter from an existing node", rangeDesc: twoVotersAndALearner, - changes: roachpb.ReplicationChanges{ + changes: kvpb.ReplicationChanges{ {ChangeType: roachpb.REMOVE_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 1}}, }, }, { name: "remove a voter from the wrong node", rangeDesc: twoVotersAndALearner, - changes: roachpb.ReplicationChanges{ + changes: kvpb.ReplicationChanges{ {ChangeType: roachpb.REMOVE_VOTER, Target: roachpb.ReplicationTarget{NodeID: 2, StoreID: 2}}, }, shouldFail: true, @@ -164,7 +165,7 @@ func TestValidateReplicationChanges(t *testing.T) { { name: "remove a voter from the wrong store", rangeDesc: twoVotersAndALearner, - changes: roachpb.ReplicationChanges{ + changes: kvpb.ReplicationChanges{ {ChangeType: roachpb.REMOVE_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 2}}, }, shouldFail: true, @@ -173,7 +174,7 @@ func TestValidateReplicationChanges(t *testing.T) { { name: "rebalance within a node", rangeDesc: twoVotersAndALearner, - changes: roachpb.ReplicationChanges{ + changes: kvpb.ReplicationChanges{ {ChangeType: roachpb.ADD_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 2}}, {ChangeType: roachpb.REMOVE_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 1}}, }, @@ -181,7 +182,7 @@ func TestValidateReplicationChanges(t *testing.T) { { name: "rebalance within a node but attempt to remove from the wrong one", rangeDesc: twoVotersAndALearner, - changes: roachpb.ReplicationChanges{ + changes: kvpb.ReplicationChanges{ {ChangeType: roachpb.ADD_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 5}}, {ChangeType: roachpb.REMOVE_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 2}}, }, @@ -191,7 +192,7 @@ func TestValidateReplicationChanges(t *testing.T) { { name: "re-add an existing voter", rangeDesc: twoVotersAndALearner, - changes: roachpb.ReplicationChanges{ + changes: kvpb.ReplicationChanges{ {ChangeType: roachpb.ADD_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 1}}, }, shouldFail: true, @@ -200,7 +201,7 @@ func TestValidateReplicationChanges(t *testing.T) { { name: "add voter to a node that already has one", rangeDesc: twoVotersAndALearner, - changes: roachpb.ReplicationChanges{ + changes: kvpb.ReplicationChanges{ {ChangeType: roachpb.ADD_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 2}}, }, shouldFail: true, @@ -209,7 +210,7 @@ func TestValidateReplicationChanges(t *testing.T) { { name: "add non-voter to a store that already has one", rangeDesc: oneVoterAndOneNonVoter, - changes: roachpb.ReplicationChanges{ + changes: kvpb.ReplicationChanges{ {ChangeType: roachpb.ADD_NON_VOTER, Target: roachpb.ReplicationTarget{NodeID: 2, StoreID: 2}}, }, shouldFail: true, @@ -218,7 +219,7 @@ func TestValidateReplicationChanges(t *testing.T) { { name: "add non-voter to a node that already has one", rangeDesc: oneVoterAndOneNonVoter, - changes: roachpb.ReplicationChanges{ + changes: kvpb.ReplicationChanges{ {ChangeType: roachpb.ADD_NON_VOTER, Target: roachpb.ReplicationTarget{NodeID: 2, StoreID: 5}}, }, shouldFail: true, @@ -227,7 +228,7 @@ func TestValidateReplicationChanges(t *testing.T) { { name: "add non-voter to a store that already has a voter", rangeDesc: oneVoterAndOneNonVoter, - changes: roachpb.ReplicationChanges{ + changes: kvpb.ReplicationChanges{ {ChangeType: roachpb.ADD_NON_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 1}}, }, shouldFail: true, @@ -236,7 +237,7 @@ func TestValidateReplicationChanges(t *testing.T) { { name: "try to rebalance within a node, but also add an extra", rangeDesc: twoVotersAndALearner, - changes: roachpb.ReplicationChanges{ + changes: kvpb.ReplicationChanges{ {ChangeType: roachpb.REMOVE_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 1}}, {ChangeType: roachpb.ADD_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 2}}, {ChangeType: roachpb.ADD_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 5}}, @@ -247,7 +248,7 @@ func TestValidateReplicationChanges(t *testing.T) { { name: "try to add twice to the same node", rangeDesc: twoVotersAndALearner, - changes: roachpb.ReplicationChanges{ + changes: kvpb.ReplicationChanges{ {ChangeType: roachpb.ADD_VOTER, Target: roachpb.ReplicationTarget{NodeID: 5, StoreID: 6}}, {ChangeType: roachpb.ADD_VOTER, Target: roachpb.ReplicationTarget{NodeID: 5, StoreID: 5}}, }, @@ -257,7 +258,7 @@ func TestValidateReplicationChanges(t *testing.T) { { name: "try to remove twice from the same store, while the range only has 1 replica", rangeDesc: oneReplica, - changes: roachpb.ReplicationChanges{ + changes: kvpb.ReplicationChanges{ {ChangeType: roachpb.REMOVE_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 1}}, {ChangeType: roachpb.REMOVE_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 1}}, }, @@ -267,7 +268,7 @@ func TestValidateReplicationChanges(t *testing.T) { { name: "try to remove twice from the same node", rangeDesc: twoVotersAndALearner, - changes: roachpb.ReplicationChanges{ + changes: kvpb.ReplicationChanges{ {ChangeType: roachpb.REMOVE_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 1}}, {ChangeType: roachpb.REMOVE_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 2}}, }, @@ -276,7 +277,7 @@ func TestValidateReplicationChanges(t *testing.T) { { name: "try to add on a node that already has a learner", rangeDesc: twoVotersAndALearner, - changes: roachpb.ReplicationChanges{ + changes: kvpb.ReplicationChanges{ {ChangeType: roachpb.ADD_VOTER, Target: roachpb.ReplicationTarget{NodeID: 4, StoreID: 5}}, }, shouldFail: true, @@ -285,7 +286,7 @@ func TestValidateReplicationChanges(t *testing.T) { { name: "add/remove multiple replicas", rangeDesc: twoVotersAndALearner, - changes: roachpb.ReplicationChanges{ + changes: kvpb.ReplicationChanges{ {ChangeType: roachpb.ADD_VOTER, Target: roachpb.ReplicationTarget{NodeID: 2, StoreID: 2}}, {ChangeType: roachpb.ADD_VOTER, Target: roachpb.ReplicationTarget{NodeID: 5, StoreID: 5}}, {ChangeType: roachpb.ADD_VOTER, Target: roachpb.ReplicationTarget{NodeID: 6, StoreID: 6}}, @@ -300,21 +301,21 @@ func TestValidateReplicationChanges(t *testing.T) { // Regression test for #60545. name: "remove a learner from a node that has two replicas", rangeDesc: twoReplicasOnOneNode, - changes: roachpb.ReplicationChanges{ + changes: kvpb.ReplicationChanges{ {ChangeType: roachpb.REMOVE_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 3}}, }, }, { name: "remove a voter from a node that has two replicas", rangeDesc: twoReplicasOnOneNode, - changes: roachpb.ReplicationChanges{ + changes: kvpb.ReplicationChanges{ {ChangeType: roachpb.REMOVE_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 1}}, }, }, { name: "remove a replica with the wrong type from a node that has two replicas", rangeDesc: twoReplicasOnOneNode, - changes: roachpb.ReplicationChanges{ + changes: kvpb.ReplicationChanges{ {ChangeType: roachpb.REMOVE_NON_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 2}}, }, shouldFail: true, @@ -323,14 +324,14 @@ func TestValidateReplicationChanges(t *testing.T) { { name: "add to a different node while one node is in the midst of a lateral rebalance", rangeDesc: twoReplicasOnOneNode, - changes: roachpb.ReplicationChanges{ + changes: kvpb.ReplicationChanges{ {ChangeType: roachpb.ADD_VOTER, Target: roachpb.ReplicationTarget{NodeID: 4, StoreID: 4}}, }, }, { name: "add-remove to a node that is in the middle of a lateral rebalance", rangeDesc: twoReplicasOnOneNode, - changes: roachpb.ReplicationChanges{ + changes: kvpb.ReplicationChanges{ {ChangeType: roachpb.ADD_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 5}}, {ChangeType: roachpb.REMOVE_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 3}}, }, @@ -340,7 +341,7 @@ func TestValidateReplicationChanges(t *testing.T) { { name: "remove two replicas from a node that is in the middle of a lateral rebalance", rangeDesc: twoReplicasOnOneNode, - changes: roachpb.ReplicationChanges{ + changes: kvpb.ReplicationChanges{ {ChangeType: roachpb.REMOVE_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 1}}, {ChangeType: roachpb.REMOVE_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 3}}, }, @@ -350,7 +351,7 @@ func TestValidateReplicationChanges(t *testing.T) { { name: "remove then add within a node", rangeDesc: twoVotersAndALearner, - changes: roachpb.ReplicationChanges{ + changes: kvpb.ReplicationChanges{ {ChangeType: roachpb.REMOVE_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 1}}, {ChangeType: roachpb.ADD_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 2}}, }, @@ -360,7 +361,7 @@ func TestValidateReplicationChanges(t *testing.T) { { name: "add to a node when we only have one replica", rangeDesc: oneReplica, - changes: roachpb.ReplicationChanges{ + changes: kvpb.ReplicationChanges{ {ChangeType: roachpb.ADD_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 2}}, }, }, @@ -368,7 +369,7 @@ func TestValidateReplicationChanges(t *testing.T) { name: "adding a non-voter where we already have a voting replica, without an accompanying" + " removal of that voter", rangeDesc: oneReplica, - changes: roachpb.ReplicationChanges{ + changes: kvpb.ReplicationChanges{ {ChangeType: roachpb.ADD_NON_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 1}}, }, shouldFail: true, @@ -377,7 +378,7 @@ func TestValidateReplicationChanges(t *testing.T) { { name: "voter demotion", rangeDesc: oneReplica, - changes: roachpb.ReplicationChanges{ + changes: kvpb.ReplicationChanges{ {ChangeType: roachpb.ADD_NON_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 1}}, {ChangeType: roachpb.REMOVE_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 1}}, }, @@ -385,7 +386,7 @@ func TestValidateReplicationChanges(t *testing.T) { { name: "non-voter promotion", rangeDesc: oneVoterAndOneNonVoter, - changes: roachpb.ReplicationChanges{ + changes: kvpb.ReplicationChanges{ {ChangeType: roachpb.ADD_VOTER, Target: roachpb.ReplicationTarget{NodeID: 2, StoreID: 2}}, {ChangeType: roachpb.REMOVE_NON_VOTER, Target: roachpb.ReplicationTarget{NodeID: 2, StoreID: 2}}, }, @@ -393,7 +394,7 @@ func TestValidateReplicationChanges(t *testing.T) { { name: "swapping voter with non-voter", rangeDesc: oneVoterAndOneNonVoter, - changes: roachpb.ReplicationChanges{ + changes: kvpb.ReplicationChanges{ {ChangeType: roachpb.ADD_NON_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 1}}, {ChangeType: roachpb.REMOVE_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 1}}, {ChangeType: roachpb.ADD_VOTER, Target: roachpb.ReplicationTarget{NodeID: 2, StoreID: 2}}, @@ -403,7 +404,7 @@ func TestValidateReplicationChanges(t *testing.T) { { name: "trying to promote a non-voter that doesnt exist", rangeDesc: oneVoterAndOneNonVoter, - changes: roachpb.ReplicationChanges{ + changes: kvpb.ReplicationChanges{ {ChangeType: roachpb.ADD_VOTER, Target: roachpb.ReplicationTarget{NodeID: 3, StoreID: 3}}, {ChangeType: roachpb.REMOVE_NON_VOTER, Target: roachpb.ReplicationTarget{NodeID: 3, StoreID: 3}}, }, @@ -428,7 +429,7 @@ func TestSynthesizeTargetsByChangeType(t *testing.T) { defer leaktest.AfterTest(t)() type testCase struct { name string - changes []roachpb.ReplicationChange + changes []kvpb.ReplicationChange expPromotions, expDemotions []int32 expVoterAdditions, expVoterRemovals []int32 expNonVoterAdditions, expNonVoterRemovals []int32 @@ -454,35 +455,35 @@ func TestSynthesizeTargetsByChangeType(t *testing.T) { tests := []testCase{ { name: "simple voter addition", - changes: []roachpb.ReplicationChange{ + changes: []kvpb.ReplicationChange{ {ChangeType: roachpb.ADD_VOTER, Target: mkTarget(2)}, }, expVoterAdditions: []int32{2}, }, { name: "simple voter removal", - changes: []roachpb.ReplicationChange{ + changes: []kvpb.ReplicationChange{ {ChangeType: roachpb.REMOVE_VOTER, Target: mkTarget(2)}, }, expVoterRemovals: []int32{2}, }, { name: "simple non-voter addition", - changes: []roachpb.ReplicationChange{ + changes: []kvpb.ReplicationChange{ {ChangeType: roachpb.ADD_NON_VOTER, Target: mkTarget(2)}, }, expNonVoterAdditions: []int32{2}, }, { name: "simple non-voter removal", - changes: []roachpb.ReplicationChange{ + changes: []kvpb.ReplicationChange{ {ChangeType: roachpb.REMOVE_NON_VOTER, Target: mkTarget(2)}, }, expNonVoterRemovals: []int32{2}, }, { name: "promote non_voter to voter", - changes: []roachpb.ReplicationChange{ + changes: []kvpb.ReplicationChange{ {ChangeType: roachpb.ADD_VOTER, Target: mkTarget(2)}, {ChangeType: roachpb.REMOVE_NON_VOTER, Target: mkTarget(2)}, }, @@ -490,7 +491,7 @@ func TestSynthesizeTargetsByChangeType(t *testing.T) { }, { name: "demote voter to non_voter", - changes: []roachpb.ReplicationChange{ + changes: []kvpb.ReplicationChange{ {ChangeType: roachpb.ADD_NON_VOTER, Target: mkTarget(1)}, {ChangeType: roachpb.REMOVE_VOTER, Target: mkTarget(1)}, }, @@ -498,7 +499,7 @@ func TestSynthesizeTargetsByChangeType(t *testing.T) { }, { name: "swap voter with non_voter", - changes: []roachpb.ReplicationChange{ + changes: []kvpb.ReplicationChange{ {ChangeType: roachpb.ADD_NON_VOTER, Target: mkTarget(1)}, {ChangeType: roachpb.REMOVE_VOTER, Target: mkTarget(1)}, {ChangeType: roachpb.ADD_VOTER, Target: mkTarget(2)}, @@ -509,7 +510,7 @@ func TestSynthesizeTargetsByChangeType(t *testing.T) { }, { name: "swap with simple addition", - changes: []roachpb.ReplicationChange{ + changes: []kvpb.ReplicationChange{ {ChangeType: roachpb.ADD_NON_VOTER, Target: mkTarget(1)}, {ChangeType: roachpb.REMOVE_VOTER, Target: mkTarget(1)}, {ChangeType: roachpb.ADD_VOTER, Target: mkTarget(2)}, @@ -522,7 +523,7 @@ func TestSynthesizeTargetsByChangeType(t *testing.T) { }, { name: "swap with simple removal", - changes: []roachpb.ReplicationChange{ + changes: []kvpb.ReplicationChange{ {ChangeType: roachpb.ADD_NON_VOTER, Target: mkTarget(1)}, {ChangeType: roachpb.REMOVE_VOTER, Target: mkTarget(1)}, {ChangeType: roachpb.ADD_VOTER, Target: mkTarget(2)}, @@ -535,7 +536,7 @@ func TestSynthesizeTargetsByChangeType(t *testing.T) { }, { name: "swap with addition promotion", - changes: []roachpb.ReplicationChange{ + changes: []kvpb.ReplicationChange{ {ChangeType: roachpb.ADD_NON_VOTER, Target: mkTarget(1)}, {ChangeType: roachpb.REMOVE_VOTER, Target: mkTarget(1)}, {ChangeType: roachpb.ADD_VOTER, Target: mkTarget(2)}, @@ -548,7 +549,7 @@ func TestSynthesizeTargetsByChangeType(t *testing.T) { }, { name: "swap with additional demotion", - changes: []roachpb.ReplicationChange{ + changes: []kvpb.ReplicationChange{ {ChangeType: roachpb.ADD_NON_VOTER, Target: mkTarget(1)}, {ChangeType: roachpb.REMOVE_VOTER, Target: mkTarget(1)}, {ChangeType: roachpb.ADD_VOTER, Target: mkTarget(2)}, @@ -561,7 +562,7 @@ func TestSynthesizeTargetsByChangeType(t *testing.T) { }, { name: "two swaps", - changes: []roachpb.ReplicationChange{ + changes: []kvpb.ReplicationChange{ {ChangeType: roachpb.ADD_NON_VOTER, Target: mkTarget(1)}, {ChangeType: roachpb.REMOVE_VOTER, Target: mkTarget(1)}, {ChangeType: roachpb.ADD_VOTER, Target: mkTarget(2)}, @@ -576,7 +577,7 @@ func TestSynthesizeTargetsByChangeType(t *testing.T) { }, { name: "all at once", - changes: []roachpb.ReplicationChange{ + changes: []kvpb.ReplicationChange{ {ChangeType: roachpb.ADD_NON_VOTER, Target: mkTarget(1)}, {ChangeType: roachpb.REMOVE_VOTER, Target: mkTarget(1)}, {ChangeType: roachpb.ADD_VOTER, Target: mkTarget(2)}, diff --git a/pkg/kv/kvserver/replica_consistency.go b/pkg/kv/kvserver/replica_consistency.go index d769def1164c..28071b29bb31 100644 --- a/pkg/kv/kvserver/replica_consistency.go +++ b/pkg/kv/kvserver/replica_consistency.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" @@ -78,26 +79,26 @@ type replicaChecksum struct { // terminate suspicious nodes. This behavior should be lifted to the consistency // checker queue in the future. func (r *Replica) CheckConsistency( - ctx context.Context, req roachpb.CheckConsistencyRequest, -) (roachpb.CheckConsistencyResponse, *roachpb.Error) { - return r.checkConsistencyImpl(ctx, roachpb.ComputeChecksumRequest{ - RequestHeader: roachpb.RequestHeader{Key: r.Desc().StartKey.AsRawKey()}, + ctx context.Context, req kvpb.CheckConsistencyRequest, +) (kvpb.CheckConsistencyResponse, *kvpb.Error) { + return r.checkConsistencyImpl(ctx, kvpb.ComputeChecksumRequest{ + RequestHeader: kvpb.RequestHeader{Key: r.Desc().StartKey.AsRawKey()}, Version: batcheval.ReplicaChecksumVersion, Mode: req.Mode, }) } func (r *Replica) checkConsistencyImpl( - ctx context.Context, args roachpb.ComputeChecksumRequest, -) (roachpb.CheckConsistencyResponse, *roachpb.Error) { - isQueue := args.Mode == roachpb.ChecksumMode_CHECK_VIA_QUEUE + ctx context.Context, args kvpb.ComputeChecksumRequest, +) (kvpb.CheckConsistencyResponse, *kvpb.Error) { + isQueue := args.Mode == kvpb.ChecksumMode_CHECK_VIA_QUEUE results, err := r.runConsistencyCheck(ctx, args) if err != nil { - return roachpb.CheckConsistencyResponse{}, roachpb.NewError(err) + return kvpb.CheckConsistencyResponse{}, kvpb.NewError(err) } - res := roachpb.CheckConsistencyResponse_Result{RangeID: r.RangeID} + res := kvpb.CheckConsistencyResponse_Result{RangeID: r.RangeID} shaToIdxs := map[string][]int{} var missing []ConsistencyCheckResult @@ -177,27 +178,27 @@ func (r *Replica) checkConsistencyImpl( } res.StartKey = []byte(args.Key) - res.Status = roachpb.CheckConsistencyResponse_RANGE_CONSISTENT + res.Status = kvpb.CheckConsistencyResponse_RANGE_CONSISTENT if minoritySHA != "" { - res.Status = roachpb.CheckConsistencyResponse_RANGE_INCONSISTENT - } else if args.Mode != roachpb.ChecksumMode_CHECK_STATS && haveDelta { + res.Status = kvpb.CheckConsistencyResponse_RANGE_INCONSISTENT + } else if args.Mode != kvpb.ChecksumMode_CHECK_STATS && haveDelta { if delta.ContainsEstimates > 0 { // When ContainsEstimates is set, it's generally expected that we'll get a different // result when we recompute from scratch. - res.Status = roachpb.CheckConsistencyResponse_RANGE_CONSISTENT_STATS_ESTIMATED + res.Status = kvpb.CheckConsistencyResponse_RANGE_CONSISTENT_STATS_ESTIMATED } else { // When ContainsEstimates is unset, we expect the recomputation to agree with the stored stats. // If that's not the case, that's a problem: it could be a bug in the stats computation // or stats maintenance, but it could also hint at the replica having diverged from its peers. - res.Status = roachpb.CheckConsistencyResponse_RANGE_CONSISTENT_STATS_INCORRECT + res.Status = kvpb.CheckConsistencyResponse_RANGE_CONSISTENT_STATS_INCORRECT } res.Detail += fmt.Sprintf("delta (stats-computed): %+v\n", enginepb.MVCCStats(results[0].Response.Delta)) } else if len(missing) > 0 { // No inconsistency was detected, but we didn't manage to inspect all replicas. - res.Status = roachpb.CheckConsistencyResponse_RANGE_INDETERMINATE + res.Status = kvpb.CheckConsistencyResponse_RANGE_INDETERMINATE } - var resp roachpb.CheckConsistencyResponse + var resp kvpb.CheckConsistencyResponse resp.Result = append(resp.Result, res) // Bail out at this point except if the queue is the caller. All of the stuff @@ -232,11 +233,11 @@ func (r *Replica) checkConsistencyImpl( log.Infof(ctx, "triggering stats recomputation to resolve delta of %+v", results[0].Response.Delta) var b kv.Batch - b.AddRawRequest(&roachpb.RecomputeStatsRequest{ - RequestHeader: roachpb.RequestHeader{Key: args.Key}, + b.AddRawRequest(&kvpb.RecomputeStatsRequest{ + RequestHeader: kvpb.RequestHeader{Key: args.Key}, }) err := r.store.db.Run(ctx, &b) - return resp, roachpb.NewError(err) + return resp, kvpb.NewError(err) } if args.Checkpoint { @@ -314,7 +315,7 @@ func (r *Replica) collectChecksumFromReplica( // upon). Requires that the computation succeeds on at least one replica, and // puts an arbitrary successful result first in the returned slice. func (r *Replica) runConsistencyCheck( - ctx context.Context, req roachpb.ComputeChecksumRequest, + ctx context.Context, req kvpb.ComputeChecksumRequest, ) ([]ConsistencyCheckResult, error) { // Send a ComputeChecksum which will trigger computation of the checksum on // all replicas. @@ -322,7 +323,7 @@ func (r *Replica) runConsistencyCheck( if pErr != nil { return nil, pErr.GoError() } - ccRes := res.(*roachpb.ComputeChecksumResponse) + ccRes := res.(*kvpb.ComputeChecksumResponse) replicas := r.Desc().Replicas().Descriptors() resultCh := make(chan ConsistencyCheckResult, len(replicas)) @@ -495,10 +496,10 @@ func CalcReplicaDigest( ctx context.Context, desc roachpb.RangeDescriptor, snap storage.Reader, - mode roachpb.ChecksumMode, + mode kvpb.ChecksumMode, limiter *quotapool.RateLimiter, ) (*ReplicaDigest, error) { - statsOnly := mode == roachpb.ChecksumMode_CHECK_STATS + statsOnly := mode == kvpb.ChecksumMode_CHECK_STATS // Iterate over all the data in the range. var intBuf [8]byte diff --git a/pkg/kv/kvserver/replica_consistency_test.go b/pkg/kv/kvserver/replica_consistency_test.go index 0a3f545f824f..b813c959ae07 100644 --- a/pkg/kv/kvserver/replica_consistency_test.go +++ b/pkg/kv/kvserver/replica_consistency_test.go @@ -17,6 +17,7 @@ import ( "testing" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -48,7 +49,7 @@ func TestReplicaChecksumVersion(t *testing.T) { testutils.RunTrueAndFalse(t, "matchingVersion", func(t *testing.T, matchingVersion bool) { cc := kvserverpb.ComputeChecksum{ ChecksumID: uuid.FastMakeV4(), - Mode: roachpb.ChecksumMode_CHECK_FULL, + Mode: kvpb.ChecksumMode_CHECK_FULL, } if matchingVersion { cc.Version = batcheval.ReplicaChecksumVersion @@ -183,7 +184,7 @@ func TestGetChecksumNotSuccessfulExitConditions(t *testing.T) { startChecksumTask := func(ctx context.Context, id uuid.UUID) error { return tc.repl.computeChecksumPostApply(ctx, kvserverpb.ComputeChecksum{ ChecksumID: id, - Mode: roachpb.ChecksumMode_CHECK_FULL, + Mode: kvpb.ChecksumMode_CHECK_FULL, Version: batcheval.ReplicaChecksumVersion, }) } @@ -274,7 +275,7 @@ func TestReplicaChecksumSHA512(t *testing.T) { // Hash the empty state. unlim := quotapool.NewRateLimiter("test", quotapool.Inf(), 0) - rd, err := CalcReplicaDigest(ctx, desc, eng, roachpb.ChecksumMode_CHECK_FULL, unlim) + rd, err := CalcReplicaDigest(ctx, desc, eng, kvpb.ChecksumMode_CHECK_FULL, unlim) require.NoError(t, err) fmt.Fprintf(sb, "checksum0: %x\n", rd.SHA512) @@ -312,13 +313,13 @@ func TestReplicaChecksumSHA512(t *testing.T) { require.NoError(t, storage.MVCCPut(ctx, eng, nil, key, ts, localTS, value, nil)) } - rd, err = CalcReplicaDigest(ctx, desc, eng, roachpb.ChecksumMode_CHECK_FULL, unlim) + rd, err = CalcReplicaDigest(ctx, desc, eng, kvpb.ChecksumMode_CHECK_FULL, unlim) require.NoError(t, err) fmt.Fprintf(sb, "checksum%d: %x\n", i+1, rd.SHA512) } // Run another check to obtain stats for the final state. - rd, err = CalcReplicaDigest(ctx, desc, eng, roachpb.ChecksumMode_CHECK_FULL, unlim) + rd, err = CalcReplicaDigest(ctx, desc, eng, kvpb.ChecksumMode_CHECK_FULL, unlim) require.NoError(t, err) jsonpb := protoutil.JSONPb{Indent: " "} json, err := jsonpb.Marshal(&rd.RecomputedMS) diff --git a/pkg/kv/kvserver/replica_corruption.go b/pkg/kv/kvserver/replica_corruption.go index 9b4486808f10..c1d500bae05a 100644 --- a/pkg/kv/kvserver/replica_corruption.go +++ b/pkg/kv/kvserver/replica_corruption.go @@ -15,7 +15,7 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/storage/fs" "github.com/cockroachdb/cockroach/pkg/util/log" ) @@ -38,8 +38,8 @@ import ( // @bdarnell remarks: Corruption errors should be rare so we may want the store // to just recompute its stats in the background when one occurs. func (r *Replica) setCorruptRaftMuLocked( - ctx context.Context, cErr *roachpb.ReplicaCorruptionError, -) *roachpb.Error { + ctx context.Context, cErr *kvpb.ReplicaCorruptionError, +) *kvpb.Error { r.readOnlyCmdMu.Lock() defer r.readOnlyCmdMu.Unlock() r.mu.Lock() @@ -68,5 +68,5 @@ A file preventing this node from restarting was placed at: } log.FatalfDepth(ctx, 1, "replica is corrupted: %s", cErr) - return roachpb.NewError(cErr) + return kvpb.NewError(cErr) } diff --git a/pkg/kv/kvserver/replica_destroy.go b/pkg/kv/kvserver/replica_destroy.go index d24dcc032665..022c72d50145 100644 --- a/pkg/kv/kvserver/replica_destroy.go +++ b/pkg/kv/kvserver/replica_destroy.go @@ -15,6 +15,7 @@ import ( "fmt" "math" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/apply" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvstorage" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -179,8 +180,8 @@ func (r *Replica) disconnectReplicationRaftMuLocked(ctx context.Context) { // NB: each proposal needs its own version of the error (i.e. don't try to // share the error across proposals). p.finishApplication(ctx, proposalResult{ - Err: roachpb.NewError( - roachpb.NewAmbiguousResultError(apply.ErrRemoved)), + Err: kvpb.NewError( + kvpb.NewAmbiguousResultError(apply.ErrRemoved)), }) } r.mu.internalRaftGroup = nil diff --git a/pkg/kv/kvserver/replica_eval_context_span.go b/pkg/kv/kvserver/replica_eval_context_span.go index f626b10e5f9c..40f86e4e9372 100644 --- a/pkg/kv/kvserver/replica_eval_context_span.go +++ b/pkg/kv/kvserver/replica_eval_context_span.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/abortspan" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" @@ -146,7 +147,7 @@ func (rec SpanSetReplicaEvalContext) GetMaxSplitCPU(ctx context.Context) (float6 // for details about its arguments, return values, and preconditions. func (rec SpanSetReplicaEvalContext) CanCreateTxnRecord( ctx context.Context, txnID uuid.UUID, txnKey []byte, txnMinTS hlc.Timestamp, -) (bool, roachpb.TransactionAbortedReason) { +) (bool, kvpb.TransactionAbortedReason) { rec.ss.AssertAllowed(spanset.SpanReadOnly, roachpb.Span{Key: keys.TransactionKey(txnKey, txnID)}, ) diff --git a/pkg/kv/kvserver/replica_evaluate.go b/pkg/kv/kvserver/replica_evaluate.go index 9b838c7efa23..7fa8d4eba5fc 100644 --- a/pkg/kv/kvserver/replica_evaluate.go +++ b/pkg/kv/kvserver/replica_evaluate.go @@ -16,6 +16,7 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" @@ -42,8 +43,8 @@ import ( // the input slice, or has been shallow-copied appropriately to avoid // mutating the original requests). func optimizePuts( - reader storage.Reader, origReqs []roachpb.RequestUnion, distinctSpans bool, -) []roachpb.RequestUnion { + reader storage.Reader, origReqs []kvpb.RequestUnion, distinctSpans bool, +) []kvpb.RequestUnion { var minKey, maxKey roachpb.Key var unique map[string]struct{} if !distinctSpans { @@ -70,15 +71,15 @@ func optimizePuts( firstUnoptimizedIndex := len(origReqs) for i, r := range origReqs { switch t := r.GetInner().(type) { - case *roachpb.PutRequest: + case *kvpb.PutRequest: if maybeAddPut(t.Key) { continue } - case *roachpb.ConditionalPutRequest: + case *kvpb.ConditionalPutRequest: if maybeAddPut(t.Key) { continue } - case *roachpb.InitPutRequest: + case *kvpb.InitPutRequest: if maybeAddPut(t.Key) { continue } @@ -117,20 +118,20 @@ func optimizePuts( } // Set the prefix of the run which is being written to virgin // keyspace to "blindly" put values. - reqs := append([]roachpb.RequestUnion(nil), origReqs...) + reqs := append([]kvpb.RequestUnion(nil), origReqs...) for i := range reqs[:firstUnoptimizedIndex] { inner := reqs[i].GetInner() if iterKey == nil || bytes.Compare(iterKey, inner.Header().Key) > 0 { switch t := inner.(type) { - case *roachpb.PutRequest: + case *kvpb.PutRequest: shallow := *t shallow.Blind = true reqs[i].MustSetInner(&shallow) - case *roachpb.ConditionalPutRequest: + case *kvpb.ConditionalPutRequest: shallow := *t shallow.Blind = true reqs[i].MustSetInner(&shallow) - case *roachpb.InitPutRequest: + case *kvpb.InitPutRequest: shallow := *t shallow.Blind = true reqs[i].MustSetInner(&shallow) @@ -151,12 +152,12 @@ func evaluateBatch( readWriter storage.ReadWriter, rec batcheval.EvalContext, ms *enginepb.MVCCStats, - ba *roachpb.BatchRequest, + ba *kvpb.BatchRequest, g *concurrency.Guard, st *kvserverpb.LeaseStatus, ui uncertainty.Interval, evalPath batchEvalPath, -) (_ *roachpb.BatchResponse, _ result.Result, retErr *roachpb.Error) { +) (_ *kvpb.BatchResponse, _ result.Result, retErr *kvpb.Error) { defer func() { // Ensure that errors don't carry the WriteTooOld flag set. The client // handles non-error responses with the WriteTooOld flag set, and errors @@ -226,7 +227,7 @@ func evaluateBatch( // WriteTooOldError in order to find out if there's more conflicts and chose // a final write timestamp. var writeTooOldState struct { - err *roachpb.WriteTooOldError + err *kvpb.WriteTooOldError // cantDeferWTOE is set when a WriteTooOldError cannot be deferred past the // end of the current batch. cantDeferWTOE bool @@ -313,14 +314,14 @@ func evaluateBatch( if err != nil { // If an EndTxn wants to restart because of a write too old, we // might have a better error to return to the client. - if retErr := (*roachpb.TransactionRetryError)(nil); errors.As(err, &retErr) && - retErr.Reason == roachpb.RETRY_WRITE_TOO_OLD && - args.Method() == roachpb.EndTxn && writeTooOldState.err != nil { + if retErr := (*kvpb.TransactionRetryError)(nil); errors.As(err, &retErr) && + retErr.Reason == kvpb.RETRY_WRITE_TOO_OLD && + args.Method() == kvpb.EndTxn && writeTooOldState.err != nil { err = writeTooOldState.err // Don't defer this error. We could perhaps rely on the client observing // the WriteTooOld flag and retry the batch, but we choose not too. writeTooOldState.cantDeferWTOE = true - } else if wtoErr := (*roachpb.WriteTooOldError)(nil); errors.As(err, &wtoErr) { + } else if wtoErr := (*kvpb.WriteTooOldError)(nil); errors.As(err, &wtoErr) { // We got a WriteTooOldError. We continue on to run all // commands in the batch in order to determine the highest // timestamp for more efficient retries. If the batch is @@ -361,7 +362,7 @@ func evaluateBatch( // speculative result, or leave behind an unreplicated lock that won't // prevent the request for evaluating again at the same sequence number // but at a bumped timestamp. - if !roachpb.IsBlindWrite(args) { + if !kvpb.IsBlindWrite(args) { writeTooOldState.cantDeferWTOE = true } @@ -395,7 +396,7 @@ func evaluateBatch( } if err != nil { - pErr := roachpb.NewErrorWithTxn(err, baHeader.Txn) + pErr := kvpb.NewErrorWithTxn(err, baHeader.Txn) // Initialize the error index. pErr.SetErrorIndex(int32(index)) @@ -423,7 +424,7 @@ func evaluateBatch( // limit). We have to check the ResumeReason as well, since e.g. a Scan // response may not include the value that pushed it across the limit. if baHeader.TargetBytes > 0 { - if h.ResumeReason == roachpb.RESUME_BYTE_LIMIT { + if h.ResumeReason == kvpb.RESUME_BYTE_LIMIT { baHeader.TargetBytes = -1 } else if baHeader.TargetBytes > h.NumBytes { baHeader.TargetBytes -= h.NumBytes @@ -442,7 +443,7 @@ func evaluateBatch( if writeTooOldState.cantDeferWTOE { // NB: we can't do any error wrapping here yet due to compatibility with 20.2 nodes; // there needs to be an ErrorDetail here. - return nil, mergedResult, roachpb.NewErrorWithTxn(writeTooOldState.err, baHeader.Txn) + return nil, mergedResult, kvpb.NewErrorWithTxn(writeTooOldState.err, baHeader.Txn) } // The batch evaluation will not return an error (i.e. either everything went @@ -470,7 +471,7 @@ func evaluateBatch( } // evaluateCommand delegates to the eval method for the given -// roachpb.Request. The returned Result may be partially valid +// kvpb.Request. The returned Result may be partially valid // even if an error is returned. maxKeys is the number of scan results // remaining for this batch (MaxInt64 for no limit). func evaluateCommand( @@ -478,9 +479,9 @@ func evaluateCommand( readWriter storage.ReadWriter, rec batcheval.EvalContext, ms *enginepb.MVCCStats, - h roachpb.Header, - args roachpb.Request, - reply roachpb.Response, + h kvpb.Header, + args kvpb.Request, + reply kvpb.Response, g *concurrency.Guard, st *kvserverpb.LeaseStatus, ui uncertainty.Interval, @@ -559,9 +560,9 @@ func evaluateCommand( // timestamp. func canDoServersideRetry( ctx context.Context, - pErr *roachpb.Error, - ba *roachpb.BatchRequest, - br *roachpb.BatchResponse, + pErr *kvpb.Error, + ba *kvpb.BatchRequest, + br *kvpb.BatchResponse, g *concurrency.Guard, deadline hlc.Timestamp, ) bool { @@ -572,8 +573,8 @@ func canDoServersideRetry( if !deadline.IsEmpty() { log.Fatal(ctx, "deadline passed for transactional request") } - if etArg, ok := ba.GetArg(roachpb.EndTxn); ok { - et := etArg.(*roachpb.EndTxnRequest) + if etArg, ok := ba.GetArg(kvpb.EndTxn); ok { + et := etArg.(*kvpb.EndTxnRequest) deadline = et.Deadline } } @@ -582,7 +583,7 @@ func canDoServersideRetry( if ba.Txn != nil { if pErr != nil { var ok bool - ok, newTimestamp = roachpb.TransactionRefreshTimestamp(pErr) + ok, newTimestamp = kvpb.TransactionRefreshTimestamp(pErr) if !ok { return false } @@ -597,10 +598,10 @@ func canDoServersideRetry( log.Fatalf(ctx, "canDoServersideRetry called for non-txn request without error") } switch tErr := pErr.GetDetail().(type) { - case *roachpb.WriteTooOldError: + case *kvpb.WriteTooOldError: newTimestamp = tErr.RetryTimestamp() - case *roachpb.ReadWithinUncertaintyIntervalError: + case *kvpb.ReadWithinUncertaintyIntervalError: newTimestamp = tErr.RetryTimestamp() default: @@ -619,19 +620,19 @@ func canDoServersideRetry( // table first), bump the ts cache, release latches and then proceed with // evaluation. Only non-locking read requests that aren't being evaluated under // the `OptimisticEval` path are eligible for this optimization. -func canReadOnlyRequestDropLatchesBeforeEval(ba *roachpb.BatchRequest, g *concurrency.Guard) bool { +func canReadOnlyRequestDropLatchesBeforeEval(ba *kvpb.BatchRequest, g *concurrency.Guard) bool { if g == nil { // NB: A nil guard indicates that the caller is not holding latches. return false } switch ba.Header.ReadConsistency { - case roachpb.CONSISTENT: + case kvpb.CONSISTENT: // TODO(aayush): INCONSISTENT and READ_UNCOMMITTED reads do not care about // resolving lock conflicts at all. Yet, they can still drop latches early and // evaluate once they've pinned their pebble engine state. We should consider // supporting this by letting these kinds of requests drop latches early while // also skipping the initial validation step of scanning the lock table. - case roachpb.INCONSISTENT, roachpb.READ_UNCOMMITTED: + case kvpb.INCONSISTENT, kvpb.READ_UNCOMMITTED: return false default: panic(fmt.Sprintf("unexpected ReadConsistency: %s", ba.Header.ReadConsistency)) @@ -670,7 +671,7 @@ func canReadOnlyRequestDropLatchesBeforeEval(ba *roachpb.BatchRequest, g *concur for _, req := range ba.Requests { inner := req.GetInner() switch inner.(type) { - case *roachpb.ExportRequest, *roachpb.GetRequest, *roachpb.ScanRequest, *roachpb.ReverseScanRequest: + case *kvpb.ExportRequest, *kvpb.GetRequest, *kvpb.ScanRequest, *kvpb.ReverseScanRequest: default: return false } diff --git a/pkg/kv/kvserver/replica_evaluate_test.go b/pkg/kv/kvserver/replica_evaluate_test.go index 31f0241e2eba..0ea68f2354cc 100644 --- a/pkg/kv/kvserver/replica_evaluate_test.go +++ b/pkg/kv/kvserver/replica_evaluate_test.go @@ -15,6 +15,7 @@ import ( "fmt" "testing" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/abortspan" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" @@ -698,7 +699,7 @@ func TestEvaluateBatch(t *testing.T) { type data struct { batcheval.MockEvalCtx - ba roachpb.BatchRequest + ba kvpb.BatchRequest idKey kvserverbase.CmdIDKey eng storage.Engine ms enginepb.MVCCStats @@ -707,9 +708,9 @@ type data struct { type resp struct { d *data - br *roachpb.BatchResponse + br *kvpb.BatchResponse res result.Result - pErr *roachpb.Error + pErr *kvpb.Error } type testCase struct { @@ -747,11 +748,11 @@ func verifyScanResult(t *testing.T, r resp, keysPerResp ...[]string) { scan := r.br.Responses[i].GetInner() var rows []roachpb.KeyValue switch req := scan.(type) { - case *roachpb.ScanResponse: + case *kvpb.ScanResponse: rows = req.Rows - case *roachpb.ReverseScanResponse: + case *kvpb.ReverseScanResponse: rows = req.Rows - case *roachpb.GetResponse: + case *kvpb.GetResponse: if req.Value != nil { rows = []roachpb.KeyValue{{ Key: r.d.ba.Requests[i].GetGet().Key, diff --git a/pkg/kv/kvserver/replica_follower_read.go b/pkg/kv/kvserver/replica_follower_read.go index 15ae927df6b6..93e4600b30d0 100644 --- a/pkg/kv/kvserver/replica_follower_read.go +++ b/pkg/kv/kvserver/replica_follower_read.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/kv/kvbase" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" @@ -36,7 +37,7 @@ var FollowerReadsEnabled = settings.RegisterBoolSetting( // BatchCanBeEvaluatedOnFollower determines if a batch consists exclusively of // requests that can be evaluated on a follower replica, given a sufficiently // advanced closed timestamp. -func BatchCanBeEvaluatedOnFollower(ba *roachpb.BatchRequest) bool { +func BatchCanBeEvaluatedOnFollower(ba *kvpb.BatchRequest) bool { // Various restrictions apply to a batch for it to be successfully considered // for evaluation on a follower replica, which are described inline. // @@ -58,16 +59,16 @@ func BatchCanBeEvaluatedOnFollower(ba *roachpb.BatchRequest) bool { for _, ru := range ba.Requests { r := ru.GetInner() switch { - case roachpb.IsTransactional(r): + case kvpb.IsTransactional(r): // Transactional requests have clear semantics when served under the // closed timestamp. The request must be read-only, as follower replicas // cannot propose writes to Raft. The request also needs to be // non-locking, because unreplicated locks are only held on the // leaseholder. - if !roachpb.IsReadOnly(r) || roachpb.IsLocking(r) { + if !kvpb.IsReadOnly(r) || kvpb.IsLocking(r) { return false } - case r.Method() == roachpb.Export: + case r.Method() == kvpb.Export: // Export requests also have clear semantics when served under the closed // timestamp as well, even though they are non-transactional, as they // define the start and end timestamp to export data over. @@ -83,7 +84,7 @@ func BatchCanBeEvaluatedOnFollower(ba *roachpb.BatchRequest) bool { // non-locking, read-only requests can be served as follower reads. The batch // must be transactional and composed exclusively of this kind of request to be // accepted as a follower read. -func (r *Replica) canServeFollowerReadRLocked(ctx context.Context, ba *roachpb.BatchRequest) bool { +func (r *Replica) canServeFollowerReadRLocked(ctx context.Context, ba *kvpb.BatchRequest) bool { eligible := BatchCanBeEvaluatedOnFollower(ba) && FollowerReadsEnabled.Get(&r.store.cfg.Settings.SV) if !eligible { // We couldn't do anything with the error, propagate it. diff --git a/pkg/kv/kvserver/replica_follower_read_test.go b/pkg/kv/kvserver/replica_follower_read_test.go index a0f09c2039bf..3c10d762b904 100644 --- a/pkg/kv/kvserver/replica_follower_read_test.go +++ b/pkg/kv/kvserver/replica_follower_read_test.go @@ -16,6 +16,7 @@ import ( "testing" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -88,8 +89,8 @@ func TestCanServeFollowerRead(t *testing.T) { 0, // coordinatorNodeID ) - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} ba.Add(&gArgs) r := tc.repl r.mu.RLock() @@ -114,7 +115,7 @@ func TestCheckExecutionCanProceedAllowsFollowerReadWithInvalidLease(t *testing.T // Permit only one lease attempt. The test is flaky if we allow the lease to // be renewed by background processes. var leaseOnce sync.Once - tsc.TestingKnobs.LeaseRequestEvent = func(ts hlc.Timestamp, _ roachpb.StoreID, _ roachpb.RangeID) *roachpb.Error { + tsc.TestingKnobs.LeaseRequestEvent = func(ts hlc.Timestamp, _ roachpb.StoreID, _ roachpb.RangeID) *kvpb.Error { admitted := false leaseOnce.Do(func() { admitted = true @@ -122,7 +123,7 @@ func TestCheckExecutionCanProceedAllowsFollowerReadWithInvalidLease(t *testing.T if admitted { return nil } - return roachpb.NewErrorf("boom") + return kvpb.NewErrorf("boom") } const closedTimestampLag = time.Second closedts.TargetDuration.Override(ctx, &tsc.Settings.SV, closedTimestampLag) @@ -168,8 +169,8 @@ func TestCheckExecutionCanProceedAllowsFollowerReadWithInvalidLease(t *testing.T 0, // coordinatorNodeID ) - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{ + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{ Txn: &txn, Timestamp: txn.ReadTimestamp, } diff --git a/pkg/kv/kvserver/replica_gc_queue.go b/pkg/kv/kvserver/replica_gc_queue.go index fa70f39ac7e0..90bf2cbc405c 100644 --- a/pkg/kv/kvserver/replica_gc_queue.go +++ b/pkg/kv/kvserver/replica_gc_queue.go @@ -15,6 +15,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -233,7 +234,7 @@ func (rgcq *replicaGCQueue) process( // considering one of the metadata ranges: we must not do an inconsistent // lookup in our own copy of the range. rs, _, err := kv.RangeLookup(ctx, rgcq.db.NonTransactionalSender(), desc.StartKey.AsRawKey(), - roachpb.CONSISTENT, 0 /* prefetchNum */, false /* reverse */) + kvpb.CONSISTENT, 0 /* prefetchNum */, false /* reverse */) if err != nil { return false, err } @@ -338,7 +339,7 @@ func (rgcq *replicaGCQueue) process( if leftRepl != nil { leftDesc := leftRepl.Desc() rs, _, err := kv.RangeLookup(ctx, rgcq.db.NonTransactionalSender(), leftDesc.StartKey.AsRawKey(), - roachpb.CONSISTENT, 0 /* prefetchNum */, false /* reverse */) + kvpb.CONSISTENT, 0 /* prefetchNum */, false /* reverse */) if err != nil { return false, err } diff --git a/pkg/kv/kvserver/replica_gossip.go b/pkg/kv/kvserver/replica_gossip.go index 4027140bc8bb..f6330420738d 100644 --- a/pkg/kv/kvserver/replica_gossip.go +++ b/pkg/kv/kvserver/replica_gossip.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/gossip" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/uncertainty" @@ -83,11 +84,11 @@ func (r *Replica) MaybeGossipNodeLivenessRaftMuLocked( return nil } - ba := roachpb.BatchRequest{} + ba := kvpb.BatchRequest{} // Read at the maximum timestamp to ensure that we see the most recent // liveness record, regardless of what timestamp it is written at. ba.Timestamp = hlc.MaxTimestamp - ba.Add(&roachpb.ScanRequest{RequestHeader: roachpb.RequestHeaderFromSpan(span)}) + ba.Add(&kvpb.ScanRequest{RequestHeader: kvpb.RequestHeaderFromSpan(span)}) // Call evaluateBatch instead of Send to avoid reacquiring latches. rec := NewReplicaEvalContext( ctx, r, todoSpanSet, false, /* requiresClosedTSOlderThanStorageSnap */ @@ -107,7 +108,7 @@ func (r *Replica) MaybeGossipNodeLivenessRaftMuLocked( if len(result.Local.EncounteredIntents) > 0 { return errors.Errorf("unexpected intents on node liveness span %s: %+v", span, result.Local.EncounteredIntents) } - kvs := br.Responses[0].GetInner().(*roachpb.ScanResponse).Rows + kvs := br.Responses[0].GetInner().(*kvpb.ScanResponse).Rows log.VEventf(ctx, 2, "gossiping %d node liveness record(s) from span %s", len(kvs), span) for _, kv := range kvs { var kvLiveness, gossipLiveness livenesspb.Liveness @@ -130,13 +131,13 @@ func (r *Replica) MaybeGossipNodeLivenessRaftMuLocked( // getLeaseForGossip tries to obtain a range lease. Only one of the replicas // should gossip; the bool returned indicates whether it's us. -func (r *Replica) getLeaseForGossip(ctx context.Context) (bool, *roachpb.Error) { +func (r *Replica) getLeaseForGossip(ctx context.Context) (bool, *kvpb.Error) { // If no Gossip available (some tests) or range too fresh, noop. if r.store.Gossip() == nil || !r.IsInitialized() { - return false, roachpb.NewErrorf("no gossip or range not initialized") + return false, kvpb.NewErrorf("no gossip or range not initialized") } var hasLease bool - var pErr *roachpb.Error + var pErr *kvpb.Error if err := r.store.Stopper().RunTask( ctx, "storage.Replica: acquiring lease to gossip", func(ctx context.Context) { @@ -145,7 +146,7 @@ func (r *Replica) getLeaseForGossip(ctx context.Context) (bool, *roachpb.Error) hasLease = pErr == nil if pErr != nil { switch e := pErr.GetDetail().(type) { - case *roachpb.NotLeaseHolderError: + case *kvpb.NotLeaseHolderError: // NotLeaseHolderError means there is an active lease, but only if // the lease is non-empty; otherwise, it's likely a timeout. if !e.Lease.Empty() { @@ -157,7 +158,7 @@ func (r *Replica) getLeaseForGossip(ctx context.Context) (bool, *roachpb.Error) } } }); err != nil { - pErr = roachpb.NewError(err) + pErr = kvpb.NewError(err) } return hasLease, pErr } @@ -165,7 +166,7 @@ func (r *Replica) getLeaseForGossip(ctx context.Context) (bool, *roachpb.Error) // maybeGossipFirstRange adds the sentinel and first range metadata to gossip // if this is the first range and a range lease can be obtained. The Store // calls this periodically on first range replicas. -func (r *Replica) maybeGossipFirstRange(ctx context.Context) *roachpb.Error { +func (r *Replica) maybeGossipFirstRange(ctx context.Context) *kvpb.Error { if !r.IsFirstRange() { return nil } diff --git a/pkg/kv/kvserver/replica_learner_test.go b/pkg/kv/kvserver/replica_learner_test.go index 899f066d5fd1..48d22d3acef2 100644 --- a/pkg/kv/kvserver/replica_learner_test.go +++ b/pkg/kv/kvserver/replica_learner_test.go @@ -23,6 +23,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftutil" @@ -294,7 +295,7 @@ func TestAddReplicaWithReceiverThrottling(t *testing.T) { return err } _, err = tc.Servers[0].DB().AdminChangeReplicas(ctx, scratch, desc, - roachpb.MakeReplicationChanges(roachpb.ADD_NON_VOTER, tc.Target(2)), + kvpb.MakeReplicationChanges(roachpb.ADD_NON_VOTER, tc.Target(2)), ) replicationChange <- err return err @@ -318,7 +319,7 @@ func TestAddReplicaWithReceiverThrottling(t *testing.T) { return err } _, err = tc.Servers[0].DB().AdminChangeReplicas( - ctx, scratch, desc, roachpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(1)), + ctx, scratch, desc, kvpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(1)), ) replicationChange <- err return err @@ -473,7 +474,7 @@ func TestDelegateSnapshotFails(t *testing.T) { require.NoError(t, err) _, err = tc.Servers[0].DB().AdminChangeReplicas( - ctx, scratchKey, desc, roachpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(1)), + ctx, scratchKey, desc, kvpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(1)), ) require.True(t, testutils.IsError(err, "partitioned"), `expected partitioned error got: %+v`, err) @@ -502,7 +503,7 @@ func TestDelegateSnapshotFails(t *testing.T) { require.NoError(t, err) _, err = tc.Servers[0].DB().AdminChangeReplicas( - ctx, scratchKey, desc, roachpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(1)), + ctx, scratchKey, desc, kvpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(1)), ) log.Infof(ctx, "Err=%v", err) require.True(t, testutils.IsError(err, "partitioned"), `expected partitioned error got: %+v`, err) @@ -532,7 +533,7 @@ func TestDelegateSnapshotFails(t *testing.T) { require.NoError(t, err) _, err = tc.Servers[0].DB().AdminChangeReplicas( - ctx, scratchKey, desc, roachpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(1)), + ctx, scratchKey, desc, kvpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(1)), ) require.NoError(t, err) }) @@ -950,14 +951,14 @@ func TestSplitRetriesOnFailedExitOfJointConfig(t *testing.T) { var rangeIDAtomic int64 var rejectedCount int const maxRejects = 3 - reqFilter := func(ctx context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + reqFilter := func(ctx context.Context, ba *kvpb.BatchRequest) *kvpb.Error { rangeID := roachpb.RangeID(atomic.LoadInt64(&rangeIDAtomic)) if ba.RangeID == rangeID && ba.IsSingleTransferLeaseRequest() && rejectedCount < maxRejects { rejectedCount++ repl := ba.Requests[0].GetTransferLease().Lease.Replica status := raftutil.ReplicaStateProbe err := kvserver.NewLeaseTransferRejectedBecauseTargetMayNeedSnapshotError(repl, status) - return roachpb.NewError(err) + return kvpb.NewError(err) } return nil } @@ -1205,7 +1206,7 @@ func TestLearnerAdminChangeReplicasRace(t *testing.T) { return err } _, err = tc.Servers[0].DB().AdminChangeReplicas( - ctx, scratchStartKey, desc, roachpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(1)), + ctx, scratchStartKey, desc, kvpb.MakeReplicationChanges(roachpb.ADD_VOTER, tc.Target(1)), ) return err }) @@ -1408,12 +1409,12 @@ func TestLearnerAndVoterOutgoingFollowerRead(t *testing.T) { check := func() { ts := tc.Server(0).Clock().Now() txn := roachpb.MakeTransaction("txn", nil, 0, ts, 0, int32(tc.Server(0).SQLInstanceID())) - req := &roachpb.BatchRequest{Header: roachpb.Header{ + req := &kvpb.BatchRequest{Header: kvpb.Header{ RangeID: scratchDesc.RangeID, Timestamp: ts, Txn: &txn, }} - req.Add(&roachpb.ScanRequest{RequestHeader: roachpb.RequestHeader{ + req.Add(&kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{ Key: scratchDesc.StartKey.AsRawKey(), EndKey: scratchDesc.EndKey.AsRawKey(), }}) diff --git a/pkg/kv/kvserver/replica_probe_test.go b/pkg/kv/kvserver/replica_probe_test.go index 9cf1d784b2fa..47b389c156a2 100644 --- a/pkg/kv/kvserver/replica_probe_test.go +++ b/pkg/kv/kvserver/replica_probe_test.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -39,10 +40,10 @@ func TestReplicaProbeRequest(t *testing.T) { var seen struct { syncutil.Mutex m map[roachpb.StoreID]int - injectedErr *roachpb.Error + injectedErr *kvpb.Error } seen.m = map[roachpb.StoreID]int{} - filter := func(args kvserverbase.ApplyFilterArgs) (int, *roachpb.Error) { + filter := func(args kvserverbase.ApplyFilterArgs) (int, *kvpb.Error) { if !args.IsProbe { return 0, args.ForcedError } @@ -111,8 +112,8 @@ func TestReplicaProbeRequest(t *testing.T) { tc.AddVotersOrFatal(t, k, tc.Target(1)) tc.AddNonVotersOrFatal(t, k, tc.Target(2)) - probeReq := &roachpb.ProbeRequest{ - RequestHeader: roachpb.RequestHeader{ + probeReq := &kvpb.ProbeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: k, }, } @@ -120,15 +121,15 @@ func TestReplicaProbeRequest(t *testing.T) { // stack, with both routing policies. for _, srv := range tc.Servers { db := srv.DB() - for _, policy := range []roachpb.RoutingPolicy{ - roachpb.RoutingPolicy_LEASEHOLDER, - roachpb.RoutingPolicy_NEAREST, + for _, policy := range []kvpb.RoutingPolicy{ + kvpb.RoutingPolicy_LEASEHOLDER, + kvpb.RoutingPolicy_NEAREST, } { var b kv.Batch b.AddRawRequest(probeReq) b.Header.RoutingPolicy = policy err := db.Run(ctx, &b) - if errors.HasType(err, (*roachpb.AmbiguousResultError)(nil)) { + if errors.HasType(err, (*kvpb.AmbiguousResultError)(nil)) { // Rare but it can happen that we're proposing on a replica // that is just about to get a snapshot. In that case we'll // get: @@ -170,7 +171,7 @@ func TestReplicaProbeRequest(t *testing.T) { for _, srv := range tc.Servers { repl, _, err := srv.Stores().GetReplicaForRangeID(ctx, desc.RangeID) require.NoError(t, err) - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Add(probeReq) ba.Timestamp = srv.Clock().Now() _, pErr := repl.Send(ctx, ba) @@ -181,14 +182,14 @@ func TestReplicaProbeRequest(t *testing.T) { // back. Not sure what other error might occur in practice, but checking this // anyway gives us extra confidence in the implementation mechanics of this // request. - injErr := roachpb.NewErrorf("bang") + injErr := kvpb.NewErrorf("bang") seen.Lock() seen.injectedErr = injErr seen.Unlock() for _, srv := range tc.Servers { repl, _, err := srv.Stores().GetReplicaForRangeID(ctx, desc.RangeID) require.NoError(t, err) - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Timestamp = srv.Clock().Now() ba.Add(probeReq) _, pErr := repl.Send(ctx, ba) diff --git a/pkg/kv/kvserver/replica_proposal.go b/pkg/kv/kvserver/replica_proposal.go index 2180afacf313..c693a9c1aca6 100644 --- a/pkg/kv/kvserver/replica_proposal.go +++ b/pkg/kv/kvserver/replica_proposal.go @@ -17,6 +17,7 @@ import ( "unsafe" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" @@ -114,7 +115,7 @@ type ProposalData struct { // applies. Other than tests, we only need a few bits of the request // here; this could be replaced with isLease and isChangeReplicas // booleans. - Request *roachpb.BatchRequest + Request *kvpb.BatchRequest // leaseStatus represents the lease under which the Request was evaluated and // under which this proposal is being made. For lease requests, this is the @@ -680,8 +681,8 @@ func (r *Replica) handleReadWriteLocalEvalResult(ctx context.Context, lResult re // proposalResult indicates the result of a proposal. Exactly one of // Reply and Err is set, and it represents the result of the proposal. type proposalResult struct { - Reply *roachpb.BatchResponse - Err *roachpb.Error + Reply *kvpb.BatchResponse + Err *kvpb.Error EncounteredIntents []roachpb.Intent EndTxns []result.EndTxnIntents } @@ -702,13 +703,13 @@ type proposalResult struct { func (r *Replica) evaluateProposal( ctx context.Context, idKey kvserverbase.CmdIDKey, - ba *roachpb.BatchRequest, + ba *kvpb.BatchRequest, g *concurrency.Guard, st *kvserverpb.LeaseStatus, ui uncertainty.Interval, -) (*result.Result, bool, *roachpb.Error) { +) (*result.Result, bool, *kvpb.Error) { if ba.Timestamp.IsEmpty() { - return nil, false, roachpb.NewErrorf("can't propose Raft command with zero timestamp") + return nil, false, kvpb.NewErrorf("can't propose Raft command with zero timestamp") } // Evaluate the commands. If this returns without an error, the batch should @@ -731,7 +732,7 @@ func (r *Replica) evaluateProposal( } if pErr != nil { - if _, ok := pErr.GetDetail().(*roachpb.ReplicaCorruptionError); ok { + if _, ok := pErr.GetDetail().(*kvpb.ReplicaCorruptionError); ok { return &res, false /* needConsensus */, pErr } @@ -799,16 +800,16 @@ func (r *Replica) evaluateProposal( // requestToProposal converts a BatchRequest into a ProposalData, by // evaluating it. The returned ProposalData is partially valid even -// on a non-nil *roachpb.Error and should be proposed through Raft +// on a non-nil *kvpb.Error and should be proposed through Raft // if ProposalData.command is non-nil. func (r *Replica) requestToProposal( ctx context.Context, idKey kvserverbase.CmdIDKey, - ba *roachpb.BatchRequest, + ba *kvpb.BatchRequest, g *concurrency.Guard, st *kvserverpb.LeaseStatus, ui uncertainty.Interval, -) (*ProposalData, *roachpb.Error) { +) (*ProposalData, *kvpb.Error) { res, needConsensus, pErr := r.evaluateProposal(ctx, idKey, ba, g, st, ui) // Fill out the results even if pErr != nil; we'll return the error below. diff --git a/pkg/kv/kvserver/replica_proposal_bench_test.go b/pkg/kv/kvserver/replica_proposal_bench_test.go index 95a4ba970bbe..9630047be012 100644 --- a/pkg/kv/kvserver/replica_proposal_bench_test.go +++ b/pkg/kv/kvserver/replica_proposal_bench_test.go @@ -16,6 +16,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" @@ -81,8 +82,8 @@ func runBenchmarkReplicaProposal(b *testing.B, bytes int64, withFollower bool) { value := roachpb.MakeValueFromString( randutil.RandString(rng, int(bytes), randutil.PrintableKeyAlphabet), ) - req := roachpb.NewPut(k, value) - var ba roachpb.BatchRequest + req := kvpb.NewPut(k, value) + var ba kvpb.BatchRequest ba.Add(req) repl, _, err := tc.Server(0).GetStores().(*kvserver.Stores).GetReplicaForRangeID( diff --git a/pkg/kv/kvserver/replica_proposal_buf.go b/pkg/kv/kvserver/replica_proposal_buf.go index 13272a7e3cf8..2a5de9ba2e49 100644 --- a/pkg/kv/kvserver/replica_proposal_buf.go +++ b/pkg/kv/kvserver/replica_proposal_buf.go @@ -15,6 +15,7 @@ import ( "sync" "sync/atomic" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/tracker" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" @@ -1263,8 +1264,8 @@ func (rp *replicaProposer) rejectProposalWithRedirectLocked( r.store.metrics.LeaseRequestErrorCount.Inc(1) redirectRep, _ /* ok */ := rangeDesc.GetReplicaDescriptorByID(redirectTo) log.VEventf(ctx, 2, "redirecting proposal to node %s; request: %s", redirectRep.NodeID, prop.Request) - rp.rejectProposalWithErrLocked(ctx, prop, roachpb.NewError( - roachpb.NewNotLeaseHolderErrorWithSpeculativeLease( + rp.rejectProposalWithErrLocked(ctx, prop, kvpb.NewError( + kvpb.NewNotLeaseHolderErrorWithSpeculativeLease( redirectRep, storeID, rangeDesc, @@ -1282,11 +1283,11 @@ func (rp *replicaProposer) rejectProposalWithLeaseTransferRejectedLocked( log.VEventf(ctx, 2, "not proposing lease transfer because the target %s may "+ "need a snapshot: %s", lease.Replica, reason) err := NewLeaseTransferRejectedBecauseTargetMayNeedSnapshotError(lease.Replica, reason) - rp.rejectProposalWithErrLocked(ctx, prop, roachpb.NewError(err)) + rp.rejectProposalWithErrLocked(ctx, prop, kvpb.NewError(err)) } func (rp *replicaProposer) rejectProposalWithErrLocked( - ctx context.Context, prop *ProposalData, pErr *roachpb.Error, + ctx context.Context, prop *ProposalData, pErr *kvpb.Error, ) { (*Replica)(rp).cleanupFailedProposalLocked(prop) prop.finishApplication(ctx, proposalResult{Err: pErr}) diff --git a/pkg/kv/kvserver/replica_proposal_buf_test.go b/pkg/kv/kvserver/replica_proposal_buf_test.go index a01b69123db7..77d542d1b061 100644 --- a/pkg/kv/kvserver/replica_proposal_buf_test.go +++ b/pkg/kv/kvserver/replica_proposal_buf_test.go @@ -18,6 +18,7 @@ import ( "testing" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/tracker" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" @@ -255,32 +256,32 @@ type proposalCreator struct { } func (pc proposalCreator) newPutProposal(ts hlc.Timestamp) *ProposalData { - ba := &roachpb.BatchRequest{} - ba.Add(&roachpb.PutRequest{}) + ba := &kvpb.BatchRequest{} + ba.Add(&kvpb.PutRequest{}) ba.Timestamp = ts return pc.newProposal(ba) } func (pc proposalCreator) newLeaseRequestProposal(lease roachpb.Lease) *ProposalData { - ba := &roachpb.BatchRequest{} - ba.Add(&roachpb.RequestLeaseRequest{Lease: lease, PrevLease: pc.lease.Lease}) + ba := &kvpb.BatchRequest{} + ba.Add(&kvpb.RequestLeaseRequest{Lease: lease, PrevLease: pc.lease.Lease}) return pc.newProposal(ba) } func (pc proposalCreator) newLeaseTransferProposal(lease roachpb.Lease) *ProposalData { - ba := &roachpb.BatchRequest{} - ba.Add(&roachpb.TransferLeaseRequest{Lease: lease, PrevLease: pc.lease.Lease}) + ba := &kvpb.BatchRequest{} + ba.Add(&kvpb.TransferLeaseRequest{Lease: lease, PrevLease: pc.lease.Lease}) return pc.newProposal(ba) } -func (pc proposalCreator) newProposal(ba *roachpb.BatchRequest) *ProposalData { +func (pc proposalCreator) newProposal(ba *kvpb.BatchRequest) *ProposalData { var lease *roachpb.Lease var isLeaseRequest bool switch v := ba.Requests[0].GetInner().(type) { - case *roachpb.RequestLeaseRequest: + case *kvpb.RequestLeaseRequest: lease = &v.Lease isLeaseRequest = true - case *roachpb.TransferLeaseRequest: + case *kvpb.TransferLeaseRequest: lease = &v.Lease } p := &ProposalData{ diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go index 39d78afde7cf..b91c15aa82b3 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -18,6 +18,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/apply" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/poison" @@ -105,7 +106,7 @@ func makeIDKey() kvserverbase.CmdIDKey { // which case the other returned values are zero. func (r *Replica) evalAndPropose( ctx context.Context, - ba *roachpb.BatchRequest, + ba *kvpb.BatchRequest, g *concurrency.Guard, st *kvserverpb.LeaseStatus, ui uncertainty.Interval, @@ -115,7 +116,7 @@ func (r *Replica) evalAndPropose( func(), kvserverbase.CmdIDKey, *kvadmission.StoreWriteBytes, - *roachpb.Error, + *kvpb.Error, ) { defer tok.DoneIfNotMoved(ctx) idKey := makeIDKey() @@ -127,7 +128,7 @@ func (r *Replica) evalAndPropose( if isConcurrencyRetryError(pErr) { pErr = maybeAttachLease(pErr, &st.Lease) return nil, nil, "", nil, pErr - } else if _, ok := pErr.GetDetail().(*roachpb.ReplicaCorruptionError); ok { + } else if _, ok := pErr.GetDetail().(*kvpb.ReplicaCorruptionError); ok { return nil, nil, "", nil, pErr } @@ -147,7 +148,7 @@ func (r *Replica) evalAndPropose( // in an error. if proposal.command == nil { if proposal.Local.RequiresRaft() { - return nil, nil, "", nil, roachpb.NewError(errors.AssertionFailedf( + return nil, nil, "", nil, kvpb.NewError(errors.AssertionFailedf( "proposal resulting from batch %s erroneously bypassed Raft", ba)) } intents := proposal.Local.DetachEncounteredIntents() @@ -192,7 +193,7 @@ func (r *Replica) evalAndPropose( // Disallow async consensus for commands with EndTxnIntents because // any !Always EndTxnIntent can't be cleaned up until after the // command succeeds. - return nil, nil, "", writeBytes, roachpb.NewErrorf("cannot perform consensus asynchronously for "+ + return nil, nil, "", writeBytes, kvpb.NewErrorf("cannot perform consensus asynchronously for "+ "proposal with EndTxnIntents=%v; %v", ets, ba) } @@ -202,7 +203,7 @@ func (r *Replica) evalAndPropose( // Signal the proposal's response channel immediately. reply := *proposal.Local.Reply - reply.Responses = append([]roachpb.ResponseUnion(nil), reply.Responses...) + reply.Responses = append([]kvpb.ResponseUnion(nil), reply.Responses...) pr := proposalResult{ Reply: &reply, EncounteredIntents: proposal.Local.DetachEncounteredIntents(), @@ -226,9 +227,9 @@ func (r *Replica) evalAndPropose( // to mutate state. var seq roachpb.LeaseSequence switch t := ba.Requests[0].GetInner().(type) { - case *roachpb.RequestLeaseRequest: + case *kvpb.RequestLeaseRequest: seq = t.PrevLease.Sequence - case *roachpb.TransferLeaseRequest: + case *kvpb.TransferLeaseRequest: seq = t.PrevLease.Sequence default: } @@ -256,14 +257,14 @@ func (r *Replica) evalAndPropose( // behavior. quotaSize := uint64(proposal.command.Size()) if maxSize := uint64(kvserverbase.MaxCommandSize.Get(&r.store.cfg.Settings.SV)); quotaSize > maxSize { - return nil, nil, "", nil, roachpb.NewError(errors.Errorf( + return nil, nil, "", nil, kvpb.NewError(errors.Errorf( "command is too large: %d bytes (max: %d)", quotaSize, maxSize, )) } var err error proposal.quotaAlloc, err = r.maybeAcquireProposalQuota(ctx, quotaSize) if err != nil { - return nil, nil, "", nil, roachpb.NewError(err) + return nil, nil, "", nil, kvpb.NewError(err) } // Make sure we clean up the proposal if we fail to insert it into the // proposal buffer successfully. This ensures that we always release any @@ -334,7 +335,7 @@ func (r *Replica) evalAndPropose( // for details. func (r *Replica) propose( ctx context.Context, p *ProposalData, tok TrackedRequestToken, -) (pErr *roachpb.Error) { +) (pErr *kvpb.Error) { defer tok.DoneIfNotMoved(ctx) defer func() { @@ -347,13 +348,13 @@ func (r *Replica) propose( if p.command.MaxLeaseIndex > 0 { // MaxLeaseIndex should not be populated now. It is set only when the proposal buffer // flushes this proposal into the local raft instance. - return roachpb.NewError(errors.AssertionFailedf("MaxLeaseIndex set: %+v", p)) + return kvpb.NewError(errors.AssertionFailedf("MaxLeaseIndex set: %+v", p)) } if p.encodedCommand != nil { // This indicates someone took an existing proposal and handed it to this method // again. The caller needs to properly reset the proposal if they're going to do // that. - return roachpb.NewError(errors.AssertionFailedf("encodedCommand set: %+v", p)) + return kvpb.NewError(errors.AssertionFailedf("encodedCommand set: %+v", p)) } // Determine the encoding style for the Raft command. @@ -409,7 +410,7 @@ func (r *Replica) propose( // See also https://github.com/cockroachdb/cockroach/issues/67740. lhDesc, err := r.GetReplicaDescriptor() if err != nil { - return roachpb.NewError(err) + return kvpb.NewError(err) } proposedDesc := p.command.ReplicatedEvalResult.State.Desc // This is a reconfiguration command, we make sure the proposed @@ -426,7 +427,7 @@ func (r *Replica) propose( "remove self (leaseholder); lhRemovalAllowed: %v; current desc: %v; proposed desc: %v", lhDesc, crt, true /* lhRemovalAllowed */, r.Desc(), proposedDesc), errMarkInvalidReplicationChange) log.Errorf(p.ctx, "%v", e) - return roachpb.NewError(e) + return kvpb.NewError(e) } } else if p.command.ReplicatedEvalResult.AddSSTable != nil { log.VEvent(p.ctx, 4, "sideloadable proposal detected") @@ -434,7 +435,7 @@ func (r *Replica) propose( r.store.metrics.AddSSTableProposals.Inc(1) if p.command.ReplicatedEvalResult.AddSSTable.Data == nil { - return roachpb.NewErrorf("cannot sideload empty SSTable") + return kvpb.NewErrorf("cannot sideload empty SSTable") } } else if log.V(4) { log.Infof(p.ctx, "proposing command %x: %s", p.idKey, p.Request.Summary()) @@ -457,7 +458,7 @@ func (r *Replica) propose( // Encode body of command. data = data[:preLen+cmdLen] if _, err := protoutil.MarshalTo(p.command, data[preLen:]); err != nil { - return roachpb.NewError(err) + return kvpb.NewError(err) } p.encodedCommand = data @@ -492,7 +493,7 @@ func (r *Replica) propose( // on the field. err := r.mu.proposalBuf.Insert(ctx, p, tok.Move(ctx)) if err != nil { - return roachpb.NewError(err) + return kvpb.NewError(err) } return nil } @@ -1270,7 +1271,7 @@ func (r *Replica) hasRaftReadyRLocked() bool { // replicated commands should be considered "stuck" and should trip the // per-Replica circuit breaker. The boolean indicates whether this // mechanism is enabled; if it isn't no action should be taken. -func (r *Replica) slowReplicationThreshold(ba *roachpb.BatchRequest) (time.Duration, bool) { +func (r *Replica) slowReplicationThreshold(ba *kvpb.BatchRequest) (time.Duration, bool) { if knobs := r.store.TestingKnobs(); knobs != nil && knobs.SlowReplicationThresholdOverride != nil { if dur := knobs.SlowReplicationThresholdOverride(ba); dur > 0 { return dur, true @@ -1316,7 +1317,7 @@ func (r *Replica) refreshProposalsLocked( log.Fatalf(ctx, "refreshAtDelta specified for reason %s != reasonTicks", reason) } - var maxSlowProposalDurationRequest *roachpb.BatchRequest + var maxSlowProposalDurationRequest *kvpb.BatchRequest // TODO(tbg): don't track exempt requests for tripping the breaker? var maxSlowProposalDuration time.Duration var slowProposalCount int64 @@ -1354,8 +1355,8 @@ func (r *Replica) refreshProposalsLocked( r.cleanupFailedProposalLocked(p) log.Eventf(p.ctx, "retry proposal %x: %s", p.idKey, reason) p.finishApplication(ctx, proposalResult{ - Err: roachpb.NewError( - roachpb.NewAmbiguousResultErrorf( + Err: kvpb.NewError( + kvpb.NewAmbiguousResultErrorf( "unable to determine whether command was applied via snapshot", ), ), @@ -1423,7 +1424,7 @@ func (r *Replica) refreshProposalsLocked( if err := r.mu.proposalBuf.ReinsertLocked(ctx, p); err != nil { r.cleanupFailedProposalLocked(p) p.finishApplication(ctx, proposalResult{ - Err: roachpb.NewError(roachpb.NewAmbiguousResultError(err)), + Err: kvpb.NewError(kvpb.NewAmbiguousResultError(err)), }) } } @@ -1439,10 +1440,10 @@ func (r *Replica) poisonInflightLatches(err error) { // TODO(tbg): find out how `p.ec.done()` can have been called at this point, // See: https://github.com/cockroachdb/cockroach/issues/86547 if p.ec.g != nil && p.ec.g.Req.PoisonPolicy == poison.Policy_Error { - aErr := roachpb.NewAmbiguousResultError(err) + aErr := kvpb.NewAmbiguousResultError(err) // NB: this does not release the request's latches. It's important that // the latches stay in place, since the command could still apply. - p.signalProposalResult(proposalResult{Err: roachpb.NewError(aErr)}) + p.signalProposalResult(proposalResult{Err: kvpb.NewError(aErr)}) } } } @@ -2246,7 +2247,7 @@ func (r *Replica) acquireSplitLock( ) // If getOrCreateReplica returns RaftGroupDeletedError we know that the RHS // has already been removed. This case is handled properly in splitPostApply. - if errors.HasType(err, (*roachpb.RaftGroupDeletedError)(nil)) { + if errors.HasType(err, (*kvpb.RaftGroupDeletedError)(nil)) { return func() {}, nil } if err != nil { diff --git a/pkg/kv/kvserver/replica_raftstorage.go b/pkg/kv/kvserver/replica_raftstorage.go index 9cdf1328c120..2519fa21f254 100644 --- a/pkg/kv/kvserver/replica_raftstorage.go +++ b/pkg/kv/kvserver/replica_raftstorage.go @@ -15,6 +15,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvstorage" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/logstore" @@ -516,7 +517,7 @@ func (r *Replica) applySnapshot( sr.readOnlyCmdMu.Lock() sr.mu.Lock() sr.mu.destroyStatus.Set( - roachpb.NewRangeNotFoundError(sr.RangeID, sr.store.StoreID()), + kvpb.NewRangeNotFoundError(sr.RangeID, sr.store.StoreID()), destroyReasonRemoved) sr.mu.Unlock() sr.readOnlyCmdMu.Unlock() @@ -687,7 +688,7 @@ func (r *Replica) applySnapshot( // each raft command. These will be lost during a snapshot, so disconnect // the rangefeed, if one exists. r.disconnectRangefeedWithReason( - roachpb.RangeFeedRetryError_REASON_RAFT_SNAPSHOT, + kvpb.RangeFeedRetryError_REASON_RAFT_SNAPSHOT, ) // Update the replica's cached byte thresholds. This is a no-op if the system diff --git a/pkg/kv/kvserver/replica_range_lease.go b/pkg/kv/kvserver/replica_range_lease.go index 9cff7cb1116c..f4fba68ee86a 100644 --- a/pkg/kv/kvserver/replica_range_lease.go +++ b/pkg/kv/kvserver/replica_range_lease.go @@ -50,6 +50,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/constraint" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" @@ -83,11 +84,11 @@ var leaseStatusLogLimiter = func() *log.EveryN { // leaseRequestHandle is a handle to an asynchronous lease request. type leaseRequestHandle struct { p *pendingLeaseRequest - c chan *roachpb.Error + c chan *kvpb.Error } // C returns the channel where the lease request's result will be sent on. -func (h *leaseRequestHandle) C() <-chan *roachpb.Error { +func (h *leaseRequestHandle) C() <-chan *kvpb.Error { if h.c == nil { panic("handle already canceled") } @@ -115,7 +116,7 @@ func (h *leaseRequestHandle) Cancel() { // resolve notifies the handle of the request's result. // // Requires repl.mu is exclusively locked. -func (h *leaseRequestHandle) resolve(pErr *roachpb.Error) { h.c <- pErr } +func (h *leaseRequestHandle) resolve(pErr *kvpb.Error) { h.c <- pErr } // pendingLeaseRequest coalesces RequestLease requests and lets // callers join an in-progress lease request and wait for the result. @@ -209,7 +210,7 @@ func (p *pendingLeaseRequest) InitOrJoinRequest( // We can't join the request in progress. // TODO(nvanbenschoten): should this return a LeaseRejectedError? Should // it cancel and replace the request in progress? Reconsider. - return p.newResolvedHandle(roachpb.NewErrorf( + return p.newResolvedHandle(kvpb.NewErrorf( "request for different replica in progress (requesting: %+v, in progress: %+v)", nextLeaseHolder.ReplicaID, nextLease.Replica.ReplicaID)) } @@ -247,7 +248,7 @@ func (p *pendingLeaseRequest) InitOrJoinRequest( // No request in progress. Let's propose a Lease command asynchronously. llHandle := p.newHandle() - reqHeader := roachpb.RequestHeader{ + reqHeader := kvpb.RequestHeader{ Key: startKey, } reqLease := roachpb.Lease{ @@ -279,7 +280,7 @@ func (p *pendingLeaseRequest) InitOrJoinRequest( // Get the liveness for the next lease holder and set the epoch in the lease request. l, ok := p.repl.store.cfg.NodeLiveness.GetLiveness(nextLeaseHolder.NodeID) if !ok || l.Epoch == 0 { - llHandle.resolve(roachpb.NewError(&roachpb.LeaseRejectedError{ + llHandle.resolve(kvpb.NewError(&kvpb.LeaseRejectedError{ Existing: status.Lease, Requested: reqLease, Message: fmt.Sprintf("couldn't request lease for %+v: %v", nextLeaseHolder, liveness.ErrRecordCacheMiss), @@ -289,9 +290,9 @@ func (p *pendingLeaseRequest) InitOrJoinRequest( reqLease.Epoch = l.Epoch } - var leaseReq roachpb.Request + var leaseReq kvpb.Request if transfer { - leaseReq = &roachpb.TransferLeaseRequest{ + leaseReq = &kvpb.TransferLeaseRequest{ RequestHeader: reqHeader, Lease: reqLease, PrevLease: status.Lease, @@ -306,7 +307,7 @@ func (p *pendingLeaseRequest) InitOrJoinRequest( log.Fatal(ctx, "bypassSafetyChecks not supported for RequestLeaseRequest") } minProposedTS := p.repl.mu.minLeaseProposedTS - leaseReq = &roachpb.RequestLeaseRequest{ + leaseReq = &kvpb.RequestLeaseRequest{ RequestHeader: reqHeader, Lease: reqLease, // PrevLease must match for our lease to be accepted. If another @@ -321,8 +322,8 @@ func (p *pendingLeaseRequest) InitOrJoinRequest( // We failed to start the asynchronous task. Send a blank NotLeaseHolderError // back to indicate that we have no idea who the range lease holder might // be; we've withdrawn from active duty. - llHandle.resolve(roachpb.NewError( - roachpb.NewNotLeaseHolderError(roachpb.Lease{}, p.repl.store.StoreID(), p.repl.mu.state.Desc, + llHandle.resolve(kvpb.NewError( + kvpb.NewNotLeaseHolderError(roachpb.Lease{}, p.repl.store.StoreID(), p.repl.mu.state.Desc, "lease acquisition task couldn't be started; node is shutting down"))) return llHandle } @@ -346,7 +347,7 @@ func (p *pendingLeaseRequest) requestLeaseAsync( nextLeaseHolder roachpb.ReplicaDescriptor, reqLease roachpb.Lease, status kvserverpb.LeaseStatus, - leaseReq roachpb.Request, + leaseReq kvpb.Request, ) error { // Create a new context. We multiplex the cancellation of all contexts onto // this new one, canceling it if all coalesced requests timeout/cancel. @@ -419,7 +420,7 @@ func (p *pendingLeaseRequest) requestLeaseAsync( for llHandle := range p.llHandles { // Don't send the same transaction object twice; this can lead to races. if err != nil { - pErr := roachpb.NewError(err) + pErr := kvpb.NewError(err) // TODO(tbg): why? pErr.SetTxn(pErr.GetTxn()) llHandle.resolve(pErr) @@ -446,7 +447,7 @@ func (p *pendingLeaseRequest) requestLease( nextLeaseHolder roachpb.ReplicaDescriptor, reqLease roachpb.Lease, status kvserverpb.LeaseStatus, - leaseReq roachpb.Request, + leaseReq kvpb.Request, ) error { // If requesting an epoch-based lease & current state is expired, // potentially heartbeat our own liveness or increment epoch of @@ -515,7 +516,7 @@ func (p *pendingLeaseRequest) requestLease( // isn't valid. In particular, if it was ours but we failed to reacquire // it (e.g. because our heartbeat failed due to a stalled disk) then we // don't want DistSender to retry us. - return roachpb.NewNotLeaseHolderError(roachpb.Lease{}, p.repl.store.StoreID(), p.repl.Desc(), + return kvpb.NewNotLeaseHolderError(roachpb.Lease{}, p.repl.store.StoreID(), p.repl.Desc(), fmt.Sprintf("failed to manipulate liveness record: %s", err)) } } @@ -528,7 +529,7 @@ func (p *pendingLeaseRequest) requestLease( // solution to the below issue: // // https://github.com/cockroachdb/cockroach/issues/37906 - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Timestamp = p.repl.store.Clock().Now() ba.RangeID = p.repl.RangeID // NB: @@ -560,7 +561,7 @@ func (p *pendingLeaseRequest) requestLease( func (p *pendingLeaseRequest) JoinRequest() *leaseRequestHandle { llHandle := p.newHandle() if _, ok := p.RequestPending(); !ok { - llHandle.resolve(roachpb.NewErrorf("no request in progress")) + llHandle.resolve(kvpb.NewErrorf("no request in progress")) return llHandle } p.llHandles[llHandle] = struct{}{} @@ -593,13 +594,13 @@ func (p *pendingLeaseRequest) TransferInProgress(replicaID roachpb.ReplicaID) bo func (p *pendingLeaseRequest) newHandle() *leaseRequestHandle { return &leaseRequestHandle{ p: p, - c: make(chan *roachpb.Error, 1), + c: make(chan *kvpb.Error, 1), } } // newResolvedHandle creates a new leaseRequestHandle referencing the pending // lease request. It then resolves the handle with the provided error. -func (p *pendingLeaseRequest) newResolvedHandle(pErr *roachpb.Error) *leaseRequestHandle { +func (p *pendingLeaseRequest) newResolvedHandle(pErr *kvpb.Error) *leaseRequestHandle { h := p.newHandle() h.resolve(pErr) return h @@ -823,7 +824,7 @@ func (r *Replica) requestLeaseLocked( // Propose a Raft command to get a lease for this replica. repDesc, err := r.getReplicaDescriptorRLocked() if err != nil { - return r.mu.pendingLeaseRequest.newResolvedHandle(roachpb.NewError(err)) + return r.mu.pendingLeaseRequest.newResolvedHandle(kvpb.NewError(err)) } return r.mu.pendingLeaseRequest.InitOrJoinRequest( ctx, repDesc, status, r.mu.state.Desc.StartKey.AsRawKey(), @@ -872,7 +873,7 @@ func (r *Replica) AdminTransferLease( } desc := r.mu.state.Desc if !status.Lease.OwnedBy(r.store.StoreID()) { - return nil, nil, roachpb.NewNotLeaseHolderError(status.Lease, r.store.StoreID(), desc, + return nil, nil, kvpb.NewNotLeaseHolderError(status.Lease, r.store.StoreID(), desc, "can't transfer the lease because this store doesn't own it") } // Verify the target is a replica of the range. @@ -894,7 +895,7 @@ func (r *Replica) AdminTransferLease( } // Another transfer is in progress, and it's not transferring to the // same replica we'd like. - return nil, nil, roachpb.NewNotLeaseHolderError(nextLease, r.store.StoreID(), desc, + return nil, nil, kvpb.NewNotLeaseHolderError(nextLease, r.store.StoreID(), desc, "another transfer to a different store is in progress") } @@ -1105,7 +1106,7 @@ func (r *Replica) leaseGoodToGoForStatusRLocked( } if !st.IsValid() { // Case (2): invalid lease. - return false, &roachpb.InvalidLeaseError{} + return false, &kvpb.InvalidLeaseError{} } if !st.Lease.OwnedBy(r.store.StoreID()) { // Case (3): not leaseholder. @@ -1147,7 +1148,7 @@ func (r *Replica) leaseGoodToGoForStatusRLocked( } // Otherwise, if the lease is currently held by another replica, redirect // to the holder. - return false, roachpb.NewNotLeaseHolderError( + return false, kvpb.NewNotLeaseHolderError( st.Lease, r.store.StoreID(), r.descRLocked(), "lease held by different store", ) } @@ -1183,7 +1184,7 @@ func (r *Replica) leaseGoodToGo( // Reads, however, must wait. func (r *Replica) redirectOnOrAcquireLease( ctx context.Context, -) (kvserverpb.LeaseStatus, *roachpb.Error) { +) (kvserverpb.LeaseStatus, *kvpb.Error) { return r.redirectOnOrAcquireLeaseForRequest(ctx, hlc.Timestamp{}, r.breaker.Signal()) } @@ -1200,7 +1201,7 @@ func (r *Replica) TestingAcquireLease(ctx context.Context) (kvserverpb.LeaseStat // the request is operating at the current time. func (r *Replica) redirectOnOrAcquireLeaseForRequest( ctx context.Context, reqTS hlc.Timestamp, brSig signaller, -) (status kvserverpb.LeaseStatus, pErr *roachpb.Error) { +) (status kvserverpb.LeaseStatus, pErr *kvpb.Error) { // Does not use RunWithTimeout(), because we do not want to mask the // NotLeaseHolderError on context cancellation. ctx, cancel := context.WithTimeout(ctx, r.store.cfg.RangeLeaseAcquireTimeout()) // nolint:context @@ -1215,20 +1216,20 @@ func (r *Replica) redirectOnOrAcquireLeaseForRequest( r.maybeExtendLeaseAsync(ctx, status) } return status, nil - } else if !errors.HasType(err, (*roachpb.InvalidLeaseError)(nil)) { - return kvserverpb.LeaseStatus{}, roachpb.NewError(err) + } else if !errors.HasType(err, (*kvpb.InvalidLeaseError)(nil)) { + return kvserverpb.LeaseStatus{}, kvpb.NewError(err) } } if err := brSig.Err(); err != nil { - return kvserverpb.LeaseStatus{}, roachpb.NewError(err) + return kvserverpb.LeaseStatus{}, kvpb.NewError(err) } // Loop until the lease is held or the replica ascertains the actual lease // holder. Returns also on context.Done() (timeout or cancellation). for attempt := 1; ; attempt++ { now = r.store.Clock().NowAsClockTimestamp() - llHandle, status, transfer, pErr := func() (*leaseRequestHandle, kvserverpb.LeaseStatus, bool, *roachpb.Error) { + llHandle, status, transfer, pErr := func() (*leaseRequestHandle, kvserverpb.LeaseStatus, bool, *kvpb.Error) { r.mu.Lock() defer r.mu.Unlock() @@ -1239,7 +1240,7 @@ func (r *Replica) redirectOnOrAcquireLeaseForRequest( // unsuccessfully before redirecting or retrying. repDesc, err := r.getReplicaDescriptorRLocked() if err != nil { - return nil, kvserverpb.LeaseStatus{}, false, roachpb.NewError(err) + return nil, kvserverpb.LeaseStatus{}, false, kvpb.NewError(err) } if ok := r.mu.pendingLeaseRequest.TransferInProgress(repDesc.ReplicaID); ok { return r.mu.pendingLeaseRequest.JoinRequest(), kvserverpb.LeaseStatus{}, true /* transfer */, nil @@ -1254,8 +1255,8 @@ func (r *Replica) redirectOnOrAcquireLeaseForRequest( msg = "lease state could not be determined" } log.VEventf(ctx, 2, "%s", msg) - return nil, kvserverpb.LeaseStatus{}, false, roachpb.NewError( - roachpb.NewNotLeaseHolderError(roachpb.Lease{}, r.store.StoreID(), r.mu.state.Desc, msg)) + return nil, kvserverpb.LeaseStatus{}, false, kvpb.NewError( + kvpb.NewNotLeaseHolderError(roachpb.Lease{}, r.store.StoreID(), r.mu.state.Desc, msg)) case kvserverpb.LeaseState_VALID, kvserverpb.LeaseState_UNUSABLE: if !status.Lease.OwnedBy(r.store.StoreID()) { @@ -1267,8 +1268,8 @@ func (r *Replica) redirectOnOrAcquireLeaseForRequest( } // Otherwise, if the lease is currently held by another replica, redirect // to the holder. - return nil, kvserverpb.LeaseStatus{}, false, roachpb.NewError( - roachpb.NewNotLeaseHolderError(status.Lease, r.store.StoreID(), r.mu.state.Desc, + return nil, kvserverpb.LeaseStatus{}, false, kvpb.NewError( + kvpb.NewNotLeaseHolderError(status.Lease, r.store.StoreID(), r.mu.state.Desc, "lease held by different store")) } @@ -1295,11 +1296,11 @@ func (r *Replica) redirectOnOrAcquireLeaseForRequest( return r.requestLeaseLocked(ctx, status), kvserverpb.LeaseStatus{}, false, nil } // If lease is currently held by another, redirect to holder. - return nil, kvserverpb.LeaseStatus{}, false, roachpb.NewError( - roachpb.NewNotLeaseHolderError(status.Lease, r.store.StoreID(), r.mu.state.Desc, "lease proscribed")) + return nil, kvserverpb.LeaseStatus{}, false, kvpb.NewError( + kvpb.NewNotLeaseHolderError(status.Lease, r.store.StoreID(), r.mu.state.Desc, "lease proscribed")) default: - return nil, kvserverpb.LeaseStatus{}, false, roachpb.NewErrorf("unknown lease status state %v", status) + return nil, kvserverpb.LeaseStatus{}, false, kvpb.NewErrorf("unknown lease status state %v", status) } }() if pErr != nil { @@ -1320,7 +1321,7 @@ func (r *Replica) redirectOnOrAcquireLeaseForRequest( // timestamp is not covered by the new lease (though we try to protect // against this in checkRequestTimeRLocked). So instead of assuming // anything, we iterate and check again. - pErr = func() (pErr *roachpb.Error) { + pErr = func() (pErr *kvpb.Error) { slowTimer := timeutil.NewTimer() defer slowTimer.Stop() slowTimer.Reset(base.SlowRequestThreshold) @@ -1337,14 +1338,14 @@ func (r *Replica) redirectOnOrAcquireLeaseForRequest( if pErr != nil { goErr := pErr.GoError() switch { - case errors.HasType(goErr, (*roachpb.AmbiguousResultError)(nil)): + case errors.HasType(goErr, (*kvpb.AmbiguousResultError)(nil)): // This can happen if the RequestLease command we sent has been // applied locally through a snapshot: the RequestLeaseRequest // cannot be reproposed so we get this ambiguity. // We'll just loop around. return nil - case errors.HasType(goErr, (*roachpb.LeaseRejectedError)(nil)): - var tErr *roachpb.LeaseRejectedError + case errors.HasType(goErr, (*kvpb.LeaseRejectedError)(nil)): + var tErr *kvpb.LeaseRejectedError errors.As(goErr, &tErr) if tErr.Existing.OwnedBy(r.store.StoreID()) { // The RequestLease command we sent was rejected because another @@ -1363,13 +1364,13 @@ func (r *Replica) redirectOnOrAcquireLeaseForRequest( if _, descErr := r.GetReplicaDescriptor(); descErr != nil { err = descErr } else if st := r.CurrentLeaseStatus(ctx); !st.IsValid() { - err = roachpb.NewNotLeaseHolderError(roachpb.Lease{}, r.store.StoreID(), r.Desc(), + err = kvpb.NewNotLeaseHolderError(roachpb.Lease{}, r.store.StoreID(), r.Desc(), "lease acquisition attempt lost to another lease, which has expired in the meantime") } else { - err = roachpb.NewNotLeaseHolderError(st.Lease, r.store.StoreID(), r.Desc(), + err = kvpb.NewNotLeaseHolderError(st.Lease, r.store.StoreID(), r.Desc(), "lease acquisition attempt lost to another lease") } - pErr = roachpb.NewError(err) + pErr = kvpb.NewError(err) } return pErr } @@ -1379,7 +1380,7 @@ func (r *Replica) redirectOnOrAcquireLeaseForRequest( llHandle.Cancel() err := brSig.Err() log.VErrEventf(ctx, 2, "lease acquisition failed: %s", err) - return roachpb.NewError(err) + return kvpb.NewError(err) case <-slowTimer.C: slowTimer.Read = true log.Warningf(ctx, "have been waiting %s attempting to acquire lease (%d attempts)", @@ -1392,11 +1393,11 @@ func (r *Replica) redirectOnOrAcquireLeaseForRequest( case <-ctx.Done(): llHandle.Cancel() log.VErrEventf(ctx, 2, "lease acquisition failed: %s", ctx.Err()) - return roachpb.NewError(roachpb.NewNotLeaseHolderError(roachpb.Lease{}, r.store.StoreID(), r.Desc(), + return kvpb.NewError(kvpb.NewNotLeaseHolderError(roachpb.Lease{}, r.store.StoreID(), r.Desc(), "lease acquisition canceled because context canceled")) case <-r.store.Stopper().ShouldQuiesce(): llHandle.Cancel() - return roachpb.NewError(roachpb.NewNotLeaseHolderError(roachpb.Lease{}, r.store.StoreID(), r.Desc(), + return kvpb.NewError(kvpb.NewNotLeaseHolderError(roachpb.Lease{}, r.store.StoreID(), r.Desc(), "lease acquisition canceled because node is stopping")) } } diff --git a/pkg/kv/kvserver/replica_rangefeed.go b/pkg/kv/kvserver/replica_rangefeed.go index 8852c0a90d1b..e5c6c03ae12a 100644 --- a/pkg/kv/kvserver/replica_rangefeed.go +++ b/pkg/kv/kvserver/replica_rangefeed.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/intentresolver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" @@ -63,7 +64,7 @@ var RangeFeedRefreshInterval = settings.RegisterDurationSetting( // support for concurrent calls to Send. Note that the default implementation of // grpc.Stream is not safe for concurrent calls to Send. type lockedRangefeedStream struct { - wrapped roachpb.RangeFeedEventSink + wrapped kvpb.RangeFeedEventSink sendMu syncutil.Mutex } @@ -71,7 +72,7 @@ func (s *lockedRangefeedStream) Context() context.Context { return s.wrapped.Context() } -func (s *lockedRangefeedStream) Send(e *roachpb.RangeFeedEvent) error { +func (s *lockedRangefeedStream) Send(e *kvpb.RangeFeedEvent) error { s.sendMu.Lock() defer s.sendMu.Unlock() return s.wrapped.Send(e) @@ -96,7 +97,7 @@ func (tp *rangefeedTxnPusher) PushTxns( pushTxnMap[txn.ID] = txn } - h := roachpb.Header{ + h := kvpb.Header{ Timestamp: ts, Txn: &roachpb.Transaction{ TxnMeta: enginepb.TxnMeta{ @@ -106,7 +107,7 @@ func (tp *rangefeedTxnPusher) PushTxns( } pushedTxnMap, pErr := tp.ir.MaybePushTransactions( - ctx, pushTxnMap, h, roachpb.PUSH_TIMESTAMP, false, /* skipIfInFlight */ + ctx, pushTxnMap, h, kvpb.PUSH_TIMESTAMP, false, /* skipIfInFlight */ ) if pErr != nil { return nil, pErr.GoError() @@ -134,29 +135,29 @@ func (tp *rangefeedTxnPusher) ResolveIntents( // complete. The surrounding store's ConcurrentRequestLimiter is used to limit // the number of rangefeeds using catch-up iterators at the same time. func (r *Replica) RangeFeed( - args *roachpb.RangeFeedRequest, stream roachpb.RangeFeedEventSink, pacer *admission.Pacer, -) *roachpb.Error { + args *kvpb.RangeFeedRequest, stream kvpb.RangeFeedEventSink, pacer *admission.Pacer, +) *kvpb.Error { return r.rangeFeedWithRangeID(r.RangeID, args, stream, pacer) } func (r *Replica) rangeFeedWithRangeID( _forStacks roachpb.RangeID, - args *roachpb.RangeFeedRequest, - stream roachpb.RangeFeedEventSink, + args *kvpb.RangeFeedRequest, + stream kvpb.RangeFeedEventSink, pacer *admission.Pacer, -) *roachpb.Error { +) *kvpb.Error { ctx := r.AnnotateCtx(stream.Context()) rSpan, err := keys.SpanAddr(args.Span) if err != nil { - return roachpb.NewError(err) + return kvpb.NewError(err) } if err := r.ensureClosedTimestampStarted(ctx); err != nil { - if err := stream.Send(&roachpb.RangeFeedEvent{Error: &roachpb.RangeFeedError{ + if err := stream.Send(&kvpb.RangeFeedEvent{Error: &kvpb.RangeFeedError{ Error: *err, }}); err != nil { - return roachpb.NewError(err) + return kvpb.NewError(err) } return nil } @@ -177,7 +178,7 @@ func (r *Replica) rangeFeedWithRangeID( } lockedStream := &lockedRangefeedStream{wrapped: stream} - errC := make(chan *roachpb.Error, 1) + errC := make(chan *kvpb.Error, 1) // If we will be using a catch-up iterator, wait for the limiter here before // locking raftMu. @@ -187,7 +188,7 @@ func (r *Replica) rangeFeedWithRangeID( usingCatchUpIter = true alloc, err := r.store.limiters.ConcurrentRangefeedIters.Begin(ctx) if err != nil { - return roachpb.NewError(err) + return kvpb.NewError(err) } // Finish the iterator limit if we exit before the iterator finishes. // The release function will be hooked into the Close method on the @@ -211,7 +212,7 @@ func (r *Replica) rangeFeedWithRangeID( r.raftMu.Lock() if err := r.checkExecutionCanProceedForRangeFeed(ctx, rSpan, checkTS); err != nil { r.raftMu.Unlock() - return roachpb.NewError(err) + return kvpb.NewError(err) } // Register the stream with a catch-up iterator. @@ -330,7 +331,7 @@ func (r *Replica) registerWithRangefeedRaftMuLocked( catchUpIter rangefeed.CatchUpIteratorConstructor, withDiff bool, stream rangefeed.Stream, - errC chan<- *roachpb.Error, + errC chan<- *kvpb.Error, ) *rangefeed.Processor { defer logSlowRangefeedRegistration(ctx)() @@ -400,7 +401,7 @@ func (r *Replica) registerWithRangefeedRaftMuLocked( // due to stopping, but before it enters the quiescing state, then the select // below will fall through to the panic. if err := p.Start(r.store.Stopper(), rtsIter); err != nil { - errC <- roachpb.NewError(err) + errC <- kvpb.NewError(err) return nil } @@ -413,7 +414,7 @@ func (r *Replica) registerWithRangefeedRaftMuLocked( if !reg { select { case <-r.store.Stopper().ShouldQuiesce(): - errC <- roachpb.NewError(&roachpb.NodeUnavailableError{}) + errC <- kvpb.NewError(&kvpb.NodeUnavailableError{}) return nil default: panic("unexpected Stopped processor") @@ -452,7 +453,7 @@ func (r *Replica) maybeDisconnectEmptyRangefeed(p *rangefeed.Processor) { // disconnectRangefeedWithErr broadcasts the provided error to all rangefeed // registrations and tears down the provided rangefeed Processor. -func (r *Replica) disconnectRangefeedWithErr(p *rangefeed.Processor, pErr *roachpb.Error) { +func (r *Replica) disconnectRangefeedWithErr(p *rangefeed.Processor, pErr *kvpb.Error) { p.StopWithErr(pErr) r.unsetRangefeedProcessor(p) } @@ -460,7 +461,7 @@ func (r *Replica) disconnectRangefeedWithErr(p *rangefeed.Processor, pErr *roach // disconnectRangefeedSpanWithErr broadcasts the provided error to all rangefeed // registrations that overlap the given span. Tears down the rangefeed Processor // if it has no remaining registrations. -func (r *Replica) disconnectRangefeedSpanWithErr(span roachpb.Span, pErr *roachpb.Error) { +func (r *Replica) disconnectRangefeedSpanWithErr(span roachpb.Span, pErr *kvpb.Error) { p := r.getRangefeedProcessor() if p == nil { return @@ -472,12 +473,12 @@ func (r *Replica) disconnectRangefeedSpanWithErr(span roachpb.Span, pErr *roachp // disconnectRangefeedWithReason broadcasts the provided rangefeed retry reason // to all rangefeed registrations and tears down the active rangefeed Processor. // No-op if a rangefeed is not active. -func (r *Replica) disconnectRangefeedWithReason(reason roachpb.RangeFeedRetryError_Reason) { +func (r *Replica) disconnectRangefeedWithReason(reason kvpb.RangeFeedRetryError_Reason) { p := r.getRangefeedProcessor() if p == nil { return } - pErr := roachpb.NewError(roachpb.NewRangeFeedRetryError(reason)) + pErr := kvpb.NewError(kvpb.NewRangeFeedRetryError(reason)) r.disconnectRangefeedWithErr(p, pErr) } @@ -566,7 +567,7 @@ func (r *Replica) handleLogicalOpLogRaftMuLocked( // registrations, we're forced to throw an error. The rangefeed clients // can reconnect at a later time, at which point all new Raft commands // should have logical op logs. - r.disconnectRangefeedWithReason(roachpb.RangeFeedRetryError_REASON_LOGICAL_OPS_MISSING) + r.disconnectRangefeedWithReason(kvpb.RangeFeedRetryError_REASON_LOGICAL_OPS_MISSING) return } if len(ops.Ops) == 0 { @@ -631,7 +632,7 @@ func (r *Replica) handleLogicalOpLogRaftMuLocked( err = errors.New("value missing in reader") } if err != nil { - r.disconnectRangefeedWithErr(p, roachpb.NewErrorf( + r.disconnectRangefeedWithErr(p, kvpb.NewErrorf( "error consuming %T for key %v @ ts %v: %v", op, key, ts, err, )) return @@ -753,7 +754,7 @@ func (r *Replica) handleClosedTimestampUpdateRaftMuLocked( // receiving closed timestamp updates for this replica's range. Note that this // forces a valid lease to exist on the range and so can be reasonably expensive // if there is not already a valid lease. -func (r *Replica) ensureClosedTimestampStarted(ctx context.Context) *roachpb.Error { +func (r *Replica) ensureClosedTimestampStarted(ctx context.Context) *kvpb.Error { // Make sure there's a valid lease. If there's no lease, nobody's sending // closed timestamp updates. lease := r.CurrentLeaseStatus(ctx) @@ -770,18 +771,18 @@ func (r *Replica) ensureClosedTimestampStarted(ctx context.Context) *roachpb.Err err := contextutil.RunWithTimeout(ctx, "read forcing lease acquisition", 5*time.Second, func(ctx context.Context) error { var b kv.Batch - liReq := &roachpb.LeaseInfoRequest{} + liReq := &kvpb.LeaseInfoRequest{} liReq.Key = r.Desc().StartKey.AsRawKey() b.AddRawRequest(liReq) return r.store.DB().Run(ctx, &b) }) if err != nil { if errors.HasType(err, (*contextutil.TimeoutError)(nil)) { - err = &roachpb.RangeFeedRetryError{ - Reason: roachpb.RangeFeedRetryError_REASON_NO_LEASEHOLDER, + err = &kvpb.RangeFeedRetryError{ + Reason: kvpb.RangeFeedRetryError_REASON_NO_LEASEHOLDER, } } - return roachpb.NewError(err) + return kvpb.NewError(err) } } return nil diff --git a/pkg/kv/kvserver/replica_rangefeed_test.go b/pkg/kv/kvserver/replica_rangefeed_test.go index 469b3cf032fa..d9160a8df3b9 100644 --- a/pkg/kv/kvserver/replica_rangefeed_test.go +++ b/pkg/kv/kvserver/replica_rangefeed_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" @@ -45,13 +46,13 @@ import ( "google.golang.org/grpc/metadata" ) -// testStream is a mock implementation of roachpb.Internal_RangeFeedServer. +// testStream is a mock implementation of kvpb.Internal_RangeFeedServer. type testStream struct { ctx context.Context cancel func() mu struct { syncutil.Mutex - events []*roachpb.RangeFeedEvent + events []*kvpb.RangeFeedEvent } } @@ -74,14 +75,14 @@ func (s *testStream) Cancel() { s.cancel() } -func (s *testStream) Send(e *roachpb.RangeFeedEvent) error { +func (s *testStream) Send(e *kvpb.RangeFeedEvent) error { s.mu.Lock() defer s.mu.Unlock() s.mu.events = append(s.mu.events, e) return nil } -func (s *testStream) Events() []*roachpb.RangeFeedEvent { +func (s *testStream) Events() []*kvpb.RangeFeedEvent { s.mu.Lock() defer s.mu.Unlock() return s.mu.events @@ -130,13 +131,13 @@ func TestReplicaRangefeed(t *testing.T) { initTime := ts.Clock().Now() ts1 := initTime.Add(0, 1) incArgs := incrementArgs(roachpb.Key("b"), 9) - if _, pErr := kv.SendWrappedWith(ctx, db, roachpb.Header{Timestamp: ts1}, incArgs); pErr != nil { + if _, pErr := kv.SendWrappedWith(ctx, db, kvpb.Header{Timestamp: ts1}, incArgs); pErr != nil { t.Fatal(pErr) } tc.WaitForValues(t, roachpb.Key("b"), []int64{9, 9, 9, 9, 9}) streams := make([]*testStream, numNodes) - streamErrC := make(chan *roachpb.Error, numNodes) + streamErrC := make(chan *kvpb.Error, numNodes) rangefeedSpan := roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("z")} for i := 0; i < numNodes; i++ { stream := newTestStream() @@ -147,8 +148,8 @@ func TestReplicaRangefeed(t *testing.T) { t.Fatal(err) } go func(i int) { - req := roachpb.RangeFeedRequest{ - Header: roachpb.Header{ + req := kvpb.RangeFeedRequest{ + Header: kvpb.Header{ Timestamp: initTime, RangeID: rangeID, }, @@ -161,7 +162,7 @@ func TestReplicaRangefeed(t *testing.T) { }(i) } - checkForExpEvents := func(expEvents []*roachpb.RangeFeedEvent) { + checkForExpEvents := func(expEvents []*kvpb.RangeFeedEvent) { t.Helper() // SSTs may not be equal byte-for-byte due to AddSSTable rewrites. We nil @@ -186,7 +187,7 @@ func TestReplicaRangefeed(t *testing.T) { } for _, stream := range streams { - var events []*roachpb.RangeFeedEvent + var events []*kvpb.RangeFeedEvent testutils.SucceedsSoon(t, func() error { if len(streamErrC) > 0 { // Break if the error channel is already populated. @@ -195,7 +196,7 @@ func TestReplicaRangefeed(t *testing.T) { events = stream.Events() // Filter out checkpoints. Those are not deterministic; they can come at any time. - var filteredEvents []*roachpb.RangeFeedEvent + var filteredEvents []*kvpb.RangeFeedEvent for _, e := range events { if e.Checkpoint != nil { continue @@ -275,8 +276,8 @@ func TestReplicaRangefeed(t *testing.T) { expVal1 := roachpb.Value{Timestamp: ts1} expVal1.SetInt(9) expVal1.InitChecksum(roachpb.Key("b")) - expEvents := []*roachpb.RangeFeedEvent{ - {Val: &roachpb.RangeFeedValue{ + expEvents := []*kvpb.RangeFeedEvent{ + {Val: &kvpb.RangeFeedValue{ Key: roachpb.Key("b"), Value: expVal1, }}, } @@ -285,7 +286,7 @@ func TestReplicaRangefeed(t *testing.T) { // Insert a key non-transactionally. ts2 := initTime.Add(0, 2) pArgs := putArgs(roachpb.Key("c"), []byte("val2")) - _, err := kv.SendWrappedWith(ctx, db, roachpb.Header{Timestamp: ts2}, pArgs) + _, err := kv.SendWrappedWith(ctx, db, kvpb.Header{Timestamp: ts2}, pArgs) if err != nil { t.Fatal(err) } @@ -312,7 +313,7 @@ func TestReplicaRangefeed(t *testing.T) { // Update the originally incremented key non-transactionally. ts4 := initTime.Add(0, 4) - _, err = kv.SendWrappedWith(ctx, db, roachpb.Header{Timestamp: ts4}, incArgs) + _, err = kv.SendWrappedWith(ctx, db, kvpb.Header{Timestamp: ts4}, incArgs) if err != nil { t.Fatal(err) } @@ -405,27 +406,27 @@ func TestReplicaRangefeed(t *testing.T) { expVal7q.Timestamp = ts7 expVal1NoTS, expVal4NoTS := expVal1, expVal4 expVal1NoTS.Timestamp, expVal4NoTS.Timestamp = hlc.Timestamp{}, hlc.Timestamp{} - expEvents = append(expEvents, []*roachpb.RangeFeedEvent{ - {Val: &roachpb.RangeFeedValue{ + expEvents = append(expEvents, []*kvpb.RangeFeedEvent{ + {Val: &kvpb.RangeFeedValue{ Key: roachpb.Key("c"), Value: expVal2, }}, - {Val: &roachpb.RangeFeedValue{ + {Val: &kvpb.RangeFeedValue{ Key: roachpb.Key("m"), Value: expVal3, }}, - {Val: &roachpb.RangeFeedValue{ + {Val: &kvpb.RangeFeedValue{ Key: roachpb.Key("b"), Value: expVal4, PrevValue: expVal1NoTS, }}, - {Val: &roachpb.RangeFeedValue{ + {Val: &kvpb.RangeFeedValue{ Key: roachpb.Key("b"), Value: expVal5, PrevValue: expVal4NoTS, }}, - {SST: &roachpb.RangeFeedSSTable{ + {SST: &kvpb.RangeFeedSSTable{ // Binary representation of Data may be modified by SST rewrite, see checkForExpEvents. Data: expSST, Span: expSSTSpan, WriteTS: ts6, }}, - {Val: &roachpb.RangeFeedValue{ + {Val: &kvpb.RangeFeedValue{ Key: roachpb.Key("b"), Value: expVal7b, PrevValue: expVal6b, }}, - {Val: &roachpb.RangeFeedValue{ + {Val: &kvpb.RangeFeedValue{ Key: roachpb.Key("q"), Value: expVal7q, PrevValue: expVal6q, }}, }...) @@ -444,20 +445,20 @@ func TestReplicaRangefeed(t *testing.T) { // Bump the GC threshold and assert that RangeFeed below the timestamp will // catch an error. - gcReq := &roachpb.GCRequest{ + gcReq := &kvpb.GCRequest{ Threshold: initTime.Add(0, 1), } gcReq.Key = startKey gcReq.EndKey = firstStore.LookupReplica(startKey).Desc().EndKey.AsRawKey() - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.RangeID = rangeID ba.Add(gcReq) if _, pErr := firstStore.Send(ctx, ba); pErr != nil { t.Fatal(pErr) } - req := roachpb.RangeFeedRequest{ - Header: roachpb.Header{ + req := kvpb.RangeFeedRequest{ + Header: kvpb.Header{ Timestamp: initTime, RangeID: rangeID, }, @@ -540,21 +541,21 @@ func TestReplicaRangefeedErrors(t *testing.T) { } waitForInitialCheckpointAcrossSpan := func( - t *testing.T, stream *testStream, streamErrC <-chan *roachpb.Error, span roachpb.Span, + t *testing.T, stream *testStream, streamErrC <-chan *kvpb.Error, span roachpb.Span, ) { t.Helper() - noResolveTimestampEvent := roachpb.RangeFeedEvent{ - Checkpoint: &roachpb.RangeFeedCheckpoint{ + noResolveTimestampEvent := kvpb.RangeFeedEvent{ + Checkpoint: &kvpb.RangeFeedCheckpoint{ Span: span, ResolvedTS: hlc.Timestamp{}, }, } - resolveTimestampEvent := roachpb.RangeFeedEvent{ - Checkpoint: &roachpb.RangeFeedCheckpoint{ + resolveTimestampEvent := kvpb.RangeFeedEvent{ + Checkpoint: &kvpb.RangeFeedCheckpoint{ Span: span, }, } - var events []*roachpb.RangeFeedEvent + var events []*kvpb.RangeFeedEvent testutils.SucceedsSoon(t, func() error { if len(streamErrC) > 0 { // Break if the error channel is already populated. @@ -569,13 +570,13 @@ func TestReplicaRangefeedErrors(t *testing.T) { if len(streamErrC) > 0 { t.Fatalf("unexpected error from stream: %v", <-streamErrC) } - expEvents := []*roachpb.RangeFeedEvent{&noResolveTimestampEvent} + expEvents := []*kvpb.RangeFeedEvent{&noResolveTimestampEvent} if len(events) > 1 { // Unfortunately there is a timing issue here and the range feed may // publish two checkpoints, one with a resolvedTs and one without, so we // check for either case. resolveTimestampEvent.Checkpoint.ResolvedTS = events[1].Checkpoint.ResolvedTS - expEvents = []*roachpb.RangeFeedEvent{ + expEvents = []*kvpb.RangeFeedEvent{ &noResolveTimestampEvent, &resolveTimestampEvent, } @@ -587,14 +588,14 @@ func TestReplicaRangefeedErrors(t *testing.T) { } assertRangefeedRetryErr := func( - t *testing.T, pErr *roachpb.Error, expReason roachpb.RangeFeedRetryError_Reason, + t *testing.T, pErr *kvpb.Error, expReason kvpb.RangeFeedRetryError_Reason, ) { t.Helper() - expErr := roachpb.NewRangeFeedRetryError(expReason) + expErr := kvpb.NewRangeFeedRetryError(expReason) if pErr == nil { t.Fatalf("got nil error for RangeFeed: expecting %v", expErr) } - rfErr, ok := pErr.GetDetail().(*roachpb.RangeFeedRetryError) + rfErr, ok := pErr.GetDetail().(*kvpb.RangeFeedRetryError) if !ok { t.Fatalf("got incorrect error for RangeFeed: %v; expecting %v", pErr, expErr) } @@ -604,14 +605,14 @@ func TestReplicaRangefeedErrors(t *testing.T) { } } - t.Run(roachpb.RangeFeedRetryError_REASON_REPLICA_REMOVED.String(), func(t *testing.T) { + t.Run(kvpb.RangeFeedRetryError_REASON_REPLICA_REMOVED.String(), func(t *testing.T) { const removeStore = 2 tc, rangeID := setup(t, base.TestingKnobs{}) defer tc.Stopper().Stop(ctx) // Establish a rangefeed on the replica we plan to remove. stream := newTestStream() - streamErrC := make(chan *roachpb.Error, 1) + streamErrC := make(chan *kvpb.Error, 1) rangefeedSpan := mkSpan("a", "z") ts := tc.Servers[removeStore] store, err := ts.Stores().GetStore(ts.GetFirstStoreID()) @@ -619,8 +620,8 @@ func TestReplicaRangefeedErrors(t *testing.T) { t.Fatal(err) } go func() { - req := roachpb.RangeFeedRequest{ - Header: roachpb.Header{ + req := kvpb.RangeFeedRequest{ + Header: kvpb.Header{ RangeID: rangeID, }, Span: rangefeedSpan, @@ -638,15 +639,15 @@ func TestReplicaRangefeedErrors(t *testing.T) { // Check the error. pErr := <-streamErrC - assertRangefeedRetryErr(t, pErr, roachpb.RangeFeedRetryError_REASON_REPLICA_REMOVED) + assertRangefeedRetryErr(t, pErr, kvpb.RangeFeedRetryError_REASON_REPLICA_REMOVED) }) - t.Run(roachpb.RangeFeedRetryError_REASON_RANGE_SPLIT.String(), func(t *testing.T) { + t.Run(kvpb.RangeFeedRetryError_REASON_RANGE_SPLIT.String(), func(t *testing.T) { tc, rangeID := setup(t, base.TestingKnobs{}) defer tc.Stopper().Stop(ctx) // Establish a rangefeed on the replica we plan to split. stream := newTestStream() - streamErrC := make(chan *roachpb.Error, 1) + streamErrC := make(chan *kvpb.Error, 1) rangefeedSpan := mkSpan("a", "z") ts := tc.Servers[0] store, err := ts.Stores().GetStore(ts.GetFirstStoreID()) @@ -654,8 +655,8 @@ func TestReplicaRangefeedErrors(t *testing.T) { t.Fatal(err) } go func() { - req := roachpb.RangeFeedRequest{ - Header: roachpb.Header{ + req := kvpb.RangeFeedRequest{ + Header: kvpb.Header{ RangeID: rangeID, }, Span: rangefeedSpan, @@ -673,9 +674,9 @@ func TestReplicaRangefeedErrors(t *testing.T) { // Check the error. pErr := <-streamErrC - assertRangefeedRetryErr(t, pErr, roachpb.RangeFeedRetryError_REASON_RANGE_SPLIT) + assertRangefeedRetryErr(t, pErr, kvpb.RangeFeedRetryError_REASON_RANGE_SPLIT) }) - t.Run(roachpb.RangeFeedRetryError_REASON_RANGE_MERGED.String(), func(t *testing.T) { + t.Run(kvpb.RangeFeedRetryError_REASON_RANGE_MERGED.String(), func(t *testing.T) { tc, rangeID := setup(t, base.TestingKnobs{}) defer tc.Stopper().Stop(ctx) @@ -696,11 +697,11 @@ func TestReplicaRangefeedErrors(t *testing.T) { // Establish a rangefeed on the left replica. streamLeft := newTestStream() - streamLeftErrC := make(chan *roachpb.Error, 1) + streamLeftErrC := make(chan *kvpb.Error, 1) rangefeedLeftSpan := roachpb.Span{Key: mkKey("a"), EndKey: splitKey} go func() { - req := roachpb.RangeFeedRequest{ - Header: roachpb.Header{ + req := kvpb.RangeFeedRequest{ + Header: kvpb.Header{ RangeID: rangeID, }, Span: rangefeedLeftSpan, @@ -712,11 +713,11 @@ func TestReplicaRangefeedErrors(t *testing.T) { // Establish a rangefeed on the right replica. streamRight := newTestStream() - streamRightErrC := make(chan *roachpb.Error, 1) + streamRightErrC := make(chan *kvpb.Error, 1) rangefeedRightSpan := roachpb.Span{Key: splitKey, EndKey: mkKey("z")} go func() { - req := roachpb.RangeFeedRequest{ - Header: roachpb.Header{ + req := kvpb.RangeFeedRequest{ + Header: kvpb.Header{ RangeID: rightRangeID, }, Span: rangefeedRightSpan, @@ -738,10 +739,10 @@ func TestReplicaRangefeedErrors(t *testing.T) { // Check the errors. pErrLeft, pErrRight := <-streamLeftErrC, <-streamRightErrC - assertRangefeedRetryErr(t, pErrLeft, roachpb.RangeFeedRetryError_REASON_RANGE_MERGED) - assertRangefeedRetryErr(t, pErrRight, roachpb.RangeFeedRetryError_REASON_RANGE_MERGED) + assertRangefeedRetryErr(t, pErrLeft, kvpb.RangeFeedRetryError_REASON_RANGE_MERGED) + assertRangefeedRetryErr(t, pErrRight, kvpb.RangeFeedRetryError_REASON_RANGE_MERGED) }) - t.Run(roachpb.RangeFeedRetryError_REASON_RAFT_SNAPSHOT.String(), func(t *testing.T) { + t.Run(kvpb.RangeFeedRetryError_REASON_RAFT_SNAPSHOT.String(), func(t *testing.T) { tc, rangeID := setup(t, base.TestingKnobs{}) defer tc.Stopper().Stop(ctx) @@ -770,11 +771,11 @@ func TestReplicaRangefeedErrors(t *testing.T) { // Establish a rangefeed on the replica we plan to partition. stream := newTestStream() - streamErrC := make(chan *roachpb.Error, 1) + streamErrC := make(chan *kvpb.Error, 1) rangefeedSpan := mkSpan("a", "z") go func() { - req := roachpb.RangeFeedRequest{ - Header: roachpb.Header{ + req := kvpb.RangeFeedRequest{ + Header: kvpb.Header{ RangeID: rangeID, }, Span: rangefeedSpan, @@ -860,9 +861,9 @@ func TestReplicaRangefeedErrors(t *testing.T) { // Check the error. pErr := <-streamErrC - assertRangefeedRetryErr(t, pErr, roachpb.RangeFeedRetryError_REASON_RAFT_SNAPSHOT) + assertRangefeedRetryErr(t, pErr, kvpb.RangeFeedRetryError_REASON_RAFT_SNAPSHOT) }) - t.Run(roachpb.RangeFeedRetryError_REASON_LOGICAL_OPS_MISSING.String(), func(t *testing.T) { + t.Run(kvpb.RangeFeedRetryError_REASON_LOGICAL_OPS_MISSING.String(), func(t *testing.T) { knobs := base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ // This test splits off a range manually from system table ranges. @@ -895,13 +896,13 @@ func TestReplicaRangefeedErrors(t *testing.T) { // Establish a rangefeed. stream := newTestStream() - streamErrC := make(chan *roachpb.Error, 1) + streamErrC := make(chan *kvpb.Error, 1) endKey := keys.ScratchRangeMax rangefeedSpan := roachpb.Span{Key: startKey, EndKey: endKey} go func() { - req := roachpb.RangeFeedRequest{ - Header: roachpb.Header{ + req := kvpb.RangeFeedRequest{ + Header: kvpb.Header{ RangeID: rightRangeID, }, Span: rangefeedSpan, @@ -928,7 +929,7 @@ func TestReplicaRangefeedErrors(t *testing.T) { // Check the error. pErr := <-streamErrC - assertRangefeedRetryErr(t, pErr, roachpb.RangeFeedRetryError_REASON_LOGICAL_OPS_MISSING) + assertRangefeedRetryErr(t, pErr, kvpb.RangeFeedRetryError_REASON_LOGICAL_OPS_MISSING) }) t.Run("range key mismatch", func(t *testing.T) { knobs := base.TestingKnobs{ @@ -970,14 +971,14 @@ func TestReplicaRangefeedErrors(t *testing.T) { // Attempt to establish a rangefeed, sending the request to the LHS. stream := newTestStream() - streamErrC := make(chan *roachpb.Error, 1) + streamErrC := make(chan *kvpb.Error, 1) endKey := keys.ScratchRangeMax rangefeedSpan := roachpb.Span{Key: startKey, EndKey: endKey} go func() { - req := roachpb.RangeFeedRequest{ - Header: roachpb.Header{ + req := kvpb.RangeFeedRequest{ + Header: kvpb.Header{ RangeID: leftRangeID, }, Span: rangefeedSpan, @@ -989,7 +990,7 @@ func TestReplicaRangefeedErrors(t *testing.T) { // Check the error. pErr := <-streamErrC - if _, ok := pErr.GetDetail().(*roachpb.RangeKeyMismatchError); !ok { + if _, ok := pErr.GetDetail().(*kvpb.RangeKeyMismatchError); !ok { t.Fatalf("got incorrect error for RangeFeed: %v; expecting RangeKeyMismatchError", pErr) } @@ -997,8 +998,8 @@ func TestReplicaRangefeedErrors(t *testing.T) { // encounter errors. stream = newTestStream() go func() { - req := roachpb.RangeFeedRequest{ - Header: roachpb.Header{ + req := kvpb.RangeFeedRequest{ + Header: kvpb.Header{ RangeID: rightRangeID, }, Span: rangefeedSpan, @@ -1042,10 +1043,10 @@ func TestReplicaRangefeedMVCCHistoryMutationError(t *testing.T) { // Set up a rangefeed across a-c. stream := newTestStream() - streamErrC := make(chan *roachpb.Error, 1) + streamErrC := make(chan *kvpb.Error, 1) go func() { - req := roachpb.RangeFeedRequest{ - Header: roachpb.Header{RangeID: rangeID}, + req := kvpb.RangeFeedRequest{ + Header: kvpb.Header{RangeID: rangeID}, Span: roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("c")}, } timer := time.AfterFunc(10*time.Second, stream.Cancel) @@ -1068,8 +1069,8 @@ func TestReplicaRangefeedMVCCHistoryMutationError(t *testing.T) { // Apply a ClearRange command that mutates MVCC history across c-e. // This does not overlap with the rangefeed registration, and should // not disconnect it. - _, pErr = kv.SendWrapped(ctx, store.TestSender(), &roachpb.ClearRangeRequest{ - RequestHeader: roachpb.RequestHeader{ + _, pErr = kv.SendWrapped(ctx, store.TestSender(), &kvpb.ClearRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: roachpb.Key("c"), EndKey: roachpb.Key("e"), }, @@ -1081,8 +1082,8 @@ func TestReplicaRangefeedMVCCHistoryMutationError(t *testing.T) { // Apply a ClearRange command that mutates MVCC history across b-e. // This overlaps with the rangefeed, and should disconnect it. - _, pErr = kv.SendWrapped(ctx, store.TestSender(), &roachpb.ClearRangeRequest{ - RequestHeader: roachpb.RequestHeader{ + _, pErr = kv.SendWrapped(ctx, store.TestSender(), &kvpb.ClearRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: roachpb.Key("b"), EndKey: roachpb.Key("e"), }, @@ -1091,9 +1092,9 @@ func TestReplicaRangefeedMVCCHistoryMutationError(t *testing.T) { select { case pErr = <-streamErrC: require.NotNil(t, pErr) - var mvccErr *roachpb.MVCCHistoryMutationError + var mvccErr *kvpb.MVCCHistoryMutationError require.ErrorAs(t, pErr.GoError(), &mvccErr) - require.Equal(t, &roachpb.MVCCHistoryMutationError{ + require.Equal(t, &kvpb.MVCCHistoryMutationError{ Span: roachpb.Span{Key: roachpb.Key("b"), EndKey: roachpb.Key("e")}, }, mvccErr) case <-time.After(time.Second): @@ -1244,7 +1245,7 @@ func TestRangefeedCheckpointsRecoverFromLeaseExpiration(t *testing.T) { WallClock: manualClock, }, Store: &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(ctx context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + TestingRequestFilter: func(ctx context.Context, ba *kvpb.BatchRequest) *kvpb.Error { // Once reject is set, the test wants full control over the requests // evaluating on the scratch range. On that range, we'll reject // everything that's not triggered by the test because we want to only @@ -1269,7 +1270,7 @@ func TestRangefeedCheckpointsRecoverFromLeaseExpiration(t *testing.T) { return nil } log.Infof(ctx, "test rejecting request: %s", ba) - return roachpb.NewErrorf("test injected error") + return kvpb.NewErrorf("test injected error") }, }, }, @@ -1419,7 +1420,7 @@ func TestNewRangefeedForceLeaseRetry(t *testing.T) { WallClock: manualClock, }, Store: &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(ctx context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + TestingRequestFilter: func(ctx context.Context, ba *kvpb.BatchRequest) *kvpb.Error { // Once reject is set, the test wants full control over the requests // evaluating on the scratch range. On that range, we'll reject @@ -1442,7 +1443,7 @@ func TestNewRangefeedForceLeaseRetry(t *testing.T) { mockTimeout := contextutil.RunWithTimeout(ctx, "test", 0, func(ctx context.Context) error { <-ctx.Done(); return ctx.Err() }) timeoutSimulated = true - return roachpb.NewError(mockTimeout) + return kvpb.NewError(mockTimeout) } log.Infof(ctx, "lease succeeds this time") return nil @@ -1452,7 +1453,7 @@ func TestNewRangefeedForceLeaseRetry(t *testing.T) { return nil } log.Infof(ctx, "test rejecting request: %s", ba) - return roachpb.NewErrorf("test injected error") + return kvpb.NewErrorf("test injected error") }, }, }, diff --git a/pkg/kv/kvserver/replica_rankings_test.go b/pkg/kv/kvserver/replica_rankings_test.go index 3d3f8f5fef7c..0f8770d8459e 100644 --- a/pkg/kv/kvserver/replica_rankings_test.go +++ b/pkg/kv/kvserver/replica_rankings_test.go @@ -18,6 +18,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator" aload "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/load" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/load" @@ -115,18 +116,18 @@ func TestAddSSTQPSStat(t *testing.T) { sst, start, end := storageutils.MakeSST(t, ts.ClusterSettings(), sstKeys) requestSize := float64(len(sst)) - sstReq := &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, + sstReq := &kvpb.AddSSTableRequest{ + RequestHeader: kvpb.RequestHeader{Key: start, EndKey: end}, Data: sst, MVCCStats: storageutils.SSTStats(t, sst, 0), } - get := &roachpb.GetRequest{ - RequestHeader: roachpb.RequestHeader{Key: start}, + get := &kvpb.GetRequest{ + RequestHeader: kvpb.RequestHeader{Key: start}, } - addSSTBA := &roachpb.BatchRequest{} - nonSSTBA := &roachpb.BatchRequest{} + addSSTBA := &kvpb.BatchRequest{} + nonSSTBA := &kvpb.BatchRequest{} addSSTBA.Add(sstReq) nonSSTBA.Add(get) @@ -137,7 +138,7 @@ func TestAddSSTQPSStat(t *testing.T) { testCases := []struct { addsstRequestFactor int expectedQPS float64 - ba *roachpb.BatchRequest + ba *kvpb.BatchRequest }{ {0, 1, addSSTBA}, {100, 1, nonSSTBA}, @@ -184,9 +185,9 @@ func TestAddSSTQPSStat(t *testing.T) { } // genVariableRead returns a batch request containing, start-end sequential key reads. -func genVariableRead(ctx context.Context, start, end roachpb.Key) *roachpb.BatchRequest { - scan := roachpb.NewScan(start, end, false) - readBa := &roachpb.BatchRequest{} +func genVariableRead(ctx context.Context, start, end roachpb.Key) *kvpb.BatchRequest { + scan := kvpb.NewScan(start, end, false) + readBa := &kvpb.BatchRequest{} readBa.Add(scan) return readBa } @@ -318,35 +319,35 @@ func TestReadLoadMetricAccounting(t *testing.T) { nextKey = nextKey.Next() } sst, start, end := storageutils.MakeSST(t, ts.ClusterSettings(), sstKeys) - sstReq := &roachpb.AddSSTableRequest{ - RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, + sstReq := &kvpb.AddSSTableRequest{ + RequestHeader: kvpb.RequestHeader{Key: start, EndKey: end}, Data: sst, MVCCStats: storageutils.SSTStats(t, sst, 0), } - addSSTBA := &roachpb.BatchRequest{} + addSSTBA := &kvpb.BatchRequest{} addSSTBA.Add(sstReq) // Send an AddSSTRequest once to create the key range. _, pErr := db.NonTransactionalSender().Send(ctx, addSSTBA) require.Nil(t, pErr) - get := &roachpb.GetRequest{ - RequestHeader: roachpb.RequestHeader{Key: start}, + get := &kvpb.GetRequest{ + RequestHeader: kvpb.RequestHeader{Key: start}, } - getReadBA := &roachpb.BatchRequest{} + getReadBA := &kvpb.BatchRequest{} getReadBA.Add(get) - scan := &roachpb.ScanRequest{ - RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, + scan := &kvpb.ScanRequest{ + RequestHeader: kvpb.RequestHeader{Key: start, EndKey: end}, } - scanReadBA := &roachpb.BatchRequest{} + scanReadBA := &kvpb.BatchRequest{} scanReadBA.Add(scan) testCases := []struct { - ba *roachpb.BatchRequest + ba *kvpb.BatchRequest expectedRQPS float64 expectedWPS float64 expectedRPS float64 diff --git a/pkg/kv/kvserver/replica_rate_limit.go b/pkg/kv/kvserver/replica_rate_limit.go index d98bba155088..10fe78173cf1 100644 --- a/pkg/kv/kvserver/replica_rate_limit.go +++ b/pkg/kv/kvserver/replica_rate_limit.go @@ -13,6 +13,7 @@ package kvserver import ( "context" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcostmodel" "github.com/cockroachdb/cockroach/pkg/roachpb" ) @@ -20,7 +21,7 @@ import ( // maybeRateLimitBatch may block the batch waiting to be rate-limited. Note that // the replica must be initialized and thus there is no synchronization issue // on the tenantRateLimiter. -func (r *Replica) maybeRateLimitBatch(ctx context.Context, ba *roachpb.BatchRequest) error { +func (r *Replica) maybeRateLimitBatch(ctx context.Context, ba *kvpb.BatchRequest) error { if r.tenantLimiter == nil { return nil } @@ -35,7 +36,7 @@ func (r *Replica) maybeRateLimitBatch(ctx context.Context, ba *roachpb.BatchRequ // recordImpactOnRateLimiter is used to record a read against the tenant rate // limiter. func (r *Replica) recordImpactOnRateLimiter( - ctx context.Context, br *roachpb.BatchResponse, isReadOnly bool, + ctx context.Context, br *kvpb.BatchResponse, isReadOnly bool, ) { if r.tenantLimiter == nil || br == nil || !isReadOnly { return diff --git a/pkg/kv/kvserver/replica_read.go b/pkg/kv/kvserver/replica_read.go index fd3309147883..ed39fc05d42a 100644 --- a/pkg/kv/kvserver/replica_read.go +++ b/pkg/kv/kvserver/replica_read.go @@ -14,6 +14,7 @@ import ( "context" "sync" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" @@ -41,12 +42,12 @@ import ( // iterator to evaluate the batch and then updates the timestamp cache to // reflect the key spans that it read. func (r *Replica) executeReadOnlyBatch( - ctx context.Context, ba *roachpb.BatchRequest, g *concurrency.Guard, + ctx context.Context, ba *kvpb.BatchRequest, g *concurrency.Guard, ) ( - br *roachpb.BatchResponse, + br *kvpb.BatchResponse, _ *concurrency.Guard, _ *kvadmission.StoreWriteBytes, - pErr *roachpb.Error, + pErr *kvpb.Error, ) { r.readOnlyCmdMu.RLock() defer r.readOnlyCmdMu.RUnlock() @@ -54,7 +55,7 @@ func (r *Replica) executeReadOnlyBatch( // Verify that the batch can be executed. st, err := r.checkExecutionCanProceedBeforeStorageSnapshot(ctx, ba, g) if err != nil { - return nil, g, nil, roachpb.NewError(err) + return nil, g, nil, kvpb.NewError(err) } if fn := r.store.TestingKnobs().PreStorageSnapshotButChecksCompleteInterceptor; fn != nil { @@ -82,7 +83,7 @@ func (r *Replica) executeReadOnlyBatch( // based off the state of the engine as of this point and are mutually // consistent. if err := rw.PinEngineStateForIterators(); err != nil { - return nil, g, nil, roachpb.NewError(err) + return nil, g, nil, kvpb.NewError(err) } if util.RaceEnabled { rw = spanset.NewReadWriterAt(rw, g.LatchSpans(), ba.Timestamp) @@ -90,7 +91,7 @@ func (r *Replica) executeReadOnlyBatch( defer rw.Close() if err := r.checkExecutionCanProceedAfterStorageSnapshot(ctx, ba, st); err != nil { - return nil, g, nil, roachpb.NewError(err) + return nil, g, nil, kvpb.NewError(err) } ok, stillNeedsInterleavedIntents, pErr := r.canDropLatchesBeforeEval(ctx, rw, ba, g, st) if pErr != nil { @@ -139,7 +140,7 @@ func (r *Replica) executeReadOnlyBatch( // conflicts for by using collectSpansRead as done below in the // non-error path. if !g.CheckOptimisticNoLatchConflicts() { - return nil, g, nil, roachpb.NewError(roachpb.NewOptimisticEvalConflictsError()) + return nil, g, nil, kvpb.NewError(kvpb.NewOptimisticEvalConflictsError()) } } pErr = maybeAttachLease(pErr, &st.Lease) @@ -153,19 +154,19 @@ func (r *Replica) executeReadOnlyBatch( // the response. latchSpansRead, lockSpansRead, err := r.collectSpansRead(ba, br) if err != nil { - return nil, g, nil, roachpb.NewError(err) + return nil, g, nil, kvpb.NewError(err) } defer latchSpansRead.Release() defer lockSpansRead.Release() if ok := g.CheckOptimisticNoConflicts(latchSpansRead, lockSpansRead); !ok { - return nil, g, nil, roachpb.NewError(roachpb.NewOptimisticEvalConflictsError()) + return nil, g, nil, kvpb.NewError(kvpb.NewOptimisticEvalConflictsError()) } } else { // There was an error, that was not classified as a concurrency retry // error, and this request was not holding latches. This should be rare, // and in the interest of not having subtle correctness bugs, we retry // pessimistically. - return nil, g, nil, roachpb.NewError(roachpb.NewOptimisticEvalConflictsError()) + return nil, g, nil, kvpb.NewError(kvpb.NewOptimisticEvalConflictsError()) } } @@ -200,7 +201,7 @@ func (r *Replica) executeReadOnlyBatch( // if the request originated from the local node which means the local // range descriptor cache has an in-flight RangeLookup request which // prohibits any concurrent requests for the same range. See #17760. - allowSyncProcessing := ba.ReadConsistency == roachpb.CONSISTENT && + allowSyncProcessing := ba.ReadConsistency == kvpb.CONSISTENT && ba.WaitPolicy != lock.WaitPolicy_SkipLocked if err := r.store.intentResolver.CleanupIntentsAsync( ctx, @@ -237,9 +238,9 @@ func (r *Replica) executeReadOnlyBatch( func (r *Replica) updateTimestampCacheAndDropLatches( ctx context.Context, g *concurrency.Guard, - ba *roachpb.BatchRequest, - br *roachpb.BatchResponse, - pErr *roachpb.Error, + ba *kvpb.BatchRequest, + br *kvpb.BatchResponse, + pErr *kvpb.Error, st kvserverpb.LeaseStatus, ) { ec := endCmds{repl: r, g: g, st: st} @@ -281,10 +282,10 @@ var allowDroppingLatchesBeforeEval = settings.RegisterBoolSetting( func (r *Replica) canDropLatchesBeforeEval( ctx context.Context, rw storage.ReadWriter, - ba *roachpb.BatchRequest, + ba *kvpb.BatchRequest, g *concurrency.Guard, st kvserverpb.LeaseStatus, -) (ok, stillNeedsIntentInterleaving bool, pErr *roachpb.Error) { +) (ok, stillNeedsIntentInterleaving bool, pErr *kvpb.Error) { if !allowDroppingLatchesBeforeEval.Get(&r.store.cfg.Settings.SV) || !canReadOnlyRequestDropLatchesBeforeEval(ba, g) { // If the request does not qualify, we neither drop latches nor use a @@ -311,7 +312,7 @@ func (r *Replica) canDropLatchesBeforeEval( ctx, rw, txnID, ba.Header.Timestamp, start, end, seq, &intents, maxIntents, ) if err != nil { - return false /* ok */, true /* stillNeedsIntentInterleaving */, roachpb.NewError( + return false /* ok */, true /* stillNeedsIntentInterleaving */, kvpb.NewError( errors.Wrap(err, "scanning intents"), ) } @@ -322,7 +323,7 @@ func (r *Replica) canDropLatchesBeforeEval( } if len(intents) > 0 { return false /* ok */, false /* stillNeedsIntentInterleaving */, maybeAttachLease( - roachpb.NewError(&roachpb.WriteIntentError{Intents: intents}), &st.Lease, + kvpb.NewError(&kvpb.WriteIntentError{Intents: intents}), &st.Lease, ) } // If there were no conflicts, then the request can drop its latches and @@ -398,12 +399,12 @@ func (r *Replica) executeReadOnlyBatchWithServersideRefreshes( ctx context.Context, rw storage.ReadWriter, rec batcheval.EvalContext, - ba *roachpb.BatchRequest, + ba *kvpb.BatchRequest, g *concurrency.Guard, st *kvserverpb.LeaseStatus, ui uncertainty.Interval, evalPath batchEvalPath, -) (br *roachpb.BatchResponse, res result.Result, pErr *roachpb.Error) { +) (br *kvpb.BatchResponse, res result.Result, pErr *kvpb.Error) { log.Event(ctx, "executing read-only batch") var rootMonitor *mon.BytesMonitor @@ -424,7 +425,7 @@ func (r *Replica) executeReadOnlyBatchWithServersideRefreshes( // TODO(sumeer): for multi-tenant KV we should be accounting on a per-tenant // basis and not letting a single tenant consume all the memory (we could // place a limit equal to total/2). - if ba.AdmissionHeader.SourceLocation != roachpb.AdmissionHeader_LOCAL || + if ba.AdmissionHeader.SourceLocation != kvpb.AdmissionHeader_LOCAL || ba.AdmissionHeader.NoMemoryReservedAtSource { // rootMonitor will never be nil in production settings, but it can be nil // for tests that do not have a monitor. @@ -492,8 +493,8 @@ func (r *Replica) executeReadOnlyBatchWithServersideRefreshes( } func (r *Replica) handleReadOnlyLocalEvalResult( - ctx context.Context, ba *roachpb.BatchRequest, lResult result.LocalResult, -) *roachpb.Error { + ctx context.Context, ba *kvpb.BatchRequest, lResult result.LocalResult, +) *kvpb.Error { // Fields for which no action is taken in this method are zeroed so that // they don't trigger an assertion at the end of the method (which checks // that all fields were handled). @@ -520,17 +521,17 @@ func (r *Replica) handleReadOnlyLocalEvalResult( // spans in the responses, to construct the effective spans that were read, // and uses that to compute the latch and lock spans. func (r *Replica) collectSpansRead( - ba *roachpb.BatchRequest, br *roachpb.BatchResponse, + ba *kvpb.BatchRequest, br *kvpb.BatchResponse, ) (latchSpans, lockSpans *spanset.SpanSet, _ error) { baCopy := *ba - baCopy.Requests = make([]roachpb.RequestUnion, 0, len(ba.Requests)) + baCopy.Requests = make([]kvpb.RequestUnion, 0, len(ba.Requests)) for i := 0; i < len(ba.Requests); i++ { baReq := ba.Requests[i] req := baReq.GetInner() header := req.Header() resp := br.Responses[i].GetInner() - if ba.WaitPolicy == lock.WaitPolicy_SkipLocked && roachpb.CanSkipLocked(req) { + if ba.WaitPolicy == lock.WaitPolicy_SkipLocked && kvpb.CanSkipLocked(req) { if ba.IndexFetchSpec != nil { return nil, nil, errors.AssertionFailedf("unexpectedly IndexFetchSpec is set with SKIP LOCKED wait policy") } @@ -549,18 +550,18 @@ func (r *Replica) collectSpansRead( // // This is similar to how the timestamp cache and refresh spans handle the // SkipLocked wait policy. - if err := roachpb.ResponseKeyIterate(req, resp, func(key roachpb.Key) { + if err := kvpb.ResponseKeyIterate(req, resp, func(key roachpb.Key) { // TODO(nvanbenschoten): we currently perform a per-response key memory // allocation. If this becomes an issue, we could pre-allocate chunks of // these structs to amortize the cost. getAlloc := new(struct { - get roachpb.GetRequest - union roachpb.RequestUnion_Get + get kvpb.GetRequest + union kvpb.RequestUnion_Get }) getAlloc.get.Key = key - getAlloc.get.KeyLocking = req.(roachpb.LockingReadRequest).KeyLockingStrength() + getAlloc.get.KeyLocking = req.(kvpb.LockingReadRequest).KeyLockingStrength() getAlloc.union.Get = &getAlloc.get - ru := roachpb.RequestUnion{Value: &getAlloc.union} + ru := kvpb.RequestUnion{Value: &getAlloc.union} baCopy.Requests = append(baCopy.Requests, ru) }); err != nil { return nil, nil, err @@ -575,10 +576,10 @@ func (r *Replica) collectSpansRead( } switch t := resp.(type) { - case *roachpb.GetResponse: + case *kvpb.GetResponse: // The request did not evaluate. Ignore it. continue - case *roachpb.ScanResponse: + case *kvpb.ScanResponse: if header.Key.Equal(t.ResumeSpan.Key) { // The request did not evaluate. Ignore it. continue @@ -587,7 +588,7 @@ func (r *Replica) collectSpansRead( // ResumeSpan.Key has not been read and becomes the exclusive end key of // what was read. header.EndKey = t.ResumeSpan.Key - case *roachpb.ReverseScanResponse: + case *kvpb.ReverseScanResponse: if header.EndKey.Equal(t.ResumeSpan.EndKey) { // The request did not evaluate. Ignore it. continue @@ -602,7 +603,7 @@ func (r *Replica) collectSpansRead( continue } // The ResumeSpan has changed the header. - var ru roachpb.RequestUnion + var ru kvpb.RequestUnion req = req.ShallowCopy() req.SetHeader(header) ru.MustSetInner(req) @@ -616,7 +617,7 @@ func (r *Replica) collectSpansRead( return latchSpans, lockSpans, err } -func getBatchResponseReadStats(br *roachpb.BatchResponse) (float64, float64) { +func getBatchResponseReadStats(br *kvpb.BatchResponse) (float64, float64) { var keys, bytes float64 for _, reply := range br.Responses { h := reply.GetInner().Header() diff --git a/pkg/kv/kvserver/replica_send.go b/pkg/kv/kvserver/replica_send.go index 96be9a7f7242..3d36d1ff81ec 100644 --- a/pkg/kv/kvserver/replica_send.go +++ b/pkg/kv/kvserver/replica_send.go @@ -15,6 +15,7 @@ import ( "reflect" "runtime/pprof" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" @@ -111,8 +112,8 @@ var optimisticEvalLimitedScans = settings.RegisterBoolSetting( // to commit the command, then signaling proposer and // applying the command) func (r *Replica) Send( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, *kvpb.Error) { br, writeBytes, pErr := r.SendWithWriteBytes(ctx, ba) writeBytes.Release() return br, pErr @@ -121,8 +122,8 @@ func (r *Replica) Send( // SendWithWriteBytes is the implementation of Send with an additional // *StoreWriteBytes return value. func (r *Replica) SendWithWriteBytes( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, *kvadmission.StoreWriteBytes, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, *kvadmission.StoreWriteBytes, *kvpb.Error) { if r.store.cfg.Settings.CPUProfileType() == cluster.CPUProfileWithLabels { defer pprof.SetGoroutineLabels(ctx) // Note: the defer statement captured the previous context. @@ -145,23 +146,23 @@ func (r *Replica) SendWithWriteBytes( isReadOnly := ba.IsReadOnly() if err := r.checkBatchRequest(ba, isReadOnly); err != nil { - return nil, nil, roachpb.NewError(err) + return nil, nil, kvpb.NewError(err) } if err := r.maybeBackpressureBatch(ctx, ba); err != nil { - return nil, nil, roachpb.NewError(err) + return nil, nil, kvpb.NewError(err) } if err := r.maybeRateLimitBatch(ctx, ba); err != nil { - return nil, nil, roachpb.NewError(err) + return nil, nil, kvpb.NewError(err) } if err := r.maybeCommitWaitBeforeCommitTrigger(ctx, ba); err != nil { - return nil, nil, roachpb.NewError(err) + return nil, nil, kvpb.NewError(err) } // NB: must be performed before collecting request spans. ba, err := maybeStripInFlightWrites(ba) if err != nil { - return nil, nil, roachpb.NewError(err) + return nil, nil, kvpb.NewError(err) } if filter := r.store.cfg.TestingKnobs.TestingRequestFilter; filter != nil { @@ -171,8 +172,8 @@ func (r *Replica) SendWithWriteBytes( } // Differentiate between read-write, read-only, and admin. - var br *roachpb.BatchResponse - var pErr *roachpb.Error + var br *kvpb.BatchResponse + var pErr *kvpb.Error var writeBytes *kvadmission.StoreWriteBytes if isReadOnly { log.Event(ctx, "read-only path") @@ -225,13 +226,13 @@ func (r *Replica) SendWithWriteBytes( // committing with its current provisional commit timestamp, it will not need to // commit wait any further. func (r *Replica) maybeCommitWaitBeforeCommitTrigger( - ctx context.Context, ba *roachpb.BatchRequest, + ctx context.Context, ba *kvpb.BatchRequest, ) error { - args, hasET := ba.GetArg(roachpb.EndTxn) + args, hasET := ba.GetArg(kvpb.EndTxn) if !hasET { return nil } - et := args.(*roachpb.EndTxnRequest) + et := args.(*kvpb.EndTxnRequest) if !et.Commit || et.InternalCommitTrigger == nil { // Not committing with a commit trigger. return nil @@ -303,7 +304,7 @@ func (r *Replica) maybeCommitWaitBeforeCommitTrigger( // maybeAddRangeInfoToResponse populates br.RangeInfo if the client doesn't // have up-to-date info about the range's descriptor and lease. func (r *Replica) maybeAddRangeInfoToResponse( - ctx context.Context, ba *roachpb.BatchRequest, br *roachpb.BatchResponse, + ctx context.Context, ba *kvpb.BatchRequest, br *kvpb.BatchResponse, ) { // Compare the client's info with the replica's info to detect if the client // has stale knowledge. Note that the client can have more recent knowledge @@ -379,8 +380,8 @@ func (r *Replica) maybeAddRangeInfoToResponse( // the function returns one of these errors, it must also pass ownership of the // concurrency guard back to the caller. type batchExecutionFn func( - *Replica, context.Context, *roachpb.BatchRequest, *concurrency.Guard, -) (*roachpb.BatchResponse, *concurrency.Guard, *kvadmission.StoreWriteBytes, *roachpb.Error) + *Replica, context.Context, *kvpb.BatchRequest, *concurrency.Guard, +) (*kvpb.BatchResponse, *concurrency.Guard, *kvadmission.StoreWriteBytes, *kvpb.Error) var _ batchExecutionFn = (*Replica).executeWriteBatch var _ batchExecutionFn = (*Replica).executeReadOnlyBatch @@ -400,8 +401,8 @@ var _ batchExecutionFn = (*Replica).executeReadOnlyBatch // a TransactionPushError it will propagate the error back to this method, which // handles the process of retrying batch execution after addressing the error. func (r *Replica) executeBatchWithConcurrencyRetries( - ctx context.Context, ba *roachpb.BatchRequest, fn batchExecutionFn, -) (br *roachpb.BatchResponse, writeBytes *kvadmission.StoreWriteBytes, pErr *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, fn batchExecutionFn, +) (br *kvpb.BatchResponse, writeBytes *kvadmission.StoreWriteBytes, pErr *kvpb.Error) { // Try to execute command; exit retry loop on success. var latchSpans, lockSpans *spanset.SpanSet var requestEvalKind concurrency.RequestEvalKind @@ -421,7 +422,7 @@ func (r *Replica) executeBatchWithConcurrencyRetries( for { // Exit loop if context has been canceled or timed out. if err := ctx.Err(); err != nil { - return nil, nil, roachpb.NewError(errors.Wrap(err, "aborted during Replica.Send")) + return nil, nil, kvpb.NewError(errors.Wrap(err, "aborted during Replica.Send")) } // Determine the maximal set of key spans that the batch will operate on. @@ -434,7 +435,7 @@ func (r *Replica) executeBatchWithConcurrencyRetries( var err error latchSpans, lockSpans, requestEvalKind, err = r.collectSpans(ba) if err != nil { - return nil, nil, roachpb.NewError(err) + return nil, nil, kvpb.NewError(err) } } @@ -442,7 +443,7 @@ func (r *Replica) executeBatchWithConcurrencyRetries( // this request completes. After latching, wait on any conflicting locks // to ensure that the request has full isolation during evaluation. This // returns a request guard that must be eventually released. - var resp []roachpb.ResponseUnion + var resp []kvpb.ResponseUnion g, resp, pErr = r.concMgr.SequenceReq(ctx, g, concurrency.Request{ Txn: ba.Txn, Timestamp: ba.Timestamp, @@ -466,14 +467,14 @@ func (r *Replica) executeBatchWithConcurrencyRetries( // circuit.ErrBreakerOpen into the chain as well so that we have the // invariant that all replica circuit breaker errors contain both // ErrBreakerOpen and ReplicaUnavailableError. - pErr = roachpb.NewError(r.replicaUnavailableError(errors.CombineErrors( + pErr = kvpb.NewError(r.replicaUnavailableError(errors.CombineErrors( errors.Mark(poisonErr, circuit.ErrBreakerOpen), r.breaker.Signal().Err(), ))) } return nil, nil, pErr } else if resp != nil { - br = new(roachpb.BatchResponse) + br = new(kvpb.BatchResponse) br.Responses = resp return br, nil, nil } @@ -518,19 +519,19 @@ func (r *Replica) executeBatchWithConcurrencyRetries( requestEvalKind = concurrency.PessimisticEval switch t := pErr.GetDetail().(type) { - case *roachpb.WriteIntentError: + case *kvpb.WriteIntentError: // Drop latches, but retain lock wait-queues. g.AssertLatches() if g, pErr = r.handleWriteIntentError(ctx, ba, g, pErr, t); pErr != nil { return nil, nil, pErr } - case *roachpb.TransactionPushError: + case *kvpb.TransactionPushError: // Drop latches, but retain lock wait-queues. g.AssertLatches() if g, pErr = r.handleTransactionPushError(ctx, ba, g, pErr, t); pErr != nil { return nil, nil, pErr } - case *roachpb.IndeterminateCommitError: + case *kvpb.IndeterminateCommitError: dropLatchesAndLockWaitQueues(true /* reuseLatchAndLockSpans */) // Then launch a task to handle the indeterminate commit error. No error // is returned if the transaction is recovered successfully to either a @@ -538,7 +539,7 @@ func (r *Replica) executeBatchWithConcurrencyRetries( if pErr = r.handleIndeterminateCommitError(ctx, ba, pErr, t); pErr != nil { return nil, nil, pErr } - case *roachpb.ReadWithinUncertaintyIntervalError: + case *kvpb.ReadWithinUncertaintyIntervalError: // If the batch is able to perform a server-side retry in order to avoid // the uncertainty error, it will have a new timestamp. Force a refresh of // the latch and lock spans. @@ -554,7 +555,7 @@ func (r *Replica) executeBatchWithConcurrencyRetries( if pErr != nil { return nil, nil, pErr } - case *roachpb.InvalidLeaseError: + case *kvpb.InvalidLeaseError: dropLatchesAndLockWaitQueues(true /* reuseLatchAndLockSpans */) // Then attempt to acquire the lease if not currently held by any // replica or redirect to the current leaseholder if currently held @@ -562,13 +563,13 @@ func (r *Replica) executeBatchWithConcurrencyRetries( if pErr = r.handleInvalidLeaseError(ctx, ba); pErr != nil { return nil, nil, pErr } - case *roachpb.MergeInProgressError: + case *kvpb.MergeInProgressError: dropLatchesAndLockWaitQueues(true /* reuseLatchAndLockSpans */) // Then listen for the merge to complete. if pErr = r.handleMergeInProgressError(ctx, ba, pErr, t); pErr != nil { return nil, nil, pErr } - case *roachpb.OptimisticEvalConflictsError: + case *kvpb.OptimisticEvalConflictsError: // We are deliberately not dropping latches. Note that the latches are // also optimistically acquired, in the sense of being inserted but not // waited on. The next iteration will wait on these latches to ensure @@ -591,24 +592,24 @@ func (r *Replica) executeBatchWithConcurrencyRetries( // concurrency manager, and executing the request again. The one exception is // OptimisticEvalConflictsError, where there is no need to drop latches, and // the request can immediately proceed to retrying pessimistically. -func isConcurrencyRetryError(pErr *roachpb.Error) bool { +func isConcurrencyRetryError(pErr *kvpb.Error) bool { switch pErr.GetDetail().(type) { - case *roachpb.WriteIntentError: + case *kvpb.WriteIntentError: // If a request hits a WriteIntentError, it adds the conflicting intent // to the lockTable through a process called "lock discovery". It then // waits in the lock's wait-queue during its next sequencing pass. - case *roachpb.TransactionPushError: + case *kvpb.TransactionPushError: // If a PushTxn request hits a TransactionPushError, it attempted to // push another transactions record but did not succeed. It enqueues the // pushee transaction in the txnWaitQueue and waits on the record to // change or expire during its next sequencing pass. - case *roachpb.IndeterminateCommitError: + case *kvpb.IndeterminateCommitError: // If a PushTxn hits a IndeterminateCommitError, it attempted to push an // expired transaction record in the STAGING state. It's unclear whether // the pushee is aborted or committed, so the request must kick off the // "transaction recovery procedure" to resolve this ambiguity before // retrying. - case *roachpb.ReadWithinUncertaintyIntervalError: + case *kvpb.ReadWithinUncertaintyIntervalError: // If a request hits a ReadWithinUncertaintyIntervalError, it was performing // a non-locking read [1] and encountered a (committed or provisional) write // within the uncertainty interval of the reader. Depending on the state of @@ -636,21 +637,21 @@ func isConcurrencyRetryError(pErr *roachpb.Error) bool { // WriteTooOld error. It's uncertainty interval does not matter. // [2] in practice, this is enforced by tryBumpBatchTimestamp's call to // (*concurrency.Guard).IsolatedAtLaterTimestamps. - case *roachpb.InvalidLeaseError: + case *kvpb.InvalidLeaseError: // If a request hits an InvalidLeaseError, the replica it is being // evaluated against does not have a valid lease under which it can // serve the request. The request cannot proceed until a new lease is // acquired. If the acquisition process determines that the lease now // lives elsewhere, the request should be redirected (using a // NotLeaseHolderError) to the new leaseholder. - case *roachpb.MergeInProgressError: + case *kvpb.MergeInProgressError: // If a request hits a MergeInProgressError, the replica it is being // evaluated against is in the process of being merged into its left-hand // neighbor. The request cannot proceed until the range merge completes, // either successfully or unsuccessfully, so it waits before retrying. // If the merge does complete successfully, the retry will be rejected // with an error that will propagate back to the client. - case *roachpb.OptimisticEvalConflictsError: + case *kvpb.OptimisticEvalConflictsError: // Optimistic evaluation encountered a conflict. The request will // immediately retry pessimistically. default: @@ -663,8 +664,8 @@ func isConcurrencyRetryError(pErr *roachpb.Error) bool { // information about the lease that the operation which hit this error was // operating under. If the operation was performed on a follower that does not // hold the lease (e.g. a follower read), the provided lease will be empty. -func maybeAttachLease(pErr *roachpb.Error, lease *roachpb.Lease) *roachpb.Error { - if wiErr, ok := pErr.GetDetail().(*roachpb.WriteIntentError); ok { +func maybeAttachLease(pErr *kvpb.Error, lease *roachpb.Lease) *kvpb.Error { + if wiErr, ok := pErr.GetDetail().(*kvpb.WriteIntentError); ok { // If we hit an intent on the leaseholder, attach information about the // lease to WriteIntentErrors, which is necessary to keep the lock-table // in sync with the applied state. @@ -709,21 +710,21 @@ func maybeAttachLease(pErr *roachpb.Error, lease *roachpb.Lease) *roachpb.Error // in CockroachDB. In Spanner, it is the "safe time" of a replica that // determines follower read eligibility. if lease.Empty() /* followerRead */ { - return roachpb.NewErrorWithTxn(&roachpb.InvalidLeaseError{}, pErr.GetTxn()) + return kvpb.NewErrorWithTxn(&kvpb.InvalidLeaseError{}, pErr.GetTxn()) } wiErr.LeaseSequence = lease.Sequence - return roachpb.NewErrorWithTxn(wiErr, pErr.GetTxn()) + return kvpb.NewErrorWithTxn(wiErr, pErr.GetTxn()) } return pErr } func (r *Replica) handleWriteIntentError( ctx context.Context, - ba *roachpb.BatchRequest, + ba *kvpb.BatchRequest, g *concurrency.Guard, - pErr *roachpb.Error, - t *roachpb.WriteIntentError, -) (*concurrency.Guard, *roachpb.Error) { + pErr *kvpb.Error, + t *kvpb.WriteIntentError, +) (*concurrency.Guard, *kvpb.Error) { if r.store.cfg.TestingKnobs.DontPushOnWriteIntentError { return g, pErr } @@ -733,18 +734,18 @@ func (r *Replica) handleWriteIntentError( func (r *Replica) handleTransactionPushError( ctx context.Context, - ba *roachpb.BatchRequest, + ba *kvpb.BatchRequest, g *concurrency.Guard, - pErr *roachpb.Error, - t *roachpb.TransactionPushError, -) (*concurrency.Guard, *roachpb.Error) { + pErr *kvpb.Error, + t *kvpb.TransactionPushError, +) (*concurrency.Guard, *kvpb.Error) { // On a transaction push error, retry immediately if doing so will enqueue // into the txnWaitQueue in order to await further updates to the unpushed // txn's status. We check ShouldPushImmediately to avoid retrying // non-queueable PushTxnRequests (see #18191). dontRetry := r.store.cfg.TestingKnobs.DontRetryPushTxnFailures if !dontRetry && ba.IsSinglePushTxnRequest() { - pushReq := ba.Requests[0].GetInner().(*roachpb.PushTxnRequest) + pushReq := ba.Requests[0].GetInner().(*kvpb.PushTxnRequest) dontRetry = txnwait.ShouldPushImmediately(pushReq) } if dontRetry { @@ -756,11 +757,8 @@ func (r *Replica) handleTransactionPushError( } func (r *Replica) handleIndeterminateCommitError( - ctx context.Context, - ba *roachpb.BatchRequest, - pErr *roachpb.Error, - t *roachpb.IndeterminateCommitError, -) *roachpb.Error { + ctx context.Context, ba *kvpb.BatchRequest, pErr *kvpb.Error, t *kvpb.IndeterminateCommitError, +) *kvpb.Error { if r.store.cfg.TestingKnobs.DontRecoverIndeterminateCommits { return pErr } @@ -768,11 +766,11 @@ func (r *Replica) handleIndeterminateCommitError( // stuck transaction. Retry immediately if successful. if _, err := r.store.recoveryMgr.ResolveIndeterminateCommit(ctx, t); err != nil { // Do not propagate ambiguous results; assume success and retry original op. - if errors.HasType(err, (*roachpb.AmbiguousResultError)(nil)) { + if errors.HasType(err, (*kvpb.AmbiguousResultError)(nil)) { return nil } // Propagate new error. Preserve the error index. - newPErr := roachpb.NewError(err) + newPErr := kvpb.NewError(err) newPErr.Index = pErr.Index return newPErr } @@ -782,10 +780,10 @@ func (r *Replica) handleIndeterminateCommitError( func (r *Replica) handleReadWithinUncertaintyIntervalError( ctx context.Context, - ba *roachpb.BatchRequest, - pErr *roachpb.Error, - t *roachpb.ReadWithinUncertaintyIntervalError, -) (*roachpb.BatchRequest, *roachpb.Error) { + ba *kvpb.BatchRequest, + pErr *kvpb.Error, + t *kvpb.ReadWithinUncertaintyIntervalError, +) (*kvpb.BatchRequest, *kvpb.Error) { // Attempt a server-side retry of the request. Note that we pass nil for // latchSpans, because we have already released our latches and plan to // re-acquire them if the retry is allowed. @@ -809,15 +807,13 @@ func (r *Replica) handleReadWithinUncertaintyIntervalError( ctx, cancel = r.store.Stopper().WithCancelOnQuiesce(ctx) defer cancel() if err := r.Clock().SleepUntil(ctx, ba.Timestamp); err != nil { - return nil, roachpb.NewError(err) + return nil, kvpb.NewError(err) } } return ba, nil } -func (r *Replica) handleInvalidLeaseError( - ctx context.Context, ba *roachpb.BatchRequest, -) *roachpb.Error { +func (r *Replica) handleInvalidLeaseError(ctx context.Context, ba *kvpb.BatchRequest) *kvpb.Error { // On an invalid lease error, attempt to acquire a new lease. If in the // process of doing so, we determine that the lease now lives elsewhere, // redirect. @@ -828,11 +824,8 @@ func (r *Replica) handleInvalidLeaseError( } func (r *Replica) handleMergeInProgressError( - ctx context.Context, - ba *roachpb.BatchRequest, - pErr *roachpb.Error, - t *roachpb.MergeInProgressError, -) *roachpb.Error { + ctx context.Context, ba *kvpb.BatchRequest, pErr *kvpb.Error, t *kvpb.MergeInProgressError, +) *kvpb.Error { // A merge was in progress. We need to retry the command after the merge // completes, as signaled by the closing of the replica's mergeComplete // channel. Note that the merge may have already completed, in which case @@ -856,7 +849,7 @@ func (r *Replica) handleMergeInProgressError( // after the lease transfer is initiated but before the lease transfer // acquires latches. if ba.IsSingleTransferLeaseRequest() { - return roachpb.NewErrorf("cannot transfer lease while merge in progress") + return kvpb.NewErrorf("cannot transfer lease while merge in progress") } log.Event(ctx, "waiting on in-progress range merge") select { @@ -864,9 +857,9 @@ func (r *Replica) handleMergeInProgressError( // Merge complete. Retry the command. return nil case <-ctx.Done(): - return roachpb.NewError(errors.Wrap(ctx.Err(), "aborted during merge")) + return kvpb.NewError(errors.Wrap(ctx.Err(), "aborted during merge")) case <-r.store.stopper.ShouldQuiesce(): - return roachpb.NewError(&roachpb.NodeUnavailableError{}) + return kvpb.NewError(&kvpb.NodeUnavailableError{}) } } @@ -876,10 +869,10 @@ func (r *Replica) handleMergeInProgressError( // Admin commands must run on the lease holder replica. Batch support here is // limited to single-element batches; everything else catches an error. func (r *Replica) executeAdminBatch( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, *kvpb.Error) { if len(ba.Requests) != 1 { - return nil, roachpb.NewErrorf("only single-element admin batches allowed") + return nil, kvpb.NewErrorf("only single-element admin batches allowed") } args := ba.Requests[0].GetInner() @@ -897,7 +890,7 @@ func (r *Replica) executeAdminBatch( // request sends will acquire latches. for { if err := ctx.Err(); err != nil { - return nil, roachpb.NewError(err) + return nil, kvpb.NewError(err) } _, err := r.checkExecutionCanProceedRWOrAdmin(ctx, ba, nil /* g */) @@ -908,7 +901,7 @@ func (r *Replica) executeAdminBatch( break } switch { - case errors.HasType(err, (*roachpb.InvalidLeaseError)(nil)): + case errors.HasType(err, (*kvpb.InvalidLeaseError)(nil)): // If the replica does not have the lease, attempt to acquire it, or // redirect to the current leaseholder by returning an error. _, pErr := r.redirectOnOrAcquireLeaseForRequest(ctx, ba.Timestamp, r.signallerForBatch(ba)) @@ -917,45 +910,45 @@ func (r *Replica) executeAdminBatch( } // Retry... default: - return nil, roachpb.NewError(err) + return nil, kvpb.NewError(err) } } - var resp roachpb.Response - var pErr *roachpb.Error + var resp kvpb.Response + var pErr *kvpb.Error switch tArgs := args.(type) { - case *roachpb.AdminSplitRequest: - var reply roachpb.AdminSplitResponse + case *kvpb.AdminSplitRequest: + var reply kvpb.AdminSplitResponse reply, pErr = r.AdminSplit(ctx, *tArgs, "manual") resp = &reply - case *roachpb.AdminUnsplitRequest: - var reply roachpb.AdminUnsplitResponse + case *kvpb.AdminUnsplitRequest: + var reply kvpb.AdminUnsplitResponse reply, pErr = r.AdminUnsplit(ctx, *tArgs, "manual") resp = &reply - case *roachpb.AdminMergeRequest: - var reply roachpb.AdminMergeResponse + case *kvpb.AdminMergeRequest: + var reply kvpb.AdminMergeResponse reply, pErr = r.AdminMerge(ctx, *tArgs, "manual") resp = &reply - case *roachpb.AdminTransferLeaseRequest: - pErr = roachpb.NewError(r.AdminTransferLease(ctx, tArgs.Target, tArgs.BypassSafetyChecks)) - resp = &roachpb.AdminTransferLeaseResponse{} + case *kvpb.AdminTransferLeaseRequest: + pErr = kvpb.NewError(r.AdminTransferLease(ctx, tArgs.Target, tArgs.BypassSafetyChecks)) + resp = &kvpb.AdminTransferLeaseResponse{} - case *roachpb.AdminChangeReplicasRequest: + case *kvpb.AdminChangeReplicasRequest: chgs := tArgs.Changes() desc, err := r.ChangeReplicas(ctx, &tArgs.ExpDesc, kvserverpb.SnapshotRequest_REBALANCE, kvserverpb.ReasonAdminRequest, "", chgs) - pErr = roachpb.NewError(err) + pErr = kvpb.NewError(err) if pErr != nil { - resp = &roachpb.AdminChangeReplicasResponse{} + resp = &kvpb.AdminChangeReplicasResponse{} } else { - resp = &roachpb.AdminChangeReplicasResponse{ + resp = &kvpb.AdminChangeReplicasResponse{ Desc: *desc, } } - case *roachpb.AdminRelocateRangeRequest: + case *kvpb.AdminRelocateRangeRequest: // Transferring the lease to the first voting replica in the target slice is // pre-22.1 behavior. // We revert to that behavior if the request is coming @@ -966,39 +959,39 @@ func (r *Replica) executeAdminBatch( err := r.AdminRelocateRange( ctx, *r.Desc(), tArgs.VoterTargets, tArgs.NonVoterTargets, transferLeaseToFirstVoter, ) - pErr = roachpb.NewError(err) - resp = &roachpb.AdminRelocateRangeResponse{} + pErr = kvpb.NewError(err) + resp = &kvpb.AdminRelocateRangeResponse{} - case *roachpb.CheckConsistencyRequest: - var reply roachpb.CheckConsistencyResponse + case *kvpb.CheckConsistencyRequest: + var reply kvpb.CheckConsistencyResponse reply, pErr = r.CheckConsistency(ctx, *tArgs) resp = &reply - case *roachpb.AdminScatterRequest: + case *kvpb.AdminScatterRequest: reply, err := r.adminScatter(ctx, *tArgs) - pErr = roachpb.NewError(err) + pErr = kvpb.NewError(err) resp = &reply - case *roachpb.AdminVerifyProtectedTimestampRequest: + case *kvpb.AdminVerifyProtectedTimestampRequest: reply, err := r.adminVerifyProtectedTimestamp(ctx, *tArgs) - pErr = roachpb.NewError(err) + pErr = kvpb.NewError(err) resp = &reply default: - return nil, roachpb.NewErrorf("unrecognized admin command: %T", args) + return nil, kvpb.NewErrorf("unrecognized admin command: %T", args) } if pErr != nil { return nil, pErr } - br := &roachpb.BatchResponse{} + br := &kvpb.BatchResponse{} br.Add(resp) br.Txn = resp.Header().Txn return br, nil } -func (r *Replica) recordBatchRequestLoad(ctx context.Context, ba *roachpb.BatchRequest) { +func (r *Replica) recordBatchRequestLoad(ctx context.Context, ba *kvpb.BatchRequest) { if r.loadStats == nil { log.VEventf( ctx, @@ -1025,7 +1018,7 @@ func (r *Replica) recordBatchRequestLoad(ctx context.Context, ba *roachpb.BatchR // AddSSTableRequest's data size is divided by a factor and added to QPS. This // specific treatment of QPS is a special case to account for the mismatch // between AddSSTableRequest and other requests in terms of resource use. -func (r *Replica) getBatchRequestQPS(ctx context.Context, ba *roachpb.BatchRequest) float64 { +func (r *Replica) getBatchRequestQPS(ctx context.Context, ba *kvpb.BatchRequest) float64 { var count float64 = 1 // For divisors less than 1, use the default treatment of QPS. @@ -1037,7 +1030,7 @@ func (r *Replica) getBatchRequestQPS(ctx context.Context, ba *roachpb.BatchReque var addSSTSize float64 = 0 for _, req := range ba.Requests { switch t := req.GetInner().(type) { - case *roachpb.AddSSTableRequest: + case *kvpb.AddSSTableRequest: addSSTSize += float64(len(t.Data)) default: continue @@ -1065,14 +1058,14 @@ func (r *Replica) recordRequestWriteBytes(writeBytes *kvadmission.StoreWriteByte // // TODO(tschottdorf): should check that request is contained in range and that // EndTxn only occurs at the very end. -func (r *Replica) checkBatchRequest(ba *roachpb.BatchRequest, isReadOnly bool) error { +func (r *Replica) checkBatchRequest(ba *kvpb.BatchRequest, isReadOnly bool) error { if ba.Timestamp.IsEmpty() { // For transactional requests, Store.Send sets the timestamp. For non- // transactional requests, the client sets the timestamp. Either way, we // need to have a timestamp at this point. return errors.New("Replica.checkBatchRequest: batch does not have timestamp assigned") } - consistent := ba.ReadConsistency == roachpb.CONSISTENT + consistent := ba.ReadConsistency == kvpb.CONSISTENT if isReadOnly { if !consistent && ba.Txn != nil { // Disallow any inconsistent reads within txns. @@ -1086,7 +1079,7 @@ func (r *Replica) checkBatchRequest(ba *roachpb.BatchRequest, isReadOnly bool) e } func (r *Replica) collectSpans( - ba *roachpb.BatchRequest, + ba *kvpb.BatchRequest, ) (latchSpans, lockSpans *spanset.SpanSet, requestEvalKind concurrency.RequestEvalKind, _ error) { latchSpans, lockSpans = spanset.New(), spanset.New() r.mu.RLock() @@ -1105,9 +1098,9 @@ func (r *Replica) collectSpans( // use. if ba.IsLocking() { latchGuess := len(ba.Requests) - if et, ok := ba.GetArg(roachpb.EndTxn); ok { + if et, ok := ba.GetArg(kvpb.EndTxn); ok { // EndTxn declares a global write for each of its lock spans. - latchGuess += len(et.(*roachpb.EndTxnRequest).LockSpans) - 1 + latchGuess += len(et.(*kvpb.EndTxnRequest).LockSpans) - 1 } latchSpans.Reserve(spanset.SpanReadWrite, spanset.SpanGlobal, latchGuess) lockSpans.Reserve(spanset.SpanReadWrite, spanset.SpanGlobal, len(ba.Requests)) @@ -1165,9 +1158,9 @@ func (r *Replica) collectSpans( cmd.DeclareKeys(desc, &ba.Header, inner, latchSpans, lockSpans, r.Clock().MaxOffset()) if considerOptEvalForLimit { switch inner.(type) { - case *roachpb.ScanRequest, *roachpb.ReverseScanRequest: + case *kvpb.ScanRequest, *kvpb.ReverseScanRequest: hasScans = true - case *roachpb.GetRequest: + case *kvpb.GetRequest: numGets++ } } @@ -1262,7 +1255,7 @@ func (ec *endCmds) poison() { // No-op if the receiver has been zeroed out by a call to move. Idempotent and // is safe to call more than once. func (ec *endCmds) done( - ctx context.Context, ba *roachpb.BatchRequest, br *roachpb.BatchResponse, pErr *roachpb.Error, + ctx context.Context, ba *kvpb.BatchRequest, br *kvpb.BatchResponse, pErr *kvpb.Error, ) { if ec.repl == nil { // The endCmds were cleared. @@ -1274,7 +1267,7 @@ func (ec *endCmds) done( // turn; only those marked as affecting the cache are processed. However, only // do so if the request is consistent and was operating on the leaseholder // under a valid range lease. - if ba.ReadConsistency == roachpb.CONSISTENT && ec.st.State == kvserverpb.LeaseState_VALID { + if ba.ReadConsistency == kvpb.CONSISTENT && ec.st.State == kvserverpb.LeaseState_VALID { ec.repl.updateTimestampCache(ctx, &ec.st, ba, br, pErr) } diff --git a/pkg/kv/kvserver/replica_sideload_test.go b/pkg/kv/kvserver/replica_sideload_test.go index 0d9ba4252672..6375b3953b28 100644 --- a/pkg/kv/kvserver/replica_sideload_test.go +++ b/pkg/kv/kvserver/replica_sideload_test.go @@ -20,6 +20,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/logstore" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftlog" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -65,7 +66,7 @@ func TestRaftSSTableSideloadingProposal(t *testing.T) { } { - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} get := getArgs(roachpb.Key(key)) ba.Add(&get) ba.Header.RangeID = tc.repl.RangeID @@ -74,7 +75,7 @@ func TestRaftSSTableSideloadingProposal(t *testing.T) { if pErr != nil { t.Fatal(pErr) } - v := br.Responses[0].GetInner().(*roachpb.GetResponse).Value + v := br.Responses[0].GetInner().(*kvpb.GetResponse).Value if v == nil { t.Fatal("expected to read a value") } @@ -150,7 +151,7 @@ func TestRaftSSTableSideloading(t *testing.T) { // Disable log truncation to make sure our proposal stays in the log. tc.store.SetRaftLogQueueActive(false) - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.RangeID = tc.repl.RangeID // Put a sideloaded proposal on the Range. @@ -158,7 +159,7 @@ func TestRaftSSTableSideloading(t *testing.T) { origSSTData, _ := MakeSSTable(ctx, key, val, hlc.Timestamp{}.Add(0, 1)) { - var addReq roachpb.AddSSTableRequest + var addReq kvpb.AddSSTableRequest addReq.Data = origSSTData addReq.Key = roachpb.Key(key) addReq.EndKey = addReq.Key.Next() @@ -264,7 +265,7 @@ func TestRaftSSTableSideloadingTruncation(t *testing.T) { newFirstIndex := indexes[i] + 1 truncateArgs := truncateLogArgs(newFirstIndex, rangeID) log.Eventf(ctx, "truncating to index < %d", newFirstIndex) - if _, pErr := kv.SendWrappedWith(ctx, tc.Sender(), roachpb.Header{RangeID: rangeID}, &truncateArgs); pErr != nil { + if _, pErr := kv.SendWrappedWith(ctx, tc.Sender(), kvpb.Header{RangeID: rangeID}, &truncateArgs); pErr != nil { t.Fatal(pErr) } waitForTruncationForTesting(t, tc.repl, newFirstIndex, looselyCoupled) diff --git a/pkg/kv/kvserver/replica_split_load.go b/pkg/kv/kvserver/replica_split_load.go index 7d14a6bc05ab..8031f7892eec 100644 --- a/pkg/kv/kvserver/replica_split_load.go +++ b/pkg/kv/kvserver/replica_split_load.go @@ -15,6 +15,7 @@ import ( "fmt" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/split" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -199,7 +200,7 @@ func (r *Replica) loadSplitStat(ctx context.Context) loadSplitStat { // 2. len(ba.Requests) == len(br.Responses) // Assumptions are checked in executeBatchWithConcurrencyRetries. func getResponseBoundarySpan( - ba *roachpb.BatchRequest, br *roachpb.BatchResponse, + ba *kvpb.BatchRequest, br *kvpb.BatchResponse, ) (responseBoundarySpan roachpb.Span) { addSpanToBoundary := func(span roachpb.Span) { if !responseBoundarySpan.Valid() { @@ -219,10 +220,10 @@ func getResponseBoundarySpan( } switch resp.(type) { - case *roachpb.GetResponse: + case *kvpb.GetResponse: // The request did not evaluate. Ignore it. continue - case *roachpb.ScanResponse: + case *kvpb.ScanResponse: // Not reverse (->) // Request: [key...............endKey) // ResumeSpan: [key......endKey) @@ -239,7 +240,7 @@ func getResponseBoundarySpan( Key: reqHeader.Key, EndKey: resumeSpan.Key, }) - case *roachpb.ReverseScanResponse: + case *kvpb.ReverseScanResponse: // Reverse (<-) // Request: [key...............endKey) // ResumeSpan: [key......endKey) @@ -267,7 +268,7 @@ func getResponseBoundarySpan( // recordBatchForLoadBasedSplitting records the batch's spans to be considered // for load based splitting. func (r *Replica) recordBatchForLoadBasedSplitting( - ctx context.Context, ba *roachpb.BatchRequest, br *roachpb.BatchResponse, stat int, + ctx context.Context, ba *kvpb.BatchRequest, br *kvpb.BatchResponse, stat int, ) { if !r.SplitByLoadEnabled() { return diff --git a/pkg/kv/kvserver/replica_split_load_test.go b/pkg/kv/kvserver/replica_split_load_test.go index 3d04da3546b9..411855f6d35a 100644 --- a/pkg/kv/kvserver/replica_split_load_test.go +++ b/pkg/kv/kvserver/replica_split_load_test.go @@ -14,6 +14,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/stretchr/testify/assert" @@ -23,35 +24,35 @@ func roachpbKey(key uint32) roachpb.Key { return keys.SystemSQLCodec.TablePrefix(key) } -func requestHeaderWithNilEndKey(key uint32) roachpb.RequestHeader { - return roachpb.RequestHeader{ +func requestHeaderWithNilEndKey(key uint32) kvpb.RequestHeader { + return kvpb.RequestHeader{ Key: roachpbKey(key), } } -func requestHeader(key uint32, endKey uint32) roachpb.RequestHeader { - return roachpb.RequestHeader{ +func requestHeader(key uint32, endKey uint32) kvpb.RequestHeader { + return kvpb.RequestHeader{ Key: roachpbKey(key), EndKey: roachpbKey(endKey), } } -func responseHeaderWithNilResumeSpan() roachpb.ResponseHeader { - return roachpb.ResponseHeader{ +func responseHeaderWithNilResumeSpan() kvpb.ResponseHeader { + return kvpb.ResponseHeader{ ResumeSpan: nil, } } -func responseHeaderWithNilEndKey(key uint32) roachpb.ResponseHeader { - return roachpb.ResponseHeader{ +func responseHeaderWithNilEndKey(key uint32) kvpb.ResponseHeader { + return kvpb.ResponseHeader{ ResumeSpan: &roachpb.Span{ Key: roachpbKey(key), }, } } -func responseHeader(key uint32, endKey uint32) roachpb.ResponseHeader { - return roachpb.ResponseHeader{ +func responseHeader(key uint32, endKey uint32) kvpb.ResponseHeader { + return kvpb.ResponseHeader{ ResumeSpan: &roachpb.Span{ Key: roachpbKey(key), EndKey: roachpbKey(endKey), @@ -59,80 +60,80 @@ func responseHeader(key uint32, endKey uint32) roachpb.ResponseHeader { } } -func requestUnionGet(requestHeader roachpb.RequestHeader) roachpb.RequestUnion { - return roachpb.RequestUnion{ - Value: &roachpb.RequestUnion_Get{ - Get: &roachpb.GetRequest{ +func requestUnionGet(requestHeader kvpb.RequestHeader) kvpb.RequestUnion { + return kvpb.RequestUnion{ + Value: &kvpb.RequestUnion_Get{ + Get: &kvpb.GetRequest{ RequestHeader: requestHeader, }, }, } } -func responseUnionGet(responseHeader roachpb.ResponseHeader) roachpb.ResponseUnion { - return roachpb.ResponseUnion{ - Value: &roachpb.ResponseUnion_Get{ - Get: &roachpb.GetResponse{ +func responseUnionGet(responseHeader kvpb.ResponseHeader) kvpb.ResponseUnion { + return kvpb.ResponseUnion{ + Value: &kvpb.ResponseUnion_Get{ + Get: &kvpb.GetResponse{ ResponseHeader: responseHeader, }, }, } } -func requestUnionScan(requestHeader roachpb.RequestHeader) roachpb.RequestUnion { - return roachpb.RequestUnion{ - Value: &roachpb.RequestUnion_Scan{ - Scan: &roachpb.ScanRequest{ +func requestUnionScan(requestHeader kvpb.RequestHeader) kvpb.RequestUnion { + return kvpb.RequestUnion{ + Value: &kvpb.RequestUnion_Scan{ + Scan: &kvpb.ScanRequest{ RequestHeader: requestHeader, }, }, } } -func responseUnionScan(responseHeader roachpb.ResponseHeader) roachpb.ResponseUnion { - return roachpb.ResponseUnion{ - Value: &roachpb.ResponseUnion_Scan{ - Scan: &roachpb.ScanResponse{ +func responseUnionScan(responseHeader kvpb.ResponseHeader) kvpb.ResponseUnion { + return kvpb.ResponseUnion{ + Value: &kvpb.ResponseUnion_Scan{ + Scan: &kvpb.ScanResponse{ ResponseHeader: responseHeader, }, }, } } -func requestUnionReverseScan(requestHeader roachpb.RequestHeader) roachpb.RequestUnion { - return roachpb.RequestUnion{ - Value: &roachpb.RequestUnion_ReverseScan{ - ReverseScan: &roachpb.ReverseScanRequest{ +func requestUnionReverseScan(requestHeader kvpb.RequestHeader) kvpb.RequestUnion { + return kvpb.RequestUnion{ + Value: &kvpb.RequestUnion_ReverseScan{ + ReverseScan: &kvpb.ReverseScanRequest{ RequestHeader: requestHeader, }, }, } } -func responseUnionReverseScan(responseHeader roachpb.ResponseHeader) roachpb.ResponseUnion { - return roachpb.ResponseUnion{ - Value: &roachpb.ResponseUnion_ReverseScan{ - ReverseScan: &roachpb.ReverseScanResponse{ +func responseUnionReverseScan(responseHeader kvpb.ResponseHeader) kvpb.ResponseUnion { + return kvpb.ResponseUnion{ + Value: &kvpb.ResponseUnion_ReverseScan{ + ReverseScan: &kvpb.ReverseScanResponse{ ResponseHeader: responseHeader, }, }, } } -func requestUnionDeleteRange(requestHeader roachpb.RequestHeader) roachpb.RequestUnion { - return roachpb.RequestUnion{ - Value: &roachpb.RequestUnion_DeleteRange{ - DeleteRange: &roachpb.DeleteRangeRequest{ +func requestUnionDeleteRange(requestHeader kvpb.RequestHeader) kvpb.RequestUnion { + return kvpb.RequestUnion{ + Value: &kvpb.RequestUnion_DeleteRange{ + DeleteRange: &kvpb.DeleteRangeRequest{ RequestHeader: requestHeader, }, }, } } -func responseUnionDeleteRange(responseHeader roachpb.ResponseHeader) roachpb.ResponseUnion { - return roachpb.ResponseUnion{ - Value: &roachpb.ResponseUnion_DeleteRange{ - DeleteRange: &roachpb.DeleteRangeResponse{ +func responseUnionDeleteRange(responseHeader kvpb.ResponseHeader) kvpb.ResponseUnion { + return kvpb.ResponseUnion{ + Value: &kvpb.ResponseUnion_DeleteRange{ + DeleteRange: &kvpb.DeleteRangeResponse{ ResponseHeader: responseHeader, }, }, @@ -142,18 +143,18 @@ func responseUnionDeleteRange(responseHeader roachpb.ResponseHeader) roachpb.Res func TestGetResponseBoundarySpan(t *testing.T) { defer leaktest.AfterTest(t)() testCases := []struct { - ba *roachpb.BatchRequest - br *roachpb.BatchResponse + ba *kvpb.BatchRequest + br *kvpb.BatchResponse expectedResponseBoundarySpan roachpb.Span }{ { - ba: &roachpb.BatchRequest{ - Requests: []roachpb.RequestUnion{ + ba: &kvpb.BatchRequest{ + Requests: []kvpb.RequestUnion{ requestUnionGet(requestHeaderWithNilEndKey(100)), }, }, - br: &roachpb.BatchResponse{ - Responses: []roachpb.ResponseUnion{ + br: &kvpb.BatchResponse{ + Responses: []kvpb.ResponseUnion{ responseUnionGet(responseHeaderWithNilResumeSpan()), }, }, @@ -162,13 +163,13 @@ func TestGetResponseBoundarySpan(t *testing.T) { }, }, { - ba: &roachpb.BatchRequest{ - Requests: []roachpb.RequestUnion{ + ba: &kvpb.BatchRequest{ + Requests: []kvpb.RequestUnion{ requestUnionScan(requestHeader(100, 900)), }, }, - br: &roachpb.BatchResponse{ - Responses: []roachpb.ResponseUnion{ + br: &kvpb.BatchResponse{ + Responses: []kvpb.ResponseUnion{ responseUnionScan(responseHeaderWithNilResumeSpan()), }, }, @@ -178,13 +179,13 @@ func TestGetResponseBoundarySpan(t *testing.T) { }, }, { - ba: &roachpb.BatchRequest{ - Requests: []roachpb.RequestUnion{ + ba: &kvpb.BatchRequest{ + Requests: []kvpb.RequestUnion{ requestUnionScan(requestHeader(100, 900)), }, }, - br: &roachpb.BatchResponse{ - Responses: []roachpb.ResponseUnion{ + br: &kvpb.BatchResponse{ + Responses: []kvpb.ResponseUnion{ responseUnionScan(responseHeader(113, 900)), }, }, @@ -194,13 +195,13 @@ func TestGetResponseBoundarySpan(t *testing.T) { }, }, { - ba: &roachpb.BatchRequest{ - Requests: []roachpb.RequestUnion{ + ba: &kvpb.BatchRequest{ + Requests: []kvpb.RequestUnion{ requestUnionReverseScan(requestHeader(100, 900)), }, }, - br: &roachpb.BatchResponse{ - Responses: []roachpb.ResponseUnion{ + br: &kvpb.BatchResponse{ + Responses: []kvpb.ResponseUnion{ responseUnionReverseScan(responseHeader(100, 879)), }, }, @@ -210,13 +211,13 @@ func TestGetResponseBoundarySpan(t *testing.T) { }, }, { - ba: &roachpb.BatchRequest{ - Requests: []roachpb.RequestUnion{ + ba: &kvpb.BatchRequest{ + Requests: []kvpb.RequestUnion{ requestUnionDeleteRange(requestHeader(100, 900)), }, }, - br: &roachpb.BatchResponse{ - Responses: []roachpb.ResponseUnion{ + br: &kvpb.BatchResponse{ + Responses: []kvpb.ResponseUnion{ responseUnionDeleteRange(responseHeader(113, 900)), }, }, @@ -226,47 +227,47 @@ func TestGetResponseBoundarySpan(t *testing.T) { }, }, { - ba: &roachpb.BatchRequest{ - Requests: []roachpb.RequestUnion{ + ba: &kvpb.BatchRequest{ + Requests: []kvpb.RequestUnion{ requestUnionGet(requestHeaderWithNilEndKey(100)), }, }, - br: &roachpb.BatchResponse{ - Responses: []roachpb.ResponseUnion{ + br: &kvpb.BatchResponse{ + Responses: []kvpb.ResponseUnion{ responseUnionGet(responseHeaderWithNilEndKey(100)), }, }, expectedResponseBoundarySpan: roachpb.Span{}, }, { - ba: &roachpb.BatchRequest{ - Requests: []roachpb.RequestUnion{ + ba: &kvpb.BatchRequest{ + Requests: []kvpb.RequestUnion{ requestUnionScan(requestHeader(100, 900)), }, }, - br: &roachpb.BatchResponse{ - Responses: []roachpb.ResponseUnion{ + br: &kvpb.BatchResponse{ + Responses: []kvpb.ResponseUnion{ responseUnionScan(responseHeader(100, 900)), }, }, expectedResponseBoundarySpan: roachpb.Span{}, }, { - ba: &roachpb.BatchRequest{ - Requests: []roachpb.RequestUnion{ + ba: &kvpb.BatchRequest{ + Requests: []kvpb.RequestUnion{ requestUnionReverseScan(requestHeader(100, 900)), }, }, - br: &roachpb.BatchResponse{ - Responses: []roachpb.ResponseUnion{ + br: &kvpb.BatchResponse{ + Responses: []kvpb.ResponseUnion{ responseUnionReverseScan(responseHeader(100, 900)), }, }, expectedResponseBoundarySpan: roachpb.Span{}, }, { - ba: &roachpb.BatchRequest{ - Requests: []roachpb.RequestUnion{ + ba: &kvpb.BatchRequest{ + Requests: []kvpb.RequestUnion{ requestUnionScan(requestHeader(500, 600)), requestUnionReverseScan(requestHeader(475, 625)), requestUnionGet(requestHeaderWithNilEndKey(480)), @@ -274,8 +275,8 @@ func TestGetResponseBoundarySpan(t *testing.T) { requestUnionScan(requestHeader(700, 800)), }, }, - br: &roachpb.BatchResponse{ - Responses: []roachpb.ResponseUnion{ + br: &kvpb.BatchResponse{ + Responses: []kvpb.ResponseUnion{ responseUnionScan(responseHeader(550, 600)), responseUnionReverseScan(responseHeader(475, 525)), responseUnionGet(responseHeaderWithNilResumeSpan()), diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 14ddee2643dd..171d2a597f5e 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -33,6 +33,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" @@ -223,7 +224,7 @@ func (tc *testContext) StartWithStoreConfigAndVersion( } func (tc *testContext) Sender() kv.Sender { - return kv.Wrap(tc.repl, func(ba *roachpb.BatchRequest) *roachpb.BatchRequest { + return kv.Wrap(tc.repl, func(ba *kvpb.BatchRequest) *kvpb.BatchRequest { if ba.RangeID == 0 { ba.RangeID = 1 } @@ -240,14 +241,14 @@ func (tc *testContext) Sender() kv.Sender { // SendWrappedWith is a convenience function which wraps the request in a batch // and sends it func (tc *testContext) SendWrappedWith( - h roachpb.Header, args roachpb.Request, -) (roachpb.Response, *roachpb.Error) { + h kvpb.Header, args kvpb.Request, +) (kvpb.Response, *kvpb.Error) { return kv.SendWrappedWith(context.Background(), tc.Sender(), h, args) } // SendWrapped is identical to SendWrappedWith with a zero header. -func (tc *testContext) SendWrapped(args roachpb.Request) (roachpb.Response, *roachpb.Error) { - return tc.SendWrappedWith(roachpb.Header{}, args) +func (tc *testContext) SendWrapped(args kvpb.Request) (kvpb.Response, *kvpb.Error) { + return tc.SendWrappedWith(kvpb.Header{}, args) } // addBogusReplicaToRangeDesc modifies the range descriptor to include a second @@ -284,7 +285,7 @@ func (tc *testContext) addBogusReplicaToRangeDesc( // have in memory. At the time of this writing, this is not actually required // by the tests using this functionality, but it seems sane to do. ba := kv.Batch{ - Header: roachpb.Header{Timestamp: tc.Clock().Now()}, + Header: kvpb.Header{Timestamp: tc.Clock().Now()}, } descKey := keys.RangeDescriptorKey(oldDesc.StartKey) if err := updateRangeDescriptor(&ba, descKey, dbDescKV.Value.TagAndDataBytes(), &newDesc); err != nil { @@ -319,7 +320,7 @@ func newTransaction( // assignSeqNumsForReqs sets sequence numbers for each of the provided requests // given a transaction proto. It also updates the proto to reflect the incremented // sequence number. -func assignSeqNumsForReqs(txn *roachpb.Transaction, reqs ...roachpb.Request) { +func assignSeqNumsForReqs(txn *roachpb.Transaction, reqs ...kvpb.Request) { for _, ru := range reqs { txn.Sequence++ oldHeader := ru.Header() @@ -347,14 +348,14 @@ func TestIsOnePhaseCommit(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) ctx := context.Background() - withSeq := func(req roachpb.Request, seq enginepb.TxnSeq) roachpb.Request { + withSeq := func(req kvpb.Request, seq enginepb.TxnSeq) kvpb.Request { h := req.Header() h.Sequence = seq req.SetHeader(h) return req } - makeReqs := func(reqs ...roachpb.Request) []roachpb.RequestUnion { - ru := make([]roachpb.RequestUnion, len(reqs)) + makeReqs := func(reqs ...kvpb.Request) []kvpb.RequestUnion { + ru := make([]kvpb.RequestUnion, len(reqs)) for i, r := range reqs { ru[i].MustSetInner(r) } @@ -362,20 +363,20 @@ func TestIsOnePhaseCommit(t *testing.T) { } noReqs := makeReqs() - getReq := makeReqs(withSeq(&roachpb.GetRequest{}, 0)) - putReq := makeReqs(withSeq(&roachpb.PutRequest{}, 1)) - etReq := makeReqs(withSeq(&roachpb.EndTxnRequest{Commit: true}, 1)) + getReq := makeReqs(withSeq(&kvpb.GetRequest{}, 0)) + putReq := makeReqs(withSeq(&kvpb.PutRequest{}, 1)) + etReq := makeReqs(withSeq(&kvpb.EndTxnRequest{Commit: true}, 1)) txnReqs := makeReqs( - withSeq(&roachpb.PutRequest{}, 1), - withSeq(&roachpb.EndTxnRequest{Commit: true}, 2), + withSeq(&kvpb.PutRequest{}, 1), + withSeq(&kvpb.EndTxnRequest{Commit: true}, 2), ) txnReqsRequire1PC := makeReqs( - withSeq(&roachpb.PutRequest{}, 1), - withSeq(&roachpb.EndTxnRequest{Commit: true, Require1PC: true}, 2), + withSeq(&kvpb.PutRequest{}, 1), + withSeq(&kvpb.EndTxnRequest{Commit: true, Require1PC: true}, 2), ) testCases := []struct { - ru []roachpb.RequestUnion + ru []kvpb.RequestUnion isNonTxn bool canForwardTS bool isRestarted bool @@ -431,7 +432,7 @@ func TestIsOnePhaseCommit(t *testing.T) { fmt.Sprintf("%d:isNonTxn:%t,canForwardTS:%t,isRestarted:%t,isWTO:%t,isTSOff:%t", i, c.isNonTxn, c.canForwardTS, c.isRestarted, c.isWTO, c.isTSOff), func(t *testing.T) { - ba := &roachpb.BatchRequest{Requests: c.ru} + ba := &kvpb.BatchRequest{Requests: c.ru} if !c.isNonTxn { ba.Txn = newTransaction("txn", roachpb.Key("a"), 1, clock) if c.canForwardTS { @@ -498,10 +499,10 @@ func TestReplicaContains(t *testing.T) { func sendLeaseRequest(r *Replica, l *roachpb.Lease) error { ctx := context.Background() - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Timestamp = r.store.Clock().Now() st := r.CurrentLeaseStatus(ctx) - leaseReq := &roachpb.RequestLeaseRequest{ + leaseReq := &kvpb.RequestLeaseRequest{ Lease: *l, PrevLease: st.Lease, } @@ -550,16 +551,16 @@ func TestReplicaReadConsistency(t *testing.T) { txn := newTransaction("test", roachpb.Key("a"), 1, tc.Clock()) assignSeqNumsForReqs(txn, &gArgs) - if _, err := tc.SendWrappedWith(roachpb.Header{ + if _, err := tc.SendWrappedWith(kvpb.Header{ Txn: txn, - ReadConsistency: roachpb.READ_UNCOMMITTED, + ReadConsistency: kvpb.READ_UNCOMMITTED, }, &gArgs); err == nil { t.Errorf("expected error on read uncommitted read within a txn") } - if _, err := tc.SendWrappedWith(roachpb.Header{ + if _, err := tc.SendWrappedWith(kvpb.Header{ Txn: txn, - ReadConsistency: roachpb.INCONSISTENT, + ReadConsistency: kvpb.INCONSISTENT, }, &gArgs); err == nil { t.Errorf("expected error on inconsistent read within a txn") } @@ -577,22 +578,22 @@ func TestReplicaReadConsistency(t *testing.T) { } // Send without Txn. - _, pErr := tc.SendWrappedWith(roachpb.Header{ - ReadConsistency: roachpb.CONSISTENT, + _, pErr := tc.SendWrappedWith(kvpb.Header{ + ReadConsistency: kvpb.CONSISTENT, }, &gArgs) - if _, ok := pErr.GetDetail().(*roachpb.NotLeaseHolderError); !ok { + if _, ok := pErr.GetDetail().(*kvpb.NotLeaseHolderError); !ok { t.Errorf("expected not lease holder error; got %s", pErr) } - _, pErr = tc.SendWrappedWith(roachpb.Header{ - ReadConsistency: roachpb.READ_UNCOMMITTED, + _, pErr = tc.SendWrappedWith(kvpb.Header{ + ReadConsistency: kvpb.READ_UNCOMMITTED, }, &gArgs) - if _, ok := pErr.GetDetail().(*roachpb.NotLeaseHolderError); !ok { + if _, ok := pErr.GetDetail().(*kvpb.NotLeaseHolderError); !ok { t.Errorf("expected not lease holder error; got %s", pErr) } - if _, pErr := tc.SendWrappedWith(roachpb.Header{ - ReadConsistency: roachpb.INCONSISTENT, + if _, pErr := tc.SendWrappedWith(kvpb.Header{ + ReadConsistency: kvpb.INCONSISTENT, }, &gArgs); pErr != nil { t.Errorf("expected success reading with inconsistent: %s", pErr) } @@ -617,7 +618,7 @@ func TestBehaviorDuringLeaseTransfer(t *testing.T) { tsc := TestStoreConfig(clock) var leaseAcquisitionTrap atomic.Value tsc.TestingKnobs.DisableAutomaticLeaseRenewal = true - tsc.TestingKnobs.LeaseRequestEvent = func(ts hlc.Timestamp, _ roachpb.StoreID, _ roachpb.RangeID) *roachpb.Error { + tsc.TestingKnobs.LeaseRequestEvent = func(ts hlc.Timestamp, _ roachpb.StoreID, _ roachpb.RangeID) *kvpb.Error { val := leaseAcquisitionTrap.Load() if val == nil { return nil @@ -630,8 +631,8 @@ func TestBehaviorDuringLeaseTransfer(t *testing.T) { } transferSem := make(chan struct{}) tsc.TestingKnobs.EvalKnobs.TestingPostEvalFilter = - func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { - if _, ok := filterArgs.Req.(*roachpb.TransferLeaseRequest); ok { + func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { + if _, ok := filterArgs.Req.(*kvpb.TransferLeaseRequest); ok { // Notify the test that the transfer has been trapped. transferSem <- struct{}{} // Wait for the test to unblock the transfer. @@ -639,7 +640,7 @@ func TestBehaviorDuringLeaseTransfer(t *testing.T) { if !transferSucceeds { // Return an error, so that the pendingLeaseRequest // considers the transfer failed. - return roachpb.NewErrorf("injected transfer error") + return kvpb.NewErrorf("injected transfer error") } return nil } @@ -737,7 +738,7 @@ func TestBehaviorDuringLeaseTransfer(t *testing.T) { // target. err = <-readResChan require.Error(t, err) - var lErr *roachpb.NotLeaseHolderError + var lErr *kvpb.NotLeaseHolderError require.True(t, errors.As(err, &lErr)) require.Equal(t, secondReplica.StoreID, lErr.Lease.Replica.StoreID) } else { @@ -794,10 +795,10 @@ func TestApplyCmdLeaseError(t *testing.T) { t.Fatal(err) } - _, pErr := tc.SendWrappedWith(roachpb.Header{ + _, pErr := tc.SendWrappedWith(kvpb.Header{ Timestamp: tc.Clock().Now().Add(-100, 0), }, &pArgs) - if _, ok := pErr.GetDetail().(*roachpb.NotLeaseHolderError); !ok { + if _, ok := pErr.GetDetail().(*kvpb.NotLeaseHolderError); !ok { t.Fatalf("expected not lease holder error in return, got %v", pErr) } } @@ -832,7 +833,7 @@ func TestLeaseReplicaNotInDesc(t *testing.T) { ) pErr := fr.ForcedError tc.repl.mu.Unlock() - if _, isErr := pErr.GetDetail().(*roachpb.LeaseRejectedError); !isErr { + if _, isErr := pErr.GetDetail().(*kvpb.LeaseRejectedError); !isErr { t.Fatal(pErr) } else if !testutils.IsPError(pErr, "replica not part of range") { t.Fatal(pErr) @@ -866,8 +867,8 @@ func TestReplicaRangeMismatchRedirect(t *testing.T) { } gArgs := getArgs(roachpb.Key("b")) - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{ + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{ RangeID: 1, } ba.Add(&gArgs) @@ -900,9 +901,9 @@ func TestReplicaLease(t *testing.T) { defer stopper.Stop(ctx) var filterErr atomic.Value - applyFilter := func(args kvserverbase.ApplyFilterArgs) (int, *roachpb.Error) { + applyFilter := func(args kvserverbase.ApplyFilterArgs) (int, *kvpb.Error) { if pErr := filterErr.Load(); pErr != nil { - return 0, pErr.(*roachpb.Error) + return 0, pErr.(*kvpb.Error) } return 0, nil } @@ -929,10 +930,10 @@ func TestReplicaLease(t *testing.T) { EvalCtx: NewReplicaEvalContext( ctx, tc.repl, allSpans(), false, /* requiresClosedTSOlderThanStorageSnap */ ), - Args: &roachpb.RequestLeaseRequest{ + Args: &kvpb.RequestLeaseRequest{ Lease: lease, }, - }, &roachpb.RequestLeaseResponse{}); !testutils.IsError(err, "replica not found") { + }, &kvpb.RequestLeaseResponse{}); !testutils.IsError(err, "replica not found") { t.Fatalf("unexpected error: %+v", err) } } @@ -955,7 +956,7 @@ func TestReplicaLease(t *testing.T) { { _, pErr := tc.repl.redirectOnOrAcquireLease(ctx) - if lErr, ok := pErr.GetDetail().(*roachpb.NotLeaseHolderError); !ok || lErr == nil { + if lErr, ok := pErr.GetDetail().(*kvpb.NotLeaseHolderError); !ok || lErr == nil { t.Fatalf("wanted NotLeaseHolderError, got %s", pErr) } } @@ -967,12 +968,12 @@ func TestReplicaLease(t *testing.T) { } // Verify that command returns NotLeaseHolderError when lease is rejected. - filterErr.Store(roachpb.NewError(&roachpb.LeaseRejectedError{Message: "replica not found"})) + filterErr.Store(kvpb.NewError(&kvpb.LeaseRejectedError{Message: "replica not found"})) { _, err := tc.repl.redirectOnOrAcquireLease(ctx) - if _, ok := err.GetDetail().(*roachpb.NotLeaseHolderError); !ok { - t.Fatalf("expected %T, got %s", &roachpb.NotLeaseHolderError{}, err) + if _, ok := err.GetDetail().(*kvpb.NotLeaseHolderError); !ok { + t.Fatalf("expected %T, got %s", &kvpb.NotLeaseHolderError{}, err) } } } @@ -1005,30 +1006,30 @@ func TestReplicaNotLeaseHolderError(t *testing.T) { t.Fatal(err) } - header := roachpb.RequestHeader{ + header := kvpb.RequestHeader{ Key: roachpb.Key("a"), } - testCases := []roachpb.Request{ + testCases := []kvpb.Request{ // Admin split covers admin commands. - &roachpb.AdminSplitRequest{ + &kvpb.AdminSplitRequest{ RequestHeader: header, SplitKey: roachpb.Key("a"), }, // Get covers read-only commands. - &roachpb.GetRequest{ + &kvpb.GetRequest{ RequestHeader: header, }, // Put covers read-write commands. - &roachpb.PutRequest{ + &kvpb.PutRequest{ RequestHeader: header, Value: roachpb.MakeValueFromString("value"), }, } for i, test := range testCases { - _, pErr := tc.SendWrappedWith(roachpb.Header{Timestamp: now.ToTimestamp()}, test) + _, pErr := tc.SendWrappedWith(kvpb.Header{Timestamp: now.ToTimestamp()}, test) - if _, ok := pErr.GetDetail().(*roachpb.NotLeaseHolderError); !ok { + if _, ok := pErr.GetDetail().(*kvpb.NotLeaseHolderError); !ok { t.Errorf("%d: expected not lease holder error: %s", i, pErr) } } @@ -1336,9 +1337,9 @@ func TestReplicaLeaseRejectUnknownRaftNodeID(t *testing.T) { }, } st := tc.repl.CurrentLeaseStatus(ctx) - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Timestamp = tc.repl.store.Clock().Now() - ba.Add(&roachpb.RequestLeaseRequest{Lease: *lease}) + ba.Add(&kvpb.RequestLeaseRequest{Lease: *lease}) _, tok := tc.repl.mu.proposalBuf.TrackEvaluatingRequest(ctx, hlc.MinTimestamp) ch, _, _, _, pErr := tc.repl.evalAndPropose(ctx, ba, allSpansGuard(), &st, uncertainty.Interval{}, tok.Move(ctx)) if pErr == nil { @@ -1382,51 +1383,51 @@ func TestReplicaGossipFirstRange(t *testing.T) { } } -func getArgs(key []byte) roachpb.GetRequest { - return roachpb.GetRequest{ - RequestHeader: roachpb.RequestHeader{ +func getArgs(key []byte) kvpb.GetRequest { + return kvpb.GetRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, }, } } -func putArgs(key roachpb.Key, value []byte) roachpb.PutRequest { - return roachpb.PutRequest{ - RequestHeader: roachpb.RequestHeader{ +func putArgs(key roachpb.Key, value []byte) kvpb.PutRequest { + return kvpb.PutRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, }, Value: roachpb.MakeValueFromBytes(value), } } -func cPutArgs(key roachpb.Key, value, expValue []byte) roachpb.ConditionalPutRequest { +func cPutArgs(key roachpb.Key, value, expValue []byte) kvpb.ConditionalPutRequest { if expValue != nil { expValue = roachpb.MakeValueFromBytes(expValue).TagAndDataBytes() } - req := roachpb.NewConditionalPut(key, roachpb.MakeValueFromBytes(value), expValue, false /* allowNotExist */) - return *req.(*roachpb.ConditionalPutRequest) + req := kvpb.NewConditionalPut(key, roachpb.MakeValueFromBytes(value), expValue, false /* allowNotExist */) + return *req.(*kvpb.ConditionalPutRequest) } -func iPutArgs(key roachpb.Key, value []byte) roachpb.InitPutRequest { - return roachpb.InitPutRequest{ - RequestHeader: roachpb.RequestHeader{ +func iPutArgs(key roachpb.Key, value []byte) kvpb.InitPutRequest { + return kvpb.InitPutRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, }, Value: roachpb.MakeValueFromBytes(value), } } -func deleteArgs(key roachpb.Key) roachpb.DeleteRequest { - return roachpb.DeleteRequest{ - RequestHeader: roachpb.RequestHeader{ +func deleteArgs(key roachpb.Key) kvpb.DeleteRequest { + return kvpb.DeleteRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, }, } } -func deleteRangeArgs(key, endKey roachpb.Key) roachpb.DeleteRangeRequest { - return roachpb.DeleteRangeRequest{ - RequestHeader: roachpb.RequestHeader{ +func deleteRangeArgs(key, endKey roachpb.Key) kvpb.DeleteRangeRequest { + return kvpb.DeleteRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, EndKey: endKey, }, @@ -1435,7 +1436,7 @@ func deleteRangeArgs(key, endKey roachpb.Key) roachpb.DeleteRangeRequest { // readOrWriteArgs returns either get or put arguments depending on // value of "read". Get for true; Put for false. -func readOrWriteArgs(key roachpb.Key, read bool) roachpb.Request { +func readOrWriteArgs(key roachpb.Key, read bool) kvpb.Request { if read { gArgs := getArgs(key) return &gArgs @@ -1444,45 +1445,45 @@ func readOrWriteArgs(key roachpb.Key, read bool) roachpb.Request { return &pArgs } -func incrementArgs(key []byte, inc int64) *roachpb.IncrementRequest { - return &roachpb.IncrementRequest{ - RequestHeader: roachpb.RequestHeader{ +func incrementArgs(key []byte, inc int64) *kvpb.IncrementRequest { + return &kvpb.IncrementRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, }, Increment: inc, } } -func scanArgsString(s, e string) *roachpb.ScanRequest { - return &roachpb.ScanRequest{ - RequestHeader: roachpb.RequestHeader{Key: roachpb.Key(s), EndKey: roachpb.Key(e)}, +func scanArgsString(s, e string) *kvpb.ScanRequest { + return &kvpb.ScanRequest{ + RequestHeader: kvpb.RequestHeader{Key: roachpb.Key(s), EndKey: roachpb.Key(e)}, } } -func getArgsString(k string) *roachpb.GetRequest { - return &roachpb.GetRequest{ - RequestHeader: roachpb.RequestHeader{Key: roachpb.Key(k)}, +func getArgsString(k string) *kvpb.GetRequest { + return &kvpb.GetRequest{ + RequestHeader: kvpb.RequestHeader{Key: roachpb.Key(k)}, } } -func scanArgs(start, end []byte) *roachpb.ScanRequest { - return &roachpb.ScanRequest{ - RequestHeader: roachpb.RequestHeader{ +func scanArgs(start, end []byte) *kvpb.ScanRequest { + return &kvpb.ScanRequest{ + RequestHeader: kvpb.RequestHeader{ Key: start, EndKey: end, }, } } -func revScanArgsString(s, e string) *roachpb.ReverseScanRequest { - return &roachpb.ReverseScanRequest{ - RequestHeader: roachpb.RequestHeader{Key: roachpb.Key(s), EndKey: roachpb.Key(e)}, +func revScanArgsString(s, e string) *kvpb.ReverseScanRequest { + return &kvpb.ReverseScanRequest{ + RequestHeader: kvpb.RequestHeader{Key: roachpb.Key(s), EndKey: roachpb.Key(e)}, } } -func revScanArgs(start, end []byte) *roachpb.ReverseScanRequest { - return &roachpb.ReverseScanRequest{ - RequestHeader: roachpb.RequestHeader{ +func revScanArgs(start, end []byte) *kvpb.ReverseScanRequest { + return &kvpb.ReverseScanRequest{ + RequestHeader: kvpb.RequestHeader{ Key: start, EndKey: end, }, @@ -1491,31 +1492,31 @@ func revScanArgs(start, end []byte) *roachpb.ReverseScanRequest { func heartbeatArgs( txn *roachpb.Transaction, now hlc.Timestamp, -) (roachpb.HeartbeatTxnRequest, roachpb.Header) { - return roachpb.HeartbeatTxnRequest{ - RequestHeader: roachpb.RequestHeader{ +) (kvpb.HeartbeatTxnRequest, kvpb.Header) { + return kvpb.HeartbeatTxnRequest{ + RequestHeader: kvpb.RequestHeader{ Key: txn.Key, }, Now: now, - }, roachpb.Header{Txn: txn} + }, kvpb.Header{Txn: txn} } // endTxnArgs creates a EndTxnRequest. By leaving the Sequence field 0, the // request will not qualify for 1PC. -func endTxnArgs(txn *roachpb.Transaction, commit bool) (roachpb.EndTxnRequest, roachpb.Header) { - return roachpb.EndTxnRequest{ - RequestHeader: roachpb.RequestHeader{ +func endTxnArgs(txn *roachpb.Transaction, commit bool) (kvpb.EndTxnRequest, kvpb.Header) { + return kvpb.EndTxnRequest{ + RequestHeader: kvpb.RequestHeader{ Key: txn.Key, // not allowed when going through TxnCoordSender, but we're not }, Commit: commit, - }, roachpb.Header{Txn: txn} + }, kvpb.Header{Txn: txn} } func pushTxnArgs( - pusher, pushee *roachpb.Transaction, pushType roachpb.PushTxnType, -) roachpb.PushTxnRequest { - return roachpb.PushTxnRequest{ - RequestHeader: roachpb.RequestHeader{ + pusher, pushee *roachpb.Transaction, pushType kvpb.PushTxnType, +) kvpb.PushTxnRequest { + return kvpb.PushTxnRequest{ + RequestHeader: kvpb.RequestHeader{ Key: pushee.Key, }, PushTo: pusher.WriteTimestamp.Next(), @@ -1525,9 +1526,9 @@ func pushTxnArgs( } } -func recoverTxnArgs(txn *roachpb.Transaction, implicitlyCommitted bool) roachpb.RecoverTxnRequest { - return roachpb.RecoverTxnRequest{ - RequestHeader: roachpb.RequestHeader{ +func recoverTxnArgs(txn *roachpb.Transaction, implicitlyCommitted bool) kvpb.RecoverTxnRequest { + return kvpb.RecoverTxnRequest{ + RequestHeader: kvpb.RequestHeader{ Key: txn.Key, }, Txn: txn.TxnMeta, @@ -1535,9 +1536,9 @@ func recoverTxnArgs(txn *roachpb.Transaction, implicitlyCommitted bool) roachpb. } } -func queryTxnArgs(txn enginepb.TxnMeta, waitForUpdate bool) roachpb.QueryTxnRequest { - return roachpb.QueryTxnRequest{ - RequestHeader: roachpb.RequestHeader{ +func queryTxnArgs(txn enginepb.TxnMeta, waitForUpdate bool) kvpb.QueryTxnRequest { + return kvpb.QueryTxnRequest{ + RequestHeader: kvpb.RequestHeader{ Key: txn.Key, }, Txn: txn, @@ -1545,11 +1546,9 @@ func queryTxnArgs(txn enginepb.TxnMeta, waitForUpdate bool) roachpb.QueryTxnRequ } } -func queryIntentArgs( - key []byte, txn enginepb.TxnMeta, errIfMissing bool, -) roachpb.QueryIntentRequest { - return roachpb.QueryIntentRequest{ - RequestHeader: roachpb.RequestHeader{ +func queryIntentArgs(key []byte, txn enginepb.TxnMeta, errIfMissing bool) kvpb.QueryIntentRequest { + return kvpb.QueryIntentRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, }, Txn: txn, @@ -1557,9 +1556,9 @@ func queryIntentArgs( } } -func queryLocksArgs(key, endKey []byte, includeUncontended bool) roachpb.QueryLocksRequest { - return roachpb.QueryLocksRequest{ - RequestHeader: roachpb.RequestHeader{ +func queryLocksArgs(key, endKey []byte, includeUncontended bool) kvpb.QueryLocksRequest { + return kvpb.QueryLocksRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, EndKey: endKey, }, @@ -1569,48 +1568,48 @@ func queryLocksArgs(key, endKey []byte, includeUncontended bool) roachpb.QueryLo func resolveIntentRangeArgsString( s, e string, txn enginepb.TxnMeta, status roachpb.TransactionStatus, -) *roachpb.ResolveIntentRangeRequest { - return &roachpb.ResolveIntentRangeRequest{ - RequestHeader: roachpb.RequestHeader{Key: roachpb.Key(s), EndKey: roachpb.Key(e)}, +) *kvpb.ResolveIntentRangeRequest { + return &kvpb.ResolveIntentRangeRequest{ + RequestHeader: kvpb.RequestHeader{Key: roachpb.Key(s), EndKey: roachpb.Key(e)}, IntentTxn: txn, Status: status, } } -func internalMergeArgs(key []byte, value roachpb.Value) roachpb.MergeRequest { - return roachpb.MergeRequest{ - RequestHeader: roachpb.RequestHeader{ +func internalMergeArgs(key []byte, value roachpb.Value) kvpb.MergeRequest { + return kvpb.MergeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, }, Value: value, } } -func truncateLogArgs(index uint64, rangeID roachpb.RangeID) roachpb.TruncateLogRequest { - return roachpb.TruncateLogRequest{ +func truncateLogArgs(index uint64, rangeID roachpb.RangeID) kvpb.TruncateLogRequest { + return kvpb.TruncateLogRequest{ Index: index, RangeID: rangeID, } } -func gcKey(key roachpb.Key, timestamp hlc.Timestamp) roachpb.GCRequest_GCKey { - return roachpb.GCRequest_GCKey{ +func gcKey(key roachpb.Key, timestamp hlc.Timestamp) kvpb.GCRequest_GCKey { + return kvpb.GCRequest_GCKey{ Key: key, Timestamp: timestamp, } } -func recomputeStatsArgs(key roachpb.Key) roachpb.RecomputeStatsRequest { - return roachpb.RecomputeStatsRequest{ - RequestHeader: roachpb.RequestHeader{ +func recomputeStatsArgs(key roachpb.Key) kvpb.RecomputeStatsRequest { + return kvpb.RecomputeStatsRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, }, } } -func gcArgs(startKey []byte, endKey []byte, keys ...roachpb.GCRequest_GCKey) roachpb.GCRequest { - return roachpb.GCRequest{ - RequestHeader: roachpb.RequestHeader{ +func gcArgs(startKey []byte, endKey []byte, keys ...kvpb.GCRequest_GCKey) kvpb.GCRequest { + return kvpb.GCRequest{ + RequestHeader: kvpb.RequestHeader{ Key: startKey, EndKey: endKey, }, @@ -1618,9 +1617,9 @@ func gcArgs(startKey []byte, endKey []byte, keys ...roachpb.GCRequest_GCKey) roa } } -func clearRangeArgs(startKey, endKey roachpb.Key) roachpb.ClearRangeRequest { - return roachpb.ClearRangeRequest{ - RequestHeader: roachpb.RequestHeader{ +func clearRangeArgs(startKey, endKey roachpb.Key) kvpb.ClearRangeRequest { + return kvpb.ClearRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: startKey, EndKey: endKey, }, @@ -1639,9 +1638,9 @@ func TestOptimizePuts(t *testing.T) { defer stopper.Stop(ctx) tc.Start(ctx, t, stopper) - pArgs := make([]roachpb.PutRequest, optimizePutThreshold) - cpArgs := make([]roachpb.ConditionalPutRequest, optimizePutThreshold) - ipArgs := make([]roachpb.InitPutRequest, optimizePutThreshold) + pArgs := make([]kvpb.PutRequest, optimizePutThreshold) + cpArgs := make([]kvpb.ConditionalPutRequest, optimizePutThreshold) + ipArgs := make([]kvpb.InitPutRequest, optimizePutThreshold) for i := 0; i < optimizePutThreshold; i++ { pArgs[i] = putArgs([]byte(fmt.Sprintf("%02d", i)), []byte("1")) cpArgs[i] = cPutArgs([]byte(fmt.Sprintf("%02d", i)), []byte("1"), []byte("0")) @@ -1652,13 +1651,13 @@ func TestOptimizePuts(t *testing.T) { testCases := []struct { exKey roachpb.Key exEndKey roachpb.Key // MVCC range key - reqs []roachpb.Request + reqs []kvpb.Request expBlind []bool }{ // No existing keys, single put. { nil, nil, - []roachpb.Request{ + []kvpb.Request{ &pArgs[0], }, []bool{ @@ -1668,7 +1667,7 @@ func TestOptimizePuts(t *testing.T) { // No existing keys, nine puts. { nil, nil, - []roachpb.Request{ + []kvpb.Request{ &pArgs[0], &pArgs[1], &pArgs[2], &pArgs[3], &pArgs[4], &pArgs[5], &pArgs[6], &pArgs[7], &pArgs[8], }, []bool{ @@ -1678,7 +1677,7 @@ func TestOptimizePuts(t *testing.T) { // No existing keys, ten puts. { nil, nil, - []roachpb.Request{ + []kvpb.Request{ &pArgs[0], &pArgs[1], &pArgs[2], &pArgs[3], &pArgs[4], &pArgs[5], &pArgs[6], &pArgs[7], &pArgs[8], &pArgs[9], }, []bool{ @@ -1688,7 +1687,7 @@ func TestOptimizePuts(t *testing.T) { // Existing key at "0", ten conditional puts. { roachpb.Key("0"), nil, - []roachpb.Request{ + []kvpb.Request{ &cpArgs[0], &cpArgs[1], &cpArgs[2], &cpArgs[3], &cpArgs[4], &cpArgs[5], &cpArgs[6], &cpArgs[7], &cpArgs[8], &cpArgs[9], }, []bool{ @@ -1698,7 +1697,7 @@ func TestOptimizePuts(t *testing.T) { // Existing key at "0", ten init puts. { roachpb.Key("0"), nil, - []roachpb.Request{ + []kvpb.Request{ &ipArgs[0], &ipArgs[1], &ipArgs[2], &ipArgs[3], &ipArgs[4], &ipArgs[5], &ipArgs[6], &ipArgs[7], &ipArgs[8], &ipArgs[9], }, []bool{ @@ -1708,7 +1707,7 @@ func TestOptimizePuts(t *testing.T) { // Existing key at 11, mixed put types. { roachpb.Key("11"), nil, - []roachpb.Request{ + []kvpb.Request{ &pArgs[0], &cpArgs[1], &pArgs[2], &cpArgs[3], &ipArgs[4], &ipArgs[5], &pArgs[6], &cpArgs[7], &pArgs[8], &ipArgs[9], }, []bool{ @@ -1718,7 +1717,7 @@ func TestOptimizePuts(t *testing.T) { // Existing key at 00, ten puts, expect nothing blind. { roachpb.Key("00"), nil, - []roachpb.Request{ + []kvpb.Request{ &pArgs[0], &pArgs[1], &pArgs[2], &pArgs[3], &pArgs[4], &pArgs[5], &pArgs[6], &pArgs[7], &pArgs[8], &pArgs[9], }, []bool{ @@ -1728,7 +1727,7 @@ func TestOptimizePuts(t *testing.T) { // Existing key at 00, ten puts in reverse order, expect nothing blind. { roachpb.Key("00"), nil, - []roachpb.Request{ + []kvpb.Request{ &pArgs[9], &pArgs[8], &pArgs[7], &pArgs[6], &pArgs[5], &pArgs[4], &pArgs[3], &pArgs[2], &pArgs[1], &pArgs[0], }, []bool{ @@ -1738,7 +1737,7 @@ func TestOptimizePuts(t *testing.T) { // Existing key at 05, ten puts, expect first five puts are blind. { roachpb.Key("05"), nil, - []roachpb.Request{ + []kvpb.Request{ &pArgs[0], &pArgs[1], &pArgs[2], &pArgs[3], &pArgs[4], &pArgs[5], &pArgs[6], &pArgs[7], &pArgs[8], &pArgs[9], }, []bool{ @@ -1748,7 +1747,7 @@ func TestOptimizePuts(t *testing.T) { // Existing key at 09, ten puts, expect first nine puts are blind. { roachpb.Key("09"), nil, - []roachpb.Request{ + []kvpb.Request{ &pArgs[0], &pArgs[1], &pArgs[2], &pArgs[3], &pArgs[4], &pArgs[5], &pArgs[6], &pArgs[7], &pArgs[8], &pArgs[9], }, []bool{ @@ -1758,7 +1757,7 @@ func TestOptimizePuts(t *testing.T) { // No existing key, ten puts + inc + ten cputs. { nil, nil, - []roachpb.Request{ + []kvpb.Request{ &pArgs[0], &pArgs[1], &pArgs[2], &pArgs[3], &pArgs[4], &pArgs[5], &pArgs[6], &pArgs[7], &pArgs[8], &pArgs[9], incArgs, &cpArgs[0], &cpArgs[1], &cpArgs[2], &cpArgs[3], &cpArgs[4], &cpArgs[5], &cpArgs[6], &cpArgs[7], &cpArgs[8], &cpArgs[9], }, @@ -1770,7 +1769,7 @@ func TestOptimizePuts(t *testing.T) { // Duplicate put at 11th key; should see ten puts. { nil, nil, - []roachpb.Request{ + []kvpb.Request{ &pArgs[0], &pArgs[1], &pArgs[2], &pArgs[3], &pArgs[4], &pArgs[5], &pArgs[6], &pArgs[7], &pArgs[8], &pArgs[9], &pArgs[9], }, []bool{ @@ -1780,7 +1779,7 @@ func TestOptimizePuts(t *testing.T) { // Duplicate cput at 11th key; should see ten puts. { nil, nil, - []roachpb.Request{ + []kvpb.Request{ &pArgs[0], &pArgs[1], &pArgs[2], &pArgs[3], &pArgs[4], &pArgs[5], &pArgs[6], &pArgs[7], &pArgs[8], &pArgs[9], &cpArgs[9], }, []bool{ @@ -1790,7 +1789,7 @@ func TestOptimizePuts(t *testing.T) { // Duplicate iput at 11th key; should see ten puts. { nil, nil, - []roachpb.Request{ + []kvpb.Request{ &pArgs[0], &pArgs[1], &pArgs[2], &pArgs[3], &pArgs[4], &pArgs[5], &pArgs[6], &pArgs[7], &pArgs[8], &pArgs[9], &ipArgs[9], }, []bool{ @@ -1800,7 +1799,7 @@ func TestOptimizePuts(t *testing.T) { // Duplicate cput at 10th key; should see ten cputs. { nil, nil, - []roachpb.Request{ + []kvpb.Request{ &cpArgs[0], &cpArgs[1], &cpArgs[2], &cpArgs[3], &cpArgs[4], &cpArgs[5], &cpArgs[6], &cpArgs[7], &cpArgs[8], &cpArgs[9], &cpArgs[9], }, []bool{ @@ -1810,7 +1809,7 @@ func TestOptimizePuts(t *testing.T) { // Existing range key at 00-20, ten puts, expect no blind. { roachpb.Key("00"), roachpb.Key("20"), - []roachpb.Request{ + []kvpb.Request{ &pArgs[0], &pArgs[1], &pArgs[2], &pArgs[3], &pArgs[4], &pArgs[5], &pArgs[6], &pArgs[7], &pArgs[8], &pArgs[9], }, []bool{ @@ -1820,7 +1819,7 @@ func TestOptimizePuts(t *testing.T) { // Existing range key at 05-08, ten puts, expect first five puts are blind. { roachpb.Key("05"), roachpb.Key("08"), - []roachpb.Request{ + []kvpb.Request{ &pArgs[0], &pArgs[1], &pArgs[2], &pArgs[3], &pArgs[4], &pArgs[5], &pArgs[6], &pArgs[7], &pArgs[8], &pArgs[9], }, []bool{ @@ -1830,7 +1829,7 @@ func TestOptimizePuts(t *testing.T) { // Existing range key at 20-21, ten puts, expect all blind. { roachpb.Key("20"), roachpb.Key("21"), - []roachpb.Request{ + []kvpb.Request{ &pArgs[0], &pArgs[1], &pArgs[2], &pArgs[3], &pArgs[4], &pArgs[5], &pArgs[6], &pArgs[7], &pArgs[8], &pArgs[9], }, []bool{ @@ -1847,16 +1846,16 @@ func TestOptimizePuts(t *testing.T) { require.NoError(t, storage.MVCCPut(ctx, tc.engine, nil, c.exKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("foo"), nil)) } - batch := roachpb.BatchRequest{} + batch := kvpb.BatchRequest{} for _, r := range c.reqs { batch.Add(r) } // Make a deep clone of the requests slice. We need a deep clone // because the regression which is prevented here changed data on the // individual requests, and not the slice. - goldenRequests := append([]roachpb.RequestUnion(nil), batch.Requests...) + goldenRequests := append([]kvpb.RequestUnion(nil), batch.Requests...) for i := range goldenRequests { - clone := protoutil.Clone(goldenRequests[i].GetInner()).(roachpb.Request) + clone := protoutil.Clone(goldenRequests[i].GetInner()).(kvpb.Request) goldenRequests[i].MustSetInner(clone) } // Save the original slice, allowing us to assert that it doesn't @@ -1872,13 +1871,13 @@ func TestOptimizePuts(t *testing.T) { blind := []bool{} for _, r := range batch.Requests { switch t := r.GetInner().(type) { - case *roachpb.PutRequest: + case *kvpb.PutRequest: blind = append(blind, t.Blind) t.Blind = false - case *roachpb.ConditionalPutRequest: + case *kvpb.ConditionalPutRequest: blind = append(blind, t.Blind) t.Blind = false - case *roachpb.InitPutRequest: + case *kvpb.InitPutRequest: blind = append(blind, t.Blind) t.Blind = false default: @@ -1906,7 +1905,7 @@ func TestAcquireLease(t *testing.T) { gArgs := getArgs([]byte("a")) pArgs := putArgs([]byte("b"), []byte("1")) - for _, test := range []roachpb.Request{ + for _, test := range []kvpb.Request{ &gArgs, &pArgs, } { @@ -1940,7 +1939,7 @@ func TestAcquireLease(t *testing.T) { tc.manualClock.MustAdvanceTo(leaseExpiry(tc.repl)) ts := tc.Clock().Now().Next() - if _, pErr := tc.SendWrappedWith(roachpb.Header{Timestamp: ts}, test); pErr != nil { + if _, pErr := tc.SendWrappedWith(kvpb.Header{Timestamp: ts}, test); pErr != nil { t.Error(pErr) } if !tc.repl.OwnsValidLease(ctx, ts.UnsafeToClockTimestamp()) { @@ -1979,7 +1978,7 @@ func TestAcquireLease(t *testing.T) { // TestLeaseConcurrent requests the lease multiple times, all of which // will join the same LeaseRequest command. This exercises the cloning of -// the *roachpb.Error to ensure that each requestor gets a distinct +// the *kvpb.Error to ensure that each requestor gets a distinct // error object (which prevents regression of #6111) func TestLeaseConcurrent(t *testing.T) { defer leaktest.AfterTest(t)() @@ -2004,8 +2003,8 @@ func TestLeaseConcurrent(t *testing.T) { // with an AmbiguousResultError. cfg.TestingKnobs.DisableRefreshReasonNewLeader = true cfg.TestingKnobs.DisableRefreshReasonNewLeaderOrConfigChange = true - cfg.TestingKnobs.TestingProposalFilter = func(args kvserverbase.ProposalFilterArgs) *roachpb.Error { - ll, ok := args.Req.Requests[0].GetInner().(*roachpb.RequestLeaseRequest) + cfg.TestingKnobs.TestingProposalFilter = func(args kvserverbase.ProposalFilterArgs) *kvpb.Error { + ll, ok := args.Req.Requests[0].GetInner().(*kvpb.RequestLeaseRequest) if !ok || atomic.LoadInt32(&active) == 0 { return nil } @@ -2018,7 +2017,7 @@ func TestLeaseConcurrent(t *testing.T) { // Wait for all lease requests to join the same LeaseRequest command. wg.Wait() if withError { - return roachpb.NewErrorf(origMsg) + return kvpb.NewErrorf(origMsg) } return nil } @@ -2027,7 +2026,7 @@ func TestLeaseConcurrent(t *testing.T) { atomic.StoreInt32(&active, 1) tc.manualClock.MustAdvanceTo(leaseExpiry(tc.repl)) now := tc.Clock().NowAsClockTimestamp() - pErrCh := make(chan *roachpb.Error, num) + pErrCh := make(chan *kvpb.Error, num) for i := 0; i < num; i++ { if err := stopper.RunAsyncTask(ctx, "test", func(ctx context.Context) { tc.repl.mu.Lock() @@ -2046,7 +2045,7 @@ func TestLeaseConcurrent(t *testing.T) { } } - pErrs := make([]*roachpb.Error, num) + pErrs := make([]*kvpb.Error, num) for i := range pErrs { // Make sure all of the responses are in (just so that we can // mess with the "original" error knowing that all of the @@ -2091,16 +2090,16 @@ func TestReplicaUpdateTSCache(t *testing.T) { ts1 := tc.Clock().Now().WithSynthetic(synthetic) gArgs := getArgs([]byte("a")) - if _, pErr := tc.SendWrappedWith(roachpb.Header{Timestamp: ts1}, &gArgs); pErr != nil { + if _, pErr := tc.SendWrappedWith(kvpb.Header{Timestamp: ts1}, &gArgs); pErr != nil { t.Error(pErr) } // Set clock to time 2s for write. tc.manualClock.MustAdvanceTo(timeutil.Unix(2, 0)) ts2 := tc.Clock().Now().WithSynthetic(synthetic) key := roachpb.Key([]byte("b")) - drArgs := roachpb.NewDeleteRange(key, key.Next(), false /* returnKeys */) + drArgs := kvpb.NewDeleteRange(key, key.Next(), false /* returnKeys */) - if _, pErr := tc.SendWrappedWith(roachpb.Header{Timestamp: ts2}, drArgs); pErr != nil { + if _, pErr := tc.SendWrappedWith(kvpb.Header{Timestamp: ts2}, drArgs); pErr != nil { t.Error(pErr) } // Verify the timestamp cache has rTS=1s and wTS=0s for "a". @@ -2189,7 +2188,7 @@ func TestReplicaLatching(t *testing.T) { tc := testContext{} tsc := TestStoreConfig(nil) tsc.TestingKnobs.EvalKnobs.TestingEvalFilter = - func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { + func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { if filterArgs.Hdr.UserPriority == blockingPriority && filterArgs.Index == 0 { blockingStart <- struct{}{} <-blockingDone @@ -2203,8 +2202,8 @@ func TestReplicaLatching(t *testing.T) { defer close(blockingDone) // make sure teardown can happen - sendWithHeader := func(header roachpb.Header, args roachpb.Request) *roachpb.Error { - ba := &roachpb.BatchRequest{} + sendWithHeader := func(header kvpb.Header, args kvpb.Request) *kvpb.Error { + ba := &kvpb.BatchRequest{} ba.Header = header ba.Add(args) @@ -2231,10 +2230,10 @@ func TestReplicaLatching(t *testing.T) { } // Asynchronously put a value to the range with blocking enabled. - cmd1Done := make(chan *roachpb.Error, 1) + cmd1Done := make(chan *kvpb.Error, 1) if err := stopper.RunAsyncTask(context.Background(), "test", func(_ context.Context) { args := readOrWriteArgs(key1, test.cmd1Read) - cmd1Done <- sendWithHeader(roachpb.Header{ + cmd1Done <- sendWithHeader(kvpb.Header{ UserPriority: blockingPriority, }, args) }); err != nil { @@ -2248,19 +2247,19 @@ func TestReplicaLatching(t *testing.T) { } // First, try a command for same key as cmd1 to verify whether it blocks. - cmd2Done := make(chan *roachpb.Error, 1) + cmd2Done := make(chan *kvpb.Error, 1) if err := stopper.RunAsyncTask(context.Background(), "", func(_ context.Context) { args := readOrWriteArgs(key1, test.cmd2Read) - cmd2Done <- sendWithHeader(roachpb.Header{}, args) + cmd2Done <- sendWithHeader(kvpb.Header{}, args) }); err != nil { t.Fatal(err) } // Next, try read for a non-impacted key--should go through immediately. - cmd3Done := make(chan *roachpb.Error, 1) + cmd3Done := make(chan *kvpb.Error, 1) if err := stopper.RunAsyncTask(context.Background(), "", func(_ context.Context) { args := readOrWriteArgs(key2, true) - cmd3Done <- sendWithHeader(roachpb.Header{}, args) + cmd3Done <- sendWithHeader(kvpb.Header{}, args) }); err != nil { t.Fatal(err) } @@ -2339,9 +2338,9 @@ func TestReplicaLatchingInconsistent(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, rc := range []roachpb.ReadConsistencyType{ - roachpb.READ_UNCOMMITTED, - roachpb.INCONSISTENT, + for _, rc := range []kvpb.ReadConsistencyType{ + kvpb.READ_UNCOMMITTED, + kvpb.INCONSISTENT, } { t.Run(rc.String(), func(t *testing.T) { key := roachpb.Key("key1") @@ -2351,11 +2350,11 @@ func TestReplicaLatchingInconsistent(t *testing.T) { tc := testContext{} tsc := TestStoreConfig(nil) tsc.TestingKnobs.EvalKnobs.TestingEvalFilter = - func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { - if put, ok := filterArgs.Req.(*roachpb.PutRequest); ok { + func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { + if put, ok := filterArgs.Req.(*kvpb.PutRequest); ok { putBytes, err := put.Value.GetBytes() if err != nil { - return roachpb.NewErrorWithTxn(err, filterArgs.Hdr.Txn) + return kvpb.NewErrorWithTxn(err, filterArgs.Hdr.Txn) } if bytes.Equal(put.Key, key) && bytes.Equal(putBytes, []byte{1}) { // Absence of replay protection can mean that we end up here @@ -2374,7 +2373,7 @@ func TestReplicaLatchingInconsistent(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(ctx) tc.StartWithStoreConfig(ctx, t, stopper, tsc) - cmd1Done := make(chan *roachpb.Error) + cmd1Done := make(chan *kvpb.Error) go func() { args := putArgs(key, []byte{1}) @@ -2385,11 +2384,11 @@ func TestReplicaLatchingInconsistent(t *testing.T) { <-blockingStart // An inconsistent read to the key won't wait. - cmd2Done := make(chan *roachpb.Error) - go func(rc roachpb.ReadConsistencyType) { + cmd2Done := make(chan *kvpb.Error) + go func(rc kvpb.ReadConsistencyType) { args := getArgs(key) - _, pErr := tc.SendWrappedWith(roachpb.Header{ + _, pErr := tc.SendWrappedWith(kvpb.Header{ ReadConsistency: rc, }, &args) cmd2Done <- pErr @@ -2429,7 +2428,7 @@ func TestReplicaLatchingSelfOverlap(t *testing.T) { testutils.RunTrueAndFalse(t, "cmd1Read", func(t *testing.T, cmd1Read bool) { testutils.RunTrueAndFalse(t, "cmd2Read", func(t *testing.T, cmd2Read bool) { key := fmt.Sprintf("%v,%v", cmd1Read, cmd2Read) - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Add(readOrWriteArgs(roachpb.Key(key), cmd1Read)) ba.Add(readOrWriteArgs(roachpb.Key(key), cmd2Read)) @@ -2438,7 +2437,7 @@ func TestReplicaLatchingSelfOverlap(t *testing.T) { defer cancel() _, pErr := tc.Sender().Send(ctx, ba) if pErr != nil { - if _, ok := pErr.GetDetail().(*roachpb.WriteTooOldError); ok && !cmd1Read && !cmd2Read { + if _, ok := pErr.GetDetail().(*kvpb.WriteTooOldError); ok && !cmd1Read && !cmd2Read { // WriteTooOldError is expected in the write/write case because we don't // allow self-overlapping non-transactional batches. } else { @@ -2465,15 +2464,15 @@ func TestReplicaLatchingTimestampNonInterference(t *testing.T) { tc := testContext{} tsc := TestStoreConfig(nil) tsc.TestingKnobs.EvalKnobs.TestingEvalFilter = - func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { + func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { // Make sure the direct GC path doesn't interfere with this test. if !filterArgs.Req.Header().Key.Equal(blockKey.Load().(roachpb.Key)) { return nil } - if filterArgs.Req.Method() == roachpb.Get && blockReader.Load().(bool) { + if filterArgs.Req.Method() == kvpb.Get && blockReader.Load().(bool) { waitForRequestBlocked <- struct{}{} <-blockCh - } else if filterArgs.Req.Method() == roachpb.Put && blockWriter.Load().(bool) { + } else if filterArgs.Req.Method() == kvpb.Put && blockWriter.Load().(bool) { waitForRequestBlocked <- struct{}{} <-blockCh } @@ -2571,13 +2570,13 @@ func TestReplicaLatchingTimestampNonInterference(t *testing.T) { blockReader.Store(false) blockWriter.Store(false) blockKey.Store(test.key) - errCh := make(chan *roachpb.Error, 2) + errCh := make(chan *kvpb.Error, 2) - baR := &roachpb.BatchRequest{} + baR := &kvpb.BatchRequest{} baR.Timestamp = test.readerTS gArgs := getArgs(test.key) baR.Add(&gArgs) - baW := &roachpb.BatchRequest{} + baW := &kvpb.BatchRequest{} baW.Timestamp = test.writerTS pArgs := putArgs(test.key, []byte("value")) baW.Add(&pArgs) @@ -2638,11 +2637,11 @@ func TestReplicaLatchingSplitDeclaresWrites(t *testing.T) { defer log.Scope(t).Close(t) var spans spanset.SpanSet - cmd, _ := batcheval.LookupCommand(roachpb.EndTxn) + cmd, _ := batcheval.LookupCommand(kvpb.EndTxn) cmd.DeclareKeys( &roachpb.RangeDescriptor{StartKey: roachpb.RKey("a"), EndKey: roachpb.RKey("e")}, - &roachpb.Header{}, - &roachpb.EndTxnRequest{ + &kvpb.Header{}, + &kvpb.EndTxnRequest{ InternalCommitTrigger: &roachpb.InternalCommitTrigger{ SplitTrigger: &roachpb.SplitTrigger{ LeftDesc: roachpb.RangeDescriptor{ @@ -2688,7 +2687,7 @@ func TestReplicaLatchingOptimisticEvaluationKeyLimit(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) testutils.RunTrueAndFalse(t, "point-reads", func(t *testing.T, pointReads bool) { - baRead := &roachpb.BatchRequest{} + baRead := &kvpb.BatchRequest{} if pointReads { gArgs1, gArgs2 := getArgsString("a"), getArgsString("b") gArgs3, gArgs4 := getArgsString("c"), getArgsString("d") @@ -2709,12 +2708,12 @@ func TestReplicaLatchingOptimisticEvaluationKeyLimit(t *testing.T) { tc := testContext{} tsc := TestStoreConfig(nil) tsc.TestingKnobs.EvalKnobs.TestingEvalFilter = - func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { + func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { // Make sure the direct GC path doesn't interfere with this test. if !filterArgs.Req.Header().Key.Equal(blockKey.Load().(roachpb.Key)) { return nil } - if filterArgs.Req.Method() == roachpb.Put && blockWriter.Load().(bool) { + if filterArgs.Req.Method() == kvpb.Put && blockWriter.Load().(bool) { blockedCh <- struct{}{} <-blockCh } @@ -2757,7 +2756,7 @@ func TestReplicaLatchingOptimisticEvaluationKeyLimit(t *testing.T) { } for _, test := range testCases { t.Run(fmt.Sprintf("%+v", test), func(t *testing.T) { - errCh := make(chan *roachpb.Error, 2) + errCh := make(chan *kvpb.Error, 2) pArgs := putArgs([]byte(test.writeKey), []byte("value")) blockKey.Store(roachpb.Key(test.writeKey)) blockWriter.Store(true) @@ -2816,7 +2815,7 @@ func TestReplicaLatchingOptimisticEvaluationSkipLocked(t *testing.T) { defer log.Scope(t).Close(t) testutils.RunTrueAndFalse(t, "point-reads", func(t *testing.T, pointReads bool) { testutils.RunTrueAndFalse(t, "locking-reads", func(t *testing.T, lockingReads bool) { - baRead := &roachpb.BatchRequest{} + baRead := &kvpb.BatchRequest{} baRead.WaitPolicy = lock.WaitPolicy_SkipLocked if pointReads { gArgs1, gArgs2 := getArgsString("a"), getArgsString("b") @@ -2848,13 +2847,13 @@ func TestReplicaLatchingOptimisticEvaluationSkipLocked(t *testing.T) { tc := testContext{} tsc := TestStoreConfig(nil) tsc.TestingKnobs.EvalKnobs.TestingEvalFilter = - func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { + func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { // Make sure the direct GC path doesn't interfere with this test. reqKey := filterArgs.Req.Header().Key if !reqKey.Equal(blockKey1) && !reqKey.Equal(blockKey2) { return nil } - if filterArgs.Req.Method() == roachpb.Put && blockWriters.Load().(bool) { + if filterArgs.Req.Method() == kvpb.Put && blockWriters.Load().(bool) { blockedCh <- struct{}{} <-blockCh } @@ -2876,7 +2875,7 @@ func TestReplicaLatchingOptimisticEvaluationSkipLocked(t *testing.T) { // read evaluates, the read skips over the key and does not conflict with // the write's latches. txn := newTransaction("locker", blockKey1, 0, tc.Clock()) - txnH := roachpb.Header{Txn: txn} + txnH := kvpb.Header{Txn: txn} putArgs1 := putArgs(blockKey1, []byte("value")) _, pErr := tc.SendWrappedWith(txnH, &putArgs1) require.Nil(t, pErr) @@ -2885,7 +2884,7 @@ func TestReplicaLatchingOptimisticEvaluationSkipLocked(t *testing.T) { // likely an even more common cause of blocking today. However, we use a Put // here because we may stop acquiring latches during intent resolution in // the future and don't want this test to break when we do. - errCh := make(chan *roachpb.Error, 3) + errCh := make(chan *kvpb.Error, 3) blockWriters.Store(true) go func() { _, pErr := tc.SendWrappedWith(txnH, &putArgs1) @@ -2907,17 +2906,17 @@ func TestReplicaLatchingOptimisticEvaluationSkipLocked(t *testing.T) { blockWriters.Store(false) var respKeys []roachpb.Key go func() { - errCh <- func() *roachpb.Error { + errCh <- func() *kvpb.Error { br, pErr := tc.Sender().Send(ctx, baRead) if pErr != nil { return pErr } for i, req := range baRead.Requests { resp := br.Responses[i] - if err := roachpb.ResponseKeyIterate(req.GetInner(), resp.GetInner(), func(k roachpb.Key) { + if err := kvpb.ResponseKeyIterate(req.GetInner(), resp.GetInner(), func(k roachpb.Key) { respKeys = append(respKeys, k) }); err != nil { - return roachpb.NewError(err) + return kvpb.NewError(err) } } return nil @@ -2957,12 +2956,12 @@ func TestReplicaUseTSCache(t *testing.T) { readTS := tc.Clock().Now().WithSynthetic(synthetic) args := getArgs([]byte("a")) - _, pErr := tc.SendWrappedWith(roachpb.Header{Timestamp: readTS}, &args) + _, pErr := tc.SendWrappedWith(kvpb.Header{Timestamp: readTS}, &args) require.Nil(t, pErr) // Perform a conflicting write. Should get bumped. pArgs := putArgs([]byte("a"), []byte("value")) - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Add(&pArgs) ba.Timestamp = startTS @@ -3006,7 +3005,7 @@ func TestReplicaTSCacheForwardsIntentTS(t *testing.T) { gArgs := getArgs(keyGet) drArgs := deleteRangeArgs(keyDeleteRange, keyDeleteRange.Next()) assignSeqNumsForReqs(txnNew, &gArgs, &drArgs) - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Header.Txn = txnNew ba.Add(&gArgs, &drArgs) if _, pErr := tc.Sender().Send(ctx, ba); pErr != nil { @@ -3022,7 +3021,7 @@ func TestReplicaTSCacheForwardsIntentTS(t *testing.T) { t.Run(string(key), func(t *testing.T) { pArgs := putArgs(key, []byte("foo")) assignSeqNumsForReqs(txnOld, &pArgs) - if _, pErr := tc.SendWrappedWith(roachpb.Header{Txn: txnOld}, &pArgs); pErr != nil { + if _, pErr := tc.SendWrappedWith(kvpb.Header{Txn: txnOld}, &pArgs); pErr != nil { t.Fatal(pErr) } iter := tc.engine.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{Prefix: true}) @@ -3064,8 +3063,8 @@ func TestConditionalPutUpdatesTSCacheOnError(t *testing.T) { // CPut args which expect value "1" to write "0". key := []byte("a") cpArgs1 := cPutArgs(key, []byte("1"), []byte("0")) - _, pErr := tc.SendWrappedWith(roachpb.Header{Timestamp: t2}, &cpArgs1) - if cfErr, ok := pErr.GetDetail().(*roachpb.ConditionFailedError); !ok { + _, pErr := tc.SendWrappedWith(kvpb.Header{Timestamp: t2}, &cpArgs1) + if cfErr, ok := pErr.GetDetail().(*kvpb.ConditionFailedError); !ok { t.Errorf("expected ConditionFailedError; got %v", pErr) } else if cfErr.ActualValue != nil { t.Errorf("expected empty actual value; got %s", cfErr.ActualValue) @@ -3076,7 +3075,7 @@ func TestConditionalPutUpdatesTSCacheOnError(t *testing.T) { txnEarly := newTransaction("test", key, 1, tc.Clock()) txnEarly.ReadTimestamp, txnEarly.WriteTimestamp = t1, t1 cpArgs2 := cPutArgs(key, []byte("value"), nil) - resp, pErr := tc.SendWrappedWith(roachpb.Header{Txn: txnEarly}, &cpArgs2) + resp, pErr := tc.SendWrappedWith(kvpb.Header{Txn: txnEarly}, &cpArgs2) if pErr != nil { t.Fatal(pErr) } else if respTS := resp.Header().Txn.WriteTimestamp; respTS != t2Next { @@ -3089,7 +3088,7 @@ func TestConditionalPutUpdatesTSCacheOnError(t *testing.T) { t3 := makeTS(3*time.Second.Nanoseconds(), 0) tc.manualClock.MustAdvanceTo(t3.GoTime()) _, pErr = tc.SendWrapped(&cpArgs1) - if _, ok := pErr.GetDetail().(*roachpb.WriteIntentError); !ok { + if _, ok := pErr.GetDetail().(*kvpb.WriteIntentError); !ok { t.Errorf("expected WriteIntentError; got %v", pErr) } @@ -3097,8 +3096,8 @@ func TestConditionalPutUpdatesTSCacheOnError(t *testing.T) { // a later timestamp. This should succeed and should not update // the timestamp cache. abortIntent := func(s roachpb.Span, abortTxn *roachpb.Transaction) { - if _, pErr = tc.SendWrapped(&roachpb.ResolveIntentRequest{ - RequestHeader: roachpb.RequestHeaderFromSpan(s), + if _, pErr = tc.SendWrapped(&kvpb.ResolveIntentRequest{ + RequestHeader: kvpb.RequestHeaderFromSpan(s), IntentTxn: abortTxn.TxnMeta, Status: roachpb.ABORTED, }); pErr != nil { @@ -3108,7 +3107,7 @@ func TestConditionalPutUpdatesTSCacheOnError(t *testing.T) { abortIntent(cpArgs2.Span(), txnEarly) txnLater := *txnEarly txnLater.ReadTimestamp, txnLater.WriteTimestamp = t3, t3 - resp, pErr = tc.SendWrappedWith(roachpb.Header{Txn: &txnLater}, &cpArgs2) + resp, pErr = tc.SendWrappedWith(kvpb.Header{Txn: &txnLater}, &cpArgs2) if pErr != nil { t.Fatal(pErr) } else if respTS := resp.Header().Txn.WriteTimestamp; respTS != t3 { @@ -3120,7 +3119,7 @@ func TestConditionalPutUpdatesTSCacheOnError(t *testing.T) { // or fourth (successful) conditional put. Only the conditional put that // hit a ConditionFailedError should update the timestamp cache. abortIntent(cpArgs2.Span(), &txnLater) - resp, pErr = tc.SendWrappedWith(roachpb.Header{Txn: txnEarly}, &cpArgs2) + resp, pErr = tc.SendWrappedWith(kvpb.Header{Txn: txnEarly}, &cpArgs2) if pErr != nil { t.Fatal(pErr) } else if respTS := resp.Header().Txn.WriteTimestamp; respTS != t2Next { @@ -3156,8 +3155,8 @@ func TestInitPutUpdatesTSCacheOnError(t *testing.T) { // InitPut args to write "1" to same key. Should fail. ipArgs2 := iPutArgs(key, []byte("1")) - _, pErr = tc.SendWrappedWith(roachpb.Header{Timestamp: t2}, &ipArgs2) - if cfErr, ok := pErr.GetDetail().(*roachpb.ConditionFailedError); !ok { + _, pErr = tc.SendWrappedWith(kvpb.Header{Timestamp: t2}, &ipArgs2) + if cfErr, ok := pErr.GetDetail().(*kvpb.ConditionFailedError); !ok { t.Errorf("expected ConditionFailedError; got %v", pErr) } else if valueBytes, err := cfErr.ActualValue.GetBytes(); err != nil { t.Fatal(err) @@ -3169,7 +3168,7 @@ func TestInitPutUpdatesTSCacheOnError(t *testing.T) { // ensure it is pushed. txnEarly := newTransaction("test", key, 1, tc.Clock()) txnEarly.ReadTimestamp, txnEarly.WriteTimestamp = t1, t1 - resp, pErr := tc.SendWrappedWith(roachpb.Header{Txn: txnEarly}, &ipArgs1) + resp, pErr := tc.SendWrappedWith(kvpb.Header{Txn: txnEarly}, &ipArgs1) if pErr != nil { t.Fatal(pErr) } else if respTS := resp.Header().Txn.WriteTimestamp; respTS != t2Next { @@ -3182,7 +3181,7 @@ func TestInitPutUpdatesTSCacheOnError(t *testing.T) { t3 := makeTS(3*time.Second.Nanoseconds(), 0) tc.manualClock.MustAdvanceTo(t3.GoTime()) _, pErr = tc.SendWrapped(&ipArgs2) - if _, ok := pErr.GetDetail().(*roachpb.WriteIntentError); !ok { + if _, ok := pErr.GetDetail().(*kvpb.WriteIntentError); !ok { t.Errorf("expected WriteIntentError; got %v", pErr) } @@ -3190,8 +3189,8 @@ func TestInitPutUpdatesTSCacheOnError(t *testing.T) { // timestamp. This should succeed and should not update the // timestamp cache. abortIntent := func(s roachpb.Span, abortTxn *roachpb.Transaction) { - if _, pErr = tc.SendWrapped(&roachpb.ResolveIntentRequest{ - RequestHeader: roachpb.RequestHeaderFromSpan(s), + if _, pErr = tc.SendWrapped(&kvpb.ResolveIntentRequest{ + RequestHeader: kvpb.RequestHeaderFromSpan(s), IntentTxn: abortTxn.TxnMeta, Status: roachpb.ABORTED, }); pErr != nil { @@ -3201,7 +3200,7 @@ func TestInitPutUpdatesTSCacheOnError(t *testing.T) { abortIntent(ipArgs1.Span(), txnEarly) txnLater := *txnEarly txnLater.ReadTimestamp, txnLater.WriteTimestamp = t3, t3 - resp, pErr = tc.SendWrappedWith(roachpb.Header{Txn: &txnLater}, &ipArgs1) + resp, pErr = tc.SendWrappedWith(kvpb.Header{Txn: &txnLater}, &ipArgs1) if pErr != nil { t.Fatal(pErr) } else if respTS := resp.Header().Txn.WriteTimestamp; respTS != t3 { @@ -3213,7 +3212,7 @@ func TestInitPutUpdatesTSCacheOnError(t *testing.T) { // or fourth (successful) init put. Only the init put that hit a // ConditionFailedError should update the timestamp cache. abortIntent(ipArgs1.Span(), &txnLater) - resp, pErr = tc.SendWrappedWith(roachpb.Header{Txn: txnEarly}, &ipArgs1) + resp, pErr = tc.SendWrappedWith(kvpb.Header{Txn: txnEarly}, &ipArgs1) if pErr != nil { t.Fatal(pErr) } else if respTS := resp.Header().Txn.WriteTimestamp; respTS != t2Next { @@ -3227,9 +3226,9 @@ func TestReplicaNoTSCacheInconsistent(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, rc := range []roachpb.ReadConsistencyType{ - roachpb.READ_UNCOMMITTED, - roachpb.INCONSISTENT, + for _, rc := range []kvpb.ReadConsistencyType{ + kvpb.READ_UNCOMMITTED, + kvpb.INCONSISTENT, } { t.Run(rc.String(), func(t *testing.T) { tc := testContext{} @@ -3243,7 +3242,7 @@ func TestReplicaNoTSCacheInconsistent(t *testing.T) { args := getArgs([]byte("a")) ts := tc.Clock().Now() - _, pErr := tc.SendWrappedWith(roachpb.Header{ + _, pErr := tc.SendWrappedWith(kvpb.Header{ Timestamp: ts, ReadConsistency: rc, }, &args) @@ -3253,8 +3252,8 @@ func TestReplicaNoTSCacheInconsistent(t *testing.T) { } pArgs := putArgs([]byte("a"), []byte("value")) - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Timestamp: hlc.Timestamp{WallTime: 0, Logical: 1}} + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Timestamp: hlc.Timestamp{WallTime: 0, Logical: 1}} ba.Add(&pArgs) br, pErr := tc.Sender().Send(context.Background(), ba) if pErr != nil { @@ -3290,7 +3289,7 @@ func TestReplicaNoTSCacheUpdateOnFailure(t *testing.T) { pArgs := putArgs(key, []byte("value")) assignSeqNumsForReqs(txn, &pArgs) - _, pErr := tc.SendWrappedWith(roachpb.Header{ + _, pErr := tc.SendWrappedWith(kvpb.Header{ Txn: txn, }, &pArgs) if pErr != nil { @@ -3301,14 +3300,14 @@ func TestReplicaNoTSCacheUpdateOnFailure(t *testing.T) { args := readOrWriteArgs(key, read) ts := tc.Clock().Now() // later timestamp - _, pErr = tc.SendWrappedWith(roachpb.Header{Timestamp: ts}, args) - if _, ok := pErr.GetDetail().(*roachpb.WriteIntentError); !ok { + _, pErr = tc.SendWrappedWith(kvpb.Header{Timestamp: ts}, args) + if _, ok := pErr.GetDetail().(*kvpb.WriteIntentError); !ok { t.Errorf("expected WriteIntentError; got %v", pErr) } // Write the intent again -- should not have its timestamp upgraded! - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: txn} + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: txn} ba.Add(&pArgs) assignSeqNumsForReqs(txn, &pArgs) br, pErr := tc.Sender().Send(context.Background(), ba) @@ -3341,15 +3340,15 @@ func TestReplicaNoTSCacheIncrementWithinTxn(t *testing.T) { gArgs := getArgs(key) assignSeqNumsForReqs(txn, &gArgs) - if _, pErr := tc.SendWrappedWith(roachpb.Header{ + if _, pErr := tc.SendWrappedWith(kvpb.Header{ Txn: txn, }, &gArgs); pErr != nil { t.Fatal(pErr) } // Now try a write and verify timestamp isn't incremented. - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: txn} + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: txn} pArgs := putArgs(key, []byte("value")) ba.Add(&pArgs) assignSeqNumsForReqs(txn, &pArgs) @@ -3362,12 +3361,12 @@ func TestReplicaNoTSCacheIncrementWithinTxn(t *testing.T) { } // Resolve the intent. - rArgs := &roachpb.ResolveIntentRequest{ + rArgs := &kvpb.ResolveIntentRequest{ RequestHeader: pArgs.Header(), IntentTxn: txn.TxnMeta, Status: roachpb.COMMITTED, } - if _, pErr = tc.SendWrappedWith(roachpb.Header{Timestamp: txn.WriteTimestamp}, rArgs); pErr != nil { + if _, pErr = tc.SendWrappedWith(kvpb.Header{Timestamp: txn.WriteTimestamp}, rArgs); pErr != nil { t.Fatal(pErr) } @@ -3376,8 +3375,8 @@ func TestReplicaNoTSCacheIncrementWithinTxn(t *testing.T) { expTS := ts expTS.Logical++ - ba = &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Timestamp: ts} + ba = &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Timestamp: ts} ba.Add(&pArgs) assignSeqNumsForReqs(txn, &pArgs) br, pErr = tc.Sender().Send(context.Background(), ba) @@ -3416,7 +3415,7 @@ func TestReplicaAbortSpanReadError(t *testing.T) { args := incrementArgs(k, 1) assignSeqNumsForReqs(txn, args) - if _, pErr := tc.SendWrappedWith(roachpb.Header{ + if _, pErr := tc.SendWrappedWith(kvpb.Header{ Txn: txn, }, args); pErr != nil { t.Fatal(pErr) @@ -3430,7 +3429,7 @@ func TestReplicaAbortSpanReadError(t *testing.T) { } // Now try increment again and verify error. - _, pErr := tc.SendWrappedWith(roachpb.Header{ + _, pErr := tc.SendWrappedWith(kvpb.Header{ Txn: txn, }, args) if !testutils.IsPError(pErr, "replica corruption") { @@ -3483,8 +3482,8 @@ func TestReplicaTxnIdempotency(t *testing.T) { defer stopper.Stop(ctx) tc.Start(ctx, t, stopper) - runWithTxn := func(txn *roachpb.Transaction, reqs ...roachpb.Request) error { - ba := &roachpb.BatchRequest{} + runWithTxn := func(txn *roachpb.Transaction, reqs ...kvpb.Request) error { + ba := &kvpb.BatchRequest{} ba.Header.Txn = txn ba.Add(reqs...) _, pErr := tc.Sender().Send(ctx, ba) @@ -3493,11 +3492,11 @@ func TestReplicaTxnIdempotency(t *testing.T) { keyAtSeqHasVal := func(txn *roachpb.Transaction, key []byte, seq enginepb.TxnSeq, val *roachpb.Value) error { args := getArgs(key) args.Sequence = seq - resp, pErr := tc.SendWrappedWith(roachpb.Header{Txn: txn}, &args) + resp, pErr := tc.SendWrappedWith(kvpb.Header{Txn: txn}, &args) if pErr != nil { return pErr.GoError() } - foundVal := resp.(*roachpb.GetResponse).Value + foundVal := resp.(*kvpb.GetResponse).Value if (foundVal == nil) == (val == nil) { if foundVal == nil { return nil @@ -3922,7 +3921,7 @@ func TestEndTxnDeadline(t *testing.T) { assignSeqNumsForReqs(txn, &put) if _, pErr := kv.SendWrappedWith( - ctx, tc.Sender(), roachpb.Header{Txn: txn}, &put, + ctx, tc.Sender(), kvpb.Header{Txn: txn}, &put, ); pErr != nil { t.Fatal(pErr) } @@ -3956,8 +3955,8 @@ func TestEndTxnDeadline(t *testing.T) { fallthrough case 2: // Past deadline. - retErr, ok := pErr.GetDetail().(*roachpb.TransactionRetryError) - if !ok || retErr.Reason != roachpb.RETRY_COMMIT_DEADLINE_EXCEEDED { + retErr, ok := pErr.GetDetail().(*kvpb.TransactionRetryError) + if !ok || retErr.Reason != kvpb.RETRY_COMMIT_DEADLINE_EXCEEDED { t.Fatalf("expected deadline exceeded, got: %v", pErr) } case 3: @@ -3988,12 +3987,12 @@ func TestSerializableDeadline(t *testing.T) { tc.manualClock.Advance(100) pusher := newTransaction( "test pusher", key, roachpb.MaxUserPriority, tc.Clock()) - pushReq := pushTxnArgs(pusher, txn, roachpb.PUSH_TIMESTAMP) + pushReq := pushTxnArgs(pusher, txn, kvpb.PUSH_TIMESTAMP) resp, pErr := tc.SendWrapped(&pushReq) if pErr != nil { t.Fatal(pErr) } - updatedPushee := resp.(*roachpb.PushTxnResponse).PusheeTxn + updatedPushee := resp.(*kvpb.PushTxnResponse).PusheeTxn if updatedPushee.Status != roachpb.PENDING { t.Fatalf("expected pushee to still be alive, but got %+v", updatedPushee) } @@ -4009,7 +4008,7 @@ func TestSerializableDeadline(t *testing.T) { t.Fatalf("expected %q, got: nil", expectedErrMsg) } err := pErr.GoError() - if _, ok := pErr.GetDetail().(*roachpb.TransactionRetryError); !ok || + if _, ok := pErr.GetDetail().(*kvpb.TransactionRetryError); !ok || !testutils.IsError(err, expectedErrMsg) { t.Fatalf("expected %q, got: %s (%T)", expectedErrMsg, err, pErr.GoError()) @@ -4042,18 +4041,18 @@ func TestCreateTxnRecordAfterPushAndGC(t *testing.T) { // This pushee should never be allowed to write a txn record because it // will be aborted before it even tries. pushee := newTransaction("pushee", key, 1, tc.Clock()) - pushReq := pushTxnArgs(pusher, pushee, roachpb.PUSH_ABORT) + pushReq := pushTxnArgs(pusher, pushee, kvpb.PUSH_ABORT) pushReq.Force = true resp, pErr := tc.SendWrapped(&pushReq) if pErr != nil { t.Fatal(pErr) } - abortedPushee := resp.(*roachpb.PushTxnResponse).PusheeTxn + abortedPushee := resp.(*kvpb.PushTxnResponse).PusheeTxn if abortedPushee.Status != roachpb.ABORTED { t.Fatalf("expected push to abort pushee, got %+v", abortedPushee) } - gcHeader := roachpb.RequestHeader{ + gcHeader := kvpb.RequestHeader{ Key: desc.StartKey.AsRawKey(), EndKey: desc.EndKey.AsRawKey(), } @@ -4062,13 +4061,13 @@ func TestCreateTxnRecordAfterPushAndGC(t *testing.T) { // would after a period of inactivity, while our pushee txn is unaware and // may have written intents elsewhere. { - gcReq := roachpb.GCRequest{ + gcReq := kvpb.GCRequest{ RequestHeader: gcHeader, - Keys: []roachpb.GCRequest_GCKey{ + Keys: []kvpb.GCRequest_GCKey{ {Key: keys.TransactionKey(pushee.Key, pushee.ID)}, }, } - if _, pErr := tc.SendWrappedWith(roachpb.Header{RangeID: 1}, &gcReq); pErr != nil { + if _, pErr := tc.SendWrappedWith(kvpb.Header{RangeID: 1}, &gcReq); pErr != nil { t.Fatal(pErr) } } @@ -4138,13 +4137,13 @@ func TestEndTxnDeadline_1PC(t *testing.T) { // Past deadline. et.Deadline = txn.WriteTimestamp.Prev() - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Header = etH ba.Add(&put, &et) assignSeqNumsForReqs(txn, &put, &et) _, pErr := tc.Sender().Send(ctx, ba) - retErr, ok := pErr.GetDetail().(*roachpb.TransactionRetryError) - if !ok || retErr.Reason != roachpb.RETRY_COMMIT_DEADLINE_EXCEEDED { + retErr, ok := pErr.GetDetail().(*kvpb.TransactionRetryError) + if !ok || retErr.Reason != kvpb.RETRY_COMMIT_DEADLINE_EXCEEDED { t.Fatalf("expected deadline exceeded, got: %v", pErr) } } @@ -4175,12 +4174,12 @@ func Test1PCTransactionWriteTimestamp(t *testing.T) { } // Now verify that the write triggers a retry. - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Header = etH ba.Add(&put, &et) assignSeqNumsForReqs(txn, &put, &et) _, pErr := tc.Sender().Send(ctx, ba) - if _, ok := pErr.GetDetail().(*roachpb.TransactionRetryError); !ok { + if _, ok := pErr.GetDetail().(*kvpb.TransactionRetryError); !ok { t.Errorf("expected retry error; got %s", pErr) } } @@ -4207,7 +4206,7 @@ func TestEndTxnWithMalformedSplitTrigger(t *testing.T) { txn := newTransaction("test", key, 1, tc.Clock()) pArgs := putArgs(key, []byte("only here to make this a rw transaction")) assignSeqNumsForReqs(txn, &pArgs) - if _, pErr := kv.SendWrappedWith(ctx, tc.Sender(), roachpb.Header{ + if _, pErr := kv.SendWrappedWith(ctx, tc.Sender(), kvpb.Header{ Txn: txn, }, &pArgs); pErr != nil { t.Fatal(pErr) @@ -4260,7 +4259,7 @@ func TestEndTxnBeforeHeartbeat(t *testing.T) { testutils.RunTrueAndFalse(t, "commit", func(t *testing.T, commit bool) { key = roachpb.Key(key).Next() txn := newTransaction("test", key, 1, tc.Clock()) - h := roachpb.Header{Txn: txn} + h := kvpb.Header{Txn: txn} put := putArgs(key, key) assignSeqNumsForReqs(txn, &put) @@ -4274,7 +4273,7 @@ func TestEndTxnBeforeHeartbeat(t *testing.T) { if pErr != nil { t.Fatal(pErr) } - reply := resp.(*roachpb.EndTxnResponse) + reply := resp.(*kvpb.EndTxnResponse) expStatus := roachpb.COMMITTED if !commit { expStatus = roachpb.ABORTED @@ -4290,7 +4289,7 @@ func TestEndTxnBeforeHeartbeat(t *testing.T) { if pErr != nil { t.Error(pErr) } - hBR := resp.(*roachpb.HeartbeatTxnResponse) + hBR := resp.(*kvpb.HeartbeatTxnResponse) if hBR.Txn.Status != expStatus { t.Errorf("expected transaction status to be %s, but got %s", expStatus, hBR.Txn.Status) } @@ -4311,7 +4310,7 @@ func TestEndTxnAfterHeartbeat(t *testing.T) { key := roachpb.Key("a") testutils.RunTrueAndFalse(t, "commit", func(t *testing.T, commit bool) { txn := newTransaction("test", key, 1, tc.Clock()) - h := roachpb.Header{Txn: txn} + h := kvpb.Header{Txn: txn} put := putArgs(key, key) assignSeqNumsForReqs(txn, &put) @@ -4325,7 +4324,7 @@ func TestEndTxnAfterHeartbeat(t *testing.T) { if pErr != nil { t.Fatal(pErr) } - hBR := resp.(*roachpb.HeartbeatTxnResponse) + hBR := resp.(*kvpb.HeartbeatTxnResponse) if hBR.Txn.Status != roachpb.PENDING { t.Errorf("expected transaction status to be %s, but got %s", hBR.Txn.Status, roachpb.PENDING) } @@ -4336,7 +4335,7 @@ func TestEndTxnAfterHeartbeat(t *testing.T) { if pErr != nil { t.Error(pErr) } - reply := resp.(*roachpb.EndTxnResponse) + reply := resp.(*kvpb.EndTxnResponse) expStatus := roachpb.COMMITTED if !commit { expStatus = roachpb.ABORTED @@ -4380,13 +4379,13 @@ func TestEndTxnWithPushedTimestamp(t *testing.T) { put := putArgs(key, []byte("value")) assignSeqNumsForReqs(pushee, &put) if _, pErr := kv.SendWrappedWith( - ctx, tc.Sender(), roachpb.Header{Txn: pushee}, &put, + ctx, tc.Sender(), kvpb.Header{Txn: pushee}, &put, ); pErr != nil { t.Fatal(pErr) } // Push pushee txn. - pushTxn := pushTxnArgs(pusher, pushee, roachpb.PUSH_TIMESTAMP) + pushTxn := pushTxnArgs(pusher, pushee, kvpb.PUSH_TIMESTAMP) pushTxn.Key = pusher.Key if _, pErr := tc.SendWrapped(&pushTxn); pErr != nil { t.Error(pErr) @@ -4398,7 +4397,7 @@ func TestEndTxnWithPushedTimestamp(t *testing.T) { resp, pErr := tc.SendWrappedWith(h, &endTxn) if test.expErr { - if _, ok := pErr.GetDetail().(*roachpb.TransactionRetryError); !ok { + if _, ok := pErr.GetDetail().(*kvpb.TransactionRetryError); !ok { t.Errorf("%d: expected retry error; got %s", i, pErr) } } else { @@ -4409,7 +4408,7 @@ func TestEndTxnWithPushedTimestamp(t *testing.T) { if !test.commit { expStatus = roachpb.ABORTED } - reply := resp.(*roachpb.EndTxnResponse) + reply := resp.(*kvpb.EndTxnResponse) if reply.Txn.Status != expStatus { t.Errorf("%d: expected transaction status to be %s; got %s", i, expStatus, reply.Txn.Status) } @@ -4433,7 +4432,7 @@ func TestEndTxnWithIncrementedEpoch(t *testing.T) { txn := newTransaction("test", key, 1, tc.Clock()) put := putArgs(key, key) assignSeqNumsForReqs(txn, &put) - if _, pErr := kv.SendWrappedWith(ctx, tc.Sender(), roachpb.Header{Txn: txn}, &put); pErr != nil { + if _, pErr := kv.SendWrappedWith(ctx, tc.Sender(), kvpb.Header{Txn: txn}, &put); pErr != nil { t.Fatal(pErr) } @@ -4455,7 +4454,7 @@ func TestEndTxnWithIncrementedEpoch(t *testing.T) { if pErr != nil { t.Error(pErr) } - reply := resp.(*roachpb.EndTxnResponse) + reply := resp.(*kvpb.EndTxnResponse) if reply.Txn.Status != roachpb.COMMITTED { t.Errorf("expected transaction status to be COMMITTED; got %s", reply.Txn.Status) } @@ -4580,7 +4579,7 @@ func TestEndTxnRollbackAbortedTransaction(t *testing.T) { put := putArgs(key, key) assignSeqNumsForReqs(txn, &put) if _, pErr := kv.SendWrappedWith( - ctx, tc.Sender(), roachpb.Header{Txn: txn}, &put, + ctx, tc.Sender(), kvpb.Header{Txn: txn}, &put, ); pErr != nil { t.Fatal(pErr) } @@ -4590,20 +4589,20 @@ func TestEndTxnRollbackAbortedTransaction(t *testing.T) { // Abort the transaction by pushing it with maximum priority. pusher := newTransaction("test", key, 1, tc.Clock()) pusher.Priority = enginepb.MaxTxnPriority - pushArgs := pushTxnArgs(pusher, txn, roachpb.PUSH_ABORT) + pushArgs := pushTxnArgs(pusher, txn, kvpb.PUSH_ABORT) if _, pErr := tc.SendWrapped(&pushArgs); pErr != nil { t.Fatal(pErr) } // Check that the intent has not yet been resolved. - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} gArgs := getArgs(key) ba.Add(&gArgs) if err := ba.SetActiveTimestamp(tc.Clock()); err != nil { t.Fatal(err) } _, pErr := tc.Sender().Send(ctx, ba) - if _, ok := pErr.GetDetail().(*roachpb.WriteIntentError); !ok { + if _, ok := pErr.GetDetail().(*kvpb.WriteIntentError); !ok { t.Errorf("expected write intent error, but got %s", pErr) } @@ -4634,7 +4633,7 @@ func TestEndTxnRollbackAbortedTransaction(t *testing.T) { if pErr != nil { t.Fatal(pErr) } - reply := resp.(*roachpb.EndTxnResponse) + reply := resp.(*kvpb.EndTxnResponse) if reply.Txn.Status != roachpb.ABORTED { t.Errorf("expected transaction status to be ABORTED; got %s", reply.Txn.Status) } @@ -4663,12 +4662,12 @@ func TestRPCRetryProtectionInTxn(t *testing.T) { txn := newTransaction("test", key, 1, tc.Clock()) // Send a batch with put & end txn. - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.CanForwardReadTimestamp = noPriorReads put := putArgs(key, []byte("value")) et, _ := endTxnArgs(txn, true) et.LockSpans = []roachpb.Span{{Key: key, EndKey: nil}} - ba.Header = roachpb.Header{Txn: txn} + ba.Header = kvpb.Header{Txn: txn} ba.Add(&put) ba.Add(&et) assignSeqNumsForReqs(txn, &put, &et) @@ -4716,23 +4715,23 @@ func TestErrorsDontCarryWriteTooOldFlag(t *testing.T) { // Write a value outside of the txn to cause a WriteTooOldError later. put := putArgs(keyA, []byte("val1")) { - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Add(&put) _, pErr := tc.Sender().Send(ctx, ba) require.Nil(t, pErr) } - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} // This put will cause the WriteTooOld flag to be set. put = putArgs(keyA, []byte("val2")) // This will cause a ConditionFailedError. cput := cPutArgs(keyB, []byte("missing"), []byte("newVal")) - ba.Header = roachpb.Header{Txn: &txn} + ba.Header = kvpb.Header{Txn: &txn} ba.Add(&put) ba.Add(&cput) assignSeqNumsForReqs(&txn, &put, &cput) _, pErr := tc.Sender().Send(ctx, ba) - require.IsType(t, pErr.GetDetail(), &roachpb.ConditionFailedError{}) + require.IsType(t, pErr.GetDetail(), &kvpb.ConditionFailedError{}) require.False(t, pErr.GetTxn().WriteTooOld) } @@ -4763,9 +4762,9 @@ func TestBatchRetryCantCommitIntents(t *testing.T) { txn := newTransaction("test", key, 1, tc.Clock()) // Send a put for keyA. - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} put := putArgs(key, []byte("value")) - ba.Header = roachpb.Header{Txn: txn} + ba.Header = kvpb.Header{Txn: txn} ba.Add(&put) assignSeqNumsForReqs(txn, &put) if err := ba.SetActiveTimestamp(tc.Clock()); err != nil { @@ -4777,10 +4776,10 @@ func TestBatchRetryCantCommitIntents(t *testing.T) { } // Send a put for keyB. - ba2 := &roachpb.BatchRequest{} + ba2 := &kvpb.BatchRequest{} putB := putArgs(keyB, []byte("value")) putTxn := br.Txn.Clone() - ba2.Header = roachpb.Header{Txn: putTxn} + ba2.Header = kvpb.Header{Txn: putTxn} ba2.Add(&putB) assignSeqNumsForReqs(putTxn, &putB) br, pErr = tc.Sender().Send(ctx, ba2) @@ -4822,7 +4821,7 @@ func TestBatchRetryCantCommitIntents(t *testing.T) { // Intent should have been created. gArgs := getArgs(key) _, pErr = tc.SendWrapped(&gArgs) - if _, ok := pErr.GetDetail().(*roachpb.WriteIntentError); !ok { + if _, ok := pErr.GetDetail().(*kvpb.WriteIntentError); !ok { t.Errorf("expected WriteIntentError, got: %v", pErr) } @@ -4842,7 +4841,7 @@ func TestBatchRetryCantCommitIntents(t *testing.T) { // Expect that the txn left behind an intent on key A. gArgs = getArgs(key) _, pErr = tc.SendWrapped(&gArgs) - if _, ok := pErr.GetDetail().(*roachpb.WriteIntentError); !ok { + if _, ok := pErr.GetDetail().(*kvpb.WriteIntentError); !ok { t.Errorf("expected WriteIntentError, got: %v", pErr) } } @@ -4856,10 +4855,10 @@ func TestEndTxnLocalGC(t *testing.T) { tc := testContext{} tsc := TestStoreConfig(nil) tsc.TestingKnobs.EvalKnobs.TestingEvalFilter = - func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { + func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { // Make sure the direct GC path doesn't interfere with this test. - if filterArgs.Req.Method() == roachpb.GC { - return roachpb.NewErrorWithTxn(errors.Errorf("boom"), filterArgs.Hdr.Txn) + if filterArgs.Req.Method() == kvpb.GC { + return kvpb.NewErrorWithTxn(errors.Errorf("boom"), filterArgs.Hdr.Txn) } return nil } @@ -4890,7 +4889,7 @@ func TestEndTxnLocalGC(t *testing.T) { txn := newTransaction("test", key, 1, tc.Clock()) put := putArgs(putKey, key) assignSeqNumsForReqs(txn, &put) - if _, pErr := kv.SendWrappedWith(ctx, tc.Sender(), roachpb.Header{Txn: txn}, &put); pErr != nil { + if _, pErr := kv.SendWrappedWith(ctx, tc.Sender(), kvpb.Header{Txn: txn}, &put); pErr != nil { t.Fatal(pErr) } putKey = putKey.Next() // for the next iteration @@ -4926,14 +4925,14 @@ func setupResolutionTest( // values are unexpected. txn.Epoch++ pArgs := putArgs(key, []byte("value")) - h := roachpb.Header{Txn: txn} + h := kvpb.Header{Txn: txn} assignSeqNumsForReqs(txn, &pArgs) if _, pErr := kv.SendWrappedWith(context.Background(), tc.Sender(), h, &pArgs); pErr != nil { t.Fatal(pErr) } { - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Header = h ba.RangeID = newRepl.RangeID if err := ba.SetActiveTimestamp(newRepl.store.Clock()); err != nil { @@ -4968,10 +4967,10 @@ func TestEndTxnResolveOnlyLocalIntents(t *testing.T) { key := roachpb.Key("a") splitKey := roachpb.RKey(key).Next() tsc.TestingKnobs.EvalKnobs.TestingEvalFilter = - func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { - if filterArgs.Req.Method() == roachpb.ResolveIntent && + func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { + if filterArgs.Req.Method() == kvpb.ResolveIntent && filterArgs.Req.Header().Key.Equal(splitKey.AsRawKey()) { - return roachpb.NewErrorWithTxn(errors.Errorf("boom"), filterArgs.Hdr.Txn) + return kvpb.NewErrorWithTxn(errors.Errorf("boom"), filterArgs.Hdr.Txn) } return nil } @@ -4985,7 +4984,7 @@ func TestEndTxnResolveOnlyLocalIntents(t *testing.T) { // Check if the intent in the other range has not yet been resolved. { - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Header.RangeID = newRepl.RangeID gArgs := getArgs(splitKey) ba.Add(&gArgs) @@ -4993,7 +4992,7 @@ func TestEndTxnResolveOnlyLocalIntents(t *testing.T) { t.Fatal(err) } _, pErr := newRepl.Send(ctx, ba) - if _, ok := pErr.GetDetail().(*roachpb.WriteIntentError); !ok { + if _, ok := pErr.GetDetail().(*kvpb.WriteIntentError); !ok { t.Errorf("expected write intent error, but got %s", pErr) } } @@ -5003,7 +5002,7 @@ func TestEndTxnResolveOnlyLocalIntents(t *testing.T) { if pErr != nil { t.Fatal(pErr) } - hbResp := reply.(*roachpb.HeartbeatTxnResponse) + hbResp := reply.(*kvpb.HeartbeatTxnResponse) expIntents := []roachpb.Span{{Key: splitKey.AsRawKey()}} if !reflect.DeepEqual(hbResp.Txn.LockSpans, expIntents) { t.Fatalf("expected persisted intents %v, got %v", @@ -5036,13 +5035,13 @@ func TestEndTxnDirectGC(t *testing.T) { rightRepl, txn := setupResolutionTest(t, tc, testKey, splitKey, false /* generate AbortSpan entry */) testutils.SucceedsSoon(t, func() error { - var gr roachpb.GetResponse + var gr kvpb.GetResponse if _, err := batcheval.Get( ctx, tc.engine, batcheval.CommandArgs{ EvalCtx: NewReplicaEvalContext( ctx, tc.repl, allSpans(), false, /* requiresClosedTSOlderThanStorageSnap */ ), - Args: &roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{ + Args: &kvpb.GetRequest{RequestHeader: kvpb.RequestHeader{ Key: keys.TransactionKey(txn.Key, txn.ID), }}, }, @@ -5082,12 +5081,12 @@ func TestEndTxnDirectGCFailure(t *testing.T) { var count int64 tsc := TestStoreConfig(nil) tsc.TestingKnobs.EvalKnobs.TestingEvalFilter = - func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { - if filterArgs.Req.Method() == roachpb.ResolveIntent && + func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { + if filterArgs.Req.Method() == kvpb.ResolveIntent && filterArgs.Req.Header().Key.Equal(splitKey.AsRawKey()) { atomic.AddInt64(&count, 1) - return roachpb.NewErrorWithTxn(errors.Errorf("boom"), filterArgs.Hdr.Txn) - } else if filterArgs.Req.Method() == roachpb.GC { + return kvpb.NewErrorWithTxn(errors.Errorf("boom"), filterArgs.Hdr.Txn) + } else if filterArgs.Req.Method() == kvpb.GC { // Can't fatal since we're on a goroutine. This'll do it. t.Error(errors.Errorf("unexpected GCRequest: %+v", filterArgs.Req)) } @@ -5135,14 +5134,14 @@ func TestEndTxnDirectGC_1PC(t *testing.T) { et.LockSpans = []roachpb.Span{{Key: key}} assignSeqNumsForReqs(txn, &put, &et) - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Header = etH ba.Add(&put, &et) br, err := tc.Sender().Send(ctx, ba) if err != nil { t.Fatalf("commit=%t: %+v", commit, err) } - etArgs, ok := br.Responses[len(br.Responses)-1].GetInner().(*roachpb.EndTxnResponse) + etArgs, ok := br.Responses[len(br.Responses)-1].GetInner().(*kvpb.EndTxnResponse) if !ok || (!etArgs.OnePhaseCommit && commit) { t.Errorf("commit=%t: expected one phase commit", commit) } @@ -5169,10 +5168,10 @@ func TestReplicaTransactionRequires1PC(t *testing.T) { injectErrorOnKey.Store(roachpb.Key("")) tsc.TestingKnobs.EvalKnobs.TestingEvalFilter = - func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { - if filterArgs.Req.Method() == roachpb.Put && + func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { + if filterArgs.Req.Method() == kvpb.Put && injectErrorOnKey.Load().(roachpb.Key).Equal(filterArgs.Req.Header().Key) { - return roachpb.NewErrorf("injected error") + return kvpb.NewErrorf("injected error") } return nil } @@ -5211,7 +5210,7 @@ func TestReplicaTransactionRequires1PC(t *testing.T) { key := roachpb.Key(fmt.Sprintf("%d", i)) // Create the 1PC batch. - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} txn := newTransaction("test", key, 1, tc.Clock()) put := putArgs(key, []byte("value")) et, etH := endTxnArgs(txn, true) @@ -5253,8 +5252,8 @@ func TestReplicaEndTxnWithRequire1PC(t *testing.T) { key := roachpb.Key("a") txn := newTransaction("test", key, 1, tc.Clock()) - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: txn} + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: txn} put := putArgs(key, []byte("value")) ba.Add(&put) assignSeqNumsForReqs(txn, &put) @@ -5264,7 +5263,7 @@ func TestReplicaEndTxnWithRequire1PC(t *testing.T) { et, etH := endTxnArgs(txn, true) et.Require1PC = true - ba = &roachpb.BatchRequest{} + ba = &kvpb.BatchRequest{} ba.Header = etH ba.Add(&et) assignSeqNumsForReqs(txn, &et) @@ -5299,25 +5298,25 @@ func TestAbortSpanPoisonOnResolve(t *testing.T) { pusher.Priority = enginepb.MaxTxnPriority pushee.Priority = enginepb.MinTxnPriority // pusher will win - inc := func(actor *roachpb.Transaction, k roachpb.Key) (*roachpb.IncrementResponse, *roachpb.Error) { - incArgs := &roachpb.IncrementRequest{ - RequestHeader: roachpb.RequestHeader{Key: k}, Increment: 123, + inc := func(actor *roachpb.Transaction, k roachpb.Key) (*kvpb.IncrementResponse, *kvpb.Error) { + incArgs := &kvpb.IncrementRequest{ + RequestHeader: kvpb.RequestHeader{Key: k}, Increment: 123, } assignSeqNumsForReqs(actor, incArgs) - reply, pErr := kv.SendWrappedWith(ctx, tc.store, roachpb.Header{ + reply, pErr := kv.SendWrappedWith(ctx, tc.store, kvpb.Header{ Txn: actor, RangeID: 1, }, incArgs) if pErr != nil { return nil, pErr } - return reply.(*roachpb.IncrementResponse), nil + return reply.(*kvpb.IncrementResponse), nil } - get := func(actor *roachpb.Transaction, k roachpb.Key) *roachpb.Error { + get := func(actor *roachpb.Transaction, k roachpb.Key) *kvpb.Error { gArgs := getArgs(k) assignSeqNumsForReqs(actor, &gArgs) - _, pErr := kv.SendWrappedWith(ctx, tc.store, roachpb.Header{ + _, pErr := kv.SendWrappedWith(ctx, tc.store, kvpb.Header{ Txn: actor, RangeID: 1, }, &gArgs) @@ -5331,21 +5330,21 @@ func TestAbortSpanPoisonOnResolve(t *testing.T) { // Have the pusher run into the intent. That pushes our pushee and // resolves the intent, which in turn should poison the AbortSpan. - var assert func(*roachpb.Error) error + var assert func(*kvpb.Error) error if abort { // Write/Write conflict will abort pushee. if _, pErr := inc(pusher, key); pErr != nil { t.Fatal(pErr) } - assert = func(pErr *roachpb.Error) error { - if _, ok := pErr.GetDetail().(*roachpb.TransactionAbortedError); !ok { + assert = func(pErr *kvpb.Error) error { + if _, ok := pErr.GetDetail().(*kvpb.TransactionAbortedError); !ok { return errors.Errorf("abort=%t: expected txn abort, got %s", abort, pErr) } return nil } } else { // Verify we're not poisoned. - assert = func(pErr *roachpb.Error) error { + assert = func(pErr *kvpb.Error) error { if pErr != nil { return errors.Errorf("abort=%t: unexpected: %s", abort, pErr) } @@ -5395,7 +5394,7 @@ func TestAbortSpanPoisonOnResolve(t *testing.T) { } } -// TestAbortSpanError verifies that roachpb.Errors returned by checkIfTxnAborted +// TestAbortSpanError verifies that kvpb.Errors returned by checkIfTxnAborted // have txns that are identical to txns stored in Transaction{Retry,Aborted}Error. func TestAbortSpanError(t *testing.T) { defer leaktest.AfterTest(t)() @@ -5427,7 +5426,7 @@ func TestAbortSpanError(t *testing.T) { ec := newEvalContextImpl(ctx, tc.repl, false /* requireClosedTS */) rec := &SpanSetReplicaEvalContext{ec, *allSpans()} pErr := checkIfTxnAborted(ctx, rec, tc.engine, txn) - if _, ok := pErr.GetDetail().(*roachpb.TransactionAbortedError); ok { + if _, ok := pErr.GetDetail().(*kvpb.TransactionAbortedError); ok { expected := txn.Clone() expected.WriteTimestamp = txn.WriteTimestamp expected.Priority = priority @@ -5453,7 +5452,7 @@ func TestPushTxnBadKey(t *testing.T) { pusher := newTransaction("test", roachpb.Key("a"), 1, tc.Clock()) pushee := newTransaction("test", roachpb.Key("b"), 1, tc.Clock()) - args := pushTxnArgs(pusher, pushee, roachpb.PUSH_ABORT) + args := pushTxnArgs(pusher, pushee, kvpb.PUSH_ABORT) args.Key = pusher.Key if _, pErr := tc.SendWrapped(&args); !testutils.IsPError(pErr, ".*should match pushee.*") { @@ -5496,7 +5495,7 @@ func TestPushTxnAlreadyCommittedOrAborted(t *testing.T) { // Begin the pushee's transaction. put := putArgs(key, key) assignSeqNumsForReqs(pushee, &put) - if _, pErr := kv.SendWrappedWith(ctx, tc.Sender(), roachpb.Header{Txn: pushee}, &put); pErr != nil { + if _, pErr := kv.SendWrappedWith(ctx, tc.Sender(), kvpb.Header{Txn: pushee}, &put); pErr != nil { t.Fatal(pErr) } // End the pushee's transaction. @@ -5507,12 +5506,12 @@ func TestPushTxnAlreadyCommittedOrAborted(t *testing.T) { } // Now try to push what's already committed or aborted. - args := pushTxnArgs(pusher, pushee, roachpb.PUSH_ABORT) + args := pushTxnArgs(pusher, pushee, kvpb.PUSH_ABORT) resp, pErr := tc.SendWrapped(&args) if pErr != nil { t.Fatal(pErr) } - reply := resp.(*roachpb.PushTxnResponse) + reply := resp.(*kvpb.PushTxnResponse) // We expect the push to return an ABORTED transaction record for all // cases except when the transaction is COMMITTED and its record is not @@ -5578,16 +5577,16 @@ func TestPushTxnUpgradeExistingTxn(t *testing.T) { // Now, attempt to push the transaction using updated timestamp. pushee.WriteTimestamp = pushTS - args := pushTxnArgs(pusher, pushee, roachpb.PUSH_ABORT) + args := pushTxnArgs(pusher, pushee, kvpb.PUSH_ABORT) // Set header timestamp to the maximum of the pusher and pushee timestamps. - h := roachpb.Header{Timestamp: args.PushTo} + h := kvpb.Header{Timestamp: args.PushTo} h.Timestamp.Forward(pushee.WriteTimestamp) resp, pErr := tc.SendWrappedWith(h, &args) if pErr != nil { t.Fatal(pErr) } - reply := resp.(*roachpb.PushTxnResponse) + reply := resp.(*kvpb.PushTxnResponse) expTxnRecord := pushee.AsRecord() expTxn := expTxnRecord.AsTransaction() expTxn.Priority = enginepb.MaxTxnPriority - 1 @@ -5628,20 +5627,20 @@ func TestPushTxnQueryPusheeHasNewerVersion(t *testing.T) { put := putArgs(key, key) assignSeqNumsForReqs(pushee, &put) - if _, pErr := kv.SendWrappedWith(ctx, tc.Sender(), roachpb.Header{Txn: pushee}, &put); pErr != nil { + if _, pErr := kv.SendWrappedWith(ctx, tc.Sender(), kvpb.Header{Txn: pushee}, &put); pErr != nil { t.Fatal(pErr) } // Make sure the pushee in the request has updated information on the pushee. // Since the pushee has higher priority than the pusher, the push should fail. pushee.Priority = 4 - args := pushTxnArgs(pusher, pushee, roachpb.PUSH_ABORT) + args := pushTxnArgs(pusher, pushee, kvpb.PUSH_ABORT) _, pErr := tc.SendWrapped(&args) if pErr == nil { t.Fatalf("unexpected push success") } - if _, ok := pErr.GetDetail().(*roachpb.TransactionPushError); !ok { + if _, ok := pErr.GetDetail().(*kvpb.TransactionPushError); !ok { t.Errorf("expected txn push error: %s", pErr) } } @@ -5670,72 +5669,72 @@ func TestPushTxnHeartbeatTimeout(t *testing.T) { status roachpb.TransactionStatus // -1 for no record heartbeatOffset int64 // nanoseconds from original timestamp, 0 for no heartbeat timeOffset int64 // nanoseconds from original timestamp - pushType roachpb.PushTxnType + pushType kvpb.PushTxnType expErr string }{ // Avoid using offsets that result in outcomes that depend on logical // ticks. - {roachpb.PENDING, 0, 1, roachpb.PUSH_TIMESTAMP, txnPushError}, - {roachpb.PENDING, 0, 1, roachpb.PUSH_ABORT, txnPushError}, - {roachpb.PENDING, 0, 1, roachpb.PUSH_TOUCH, txnPushError}, - {roachpb.PENDING, 0, ns, roachpb.PUSH_TIMESTAMP, txnPushError}, - {roachpb.PENDING, 0, ns, roachpb.PUSH_ABORT, txnPushError}, - {roachpb.PENDING, 0, ns, roachpb.PUSH_TOUCH, txnPushError}, - {roachpb.PENDING, 0, m*ns - 1, roachpb.PUSH_TIMESTAMP, txnPushError}, - {roachpb.PENDING, 0, m*ns - 1, roachpb.PUSH_ABORT, txnPushError}, - {roachpb.PENDING, 0, m*ns - 1, roachpb.PUSH_TOUCH, txnPushError}, - {roachpb.PENDING, 0, m*ns + 1, roachpb.PUSH_TIMESTAMP, noError}, - {roachpb.PENDING, 0, m*ns + 1, roachpb.PUSH_ABORT, noError}, - {roachpb.PENDING, 0, m*ns + 1, roachpb.PUSH_TOUCH, noError}, - {roachpb.PENDING, ns, m*ns + 1, roachpb.PUSH_TIMESTAMP, txnPushError}, - {roachpb.PENDING, ns, m*ns + 1, roachpb.PUSH_ABORT, txnPushError}, - {roachpb.PENDING, ns, m*ns + 1, roachpb.PUSH_TOUCH, txnPushError}, - {roachpb.PENDING, ns, (m+1)*ns - 1, roachpb.PUSH_TIMESTAMP, txnPushError}, - {roachpb.PENDING, ns, (m+1)*ns - 1, roachpb.PUSH_ABORT, txnPushError}, - {roachpb.PENDING, ns, (m+1)*ns - 1, roachpb.PUSH_TOUCH, txnPushError}, - {roachpb.PENDING, ns, (m+1)*ns + 1, roachpb.PUSH_TIMESTAMP, noError}, - {roachpb.PENDING, ns, (m+1)*ns + 1, roachpb.PUSH_ABORT, noError}, - {roachpb.PENDING, ns, (m+1)*ns + 1, roachpb.PUSH_TOUCH, noError}, + {roachpb.PENDING, 0, 1, kvpb.PUSH_TIMESTAMP, txnPushError}, + {roachpb.PENDING, 0, 1, kvpb.PUSH_ABORT, txnPushError}, + {roachpb.PENDING, 0, 1, kvpb.PUSH_TOUCH, txnPushError}, + {roachpb.PENDING, 0, ns, kvpb.PUSH_TIMESTAMP, txnPushError}, + {roachpb.PENDING, 0, ns, kvpb.PUSH_ABORT, txnPushError}, + {roachpb.PENDING, 0, ns, kvpb.PUSH_TOUCH, txnPushError}, + {roachpb.PENDING, 0, m*ns - 1, kvpb.PUSH_TIMESTAMP, txnPushError}, + {roachpb.PENDING, 0, m*ns - 1, kvpb.PUSH_ABORT, txnPushError}, + {roachpb.PENDING, 0, m*ns - 1, kvpb.PUSH_TOUCH, txnPushError}, + {roachpb.PENDING, 0, m*ns + 1, kvpb.PUSH_TIMESTAMP, noError}, + {roachpb.PENDING, 0, m*ns + 1, kvpb.PUSH_ABORT, noError}, + {roachpb.PENDING, 0, m*ns + 1, kvpb.PUSH_TOUCH, noError}, + {roachpb.PENDING, ns, m*ns + 1, kvpb.PUSH_TIMESTAMP, txnPushError}, + {roachpb.PENDING, ns, m*ns + 1, kvpb.PUSH_ABORT, txnPushError}, + {roachpb.PENDING, ns, m*ns + 1, kvpb.PUSH_TOUCH, txnPushError}, + {roachpb.PENDING, ns, (m+1)*ns - 1, kvpb.PUSH_TIMESTAMP, txnPushError}, + {roachpb.PENDING, ns, (m+1)*ns - 1, kvpb.PUSH_ABORT, txnPushError}, + {roachpb.PENDING, ns, (m+1)*ns - 1, kvpb.PUSH_TOUCH, txnPushError}, + {roachpb.PENDING, ns, (m+1)*ns + 1, kvpb.PUSH_TIMESTAMP, noError}, + {roachpb.PENDING, ns, (m+1)*ns + 1, kvpb.PUSH_ABORT, noError}, + {roachpb.PENDING, ns, (m+1)*ns + 1, kvpb.PUSH_TOUCH, noError}, // If the transaction record is STAGING then any case that previously // returned a TransactionPushError will continue to return that error, // but any case that previously succeeded in pushing the transaction // will now return an IndeterminateCommitError. - {roachpb.STAGING, 0, 1, roachpb.PUSH_TIMESTAMP, txnPushError}, - {roachpb.STAGING, 0, 1, roachpb.PUSH_ABORT, txnPushError}, - {roachpb.STAGING, 0, 1, roachpb.PUSH_TOUCH, txnPushError}, - {roachpb.STAGING, 0, ns, roachpb.PUSH_TIMESTAMP, txnPushError}, - {roachpb.STAGING, 0, ns, roachpb.PUSH_ABORT, txnPushError}, - {roachpb.STAGING, 0, ns, roachpb.PUSH_TOUCH, txnPushError}, - {roachpb.STAGING, 0, m*ns - 1, roachpb.PUSH_TIMESTAMP, txnPushError}, - {roachpb.STAGING, 0, m*ns - 1, roachpb.PUSH_ABORT, txnPushError}, - {roachpb.STAGING, 0, m*ns - 1, roachpb.PUSH_TOUCH, txnPushError}, - {roachpb.STAGING, 0, m*ns + 1, roachpb.PUSH_TIMESTAMP, indetCommitError}, - {roachpb.STAGING, 0, m*ns + 1, roachpb.PUSH_ABORT, indetCommitError}, - {roachpb.STAGING, 0, m*ns + 1, roachpb.PUSH_TOUCH, indetCommitError}, - {roachpb.STAGING, ns, m*ns + 1, roachpb.PUSH_TIMESTAMP, txnPushError}, - {roachpb.STAGING, ns, m*ns + 1, roachpb.PUSH_ABORT, txnPushError}, - {roachpb.STAGING, ns, m*ns + 1, roachpb.PUSH_TOUCH, txnPushError}, - {roachpb.STAGING, ns, (m+1)*ns - 1, roachpb.PUSH_TIMESTAMP, txnPushError}, - {roachpb.STAGING, ns, (m+1)*ns - 1, roachpb.PUSH_ABORT, txnPushError}, - {roachpb.STAGING, ns, (m+1)*ns - 1, roachpb.PUSH_TOUCH, txnPushError}, - {roachpb.STAGING, ns, (m+1)*ns + 1, roachpb.PUSH_TIMESTAMP, indetCommitError}, - {roachpb.STAGING, ns, (m+1)*ns + 1, roachpb.PUSH_ABORT, indetCommitError}, - {roachpb.STAGING, ns, (m+1)*ns + 1, roachpb.PUSH_TOUCH, indetCommitError}, + {roachpb.STAGING, 0, 1, kvpb.PUSH_TIMESTAMP, txnPushError}, + {roachpb.STAGING, 0, 1, kvpb.PUSH_ABORT, txnPushError}, + {roachpb.STAGING, 0, 1, kvpb.PUSH_TOUCH, txnPushError}, + {roachpb.STAGING, 0, ns, kvpb.PUSH_TIMESTAMP, txnPushError}, + {roachpb.STAGING, 0, ns, kvpb.PUSH_ABORT, txnPushError}, + {roachpb.STAGING, 0, ns, kvpb.PUSH_TOUCH, txnPushError}, + {roachpb.STAGING, 0, m*ns - 1, kvpb.PUSH_TIMESTAMP, txnPushError}, + {roachpb.STAGING, 0, m*ns - 1, kvpb.PUSH_ABORT, txnPushError}, + {roachpb.STAGING, 0, m*ns - 1, kvpb.PUSH_TOUCH, txnPushError}, + {roachpb.STAGING, 0, m*ns + 1, kvpb.PUSH_TIMESTAMP, indetCommitError}, + {roachpb.STAGING, 0, m*ns + 1, kvpb.PUSH_ABORT, indetCommitError}, + {roachpb.STAGING, 0, m*ns + 1, kvpb.PUSH_TOUCH, indetCommitError}, + {roachpb.STAGING, ns, m*ns + 1, kvpb.PUSH_TIMESTAMP, txnPushError}, + {roachpb.STAGING, ns, m*ns + 1, kvpb.PUSH_ABORT, txnPushError}, + {roachpb.STAGING, ns, m*ns + 1, kvpb.PUSH_TOUCH, txnPushError}, + {roachpb.STAGING, ns, (m+1)*ns - 1, kvpb.PUSH_TIMESTAMP, txnPushError}, + {roachpb.STAGING, ns, (m+1)*ns - 1, kvpb.PUSH_ABORT, txnPushError}, + {roachpb.STAGING, ns, (m+1)*ns - 1, kvpb.PUSH_TOUCH, txnPushError}, + {roachpb.STAGING, ns, (m+1)*ns + 1, kvpb.PUSH_TIMESTAMP, indetCommitError}, + {roachpb.STAGING, ns, (m+1)*ns + 1, kvpb.PUSH_ABORT, indetCommitError}, + {roachpb.STAGING, ns, (m+1)*ns + 1, kvpb.PUSH_TOUCH, indetCommitError}, // Even when a transaction record doesn't exist, if the timestamp // from the PushTxn request indicates sufficiently recent client // activity, the push will fail. - {-1, 0, 1, roachpb.PUSH_TIMESTAMP, txnPushError}, - {-1, 0, 1, roachpb.PUSH_ABORT, txnPushError}, - {-1, 0, 1, roachpb.PUSH_TOUCH, txnPushError}, - {-1, 0, ns, roachpb.PUSH_TIMESTAMP, txnPushError}, - {-1, 0, ns, roachpb.PUSH_ABORT, txnPushError}, - {-1, 0, ns, roachpb.PUSH_TOUCH, txnPushError}, - {-1, 0, m*ns - 1, roachpb.PUSH_TIMESTAMP, txnPushError}, - {-1, 0, m*ns - 1, roachpb.PUSH_ABORT, txnPushError}, - {-1, 0, m*ns - 1, roachpb.PUSH_TOUCH, txnPushError}, - {-1, 0, m*ns + 1, roachpb.PUSH_TIMESTAMP, noError}, - {-1, 0, m*ns + 1, roachpb.PUSH_ABORT, noError}, - {-1, 0, m*ns + 1, roachpb.PUSH_TOUCH, noError}, + {-1, 0, 1, kvpb.PUSH_TIMESTAMP, txnPushError}, + {-1, 0, 1, kvpb.PUSH_ABORT, txnPushError}, + {-1, 0, 1, kvpb.PUSH_TOUCH, txnPushError}, + {-1, 0, ns, kvpb.PUSH_TIMESTAMP, txnPushError}, + {-1, 0, ns, kvpb.PUSH_ABORT, txnPushError}, + {-1, 0, ns, kvpb.PUSH_TOUCH, txnPushError}, + {-1, 0, m*ns - 1, kvpb.PUSH_TIMESTAMP, txnPushError}, + {-1, 0, m*ns - 1, kvpb.PUSH_ABORT, txnPushError}, + {-1, 0, m*ns - 1, kvpb.PUSH_TOUCH, txnPushError}, + {-1, 0, m*ns + 1, kvpb.PUSH_TIMESTAMP, noError}, + {-1, 0, m*ns + 1, kvpb.PUSH_ABORT, noError}, + {-1, 0, m*ns + 1, kvpb.PUSH_TOUCH, noError}, } for i, test := range testCases { @@ -5776,7 +5775,7 @@ func TestPushTxnHeartbeatTimeout(t *testing.T) { // Now, attempt to push the transaction. args := pushTxnArgs(pusher, pushee, test.pushType) args.PushTo = pushee.ReadTimestamp.Add(0, 1) - h := roachpb.Header{Timestamp: args.PushTo} + h := kvpb.Header{Timestamp: args.PushTo} // Set the manual clock to the txn start time + offset. This is the time // source used to detect transaction expiration. We make sure to set it @@ -5789,7 +5788,7 @@ func TestPushTxnHeartbeatTimeout(t *testing.T) { t.Fatalf("%d: expected error %q; got %v, args=%+v, reply=%+v", i, test.expErr, pErr, args, reply) } if reply != nil { - if txn := reply.(*roachpb.PushTxnResponse).PusheeTxn; txn.Status != roachpb.ABORTED { + if txn := reply.(*kvpb.PushTxnResponse).PusheeTxn; txn.Status != roachpb.ABORTED { t.Errorf("%d: expected aborted transaction, got %s", i, txn) } } @@ -5813,28 +5812,28 @@ func TestResolveIntentPushTxnReplyTxn(t *testing.T) { txn := newTransaction("test", roachpb.Key("test"), 1, tc.Clock()) txnPushee := txn.Clone() - pa := pushTxnArgs(txn, txnPushee, roachpb.PUSH_ABORT) + pa := pushTxnArgs(txn, txnPushee, kvpb.PUSH_ABORT) pa.Force = true var ms enginepb.MVCCStats - var ra roachpb.ResolveIntentRequest - var rra roachpb.ResolveIntentRangeRequest + var ra kvpb.ResolveIntentRequest + var rra kvpb.ResolveIntentRangeRequest - h := roachpb.Header{Txn: txn, Timestamp: tc.Clock().Now()} + h := kvpb.Header{Txn: txn, Timestamp: tc.Clock().Now()} // Should not be able to push or resolve in a transaction. - if _, err := batcheval.PushTxn(ctx, b, batcheval.CommandArgs{Stats: &ms, Header: h, Args: &pa}, &roachpb.PushTxnResponse{}); !testutils.IsError(err, batcheval.ErrTransactionUnsupported.Error()) { + if _, err := batcheval.PushTxn(ctx, b, batcheval.CommandArgs{Stats: &ms, Header: h, Args: &pa}, &kvpb.PushTxnResponse{}); !testutils.IsError(err, batcheval.ErrTransactionUnsupported.Error()) { t.Fatalf("transactional PushTxn returned unexpected error: %+v", err) } - if _, err := batcheval.ResolveIntent(ctx, b, batcheval.CommandArgs{Stats: &ms, Header: h, Args: &ra}, &roachpb.ResolveIntentResponse{}); !testutils.IsError(err, batcheval.ErrTransactionUnsupported.Error()) { + if _, err := batcheval.ResolveIntent(ctx, b, batcheval.CommandArgs{Stats: &ms, Header: h, Args: &ra}, &kvpb.ResolveIntentResponse{}); !testutils.IsError(err, batcheval.ErrTransactionUnsupported.Error()) { t.Fatalf("transactional ResolveIntent returned unexpected error: %+v", err) } - if _, err := batcheval.ResolveIntentRange(ctx, b, batcheval.CommandArgs{Stats: &ms, Header: h, Args: &rra}, &roachpb.ResolveIntentRangeResponse{}); !testutils.IsError(err, batcheval.ErrTransactionUnsupported.Error()) { + if _, err := batcheval.ResolveIntentRange(ctx, b, batcheval.CommandArgs{Stats: &ms, Header: h, Args: &rra}, &kvpb.ResolveIntentRangeResponse{}); !testutils.IsError(err, batcheval.ErrTransactionUnsupported.Error()) { t.Fatalf("transactional ResolveIntentRange returned unexpected error: %+v", err) } // Should not get a transaction back from PushTxn. It used to erroneously // return args.PusherTxn. - h = roachpb.Header{Timestamp: tc.Clock().Now()} - var reply roachpb.PushTxnResponse + h = kvpb.Header{Timestamp: tc.Clock().Now()} + var reply kvpb.PushTxnResponse ec := newEvalContextImpl( ctx, tc.repl, @@ -5875,23 +5874,23 @@ func TestPushTxnPriorities(t *testing.T) { testCases := []struct { pusherPriority, pusheePriority enginepb.TxnPriority pusherTS, pusheeTS hlc.Timestamp - pushType roachpb.PushTxnType + pushType kvpb.PushTxnType expSuccess bool }{ // Pusher with higher priority succeeds. - {enginepb.MaxTxnPriority, enginepb.MinTxnPriority, ts1, ts1, roachpb.PUSH_TIMESTAMP, true}, - {enginepb.MaxTxnPriority, enginepb.MinTxnPriority, ts1, ts1, roachpb.PUSH_ABORT, true}, + {enginepb.MaxTxnPriority, enginepb.MinTxnPriority, ts1, ts1, kvpb.PUSH_TIMESTAMP, true}, + {enginepb.MaxTxnPriority, enginepb.MinTxnPriority, ts1, ts1, kvpb.PUSH_ABORT, true}, // Pusher with lower priority fails. - {enginepb.MinTxnPriority, enginepb.MaxTxnPriority, ts1, ts1, roachpb.PUSH_ABORT, false}, - {enginepb.MinTxnPriority, enginepb.MaxTxnPriority, ts1, ts1, roachpb.PUSH_TIMESTAMP, false}, + {enginepb.MinTxnPriority, enginepb.MaxTxnPriority, ts1, ts1, kvpb.PUSH_ABORT, false}, + {enginepb.MinTxnPriority, enginepb.MaxTxnPriority, ts1, ts1, kvpb.PUSH_TIMESTAMP, false}, // Pusher with lower priority fails, even with older txn timestamp. - {enginepb.MinTxnPriority, enginepb.MaxTxnPriority, ts1, ts2, roachpb.PUSH_ABORT, false}, + {enginepb.MinTxnPriority, enginepb.MaxTxnPriority, ts1, ts2, kvpb.PUSH_ABORT, false}, // Pusher has lower priority, but older txn timestamp allows success if // !abort since there's nothing to do. - {enginepb.MinTxnPriority, enginepb.MaxTxnPriority, ts1, ts2, roachpb.PUSH_TIMESTAMP, true}, + {enginepb.MinTxnPriority, enginepb.MaxTxnPriority, ts1, ts2, kvpb.PUSH_TIMESTAMP, true}, // When touching, priority never wins. - {enginepb.MaxTxnPriority, enginepb.MinTxnPriority, ts1, ts1, roachpb.PUSH_TOUCH, false}, - {enginepb.MinTxnPriority, enginepb.MaxTxnPriority, ts1, ts1, roachpb.PUSH_TOUCH, false}, + {enginepb.MaxTxnPriority, enginepb.MinTxnPriority, ts1, ts1, kvpb.PUSH_TOUCH, false}, + {enginepb.MinTxnPriority, enginepb.MaxTxnPriority, ts1, ts1, kvpb.PUSH_TOUCH, false}, } for i, test := range testCases { @@ -5912,14 +5911,14 @@ func TestPushTxnPriorities(t *testing.T) { put := putArgs(key, key) assignSeqNumsForReqs(pushee, &put) - if _, pErr := kv.SendWrappedWith(context.Background(), tc.Sender(), roachpb.Header{Txn: pushee}, &put); pErr != nil { + if _, pErr := kv.SendWrappedWith(context.Background(), tc.Sender(), kvpb.Header{Txn: pushee}, &put); pErr != nil { t.Fatal(pErr) } // Now, attempt to push the transaction with intent epoch set appropriately. args := pushTxnArgs(pusher, pushee, test.pushType) // Set header timestamp to the maximum of the pusher and pushee timestamps. - h := roachpb.Header{Timestamp: args.PushTo} + h := kvpb.Header{Timestamp: args.PushTo} h.Timestamp.Forward(pushee.MinTimestamp) _, pErr := tc.SendWrappedWith(h, &args) @@ -5927,7 +5926,7 @@ func TestPushTxnPriorities(t *testing.T) { t.Errorf("expected success on trial %d? %t; got err %s", i, test.expSuccess, pErr) } if pErr != nil { - if _, ok := pErr.GetDetail().(*roachpb.TransactionPushError); !ok { + if _, ok := pErr.GetDetail().(*kvpb.TransactionPushError); !ok { t.Errorf("expected txn push error: %s", pErr) } } @@ -5958,20 +5957,20 @@ func TestPushTxnPushTimestamp(t *testing.T) { key := roachpb.Key("a") put := putArgs(key, key) assignSeqNumsForReqs(pushee, &put) - if _, pErr := kv.SendWrappedWith(ctx, tc.Sender(), roachpb.Header{Txn: pushee}, &put); pErr != nil { + if _, pErr := kv.SendWrappedWith(ctx, tc.Sender(), kvpb.Header{Txn: pushee}, &put); pErr != nil { t.Fatal(pErr) } // Now, push the transaction using a PUSH_TIMESTAMP push request. - args := pushTxnArgs(pusher, pushee, roachpb.PUSH_TIMESTAMP) + args := pushTxnArgs(pusher, pushee, kvpb.PUSH_TIMESTAMP) - resp, pErr := tc.SendWrappedWith(roachpb.Header{Timestamp: args.PushTo}, &args) + resp, pErr := tc.SendWrappedWith(kvpb.Header{Timestamp: args.PushTo}, &args) if pErr != nil { t.Fatalf("unexpected error on push: %s", pErr) } expTS := pusher.WriteTimestamp expTS.Logical++ - reply := resp.(*roachpb.PushTxnResponse) + reply := resp.(*kvpb.PushTxnResponse) if reply.PusheeTxn.WriteTimestamp != expTS { t.Errorf("expected timestamp to be pushed to %+v; got %+v", expTS, reply.PusheeTxn.WriteTimestamp) } @@ -6003,18 +6002,18 @@ func TestPushTxnPushTimestampAlreadyPushed(t *testing.T) { key := roachpb.Key("a") put := putArgs(key, key) assignSeqNumsForReqs(pushee, &put) - if _, pErr := kv.SendWrappedWith(ctx, tc.Sender(), roachpb.Header{Txn: pushee}, &put); pErr != nil { + if _, pErr := kv.SendWrappedWith(ctx, tc.Sender(), kvpb.Header{Txn: pushee}, &put); pErr != nil { t.Fatal(pErr) } // Now, push the transaction using a PUSH_TIMESTAMP push request. - args := pushTxnArgs(pusher, pushee, roachpb.PUSH_TIMESTAMP) + args := pushTxnArgs(pusher, pushee, kvpb.PUSH_TIMESTAMP) - resp, pErr := tc.SendWrappedWith(roachpb.Header{Timestamp: args.PushTo}, &args) + resp, pErr := tc.SendWrappedWith(kvpb.Header{Timestamp: args.PushTo}, &args) if pErr != nil { t.Fatalf("unexpected pError on push: %s", pErr) } - reply := resp.(*roachpb.PushTxnResponse) + reply := resp.(*kvpb.PushTxnResponse) if reply.PusheeTxn.WriteTimestamp != pushee.WriteTimestamp { t.Errorf("expected timestamp to be equal to original %+v; got %+v", pushee.WriteTimestamp, reply.PusheeTxn.WriteTimestamp) } @@ -6054,7 +6053,7 @@ func TestPushTxnSerializableRestart(t *testing.T) { // Write to a key. put := putArgs(key, []byte("foo")) assignSeqNumsForReqs(pushee, &put) - resp, pErr := kv.SendWrappedWith(ctx, tc.Sender(), roachpb.Header{Txn: pushee}, &put) + resp, pErr := kv.SendWrappedWith(ctx, tc.Sender(), kvpb.Header{Txn: pushee}, &put) if pErr != nil { t.Fatal(pErr) } @@ -6064,7 +6063,7 @@ func TestPushTxnSerializableRestart(t *testing.T) { etArgs, h := endTxnArgs(pushee, true /* commit */) assignSeqNumsForReqs(pushee, &etArgs) _, pErr = tc.SendWrappedWith(h, &etArgs) - if _, ok := pErr.GetDetail().(*roachpb.TransactionRetryError); !ok { + if _, ok := pErr.GetDetail().(*kvpb.TransactionRetryError); !ok { t.Fatalf("expected retry error; got %s", pErr) } pusheeCopy := *pushee @@ -6072,20 +6071,20 @@ func TestPushTxnSerializableRestart(t *testing.T) { // Next push pushee to advance timestamp of txn record. pusher.WriteTimestamp = tc.repl.store.Clock().Now() - args := pushTxnArgs(pusher, &pusheeCopy, roachpb.PUSH_TIMESTAMP) + args := pushTxnArgs(pusher, &pusheeCopy, kvpb.PUSH_TIMESTAMP) if _, pErr := tc.SendWrapped(&args); pErr != nil { t.Fatal(pErr) } // Try to end pushed transaction at restart timestamp, which is // earlier than its now-pushed timestamp. Should fail. - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Add(&put) ba.Add(&etArgs) ba.Header.Txn = pushee assignSeqNumsForReqs(pushee, &put, &etArgs) _, pErr = tc.Sender().Send(ctx, ba) - if _, ok := pErr.GetDetail().(*roachpb.TransactionRetryError); !ok { + if _, ok := pErr.GetDetail().(*kvpb.TransactionRetryError); !ok { t.Fatalf("expected retry error; got %s", pErr) } // Verify that the returned transaction has timestamp equal to the pushed @@ -6115,7 +6114,7 @@ func TestQueryIntentRequest(t *testing.T) { pArgs := putArgs(key1, []byte("value1")) assignSeqNumsForReqs(txn, &pArgs) - if _, pErr := tc.SendWrappedWith(roachpb.Header{Txn: txn}, &pArgs); pErr != nil { + if _, pErr := tc.SendWrappedWith(kvpb.Header{Txn: txn}, &pArgs); pErr != nil { t.Fatal(pErr) } @@ -6126,7 +6125,7 @@ func TestQueryIntentRequest(t *testing.T) { expectIntent bool, ) { t.Helper() - var h roachpb.Header + var h kvpb.Header if baTxn != nil { h.Txn = baTxn } else { @@ -6137,11 +6136,11 @@ func TestQueryIntentRequest(t *testing.T) { if errIfMissing && !expectIntent { ownIntent := baTxn != nil if ownIntent && txnMeta.WriteTimestamp.Less(txn.WriteTimestamp) { - if _, ok := pErr.GetDetail().(*roachpb.TransactionRetryError); !ok { + if _, ok := pErr.GetDetail().(*kvpb.TransactionRetryError); !ok { t.Fatalf("expected TransactionRetryError, found %v %v", txnMeta, pErr) } } else { - if _, ok := pErr.GetDetail().(*roachpb.IntentMissingError); !ok { + if _, ok := pErr.GetDetail().(*kvpb.IntentMissingError); !ok { t.Fatalf("expected IntentMissingError, found %v", pErr) } } @@ -6149,7 +6148,7 @@ func TestQueryIntentRequest(t *testing.T) { if pErr != nil { t.Fatal(pErr) } - if e, a := expectIntent, qiRes.(*roachpb.QueryIntentResponse).FoundIntent; e != a { + if e, a := expectIntent, qiRes.(*kvpb.QueryIntentResponse).FoundIntent; e != a { t.Fatalf("expected FoundIntent=%t but FoundIntent=%t", e, a) } } @@ -6213,8 +6212,8 @@ func TestQueryIntentRequest(t *testing.T) { txnCopy := *txn pArgs2 := putArgs(keyPrevent, []byte("value2")) assignSeqNumsForReqs(&txnCopy, &pArgs2) - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txnCopy} + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txnCopy} ba.Add(&pArgs2) br, pErr := tc.Sender().Send(context.Background(), ba) if pErr != nil { @@ -6244,14 +6243,14 @@ func TestReplicaResolveIntentRange(t *testing.T) { for _, key := range keys { pArgs := putArgs(key, []byte("value1")) assignSeqNumsForReqs(txn, &pArgs) - if _, pErr := tc.SendWrappedWith(roachpb.Header{Txn: txn}, &pArgs); pErr != nil { + if _, pErr := tc.SendWrappedWith(kvpb.Header{Txn: txn}, &pArgs); pErr != nil { t.Fatal(pErr) } } // Resolve the intents. - rArgs := &roachpb.ResolveIntentRangeRequest{ - RequestHeader: roachpb.RequestHeader{ + rArgs := &kvpb.ResolveIntentRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: roachpb.Key("a"), EndKey: roachpb.Key("c"), }, @@ -6269,7 +6268,7 @@ func TestReplicaResolveIntentRange(t *testing.T) { if pErr != nil { t.Fatalf("unexpected error on scan: %s", pErr) } - sReply := reply.(*roachpb.ScanResponse) + sReply := reply.(*kvpb.ScanResponse) if len(sReply.Rows) != 2 { t.Errorf("expected 2 rows; got %v", sReply.Rows) } @@ -6346,7 +6345,7 @@ func TestRangeStatsComputation(t *testing.T) { txn.ID = uuid assignSeqNumsForReqs(txn, &pArgs) - if _, pErr := tc.SendWrappedWith(roachpb.Header{Txn: txn}, &pArgs); pErr != nil { + if _, pErr := tc.SendWrappedWith(kvpb.Header{Txn: txn}, &pArgs); pErr != nil { t.Fatal(pErr) } expMS = baseStats @@ -6366,8 +6365,8 @@ func TestRangeStatsComputation(t *testing.T) { } // Resolve the 2nd value. - rArgs := &roachpb.ResolveIntentRequest{ - RequestHeader: roachpb.RequestHeader{ + rArgs := &kvpb.ResolveIntentRequest{ + RequestHeader: kvpb.RequestHeader{ Key: pArgs.Key, }, IntentTxn: txn.TxnMeta, @@ -6462,7 +6461,7 @@ func TestMerge(t *testing.T) { if pErr != nil { t.Fatalf("unexpected error from Get: %s", pErr) } - resp := reply.(*roachpb.GetResponse) + resp := reply.(*kvpb.GetResponse) if resp.Value == nil { t.Fatal("GetResponse had nil value") } @@ -6495,8 +6494,8 @@ func TestConditionFailedError(t *testing.T) { } cpArgs := cPutArgs(key, value, []byte("moo")) - _, pErr := tc.SendWrappedWith(roachpb.Header{Timestamp: hlc.MinTimestamp}, &cpArgs) - if cErr, ok := pErr.GetDetail().(*roachpb.ConditionFailedError); pErr == nil || !ok { + _, pErr := tc.SendWrappedWith(kvpb.Header{Timestamp: hlc.MinTimestamp}, &cpArgs) + if cErr, ok := pErr.GetDetail().(*kvpb.ConditionFailedError); pErr == nil || !ok { t.Fatalf("expected ConditionFailedError, got %T with content %+v", pErr, pErr) } else if valueBytes, err := cErr.ActualValue.GetBytes(); err != nil { t.Fatal(err) @@ -6555,7 +6554,7 @@ func TestAppliedIndex(t *testing.T) { if pErr != nil { t.Fatal(pErr) } - reply := resp.(*roachpb.IncrementResponse) + reply := resp.(*kvpb.IncrementResponse) sum += i if reply.NewValue != sum { @@ -6586,9 +6585,9 @@ func TestReplicaCorruption(t *testing.T) { tsc := TestStoreConfig(nil) tsc.TestingKnobs.EvalKnobs.TestingEvalFilter = - func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { + func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { if filterArgs.Req.Header().Key.Equal(roachpb.Key("boom")) { - return roachpb.NewError(roachpb.NewReplicaCorruptionError(errors.New("boom"))) + return kvpb.NewError(kvpb.NewReplicaCorruptionError(errors.New("boom"))) } return nil } @@ -6648,7 +6647,7 @@ func TestChangeReplicasDuplicateError(t *testing.T) { if _, err := tc.addBogusReplicaToRangeDesc(context.Background()); err != nil { t.Fatalf("Unexpected error %v", err) } - chgs := roachpb.MakeReplicationChanges(typ, roachpb.ReplicationTarget{ + chgs := kvpb.MakeReplicationChanges(typ, roachpb.ReplicationTarget{ NodeID: tc.store.Ident.NodeID, StoreID: 9999, }) @@ -6692,7 +6691,7 @@ func TestReplicaDanglingMetaIntent(t *testing.T) { key := roachpb.Key("a") // Get original meta2 descriptor. - rs, _, err := kv.RangeLookup(ctx, tc.Sender(), key, roachpb.READ_UNCOMMITTED, 0, reverse) + rs, _, err := kv.RangeLookup(ctx, tc.Sender(), key, kvpb.READ_UNCOMMITTED, 0, reverse) if err != nil { t.Fatal(err) } @@ -6717,7 +6716,7 @@ func TestReplicaDanglingMetaIntent(t *testing.T) { // priority). pArgs := putArgs(keys.RangeMetaKey(roachpb.RKey(key)).AsRawKey(), data) assignSeqNumsForReqs(txn, &pArgs) - if _, pErr := kv.SendWrappedWith(ctx, tc.Sender(), roachpb.Header{Txn: txn}, &pArgs); pErr != nil { + if _, pErr := kv.SendWrappedWith(ctx, tc.Sender(), kvpb.Header{Txn: txn}, &pArgs); pErr != nil { t.Fatal(pErr) } @@ -6727,7 +6726,7 @@ func TestReplicaDanglingMetaIntent(t *testing.T) { // // Note that 'A' < 'a'. newKey := roachpb.Key{'A'} - rs, _, err = kv.RangeLookup(ctx, tc.Sender(), newKey, roachpb.READ_UNCOMMITTED, 0, reverse) + rs, _, err = kv.RangeLookup(ctx, tc.Sender(), newKey, kvpb.READ_UNCOMMITTED, 0, reverse) if err != nil { t.Fatal(err) } @@ -6742,8 +6741,8 @@ func TestReplicaDanglingMetaIntent(t *testing.T) { } // Switch to consistent lookups, which should run into the intent. - _, _, err = kv.RangeLookup(ctx, tc.Sender(), newKey, roachpb.CONSISTENT, 0, reverse) - if !errors.HasType(err, (*roachpb.WriteIntentError)(nil)) { + _, _, err = kv.RangeLookup(ctx, tc.Sender(), newKey, kvpb.CONSISTENT, 0, reverse) + if !errors.HasType(err, (*kvpb.WriteIntentError)(nil)) { t.Fatalf("expected WriteIntentError, not %s", err) } }) @@ -6801,14 +6800,14 @@ func TestReplicaLookupUseReverseScan(t *testing.T) { pArgs := putArgs(keys.RangeMetaKey(r.EndKey).AsRawKey(), data) assignSeqNumsForReqs(txn, &pArgs) - if _, pErr := tc.SendWrappedWith(roachpb.Header{Txn: txn}, &pArgs); pErr != nil { + if _, pErr := tc.SendWrappedWith(kvpb.Header{Txn: txn}, &pArgs); pErr != nil { t.Fatal(pErr) } } // Resolve the intents. - rArgs := &roachpb.ResolveIntentRangeRequest{ - RequestHeader: roachpb.RequestHeader{ + rArgs := &kvpb.ResolveIntentRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: keys.RangeMetaKey(roachpb.RKey("a")).AsRawKey(), EndKey: keys.RangeMetaKey(roachpb.RKey("z")).AsRawKey(), }, @@ -6824,7 +6823,7 @@ func TestReplicaLookupUseReverseScan(t *testing.T) { // Test reverse RangeLookup scan without intents. for _, c := range testCases { rs, _, err := kv.RangeLookup(ctx, tc.Sender(), roachpb.Key(c.key), - roachpb.READ_UNCOMMITTED, 0, true) + kvpb.READ_UNCOMMITTED, 0, true) if err != nil { t.Fatal(err) } @@ -6845,7 +6844,7 @@ func TestReplicaLookupUseReverseScan(t *testing.T) { pArgs := putArgs(keys.RangeMetaKey(r.EndKey).AsRawKey(), data) assignSeqNumsForReqs(txn, &pArgs) - if _, pErr := tc.SendWrappedWith(roachpb.Header{Txn: txn}, &pArgs); pErr != nil { + if _, pErr := tc.SendWrappedWith(kvpb.Header{Txn: txn}, &pArgs); pErr != nil { t.Fatal(pErr) } } @@ -6853,7 +6852,7 @@ func TestReplicaLookupUseReverseScan(t *testing.T) { // Test reverse RangeLookup scan with intents. for _, c := range testCases { rs, _, err := kv.RangeLookup(ctx, tc.Sender(), roachpb.Key(c.key), - roachpb.READ_UNCOMMITTED, 0, true) + kvpb.READ_UNCOMMITTED, 0, true) if err != nil { t.Fatal(err) } @@ -6894,7 +6893,7 @@ func TestRangeLookup(t *testing.T) { for i, c := range testCases { rs, _, err := kv.RangeLookup(ctx, tc.Sender(), c.key.AsRawKey(), - roachpb.CONSISTENT, 0, c.reverse) + kvpb.CONSISTENT, 0, c.reverse) if err != nil { if c.expected != nil { t.Fatal(err) @@ -6916,11 +6915,11 @@ func TestRequestLeaderEncounterGroupDeleteError(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(ctx) - // Mock propose to return a roachpb.RaftGroupDeletedError. + // Mock propose to return a kvpb.RaftGroupDeletedError. var active int32 - proposeFn := func(fArgs kvserverbase.ProposalFilterArgs) *roachpb.Error { + proposeFn := func(fArgs kvserverbase.ProposalFilterArgs) *kvpb.Error { if atomic.LoadInt32(&active) == 1 { - return roachpb.NewError(&roachpb.RaftGroupDeletedError{}) + return kvpb.NewError(&kvpb.RaftGroupDeletedError{}) } return nil } @@ -6935,11 +6934,11 @@ func TestRequestLeaderEncounterGroupDeleteError(t *testing.T) { gArgs := getArgs(roachpb.Key("a")) // Force the read command request a new lease. manual.MustAdvanceTo(leaseExpiry(tc.repl)) - _, pErr := kv.SendWrappedWith(ctx, tc.store, roachpb.Header{ + _, pErr := kv.SendWrappedWith(ctx, tc.store, kvpb.Header{ Timestamp: tc.Clock().Now(), RangeID: 1, }, &gArgs) - if _, ok := pErr.GetDetail().(*roachpb.RangeNotFoundError); !ok { + if _, ok := pErr.GetDetail().(*kvpb.RangeNotFoundError); !ok { t.Fatalf("expected a RangeNotFoundError, get %s", pErr) } } @@ -7022,22 +7021,22 @@ func TestBatchErrorWithIndex(t *testing.T) { defer stopper.Stop(ctx) tc.Start(ctx, t, stopper) - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} // This one succeeds. - ba.Add(&roachpb.PutRequest{ - RequestHeader: roachpb.RequestHeader{Key: roachpb.Key("k")}, + ba.Add(&kvpb.PutRequest{ + RequestHeader: kvpb.RequestHeader{Key: roachpb.Key("k")}, Value: roachpb.MakeValueFromString("not nil"), }) // This one fails with a ConditionalPutError, which will populate the // returned error's index. - ba.Add(&roachpb.ConditionalPutRequest{ - RequestHeader: roachpb.RequestHeader{Key: roachpb.Key("k")}, + ba.Add(&kvpb.ConditionalPutRequest{ + RequestHeader: kvpb.RequestHeader{Key: roachpb.Key("k")}, Value: roachpb.MakeValueFromString("irrelevant"), ExpBytes: nil, // not true after above Put }) // This one is never executed. - ba.Add(&roachpb.GetRequest{ - RequestHeader: roachpb.RequestHeader{Key: roachpb.Key("k")}, + ba.Add(&kvpb.GetRequest{ + RequestHeader: kvpb.RequestHeader{Key: roachpb.Key("k")}, }) if _, pErr := tc.Sender().Send(ctx, ba); pErr == nil { @@ -7105,10 +7104,10 @@ func TestQuotaPoolReleasedOnFailedProposal(t *testing.T) { propErr := errors.New("proposal error") tsc := TestStoreConfig(nil /* clock */) - tsc.TestingKnobs.TestingProposalFilter = func(args kvserverbase.ProposalFilterArgs) *roachpb.Error { + tsc.TestingKnobs.TestingProposalFilter = func(args kvserverbase.ProposalFilterArgs) *kvpb.Error { if v := args.Ctx.Value(magicKey{}); v != nil { minQuotaSize = tc.repl.mu.proposalQuota.ApproximateQuota() + args.QuotaAlloc.Acquired() - return roachpb.NewError(propErr) + return kvpb.NewError(propErr) } return nil } @@ -7121,7 +7120,7 @@ func TestQuotaPoolReleasedOnFailedProposal(t *testing.T) { t.Fatal(pErr) } - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} pArg := putArgs(roachpb.Key("a"), make([]byte, 1<<10)) ba.Add(&pArg) ctx = context.WithValue(ctx, magicKey{}, "foo") @@ -7211,7 +7210,7 @@ func TestEntries(t *testing.T) { if _, err := kv.SendWrappedWith( ctx, tc.Sender(), - roachpb.Header{RangeID: 1}, + kvpb.Header{RangeID: 1}, &truncateArgs, ); err != nil { t.Fatal(err) @@ -7362,7 +7361,7 @@ func TestTerm(t *testing.T) { // Discard the first half of the log. truncateArgs := truncateLogArgs(indexes[5], rangeID) - if _, pErr := tc.SendWrappedWith(roachpb.Header{RangeID: 1}, &truncateArgs); pErr != nil { + if _, pErr := tc.SendWrappedWith(kvpb.Header{RangeID: 1}, &truncateArgs); pErr != nil { t.Fatal(pErr) } waitForTruncationForTesting(t, repl, indexes[5], looselyCoupled) @@ -7438,8 +7437,8 @@ func TestGCIncorrectRange(t *testing.T) { now := tc.Clock().Now() ts1 := now.Add(1, 0) ts2 := now.Add(2, 0) - ts1Header := roachpb.Header{RangeID: repl2.RangeID, Timestamp: ts1} - ts2Header := roachpb.Header{RangeID: repl2.RangeID, Timestamp: ts2} + ts1Header := kvpb.Header{RangeID: repl2.RangeID, Timestamp: ts1} + ts2Header := kvpb.Header{RangeID: repl2.RangeID, Timestamp: ts2} if _, pErr := kv.SendWrappedWith(ctx, repl2, ts1Header, &putReq); pErr != nil { t.Errorf("unexpected pError on put key request: %s", pErr) } @@ -7455,7 +7454,7 @@ func TestGCIncorrectRange(t *testing.T) { if _, pErr := kv.SendWrappedWith( ctx, repl1, - roachpb.Header{RangeID: 1, Timestamp: tc.Clock().Now()}, + kvpb.Header{RangeID: 1, Timestamp: tc.Clock().Now()}, &gcReq, ); pErr != nil { t.Errorf("unexpected pError on garbage collection request to incorrect range: %s", pErr) @@ -7465,7 +7464,7 @@ func TestGCIncorrectRange(t *testing.T) { getReq := getArgs(key) if res, pErr := kv.SendWrappedWith(ctx, repl2, ts1Header, &getReq); pErr != nil { t.Errorf("unexpected pError on get request to correct range: %s", pErr) - } else if resVal := res.(*roachpb.GetResponse).Value; resVal == nil { + } else if resVal := res.(*kvpb.GetResponse).Value; resVal == nil { t.Errorf("expected value %s to exists after GC to incorrect range but before GC to correct range, found %v", val, resVal) } @@ -7474,7 +7473,7 @@ func TestGCIncorrectRange(t *testing.T) { if _, pErr := kv.SendWrappedWith( ctx, repl2, - roachpb.Header{RangeID: repl2.RangeID, Timestamp: tc.Clock().Now()}, + kvpb.Header{RangeID: repl2.RangeID, Timestamp: tc.Clock().Now()}, &gcReq, ); pErr != nil { t.Errorf("unexpected pError on garbage collection request to correct range: %s", pErr) @@ -7483,7 +7482,7 @@ func TestGCIncorrectRange(t *testing.T) { // Make sure the key no longer exists on range 2. if res, pErr := kv.SendWrappedWith(ctx, repl2, ts1Header, &getReq); pErr != nil { t.Errorf("unexpected pError on get request to correct range: %s", pErr) - } else if resVal := res.(*roachpb.GetResponse).Value; resVal != nil { + } else if resVal := res.(*kvpb.GetResponse).Value; resVal != nil { t.Errorf("expected value at key %s to no longer exist after GC to correct range, found value %v", key, resVal) } } @@ -7502,7 +7501,7 @@ func TestReplicaCancelRaft(t *testing.T) { cfg := TestStoreConfig(nil) if !cancelEarly { cfg.TestingKnobs.TestingProposalFilter = - func(args kvserverbase.ProposalFilterArgs) *roachpb.Error { + func(args kvserverbase.ProposalFilterArgs) *kvpb.Error { for _, union := range args.Req.Requests { if union.GetInner().Header().Key.Equal(key) { cancel() @@ -7519,10 +7518,10 @@ func TestReplicaCancelRaft(t *testing.T) { if cancelEarly { cancel() } - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.RangeID = 1 - ba.Add(&roachpb.GetRequest{ - RequestHeader: roachpb.RequestHeader{Key: key}, + ba.Add(&kvpb.GetRequest{ + RequestHeader: kvpb.RequestHeader{Key: key}, }) if err := ba.SetActiveTimestamp(tc.Clock()); err != nil { t.Fatal(err) @@ -7540,7 +7539,7 @@ func TestReplicaCancelRaft(t *testing.T) { return } detail := pErr.GetDetail() - if _, ok := detail.(*roachpb.AmbiguousResultError); !ok { + if _, ok := detail.(*kvpb.AmbiguousResultError); !ok { t.Fatalf("expected AmbiguousResultError error; got %s (%T)", detail, detail) } } @@ -7577,18 +7576,18 @@ func TestReplicaAbandonProposal(t *testing.T) { } tc.repl.mu.Unlock() - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.RangeID = 1 ba.Timestamp = tc.Clock().Now() - ba.Add(&roachpb.PutRequest{ - RequestHeader: roachpb.RequestHeader{Key: []byte("acdfg")}, + ba.Add(&kvpb.PutRequest{ + RequestHeader: kvpb.RequestHeader{Key: []byte("acdfg")}, }) _, _, pErr := tc.repl.executeBatchWithConcurrencyRetries(ctx, ba, (*Replica).executeWriteBatch) if pErr == nil { t.Fatal("expected failure, but found success") } detail := pErr.GetDetail() - if _, ok := detail.(*roachpb.AmbiguousResultError); !ok { + if _, ok := detail.(*kvpb.AmbiguousResultError); !ok { t.Fatalf("expected AmbiguousResultError error; got %s (%T)", detail, detail) } @@ -7616,12 +7615,12 @@ func TestNewReplicaCorruptionError(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) for i, tc := range []struct { - errStruct *roachpb.ReplicaCorruptionError + errStruct *kvpb.ReplicaCorruptionError expErr string }{ - {roachpb.NewReplicaCorruptionError(errors.New("")), "replica corruption (processed=false)"}, - {roachpb.NewReplicaCorruptionError(errors.New("foo")), "replica corruption (processed=false): foo"}, - {roachpb.NewReplicaCorruptionError(errors.Wrap(errors.New("bar"), "foo")), "replica corruption (processed=false): foo: bar"}, + {kvpb.NewReplicaCorruptionError(errors.New("")), "replica corruption (processed=false)"}, + {kvpb.NewReplicaCorruptionError(errors.New("foo")), "replica corruption (processed=false): foo"}, + {kvpb.NewReplicaCorruptionError(errors.Wrap(errors.New("bar"), "foo")), "replica corruption (processed=false): foo: bar"}, } { // This uses fmt.Sprint because that ends up calling Error() and is the // intended use. A previous version of this test called String() directly @@ -7687,7 +7686,7 @@ func TestReplicaRetryRaftProposal(t *testing.T) { pArg := putArgs(roachpb.Key("a"), []byte("asd")) { - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Add(&pArg) ba.Timestamp = tc.Clock().Now() if _, pErr := tc.Sender().Send(ctx, ba); pErr != nil { @@ -7706,7 +7705,7 @@ func TestReplicaRetryRaftProposal(t *testing.T) { log.Infof(ctx, "test begins") - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.RangeID = 1 ba.Timestamp = tc.Clock().Now() const expInc = 123 @@ -7741,8 +7740,8 @@ func TestReplicaRetryRaftProposal(t *testing.T) { now := tc.Clock().Now().UnsafeToClockTimestamp() lease.ProposedTS = &now - ba.Add(&roachpb.RequestLeaseRequest{ - RequestHeader: roachpb.RequestHeader{ + ba.Add(&kvpb.RequestLeaseRequest{ + RequestHeader: kvpb.RequestHeader{ Key: tc.repl.Desc().StartKey.AsRawKey(), }, Lease: lease, @@ -7791,10 +7790,10 @@ func TestReplicaCancelRaftCommandProgress(t *testing.T) { var chs []chan proposalResult const num = 10 for i := 0; i < num; i++ { - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Timestamp = tc.Clock().Now() - ba.Add(&roachpb.PutRequest{ - RequestHeader: roachpb.RequestHeader{ + ba.Add(&kvpb.PutRequest{ + RequestHeader: kvpb.RequestHeader{ Key: roachpb.Key(fmt.Sprintf("k%d", i)), }, }) @@ -7863,10 +7862,10 @@ func TestReplicaBurstPendingCommandsAndRepropose(t *testing.T) { const num = 10 chs := make([]chan proposalResult, 0, num) for i := 0; i < num; i++ { - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Timestamp = tc.Clock().Now() - ba.Add(&roachpb.PutRequest{ - RequestHeader: roachpb.RequestHeader{ + ba.Add(&kvpb.PutRequest{ + RequestHeader: kvpb.RequestHeader{ Key: roachpb.Key(fmt.Sprintf("k%d", i)), }, }) @@ -7982,9 +7981,9 @@ func TestReplicaRefreshPendingCommandsTicks(t *testing.T) { for i := 0; i < 2*electionTicks; i++ { // Add another pending command on each iteration. id := fmt.Sprintf("%08d", i) - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Timestamp = tc.Clock().Now() - ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: roachpb.Key(id)}}) + ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: roachpb.Key(id)}}) st := r.CurrentLeaseStatus(ctx) cmd, pErr := r.requestToProposal(ctx, kvserverbase.CmdIDKey(id), ba, allSpansGuard(), &st, uncertainty.Interval{}) if pErr != nil { @@ -8079,7 +8078,7 @@ func TestReplicaRefreshMultiple(t *testing.T) { const incCmdID = "deadbeef" var incApplyCount int64 tsc := TestStoreConfig(nil) - tsc.TestingKnobs.TestingPostApplyFilter = func(filterArgs kvserverbase.ApplyFilterArgs) (int, *roachpb.Error) { + tsc.TestingKnobs.TestingPostApplyFilter = func(filterArgs kvserverbase.ApplyFilterArgs) (int, *kvpb.Error) { if filterArgs.ForcedError == nil && filterArgs.CmdID == incCmdID { atomic.AddInt64(&incApplyCount, 1) } @@ -8107,7 +8106,7 @@ func TestReplicaRefreshMultiple(t *testing.T) { get := getArgs(key) if resp, pErr := kv.SendWrapped(ctx, tc.Sender(), &get); pErr != nil { t.Fatal(pErr) - } else if x, err := resp.(*roachpb.GetResponse).Value.GetInt(); err != nil { + } else if x, err := resp.(*kvpb.GetResponse).Value.GetInt(); err != nil { t.Fatalf("returned non-int: %+v", err) } else if x != 3 { t.Fatalf("expected 3, got %d", x) @@ -8120,7 +8119,7 @@ func TestReplicaRefreshMultiple(t *testing.T) { // since the reproposals we're concerned with don't result in // reevaluation it doesn't matter) inc := incrementArgs(key, 1) - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Add(inc) ba.Timestamp = tc.Clock().Now() @@ -8211,7 +8210,7 @@ func TestReplicaRefreshMultiple(t *testing.T) { // raft). if resp, pErr := kv.SendWrapped(ctx, tc.Sender(), &get); pErr != nil { t.Fatal(pErr) - } else if x, err := resp.(*roachpb.GetResponse).Value.GetInt(); err != nil { + } else if x, err := resp.(*kvpb.GetResponse).Value.GetInt(); err != nil { t.Fatalf("returned non-int: %+v", err) } else if x != 4 { t.Fatalf("expected 4, got %d", x) @@ -8295,7 +8294,7 @@ func TestReplicaReproposalWithNewLeaseIndexError(t *testing.T) { // Perform a write that will first hit an illegal lease index error and // will then hit the injected error when we attempt to repropose it. - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} iArg := incrementArgs(key, 10) ba.Add(iArg) if _, pErr := tc.Sender().Send(magicCtx, ba); pErr == nil { @@ -8312,7 +8311,7 @@ func TestReplicaReproposalWithNewLeaseIndexError(t *testing.T) { gArgs := getArgs(key) if reply, pErr := tc.SendWrapped(&gArgs); pErr != nil { t.Fatal(pErr) - } else if v, err := reply.(*roachpb.GetResponse).Value.GetInt(); err != nil { + } else if v, err := reply.(*kvpb.GetResponse).Value.GetInt(); err != nil { t.Fatal(err) } else if v != initCount { t.Fatalf("expected value of %d, found %d", initCount, v) @@ -8335,8 +8334,8 @@ func TestFailureToProcessCommandClearsLocalResult(t *testing.T) { key := roachpb.Key("a") txn := newTransaction("test", key, 1, tc.Clock()) - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: txn} + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: txn} put := putArgs(key, []byte("value")) assignSeqNumsForReqs(txn, &put) ba.Add(&put) @@ -8365,7 +8364,7 @@ func TestFailureToProcessCommandClearsLocalResult(t *testing.T) { opCtx, getRecAndFinish := tracing.ContextWithRecordingSpan(ctx, tr, "test-recording") defer getRecAndFinish() - ba = &roachpb.BatchRequest{} + ba = &kvpb.BatchRequest{} et, etH := endTxnArgs(txn, true /* commit */) et.LockSpans = []roachpb.Span{{Key: key}} assignSeqNumsForReqs(txn, &et) @@ -8410,7 +8409,7 @@ func TestMVCCStatsGCCommutesWithWrites(t *testing.T) { require.NoError(t, err) write := func() hlc.Timestamp { - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} put := putArgs(key, []byte("0")) ba.Add(&put) resp, pErr := store.TestSender().Send(ctx, ba) @@ -8452,7 +8451,7 @@ func TestMVCCStatsGCCommutesWithWrites(t *testing.T) { recomputeReq := recomputeStatsArgs(key) resp, pErr := kv.SendWrapped(ctx, store.TestSender(), &recomputeReq) require.Nil(t, pErr) - delta := enginepb.MVCCStats(resp.(*roachpb.RecomputeStatsResponse).AddedDelta) + delta := enginepb.MVCCStats(resp.(*kvpb.RecomputeStatsResponse).AddedDelta) delta.AgeTo(expDelta.LastUpdateNanos) require.Equal(t, expDelta, delta) } @@ -8485,13 +8484,13 @@ func TestBatchTimestampBelowGCThreshold(t *testing.T) { // Verify a Get works. gArgs := getArgs(key) - if _, err := tc.SendWrappedWith(roachpb.Header{ + if _, err := tc.SendWrappedWith(kvpb.Header{ Timestamp: ts1, }, &gArgs); err != nil { t.Fatalf("could not get data: %+v", err) } // Verify a later Get works. - if _, err := tc.SendWrappedWith(roachpb.Header{ + if _, err := tc.SendWrappedWith(kvpb.Header{ Timestamp: ts3, }, &gArgs); err != nil { t.Fatalf("could not get data: %+v", err) @@ -8499,29 +8498,29 @@ func TestBatchTimestampBelowGCThreshold(t *testing.T) { // Put some data for use with CP later on. pArgs := putArgs(keycp, va) - if _, err := tc.SendWrappedWith(roachpb.Header{ + if _, err := tc.SendWrappedWith(kvpb.Header{ Timestamp: ts1, }, &pArgs); err != nil { t.Fatalf("could not put data: %+v", err) } // Do a GC. - gcr := roachpb.GCRequest{ + gcr := kvpb.GCRequest{ Threshold: ts2, } - if _, err := tc.SendWrappedWith(roachpb.Header{RangeID: 1}, &gcr); err != nil { + if _, err := tc.SendWrappedWith(kvpb.Header{RangeID: 1}, &gcr); err != nil { t.Fatal(err) } // Do the same Get, which should now fail. - if _, pErr := tc.SendWrappedWith(roachpb.Header{ + if _, pErr := tc.SendWrappedWith(kvpb.Header{ Timestamp: ts1, }, &gArgs); !testutils.IsPError(pErr, `batch timestamp 0.\d+,\d+ must be after replica GC threshold 0.\d+,\d+`) { t.Fatalf("unexpected error: %v", pErr) } // Verify a later Get works. - if _, pErr := tc.SendWrappedWith(roachpb.Header{ + if _, pErr := tc.SendWrappedWith(kvpb.Header{ Timestamp: ts3, }, &gArgs); pErr != nil { t.Fatal(pErr) @@ -8529,13 +8528,13 @@ func TestBatchTimestampBelowGCThreshold(t *testing.T) { // Verify an early CPut fails. cpArgs := cPutArgs(keycp, vb, va) - if _, pErr := tc.SendWrappedWith(roachpb.Header{ + if _, pErr := tc.SendWrappedWith(kvpb.Header{ Timestamp: ts2, }, &cpArgs); !testutils.IsPError(pErr, `batch timestamp 0.\d+,\d+ must be after replica GC threshold 0.\d+,\d+`) { t.Fatalf("unexpected error: %v", pErr) } // Verify a later CPut works. - if _, pErr := tc.SendWrappedWith(roachpb.Header{ + if _, pErr := tc.SendWrappedWith(kvpb.Header{ Timestamp: ts3, }, &cpArgs); pErr != nil { t.Fatal(pErr) @@ -8567,15 +8566,15 @@ func TestRefreshFromBelowGCThreshold(t *testing.T) { // Construct a Refresh{Range} request for a transaction that refreshes the // time interval (ts2, ts4]. - var refresh roachpb.Request + var refresh kvpb.Request if ranged { - refresh = &roachpb.RefreshRangeRequest{ - RequestHeader: roachpb.RequestHeader{Key: keyA, EndKey: keyB}, + refresh = &kvpb.RefreshRangeRequest{ + RequestHeader: kvpb.RequestHeader{Key: keyA, EndKey: keyB}, RefreshFrom: ts2, } } else { - refresh = &roachpb.RefreshRequest{ - RequestHeader: roachpb.RequestHeader{Key: keyA}, + refresh = &kvpb.RefreshRequest{ + RequestHeader: kvpb.RequestHeader{Key: keyA}, RefreshFrom: ts2, } } @@ -8595,12 +8594,12 @@ func TestRefreshFromBelowGCThreshold(t *testing.T) { } { t.Run(fmt.Sprintf("gcThreshold=%s", testCase.gc), func(t *testing.T) { if !testCase.gc.IsEmpty() { - gcr := roachpb.GCRequest{Threshold: testCase.gc} - _, pErr := tc.SendWrappedWith(roachpb.Header{Timestamp: testCase.gc}, &gcr) + gcr := kvpb.GCRequest{Threshold: testCase.gc} + _, pErr := tc.SendWrappedWith(kvpb.Header{Timestamp: testCase.gc}, &gcr) require.Nil(t, pErr) } - _, pErr := tc.SendWrappedWith(roachpb.Header{Txn: &txn}, refresh) + _, pErr := tc.SendWrappedWith(kvpb.Header{Txn: &txn}, refresh) if testCase.expErr { require.NotNil(t, pErr) require.Regexp(t, `batch timestamp .* must be after replica GC threshold .*`, pErr) @@ -8675,9 +8674,9 @@ func TestGCThresholdRacesWithRead(t *testing.T) { ts1 := now.Add(1, 0) ts2 := now.Add(2, 0) ts3 := now.Add(3, 0) - h1 := roachpb.Header{RangeID: desc.RangeID, Timestamp: ts1} - h2 := roachpb.Header{RangeID: desc.RangeID, Timestamp: ts2} - h3 := roachpb.Header{RangeID: desc.RangeID, Timestamp: ts3} + h1 := kvpb.Header{RangeID: desc.RangeID, Timestamp: ts1} + h2 := kvpb.Header{RangeID: desc.RangeID, Timestamp: ts2} + h3 := kvpb.Header{RangeID: desc.RangeID, Timestamp: ts3} va := []byte("a") vb := []byte("b") @@ -8700,11 +8699,11 @@ func TestGCThresholdRacesWithRead(t *testing.T) { require.NoError(t, err) testutils.SucceedsSoon(t, func() error { - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.RangeID = desc.RangeID - ba.ReadConsistency = roachpb.INCONSISTENT - ba.Add(&roachpb.QueryResolvedTimestampRequest{ - RequestHeader: roachpb.RequestHeader{Key: key, EndKey: key.Next()}, + ba.ReadConsistency = kvpb.INCONSISTENT + ba.Add(&kvpb.QueryResolvedTimestampRequest{ + RequestHeader: kvpb.RequestHeader{Key: key, EndKey: key.Next()}, }) br, pErr := reader.Send(ctx, ba) require.Nil(t, pErr) @@ -8721,8 +8720,8 @@ func TestGCThresholdRacesWithRead(t *testing.T) { resp, pErr := kv.SendWrappedWith(ctx, reader, h1, &gArgs) require.Nil(t, pErr) require.NotNil(t, resp) - require.NotNil(t, resp.(*roachpb.GetResponse).Value) - b, err := resp.(*roachpb.GetResponse).Value.GetBytes() + require.NotNil(t, resp.(*kvpb.GetResponse).Value) + b, err := resp.(*kvpb.GetResponse).Value.GetBytes() require.Nil(t, err) require.Equal(t, va, b) @@ -8768,13 +8767,13 @@ func TestGCThresholdRacesWithRead(t *testing.T) { if pErr == nil { t.Logf("read won race: %v", resp) require.NotNil(t, resp) - require.NotNil(t, resp.(*roachpb.GetResponse).Value) - b, err := resp.(*roachpb.GetResponse).Value.GetBytes() + require.NotNil(t, resp.(*kvpb.GetResponse).Value) + b, err := resp.(*kvpb.GetResponse).Value.GetBytes() require.Nil(t, err) require.Equal(t, va, b) } else { t.Logf("read lost race: %v", pErr) - gcErr := &roachpb.BatchTimestampBeforeGCError{} + gcErr := &kvpb.BatchTimestampBeforeGCError{} require.ErrorAs(t, pErr.GoError(), &gcErr) } } @@ -8803,10 +8802,10 @@ func BenchmarkMVCCGCWithForegroundTraffic(b *testing.B) { key := roachpb.Key("test") // send sends the Request with a present-time batch timestamp. - send := func(args roachpb.Request) *roachpb.BatchResponse { - var header roachpb.Header + send := func(args kvpb.Request) *kvpb.BatchResponse { + var header kvpb.Header header.Timestamp = tc.Clock().Now() - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Header = header ba.Add(args) resp, err := tc.Sender().Send(ctx, ba) @@ -8953,7 +8952,7 @@ func TestReplicaTimestampCacheBumpNotLost(t *testing.T) { txn := newTransaction("test", key, 1, tc.Clock()) minNewTS := func() hlc.Timestamp { - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} scan := scanArgs(key, tc.repl.Desc().EndKey.AsRawKey()) ba.Add(scan) @@ -8967,7 +8966,7 @@ func TestReplicaTimestampCacheBumpNotLost(t *testing.T) { return resp.Timestamp }() - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Txn = txn txnPut := putArgs(key, []byte("timestamp should be bumped")) ba.Add(&txnPut) @@ -9011,7 +9010,7 @@ func TestReplicaEvaluationNotTxnMutation(t *testing.T) { txn := newTransaction("test", key, 1, tc.Clock()) - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Txn = txn ba.Timestamp = txn.WriteTimestamp txnPut := putArgs(key, []byte("foo")) @@ -9342,7 +9341,7 @@ func TestCancelPendingCommands(t *testing.T) { proposalDropped := false tc.repl.mu.Lock() tc.repl.mu.proposalBuf.testing.submitProposalFilter = func(p *ProposalData) (drop bool, _ error) { - if _, ok := p.Request.GetArg(roachpb.Increment); ok { + if _, ok := p.Request.GetArg(kvpb.Increment); ok { if !proposalDropped { // Notify the main thread the first time we drop a proposal. close(proposalDroppedCh) @@ -9354,7 +9353,7 @@ func TestCancelPendingCommands(t *testing.T) { } tc.repl.mu.Unlock() - errChan := make(chan *roachpb.Error, 1) + errChan := make(chan *kvpb.Error, 1) go func() { incArgs := incrementArgs(roachpb.Key("a"), 1) _, pErr := kv.SendWrapped(ctx, tc.Sender(), incArgs) @@ -9372,7 +9371,7 @@ func TestCancelPendingCommands(t *testing.T) { tc.repl.disconnectReplicationRaftMuLocked(ctx) tc.repl.raftMu.Unlock() pErr := <-errChan - if _, ok := pErr.GetDetail().(*roachpb.AmbiguousResultError); !ok { + if _, ok := pErr.GetDetail().(*kvpb.AmbiguousResultError); !ok { t.Errorf("expected AmbiguousResultError, got %v", pErr) } } @@ -9386,7 +9385,7 @@ func TestNoopRequestsNotProposed(t *testing.T) { cfg := TestStoreConfig(nil) cfg.TestingKnobs.DontRetryPushTxnFailures = true - rh := roachpb.RequestHeader{Key: roachpb.Key("a")} + rh := kvpb.RequestHeader{Key: roachpb.Key("a")} txn := newTransaction( "name", rh.Key, @@ -9394,56 +9393,56 @@ func TestNoopRequestsNotProposed(t *testing.T) { cfg.Clock, ) - getReq := &roachpb.GetRequest{ + getReq := &kvpb.GetRequest{ RequestHeader: rh, } - putReq := &roachpb.PutRequest{ + putReq := &kvpb.PutRequest{ RequestHeader: rh, Value: roachpb.MakeValueFromBytes([]byte("val")), } - deleteReq := &roachpb.DeleteRequest{ + deleteReq := &kvpb.DeleteRequest{ RequestHeader: rh, } - endTxnCommitReq := &roachpb.EndTxnRequest{ + endTxnCommitReq := &kvpb.EndTxnRequest{ RequestHeader: rh, Commit: true, } - endTxnAbortReq := &roachpb.EndTxnRequest{ + endTxnAbortReq := &kvpb.EndTxnRequest{ RequestHeader: rh, Commit: true, } - hbTxnReq := &roachpb.HeartbeatTxnRequest{ + hbTxnReq := &kvpb.HeartbeatTxnRequest{ RequestHeader: rh, Now: cfg.Clock.Now(), } - pushTxnReq := &roachpb.PushTxnRequest{ - RequestHeader: roachpb.RequestHeader{ + pushTxnReq := &kvpb.PushTxnRequest{ + RequestHeader: kvpb.RequestHeader{ Key: txn.TxnMeta.Key, }, PusheeTxn: txn.TxnMeta, - PushType: roachpb.PUSH_ABORT, + PushType: kvpb.PUSH_ABORT, Force: true, } - resolveCommittedIntentReq := &roachpb.ResolveIntentRequest{ + resolveCommittedIntentReq := &kvpb.ResolveIntentRequest{ RequestHeader: rh, IntentTxn: txn.TxnMeta, Status: roachpb.COMMITTED, Poison: false, } - resolveAbortedIntentReq := &roachpb.ResolveIntentRequest{ + resolveAbortedIntentReq := &kvpb.ResolveIntentRequest{ RequestHeader: rh, IntentTxn: txn.TxnMeta, Status: roachpb.ABORTED, Poison: true, } - barrierReq := &roachpb.BarrierRequest{ + barrierReq := &kvpb.BarrierRequest{ RequestHeader: rh, } sendReq := func( - ctx context.Context, repl *Replica, req roachpb.Request, txn *roachpb.Transaction, - ) *roachpb.Error { - ba := &roachpb.BatchRequest{} + ctx context.Context, repl *Replica, req kvpb.Request, txn *roachpb.Transaction, + ) *kvpb.Error { + ba := &kvpb.BatchRequest{} ba.Header.RangeID = repl.RangeID ba.Add(req) ba.Txn = txn @@ -9456,9 +9455,9 @@ func TestNoopRequestsNotProposed(t *testing.T) { testCases := []struct { name string - setup func(context.Context, *Replica) *roachpb.Error // optional + setup func(context.Context, *Replica) *kvpb.Error // optional useTxn bool - req roachpb.Request + req kvpb.Request expFailure string // regexp pattern to match on error if not empty expProposal bool }{ @@ -9511,7 +9510,7 @@ func TestNoopRequestsNotProposed(t *testing.T) { }, { name: "end txn (commit) with auto-gc, with existing record", - setup: func(ctx context.Context, repl *Replica) *roachpb.Error { + setup: func(ctx context.Context, repl *Replica) *kvpb.Error { return sendReq(ctx, repl, hbTxnReq, txn) }, useTxn: true, @@ -9520,7 +9519,7 @@ func TestNoopRequestsNotProposed(t *testing.T) { }, { name: "end txn (abort) with auto-gc, with existing record", - setup: func(ctx context.Context, repl *Replica) *roachpb.Error { + setup: func(ctx context.Context, repl *Replica) *kvpb.Error { return sendReq(ctx, repl, hbTxnReq, txn) }, useTxn: true, @@ -9535,7 +9534,7 @@ func TestNoopRequestsNotProposed(t *testing.T) { }, { name: "push txn req", - setup: func(ctx context.Context, repl *Replica) *roachpb.Error { + setup: func(ctx context.Context, repl *Replica) *kvpb.Error { return sendReq(ctx, repl, hbTxnReq, txn) }, req: pushTxnReq, @@ -9543,7 +9542,7 @@ func TestNoopRequestsNotProposed(t *testing.T) { }, { name: "redundant push txn req", - setup: func(ctx context.Context, repl *Replica) *roachpb.Error { + setup: func(ctx context.Context, repl *Replica) *kvpb.Error { if pErr := sendReq(ctx, repl, hbTxnReq, txn); pErr != nil { return pErr } @@ -9555,7 +9554,7 @@ func TestNoopRequestsNotProposed(t *testing.T) { }, { name: "resolve committed intent req, with intent", - setup: func(ctx context.Context, repl *Replica) *roachpb.Error { + setup: func(ctx context.Context, repl *Replica) *kvpb.Error { return sendReq(ctx, repl, putReq, txn) }, req: resolveCommittedIntentReq, @@ -9569,7 +9568,7 @@ func TestNoopRequestsNotProposed(t *testing.T) { }, { name: "resolve aborted intent req, with intent", - setup: func(ctx context.Context, repl *Replica) *roachpb.Error { + setup: func(ctx context.Context, repl *Replica) *kvpb.Error { return sendReq(ctx, repl, putReq, txn) }, req: resolveAbortedIntentReq, @@ -9585,7 +9584,7 @@ func TestNoopRequestsNotProposed(t *testing.T) { }, { name: "redundant resolve aborted intent req", - setup: func(ctx context.Context, repl *Replica) *roachpb.Error { + setup: func(ctx context.Context, repl *Replica) *kvpb.Error { return sendReq(ctx, repl, resolveAbortedIntentReq, nil /* txn */) }, req: resolveAbortedIntentReq, @@ -9594,7 +9593,7 @@ func TestNoopRequestsNotProposed(t *testing.T) { }, { name: "barrier", - setup: func(ctx context.Context, repl *Replica) *roachpb.Error { + setup: func(ctx context.Context, repl *Replica) *kvpb.Error { return sendReq(ctx, repl, barrierReq, nil /* txn */) }, req: barrierReq, @@ -9629,7 +9628,7 @@ func TestNoopRequestsNotProposed(t *testing.T) { markerTS := tc.Clock().Now() repl.mu.Lock() repl.store.TestingKnobs().TestingProposalFilter = - func(args kvserverbase.ProposalFilterArgs) *roachpb.Error { + func(args kvserverbase.ProposalFilterArgs) *kvpb.Error { if args.Req.Timestamp == markerTS { atomic.AddInt32(&propCount, 1) } @@ -9637,7 +9636,7 @@ func TestNoopRequestsNotProposed(t *testing.T) { } repl.mu.Unlock() - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Timestamp = markerTS ba.RangeID = repl.RangeID if c.useTxn { @@ -9704,9 +9703,9 @@ func TestErrorInRaftApplicationClearsIntents(t *testing.T) { if err != nil { t.Fatal(err) } - storeKnobs.TestingApplyCalledTwiceFilter = func(filterArgs kvserverbase.ApplyFilterArgs) (int, *roachpb.Error) { + storeKnobs.TestingApplyCalledTwiceFilter = func(filterArgs kvserverbase.ApplyFilterArgs) (int, *kvpb.Error) { if atomic.LoadInt32(&filterActive) == 1 { - return 0, roachpb.NewErrorf("boom") + return 0, kvpb.NewErrorf("boom") } return 0, nil } @@ -9740,7 +9739,7 @@ func TestErrorInRaftApplicationClearsIntents(t *testing.T) { txn.Sequence++ etArgs, _ := endTxnArgs(txn, true /* commit */) etArgs.LockSpans = []roachpb.Span{{Key: roachpb.Key("bb")}} - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Header.Txn = txn ba.Add(&etArgs) assignSeqNumsForReqs(txn, &etArgs) @@ -9785,7 +9784,7 @@ func TestProposeWithAsyncConsensus(t *testing.T) { var filterActive int32 blockRaftApplication := make(chan struct{}) tsc.TestingKnobs.TestingApplyCalledTwiceFilter = - func(filterArgs kvserverbase.ApplyFilterArgs) (int, *roachpb.Error) { + func(filterArgs kvserverbase.ApplyFilterArgs) (int, *kvpb.Error) { if atomic.LoadInt32(&filterActive) == 1 { <-blockRaftApplication } @@ -9797,7 +9796,7 @@ func TestProposeWithAsyncConsensus(t *testing.T) { tc.StartWithStoreConfig(ctx, t, stopper, tsc) repl := tc.repl - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} key := roachpb.Key("a") put := putArgs(key, []byte("val")) ba.Add(&put) @@ -9846,7 +9845,7 @@ func TestApplyPaginatedCommittedEntries(t *testing.T) { blockRaftApplication := make(chan struct{}) blockingRaftApplication := make(chan struct{}, 1) tsc.TestingKnobs.TestingApplyCalledTwiceFilter = - func(filterArgs kvserverbase.ApplyFilterArgs) (int, *roachpb.Error) { + func(filterArgs kvserverbase.ApplyFilterArgs) (int, *kvpb.Error) { if atomic.LoadInt32(&filterActive) == 1 { select { case blockingRaftApplication <- struct{}{}: @@ -9863,7 +9862,7 @@ func TestApplyPaginatedCommittedEntries(t *testing.T) { repl := tc.repl // Block command application then propose a command to Raft. - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} key := roachpb.Key("a") put := putArgs(key, []byte("val")) ba.Add(&put) @@ -9883,13 +9882,13 @@ func TestApplyPaginatedCommittedEntries(t *testing.T) { <-blockingRaftApplication var ch chan proposalResult for i := 0; i < 50; i++ { - ba2 := &roachpb.BatchRequest{} + ba2 := &kvpb.BatchRequest{} key := roachpb.Key("a") put := putArgs(key, make([]byte, 2*tsc.RaftMaxCommittedSizePerReady)) ba2.Add(&put) ba2.Timestamp = tc.Clock().Now() - var pErr *roachpb.Error + var pErr *kvpb.Error _, tok := repl.mu.proposalBuf.TrackEvaluatingRequest(ctx, hlc.MinTimestamp) ch, _, _, _, pErr = repl.evalAndPropose(ctx, ba2, allSpansGuard(), &st, uncertainty.Interval{}, tok.Move(ctx)) if pErr != nil { @@ -10331,8 +10330,8 @@ func TestReplicaRecomputeStats(t *testing.T) { runTest := func(test testCase) { t.Run(test.name, func(t *testing.T) { - args := &roachpb.RecomputeStatsRequest{ - RequestHeader: roachpb.RequestHeader{ + args := &kvpb.RecomputeStatsRequest{ + RequestHeader: kvpb.RequestHeader{ Key: test.key, }, } @@ -10345,7 +10344,7 @@ func TestReplicaRecomputeStats(t *testing.T) { return } - delta := enginepb.MVCCStats(resp.(*roachpb.RecomputeStatsResponse).AddedDelta) + delta := enginepb.MVCCStats(resp.(*kvpb.RecomputeStatsResponse).AddedDelta) delta.AgeTo(test.expDelta.LastUpdateNanos) if delta != test.expDelta { @@ -10405,9 +10404,9 @@ func TestConsistenctQueueErrorFromCheckConsistency(t *testing.T) { cfg := TestStoreConfig(nil) cfg.TestingKnobs = StoreTestingKnobs{ - TestingRequestFilter: func(_ context.Context, ba *roachpb.BatchRequest) *roachpb.Error { - if _, ok := ba.GetArg(roachpb.ComputeChecksum); ok { - return roachpb.NewErrorf("boom") + TestingRequestFilter: func(_ context.Context, ba *kvpb.BatchRequest) *kvpb.Error { + if _, ok := ba.GetArg(kvpb.ComputeChecksum); ok { + return kvpb.NewErrorf("boom") } return nil }, @@ -10458,7 +10457,7 @@ func TestReplicaServersideRefreshes(t *testing.T) { ) return &txn } - send := func(ba *roachpb.BatchRequest) (hlc.Timestamp, error) { + send := func(ba *kvpb.BatchRequest) (hlc.Timestamp, error) { br, pErr := tc.Sender().Send(ctx, ba) if pErr != nil { return hlc.Timestamp{}, pErr.GoError() @@ -10468,7 +10467,7 @@ func TestReplicaServersideRefreshes(t *testing.T) { // Regression test for #31870. snap := tc.engine.NewSnapshot() defer snap.Close() - res, err := CalcReplicaDigest(ctx, *tc.repl.Desc(), tc.engine, roachpb.ChecksumMode_CHECK_FULL, + res, err := CalcReplicaDigest(ctx, *tc.repl.Desc(), tc.engine, kvpb.ChecksumMode_CHECK_FULL, quotapool.NewRateLimiter("test", quotapool.Inf(), 0)) if err != nil { return hlc.Timestamp{}, err @@ -10480,13 +10479,13 @@ func TestReplicaServersideRefreshes(t *testing.T) { return br.Timestamp, nil } get := func(key string) (hlc.Timestamp, error) { - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} get := getArgs(roachpb.Key(key)) ba.Add(&get) return send(ba) } put := func(key, val string) (hlc.Timestamp, error) { - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} put := putArgs(roachpb.Key(key), []byte(val)) ba.Add(&put) return send(ba) @@ -10495,7 +10494,7 @@ func TestReplicaServersideRefreshes(t *testing.T) { testCases := []struct { name string setupFn func() (hlc.Timestamp, error) // returns expected batch execution timestamp - batchFn func(hlc.Timestamp) (*roachpb.BatchRequest, hlc.Timestamp) + batchFn func(hlc.Timestamp) (*kvpb.BatchRequest, hlc.Timestamp) expErr string }{ { @@ -10503,8 +10502,8 @@ func TestReplicaServersideRefreshes(t *testing.T) { setupFn: func() (hlc.Timestamp, error) { return put("a", "put") }, - batchFn: func(ts hlc.Timestamp) (ba *roachpb.BatchRequest, expTS hlc.Timestamp) { - ba = &roachpb.BatchRequest{} + batchFn: func(ts hlc.Timestamp) (ba *kvpb.BatchRequest, expTS hlc.Timestamp) { + ba = &kvpb.BatchRequest{} ba.Timestamp = ts.Prev() expTS = ts.Next() put := putArgs(roachpb.Key("a"), []byte("put2")) @@ -10521,8 +10520,8 @@ func TestReplicaServersideRefreshes(t *testing.T) { _, _ = put("b", "put1") return put("b", "put2") }, - batchFn: func(ts hlc.Timestamp) (ba *roachpb.BatchRequest, expTS hlc.Timestamp) { - ba = &roachpb.BatchRequest{} + batchFn: func(ts hlc.Timestamp) (ba *kvpb.BatchRequest, expTS hlc.Timestamp) { + ba = &kvpb.BatchRequest{} ba.Timestamp = ts.Prev() expTS = ts.Next() cput := cPutArgs(roachpb.Key("b"), []byte("cput"), []byte("put2")) @@ -10539,8 +10538,8 @@ func TestReplicaServersideRefreshes(t *testing.T) { _, _ = put("b-iput", "put1") return put("b-iput", "put2") }, - batchFn: func(ts hlc.Timestamp) (ba *roachpb.BatchRequest, expTS hlc.Timestamp) { - ba = &roachpb.BatchRequest{} + batchFn: func(ts hlc.Timestamp) (ba *kvpb.BatchRequest, expTS hlc.Timestamp) { + ba = &kvpb.BatchRequest{} ba.Timestamp = ts.Prev() expTS = ts.Next() iput := iPutArgs(roachpb.Key("b-iput"), []byte("put2")) @@ -10559,8 +10558,8 @@ func TestReplicaServersideRefreshes(t *testing.T) { setupFn: func() (hlc.Timestamp, error) { return put("a", "put") }, - batchFn: func(ts hlc.Timestamp) (ba *roachpb.BatchRequest, expTS hlc.Timestamp) { - ba = &roachpb.BatchRequest{} + batchFn: func(ts hlc.Timestamp) (ba *kvpb.BatchRequest, expTS hlc.Timestamp) { + ba = &kvpb.BatchRequest{} ba.Timestamp = ts.Prev() get := getArgs(roachpb.Key("a")) put := putArgs(roachpb.Key("a"), []byte("put2")) @@ -10574,8 +10573,8 @@ func TestReplicaServersideRefreshes(t *testing.T) { setupFn: func() (hlc.Timestamp, error) { return get("a") }, - batchFn: func(ts hlc.Timestamp) (ba *roachpb.BatchRequest, expTS hlc.Timestamp) { - ba = &roachpb.BatchRequest{} + batchFn: func(ts hlc.Timestamp) (ba *kvpb.BatchRequest, expTS hlc.Timestamp) { + ba = &kvpb.BatchRequest{} ba.Timestamp = ts.Prev() expTS = ts.Next() put := putArgs(roachpb.Key("a"), []byte("put2")) @@ -10590,8 +10589,8 @@ func TestReplicaServersideRefreshes(t *testing.T) { _, _ = put("c-cput", "put") return put("c-cput", "put") }, - batchFn: func(ts hlc.Timestamp) (ba *roachpb.BatchRequest, expTS hlc.Timestamp) { - ba = &roachpb.BatchRequest{} + batchFn: func(ts hlc.Timestamp) (ba *kvpb.BatchRequest, expTS hlc.Timestamp) { + ba = &kvpb.BatchRequest{} ba.Txn = newTxn("c-cput", ts.Prev()) cput := cPutArgs(roachpb.Key("c-cput"), []byte("iput"), []byte("put")) ba.Add(&cput) @@ -10606,8 +10605,8 @@ func TestReplicaServersideRefreshes(t *testing.T) { setupFn: func() (hlc.Timestamp, error) { return put("c-iput", "put") }, - batchFn: func(ts hlc.Timestamp) (ba *roachpb.BatchRequest, expTS hlc.Timestamp) { - ba = &roachpb.BatchRequest{} + batchFn: func(ts hlc.Timestamp) (ba *kvpb.BatchRequest, expTS hlc.Timestamp) { + ba = &kvpb.BatchRequest{} ba.Txn = newTxn("c-iput", ts.Prev()) iput := iPutArgs(roachpb.Key("c-iput"), []byte("iput")) ba.Add(&iput) @@ -10622,8 +10621,8 @@ func TestReplicaServersideRefreshes(t *testing.T) { setupFn: func() (hlc.Timestamp, error) { return put("c-scan", "put") }, - batchFn: func(ts hlc.Timestamp) (ba *roachpb.BatchRequest, expTS hlc.Timestamp) { - ba = &roachpb.BatchRequest{} + batchFn: func(ts hlc.Timestamp) (ba *kvpb.BatchRequest, expTS hlc.Timestamp) { + ba = &kvpb.BatchRequest{} ba.Txn = newTxn("c-scan", ts.Prev()) scan := scanArgs(roachpb.Key("c-scan"), roachpb.Key("c-scan\x00")) scan.KeyLocking = lock.Exclusive @@ -10640,9 +10639,9 @@ func TestReplicaServersideRefreshes(t *testing.T) { _, _ = put("c-cput", "put") return put("c-cput", "put") }, - batchFn: func(ts hlc.Timestamp) (ba *roachpb.BatchRequest, expTS hlc.Timestamp) { + batchFn: func(ts hlc.Timestamp) (ba *kvpb.BatchRequest, expTS hlc.Timestamp) { expTS = ts.Next() - ba = &roachpb.BatchRequest{} + ba = &kvpb.BatchRequest{} ba.Txn = newTxn("c-cput", ts.Prev()) ba.CanForwardReadTimestamp = true cput := cPutArgs(roachpb.Key("c-cput"), []byte("iput"), []byte("put")) @@ -10665,8 +10664,8 @@ func TestReplicaServersideRefreshes(t *testing.T) { _, _ = put("c-iput", "put1") return put("c-iput", "put2") }, - batchFn: func(ts hlc.Timestamp) (ba *roachpb.BatchRequest, expTS hlc.Timestamp) { - ba = &roachpb.BatchRequest{} + batchFn: func(ts hlc.Timestamp) (ba *kvpb.BatchRequest, expTS hlc.Timestamp) { + ba = &kvpb.BatchRequest{} ba.Txn = newTxn("c-iput", ts.Prev()) ba.CanForwardReadTimestamp = true iput := iPutArgs(roachpb.Key("c-iput"), []byte("put2")) @@ -10683,9 +10682,9 @@ func TestReplicaServersideRefreshes(t *testing.T) { setupFn: func() (hlc.Timestamp, error) { return put("c-scan", "put") }, - batchFn: func(ts hlc.Timestamp) (ba *roachpb.BatchRequest, expTS hlc.Timestamp) { + batchFn: func(ts hlc.Timestamp) (ba *kvpb.BatchRequest, expTS hlc.Timestamp) { expTS = ts.Next() - ba = &roachpb.BatchRequest{} + ba = &kvpb.BatchRequest{} ba.Txn = newTxn("c-scan", ts.Prev()) ba.CanForwardReadTimestamp = true scan := scanArgs(roachpb.Key("c-scan"), roachpb.Key("c-scan\x00")) @@ -10702,8 +10701,8 @@ func TestReplicaServersideRefreshes(t *testing.T) { _, _ = put("d", "put") return put("d", "put") }, - batchFn: func(ts hlc.Timestamp) (ba *roachpb.BatchRequest, expTS hlc.Timestamp) { - ba = &roachpb.BatchRequest{} + batchFn: func(ts hlc.Timestamp) (ba *kvpb.BatchRequest, expTS hlc.Timestamp) { + ba = &kvpb.BatchRequest{} ba.Txn = newTxn("d", ts.Prev()) cput := cPutArgs(ba.Txn.Key, []byte("cput"), []byte("put")) et, _ := endTxnArgs(ba.Txn, true /* commit */) @@ -10720,9 +10719,9 @@ func TestReplicaServersideRefreshes(t *testing.T) { _, _ = put("e", "put") return put("e", "put") }, - batchFn: func(ts hlc.Timestamp) (ba *roachpb.BatchRequest, expTS hlc.Timestamp) { + batchFn: func(ts hlc.Timestamp) (ba *kvpb.BatchRequest, expTS hlc.Timestamp) { expTS = ts.Next() - ba = &roachpb.BatchRequest{} + ba = &kvpb.BatchRequest{} ba.Txn = newTxn("e", ts.Prev()) ba.CanForwardReadTimestamp = true // necessary to indicate serverside-refresh is possible cput := cPutArgs(ba.Txn.Key, []byte("cput"), []byte("put")) @@ -10739,9 +10738,9 @@ func TestReplicaServersideRefreshes(t *testing.T) { _, _ = put("e", "put") return put("e", "put") }, - batchFn: func(ts hlc.Timestamp) (ba *roachpb.BatchRequest, expTS hlc.Timestamp) { + batchFn: func(ts hlc.Timestamp) (ba *kvpb.BatchRequest, expTS hlc.Timestamp) { expTS = ts.Next() - ba = &roachpb.BatchRequest{} + ba = &kvpb.BatchRequest{} ba.Txn = newTxn("e", ts.Prev()) ba.CanForwardReadTimestamp = true // necessary to indicate serverside-refresh is possible cput := cPutArgs(ba.Txn.Key, []byte("cput"), []byte("put")) @@ -10762,11 +10761,11 @@ func TestReplicaServersideRefreshes(t *testing.T) { setupFn: func() (hlc.Timestamp, error) { return put("e1", "put") }, - batchFn: func(ts hlc.Timestamp) (ba *roachpb.BatchRequest, expTS hlc.Timestamp) { + batchFn: func(ts hlc.Timestamp) (ba *kvpb.BatchRequest, expTS hlc.Timestamp) { txn := newTxn("e1", ts.Prev()) // Send write to another key first to avoid 1PC. - ba = &roachpb.BatchRequest{} + ba = &kvpb.BatchRequest{} ba.Txn = txn put := putArgs([]byte("e1-other-key"), []byte("otherput")) ba.Add(&put) @@ -10775,7 +10774,7 @@ func TestReplicaServersideRefreshes(t *testing.T) { panic(err) } - ba = &roachpb.BatchRequest{} + ba = &kvpb.BatchRequest{} ba.Txn = txn // Indicate local retry is possible, even though we don't currently take // advantage of this. @@ -10805,11 +10804,11 @@ func TestReplicaServersideRefreshes(t *testing.T) { } return put("f3", "put") }, - batchFn: func(ts hlc.Timestamp) (ba *roachpb.BatchRequest, expTS hlc.Timestamp) { + batchFn: func(ts hlc.Timestamp) (ba *kvpb.BatchRequest, expTS hlc.Timestamp) { expTS = ts.Next() // We're going to execute before any of the writes in setupFn. ts.Logical = 0 - ba = &roachpb.BatchRequest{} + ba = &kvpb.BatchRequest{} ba.Timestamp = ts for i := 1; i <= 3; i++ { cput := cPutArgs(roachpb.Key(fmt.Sprintf("f%d", i)), []byte("cput"), []byte("put")) @@ -10833,11 +10832,11 @@ func TestReplicaServersideRefreshes(t *testing.T) { } return put("ga3", "put") }, - batchFn: func(ts hlc.Timestamp) (ba *roachpb.BatchRequest, expTS hlc.Timestamp) { + batchFn: func(ts hlc.Timestamp) (ba *kvpb.BatchRequest, expTS hlc.Timestamp) { expTS = ts.Next() // We're going to execute before any of the writes in setupFn. ts.Logical = 0 - ba = &roachpb.BatchRequest{} + ba = &kvpb.BatchRequest{} ba.Txn = newTxn("ga1", ts) ba.CanForwardReadTimestamp = true // necessary to indicate serverside-refresh is possible for i := 1; i <= 3; i++ { @@ -10860,10 +10859,10 @@ func TestReplicaServersideRefreshes(t *testing.T) { } return get("h") }, - batchFn: func(ts hlc.Timestamp) (ba *roachpb.BatchRequest, expTS hlc.Timestamp) { + batchFn: func(ts hlc.Timestamp) (ba *kvpb.BatchRequest, expTS hlc.Timestamp) { txn := newTxn("h", ts.Prev()) // Send write to another key first to avoid 1PC. - ba = &roachpb.BatchRequest{} + ba = &kvpb.BatchRequest{} ba.Txn = txn put := putArgs([]byte("h2"), []byte("otherput")) ba.Add(&put) @@ -10873,7 +10872,7 @@ func TestReplicaServersideRefreshes(t *testing.T) { } // Send the remainder of the transaction in another batch. expTS = ts.Next() - ba = &roachpb.BatchRequest{} + ba = &kvpb.BatchRequest{} ba.Txn = txn ba.CanForwardReadTimestamp = true // necessary to indicate serverside-refresh is possible cput := cPutArgs(ba.Txn.Key, []byte("cput"), []byte("put")) @@ -10891,8 +10890,8 @@ func TestReplicaServersideRefreshes(t *testing.T) { setupFn: func() (hlc.Timestamp, error) { return get("a") }, - batchFn: func(ts hlc.Timestamp) (ba *roachpb.BatchRequest, expTS hlc.Timestamp) { - ba = &roachpb.BatchRequest{} + batchFn: func(ts hlc.Timestamp) (ba *kvpb.BatchRequest, expTS hlc.Timestamp) { + ba = &kvpb.BatchRequest{} ba.Txn = newTxn("a", ts.Prev()) ba.CanForwardReadTimestamp = true // necessary to indicate serverside-refresh is possible expTS = ts.Next() @@ -10912,10 +10911,10 @@ func TestReplicaServersideRefreshes(t *testing.T) { setupFn: func() (hlc.Timestamp, error) { return put("lscan", "put") }, - batchFn: func(ts hlc.Timestamp) (ba *roachpb.BatchRequest, expTS hlc.Timestamp) { + batchFn: func(ts hlc.Timestamp) (ba *kvpb.BatchRequest, expTS hlc.Timestamp) { // Txn with (read_ts, write_ts) = (1, 4) finds a value with // `ts = 2`. Final timestamp should be `ts = 4`. - ba = &roachpb.BatchRequest{} + ba = &kvpb.BatchRequest{} ba.Txn = newTxn("lscan", ts.Prev()) ba.Txn.WriteTimestamp = ts.Next().Next() ba.CanForwardReadTimestamp = true @@ -10934,10 +10933,10 @@ func TestReplicaServersideRefreshes(t *testing.T) { setupFn: func() (hlc.Timestamp, error) { return put("i", "put") }, - batchFn: func(ts hlc.Timestamp) (ba *roachpb.BatchRequest, expTS hlc.Timestamp) { + batchFn: func(ts hlc.Timestamp) (ba *kvpb.BatchRequest, expTS hlc.Timestamp) { txn := newTxn("i", ts.Prev()) // Send write to another key first to avoid 1PC. - ba = &roachpb.BatchRequest{} + ba = &kvpb.BatchRequest{} ba.Txn = txn put1 := putArgs([]byte("i2"), []byte("otherput")) ba.Add(&put1) @@ -10947,7 +10946,7 @@ func TestReplicaServersideRefreshes(t *testing.T) { } // Send the remainder of the transaction in another batch. expTS = ts.Next() - ba = &roachpb.BatchRequest{} + ba = &kvpb.BatchRequest{} ba.Txn = txn ba.CanForwardReadTimestamp = true // necessary to indicate serverside-refresh is possible put2 := putArgs(ba.Txn.Key, []byte("newput")) @@ -10968,8 +10967,8 @@ func TestReplicaServersideRefreshes(t *testing.T) { setupFn: func() (hlc.Timestamp, error) { return put("a", "put") }, - batchFn: func(ts hlc.Timestamp) (ba *roachpb.BatchRequest, expTS hlc.Timestamp) { - ba = &roachpb.BatchRequest{} + batchFn: func(ts hlc.Timestamp) (ba *kvpb.BatchRequest, expTS hlc.Timestamp) { + ba = &kvpb.BatchRequest{} ba.Timestamp = ts.Prev() // NOTE: set the TimestampFromServerClock field manually. This is // usually set on the server for non-transactional requests without @@ -10990,10 +10989,10 @@ func TestReplicaServersideRefreshes(t *testing.T) { setupFn: func() (hlc.Timestamp, error) { return put("a", "put") }, - batchFn: func(ts hlc.Timestamp) (ba *roachpb.BatchRequest, expTS hlc.Timestamp) { + batchFn: func(ts hlc.Timestamp) (ba *kvpb.BatchRequest, expTS hlc.Timestamp) { expTS = ts.Next() ts = ts.Prev() - ba = &roachpb.BatchRequest{} + ba = &kvpb.BatchRequest{} ba.Txn = newTxn("a", ts) ba.Txn.GlobalUncertaintyLimit = expTS ba.CanForwardReadTimestamp = true // necessary to indicate serverside-refresh is possible @@ -11007,9 +11006,9 @@ func TestReplicaServersideRefreshes(t *testing.T) { setupFn: func() (hlc.Timestamp, error) { return put("a", "put") }, - batchFn: func(ts hlc.Timestamp) (ba *roachpb.BatchRequest, expTS hlc.Timestamp) { + batchFn: func(ts hlc.Timestamp) (ba *kvpb.BatchRequest, expTS hlc.Timestamp) { ts = ts.Prev() - ba = &roachpb.BatchRequest{} + ba = &kvpb.BatchRequest{} ba.Txn = newTxn("a", ts) ba.Txn.GlobalUncertaintyLimit = ts.Next() get := getArgs(roachpb.Key("a")) @@ -11023,10 +11022,10 @@ func TestReplicaServersideRefreshes(t *testing.T) { setupFn: func() (hlc.Timestamp, error) { return put("a", "put") }, - batchFn: func(ts hlc.Timestamp) (ba *roachpb.BatchRequest, expTS hlc.Timestamp) { + batchFn: func(ts hlc.Timestamp) (ba *kvpb.BatchRequest, expTS hlc.Timestamp) { expTS = ts.Next() ts = ts.Prev() - ba = &roachpb.BatchRequest{} + ba = &kvpb.BatchRequest{} ba.Txn = newTxn("a", ts) ba.Txn.GlobalUncertaintyLimit = expTS ba.CanForwardReadTimestamp = true // necessary to indicate serverside-refresh is possible @@ -11042,9 +11041,9 @@ func TestReplicaServersideRefreshes(t *testing.T) { setupFn: func() (hlc.Timestamp, error) { return put("a", "put") }, - batchFn: func(ts hlc.Timestamp) (ba *roachpb.BatchRequest, expTS hlc.Timestamp) { + batchFn: func(ts hlc.Timestamp) (ba *kvpb.BatchRequest, expTS hlc.Timestamp) { ts = ts.Prev() - ba = &roachpb.BatchRequest{} + ba = &kvpb.BatchRequest{} ba.Txn = newTxn("a", ts) ba.Txn.GlobalUncertaintyLimit = ts.Next() get := getArgs(roachpb.Key("a")) @@ -11122,15 +11121,15 @@ func TestReplicaPushed1PC(t *testing.T) { // this difference is difficult to observe in a test. If we had // more detailed metrics we could assert that the 1PC path was // not even attempted here. - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: &txn} + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: &txn} put := putArgs(k, []byte("two")) et, _ := endTxnArgs(&txn, true) ba.Add(&put, &et) assignSeqNumsForReqs(&txn, &put, &et) if br, pErr := tc.Sender().Send(ctx, ba); pErr == nil { t.Errorf("did not get expected error. resp=%s", br) - } else if wtoe, ok := pErr.GetDetail().(*roachpb.WriteTooOldError); !ok { + } else if wtoe, ok := pErr.GetDetail().(*kvpb.WriteTooOldError); !ok { t.Errorf("expected WriteTooOldError, got %s", wtoe) } } @@ -11165,9 +11164,9 @@ func TestReplicaNotifyLockTableOn1PC(t *testing.T) { // Create a new transaction and perform a "for update" scan. This should // acquire unreplicated, exclusive locks on the key. txn := newTransaction("test", key, 1, tc.Clock()) - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: txn} - ba.Add(roachpb.NewScan(key, key.Next(), true /* forUpdate */)) + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: txn} + ba.Add(kvpb.NewScan(key, key.Next(), true /* forUpdate */)) if _, pErr := tc.Sender().Send(ctx, ba); pErr != nil { t.Fatalf("unexpected error: %s", pErr) } @@ -11181,7 +11180,7 @@ func TestReplicaNotifyLockTableOn1PC(t *testing.T) { // block forever without the lock-table notification. We didn't need to // disable deadlock detection pushes because this is a non-transactional // write, so it never performs them. - pErrC := make(chan *roachpb.Error, 1) + pErrC := make(chan *kvpb.Error, 1) go func() { otherWrite := incrementArgs(key, 1) _, pErr := tc.SendWrapped(otherWrite) @@ -11197,7 +11196,7 @@ func TestReplicaNotifyLockTableOn1PC(t *testing.T) { // Update the locked value and commit in a single batch. This should release // the "for update" lock. - ba = &roachpb.BatchRequest{} + ba = &kvpb.BatchRequest{} incArgs := incrementArgs(key, 1) et, etH := endTxnArgs(txn, true /* commit */) et.Require1PC = true @@ -11246,9 +11245,9 @@ func TestReplicaQueryLocks(t *testing.T) { // Create a new transaction and perform a "for update" scan. This should // acquire unreplicated, exclusive locks on keys "a" and "b". txn := newTransaction("test", keyA, 1, tc.Clock()) - ba := &roachpb.BatchRequest{} - ba.Header = roachpb.Header{Txn: txn} - ba.Add(roachpb.NewScan(keyA, keyB.Next(), true /* forUpdate */)) + ba := &kvpb.BatchRequest{} + ba.Header = kvpb.Header{Txn: txn} + ba.Add(kvpb.NewScan(keyA, keyB.Next(), true /* forUpdate */)) if _, pErr := tc.Sender().Send(ctx, ba); pErr != nil { t.Fatalf("unexpected error: %s", pErr) } @@ -11256,7 +11255,7 @@ func TestReplicaQueryLocks(t *testing.T) { // Try to write to key "a" outside this transaction. Should wait on the // "for update" lock in a lock wait-queue in the concurrency manager until // the lock is released. - pErrC := make(chan *roachpb.Error, 1) + pErrC := make(chan *kvpb.Error, 1) go func() { otherWrite := incrementArgs(keyA, 1) _, pErr := tc.SendWrapped(otherWrite) @@ -11270,8 +11269,8 @@ func TestReplicaQueryLocks(t *testing.T) { case <-time.After(50 * time.Millisecond): } - var h roachpb.Header - var queryResp *roachpb.QueryLocksResponse + var h kvpb.Header + var queryResp *kvpb.QueryLocksResponse var ok bool if limitResults { h.MaxSpanRequestKeys = 1 @@ -11281,7 +11280,7 @@ func TestReplicaQueryLocks(t *testing.T) { if err != nil { t.Fatal(err) } - if queryResp, ok = resp.(*roachpb.QueryLocksResponse); !ok { + if queryResp, ok = resp.(*kvpb.QueryLocksResponse); !ok { t.Fatalf("expected QueryLocksResponse, found %v", resp) } @@ -11294,11 +11293,11 @@ func TestReplicaQueryLocks(t *testing.T) { require.Len(t, queryResp.Locks, expectedLen, "expected %d locks in response, got only %d", expectedLen, len(queryResp.Locks)) if includeUncontended && limitResults { require.NotNil(t, resp.Header().ResumeSpan, "expected resume span") - require.Equal(t, roachpb.RESUME_KEY_LIMIT, resp.Header().ResumeReason) + require.Equal(t, kvpb.RESUME_KEY_LIMIT, resp.Header().ResumeReason) require.Equal(t, roachpb.Key("b"), resp.Header().ResumeSpan.Key) } else { require.Nil(t, resp.Header().ResumeSpan) - require.Equal(t, roachpb.RESUME_UNKNOWN, resp.Header().ResumeReason) + require.Equal(t, kvpb.RESUME_UNKNOWN, resp.Header().ResumeReason) } // Validate first lock as held by txn, on key "a", at the correct ts, with a single waiter. @@ -11332,7 +11331,7 @@ func TestReplicaQueryLocks(t *testing.T) { // Update the locked value and commit in a single batch. This should release // the "for update" lock. - ba = &roachpb.BatchRequest{} + ba = &kvpb.BatchRequest{} incArgs := incrementArgs(keyA, 1) et, etH := endTxnArgs(txn, true /* commit */) et.Require1PC = true @@ -11355,12 +11354,12 @@ func TestReplicaQueryLocks(t *testing.T) { if err != nil { t.Fatal(err) } - if queryResp, ok = resp.(*roachpb.QueryLocksResponse); !ok { + if queryResp, ok = resp.(*kvpb.QueryLocksResponse); !ok { t.Fatalf("expected QueryLocksResponse, found %v", resp) } require.Empty(t, queryResp.Locks) require.Nil(t, resp.Header().ResumeSpan) - require.Equal(t, roachpb.RESUME_UNKNOWN, resp.Header().ResumeReason) + require.Equal(t, kvpb.RESUME_UNKNOWN, resp.Header().ResumeReason) }) }) } @@ -11632,13 +11631,13 @@ func TestRangeStatsRequest(t *testing.T) { // returns the same MVCC stats as the replica's in-memory state. WriteRandomDataToRange(t, tc.store, tc.repl.RangeID, keyPrefix) expMS := tc.repl.GetMVCCStats() - res, pErr := kv.SendWrappedWith(ctx, tc.Sender(), roachpb.Header{ + res, pErr := kv.SendWrappedWith(ctx, tc.Sender(), kvpb.Header{ RangeID: tc.repl.RangeID, - }, &roachpb.RangeStatsRequest{}) + }, &kvpb.RangeStatsRequest{}) if pErr != nil { t.Fatal(pErr) } - resMS := res.(*roachpb.RangeStatsResponse).MVCCStats + resMS := res.(*kvpb.RangeStatsResponse).MVCCStats require.Equal(t, expMS, resMS) // Write another key to the range and verify that the MVCC stats returned @@ -11647,13 +11646,13 @@ func TestRangeStatsRequest(t *testing.T) { if err := tc.store.DB().Put(ctx, key, "123"); err != nil { t.Fatal(err) } - res, pErr = kv.SendWrappedWith(ctx, tc.Sender(), roachpb.Header{ + res, pErr = kv.SendWrappedWith(ctx, tc.Sender(), kvpb.Header{ RangeID: tc.repl.RangeID, - }, &roachpb.RangeStatsRequest{}) + }, &kvpb.RangeStatsRequest{}) if pErr != nil { t.Fatal(pErr) } - resMS = res.(*roachpb.RangeStatsResponse).MVCCStats + resMS = res.(*kvpb.RangeStatsResponse).MVCCStats // Only verify the update is reflected in the key/value counts. Verifying // the byte count would couple this test too tightly to our encoding scheme. require.Equal(t, expMS.KeyCount+1, resMS.KeyCount) @@ -11685,7 +11684,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { pusher.Priority = enginepb.MaxTxnPriority type runFunc func(*roachpb.Transaction, hlc.Timestamp) error - sendWrappedWithErr := func(h roachpb.Header, args roachpb.Request) error { + sendWrappedWithErr := func(h kvpb.Header, args kvpb.Request) error { _, pErr := kv.SendWrappedWith(ctx, tc.Sender(), h, args) return pErr.GoError() } @@ -11780,9 +11779,9 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { { name: "push transaction (timestamp)", run: func(txn *roachpb.Transaction, now hlc.Timestamp) error { - pt := pushTxnArgs(pusher, txn, roachpb.PUSH_TIMESTAMP) + pt := pushTxnArgs(pusher, txn, kvpb.PUSH_TIMESTAMP) pt.PushTo = now - return sendWrappedWithErr(roachpb.Header{}, &pt) + return sendWrappedWithErr(kvpb.Header{}, &pt) }, // If no transaction record exists, the push (timestamp) request does // not create one. It only records its push in the tscache. @@ -11791,8 +11790,8 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { { name: "push transaction (abort)", run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { - pt := pushTxnArgs(pusher, txn, roachpb.PUSH_ABORT) - return sendWrappedWithErr(roachpb.Header{}, &pt) + pt := pushTxnArgs(pusher, txn, kvpb.PUSH_ABORT) + return sendWrappedWithErr(kvpb.Header{}, &pt) }, // If no transaction record exists, the push (abort) request does // not create one. It only records its push in the tscache. @@ -11804,7 +11803,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { name: "recover transaction (implicitly committed)", run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { rt := recoverTxnArgs(txn, true /* implicitlyCommitted */) - return sendWrappedWithErr(roachpb.Header{}, &rt) + return sendWrappedWithErr(kvpb.Header{}, &rt) }, expError: "txn record synthesized with non-ABORTED status", expTxn: noTxnRecord, @@ -11815,7 +11814,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { name: "recover transaction (not implicitly committed)", run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { rt := recoverTxnArgs(txn, false /* implicitlyCommitted */) - return sendWrappedWithErr(roachpb.Header{}, &rt) + return sendWrappedWithErr(kvpb.Header{}, &rt) }, expError: "txn record synthesized with non-ABORTED status", expTxn: noTxnRecord, @@ -11931,9 +11930,9 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { return sendWrappedWithErr(hbH, &hb) }, run: func(txn *roachpb.Transaction, now hlc.Timestamp) error { - pt := pushTxnArgs(pusher, txn, roachpb.PUSH_TIMESTAMP) + pt := pushTxnArgs(pusher, txn, kvpb.PUSH_TIMESTAMP) pt.PushTo = now - return sendWrappedWithErr(roachpb.Header{}, &pt) + return sendWrappedWithErr(kvpb.Header{}, &pt) }, // The transaction record **is not** updated in this case. Instead, the // push is communicated through the timestamp cache. When the pushee goes @@ -11948,8 +11947,8 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { return sendWrappedWithErr(hbH, &hb) }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { - pt := pushTxnArgs(pusher, txn, roachpb.PUSH_ABORT) - return sendWrappedWithErr(roachpb.Header{}, &pt) + pt := pushTxnArgs(pusher, txn, kvpb.PUSH_ABORT) + return sendWrappedWithErr(kvpb.Header{}, &pt) }, expTxn: func(txn *roachpb.Transaction, now hlc.Timestamp) roachpb.TransactionRecord { record := txnWithStatus(roachpb.ABORTED)(txn, now) @@ -12147,9 +12146,9 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { return sendWrappedWithErr(etH, &et) }, run: func(txn *roachpb.Transaction, now hlc.Timestamp) error { - pt := pushTxnArgs(pusher, txn, roachpb.PUSH_TIMESTAMP) + pt := pushTxnArgs(pusher, txn, kvpb.PUSH_TIMESTAMP) pt.PushTo = now - return sendWrappedWithErr(roachpb.Header{}, &pt) + return sendWrappedWithErr(kvpb.Header{}, &pt) }, expError: "found txn in indeterminate STAGING state", expTxn: txnWithStagingStatusAndInFlightWrites, @@ -12162,8 +12161,8 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { return sendWrappedWithErr(etH, &et) }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { - pt := pushTxnArgs(pusher, txn, roachpb.PUSH_ABORT) - return sendWrappedWithErr(roachpb.Header{}, &pt) + pt := pushTxnArgs(pusher, txn, kvpb.PUSH_ABORT) + return sendWrappedWithErr(kvpb.Header{}, &pt) }, expError: "found txn in indeterminate STAGING state", expTxn: txnWithStagingStatusAndInFlightWrites, @@ -12181,9 +12180,9 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { run: func(txn *roachpb.Transaction, now hlc.Timestamp) error { clone := txn.Clone() clone.WriteTimestamp = clone.WriteTimestamp.Add(0, 1) - pt := pushTxnArgs(pusher, clone, roachpb.PUSH_TIMESTAMP) + pt := pushTxnArgs(pusher, clone, kvpb.PUSH_TIMESTAMP) pt.PushTo = now - return sendWrappedWithErr(roachpb.Header{}, &pt) + return sendWrappedWithErr(kvpb.Header{}, &pt) }, expTxn: func(txn *roachpb.Transaction, pushTs hlc.Timestamp) roachpb.TransactionRecord { record := txnWithStatus(roachpb.ABORTED)(txn, pushTs) @@ -12205,8 +12204,8 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { run: func(txn *roachpb.Transaction, now hlc.Timestamp) error { clone := txn.Clone() clone.WriteTimestamp = clone.WriteTimestamp.Add(0, 1) - pt := pushTxnArgs(pusher, clone, roachpb.PUSH_ABORT) - return sendWrappedWithErr(roachpb.Header{}, &pt) + pt := pushTxnArgs(pusher, clone, kvpb.PUSH_ABORT) + return sendWrappedWithErr(kvpb.Header{}, &pt) }, expTxn: func(txn *roachpb.Transaction, pushTs hlc.Timestamp) roachpb.TransactionRecord { record := txnWithStatus(roachpb.ABORTED)(txn, pushTs) @@ -12228,9 +12227,9 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { run: func(txn *roachpb.Transaction, now hlc.Timestamp) error { clone := txn.Clone() clone.Restart(-1, 0, clone.WriteTimestamp.Add(0, 1)) - pt := pushTxnArgs(pusher, clone, roachpb.PUSH_TIMESTAMP) + pt := pushTxnArgs(pusher, clone, kvpb.PUSH_TIMESTAMP) pt.PushTo = now - return sendWrappedWithErr(roachpb.Header{}, &pt) + return sendWrappedWithErr(kvpb.Header{}, &pt) }, expTxn: func(txn *roachpb.Transaction, pushTs hlc.Timestamp) roachpb.TransactionRecord { record := txnWithStatus(roachpb.ABORTED)(txn, pushTs) @@ -12253,8 +12252,8 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { run: func(txn *roachpb.Transaction, now hlc.Timestamp) error { clone := txn.Clone() clone.Restart(-1, 0, clone.WriteTimestamp.Add(0, 1)) - pt := pushTxnArgs(pusher, clone, roachpb.PUSH_ABORT) - return sendWrappedWithErr(roachpb.Header{}, &pt) + pt := pushTxnArgs(pusher, clone, kvpb.PUSH_ABORT) + return sendWrappedWithErr(kvpb.Header{}, &pt) }, expTxn: func(txn *roachpb.Transaction, pushTs hlc.Timestamp) roachpb.TransactionRecord { record := txnWithStatus(roachpb.ABORTED)(txn, pushTs) @@ -12346,9 +12345,9 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { return sendWrappedWithErr(etH, &et) }, run: func(txn *roachpb.Transaction, now hlc.Timestamp) error { - pt := pushTxnArgs(pusher, txn, roachpb.PUSH_TIMESTAMP) + pt := pushTxnArgs(pusher, txn, kvpb.PUSH_TIMESTAMP) pt.PushTo = now - return sendWrappedWithErr(roachpb.Header{}, &pt) + return sendWrappedWithErr(kvpb.Header{}, &pt) }, expTxn: noTxnRecord, }, @@ -12359,8 +12358,8 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { return sendWrappedWithErr(etH, &et) }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { - pt := pushTxnArgs(pusher, txn, roachpb.PUSH_ABORT) - return sendWrappedWithErr(roachpb.Header{}, &pt) + pt := pushTxnArgs(pusher, txn, kvpb.PUSH_ABORT) + return sendWrappedWithErr(kvpb.Header{}, &pt) }, expTxn: noTxnRecord, }, @@ -12427,9 +12426,9 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { return sendWrappedWithErr(etH, &et) }, run: func(txn *roachpb.Transaction, now hlc.Timestamp) error { - pt := pushTxnArgs(pusher, txn, roachpb.PUSH_TIMESTAMP) + pt := pushTxnArgs(pusher, txn, kvpb.PUSH_TIMESTAMP) pt.PushTo = now - return sendWrappedWithErr(roachpb.Header{}, &pt) + return sendWrappedWithErr(kvpb.Header{}, &pt) }, expTxn: noTxnRecord, }, @@ -12440,8 +12439,8 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { return sendWrappedWithErr(etH, &et) }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { - pt := pushTxnArgs(pusher, txn, roachpb.PUSH_ABORT) - return sendWrappedWithErr(roachpb.Header{}, &pt) + pt := pushTxnArgs(pusher, txn, kvpb.PUSH_ABORT) + return sendWrappedWithErr(kvpb.Header{}, &pt) }, expTxn: noTxnRecord, }, @@ -12515,9 +12514,9 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { return sendWrappedWithErr(etH, &et) }, run: func(txn *roachpb.Transaction, now hlc.Timestamp) error { - pt := pushTxnArgs(pusher, txn, roachpb.PUSH_TIMESTAMP) + pt := pushTxnArgs(pusher, txn, kvpb.PUSH_TIMESTAMP) pt.PushTo = now - return sendWrappedWithErr(roachpb.Header{}, &pt) + return sendWrappedWithErr(kvpb.Header{}, &pt) }, expTxn: txnWithStatus(roachpb.ABORTED), disableTxnAutoGC: true, @@ -12529,8 +12528,8 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { return sendWrappedWithErr(etH, &et) }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { - pt := pushTxnArgs(pusher, txn, roachpb.PUSH_ABORT) - return sendWrappedWithErr(roachpb.Header{}, &pt) + pt := pushTxnArgs(pusher, txn, kvpb.PUSH_ABORT) + return sendWrappedWithErr(kvpb.Header{}, &pt) }, expTxn: txnWithStatus(roachpb.ABORTED), disableTxnAutoGC: true, @@ -12606,9 +12605,9 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { return sendWrappedWithErr(etH, &et) }, run: func(txn *roachpb.Transaction, now hlc.Timestamp) error { - pt := pushTxnArgs(pusher, txn, roachpb.PUSH_TIMESTAMP) + pt := pushTxnArgs(pusher, txn, kvpb.PUSH_TIMESTAMP) pt.PushTo = now - return sendWrappedWithErr(roachpb.Header{}, &pt) + return sendWrappedWithErr(kvpb.Header{}, &pt) }, expTxn: txnWithStatus(roachpb.COMMITTED), disableTxnAutoGC: true, @@ -12620,8 +12619,8 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { return sendWrappedWithErr(etH, &et) }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { - pt := pushTxnArgs(pusher, txn, roachpb.PUSH_ABORT) - return sendWrappedWithErr(roachpb.Header{}, &pt) + pt := pushTxnArgs(pusher, txn, kvpb.PUSH_ABORT) + return sendWrappedWithErr(kvpb.Header{}, &pt) }, expTxn: txnWithStatus(roachpb.COMMITTED), disableTxnAutoGC: true, @@ -12629,9 +12628,9 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { { name: "heartbeat transaction after push transaction (timestamp)", setup: func(txn *roachpb.Transaction, now hlc.Timestamp) error { - pt := pushTxnArgs(pusher, txn, roachpb.PUSH_TIMESTAMP) + pt := pushTxnArgs(pusher, txn, kvpb.PUSH_TIMESTAMP) pt.PushTo = now - return sendWrappedWithErr(roachpb.Header{}, &pt) + return sendWrappedWithErr(kvpb.Header{}, &pt) }, run: func(txn *roachpb.Transaction, now hlc.Timestamp) error { hb, hbH := heartbeatArgs(txn, now.Add(0, 5)) @@ -12647,9 +12646,9 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { { name: "end transaction (stage) after push transaction (timestamp)", setup: func(txn *roachpb.Transaction, now hlc.Timestamp) error { - pt := pushTxnArgs(pusher, txn, roachpb.PUSH_TIMESTAMP) + pt := pushTxnArgs(pusher, txn, kvpb.PUSH_TIMESTAMP) pt.PushTo = now - return sendWrappedWithErr(roachpb.Header{}, &pt) + return sendWrappedWithErr(kvpb.Header{}, &pt) }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { et, etH := endTxnArgs(txn, true /* commit */) @@ -12664,9 +12663,9 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { { name: "end transaction (abort) after push transaction (timestamp)", setup: func(txn *roachpb.Transaction, now hlc.Timestamp) error { - pt := pushTxnArgs(pusher, txn, roachpb.PUSH_TIMESTAMP) + pt := pushTxnArgs(pusher, txn, kvpb.PUSH_TIMESTAMP) pt.PushTo = now - return sendWrappedWithErr(roachpb.Header{}, &pt) + return sendWrappedWithErr(kvpb.Header{}, &pt) }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { et, etH := endTxnArgs(txn, false /* commit */) @@ -12679,9 +12678,9 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { { name: "end transaction (commit) after push transaction (timestamp)", setup: func(txn *roachpb.Transaction, now hlc.Timestamp) error { - pt := pushTxnArgs(pusher, txn, roachpb.PUSH_TIMESTAMP) + pt := pushTxnArgs(pusher, txn, kvpb.PUSH_TIMESTAMP) pt.PushTo = now - return sendWrappedWithErr(roachpb.Header{}, &pt) + return sendWrappedWithErr(kvpb.Header{}, &pt) }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { et, etH := endTxnArgs(txn, true /* commit */) @@ -12695,9 +12694,9 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { { name: "end transaction (one-phase commit) after push transaction (timestamp)", setup: func(txn *roachpb.Transaction, now hlc.Timestamp) error { - pt := pushTxnArgs(pusher, txn, roachpb.PUSH_TIMESTAMP) + pt := pushTxnArgs(pusher, txn, kvpb.PUSH_TIMESTAMP) pt.PushTo = now - return sendWrappedWithErr(roachpb.Header{}, &pt) + return sendWrappedWithErr(kvpb.Header{}, &pt) }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { et, etH := endTxnArgs(txn, true /* commit */) @@ -12712,9 +12711,9 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { { name: "end transaction (one-phase commit) after push transaction (abort)", setup: func(txn *roachpb.Transaction, now hlc.Timestamp) error { - pt := pushTxnArgs(pusher, txn, roachpb.PUSH_ABORT) + pt := pushTxnArgs(pusher, txn, kvpb.PUSH_ABORT) pt.PushTo = now - return sendWrappedWithErr(roachpb.Header{}, &pt) + return sendWrappedWithErr(kvpb.Header{}, &pt) }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { et, etH := endTxnArgs(txn, true /* commit */) @@ -12762,8 +12761,8 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { { name: "heartbeat transaction after push transaction (abort)", setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { - pt := pushTxnArgs(pusher, txn, roachpb.PUSH_ABORT) - return sendWrappedWithErr(roachpb.Header{}, &pt) + pt := pushTxnArgs(pusher, txn, kvpb.PUSH_ABORT) + return sendWrappedWithErr(kvpb.Header{}, &pt) }, run: func(txn *roachpb.Transaction, now hlc.Timestamp) error { hb, hbH := heartbeatArgs(txn, now) @@ -12775,8 +12774,8 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { { name: "heartbeat transaction with epoch bump after push transaction (abort)", setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { - pt := pushTxnArgs(pusher, txn, roachpb.PUSH_ABORT) - return sendWrappedWithErr(roachpb.Header{}, &pt) + pt := pushTxnArgs(pusher, txn, kvpb.PUSH_ABORT) + return sendWrappedWithErr(kvpb.Header{}, &pt) }, run: func(txn *roachpb.Transaction, now hlc.Timestamp) error { // Restart the transaction at a higher timestamp. This will @@ -12794,8 +12793,8 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { { name: "end transaction (stage) after push transaction (abort)", setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { - pt := pushTxnArgs(pusher, txn, roachpb.PUSH_ABORT) - return sendWrappedWithErr(roachpb.Header{}, &pt) + pt := pushTxnArgs(pusher, txn, kvpb.PUSH_ABORT) + return sendWrappedWithErr(kvpb.Header{}, &pt) }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { et, etH := endTxnArgs(txn, true /* commit */) @@ -12808,8 +12807,8 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { { name: "end transaction (abort) after push transaction (abort)", setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { - pt := pushTxnArgs(pusher, txn, roachpb.PUSH_ABORT) - return sendWrappedWithErr(roachpb.Header{}, &pt) + pt := pushTxnArgs(pusher, txn, kvpb.PUSH_ABORT) + return sendWrappedWithErr(kvpb.Header{}, &pt) }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { et, etH := endTxnArgs(txn, false /* commit */) @@ -12822,8 +12821,8 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { { name: "end transaction (commit) after push transaction (abort)", setup: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { - pt := pushTxnArgs(pusher, txn, roachpb.PUSH_ABORT) - return sendWrappedWithErr(roachpb.Header{}, &pt) + pt := pushTxnArgs(pusher, txn, kvpb.PUSH_ABORT) + return sendWrappedWithErr(kvpb.Header{}, &pt) }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { et, etH := endTxnArgs(txn, true /* commit */) @@ -12841,7 +12840,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { rt := recoverTxnArgs(txn, true /* implicitlyCommitted */) - return sendWrappedWithErr(roachpb.Header{}, &rt) + return sendWrappedWithErr(kvpb.Header{}, &rt) }, expError: "found PENDING record for implicitly committed transaction", expTxn: txnWithoutChanges, @@ -12857,7 +12856,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { rt := recoverTxnArgs(txn, true /* implicitlyCommitted */) - return sendWrappedWithErr(roachpb.Header{}, &rt) + return sendWrappedWithErr(kvpb.Header{}, &rt) }, expTxn: func(txn *roachpb.Transaction, now hlc.Timestamp) roachpb.TransactionRecord { record := txnWithStatus(roachpb.COMMITTED)(txn, now) @@ -12879,7 +12878,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { rt := recoverTxnArgs(txn, true /* implicitlyCommitted */) - return sendWrappedWithErr(roachpb.Header{}, &rt) + return sendWrappedWithErr(kvpb.Header{}, &rt) }, expError: "timestamp change by implicitly committed transaction", expTxn: func(txn *roachpb.Transaction, now hlc.Timestamp) roachpb.TransactionRecord { @@ -12900,7 +12899,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { rt := recoverTxnArgs(txn, true /* implicitlyCommitted */) - return sendWrappedWithErr(roachpb.Header{}, &rt) + return sendWrappedWithErr(kvpb.Header{}, &rt) }, expError: "epoch change by implicitly committed transaction", expTxn: func(txn *roachpb.Transaction, now hlc.Timestamp) roachpb.TransactionRecord { @@ -12919,7 +12918,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { rt := recoverTxnArgs(txn, true /* implicitlyCommitted */) - return sendWrappedWithErr(roachpb.Header{}, &rt) + return sendWrappedWithErr(kvpb.Header{}, &rt) }, // The transaction record was cleaned up, so RecoverTxn can't perform // the same assertion that it does in the case without eager gc. @@ -12935,7 +12934,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { rt := recoverTxnArgs(txn, true /* implicitlyCommitted */) - return sendWrappedWithErr(roachpb.Header{}, &rt) + return sendWrappedWithErr(kvpb.Header{}, &rt) }, expTxn: noTxnRecord, }, @@ -12948,7 +12947,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { rt := recoverTxnArgs(txn, true /* implicitlyCommitted */) - return sendWrappedWithErr(roachpb.Header{}, &rt) + return sendWrappedWithErr(kvpb.Header{}, &rt) }, expError: "found ABORTED record for implicitly committed transaction", expTxn: txnWithStatus(roachpb.ABORTED), @@ -12964,7 +12963,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { rt := recoverTxnArgs(txn, true /* implicitlyCommitted */) - return sendWrappedWithErr(roachpb.Header{}, &rt) + return sendWrappedWithErr(kvpb.Header{}, &rt) }, expTxn: txnWithStatus(roachpb.COMMITTED), disableTxnAutoGC: true, @@ -12978,7 +12977,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { rt := recoverTxnArgs(txn, false /* implicitlyCommitted */) - return sendWrappedWithErr(roachpb.Header{}, &rt) + return sendWrappedWithErr(kvpb.Header{}, &rt) }, expError: "cannot recover PENDING transaction", expTxn: txnWithoutChanges, @@ -12995,7 +12994,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { rt := recoverTxnArgs(txn, false /* implicitlyCommitted */) - return sendWrappedWithErr(roachpb.Header{}, &rt) + return sendWrappedWithErr(kvpb.Header{}, &rt) }, expTxn: func(txn *roachpb.Transaction, now hlc.Timestamp) roachpb.TransactionRecord { record := txn.AsRecord() @@ -13015,7 +13014,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { rt := recoverTxnArgs(txn, false /* implicitlyCommitted */) - return sendWrappedWithErr(roachpb.Header{}, &rt) + return sendWrappedWithErr(kvpb.Header{}, &rt) }, expTxn: func(txn *roachpb.Transaction, now hlc.Timestamp) roachpb.TransactionRecord { record := txnWithStatus(roachpb.ABORTED)(txn, now) @@ -13040,7 +13039,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { rt := recoverTxnArgs(txn, false /* implicitlyCommitted */) - return sendWrappedWithErr(roachpb.Header{}, &rt) + return sendWrappedWithErr(kvpb.Header{}, &rt) }, expTxn: func(txn *roachpb.Transaction, now hlc.Timestamp) roachpb.TransactionRecord { // Unchanged by the RecoverTxn request. @@ -13064,7 +13063,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { rt := recoverTxnArgs(txn, false /* implicitlyCommitted */) - return sendWrappedWithErr(roachpb.Header{}, &rt) + return sendWrappedWithErr(kvpb.Header{}, &rt) }, expTxn: func(txn *roachpb.Transaction, now hlc.Timestamp) roachpb.TransactionRecord { record := txnWithStagingStatusAndInFlightWrites(txn, now) @@ -13084,7 +13083,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { rt := recoverTxnArgs(txn, false /* implicitlyCommitted */) - return sendWrappedWithErr(roachpb.Header{}, &rt) + return sendWrappedWithErr(kvpb.Header{}, &rt) }, expTxn: noTxnRecord, }, @@ -13097,7 +13096,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { rt := recoverTxnArgs(txn, false /* implicitlyCommitted */) - return sendWrappedWithErr(roachpb.Header{}, &rt) + return sendWrappedWithErr(kvpb.Header{}, &rt) }, // The transaction record was cleaned up, so RecoverTxn can't perform // the same assertion that it does in the case without eager gc. @@ -13114,7 +13113,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { rt := recoverTxnArgs(txn, false /* implicitlyCommitted */) - return sendWrappedWithErr(roachpb.Header{}, &rt) + return sendWrappedWithErr(kvpb.Header{}, &rt) }, expTxn: txnWithStatus(roachpb.ABORTED), disableTxnAutoGC: true, @@ -13134,7 +13133,7 @@ func TestTxnRecordLifecycleTransitions(t *testing.T) { }, run: func(txn *roachpb.Transaction, _ hlc.Timestamp) error { rt := recoverTxnArgs(txn, false /* implicitlyCommitted */) - return sendWrappedWithErr(roachpb.Header{}, &rt) + return sendWrappedWithErr(kvpb.Header{}, &rt) }, expTxn: txnWithStatus(roachpb.COMMITTED), disableTxnAutoGC: true, @@ -13208,11 +13207,11 @@ func TestRollbackMissingTxnRecordNoError(t *testing.T) { key := roachpb.Key("bogus key") txn := newTransaction("test", key, roachpb.NormalUserPriority, tc.Clock()) - res, pErr := kv.SendWrappedWith(ctx, tc.Sender(), roachpb.Header{ + res, pErr := kv.SendWrappedWith(ctx, tc.Sender(), kvpb.Header{ RangeID: tc.repl.RangeID, Txn: txn, - }, &roachpb.EndTxnRequest{ - RequestHeader: roachpb.RequestHeader{ + }, &kvpb.EndTxnRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, }, Commit: false, @@ -13226,11 +13225,11 @@ func TestRollbackMissingTxnRecordNoError(t *testing.T) { // For good measure, let's take the opportunity to check replay protection for // a HeartbeatTxn arriving after the rollback. - _, pErr = kv.SendWrappedWith(ctx, tc.Sender(), roachpb.Header{ + _, pErr = kv.SendWrappedWith(ctx, tc.Sender(), kvpb.Header{ RangeID: tc.repl.RangeID, Txn: txn, - }, &roachpb.HeartbeatTxnRequest{ - RequestHeader: roachpb.RequestHeader{ + }, &kvpb.HeartbeatTxnRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, }, Now: tc.Clock().Now(), @@ -13345,7 +13344,7 @@ func TestProposalNotAcknowledgedOrReproposedAfterApplication(t *testing.T) { EnableUnconditionalRefreshesInRaftReady: true, // Set the TestingProposalFilter in order to know the CmdIDKey for our // request by detecting its txnID. - TestingProposalFilter: func(args kvserverbase.ProposalFilterArgs) *roachpb.Error { + TestingProposalFilter: func(args kvserverbase.ProposalFilterArgs) *kvpb.Error { if args.Req.Header.Txn != nil && args.Req.Header.Txn.ID == txnID { cmdID = args.CmdID } @@ -13353,7 +13352,7 @@ func TestProposalNotAcknowledgedOrReproposedAfterApplication(t *testing.T) { }, // Detect the application of the proposal to repropose it and also // invalidate the lease. - TestingApplyCalledTwiceFilter: func(args kvserverbase.ApplyFilterArgs) (retry int, pErr *roachpb.Error) { + TestingApplyCalledTwiceFilter: func(args kvserverbase.ApplyFilterArgs) (retry int, pErr *kvpb.Error) { if seen || args.CmdID != cmdID { return 0, nil } @@ -13374,7 +13373,7 @@ func TestProposalNotAcknowledgedOrReproposedAfterApplication(t *testing.T) { // reproposal by ensuring that no event is recorded after the original // proposal has been finished. return int(kvserverbase.ProposalRejectionIllegalLeaseIndex), - roachpb.NewErrorf("forced error that can be reproposed at a higher index") + kvpb.NewErrorf("forced error that can be reproposed at a higher index") }, } tc.StartWithStoreConfig(ctx, t, stopper, cfg) @@ -13382,15 +13381,15 @@ func TestProposalNotAcknowledgedOrReproposedAfterApplication(t *testing.T) { st := tc.repl.CurrentLeaseStatus(ctx) txn := newTransaction("test", key, roachpb.NormalUserPriority, tc.Clock()) txnID = txn.ID - ba := &roachpb.BatchRequest{ - Header: roachpb.Header{ + ba := &kvpb.BatchRequest{ + Header: kvpb.Header{ RangeID: tc.repl.RangeID, Txn: txn, }, } ba.Timestamp = txn.ReadTimestamp - ba.Add(&roachpb.PutRequest{ - RequestHeader: roachpb.RequestHeader{ + ba.Add(&kvpb.PutRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, }, Value: roachpb.MakeValueFromBytes([]byte("val")), @@ -13406,7 +13405,7 @@ func TestProposalNotAcknowledgedOrReproposedAfterApplication(t *testing.T) { if pErr != nil { t.Fatal(pErr) } - errCh := make(chan *roachpb.Error) + errCh := make(chan *kvpb.Error) go func() { res := <-ch sp.Finish() @@ -13456,7 +13455,7 @@ func TestReplicaTelemetryCounterForPushesDueToClosedTimestamp(t *testing.T) { keyA := append(keys.SystemSQLCodec.TablePrefix(math.MaxUint32), 'a') keyAA := append(keyA[:len(keyA):len(keyA)], 'a') rKeyA, err := keys.Addr(keyA) - putReq := func(key roachpb.Key) *roachpb.PutRequest { + putReq := func(key roachpb.Key) *kvpb.PutRequest { r := putArgs(key, []byte("foo")) return &r } @@ -13480,7 +13479,7 @@ func TestReplicaTelemetryCounterForPushesDueToClosedTimestamp(t *testing.T) { { // Test the case where no bump occurs. name: "no bump", f: func(t *testing.T, r *Replica) { - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Add(putReq(keyA)) minReadTS := r.store.Clock().Now() ba.Timestamp = minReadTS.Next() @@ -13491,7 +13490,7 @@ func TestReplicaTelemetryCounterForPushesDueToClosedTimestamp(t *testing.T) { { // Test the case where the bump occurs due to minReadTS. name: "bump due to minTS", f: func(t *testing.T, r *Replica) { - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Add(putReq(keyA)) ba.Timestamp = r.store.Clock().Now() minReadTS := ba.Timestamp.Next() @@ -13502,7 +13501,7 @@ func TestReplicaTelemetryCounterForPushesDueToClosedTimestamp(t *testing.T) { { // Test the case where we bump due to the read ts cache rather than the minReadTS. name: "bump due to later read ts cache entry", f: func(t *testing.T, r *Replica) { - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Add(putReq(keyA)) ba.Timestamp = r.store.Clock().Now() minReadTS := ba.Timestamp.Next() @@ -13515,7 +13514,7 @@ func TestReplicaTelemetryCounterForPushesDueToClosedTimestamp(t *testing.T) { // Test the case where we do initially bump due to the minReadTS but then // bump again to a higher ts due to the read ts cache. name: "higher bump due to read ts cache entry", f: func(t *testing.T, r *Replica) { - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Add(putReq(keyA)) ba.Add(putReq(keyAA)) ba.Timestamp = r.store.Clock().Now() @@ -13566,7 +13565,7 @@ func TestContainsEstimatesClampProposal(t *testing.T) { someRequestToProposal := func(tc *testContext, ctx context.Context) *ProposalData { cmdIDKey := kvserverbase.CmdIDKey("some-cmdid-key") - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Timestamp = tc.Clock().Now() req := putArgs(roachpb.Key("some-key"), []byte("some-value")) ba.Add(&req) @@ -13601,23 +13600,23 @@ func TestContainsEstimatesClampProposal(t *testing.T) { // setMockPutWithEstimates mocks the Put command (could be any) to simulate a command // that touches ContainsEstimates, in order to test request proposal behavior. func setMockPutWithEstimates(containsEstimatesDelta int64) (undo func()) { - prev, _ := batcheval.LookupCommand(roachpb.Put) + prev, _ := batcheval.LookupCommand(kvpb.Put) mockPut := func( - ctx context.Context, readWriter storage.ReadWriter, cArgs batcheval.CommandArgs, _ roachpb.Response, + ctx context.Context, readWriter storage.ReadWriter, cArgs batcheval.CommandArgs, _ kvpb.Response, ) (result.Result, error) { - args := cArgs.Args.(*roachpb.PutRequest) + args := cArgs.Args.(*kvpb.PutRequest) ms := cArgs.Stats ms.ContainsEstimates += containsEstimatesDelta ts := cArgs.Header.Timestamp return result.Result{}, storage.MVCCBlindPut(ctx, readWriter, ms, args.Key, ts, hlc.ClockTimestamp{}, args.Value, cArgs.Header.Txn) } - batcheval.UnregisterCommand(roachpb.Put) - batcheval.RegisterReadWriteCommand(roachpb.Put, batcheval.DefaultDeclareIsolatedKeys, mockPut) + batcheval.UnregisterCommand(kvpb.Put) + batcheval.RegisterReadWriteCommand(kvpb.Put, batcheval.DefaultDeclareIsolatedKeys, mockPut) return func() { - batcheval.UnregisterCommand(roachpb.Put) - batcheval.RegisterReadWriteCommand(roachpb.Put, prev.DeclareKeys, prev.EvalRW) + batcheval.UnregisterCommand(kvpb.Put) + batcheval.RegisterReadWriteCommand(kvpb.Put, prev.DeclareKeys, prev.EvalRW) } } @@ -13853,7 +13852,7 @@ func TestRangeInfoReturned(t *testing.T) { }, } { t.Run("", func(t *testing.T) { - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Add(&gArgs) ba.Header.ClientRangeInfo = test.req br, pErr := tc.Sender().Send(ctx, ba) @@ -13951,8 +13950,8 @@ func TestStoreTenantMetricsAndRateLimiterRefcount(t *testing.T) { ) // The sole range owned by tenant 123 gets merged away again. - _, pErr := leftRepl.AdminMerge(context.Background(), roachpb.AdminMergeRequest{ - RequestHeader: roachpb.RequestHeader{ + _, pErr := leftRepl.AdminMerge(context.Background(), kvpb.AdminMergeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: leftRepl.Desc().StartKey.AsRawKey(), }, }, "testing") @@ -14027,7 +14026,7 @@ func TestRangeSplitRacesWithRead(t *testing.T) { now := tc.Server(0).Clock().Now() ts1 := now.Add(1, 0) - h1 := roachpb.Header{RangeID: desc.RangeID, Timestamp: ts1} + h1 := kvpb.Header{RangeID: desc.RangeID, Timestamp: ts1} val := []byte("value") for _, k := range [][]byte{keyA, keyC} { @@ -14044,11 +14043,11 @@ func TestRangeSplitRacesWithRead(t *testing.T) { require.NoError(t, err) testutils.SucceedsSoon(t, func() error { - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.RangeID = desc.RangeID - ba.ReadConsistency = roachpb.INCONSISTENT - ba.Add(&roachpb.QueryResolvedTimestampRequest{ - RequestHeader: roachpb.RequestHeader{Key: key, EndKey: key.Next()}, + ba.ReadConsistency = kvpb.INCONSISTENT + ba.Add(&kvpb.QueryResolvedTimestampRequest{ + RequestHeader: kvpb.RequestHeader{Key: key, EndKey: key.Next()}, }) br, pErr := reader.Send(ctx, ba) require.Nil(t, pErr) @@ -14067,13 +14066,13 @@ func TestRangeSplitRacesWithRead(t *testing.T) { if pErr == nil { t.Logf("read won the race: %v", resp) require.NotNil(t, resp) - res := resp.(*roachpb.ScanResponse).Rows + res := resp.(*kvpb.ScanResponse).Rows require.Equal(t, 2, len(res)) require.Equal(t, keyA, res[0].Key) require.Equal(t, keyC, res[1].Key) } else { t.Logf("read lost the race: %v", pErr) - mismatchErr := &roachpb.RangeKeyMismatchError{} + mismatchErr := &kvpb.RangeKeyMismatchError{} require.ErrorAs(t, pErr.GoError(), &mismatchErr) return } @@ -14081,8 +14080,8 @@ func TestRangeSplitRacesWithRead(t *testing.T) { } split := func() { - splitArgs := &roachpb.AdminSplitRequest{ - RequestHeader: roachpb.RequestHeader{ + splitArgs := &kvpb.AdminSplitRequest{ + RequestHeader: kvpb.RequestHeader{ Key: splitKey, }, SplitKey: splitKey, @@ -14169,7 +14168,7 @@ func TestRangeSplitAndRHSRemovalRacesWithFollowerRead(t *testing.T) { now := tc.Server(0).Clock().Now() ts1 := now.Add(1, 0) - h1 := roachpb.Header{RangeID: desc.RangeID, Timestamp: ts1} + h1 := kvpb.Header{RangeID: desc.RangeID, Timestamp: ts1} val := []byte("value") for _, k := range [][]byte{keyA, keyC} { @@ -14185,11 +14184,11 @@ func TestRangeSplitAndRHSRemovalRacesWithFollowerRead(t *testing.T) { require.NoError(t, err) testutils.SucceedsSoon(t, func() error { - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.RangeID = desc.RangeID - ba.ReadConsistency = roachpb.INCONSISTENT - ba.Add(&roachpb.QueryResolvedTimestampRequest{ - RequestHeader: roachpb.RequestHeader{Key: key, EndKey: key.Next()}, + ba.ReadConsistency = kvpb.INCONSISTENT + ba.Add(&kvpb.QueryResolvedTimestampRequest{ + RequestHeader: kvpb.RequestHeader{Key: key, EndKey: key.Next()}, }) br, pErr := reader.Send(ctx, ba) require.Nil(t, pErr) @@ -14208,7 +14207,7 @@ func TestRangeSplitAndRHSRemovalRacesWithFollowerRead(t *testing.T) { scanArgs := scanArgs(keyA, keyD) _, pErr := kv.SendWrappedWith(ctx, reader, h1, scanArgs) require.NotNil(t, pErr) - mismatchErr := &roachpb.RangeKeyMismatchError{} + mismatchErr := &kvpb.RangeKeyMismatchError{} require.ErrorAs(t, pErr.GoError(), &mismatchErr) } @@ -14218,8 +14217,8 @@ func TestRangeSplitAndRHSRemovalRacesWithFollowerRead(t *testing.T) { case <-time.After(5 * time.Second): panic("timed out waiting for read to block") } - splitArgs := &roachpb.AdminSplitRequest{ - RequestHeader: roachpb.RequestHeader{ + splitArgs := &kvpb.AdminSplitRequest{ + RequestHeader: kvpb.RequestHeader{ Key: splitKey, }, SplitKey: splitKey, diff --git a/pkg/kv/kvserver/replica_tscache.go b/pkg/kv/kvserver/replica_tscache.go index 9023ad570e55..fb1157401589 100644 --- a/pkg/kv/kvserver/replica_tscache.go +++ b/pkg/kv/kvserver/replica_tscache.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb" @@ -33,9 +34,9 @@ import ( func (r *Replica) addToTSCacheChecked( ctx context.Context, st *kvserverpb.LeaseStatus, - ba *roachpb.BatchRequest, - br *roachpb.BatchResponse, - pErr *roachpb.Error, + ba *kvpb.BatchRequest, + br *kvpb.BatchResponse, + pErr *kvpb.Error, start, end roachpb.Key, ts hlc.Timestamp, txnID uuid.UUID, @@ -60,9 +61,9 @@ func (r *Replica) addToTSCacheChecked( func (r *Replica) updateTimestampCache( ctx context.Context, st *kvserverpb.LeaseStatus, - ba *roachpb.BatchRequest, - br *roachpb.BatchResponse, - pErr *roachpb.Error, + ba *kvpb.BatchRequest, + br *kvpb.BatchResponse, + pErr *kvpb.Error, ) { addToTSCache := func(start, end roachpb.Key, ts hlc.Timestamp, txnID uuid.UUID) { r.addToTSCacheChecked(ctx, st, ba, br, pErr, start, end, ts, txnID) @@ -81,17 +82,17 @@ func (r *Replica) updateTimestampCache( beforeEval := br == nil && pErr == nil for i, union := range ba.Requests { req := union.GetInner() - if !roachpb.UpdatesTimestampCache(req) { + if !kvpb.UpdatesTimestampCache(req) { continue } - var resp roachpb.Response + var resp kvpb.Response if br != nil { resp = br.Responses[i].GetInner() } // Skip update if there's an error and it's not for this index // or the request doesn't update the timestamp cache on errors. if pErr != nil { - if index := pErr.Index; !roachpb.UpdatesTimestampCacheOnError(req) || + if index := pErr.Index; !kvpb.UpdatesTimestampCacheOnError(req) || index == nil || int32(i) != index.Index { continue } @@ -99,7 +100,7 @@ func (r *Replica) updateTimestampCache( header := req.Header() start, end := header.Key, header.EndKey - if ba.WaitPolicy == lock.WaitPolicy_SkipLocked && roachpb.CanSkipLocked(req) { + if ba.WaitPolicy == lock.WaitPolicy_SkipLocked && kvpb.CanSkipLocked(req) { if ba.IndexFetchSpec != nil { log.Errorf(ctx, "%v", errors.AssertionFailedf("unexpectedly IndexFetchSpec is set with SKIP LOCKED wait policy")) } @@ -117,7 +118,7 @@ func (r *Replica) updateTimestampCache( // [Scan("a", "e")] -> returning ["a", "c"] // to that of a set of point read requests like: // [Get("a"), Get("c")] - if err := roachpb.ResponseKeyIterate(req, resp, func(key roachpb.Key) { + if err := kvpb.ResponseKeyIterate(req, resp, func(key roachpb.Key) { addToTSCache(key, nil, ts, txnID) }); err != nil { log.Errorf(ctx, "error iterating over response keys while "+ @@ -127,7 +128,7 @@ func (r *Replica) updateTimestampCache( } switch t := req.(type) { - case *roachpb.EndTxnRequest: + case *kvpb.EndTxnRequest: // EndTxn requests record a tombstone in the timestamp cache to ensure // replays and concurrent requests aren't able to recreate the transaction // record. @@ -137,13 +138,13 @@ func (r *Replica) updateTimestampCache( // transaction's MinTimestamp, which is consulted in CanCreateTxnRecord. key := transactionTombstoneMarker(start, txnID) addToTSCache(key, nil, ts, txnID) - case *roachpb.HeartbeatTxnRequest: + case *kvpb.HeartbeatTxnRequest: // HeartbeatTxn requests record a tombstone entry when the record is // initially written. This is used when considering potential 1PC // evaluation, avoiding checking for a transaction record on disk. key := transactionTombstoneMarker(start, txnID) addToTSCache(key, nil, ts, txnID) - case *roachpb.RecoverTxnRequest: + case *kvpb.RecoverTxnRequest: // A successful RecoverTxn request may or may not have finalized the // transaction that it was trying to recover. If so, then we record // a tombstone to the timestamp cache to ensure that replays and @@ -153,12 +154,12 @@ func (r *Replica) updateTimestampCache( // Insert the timestamp of the batch, which we asserted during // command evaluation was equal to or greater than the transaction's // MinTimestamp. - recovered := resp.(*roachpb.RecoverTxnResponse).RecoveredTxn + recovered := resp.(*kvpb.RecoverTxnResponse).RecoveredTxn if recovered.Status.IsFinalized() { key := transactionTombstoneMarker(start, recovered.ID) addToTSCache(key, nil, ts, recovered.ID) } - case *roachpb.PushTxnRequest: + case *kvpb.PushTxnRequest: // A successful PushTxn request bumps the timestamp cache for the // pushee's transaction key. The pushee will consult the timestamp // cache when creating its record - see CanCreateTxnRecord. @@ -174,7 +175,7 @@ func (r *Replica) updateTimestampCache( // then we add a "tombstone" marker to the timestamp cache wth the // pushee's minimum timestamp. This will prevent the creation of the // transaction record entirely. - pushee := resp.(*roachpb.PushTxnResponse).PusheeTxn + pushee := resp.(*kvpb.PushTxnResponse).PusheeTxn var tombstone bool switch pushee.Status { @@ -212,29 +213,29 @@ func (r *Replica) updateTimestampCache( pushTS = pushee.WriteTimestamp } addToTSCache(key, nil, pushTS, t.PusherTxn.ID) - case *roachpb.ConditionalPutRequest: + case *kvpb.ConditionalPutRequest: // ConditionalPut only updates on ConditionFailedErrors. On other // errors, no information is returned. On successful writes, the // intent already protects against writes underneath the read. - if _, ok := pErr.GetDetail().(*roachpb.ConditionFailedError); ok { + if _, ok := pErr.GetDetail().(*kvpb.ConditionFailedError); ok { addToTSCache(start, end, ts, txnID) } - case *roachpb.InitPutRequest: + case *kvpb.InitPutRequest: // InitPut only updates on ConditionFailedErrors. On other errors, // no information is returned. On successful writes, the intent // already protects against writes underneath the read. - if _, ok := pErr.GetDetail().(*roachpb.ConditionFailedError); ok { + if _, ok := pErr.GetDetail().(*kvpb.ConditionFailedError); ok { addToTSCache(start, end, ts, txnID) } - case *roachpb.GetRequest: - if !beforeEval && resp.(*roachpb.GetResponse).ResumeSpan != nil { + case *kvpb.GetRequest: + if !beforeEval && resp.(*kvpb.GetResponse).ResumeSpan != nil { // The request did not evaluate. Ignore it. continue } addToTSCache(start, end, ts, txnID) - case *roachpb.ScanRequest: - if !beforeEval && resp.(*roachpb.ScanResponse).ResumeSpan != nil { - resume := resp.(*roachpb.ScanResponse).ResumeSpan + case *kvpb.ScanRequest: + if !beforeEval && resp.(*kvpb.ScanResponse).ResumeSpan != nil { + resume := resp.(*kvpb.ScanResponse).ResumeSpan if start.Equal(resume.Key) { // The request did not evaluate. Ignore it. continue @@ -245,9 +246,9 @@ func (r *Replica) updateTimestampCache( end = resume.Key } addToTSCache(start, end, ts, txnID) - case *roachpb.ReverseScanRequest: - if !beforeEval && resp.(*roachpb.ReverseScanResponse).ResumeSpan != nil { - resume := resp.(*roachpb.ReverseScanResponse).ResumeSpan + case *kvpb.ReverseScanRequest: + if !beforeEval && resp.(*kvpb.ReverseScanResponse).ResumeSpan != nil { + resume := resp.(*kvpb.ReverseScanResponse).ResumeSpan if end.Equal(resume.EndKey) { // The request did not evaluate. Ignore it. continue @@ -259,13 +260,13 @@ func (r *Replica) updateTimestampCache( start = resume.EndKey } addToTSCache(start, end, ts, txnID) - case *roachpb.QueryIntentRequest: + case *kvpb.QueryIntentRequest: missing := false if pErr != nil { switch t := pErr.GetDetail().(type) { - case *roachpb.IntentMissingError: + case *kvpb.IntentMissingError: missing = true - case *roachpb.TransactionRetryError: + case *kvpb.TransactionRetryError: // QueryIntent will return a TxnRetry(SERIALIZABLE) error // if a transaction is querying its own intent and finds // it pushed. @@ -275,10 +276,10 @@ func (r *Replica) updateTimestampCache( // from the QueryIntent request. However, bumping the // timestamp cache wouldn't cause a correctness issue // if we found the intent. - missing = t.Reason == roachpb.RETRY_SERIALIZABLE + missing = t.Reason == kvpb.RETRY_SERIALIZABLE } } else { - missing = !resp.(*roachpb.QueryIntentResponse).FoundIntent + missing = !resp.(*kvpb.QueryIntentResponse).FoundIntent } if missing { // If the QueryIntent determined that the intent is missing @@ -324,7 +325,7 @@ func init() { // minReadTS, minReadTS (without an associated txn id) will be used instead to // adjust the batch's timestamp. func (r *Replica) applyTimestampCache( - ctx context.Context, ba *roachpb.BatchRequest, minReadTS hlc.Timestamp, + ctx context.Context, ba *kvpb.BatchRequest, minReadTS hlc.Timestamp, ) bool { // bumpedDueToMinReadTS is set to true if the highest timestamp bump encountered // below is due to the minReadTS. @@ -334,7 +335,7 @@ func (r *Replica) applyTimestampCache( for _, union := range ba.Requests { args := union.GetInner() - if roachpb.AppliesTimestampCache(args) { + if kvpb.AppliesTimestampCache(args) { header := args.Header() // Forward the timestamp if there's been a more recent read (by someone else). @@ -520,7 +521,7 @@ func (r *Replica) applyTimestampCache( // system. func (r *Replica) CanCreateTxnRecord( ctx context.Context, txnID uuid.UUID, txnKey []byte, txnMinTS hlc.Timestamp, -) (ok bool, reason roachpb.TransactionAbortedReason) { +) (ok bool, reason kvpb.TransactionAbortedReason) { // Consult the timestamp cache with the transaction's key. The timestamp cache // is used to abort transactions that don't have transaction records. // @@ -553,22 +554,22 @@ func (r *Replica) CanCreateTxnRecord( // If there were other requests in the EndTxn batch, then the client would // still have trouble reconstructing the result, but at least it could // provide a non-ambiguous error to the application. - return false, roachpb.ABORT_REASON_RECORD_ALREADY_WRITTEN_POSSIBLE_REPLAY + return false, kvpb.ABORT_REASON_RECORD_ALREADY_WRITTEN_POSSIBLE_REPLAY case uuid.Nil: lease, _ /* nextLease */ := r.GetLease() // Recognize the case where a lease started recently. Lease transfers bump // the ts cache low water mark. if tombstoneTimestamp == lease.Start.ToTimestamp() { - return false, roachpb.ABORT_REASON_NEW_LEASE_PREVENTS_TXN + return false, kvpb.ABORT_REASON_NEW_LEASE_PREVENTS_TXN } - return false, roachpb.ABORT_REASON_TIMESTAMP_CACHE_REJECTED + return false, kvpb.ABORT_REASON_TIMESTAMP_CACHE_REJECTED default: // If we find another transaction's ID then that transaction has // aborted us before our transaction record was written. It obeyed // the restriction that it couldn't create a transaction record for // us, so it recorded a tombstone cache instead to prevent us // from ever creating a transaction record. - return false, roachpb.ABORT_REASON_ABORTED_RECORD_FOUND + return false, kvpb.ABORT_REASON_ABORTED_RECORD_FOUND } } diff --git a/pkg/kv/kvserver/replica_write.go b/pkg/kv/kvserver/replica_write.go index 5d02a684e5bd..1331b0a4b44b 100644 --- a/pkg/kv/kvserver/replica_write.go +++ b/pkg/kv/kvserver/replica_write.go @@ -16,6 +16,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" @@ -78,12 +79,12 @@ var migrateApplicationTimeout = settings.RegisterDurationSetting( // as this method makes the assumption that it operates on a shallow copy (see // call to applyTimestampCache). func (r *Replica) executeWriteBatch( - ctx context.Context, ba *roachpb.BatchRequest, g *concurrency.Guard, + ctx context.Context, ba *kvpb.BatchRequest, g *concurrency.Guard, ) ( - br *roachpb.BatchResponse, + br *kvpb.BatchResponse, _ *concurrency.Guard, _ *kvadmission.StoreWriteBytes, - pErr *roachpb.Error, + pErr *kvpb.Error, ) { startTime := timeutil.Now() @@ -107,14 +108,14 @@ func (r *Replica) executeWriteBatch( // Verify that the batch can be executed. st, err := r.checkExecutionCanProceedRWOrAdmin(ctx, ba, g) if err != nil { - return nil, g, nil, roachpb.NewError(err) + return nil, g, nil, kvpb.NewError(err) } // Check the breaker. Note that we do this after // checkExecutionCanProceedBeforeStorageSnapshot, so that NotLeaseholderError // has precedence. if err := r.signallerForBatch(ba).Err(); err != nil { - return nil, g, nil, roachpb.NewError(err) + return nil, g, nil, kvpb.NewError(err) } // Compute the transaction's local uncertainty limit using observed @@ -166,7 +167,7 @@ func (r *Replica) executeWriteBatch( // Checking the context just before proposing can help avoid ambiguous errors. if err := ctx.Err(); err != nil { log.VEventf(ctx, 2, "%s before proposing: %s", err, ba.Summary()) - return nil, g, nil, roachpb.NewError(errors.Wrapf(err, "aborted before proposing")) + return nil, g, nil, kvpb.NewError(errors.Wrapf(err, "aborted before proposing")) } // If the command is proposed to Raft, ownership of and responsibility for @@ -174,7 +175,7 @@ func (r *Replica) executeWriteBatch( // evalAndPropose. ch, abandon, _, writeBytes, pErr := r.evalAndPropose(ctx, ba, g, &st, ui, tok.Move(ctx)) if pErr != nil { - if cErr, ok := pErr.GetDetail().(*roachpb.ReplicaCorruptionError); ok { + if cErr, ok := pErr.GetDetail().(*kvpb.ReplicaCorruptionError); ok { // Need to unlock here because setCorruptRaftMuLock needs readOnlyCmdMu not held. readOnlyCmdMu.RUnlock() readOnlyCmdMu = nil @@ -274,7 +275,7 @@ func (r *Replica) executeWriteBatch( // We wait for an index >= that of the migration command. r.GetLeaseAppliedIndex()) }) - propResult.Err = roachpb.NewError(applicationErr) + propResult.Err = kvpb.NewError(applicationErr) } if propResult.Err != nil && ba.IsSingleProbeRequest() && errors.Is( propResult.Err.GoError(), kvserverbase.NoopOnProbeCommandErr.GoError(), @@ -298,7 +299,7 @@ func (r *Replica) executeWriteBatch( // txn. If the resolver's async task pool is full, just skip cleanup // by setting allowSync=false, since we won't be able to // backpressure clients. - if _, ok := ba.GetArg(roachpb.EndTxn); ok { + if _, ok := ba.GetArg(kvpb.EndTxn); ok { const taskName = "async txn cleanup" _ = r.store.stopper.RunAsyncTask( r.AnnotateCtx(context.Background()), @@ -327,7 +328,7 @@ func (r *Replica) executeWriteBatch( dur := timeutil.Since(startTime) log.VEventf(ctx, 2, "context cancellation after %.2fs of attempting command %s", dur.Seconds(), ba) - return nil, nil, nil, roachpb.NewError(roachpb.NewAmbiguousResultError( + return nil, nil, nil, kvpb.NewError(kvpb.NewAmbiguousResultError( errors.Wrapf(ctx.Err(), "after %.2fs of attempting command", dur.Seconds()), )) @@ -337,7 +338,7 @@ func (r *Replica) executeWriteBatch( abandon() log.VEventf(ctx, 2, "shutdown cancellation after %0.1fs of attempting command %s", timeutil.Since(startTime).Seconds(), ba) - return nil, nil, nil, roachpb.NewError(roachpb.NewAmbiguousResultErrorf( + return nil, nil, nil, kvpb.NewError(kvpb.NewAmbiguousResultErrorf( "server shutdown")) } } @@ -346,7 +347,7 @@ func (r *Replica) executeWriteBatch( // canAttempt1PCEvaluation looks at the batch and decides whether it can be // executed as 1PC. func (r *Replica) canAttempt1PCEvaluation( - ctx context.Context, ba *roachpb.BatchRequest, g *concurrency.Guard, + ctx context.Context, ba *kvpb.BatchRequest, g *concurrency.Guard, ) bool { if !isOnePhaseCommit(ba) { return false @@ -391,11 +392,11 @@ func (r *Replica) canAttempt1PCEvaluation( func (r *Replica) evaluateWriteBatch( ctx context.Context, idKey kvserverbase.CmdIDKey, - ba *roachpb.BatchRequest, + ba *kvpb.BatchRequest, g *concurrency.Guard, st *kvserverpb.LeaseStatus, ui uncertainty.Interval, -) (storage.Batch, enginepb.MVCCStats, *roachpb.BatchResponse, result.Result, *roachpb.Error) { +) (storage.Batch, enginepb.MVCCStats, *kvpb.BatchResponse, result.Result, *kvpb.Error) { log.Event(ctx, "executing read-write batch") // If the transaction has been pushed but it can commit at the higher @@ -423,9 +424,9 @@ func (r *Replica) evaluateWriteBatch( // care about it. Note that the point of Require1PC is that we don't want to // leave locks behind in case of retriable errors, so it's better to // terminate this request early. - arg, ok := ba.GetArg(roachpb.EndTxn) - if ok && arg.(*roachpb.EndTxnRequest).Require1PC { - return nil, enginepb.MVCCStats{}, nil, result.Result{}, roachpb.NewError(kv.OnePCNotAllowedError{}) + arg, ok := ba.GetArg(kvpb.EndTxn) + if ok && arg.(*kvpb.EndTxnRequest).Require1PC { + return nil, enginepb.MVCCStats{}, nil, result.Result{}, kvpb.NewError(kv.OnePCNotAllowedError{}) } } @@ -463,11 +464,11 @@ type onePCResult struct { success onePCSuccess // pErr is set if success == onePCFailed. This is the error that should be // returned to the client for this request. - pErr *roachpb.Error + pErr *kvpb.Error // The fields below are only set when success == onePCSucceeded. stats enginepb.MVCCStats - br *roachpb.BatchResponse + br *kvpb.BatchResponse res result.Result batch storage.Batch } @@ -481,7 +482,7 @@ type onePCResult struct { func (r *Replica) evaluate1PC( ctx context.Context, idKey kvserverbase.CmdIDKey, - ba *roachpb.BatchRequest, + ba *kvpb.BatchRequest, g *concurrency.Guard, st *kvserverpb.LeaseStatus, ) (onePCRes onePCResult) { @@ -508,12 +509,12 @@ func (r *Replica) evaluate1PC( rec := NewReplicaEvalContext(ctx, r, g.LatchSpans(), ba.RequiresClosedTSOlderThanStorageSnapshot()) defer rec.Release() - var br *roachpb.BatchResponse + var br *kvpb.BatchResponse var res result.Result - var pErr *roachpb.Error + var pErr *kvpb.Error - arg, _ := ba.GetArg(roachpb.EndTxn) - etArg := arg.(*roachpb.EndTxnRequest) + arg, _ := ba.GetArg(kvpb.EndTxn) + etArg := arg.(*kvpb.EndTxnRequest) // Evaluate strippedBa. If the transaction allows, permit refreshes. ms := newMVCCStats() @@ -562,13 +563,13 @@ func (r *Replica) evaluate1PC( if err != nil { return onePCResult{ success: onePCFailed, - pErr: roachpb.NewError(errors.Wrap(err, "failed to run commit trigger")), + pErr: kvpb.NewError(errors.Wrap(err, "failed to run commit trigger")), } } if err := res.MergeAndDestroy(innerResult); err != nil { return onePCResult{ success: onePCFailed, - pErr: roachpb.NewError(err), + pErr: kvpb.NewError(err), } } } @@ -592,12 +593,12 @@ func (r *Replica) evaluate1PC( br.Txn = clonedTxn // Add placeholder response for the end transaction request. etAlloc := new(struct { - et roachpb.EndTxnResponse - union roachpb.ResponseUnion_EndTxn + et kvpb.EndTxnResponse + union kvpb.ResponseUnion_EndTxn }) etAlloc.et.OnePhaseCommit = true etAlloc.union.EndTxn = &etAlloc.et - br.Responses = append(br.Responses, roachpb.ResponseUnion{}) + br.Responses = append(br.Responses, kvpb.ResponseUnion{}) br.Responses[len(br.Responses)-1].Value = &etAlloc.union return onePCResult{ @@ -621,12 +622,12 @@ func (r *Replica) evaluateWriteBatchWithServersideRefreshes( idKey kvserverbase.CmdIDKey, rec batcheval.EvalContext, ms *enginepb.MVCCStats, - ba *roachpb.BatchRequest, + ba *kvpb.BatchRequest, g *concurrency.Guard, st *kvserverpb.LeaseStatus, ui uncertainty.Interval, deadline hlc.Timestamp, -) (batch storage.Batch, br *roachpb.BatchResponse, res result.Result, pErr *roachpb.Error) { +) (batch storage.Batch, br *kvpb.BatchResponse, res result.Result, pErr *kvpb.Error) { goldenMS := *ms for retries := 0; ; retries++ { if retries > 0 { @@ -671,11 +672,11 @@ func (r *Replica) evaluateWriteBatchWrapper( idKey kvserverbase.CmdIDKey, rec batcheval.EvalContext, ms *enginepb.MVCCStats, - ba *roachpb.BatchRequest, + ba *kvpb.BatchRequest, g *concurrency.Guard, st *kvserverpb.LeaseStatus, ui uncertainty.Interval, -) (storage.Batch, *roachpb.BatchResponse, result.Result, *roachpb.Error) { +) (storage.Batch, *kvpb.BatchResponse, result.Result, *kvpb.Error) { batch, opLogger := r.newBatchedEngine(ba, g) now := timeutil.Now() br, res, pErr := evaluateBatch(ctx, idKey, batch, rec, ms, ba, g, st, ui, readWrite) @@ -695,7 +696,7 @@ func (r *Replica) evaluateWriteBatchWrapper( // OpLogger is attached to the returned engine.Batch, recording all operations. // Its recording should be attached to the Result of request evaluation. func (r *Replica) newBatchedEngine( - ba *roachpb.BatchRequest, g *concurrency.Guard, + ba *kvpb.BatchRequest, g *concurrency.Guard, ) (storage.Batch, *storage.OpLoggerBatch) { batch := r.store.TODOEngine().NewBatch() if !batch.ConsistentIterators() { @@ -760,15 +761,15 @@ func (r *Replica) newBatchedEngine( // (4) the transaction is not in its first epoch and the EndTxn request does // // not require one phase commit. -func isOnePhaseCommit(ba *roachpb.BatchRequest) bool { +func isOnePhaseCommit(ba *kvpb.BatchRequest) bool { if ba.Txn == nil { return false } if !ba.IsCompleteTransaction() { return false } - arg, _ := ba.GetArg(roachpb.EndTxn) - etArg := arg.(*roachpb.EndTxnRequest) + arg, _ := ba.GetArg(kvpb.EndTxn) + etArg := arg.(*kvpb.EndTxnRequest) if retry, _, _ := batcheval.IsEndTxnTriggeringRetryError(ba.Txn, etArg); retry { return false } diff --git a/pkg/kv/kvserver/replicate_queue.go b/pkg/kv/kvserver/replicate_queue.go index c1cba61fba4f..198d417eb86e 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -18,6 +18,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/gossip" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/allocatorimpl" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" @@ -1190,7 +1191,7 @@ func (rq *replicateQueue) addOrReplaceVoters( // Figure out whether we should be promoting an existing non-voting replica to // a voting replica or if we ought to be adding a voter afresh. - var ops []roachpb.ReplicationChange + var ops []kvpb.ReplicationChange replDesc, found := desc.GetReplicaDescriptor(newVoter.StoreID) if found { if replDesc.Type != roachpb.NON_VOTER { @@ -1202,12 +1203,12 @@ func (rq *replicateQueue) addOrReplaceVoters( effects = effects.add(func() { rq.metrics.NonVoterPromotionsCount.Inc(1) }) - ops = roachpb.ReplicationChangesForPromotion(newVoter) + ops = kvpb.ReplicationChangesForPromotion(newVoter) } else { effects = effects.add(func() { rq.metrics.trackAddReplicaCount(allocatorimpl.VoterTarget) }) - ops = roachpb.MakeReplicationChanges(roachpb.ADD_VOTER, newVoter) + ops = kvpb.MakeReplicationChanges(roachpb.ADD_VOTER, newVoter) } if !isReplace { log.KvDistribution.Infof(ctx, "adding voter %+v: %s", @@ -1227,7 +1228,7 @@ func (rq *replicateQueue) addOrReplaceVoters( // this allocation is executed, we could be one non-voter short. This will // be handled by the replicateQueue's next attempt at this range. ops = append(ops, - roachpb.MakeReplicationChanges(roachpb.REMOVE_VOTER, roachpb.ReplicationTarget{ + kvpb.MakeReplicationChanges(roachpb.REMOVE_VOTER, roachpb.ReplicationTarget{ StoreID: removeVoter.StoreID, NodeID: removeVoter.NodeID, })...) @@ -1273,7 +1274,7 @@ func (rq *replicateQueue) addOrReplaceNonVoters( rq.metrics.trackAddReplicaCount(allocatorimpl.NonVoterTarget) }) - ops := roachpb.MakeReplicationChanges(roachpb.ADD_NON_VOTER, newNonVoter) + ops := kvpb.MakeReplicationChanges(roachpb.ADD_NON_VOTER, newNonVoter) if removeIdx < 0 { log.KvDistribution.Infof(ctx, "adding non-voter %+v: %s", newNonVoter, rangeRaftProgress(repl.RaftStatus(), existingNonVoters)) @@ -1285,7 +1286,7 @@ func (rq *replicateQueue) addOrReplaceNonVoters( log.KvDistribution.Infof(ctx, "replacing non-voter %s with %+v: %s", removeNonVoter, newNonVoter, rangeRaftProgress(repl.RaftStatus(), existingNonVoters)) ops = append(ops, - roachpb.MakeReplicationChanges(roachpb.REMOVE_NON_VOTER, roachpb.ReplicationTarget{ + kvpb.MakeReplicationChanges(roachpb.REMOVE_NON_VOTER, roachpb.ReplicationTarget{ StoreID: removeNonVoter.StoreID, NodeID: removeNonVoter.NodeID, })...) @@ -1493,7 +1494,7 @@ func (rq *replicateQueue) removeVoter( lhStore: repl.StoreID(), sideEffects: effects.f(), usage: RangeUsageInfoForRepl(repl), - chgs: roachpb.MakeReplicationChanges(roachpb.REMOVE_VOTER, removeVoter), + chgs: kvpb.MakeReplicationChanges(roachpb.REMOVE_VOTER, removeVoter), priority: kvserverpb.SnapshotRequest_UNKNOWN, // unused allocatorPriority: 0.0, // unused reason: kvserverpb.ReasonRangeOverReplicated, @@ -1535,7 +1536,7 @@ func (rq *replicateQueue) removeNonVoter( lhStore: repl.StoreID(), sideEffects: effects.f(), usage: RangeUsageInfoForRepl(repl), - chgs: roachpb.MakeReplicationChanges(roachpb.REMOVE_NON_VOTER, target), + chgs: kvpb.MakeReplicationChanges(roachpb.REMOVE_NON_VOTER, target), priority: kvserverpb.SnapshotRequest_UNKNOWN, // unused allocatorPriority: 0.0, // unused reason: kvserverpb.ReasonRangeOverReplicated, @@ -1593,7 +1594,7 @@ func (rq *replicateQueue) removeDecommissioning( lhStore: repl.StoreID(), sideEffects: effects.f(), usage: RangeUsageInfoForRepl(repl), - chgs: roachpb.MakeReplicationChanges(targetType.RemoveChangeType(), target), + chgs: kvpb.MakeReplicationChanges(targetType.RemoveChangeType(), target), priority: kvserverpb.SnapshotRequest_UNKNOWN, // unused allocatorPriority: 0.0, // unused reason: kvserverpb.ReasonStoreDecommissioning, @@ -1636,7 +1637,7 @@ func (rq *replicateQueue) removeDead( lhStore: repl.StoreID(), sideEffects: effects.f(), usage: RangeUsageInfoForRepl(repl), - chgs: roachpb.MakeReplicationChanges(targetType.RemoveChangeType(), target), + chgs: kvpb.MakeReplicationChanges(targetType.RemoveChangeType(), target), priority: kvserverpb.SnapshotRequest_UNKNOWN, // unused allocatorPriority: 0.0, // unused reason: kvserverpb.ReasonStoreDead, @@ -1785,7 +1786,7 @@ func replicationChangesForRebalance( numExistingVoters int, addTarget, removeTarget roachpb.ReplicationTarget, rebalanceTargetType allocatorimpl.TargetReplicaType, -) (chgs []roachpb.ReplicationChange, performingSwap bool, err error) { +) (chgs []kvpb.ReplicationChange, performingSwap bool, err error) { if rebalanceTargetType == allocatorimpl.VoterTarget && numExistingVoters == 1 { // If there's only one replica, the removal target is the // leaseholder and this is unsupported and will fail. However, @@ -1808,7 +1809,7 @@ func replicationChangesForRebalance( // when we know it's necessary, picking the smaller of two evils. // // See https://github.com/cockroachdb/cockroach/issues/40333. - chgs = []roachpb.ReplicationChange{ + chgs = []kvpb.ReplicationChange{ {ChangeType: roachpb.ADD_VOTER, Target: addTarget}, } log.KvDistribution.Infof(ctx, "can't swap replica due to lease; falling back to add") @@ -1837,8 +1838,8 @@ func replicationChangesForRebalance( // replica into the replicateQueue. So we expect the replicateQueue's next // attempt at rebalancing this range to rebalance the non-voter if it ends // up being in violation of the range's constraints. - promo := roachpb.ReplicationChangesForPromotion(addTarget) - demo := roachpb.ReplicationChangesForDemotion(removeTarget) + promo := kvpb.ReplicationChangesForPromotion(addTarget) + demo := kvpb.ReplicationChangesForDemotion(removeTarget) chgs = append(promo, demo...) performingSwap = true } else if found { @@ -1848,7 +1849,7 @@ func replicationChangesForRebalance( ) } else { // We have a replica to remove and one we can add, so let's swap them out. - chgs = []roachpb.ReplicationChange{ + chgs = []kvpb.ReplicationChange{ {ChangeType: roachpb.ADD_VOTER, Target: addTarget}, {ChangeType: roachpb.REMOVE_VOTER, Target: removeTarget}, } @@ -1863,7 +1864,7 @@ func replicationChangesForRebalance( " move non-voter to a store that already has a replica %s for the range", rdesc, ) } - chgs = []roachpb.ReplicationChange{ + chgs = []kvpb.ReplicationChange{ {ChangeType: roachpb.ADD_NON_VOTER, Target: addTarget}, {ChangeType: roachpb.REMOVE_NON_VOTER, Target: removeTarget}, } @@ -2021,7 +2022,7 @@ func (rq *replicateQueue) RelocateRange( func (rq *replicateQueue) changeReplicas( ctx context.Context, repl *Replica, - chgs roachpb.ReplicationChanges, + chgs kvpb.ReplicationChanges, desc *roachpb.RangeDescriptor, priority kvserverpb.SnapshotRequest_Priority, allocatorPriority float64, diff --git a/pkg/kv/kvserver/replicate_queue_test.go b/pkg/kv/kvserver/replicate_queue_test.go index ff3b929afab8..c126aa0103ad 100644 --- a/pkg/kv/kvserver/replicate_queue_test.go +++ b/pkg/kv/kvserver/replicate_queue_test.go @@ -30,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/allocatorimpl" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" @@ -1872,7 +1873,7 @@ func (h delayingRaftMessageHandler) HandleRaftRequest( ctx context.Context, req *kvserverpb.RaftMessageRequest, respStream kvserver.RaftMessageResponseStream, -) *roachpb.Error { +) *kvpb.Error { if h.rangeID != req.RangeID { return h.RaftMessageHandler.HandleRaftRequest(ctx, req, respStream) } diff --git a/pkg/kv/kvserver/reports/BUILD.bazel b/pkg/kv/kvserver/reports/BUILD.bazel index 4ecf233423c2..2633b25b6b55 100644 --- a/pkg/kv/kvserver/reports/BUILD.bazel +++ b/pkg/kv/kvserver/reports/BUILD.bazel @@ -18,6 +18,7 @@ go_library( "//pkg/config/zonepb", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/kv/kvserver/allocator/storepool", "//pkg/kv/kvserver/liveness", @@ -58,6 +59,7 @@ go_test( "//pkg/config", "//pkg/config/zonepb", "//pkg/keys", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/roachpb", "//pkg/security/securityassets", diff --git a/pkg/kv/kvserver/reports/reporter.go b/pkg/kv/kvserver/reports/reporter.go index e844231c14be..5d5b1ee43542 100644 --- a/pkg/kv/kvserver/reports/reporter.go +++ b/pkg/kv/kvserver/reports/reporter.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" @@ -272,7 +273,7 @@ func (stats *Reporter) update( func (stats *Reporter) meta1LeaseHolderStore(ctx context.Context) *kvserver.Store { const meta1RangeID = roachpb.RangeID(1) repl, store, err := stats.localStores.GetReplicaForRangeID(ctx, meta1RangeID) - if roachpb.IsRangeNotFoundError(err) { + if kvpb.IsRangeNotFoundError(err) { return nil } if err != nil { @@ -638,7 +639,7 @@ type RangeIterator interface { // the iterator is not to be used any more (except for calling Close(), which will be a no-op). // // The returned error can be a retriable one (i.e. - // *roachpb.TransactionRetryWithProtoRefreshError, possibly wrapped). In that case, the iterator + // *kvpb.TransactionRetryWithProtoRefreshError, possibly wrapped). In that case, the iterator // is reset automatically; the next Next() call ( should there be one) will // return the first descriptor. // In case of any other error, the iterator is automatically closed. @@ -754,7 +755,7 @@ func (r *meta2RangeIter) readBatch(ctx context.Context) (retErr error) { } func errIsRetriable(err error) bool { - return errors.HasType(err, (*roachpb.TransactionRetryWithProtoRefreshError)(nil)) + return errors.HasType(err, (*kvpb.TransactionRetryWithProtoRefreshError)(nil)) } // handleErr manipulates the iterator's state in response to an error. diff --git a/pkg/kv/kvserver/reports/reporter_test.go b/pkg/kv/kvserver/reports/reporter_test.go index 942c323be1fb..e22efd7c8cfd 100644 --- a/pkg/kv/kvserver/reports/reporter_test.go +++ b/pkg/kv/kvserver/reports/reporter_test.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql" @@ -458,7 +459,7 @@ func (it *erroryRangeIterator) Next(ctx context.Context) (roachpb.RangeDescripto it.injectErrAfter = -1 var err error - err = roachpb.NewTransactionRetryWithProtoRefreshError( + err = kvpb.NewTransactionRetryWithProtoRefreshError( "injected err", uuid.Nil, roachpb.Transaction{}) // Let's wrap the error to check the unwrapping. err = errors.Wrap(err, "dummy wrapper") diff --git a/pkg/kv/kvserver/reset_quorum_test.go b/pkg/kv/kvserver/reset_quorum_test.go index bea706123026..7f13e2c35b62 100644 --- a/pkg/kv/kvserver/reset_quorum_test.go +++ b/pkg/kv/kvserver/reset_quorum_test.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server" @@ -200,7 +201,7 @@ func TestResetQuorum(t *testing.T) { checkUnavailable(srv, k) // Call ResetQuorum to reset quorum on the unhealthy range. - _, err := srv.Node().(*server.Node).ResetQuorum(ctx, &roachpb.ResetQuorumRequest{ + _, err := srv.Node().(*server.Node).ResetQuorum(ctx, &kvpb.ResetQuorumRequest{ RangeID: int32(id), }) require.NoError(t, err) @@ -224,7 +225,7 @@ func TestResetQuorum(t *testing.T) { checkUnavailable(srv, k) // Call ResetQuorum to reset quorum on the unhealthy range. - _, err := srv.Node().(*server.Node).ResetQuorum(ctx, &roachpb.ResetQuorumRequest{ + _, err := srv.Node().(*server.Node).ResetQuorum(ctx, &kvpb.ResetQuorumRequest{ RangeID: int32(id), }) require.NoError(t, err) @@ -247,7 +248,7 @@ func TestResetQuorum(t *testing.T) { checkUnavailable(srv, k) // Call ResetQuorum to reset quorum on the unhealthy range. - _, err := srv.Node().(*server.Node).ResetQuorum(ctx, &roachpb.ResetQuorumRequest{ + _, err := srv.Node().(*server.Node).ResetQuorum(ctx, &kvpb.ResetQuorumRequest{ RangeID: int32(id), }) require.NoError(t, err) @@ -279,7 +280,7 @@ func TestResetQuorum(t *testing.T) { checkUnavailable(srv, k) // Call ResetQuorum to reset quorum on the unhealthy range. - _, err := srv.Node().(*server.Node).ResetQuorum(ctx, &roachpb.ResetQuorumRequest{ + _, err := srv.Node().(*server.Node).ResetQuorum(ctx, &kvpb.ResetQuorumRequest{ RangeID: int32(id), }) require.NoError(t, err) @@ -305,7 +306,7 @@ func TestResetQuorum(t *testing.T) { srv := tc.Server(0) // Call ResetQuorum to attempt to reset quorum on a healthy range. - _, err := srv.Node().(*server.Node).ResetQuorum(ctx, &roachpb.ResetQuorumRequest{ + _, err := srv.Node().(*server.Node).ResetQuorum(ctx, &kvpb.ResetQuorumRequest{ RangeID: int32(id), }) testutils.IsError(err, "targeted range to recover has not lost quorum.") @@ -317,7 +318,7 @@ func TestResetQuorum(t *testing.T) { srv := tc.Server(0) // Call ResetQuorum to attempt to reset quorum on a meta range. - _, err := srv.Node().(*server.Node).ResetQuorum(ctx, &roachpb.ResetQuorumRequest{ + _, err := srv.Node().(*server.Node).ResetQuorum(ctx, &kvpb.ResetQuorumRequest{ RangeID: int32(keys.MetaRangesID), }) testutils.IsError(err, "targeted range to recover is a meta1 or meta2 range.") diff --git a/pkg/kv/kvserver/scatter_test.go b/pkg/kv/kvserver/scatter_test.go index bea3144ddf15..67d83008d404 100644 --- a/pkg/kv/kvserver/scatter_test.go +++ b/pkg/kv/kvserver/scatter_test.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" @@ -25,9 +26,9 @@ import ( "github.com/stretchr/testify/require" ) -func adminScatterArgs(key roachpb.Key, randomizeLeases bool) *roachpb.AdminScatterRequest { - return &roachpb.AdminScatterRequest{ - RequestHeader: roachpb.RequestHeader{ +func adminScatterArgs(key roachpb.Key, randomizeLeases bool) *kvpb.AdminScatterRequest { + return &kvpb.AdminScatterRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, EndKey: key.Next(), }, diff --git a/pkg/kv/kvserver/spanlatch/BUILD.bazel b/pkg/kv/kvserver/spanlatch/BUILD.bazel index 39e419acd158..9984c73cb43d 100644 --- a/pkg/kv/kvserver/spanlatch/BUILD.bazel +++ b/pkg/kv/kvserver/spanlatch/BUILD.bazel @@ -15,6 +15,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/base", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/concurrency/poison", "//pkg/kv/kvserver/spanset", "//pkg/roachpb", diff --git a/pkg/kv/kvserver/spanlatch/manager.go b/pkg/kv/kvserver/spanlatch/manager.go index 7dac899c7d83..55a30a17afb9 100644 --- a/pkg/kv/kvserver/spanlatch/manager.go +++ b/pkg/kv/kvserver/spanlatch/manager.go @@ -16,6 +16,7 @@ import ( "unsafe" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/poison" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -568,7 +569,7 @@ func (m *Manager) waitForSignal( case <-m.stopper.ShouldQuiesce(): // While shutting down, requests may acquire // latches and never release them. - return &roachpb.NodeUnavailableError{} + return &kvpb.NodeUnavailableError{} } } } diff --git a/pkg/kv/kvserver/split_queue.go b/pkg/kv/kvserver/split_queue.go index bb16e18fa8aa..33771f922361 100644 --- a/pkg/kv/kvserver/split_queue.go +++ b/pkg/kv/kvserver/split_queue.go @@ -16,6 +16,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/telemetry" @@ -204,7 +205,7 @@ func (sq *splitQueue) process( ctx context.Context, r *Replica, confReader spanconfig.StoreReader, ) (processed bool, err error) { processed, err = sq.processAttempt(ctx, r, confReader) - if errors.HasType(err, (*roachpb.ConditionFailedError)(nil)) { + if errors.HasType(err, (*kvpb.ConditionFailedError)(nil)) { // ConditionFailedErrors are an expected outcome for range split // attempts because splits can race with other descriptor modifications. // On seeing a ConditionFailedError, don't return an error and enqueue @@ -225,8 +226,8 @@ func (sq *splitQueue) processAttempt( if splitKey := confReader.ComputeSplitKey(ctx, desc.StartKey, desc.EndKey); splitKey != nil { if _, err := r.adminSplitWithDescriptor( ctx, - roachpb.AdminSplitRequest{ - RequestHeader: roachpb.RequestHeader{ + kvpb.AdminSplitRequest{ + RequestHeader: kvpb.RequestHeader{ Key: splitKey.AsRawKey(), }, SplitKey: splitKey.AsRawKey(), @@ -250,7 +251,7 @@ func (sq *splitQueue) processAttempt( if maxBytes > 0 && size > maxBytes { if _, err := r.adminSplitWithDescriptor( ctx, - roachpb.AdminSplitRequest{}, + kvpb.AdminSplitRequest{}, desc, false, /* delayable */ fmt.Sprintf("%s above threshold size %s", humanizeutil.IBytes(size), humanizeutil.IBytes(maxBytes)), @@ -291,8 +292,8 @@ func (sq *splitQueue) processAttempt( } if _, pErr := r.adminSplitWithDescriptor( ctx, - roachpb.AdminSplitRequest{ - RequestHeader: roachpb.RequestHeader{ + kvpb.AdminSplitRequest{ + RequestHeader: kvpb.RequestHeader{ Key: splitByLoadKey, }, SplitKey: splitByLoadKey, diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index 2c485f14ece8..aab168d91406 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -32,6 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangecache" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/allocatorimpl" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" @@ -518,7 +519,7 @@ transaction, and so the transaction record will be created on the affected range. This allows us to establish a helpful invariant: INVARIANT: an intent on keys.RangeDescriptorKey is resolved atomically with -the (application of the) roachpb.EndTxnRequest committing the transaction. +the (application of the) kvpb.EndTxnRequest committing the transaction. A Replica's active configuration is dictated by its visible version of the RangeDescriptor, and the above invariant simplifies this. Without the invariant, @@ -704,7 +705,7 @@ descriptor that removes it, or it may be destroyed by the application of a merge on its left neighboring Replica, which may also occur through a snapshot. Merges are the single most complex reconfiguration operation and can only be touched upon here. At their core, they will at some point "freeze" the right-hand side -Replicas (via roachpb.SubsumeRequest) to prevent additional read or write +Replicas (via kvpb.SubsumeRequest) to prevent additional read or write activity, and also ensure that the two sets of Ranges to be merged are co-located on the same Stores as well as are all initialized. @@ -1143,7 +1144,7 @@ type ConsistencyTestingKnobs struct { // checksum, instead of log.Fatal. OnBadChecksumFatal func(roachpb.StoreIdent) - ConsistencyQueueResultHook func(response roachpb.CheckConsistencyResponse) + ConsistencyQueueResultHook func(response kvpb.CheckConsistencyResponse) } // Valid returns true if the StoreConfig is populated correctly. @@ -2120,7 +2121,7 @@ func (s *Store) startLeaseRenewer(ctx context.Context) { repl := (*Replica)(v) annotatedCtx := repl.AnnotateCtx(ctx) if _, pErr := repl.redirectOnOrAcquireLease(annotatedCtx); pErr != nil { - if _, ok := pErr.GetDetail().(*roachpb.NotLeaseHolderError); !ok { + if _, ok := pErr.GetDetail().(*kvpb.NotLeaseHolderError); !ok { log.Warningf(annotatedCtx, "failed to proactively renew lease: %s", pErr) } s.renewableLeases.Delete(k) @@ -2487,7 +2488,7 @@ func (s *Store) GetReplica(rangeID roachpb.RangeID) (*Replica, error) { if r := s.GetReplicaIfExists(rangeID); r != nil { return r, nil } - return nil, roachpb.NewRangeNotFoundError(rangeID, s.StoreID()) + return nil, kvpb.NewRangeNotFoundError(rangeID, s.StoreID()) } // GetReplicaIfExists returns the replica with the given RangeID or nil. @@ -2754,9 +2755,7 @@ func (s *Store) Descriptor(ctx context.Context, useCached bool) (*roachpb.StoreD // RangeFeed registers a rangefeed over the specified span. It sends updates to // the provided stream and returns with an optional error when the rangefeed is // complete. -func (s *Store) RangeFeed( - args *roachpb.RangeFeedRequest, stream roachpb.RangeFeedEventSink, -) *roachpb.Error { +func (s *Store) RangeFeed(args *kvpb.RangeFeedRequest, stream kvpb.RangeFeedEventSink) *kvpb.Error { if filter := s.TestingKnobs().TestingRangefeedFilter; filter != nil { if pErr := filter(args, stream); pErr != nil { @@ -2765,13 +2764,13 @@ func (s *Store) RangeFeed( } if err := verifyKeys(args.Span.Key, args.Span.EndKey, true); err != nil { - return roachpb.NewError(err) + return kvpb.NewError(err) } // Get range and add command to the range for execution. repl, err := s.GetReplica(args.RangeID) if err != nil { - return roachpb.NewError(err) + return kvpb.NewError(err) } if !repl.IsInitialized() { // (*Store).Send has an optimization for uninitialized replicas to send back @@ -2779,7 +2778,7 @@ func (s *Store) RangeFeed( // be found. RangeFeeds can always be served from followers and so don't // otherwise return NotLeaseHolderError. For simplicity we also don't return // one here. - return roachpb.NewError(roachpb.NewRangeNotFoundError(args.RangeID, s.StoreID())) + return kvpb.NewError(kvpb.NewRangeNotFoundError(args.RangeID, s.StoreID())) } tenID, _ := repl.TenantID() @@ -3553,7 +3552,7 @@ func (s *storeForTruncatorImpl) acquireReplicaForTruncator( ) replicaForTruncator { r, err := (*Store)(s).GetReplica(rangeID) if err != nil || r == nil { - // The only error we can see here is roachpb.NewRangeNotFoundError, so we + // The only error we can see here is kvpb.NewRangeNotFoundError, so we // can ignore it. return nil } diff --git a/pkg/kv/kvserver/store_create_replica.go b/pkg/kv/kvserver/store_create_replica.go index c7920adbd78a..3af5b72ba4f3 100644 --- a/pkg/kv/kvserver/store_create_replica.go +++ b/pkg/kv/kvserver/store_create_replica.go @@ -14,6 +14,7 @@ import ( "context" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvstorage" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -24,7 +25,7 @@ import ( var errRetry = errors.New("retry: orphaned replica") // getOrCreateReplica returns an existing or newly created replica with the -// given replicaID for the given rangeID, or roachpb.RaftGroupDeletedError if +// given replicaID for the given rangeID, or kvpb.RaftGroupDeletedError if // this replicaID has been deleted. A returned replica's Replica.raftMu is // locked, and the caller is responsible for unlocking it. // @@ -47,7 +48,7 @@ var errRetry = errors.New("retry: orphaned replica") // - The Replica is not being removed as seen by its Replica.mu.destroyStatus // - The RangeTombstone in storage does not see this replica as removed // -// If getOrCreateReplica returns roachpb.RaftGroupDeletedError, the guarantee is: +// If getOrCreateReplica returns kvpb.RaftGroupDeletedError, the guarantee is: // // - getOrCreateReplica will never return this replica // - Store.GetReplica(rangeID) can now only return replicas with higher IDs @@ -120,7 +121,7 @@ func (s *Store) tryGetReplica( if fromReplicaIsTooOldRLocked(repl, creatingReplica) { repl.mu.RUnlock() repl.raftMu.Unlock() - return nil, roachpb.NewReplicaTooOldError(creatingReplica.ReplicaID) + return nil, kvpb.NewReplicaTooOldError(creatingReplica.ReplicaID) } // The current replica needs to be removed, remove it and go back around. @@ -146,7 +147,7 @@ func (s *Store) tryGetReplica( // We could silently drop this message but this way we'll inform the // sender that they may no longer exist. repl.raftMu.Unlock() - return nil, &roachpb.RaftGroupDeletedError{} + return nil, &kvpb.RaftGroupDeletedError{} } if repl.replicaID != replicaID { // This case should have been caught by handleToReplicaTooOld. diff --git a/pkg/kv/kvserver/store_raft.go b/pkg/kv/kvserver/store_raft.go index e77ea4299b98..f37d0a213236 100644 --- a/pkg/kv/kvserver/store_raft.go +++ b/pkg/kv/kvserver/store_raft.go @@ -16,6 +16,7 @@ import ( "time" "unsafe" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -266,7 +267,7 @@ func (s *Store) uncoalesceBeats( // requires that s.mu is not held. func (s *Store) HandleRaftRequest( ctx context.Context, req *kvserverpb.RaftMessageRequest, respStream RaftMessageResponseStream, -) *roachpb.Error { +) *kvpb.Error { // NB: unlike the other two RaftMessageHandler methods implemented by Store, // this one doesn't need to directly run through a Stopper task because it // delegates all work through a raftScheduler, whose workers' lifetimes are @@ -324,8 +325,8 @@ func (s *Store) HandleRaftUncoalescedRequest( func (s *Store) withReplicaForRequest( ctx context.Context, req *kvserverpb.RaftMessageRequest, - f func(context.Context, *Replica) *roachpb.Error, -) *roachpb.Error { + f func(context.Context, *Replica) *kvpb.Error, +) *kvpb.Error { // Lazily create the replica. r, _, err := s.getOrCreateReplica( ctx, @@ -334,7 +335,7 @@ func (s *Store) withReplicaForRequest( &req.FromReplica, ) if err != nil { - return roachpb.NewError(err) + return kvpb.NewError(err) } defer r.raftMu.Unlock() r.setLastReplicaDescriptors(req) @@ -346,7 +347,7 @@ func (s *Store) withReplicaForRequest( // state; callers will probably want to handle this themselves at some point. func (s *Store) processRaftRequestWithReplica( ctx context.Context, r *Replica, req *kvserverpb.RaftMessageRequest, -) *roachpb.Error { +) *kvpb.Error { // Record the CPU time processing the request for this replica. This is // recorded regardless of errors that are encountered. defer r.MeasureRaftCPUNanos(grunning.Time()) @@ -375,7 +376,7 @@ func (s *Store) processRaftRequestWithReplica( if req.ToReplica.ReplicaID == 0 { log.VEventf(ctx, 1, "refusing incoming Raft message %s from %+v to %+v", req.Message.Type, req.FromReplica, req.ToReplica) - return roachpb.NewErrorf( + return kvpb.NewErrorf( "cannot recreate replica that is not a member of its range (StoreID %s not found in r%d)", r.store.StoreID(), req.RangeID, ) @@ -384,7 +385,7 @@ func (s *Store) processRaftRequestWithReplica( drop := maybeDropMsgApp(ctx, (*replicaMsgAppDropper)(r), &req.Message, req.RangeStartKey) if !drop { if err := r.stepRaftGroup(req); err != nil { - return roachpb.NewError(err) + return kvpb.NewError(err) } } return nil @@ -400,10 +401,10 @@ func (s *Store) processRaftRequestWithReplica( // will have been removed. func (s *Store) processRaftSnapshotRequest( ctx context.Context, snapHeader *kvserverpb.SnapshotRequest_Header, inSnap IncomingSnapshot, -) *roachpb.Error { +) *kvpb.Error { return s.withReplicaForRequest(ctx, &snapHeader.RaftMessageRequest, func( ctx context.Context, r *Replica, - ) (pErr *roachpb.Error) { + ) (pErr *kvpb.Error) { ctx = r.AnnotateCtx(ctx) if snapHeader.RaftMessageRequest.Message.Type != raftpb.MsgSnap { log.Fatalf(ctx, "expected snapshot: %+v", snapHeader.RaftMessageRequest) @@ -450,7 +451,7 @@ func (s *Store) processRaftSnapshotRequest( // withReplicaForRequest that this replica is not currently being removed // and we've been holding the raftMu the entire time. if err := r.stepRaftGroup(&snapHeader.RaftMessageRequest); err != nil { - return roachpb.NewError(err) + return kvpb.NewError(err) } // We've handed the snapshot to Raft, which will typically apply it (in @@ -489,12 +490,12 @@ func (s *Store) HandleRaftResponse( ctx = repl.AnnotateCtx(ctx) } switch val := resp.Union.GetValue().(type) { - case *roachpb.Error: + case *kvpb.Error: switch tErr := val.GetDetail().(type) { - case *roachpb.ReplicaTooOldError: + case *kvpb.ReplicaTooOldError: if replErr != nil { // RangeNotFoundErrors are expected here; nothing else is. - if !errors.HasType(replErr, (*roachpb.RangeNotFoundError)(nil)) { + if !errors.HasType(replErr, (*kvpb.RangeNotFoundError)(nil)) { log.Errorf(ctx, "%v", replErr) } return nil @@ -532,10 +533,10 @@ func (s *Store) HandleRaftResponse( return s.removeReplicaRaftMuLocked(ctx, repl, nextReplicaID, RemoveOptions{ DestroyData: true, }) - case *roachpb.RaftGroupDeletedError: + case *kvpb.RaftGroupDeletedError: if replErr != nil { // RangeNotFoundErrors are expected here; nothing else is. - if !errors.HasType(replErr, (*roachpb.RangeNotFoundError)(nil)) { + if !errors.HasType(replErr, (*kvpb.RangeNotFoundError)(nil)) { log.Errorf(ctx, "%v", replErr) } return nil @@ -547,7 +548,7 @@ func (s *Store) HandleRaftResponse( // other replicas are (#23994). Add it to the replica GC queue to do a // proper check. s.replicaGCQueue.AddAsync(ctx, repl, replicaGCPriorityDefault) - case *roachpb.StoreNotFoundError: + case *kvpb.StoreNotFoundError: log.Warningf(ctx, "raft error: node %d claims to not contain store %d for replica %s: %s", resp.FromReplica.NodeID, resp.FromReplica.StoreID, resp.FromReplica, val) return val.GetDetail() // close Raft connection @@ -584,7 +585,7 @@ func (s *Store) processRequestQueue(ctx context.Context, rangeID roachpb.RangeID for i := range infos { info := &infos[i] if pErr := s.withReplicaForRequest( - ctx, info.req, func(_ context.Context, r *Replica) *roachpb.Error { + ctx, info.req, func(_ context.Context, r *Replica) *kvpb.Error { return s.processRaftRequestWithReplica(r.raftCtx, r, info.req) }, ); pErr != nil { @@ -737,7 +738,7 @@ func (s *Store) processRaft(ctx context.Context) { prop.finishApplication( context.Background(), proposalResult{ - Err: roachpb.NewError(roachpb.NewAmbiguousResultErrorf("store is stopping")), + Err: kvpb.NewError(kvpb.NewAmbiguousResultErrorf("store is stopping")), }, ) } diff --git a/pkg/kv/kvserver/store_remove_replica.go b/pkg/kv/kvserver/store_remove_replica.go index e0cf6515a599..e34d06fe9bcd 100644 --- a/pkg/kv/kvserver/store_remove_replica.go +++ b/pkg/kv/kvserver/store_remove_replica.go @@ -14,6 +14,7 @@ import ( "context" "sync/atomic" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" @@ -129,7 +130,7 @@ func (s *Store) removeInitializedReplicaRaftMuLocked( } // Mark the replica as removed before deleting data. - rep.mu.destroyStatus.Set(roachpb.NewRangeNotFoundError(rep.RangeID, rep.StoreID()), + rep.mu.destroyStatus.Set(kvpb.NewRangeNotFoundError(rep.RangeID, rep.StoreID()), destroyReasonRemoved) rep.mu.Unlock() rep.readOnlyCmdMu.Unlock() @@ -167,7 +168,7 @@ func (s *Store) removeInitializedReplicaRaftMuLocked( // The replica will no longer exist, so cancel any rangefeed registrations. rep.disconnectRangefeedWithReason( - roachpb.RangeFeedRetryError_REASON_REPLICA_REMOVED, + kvpb.RangeFeedRetryError_REASON_REPLICA_REMOVED, ) // Mark the replica as destroyed and (optionally) destroy the on-disk data @@ -257,7 +258,7 @@ func (s *Store) removeUninitializedReplicaRaftMuLocked( } // Mark the replica as removed before deleting data. - rep.mu.destroyStatus.Set(roachpb.NewRangeNotFoundError(rep.RangeID, rep.StoreID()), + rep.mu.destroyStatus.Set(kvpb.NewRangeNotFoundError(rep.RangeID, rep.StoreID()), destroyReasonRemoved) rep.mu.Unlock() diff --git a/pkg/kv/kvserver/store_send.go b/pkg/kv/kvserver/store_send.go index ffc71549274e..282c1695a1c0 100644 --- a/pkg/kv/kvserver/store_send.go +++ b/pkg/kv/kvserver/store_send.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvadmission" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -44,8 +45,8 @@ import ( // of one of its writes), the response will have a transaction set which should // be used to update the client transaction object. func (s *Store) Send( - ctx context.Context, ba *roachpb.BatchRequest, -) (br *roachpb.BatchResponse, pErr *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (br *kvpb.BatchResponse, pErr *kvpb.Error) { var writeBytes *kvadmission.StoreWriteBytes br, writeBytes, pErr = s.SendWithWriteBytes(ctx, ba) writeBytes.Release() @@ -55,22 +56,22 @@ func (s *Store) Send( // SendWithWriteBytes is the implementation of Send with an additional // *StoreWriteBytes return value. func (s *Store) SendWithWriteBytes( - ctx context.Context, ba *roachpb.BatchRequest, -) (br *roachpb.BatchResponse, writeBytes *kvadmission.StoreWriteBytes, pErr *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (br *kvpb.BatchResponse, writeBytes *kvadmission.StoreWriteBytes, pErr *kvpb.Error) { // Attach any log tags from the store to the context (which normally // comes from gRPC). ctx = s.AnnotateCtx(ctx) for _, union := range ba.Requests { arg := union.GetInner() header := arg.Header() - if err := verifyKeys(header.Key, header.EndKey, roachpb.IsRange(arg)); err != nil { - return nil, nil, roachpb.NewError(errors.Wrapf(err, + if err := verifyKeys(header.Key, header.EndKey, kvpb.IsRange(arg)); err != nil { + return nil, nil, kvpb.NewError(errors.Wrapf(err, "failed to verify keys for %s", arg.Method())) } } if res, err := s.maybeThrottleBatch(ctx, ba); err != nil { - return nil, nil, roachpb.NewError(err) + return nil, nil, kvpb.NewError(err) } else if res != nil { defer res.Release() } @@ -84,7 +85,7 @@ func (s *Store) SendWithWriteBytes( } if err := ba.SetActiveTimestamp(s.Clock()); err != nil { - return nil, nil, roachpb.NewError(err) + return nil, nil, kvpb.NewError(err) } // Update our clock with the incoming request timestamp. This advances the @@ -109,7 +110,7 @@ func (s *Store) SendWithWriteBytes( // If the command appears to come from a node with a bad clock, // reject it instead of updating the local clock and proceeding. if err := s.cfg.Clock.UpdateAndCheckMaxOffset(ctx, baClockTS); err != nil { - return nil, nil, roachpb.NewError(err) + return nil, nil, kvpb.NewError(err) } } } @@ -180,7 +181,7 @@ func (s *Store) SendWithWriteBytes( // Get range and add command to the range for execution. repl, err := s.GetReplica(ba.RangeID) if err != nil { - return nil, nil, roachpb.NewError(err) + return nil, nil, kvpb.NewError(err) } if !repl.IsInitialized() { // If we have an uninitialized copy of the range, then we are probably a @@ -190,7 +191,7 @@ func (s *Store) SendWithWriteBytes( // the client that it should move on and try the next replica. Very // likely, the next replica the client tries will be initialized and will // have useful leaseholder information for the client. - return nil, nil, roachpb.NewError(&roachpb.NotLeaseHolderError{ + return nil, nil, kvpb.NewError(&kvpb.NotLeaseHolderError{ RangeID: ba.RangeID, // The replica doesn't have a range descriptor yet, so we have to build // a ReplicaDescriptor manually. @@ -217,7 +218,7 @@ func (s *Store) SendWithWriteBytes( // Augment error if necessary and return. switch t := pErr.GetDetail().(type) { - case *roachpb.RangeKeyMismatchError: + case *kvpb.RangeKeyMismatchError: // TODO(andrei): It seems silly that, if the client specified a RangeID that // doesn't match the keys it wanted to access, but this node can serve those // keys anyway, we still return a RangeKeyMismatchError to the client @@ -230,7 +231,7 @@ func (s *Store) SendWithWriteBytes( // Range from this Store. rSpan, err := keys.Range(ba.Requests) if err != nil { - return nil, nil, roachpb.NewError(err) + return nil, nil, kvpb.NewError(err) } // The kvclient thought that a particular range id covers rSpans. It was @@ -241,7 +242,7 @@ func (s *Store) SendWithWriteBytes( // that the client requested, and all the ranges in between. ri, err := t.MismatchedRange() if err != nil { - return nil, nil, roachpb.NewError(err) + return nil, nil, kvpb.NewError(err) } skipRID := ri.Desc.RangeID // We already have info on one range, so don't add it again below. startKey := ri.Desc.StartKey @@ -307,12 +308,12 @@ func (s *Store) SendWithWriteBytes( t.Ranges = append(rangeInfos, t.Ranges...) // We have to write `t` back to `pErr` so that it picks up the changes. - pErr = roachpb.NewError(t) - case *roachpb.RaftGroupDeletedError: + pErr = kvpb.NewError(t) + case *kvpb.RaftGroupDeletedError: // This error needs to be converted appropriately so that clients // will retry. - err := roachpb.NewRangeNotFoundError(repl.RangeID, repl.store.StoreID()) - pErr = roachpb.NewError(err) + err := kvpb.NewRangeNotFoundError(repl.RangeID, repl.store.StoreID()) + pErr = kvpb.NewError(err) } // Unable to retry, exit the retry loop and return an error. @@ -330,14 +331,14 @@ func (s *Store) SendWithWriteBytes( // before a request acquires latches on a range. Otherwise, the request could // inadvertently block others while being throttled. func (s *Store) maybeThrottleBatch( - ctx context.Context, ba *roachpb.BatchRequest, + ctx context.Context, ba *kvpb.BatchRequest, ) (limit.Reservation, error) { if !ba.IsSingleRequest() { return nil, nil } switch t := ba.Requests[0].GetInner().(type) { - case *roachpb.AddSSTableRequest: + case *kvpb.AddSSTableRequest: limiter := s.limiters.ConcurrentAddSSTableRequests if t.IngestAsWrites { limiter = s.limiters.ConcurrentAddSSTableAsWritesRequests @@ -362,7 +363,7 @@ func (s *Store) maybeThrottleBatch( } return res, nil - case *roachpb.ExportRequest: + case *kvpb.ExportRequest: // Limit the number of concurrent Export requests, as these often scan and // entire Range at a time and place significant read load on a Store. before := timeutil.Now() @@ -417,44 +418,44 @@ func (s *Store) maybeThrottleBatch( // For more information, see the "Server-side negotiation fast-path" section of // docs/RFCS/20210519_bounded_staleness_reads.md. func (s *Store) executeServerSideBoundedStalenessNegotiation( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchRequest, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchRequest, *kvpb.Error) { if ba.BoundedStaleness == nil { log.Fatal(ctx, "BoundedStaleness header required for server-side negotiation fast-path") } cfg := ba.BoundedStaleness if cfg.MinTimestampBound.IsEmpty() { - return ba, roachpb.NewError(errors.AssertionFailedf( + return ba, kvpb.NewError(errors.AssertionFailedf( "MinTimestampBound must be set in batch")) } if !cfg.MaxTimestampBound.IsEmpty() && cfg.MaxTimestampBound.LessEq(cfg.MinTimestampBound) { - return ba, roachpb.NewError(errors.AssertionFailedf( + return ba, kvpb.NewError(errors.AssertionFailedf( "MaxTimestampBound, if set in batch, must be greater than MinTimestampBound")) } if !ba.Timestamp.IsEmpty() { - return ba, roachpb.NewError(errors.AssertionFailedf( + return ba, kvpb.NewError(errors.AssertionFailedf( "MinTimestampBound and Timestamp cannot both be set in batch")) } if ba.Txn != nil { - return ba, roachpb.NewError(errors.AssertionFailedf( + return ba, kvpb.NewError(errors.AssertionFailedf( "MinTimestampBound and Txn cannot both be set in batch")) } // Use one or more QueryResolvedTimestampRequests to compute a resolved // timestamp over the read spans on the local replica. - queryResBa := &roachpb.BatchRequest{} + queryResBa := &kvpb.BatchRequest{} queryResBa.RangeID = ba.RangeID queryResBa.Replica = ba.Replica queryResBa.ClientRangeInfo = ba.ClientRangeInfo - queryResBa.ReadConsistency = roachpb.INCONSISTENT + queryResBa.ReadConsistency = kvpb.INCONSISTENT for _, ru := range ba.Requests { span := ru.GetInner().Header().Span() if len(span.EndKey) == 0 { // QueryResolvedTimestamp is a ranged operation. span.EndKey = span.Key.Next() } - queryResBa.Add(&roachpb.QueryResolvedTimestampRequest{ - RequestHeader: roachpb.RequestHeaderFromSpan(span), + queryResBa.Add(&kvpb.QueryResolvedTimestampRequest{ + RequestHeader: kvpb.RequestHeaderFromSpan(span), }) } @@ -484,7 +485,7 @@ func (s *Store) executeServerSideBoundedStalenessNegotiation( // to the current leaseholder. On the leaseholder, this may result in the // request blocking on conflicting transactions. if cfg.MinTimestampBoundStrict { - return ba, roachpb.NewError(roachpb.NewMinTimestampBoundUnsatisfiableError( + return ba, kvpb.NewError(kvpb.NewMinTimestampBoundUnsatisfiableError( cfg.MinTimestampBound, resTS, )) } diff --git a/pkg/kv/kvserver/store_snapshot.go b/pkg/kv/kvserver/store_snapshot.go index 969a81e5b58c..348bd1bf182f 100644 --- a/pkg/kv/kvserver/store_snapshot.go +++ b/pkg/kv/kvserver/store_snapshot.go @@ -16,6 +16,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" @@ -1000,17 +1001,17 @@ func (s *Store) receiveSnapshot( var placeholder *ReplicaPlaceholder if pErr := s.withReplicaForRequest( ctx, &header.RaftMessageRequest, func(ctx context.Context, r *Replica, - ) *roachpb.Error { + ) *kvpb.Error { var err error s.mu.Lock() defer s.mu.Unlock() placeholder, err = s.canAcceptSnapshotLocked(ctx, header) if err != nil { - return roachpb.NewError(err) + return kvpb.NewError(err) } if placeholder != nil { if err := s.addPlaceholderLocked(placeholder); err != nil { - return roachpb.NewError(err) + return kvpb.NewError(err) } } return nil diff --git a/pkg/kv/kvserver/store_test.go b/pkg/kv/kvserver/store_test.go index c873803bc9a5..d3ea0c46a7ef 100644 --- a/pkg/kv/kvserver/store_test.go +++ b/pkg/kv/kvserver/store_test.go @@ -32,6 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/allocatorimpl" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" @@ -85,7 +86,7 @@ var testIdent = roachpb.StoreIdent{ } func (s *Store) TestSender() kv.Sender { - return kv.Wrap(s, func(ba *roachpb.BatchRequest) *roachpb.BatchRequest { + return kv.Wrap(s, func(ba *kvpb.BatchRequest) *kvpb.BatchRequest { if ba.RangeID != 0 { return ba } @@ -716,7 +717,7 @@ func TestStoreRemoveReplicaDestroy(t *testing.T) { t.Fatal("replica was not marked as destroyed") } - if _, err = repl1.checkExecutionCanProceedBeforeStorageSnapshot(ctx, &roachpb.BatchRequest{}, nil /* g */); !errors.Is(err, expErr) { + if _, err = repl1.checkExecutionCanProceedBeforeStorageSnapshot(ctx, &kvpb.BatchRequest{}, nil /* g */); !errors.Is(err, expErr) { t.Fatalf("expected error %s, but got %v", expErr, err) } } @@ -921,10 +922,10 @@ func TestStoreObservedTimestamp(t *testing.T) { testCases := []struct { key roachpb.Key - check func(int64, roachpb.NodeID, roachpb.Response, *roachpb.Error) + check func(int64, roachpb.NodeID, kvpb.Response, *kvpb.Error) }{ {badKey, - func(wallNanos int64, nodeID roachpb.NodeID, _ roachpb.Response, pErr *roachpb.Error) { + func(wallNanos int64, nodeID roachpb.NodeID, _ kvpb.Response, pErr *kvpb.Error) { if pErr == nil { t.Fatal("expected an error") } @@ -943,7 +944,7 @@ func TestStoreObservedTimestamp(t *testing.T) { }}, {goodKey, - func(wallNanos int64, nodeID roachpb.NodeID, pReply roachpb.Response, pErr *roachpb.Error) { + func(wallNanos int64, nodeID roachpb.NodeID, pReply kvpb.Response, pErr *kvpb.Error) { if pErr != nil { t.Fatal(pErr) } @@ -963,9 +964,9 @@ func TestStoreObservedTimestamp(t *testing.T) { manual := timeutil.NewManualTime(timeutil.Unix(0, 123)) cfg := TestStoreConfig(hlc.NewClockForTesting(manual)) cfg.TestingKnobs.EvalKnobs.TestingEvalFilter = - func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { + func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { if bytes.Equal(filterArgs.Req.Header().Key, badKey) { - return roachpb.NewError(errors.Errorf("boom")) + return kvpb.NewError(errors.Errorf("boom")) } return nil } @@ -975,7 +976,7 @@ func TestStoreObservedTimestamp(t *testing.T) { store := createTestStoreWithConfig(ctx, t, stopper, testStoreOpts{createSystemRanges: true}, &cfg) txn := newTransaction("test", test.key, 1, store.cfg.Clock) txn.GlobalUncertaintyLimit = hlc.MaxTimestamp - h := roachpb.Header{Txn: txn} + h := kvpb.Header{Txn: txn} pArgs := putArgs(test.key, []byte("value")) assignSeqNumsForReqs(txn, &pArgs) pReply, pErr := kv.SendWrappedWith(ctx, store.TestSender(), h, &pArgs) @@ -1000,10 +1001,10 @@ func TestStoreAnnotateNow(t *testing.T) { testCases := []struct { key roachpb.Key - check func(*roachpb.BatchResponse, *roachpb.Error) + check func(*kvpb.BatchResponse, *kvpb.Error) }{ {badKey, - func(_ *roachpb.BatchResponse, pErr *roachpb.Error) { + func(_ *kvpb.BatchResponse, pErr *kvpb.Error) { if pErr == nil { t.Fatal("expected an error") } @@ -1012,7 +1013,7 @@ func TestStoreAnnotateNow(t *testing.T) { } }}, {goodKey, - func(pReply *roachpb.BatchResponse, pErr *roachpb.Error) { + func(pReply *kvpb.BatchResponse, pErr *kvpb.Error) { if pErr != nil { t.Fatal(pErr) } @@ -1027,9 +1028,9 @@ func TestStoreAnnotateNow(t *testing.T) { t.Run(test.key.String(), func(t *testing.T) { cfg := TestStoreConfig(nil) cfg.TestingKnobs.EvalKnobs.TestingEvalFilter = - func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { + func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { if bytes.Equal(filterArgs.Req.Header().Key, badKey) { - return roachpb.NewErrorWithTxn(errors.Errorf("boom"), filterArgs.Hdr.Txn) + return kvpb.NewErrorWithTxn(errors.Errorf("boom"), filterArgs.Hdr.Txn) } return nil } @@ -1043,8 +1044,8 @@ func TestStoreAnnotateNow(t *testing.T) { txn.GlobalUncertaintyLimit = hlc.MaxTimestamp assignSeqNumsForReqs(txn, &pArgs) } - ba := &roachpb.BatchRequest{ - Header: roachpb.Header{ + ba := &kvpb.BatchRequest{ + Header: kvpb.Header{ Txn: txn, Replica: desc, }, @@ -1128,7 +1129,7 @@ func TestStoreSendUpdateTime(t *testing.T) { store, _ := createTestStore(ctx, t, testStoreOpts{createSystemRanges: true}, stopper) args := getArgs([]byte("a")) now := hlc.ClockTimestamp(store.cfg.Clock.Now().Add(store.cfg.Clock.MaxOffset().Nanoseconds(), 0)) - _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{Now: now}, &args) + _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{Now: now}, &args) if pErr != nil { t.Fatal(pErr) } @@ -1149,7 +1150,7 @@ func TestStoreSendWithZeroTime(t *testing.T) { store, _ := createTestStore(ctx, t, testStoreOpts{createSystemRanges: true}, stopper) args := getArgs([]byte("a")) - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Add(&args) br, pErr := store.TestSender().Send(ctx, ba) if pErr != nil { @@ -1178,7 +1179,7 @@ func TestStoreSendWithClockOffset(t *testing.T) { args := getArgs([]byte("a")) // Set args timestamp to exceed max offset. now := hlc.ClockTimestamp(store.cfg.Clock.Now().Add(store.cfg.Clock.MaxOffset().Nanoseconds()+1, 0)) - _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{Now: now}, &args) + _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{Now: now}, &args) if !testutils.IsPError(pErr, "remote wall time is too far ahead") { t.Errorf("unexpected error: %v", pErr) } @@ -1193,8 +1194,8 @@ func TestStoreSendWithClockOffset(t *testing.T) { func splitTestRange(store *Store, splitKey roachpb.RKey, t *testing.T) *Replica { ctx := context.Background() repl := store.LookupReplica(splitKey) - _, err := repl.AdminSplit(ctx, roachpb.AdminSplitRequest{ - RequestHeader: roachpb.RequestHeader{Key: splitKey.AsRawKey()}, + _, err := repl.AdminSplit(ctx, kvpb.AdminSplitRequest{ + RequestHeader: kvpb.RequestHeader{Key: splitKey.AsRawKey()}, SplitKey: splitKey.AsRawKey(), ExpirationTime: store.Clock().Now().Add(24*time.Hour.Nanoseconds(), 0), }, "splitTestRange") @@ -1216,7 +1217,7 @@ func TestStoreSendOutOfRange(t *testing.T) { // key 'a' isn't in Range 1000 and Range 1000 doesn't exist // adjacent on this store args := getArgs([]byte("a")) - if _, err := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ + if _, err := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{ RangeID: 1000, // doesn't exist }, &args); err == nil { t.Error("expected key to be out of range") @@ -1229,7 +1230,7 @@ func TestStoreSendOutOfRange(t *testing.T) { // fail because it's before the start of the range and straddles multiple ranges // so it cannot be server side retried. scanArgs := scanArgs([]byte("a"), []byte("c")) - if _, err := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ + if _, err := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{ RangeID: repl2.RangeID, }, scanArgs); err == nil { t.Error("expected key to be out of range") @@ -1325,13 +1326,13 @@ func TestStoreResolveWriteIntent(t *testing.T) { cfg := TestStoreConfig(hlc.NewClock( manual, 1000*time.Nanosecond /* maxOffset */, 1000*time.Nanosecond /* toleratedOffset */)) cfg.TestingKnobs.EvalKnobs.TestingEvalFilter = - func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { - pr, ok := filterArgs.Req.(*roachpb.PushTxnRequest) + func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { + pr, ok := filterArgs.Req.(*kvpb.PushTxnRequest) if !ok || pr.PusherTxn.Name != "test" { return nil } if exp, act := manual.Now().UnixNano(), pr.PushTo.WallTime; exp > act { - return roachpb.NewError(fmt.Errorf("expected PushTo > WallTime, but got %d < %d:\n%+v", act, exp, pr)) + return kvpb.NewError(fmt.Errorf("expected PushTo > WallTime, but got %d < %d:\n%+v", act, exp, pr)) } return nil } @@ -1354,7 +1355,7 @@ func TestStoreResolveWriteIntent(t *testing.T) { // First lay down intent using the pushee's txn. pArgs := putArgs(key, []byte("value")) - h := roachpb.Header{Txn: pushee} + h := kvpb.Header{Txn: pushee} assignSeqNumsForReqs(pushee, &pArgs) if _, err := kv.SendWrappedWith(ctx, store.TestSender(), h, &pArgs); err != nil { t.Fatal(err) @@ -1363,7 +1364,7 @@ func TestStoreResolveWriteIntent(t *testing.T) { manual.Advance(100) // Now, try a put using the pusher's txn. h.Txn = pusher - resultCh := make(chan *roachpb.Error, 1) + resultCh := make(chan *kvpb.Error, 1) go func() { _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), h, &pArgs) resultCh <- pErr @@ -1419,7 +1420,7 @@ func TestStoreResolveWriteIntentRollback(t *testing.T) { // First lay down intent using the pushee's txn. args := incrementArgs(key, 1) - h := roachpb.Header{Txn: pushee} + h := kvpb.Header{Txn: pushee} assignSeqNumsForReqs(pushee, args) if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), h, args); pErr != nil { t.Fatal(pErr) @@ -1431,7 +1432,7 @@ func TestStoreResolveWriteIntentRollback(t *testing.T) { assignSeqNumsForReqs(pusher, args) if resp, pErr := kv.SendWrappedWith(ctx, store.TestSender(), h, args); pErr != nil { t.Errorf("expected increment to succeed: %s", pErr) - } else if reply := resp.(*roachpb.IncrementResponse); reply.NewValue != 2 { + } else if reply := resp.(*kvpb.IncrementResponse); reply.NewValue != 2 { t.Errorf("expected rollback of earlier increment to yield increment value of 2; got %d", reply.NewValue) } } @@ -1555,7 +1556,7 @@ func TestStoreResolveWriteIntentPushOnRead(t *testing.T) { { args := putArgs(key, []byte("value2")) assignSeqNumsForReqs(pushee, &args) - h := roachpb.Header{Txn: pushee} + h := kvpb.Header{Txn: pushee} if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), h, &args); pErr != nil { t.Fatal(pErr) } @@ -1598,11 +1599,11 @@ func TestStoreResolveWriteIntentPushOnRead(t *testing.T) { pusher.WriteTimestamp.Forward(readTs) gArgs := getArgs(key) assignSeqNumsForReqs(pusher, &gArgs) - repl, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{Txn: pusher}, &gArgs) + repl, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{Txn: pusher}, &gArgs) if tc.expPushError == "" { if pErr != nil { t.Errorf("expected read to succeed: %s", pErr) - } else if replyBytes, err := repl.(*roachpb.GetResponse).Value.GetBytes(); err != nil { + } else if replyBytes, err := repl.(*kvpb.GetResponse).Value.GetBytes(); err != nil { t.Fatal(err) } else if !bytes.Equal(replyBytes, []byte("value1")) { t.Errorf("expected bytes to be %q, got %q", "value1", replyBytes) @@ -1619,7 +1620,7 @@ func TestStoreResolveWriteIntentPushOnRead(t *testing.T) { assignSeqNumsForReqs(pushee, &etArgs) _, pErr = kv.SendWrappedWith(ctx, store.TestSender(), etH, &etArgs) if tc.expPusheeRetry { - if _, ok := pErr.GetDetail().(*roachpb.TransactionRetryError); !ok { + if _, ok := pErr.GetDetail().(*kvpb.TransactionRetryError); !ok { t.Errorf("expected transaction retry error; got %s", pErr) } } else { @@ -1661,12 +1662,12 @@ func TestStoreResolveWriteIntentNoTxn(t *testing.T) { getTS := store.cfg.Clock.Now() // accessed later { gArgs := getArgs(key) - if reply, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ + if reply, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{ Timestamp: getTS, UserPriority: roachpb.MaxUserPriority, }, &gArgs); pErr != nil { t.Errorf("expected read to succeed: %s", pErr) - } else if gReply := reply.(*roachpb.GetResponse); gReply.Value != nil { + } else if gReply := reply.(*kvpb.GetResponse); gReply.Value != nil { t.Errorf("expected value to be nil, got %+v", gReply.Value) } } @@ -1675,7 +1676,7 @@ func TestStoreResolveWriteIntentNoTxn(t *testing.T) { // Next, try to write outside of a transaction. We will succeed in pushing txn. putTS := store.cfg.Clock.Now() args.Value.SetBytes([]byte("value2")) - if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ + if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{ Timestamp: putTS, UserPriority: roachpb.MaxUserPriority, }, &args); pErr != nil { @@ -1716,7 +1717,7 @@ func TestStoreResolveWriteIntentNoTxn(t *testing.T) { if pErr == nil { t.Errorf("unexpected success committing transaction") } - if _, ok := pErr.GetDetail().(*roachpb.TransactionAbortedError); !ok { + if _, ok := pErr.GetDetail().(*kvpb.TransactionAbortedError); !ok { t.Errorf("expected transaction aborted error; got %s", pErr) } } @@ -1732,9 +1733,9 @@ func TestStoreReadInconsistent(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, rc := range []roachpb.ReadConsistencyType{ - roachpb.READ_UNCOMMITTED, - roachpb.INCONSISTENT, + for _, rc := range []kvpb.ReadConsistencyType{ + kvpb.READ_UNCOMMITTED, + kvpb.INCONSISTENT, } { t.Run(rc.String(), func(t *testing.T) { // The test relies on being able to commit a Txn without specifying the @@ -1769,7 +1770,7 @@ func TestStoreReadInconsistent(t *testing.T) { for _, txn := range []*roachpb.Transaction{txnA, txnB} { args.Key = txn.Key assignSeqNumsForReqs(txn, &args) - if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{Txn: txn}, &args); pErr != nil { + if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{Txn: txn}, &args); pErr != nil { t.Fatal(pErr) } } @@ -1784,24 +1785,24 @@ func TestStoreReadInconsistent(t *testing.T) { // will be able to read with both INCONSISTENT and READ_UNCOMMITTED. // With READ_UNCOMMITTED, we'll also be able to see the intent's value. gArgs := getArgs(keyA) - if reply, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ + if reply, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{ ReadConsistency: rc, }, &gArgs); pErr != nil { t.Errorf("expected read to succeed: %s", pErr) } else { - gReply := reply.(*roachpb.GetResponse) + gReply := reply.(*kvpb.GetResponse) if replyBytes, err := gReply.Value.GetBytes(); err != nil { t.Fatal(err) } else if !bytes.Equal(replyBytes, []byte("value1")) { t.Errorf("expected value %q, got %+v", []byte("value1"), reply) - } else if rc == roachpb.READ_UNCOMMITTED { + } else if rc == kvpb.READ_UNCOMMITTED { // READ_UNCOMMITTED will also return the intent. if replyIntentBytes, err := gReply.IntentValue.GetBytes(); err != nil { t.Fatal(err) } else if !bytes.Equal(replyIntentBytes, []byte("value2")) { t.Errorf("expected value %q, got %+v", []byte("value2"), reply) } - } else if rc == roachpb.INCONSISTENT { + } else if rc == kvpb.INCONSISTENT { if gReply.IntentValue != nil { t.Errorf("expected value nil, got %+v", gReply.IntentValue) } @@ -1809,23 +1810,23 @@ func TestStoreReadInconsistent(t *testing.T) { } gArgs.Key = keyB - if reply, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ + if reply, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{ ReadConsistency: rc, }, &gArgs); pErr != nil { t.Errorf("expected read to succeed: %s", pErr) } else { - gReply := reply.(*roachpb.GetResponse) + gReply := reply.(*kvpb.GetResponse) if gReply.Value != nil { // The new value of B will not be read at first. t.Errorf("expected value nil, got %+v", gReply.Value) - } else if rc == roachpb.READ_UNCOMMITTED { + } else if rc == kvpb.READ_UNCOMMITTED { // READ_UNCOMMITTED will also return the intent. if replyIntentBytes, err := gReply.IntentValue.GetBytes(); err != nil { t.Fatal(err) } else if !bytes.Equal(replyIntentBytes, []byte("value2")) { t.Errorf("expected value %q, got %+v", []byte("value2"), reply) } - } else if rc == roachpb.INCONSISTENT { + } else if rc == kvpb.INCONSISTENT { if gReply.IntentValue != nil { t.Errorf("expected value nil, got %+v", gReply.IntentValue) } @@ -1834,11 +1835,11 @@ func TestStoreReadInconsistent(t *testing.T) { // However, it will be read eventually, as B's intent can be // resolved asynchronously as txn B is committed. testutils.SucceedsSoon(t, func() error { - if reply, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ + if reply, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{ ReadConsistency: rc, }, &gArgs); pErr != nil { return errors.Errorf("expected read to succeed: %s", pErr) - } else if gReply := reply.(*roachpb.GetResponse).Value; gReply == nil { + } else if gReply := reply.(*kvpb.GetResponse).Value; gReply == nil { return errors.Errorf("value is nil") } else if replyBytes, err := gReply.GetBytes(); err != nil { return err @@ -1850,13 +1851,13 @@ func TestStoreReadInconsistent(t *testing.T) { // Scan keys and verify results. sArgs := scanArgs(keyA, keyB.Next()) - reply, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ + reply, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{ ReadConsistency: rc, }, sArgs) if pErr != nil { t.Errorf("expected scan to succeed: %s", pErr) } - sReply := reply.(*roachpb.ScanResponse) + sReply := reply.(*kvpb.ScanResponse) if l := len(sReply.Rows); l != 2 { t.Errorf("expected 2 results; got %d", l) } else if key := sReply.Rows[0].Key; !key.Equal(keyA) { @@ -1871,7 +1872,7 @@ func TestStoreReadInconsistent(t *testing.T) { t.Fatal(err) } else if !bytes.Equal(val2, []byte("value2")) { t.Errorf("expected value %q, got %q", []byte("value2"), val2) - } else if rc == roachpb.READ_UNCOMMITTED { + } else if rc == kvpb.READ_UNCOMMITTED { if l := len(sReply.IntentRows); l != 1 { t.Errorf("expected 1 intent result; got %d", l) } else if intentKey := sReply.IntentRows[0].Key; !intentKey.Equal(keyA) { @@ -1881,7 +1882,7 @@ func TestStoreReadInconsistent(t *testing.T) { } else if !bytes.Equal(intentVal1, []byte("value2")) { t.Errorf("expected intent value %q, got %q", []byte("value2"), intentVal1) } - } else if rc == roachpb.INCONSISTENT { + } else if rc == kvpb.INCONSISTENT { if l := len(sReply.IntentRows); l != 0 { t.Errorf("expected 0 intent result; got %d", l) } @@ -1889,13 +1890,13 @@ func TestStoreReadInconsistent(t *testing.T) { // Reverse scan keys and verify results. rsArgs := revScanArgs(keyA, keyB.Next()) - reply, pErr = kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ + reply, pErr = kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{ ReadConsistency: rc, }, rsArgs) if pErr != nil { t.Errorf("expected scan to succeed: %s", pErr) } - rsReply := reply.(*roachpb.ReverseScanResponse) + rsReply := reply.(*kvpb.ReverseScanResponse) if l := len(rsReply.Rows); l != 2 { t.Errorf("expected 2 results; got %d", l) } else if key := rsReply.Rows[0].Key; !key.Equal(keyB) { @@ -1910,7 +1911,7 @@ func TestStoreReadInconsistent(t *testing.T) { t.Fatal(err) } else if !bytes.Equal(val2, []byte("value1")) { t.Errorf("expected value %q, got %q", []byte("value1"), val2) - } else if rc == roachpb.READ_UNCOMMITTED { + } else if rc == kvpb.READ_UNCOMMITTED { if l := len(rsReply.IntentRows); l != 1 { t.Errorf("expected 1 intent result; got %d", l) } else if intentKey := rsReply.IntentRows[0].Key; !intentKey.Equal(keyA) { @@ -1920,7 +1921,7 @@ func TestStoreReadInconsistent(t *testing.T) { } else if !bytes.Equal(intentVal1, []byte("value2")) { t.Errorf("expected intent value %q, got %q", []byte("value2"), intentVal1) } - } else if rc == roachpb.INCONSISTENT { + } else if rc == kvpb.INCONSISTENT { if l := len(rsReply.IntentRows); l != 0 { t.Errorf("expected 0 intent result; got %d", l) } @@ -1949,7 +1950,7 @@ func TestStoreScanResumeTSCache(t *testing.T) { // Write three keys at time t0. t0 := timeutil.Unix(1, 0) manualClock.MustAdvanceTo(t0) - h := roachpb.Header{Timestamp: makeTS(t0.UnixNano(), 0)} + h := kvpb.Header{Timestamp: makeTS(t0.UnixNano(), 0)} for _, keyStr := range []string{"a", "b", "c"} { key := roachpb.Key(keyStr) putArgs := putArgs(key, []byte("value")) @@ -1969,7 +1970,7 @@ func TestStoreScanResumeTSCache(t *testing.T) { if pErr != nil { t.Fatal(pErr) } - sReply := reply.(*roachpb.ScanResponse) + sReply := reply.(*kvpb.ScanResponse) if a, e := len(sReply.Rows), 2; a != e { t.Errorf("expected %d rows; got %d", e, a) } @@ -1997,7 +1998,7 @@ func TestStoreScanResumeTSCache(t *testing.T) { if pErr != nil { t.Fatal(pErr) } - rsReply := reply.(*roachpb.ReverseScanResponse) + rsReply := reply.(*kvpb.ReverseScanResponse) if a, e := len(rsReply.Rows), 2; a != e { t.Errorf("expected %d rows; got %d", e, a) } @@ -2023,7 +2024,7 @@ func TestStoreScanResumeTSCache(t *testing.T) { t3 := timeutil.Unix(4, 0) manualClock.MustAdvanceTo(t3) h.Timestamp = makeTS(t3.UnixNano(), 0) - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Header = h ba.Add(getArgsString("a"), getArgsString("b"), getArgsString("c")) br, pErr := store.TestSender().Send(ctx, ba) @@ -2058,11 +2059,11 @@ func TestStoreSkipLockedTSCache(t *testing.T) { for _, tc := range []struct { name string - reqs []roachpb.Request + reqs []kvpb.Request }{ - {"get", []roachpb.Request{getArgsString("a"), getArgsString("b"), getArgsString("c")}}, - {"scan", []roachpb.Request{scanArgsString("a", "d")}}, - {"revscan", []roachpb.Request{revScanArgsString("a", "d")}}, + {"get", []kvpb.Request{getArgsString("a"), getArgsString("b"), getArgsString("c")}}, + {"scan", []kvpb.Request{scanArgsString("a", "d")}}, + {"revscan", []kvpb.Request{revScanArgsString("a", "d")}}, } { t.Run(tc.name, func(t *testing.T) { ctx := context.Background() @@ -2073,7 +2074,7 @@ func TestStoreSkipLockedTSCache(t *testing.T) { // Write three keys at time t0. t0 := timeutil.Unix(1, 0) manualClock.MustAdvanceTo(t0) - h := roachpb.Header{Timestamp: makeTS(t0.UnixNano(), 0)} + h := kvpb.Header{Timestamp: makeTS(t0.UnixNano(), 0)} for _, keyStr := range []string{"a", "b", "c"} { putArgs := putArgs(roachpb.Key(keyStr), []byte("value")) _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), h, &putArgs) @@ -2085,7 +2086,7 @@ func TestStoreSkipLockedTSCache(t *testing.T) { manualClock.MustAdvanceTo(t1) lockedKey := roachpb.Key("b") txn := roachpb.MakeTransaction("locker", lockedKey, 0, makeTS(t1.UnixNano(), 0), 0, 0) - txnH := roachpb.Header{Txn: &txn} + txnH := kvpb.Header{Txn: &txn} putArgs := putArgs(lockedKey, []byte("newval")) _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), txnH, &putArgs) require.Nil(t, pErr) @@ -2093,7 +2094,7 @@ func TestStoreSkipLockedTSCache(t *testing.T) { // Read the span at t2 using a SkipLocked wait policy. t2 := timeutil.Unix(3, 0) manualClock.MustAdvanceTo(t2) - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Timestamp = makeTS(t2.UnixNano(), 0) ba.WaitPolicy = lock.WaitPolicy_SkipLocked ba.Add(tc.reqs...) @@ -2104,7 +2105,7 @@ func TestStoreSkipLockedTSCache(t *testing.T) { var respKeys []string for i, ru := range br.Responses { req, resp := ba.Requests[i].GetInner(), ru.GetInner() - require.NoError(t, roachpb.ResponseKeyIterate(req, resp, func(k roachpb.Key) { + require.NoError(t, kvpb.ResponseKeyIterate(req, resp, func(k roachpb.Key) { respKeys = append(respKeys, string(k)) })) } @@ -2134,8 +2135,8 @@ func TestStoreScanIntents(t *testing.T) { countPtr := &count cfg.TestingKnobs.EvalKnobs.TestingEvalFilter = - func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { - if req, ok := filterArgs.Req.(*roachpb.ScanRequest); ok { + func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { + if req, ok := filterArgs.Req.(*kvpb.ScanRequest); ok { // Avoid counting scan requests not generated by this test, e.g. those // generated by periodic gossips. if bytes.HasPrefix(req.Key, []byte(t.Name())) { @@ -2184,7 +2185,7 @@ func TestStoreScanIntents(t *testing.T) { } args := putArgs(key, []byte(fmt.Sprintf("value%02d", j))) assignSeqNumsForReqs(txn, &args) - if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{Txn: txn}, &args); pErr != nil { + if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{Txn: txn}, &args); pErr != nil { t.Fatal(pErr) } } @@ -2192,20 +2193,20 @@ func TestStoreScanIntents(t *testing.T) { // Scan the range and verify count. Do this in a goroutine in case // it isn't expected to finish. sArgs := scanArgs(keys[0], keys[9].Next()) - var sReply *roachpb.ScanResponse + var sReply *kvpb.ScanResponse ts := store.Clock().Now() - consistency := roachpb.CONSISTENT + consistency := kvpb.CONSISTENT if !test.consistent { - consistency = roachpb.INCONSISTENT + consistency = kvpb.INCONSISTENT } - errChan := make(chan *roachpb.Error, 1) + errChan := make(chan *kvpb.Error, 1) go func() { - reply, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ + reply, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{ Timestamp: ts, ReadConsistency: consistency, }, sArgs) if pErr == nil { - sReply = reply.(*roachpb.ScanResponse) + sReply = reply.(*kvpb.ScanResponse) } errChan <- pErr }() @@ -2272,14 +2273,14 @@ func TestStoreScanIntentsRespectsLimit(t *testing.T) { Knobs: base.TestingKnobs{ Store: &StoreTestingKnobs{ TestingConcurrencyRetryFilter: func( - ctx context.Context, ba *roachpb.BatchRequest, pErr *roachpb.Error, + ctx context.Context, ba *kvpb.BatchRequest, pErr *kvpb.Error, ) { - if errors.HasType(pErr.GoError(), (*roachpb.WriteIntentError)(nil)) { + if errors.HasType(pErr.GoError(), (*kvpb.WriteIntentError)(nil)) { // Assert that the WriteIntentError has MaxIntentsPerWriteIntentErrorIntents. if trap := interceptWriteIntentErrors.Load(); trap != nil && trap.(bool) { require.Equal( t, storage.MaxIntentsPerWriteIntentErrorDefault, - len(pErr.GetDetail().(*roachpb.WriteIntentError).Intents), + len(pErr.GetDetail().(*kvpb.WriteIntentError).Intents), ) interceptWriteIntentErrors.Store(false) // Allow the writer to commit. @@ -2312,7 +2313,7 @@ func TestStoreScanIntentsRespectsLimit(t *testing.T) { key = append(key, []byte(fmt.Sprintf("%d", j))...) intentKeys = append(intentKeys, key) args := putArgs(key, []byte(fmt.Sprintf("value%07d", j))) - _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{Txn: txn}, &args) + _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{Txn: txn}, &args) require.Nil(t, pErr) } intentsLaidDownCh <- struct{}{} @@ -2361,10 +2362,10 @@ func TestStoreScanInconsistentResolvesIntents(t *testing.T) { intercept.Store(uuid.Nil) cfg := TestStoreConfig(nil) cfg.TestingKnobs.EvalKnobs.TestingEvalFilter = - func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { - req, ok := filterArgs.Req.(*roachpb.ResolveIntentRequest) + func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { + req, ok := filterArgs.Req.(*kvpb.ResolveIntentRequest) if ok && intercept.Load().(uuid.UUID).Equal(req.IntentTxn.ID) { - return roachpb.NewErrorWithTxn(errors.Errorf("boom"), filterArgs.Hdr.Txn) + return kvpb.NewErrorWithTxn(errors.Errorf("boom"), filterArgs.Hdr.Txn) } return nil } @@ -2402,7 +2403,7 @@ func TestStoreScanInconsistentResolvesIntents(t *testing.T) { testutils.SucceedsSoon(t, func() error { var b kv.Batch b.Scan(sl[0], sl[9].Next()) - b.Header.ReadConsistency = roachpb.INCONSISTENT + b.Header.ReadConsistency = kvpb.INCONSISTENT require.NoError(t, store.DB().Run(ctx, &b)) if exp, act := len(sl), len(b.Results[0].Rows); exp != act { return errors.Errorf("expected %d keys, scanned %d", exp, act) @@ -2429,7 +2430,7 @@ func TestStoreScanIntentsFromTwoTxns(t *testing.T) { txn1 := newTransaction("test1", key1, 1, store.cfg.Clock) args := putArgs(key1, []byte("value1")) assignSeqNumsForReqs(txn1, &args) - if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{Txn: txn1}, &args); pErr != nil { + if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{Txn: txn1}, &args); pErr != nil { t.Fatal(pErr) } @@ -2437,7 +2438,7 @@ func TestStoreScanIntentsFromTwoTxns(t *testing.T) { txn2 := newTransaction("test2", key2, 1, store.cfg.Clock) args = putArgs(key2, []byte("value2")) assignSeqNumsForReqs(txn2, &args) - if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{Txn: txn2}, &args); pErr != nil { + if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{Txn: txn2}, &args); pErr != nil { t.Fatal(pErr) } @@ -2449,9 +2450,9 @@ func TestStoreScanIntentsFromTwoTxns(t *testing.T) { // Scan the range and verify empty result (expired txn is aborted, // cleaning up intents). sArgs := scanArgs(key1, key2.Next()) - if reply, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{}, sArgs); pErr != nil { + if reply, pErr := kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{}, sArgs); pErr != nil { t.Fatal(pErr) - } else if sReply := reply.(*roachpb.ScanResponse); len(sReply.Rows) != 0 { + } else if sReply := reply.(*kvpb.ScanResponse); len(sReply.Rows) != 0 { t.Errorf("expected empty result; got %+v", sReply.Rows) } } @@ -2468,8 +2469,8 @@ func TestStoreScanMultipleIntents(t *testing.T) { manual := timeutil.NewManualTime(timeutil.Unix(0, 123)) cfg := TestStoreConfig(hlc.NewClockForTesting(manual)) cfg.TestingKnobs.EvalKnobs.TestingEvalFilter = - func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { - if _, ok := filterArgs.Req.(*roachpb.ResolveIntentRequest); ok { + func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { + if _, ok := filterArgs.Req.(*kvpb.ResolveIntentRequest); ok { atomic.AddInt32(&resolveCount, 1) } return nil @@ -2483,13 +2484,13 @@ func TestStoreScanMultipleIntents(t *testing.T) { key1 := roachpb.Key("key00") key10 := roachpb.Key("key09") txn := newTransaction("test", key1, 1, store.cfg.Clock) - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} for i := 0; i < 10; i++ { pArgs := putArgs(roachpb.Key(fmt.Sprintf("key%02d", i)), []byte("value")) ba.Add(&pArgs) assignSeqNumsForReqs(txn, &pArgs) } - ba.Header = roachpb.Header{Txn: txn} + ba.Header = kvpb.Header{Txn: txn} if _, pErr := store.TestSender().Send(ctx, ba); pErr != nil { t.Fatal(pErr) } @@ -2546,12 +2547,12 @@ func TestStoreBadRequests(t *testing.T) { tArgs3, tHeader3 := heartbeatArgs(txn, hlc.Timestamp{}) tHeader3.Txn.Key = roachpb.Key(tHeader3.Txn.Key).Next() - tArgs4 := pushTxnArgs(txn, txn, roachpb.PUSH_ABORT) + tArgs4 := pushTxnArgs(txn, txn, kvpb.PUSH_ABORT) tArgs4.PusheeTxn.Key = roachpb.Key(txn.Key).Next() testCases := []struct { - args roachpb.Request - header *roachpb.Header + args kvpb.Request + header *kvpb.Header err string }{ // EndKey for non-Range is invalid. @@ -2574,7 +2575,7 @@ func TestStoreBadRequests(t *testing.T) { for i, test := range testCases { t.Run("", func(t *testing.T) { if test.header == nil { - test.header = &roachpb.Header{} + test.header = &kvpb.Header{} } if test.header.Txn != nil { assignSeqNumsForReqs(test.header.Txn, test.args) @@ -2695,15 +2696,15 @@ func TestStoreGCThreshold(t *testing.T) { WallTime: 2e9, } - gcr := roachpb.GCRequest{ + gcr := kvpb.GCRequest{ // Bogus span to make it a valid request. - RequestHeader: roachpb.RequestHeader{ + RequestHeader: kvpb.RequestHeader{ Key: roachpb.Key("a"), EndKey: roachpb.Key("b"), }, Threshold: threshold, } - if _, pErr := tc.SendWrappedWith(roachpb.Header{RangeID: 1}, &gcr); pErr != nil { + if _, pErr := tc.SendWrappedWith(kvpb.Header{RangeID: 1}, &gcr); pErr != nil { t.Fatal(pErr) } diff --git a/pkg/kv/kvserver/stores.go b/pkg/kv/kvserver/stores.go index f47ed9b91f62..9e7551f6d4f2 100644 --- a/pkg/kv/kvserver/stores.go +++ b/pkg/kv/kvserver/stores.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvadmission" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -64,7 +65,7 @@ func NewStores(ambient log.AmbientContext, clock *hlc.Clock) *Stores { // the meta1 lease. Returns an error if any. func (ls *Stores) IsMeta1Leaseholder(ctx context.Context, now hlc.ClockTimestamp) (bool, error) { repl, _, err := ls.GetReplicaForRangeID(ctx, 1) - if roachpb.IsRangeNotFoundError(err) { + if kvpb.IsRangeNotFoundError(err) { return false, nil } if err != nil { @@ -95,7 +96,7 @@ func (ls *Stores) GetStore(storeID roachpb.StoreID) (*Store, error) { if value, ok := ls.storeMap.Load(int64(storeID)); ok { return (*Store)(value), nil } - return nil, roachpb.NewStoreNotFoundError(storeID) + return nil, kvpb.NewStoreNotFoundError(storeID) } // AddStore adds the specified store to the store map. @@ -153,7 +154,7 @@ func (ls *Stores) VisitStores(visitor func(s *Store) error) error { // GetReplicaForRangeID returns the replica and store which contains the // specified range. If the replica is not found on any store then -// roachpb.RangeNotFoundError will be returned. +// kvpb.RangeNotFoundError will be returned. func (ls *Stores) GetReplicaForRangeID( ctx context.Context, rangeID roachpb.RangeID, ) (*Replica, *Store, error) { @@ -169,7 +170,7 @@ func (ls *Stores) GetReplicaForRangeID( log.Fatalf(ctx, "unexpected error: %s", err) } if replica == nil { - return nil, nil, roachpb.NewRangeNotFoundError(rangeID, 0) + return nil, nil, kvpb.NewRangeNotFoundError(rangeID, 0) } return replica, store, nil } @@ -177,8 +178,8 @@ func (ls *Stores) GetReplicaForRangeID( // Send implements the client.Sender interface. The store is looked up from the // store map using the ID specified in the request. func (ls *Stores) Send( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, *kvpb.Error) { br, writeBytes, pErr := ls.SendWithWriteBytes(ctx, ba) writeBytes.Release() return br, pErr @@ -187,20 +188,20 @@ func (ls *Stores) Send( // SendWithWriteBytes is the implementation of Send with an additional // *StoreWriteBytes return value. func (ls *Stores) SendWithWriteBytes( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, *kvadmission.StoreWriteBytes, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, *kvadmission.StoreWriteBytes, *kvpb.Error) { if err := ba.ValidateForEvaluation(); err != nil { - return nil, nil, roachpb.NewError(errors.Wrapf(err, "invalid batch (%s)", ba)) + return nil, nil, kvpb.NewError(errors.Wrapf(err, "invalid batch (%s)", ba)) } store, err := ls.GetStore(ba.Replica.StoreID) if err != nil { - return nil, nil, roachpb.NewError(err) + return nil, nil, kvpb.NewError(err) } br, writeBytes, pErr := store.SendWithWriteBytes(ctx, ba) if br != nil && br.Error != nil { - panic(roachpb.ErrorUnexpectedlySet(store, br)) + panic(kvpb.ErrorUnexpectedlySet(store, br)) } return br, writeBytes, pErr } @@ -209,8 +210,8 @@ func (ls *Stores) SendWithWriteBytes( // the provided stream and returns with an optional error when the rangefeed is // complete. func (ls *Stores) RangeFeed( - args *roachpb.RangeFeedRequest, stream roachpb.RangeFeedEventSink, -) *roachpb.Error { + args *kvpb.RangeFeedRequest, stream kvpb.RangeFeedEventSink, +) *kvpb.Error { ctx := stream.Context() if args.RangeID == 0 { log.Fatal(ctx, "rangefeed request missing range ID") @@ -220,7 +221,7 @@ func (ls *Stores) RangeFeed( store, err := ls.GetStore(args.Replica.StoreID) if err != nil { - return roachpb.NewError(err) + return kvpb.NewError(err) } return store.RangeFeed(args, stream) diff --git a/pkg/kv/kvserver/stores_test.go b/pkg/kv/kvserver/stores_test.go index d7c68d864adc..e49c65936ab9 100644 --- a/pkg/kv/kvserver/stores_test.go +++ b/pkg/kv/kvserver/stores_test.go @@ -16,6 +16,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/gossip" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/logstore" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" @@ -183,7 +184,7 @@ func TestStoresGetReplicaForRangeID(t *testing.T) { if replica2 != nil { t.Fatalf("expected replica to be nil; was %v", replica2) } - expectedError := roachpb.NewRangeNotFoundError(rangeID2, 0) + expectedError := kvpb.NewRangeNotFoundError(rangeID2, 0) if err2.Error() != expectedError.Error() { t.Fatalf("expected err to be %v; was %v", expectedError, err2) } diff --git a/pkg/kv/kvserver/testing_knobs.go b/pkg/kv/kvserver/testing_knobs.go index fb033789b283..9aaf78e6d2f3 100644 --- a/pkg/kv/kvserver/testing_knobs.go +++ b/pkg/kv/kvserver/testing_knobs.go @@ -15,6 +15,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" @@ -97,7 +98,7 @@ type StoreTestingKnobs struct { // TestingResponseErrorEvent is called when an error is returned applying // a command. - TestingResponseErrorEvent func(context.Context, *roachpb.BatchRequest, error) + TestingResponseErrorEvent func(context.Context, *kvpb.BatchRequest, error) // TestingResponseFilter is called after the replica processes a // command in order for unittests to modify the batch response, @@ -106,7 +107,7 @@ type StoreTestingKnobs struct { // SlowReplicationThresholdOverride is an interceptor that allows setting a // per-Batch SlowReplicationThreshold. - SlowReplicationThresholdOverride func(ba *roachpb.BatchRequest) time.Duration + SlowReplicationThresholdOverride func(ba *kvpb.BatchRequest) time.Duration // TestingRangefeedFilter is called before a replica processes a rangefeed // in order for unit tests to modify the request, error returned to the client @@ -130,7 +131,7 @@ type StoreTestingKnobs struct { // LeaseRequestEvent, if set, is called when replica.requestLeaseLocked() is // called to acquire a new lease. This can be used to assert that a request // triggers a lease acquisition. - LeaseRequestEvent func(ts hlc.Timestamp, storeID roachpb.StoreID, rangeID roachpb.RangeID) *roachpb.Error + LeaseRequestEvent func(ts hlc.Timestamp, storeID roachpb.StoreID, rangeID roachpb.RangeID) *kvpb.Error // PinnedLeases can be used to prevent all but one store from acquiring leases on a given range. PinnedLeases *PinnedLeasesKnob // LeaseTransferBlockedOnExtensionEvent, if set, is called when @@ -327,7 +328,7 @@ type StoreTestingKnobs struct { BeforeSnapshotSSTIngestion func(IncomingSnapshot, kvserverpb.SnapshotRequest_Type, []string) error // OnRelocatedOne intercepts the return values of s.relocateOne after they // have successfully been put into effect. - OnRelocatedOne func(_ []roachpb.ReplicationChange, leaseTarget *roachpb.ReplicationTarget) + OnRelocatedOne func(_ []kvpb.ReplicationChange, leaseTarget *roachpb.ReplicationTarget) // DontIgnoreFailureToTransferLease makes `AdminRelocateRange` return an error // to its client if it failed to transfer the lease to the first voting // replica in the set of relocation targets. @@ -523,7 +524,7 @@ func (p *PinnedLeasesKnob) PinLease(rangeID roachpb.RangeID, storeID roachpb.Sto // rejectLeaseIfPinnedElsewhere is called when r is trying to acquire a lease. // It returns a NotLeaseholderError if the lease is pinned on another store. // r.mu needs to be rlocked. -func (p *PinnedLeasesKnob) rejectLeaseIfPinnedElsewhere(r *Replica) *roachpb.Error { +func (p *PinnedLeasesKnob) rejectLeaseIfPinnedElsewhere(r *Replica) *kvpb.Error { if p == nil { return nil } @@ -537,10 +538,10 @@ func (p *PinnedLeasesKnob) rejectLeaseIfPinnedElsewhere(r *Replica) *roachpb.Err repDesc, err := r.getReplicaDescriptorRLocked() if err != nil { - return roachpb.NewError(err) + return kvpb.NewError(err) } pinned, _ := r.descRLocked().GetReplicaDescriptor(pinnedStore) - return roachpb.NewError(&roachpb.NotLeaseHolderError{ + return kvpb.NewError(&kvpb.NotLeaseHolderError{ Replica: repDesc, Lease: &roachpb.Lease{ Replica: pinned, diff --git a/pkg/kv/kvserver/ts_maintenance_queue_test.go b/pkg/kv/kvserver/ts_maintenance_queue_test.go index 91e1dba2500a..4f81bea88aae 100644 --- a/pkg/kv/kvserver/ts_maintenance_queue_test.go +++ b/pkg/kv/kvserver/ts_maintenance_queue_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server" @@ -129,7 +130,7 @@ func TestTimeSeriesMaintenanceQueue(t *testing.T) { for _, k := range splitKeys { repl := store.LookupReplica(roachpb.RKey(k)) args := adminSplitArgs(k) - if _, pErr := kv.SendWrappedWith(ctx, store, roachpb.Header{ + if _, pErr := kv.SendWrappedWith(ctx, store, kvpb.Header{ RangeID: repl.RangeID, }, args); pErr != nil { t.Fatal(pErr) diff --git a/pkg/kv/kvserver/txn_recovery_integration_test.go b/pkg/kv/kvserver/txn_recovery_integration_test.go index c4438ad63cb9..d9405442f827 100644 --- a/pkg/kv/kvserver/txn_recovery_integration_test.go +++ b/pkg/kv/kvserver/txn_recovery_integration_test.go @@ -17,6 +17,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -108,7 +109,7 @@ func TestTxnRecoveryFromStaging(t *testing.T) { keyAVal := []byte("value") pArgs := putArgs(keyA, keyAVal) pArgs.Sequence = 1 - h := roachpb.Header{Txn: txn} + h := kvpb.Header{Txn: txn} if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), h, &pArgs); pErr != nil { t.Fatal(pErr) } @@ -117,7 +118,7 @@ func TestTxnRecoveryFromStaging(t *testing.T) { // conflicting operation on keyB to prevent the transaction's write to // keyB from writing at its desired timestamp. This prevents an implicit // commit state. - conflictH := roachpb.Header{Timestamp: txn.WriteTimestamp.Next()} + conflictH := kvpb.Header{Timestamp: txn.WriteTimestamp.Next()} if !tc.implicitCommit { if !tc.writeTooOld { gArgs := getArgs(keyB) @@ -166,7 +167,7 @@ func TestTxnRecoveryFromStaging(t *testing.T) { t.Fatal(pErr) } if tc.implicitCommit { - if val := gReply.(*roachpb.GetResponse).Value; val == nil { + if val := gReply.(*kvpb.GetResponse).Value; val == nil { t.Fatalf("expected non-nil value when reading key %v", keyA) } else if valBytes, err := val.GetBytes(); err != nil { t.Fatal(err) @@ -174,7 +175,7 @@ func TestTxnRecoveryFromStaging(t *testing.T) { t.Fatalf("actual value %q did not match expected value %q", valBytes, keyAVal) } } else { - if val := gReply.(*roachpb.GetResponse).Value; val != nil { + if val := gReply.(*kvpb.GetResponse).Value; val != nil { t.Fatalf("expected nil value when reading key %v; found %v", keyA, val) } } @@ -185,7 +186,7 @@ func TestTxnRecoveryFromStaging(t *testing.T) { if pErr != nil { t.Fatal(pErr) } - status := qtReply.(*roachpb.QueryTxnResponse).QueriedTxn.Status + status := qtReply.(*kvpb.QueryTxnResponse).QueriedTxn.Status expStatus := roachpb.ABORTED if tc.implicitCommit { expStatus = roachpb.COMMITTED @@ -237,14 +238,14 @@ func TestTxnRecoveryFromStagingWithHighPriority(t *testing.T) { keyAVal := []byte("value") pArgs := putArgs(keyA, keyAVal) pArgs.Sequence = 1 - h := roachpb.Header{Txn: txn} + h := kvpb.Header{Txn: txn} _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), h, &pArgs) require.Nil(t, pErr, "error: %s", pErr) // The second write may or may not be bumped. pArgs = putArgs(keyB, []byte("value2")) pArgs.Sequence = 2 - h2 := roachpb.Header{Txn: txn.Clone()} + h2 := kvpb.Header{Txn: txn.Clone()} if newEpoch { h2.Txn.BumpEpoch() } @@ -267,7 +268,7 @@ func TestTxnRecoveryFromStagingWithHighPriority(t *testing.T) { require.Equal(t, roachpb.STAGING, etReply.Header().Txn.Status) // Issue a conflicting, high-priority operation. - var conflictArgs roachpb.Request + var conflictArgs kvpb.Request if pushAbort { pArgs = putArgs(keyB, []byte("value3")) conflictArgs = &pArgs @@ -276,7 +277,7 @@ func TestTxnRecoveryFromStagingWithHighPriority(t *testing.T) { conflictArgs = &gArgs } manual.Advance(100) - conflictH := roachpb.Header{ + conflictH := kvpb.Header{ UserPriority: roachpb.MaxUserPriority, Timestamp: store.Clock().Now(), } @@ -287,7 +288,7 @@ func TestTxnRecoveryFromStagingWithHighPriority(t *testing.T) { qtArgs := queryTxnArgs(txn.TxnMeta, false /* waitForUpdate */) qtReply, pErr := kv.SendWrapped(ctx, store.TestSender(), &qtArgs) require.Nil(t, pErr, "error: %s", pErr) - qtTxn := qtReply.(*roachpb.QueryTxnResponse).QueriedTxn + qtTxn := qtReply.(*kvpb.QueryTxnResponse).QueriedTxn if !newEpoch && !newTimestamp { // The transaction was implicitly committed at its initial epoch and @@ -341,7 +342,7 @@ func TestTxnClearRangeIntents(t *testing.T) { // Create a transaction that will get stuck performing a parallel commit. txn := newTransaction("txn", keyA, 1, store.Clock()) - txnHeader := roachpb.Header{Txn: txn} + txnHeader := kvpb.Header{Txn: txn} // Issue two writes, which will be considered in-flight at the time of the // transaction's EndTxn request. @@ -368,38 +369,38 @@ func TestTxnClearRangeIntents(t *testing.T) { // Make sure intents exists for keys A and B. queryIntent := queryIntentArgs(keyA, txn.TxnMeta, false) - reply, pErr = kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{}, &queryIntent) + reply, pErr = kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{}, &queryIntent) require.Nil(t, pErr, "error: %s", pErr) - require.True(t, reply.(*roachpb.QueryIntentResponse).FoundIntent, "intent missing for %q", keyA) + require.True(t, reply.(*kvpb.QueryIntentResponse).FoundIntent, "intent missing for %q", keyA) queryIntent = queryIntentArgs(keyB, txn.TxnMeta, false) - reply, pErr = kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{}, &queryIntent) + reply, pErr = kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{}, &queryIntent) require.Nil(t, pErr, "error: %s", pErr) - require.True(t, reply.(*roachpb.QueryIntentResponse).FoundIntent, "intent missing for %q", keyB) + require.True(t, reply.(*kvpb.QueryIntentResponse).FoundIntent, "intent missing for %q", keyB) // Call ClearRange covering key B and its intent. clearRange := clearRangeArgs(clearFrom, clearTo) - _, pErr = kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{}, &clearRange) + _, pErr = kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{}, &clearRange) require.Nil(t, pErr, "error: %s", pErr) // If separated intents are enabled, all should be well. // Reading A should succeed, but B should be gone. get := getArgs(keyA) - reply, pErr = kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{}, &get) + reply, pErr = kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{}, &get) require.Nil(t, pErr, "error: %s", pErr) - require.NotNil(t, reply.(*roachpb.GetResponse).Value, "expected value for A") - value, err := reply.(*roachpb.GetResponse).Value.GetBytes() + require.NotNil(t, reply.(*kvpb.GetResponse).Value, "expected value for A") + value, err := reply.(*kvpb.GetResponse).Value.GetBytes() require.NoError(t, err) require.Equal(t, value, valueA) get = getArgs(keyB) - reply, pErr = kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{}, &get) + reply, pErr = kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{}, &get) require.Nil(t, pErr, "error: %s", pErr) - require.Nil(t, reply.(*roachpb.GetResponse).Value, "unexpected value for B") + require.Nil(t, reply.(*kvpb.GetResponse).Value, "unexpected value for B") // Query the original transaction, which should now be committed. queryTxn := queryTxnArgs(txn.TxnMeta, false) - reply, pErr = kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{}, &queryTxn) + reply, pErr = kv.SendWrappedWith(ctx, store.TestSender(), kvpb.Header{}, &queryTxn) require.Nil(t, pErr, "error: %s", pErr) - require.Equal(t, roachpb.COMMITTED, reply.(*roachpb.QueryTxnResponse).QueriedTxn.Status) + require.Equal(t, roachpb.COMMITTED, reply.(*kvpb.QueryTxnResponse).QueriedTxn.Status) } diff --git a/pkg/kv/kvserver/txn_wait_queue_test.go b/pkg/kv/kvserver/txn_wait_queue_test.go index 19bd148e7b47..32e00bdb5000 100644 --- a/pkg/kv/kvserver/txn_wait_queue_test.go +++ b/pkg/kv/kvserver/txn_wait_queue_test.go @@ -21,6 +21,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/txnwait" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -51,8 +52,8 @@ func createTxnForPushQueue(ctx context.Context, tc *testContext) (*roachpb.Trans } type RespWithErr struct { - resp *roachpb.PushTxnResponse - pErr *roachpb.Error + resp *kvpb.PushTxnResponse + pErr *kvpb.Error } func checkAllGaugesZero(tc testContext) error { @@ -103,8 +104,8 @@ func TestTxnWaitQueueEnableDisable(t *testing.T) { assert.EqualValues(tc, 1, m.PusheeWaiting.Value()) pusher := newTransaction("pusher", roachpb.Key("a"), 1, tc.Clock()) - req := roachpb.PushTxnRequest{ - PushType: roachpb.PUSH_ABORT, + req := kvpb.PushTxnRequest{ + PushType: kvpb.PUSH_ABORT, PusherTxn: *pusher, PusheeTxn: txn.TxnMeta, } @@ -186,8 +187,8 @@ func TestTxnWaitQueueCancel(t *testing.T) { t.Fatal(err) } pusher := newTransaction("pusher", roachpb.Key("a"), 1, tc.Clock()) - req := roachpb.PushTxnRequest{ - PushType: roachpb.PUSH_ABORT, + req := kvpb.PushTxnRequest{ + PushType: kvpb.PUSH_ABORT, PusherTxn: *pusher, PusheeTxn: txn.TxnMeta, } @@ -252,8 +253,8 @@ func TestTxnWaitQueueUpdateTxn(t *testing.T) { } pusher1 := newTransaction("pusher1", roachpb.Key("a"), 1, tc.Clock()) pusher2 := newTransaction("pusher2", roachpb.Key("a"), 1, tc.Clock()) - req1 := roachpb.PushTxnRequest{ - PushType: roachpb.PUSH_ABORT, + req1 := kvpb.PushTxnRequest{ + PushType: kvpb.PUSH_ABORT, PusherTxn: *pusher1, PusheeTxn: txn.TxnMeta, } @@ -362,11 +363,11 @@ func TestTxnWaitQueueTxnSilentlyCompletes(t *testing.T) { t.Fatal(err) } pusher := newTransaction("pusher", roachpb.Key("a"), 1, tc.Clock()) - req := &roachpb.PushTxnRequest{ - RequestHeader: roachpb.RequestHeader{ + req := &kvpb.PushTxnRequest{ + RequestHeader: kvpb.RequestHeader{ Key: txn.Key, }, - PushType: roachpb.PUSH_ABORT, + PushType: kvpb.PUSH_ABORT, PusherTxn: *pusher, PusheeTxn: txn.TxnMeta, } @@ -442,8 +443,8 @@ func TestTxnWaitQueueUpdateNotPushedTxn(t *testing.T) { t.Fatal(err) } pusher := newTransaction("pusher", roachpb.Key("a"), 1, tc.Clock()) - req := roachpb.PushTxnRequest{ - PushType: roachpb.PUSH_ABORT, + req := kvpb.PushTxnRequest{ + PushType: kvpb.PUSH_ABORT, PusherTxn: *pusher, PusheeTxn: txn.TxnMeta, } @@ -499,8 +500,8 @@ func TestTxnWaitQueuePusheeExpires(t *testing.T) { tc := testContext{} tsc := TestStoreConfig(clock) tsc.TestingKnobs.EvalKnobs.TestingEvalFilter = - func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { - if qtReq, ok := filterArgs.Req.(*roachpb.QueryTxnRequest); ok && bytes.Equal(qtReq.Txn.Key, txn.Key) { + func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { + if qtReq, ok := filterArgs.Req.(*kvpb.QueryTxnRequest); ok && bytes.Equal(qtReq.Txn.Key, txn.Key) { atomic.AddInt32(&queryTxnCount, 1) } return nil @@ -512,8 +513,8 @@ func TestTxnWaitQueuePusheeExpires(t *testing.T) { pusher1 := newTransaction("pusher1", roachpb.Key("a"), 1, tc.Clock()) pusher2 := newTransaction("pusher2", roachpb.Key("a"), 1, tc.Clock()) - req1 := roachpb.PushTxnRequest{ - PushType: roachpb.PUSH_ABORT, + req1 := kvpb.PushTxnRequest{ + PushType: kvpb.PUSH_ABORT, PusherTxn: *pusher1, PusheeTxn: txn.TxnMeta, } @@ -609,8 +610,8 @@ func TestTxnWaitQueuePusherUpdate(t *testing.T) { } pusher.Epoch = pushEpoch - req := roachpb.PushTxnRequest{ - PushType: roachpb.PUSH_ABORT, + req := kvpb.PushTxnRequest{ + PushType: kvpb.PUSH_ABORT, PusherTxn: *pusher, PusheeTxn: txn.TxnMeta, } @@ -676,9 +677,9 @@ func TestTxnWaitQueuePusherUpdate(t *testing.T) { } type ReqWithRespAndErr struct { - req *roachpb.PushTxnRequest - resp *roachpb.PushTxnResponse - pErr *roachpb.Error + req *kvpb.PushTxnRequest + resp *kvpb.PushTxnResponse + pErr *kvpb.Error } // TestTxnWaitQueueDependencyCycle verifies that if txn A pushes txn B @@ -706,27 +707,27 @@ func TestTxnWaitQueueDependencyCycle(t *testing.T) { t.Fatal(err) } - reqA := &roachpb.PushTxnRequest{ - RequestHeader: roachpb.RequestHeader{ + reqA := &kvpb.PushTxnRequest{ + RequestHeader: kvpb.RequestHeader{ Key: txnB.Key, }, - PushType: roachpb.PUSH_ABORT, + PushType: kvpb.PUSH_ABORT, PusherTxn: *txnA, PusheeTxn: txnB.TxnMeta, } - reqB := &roachpb.PushTxnRequest{ - RequestHeader: roachpb.RequestHeader{ + reqB := &kvpb.PushTxnRequest{ + RequestHeader: kvpb.RequestHeader{ Key: txnC.Key, }, - PushType: roachpb.PUSH_ABORT, + PushType: kvpb.PUSH_ABORT, PusherTxn: *txnB, PusheeTxn: txnC.TxnMeta, } - reqC := &roachpb.PushTxnRequest{ - RequestHeader: roachpb.RequestHeader{ + reqC := &kvpb.PushTxnRequest{ + RequestHeader: kvpb.RequestHeader{ Key: txnA.Key, }, - PushType: roachpb.PUSH_ABORT, + PushType: kvpb.PUSH_ABORT, PusherTxn: *txnC, PusheeTxn: txnA.TxnMeta, } @@ -742,10 +743,10 @@ func TestTxnWaitQueueDependencyCycle(t *testing.T) { m := tc.store.txnWaitMetrics assert.EqualValues(tc, 0, m.DeadlocksTotal.Count()) - reqs := []*roachpb.PushTxnRequest{reqA, reqB, reqC} + reqs := []*kvpb.PushTxnRequest{reqA, reqB, reqC} retCh := make(chan ReqWithRespAndErr, len(reqs)) for _, req := range reqs { - go func(req *roachpb.PushTxnRequest) { + go func(req *kvpb.PushTxnRequest) { resp, pErr := q.MaybeWaitForPush(ctx, req) retCh <- ReqWithRespAndErr{req, resp, pErr} }(req) @@ -807,19 +808,19 @@ func TestTxnWaitQueueDependencyCycleWithPriorityInversion(t *testing.T) { t.Fatal(err) } - reqA := &roachpb.PushTxnRequest{ - RequestHeader: roachpb.RequestHeader{ + reqA := &kvpb.PushTxnRequest{ + RequestHeader: kvpb.RequestHeader{ Key: txnB.Key, }, - PushType: roachpb.PUSH_ABORT, + PushType: kvpb.PUSH_ABORT, PusherTxn: *txnA, PusheeTxn: txnB.TxnMeta, } - reqB := &roachpb.PushTxnRequest{ - RequestHeader: roachpb.RequestHeader{ + reqB := &kvpb.PushTxnRequest{ + RequestHeader: kvpb.RequestHeader{ Key: txnA.Key, }, - PushType: roachpb.PUSH_ABORT, + PushType: kvpb.PUSH_ABORT, PusherTxn: *txnB, PusheeTxn: updatedTxnA.TxnMeta, } @@ -833,10 +834,10 @@ func TestTxnWaitQueueDependencyCycleWithPriorityInversion(t *testing.T) { m := tc.store.txnWaitMetrics assert.EqualValues(tc, 0, m.DeadlocksTotal.Count()) - reqs := []*roachpb.PushTxnRequest{reqA, reqB} + reqs := []*kvpb.PushTxnRequest{reqA, reqB} retCh := make(chan ReqWithRespAndErr, len(reqs)) for _, req := range reqs { - go func(req *roachpb.PushTxnRequest) { + go func(req *kvpb.PushTxnRequest) { resp, pErr := q.MaybeWaitForPush(ctx, req) retCh <- ReqWithRespAndErr{req, resp, pErr} }(req) diff --git a/pkg/kv/kvserver/txnrecovery/BUILD.bazel b/pkg/kv/kvserver/txnrecovery/BUILD.bazel index 8e15807a5871..81c0624ccca6 100644 --- a/pkg/kv/kvserver/txnrecovery/BUILD.bazel +++ b/pkg/kv/kvserver/txnrecovery/BUILD.bazel @@ -11,6 +11,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/util/hlc", "//pkg/util/log", @@ -29,6 +30,7 @@ go_test( embed = [":txnrecovery"], deps = [ "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/testutils", "//pkg/util/hlc", diff --git a/pkg/kv/kvserver/txnrecovery/manager.go b/pkg/kv/kvserver/txnrecovery/manager.go index ba22fe197380..3991c3d6be21 100644 --- a/pkg/kv/kvserver/txnrecovery/manager.go +++ b/pkg/kv/kvserver/txnrecovery/manager.go @@ -15,6 +15,7 @@ import ( "sort" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -36,7 +37,7 @@ type Manager interface { // The method may also return a transaction in any other state if it is // discovered to still be live and undergoing state transitions. ResolveIndeterminateCommit( - context.Context, *roachpb.IndeterminateCommitError, + context.Context, *kvpb.IndeterminateCommitError, ) (*roachpb.Transaction, error) // Metrics returns the Manager's metrics struct. @@ -83,7 +84,7 @@ func NewManager(ac log.AmbientContext, clock *hlc.Clock, db *kv.DB, stopper *sto // ResolveIndeterminateCommit implements the Manager interface. func (m *manager) ResolveIndeterminateCommit( - ctx context.Context, ice *roachpb.IndeterminateCommitError, + ctx context.Context, ice *kvpb.IndeterminateCommitError, ) (*roachpb.Transaction, error) { txn := &ice.StagingTxn if txn.Status != roachpb.STAGING { @@ -163,8 +164,8 @@ func (m *manager) resolveIndeterminateCommitForTxnProbe( ) (preventedIntent bool, changedTxn *roachpb.Transaction, err error) { // Create a QueryTxnRequest that we will periodically send to the // transaction's record during recovery processing. - queryTxnReq := roachpb.QueryTxnRequest{ - RequestHeader: roachpb.RequestHeader{ + queryTxnReq := kvpb.QueryTxnRequest{ + RequestHeader: kvpb.RequestHeader{ Key: txn.Key, }, Txn: txn.TxnMeta, @@ -196,12 +197,12 @@ func (m *manager) resolveIndeterminateCommitForTxnProbe( // record (e.g. if the record has been abandoned). However, it can fail if // the transaction has already refreshed at a higher timestamp in the // current epoch or restarted at a higher epoch. - queryIntentReqs := make([]roachpb.QueryIntentRequest, 0, len(txn.InFlightWrites)) + queryIntentReqs := make([]kvpb.QueryIntentRequest, 0, len(txn.InFlightWrites)) for _, w := range txn.InFlightWrites { meta := txn.TxnMeta meta.Sequence = w.Sequence - queryIntentReqs = append(queryIntentReqs, roachpb.QueryIntentRequest{ - RequestHeader: roachpb.RequestHeader{ + queryIntentReqs = append(queryIntentReqs, kvpb.QueryIntentRequest{ + RequestHeader: kvpb.RequestHeader{ Key: w.Key, }, Txn: meta, @@ -244,7 +245,7 @@ func (m *manager) resolveIndeterminateCommitForTxnProbe( // state of the transaction record has changed since we began // the recovery process. resps := b.RawResponse().Responses - queryTxnResp := resps[0].GetInner().(*roachpb.QueryTxnResponse) + queryTxnResp := resps[0].GetInner().(*kvpb.QueryTxnResponse) queriedTxn := &queryTxnResp.QueriedTxn if queriedTxn.Status.IsFinalized() || txn.Epoch < queriedTxn.Epoch || @@ -258,7 +259,7 @@ func (m *manager) resolveIndeterminateCommitForTxnProbe( // Next, look through the QueryIntentResponses to check whether // any of the in-flight writes failed. for _, ru := range resps[1:] { - queryIntentResp := ru.GetInner().(*roachpb.QueryIntentResponse) + queryIntentResp := ru.GetInner().(*kvpb.QueryIntentResponse) if !queryIntentResp.FoundIntent { return true /* preventedIntent */, nil, nil } @@ -282,8 +283,8 @@ func (m *manager) resolveIndeterminateCommitForTxnRecover( ) (*roachpb.Transaction, error) { var b kv.Batch b.Header.Timestamp = m.batchTimestamp(txn) - b.AddRawRequest(&roachpb.RecoverTxnRequest{ - RequestHeader: roachpb.RequestHeader{ + b.AddRawRequest(&kvpb.RecoverTxnRequest{ + RequestHeader: kvpb.RequestHeader{ Key: txn.Key, }, Txn: txn.TxnMeta, @@ -295,7 +296,7 @@ func (m *manager) resolveIndeterminateCommitForTxnRecover( } resps := b.RawResponse().Responses - recTxnResp := resps[0].GetInner().(*roachpb.RecoverTxnResponse) + recTxnResp := resps[0].GetInner().(*kvpb.RecoverTxnResponse) return &recTxnResp.RecoveredTxn, nil } diff --git a/pkg/kv/kvserver/txnrecovery/manager_test.go b/pkg/kv/kvserver/txnrecovery/manager_test.go index 1876e1dfbac5..5670b1ee864a 100644 --- a/pkg/kv/kvserver/txnrecovery/manager_test.go +++ b/pkg/kv/kvserver/txnrecovery/manager_test.go @@ -15,6 +15,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -29,8 +30,8 @@ func makeManager(s *kv.Sender) (Manager, *hlc.Clock, *stop.Stopper) { clock := hlc.NewClockForTesting(nil) stopper := stop.NewStopper() db := kv.NewDB(ac, kv.NonTransactionalFactoryFunc(func( - ctx context.Context, ba *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, *kvpb.Error) { return (*s).Send(ctx, ba) }), clock, stopper) return NewManager(ac, clock, db, stopper), clock, stopper @@ -90,41 +91,41 @@ func TestResolveIndeterminateCommit(t *testing.T) { } mockSender = kv.SenderFunc(func( - _ context.Context, ba *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, *roachpb.Error) { + _ context.Context, ba *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, *kvpb.Error) { // Probing Phase. assertMetrics(t, m, metricVals{attemptsPending: 1, attempts: 1}) assert.Equal(t, 3, len(ba.Requests)) - assert.IsType(t, &roachpb.QueryTxnRequest{}, ba.Requests[0].GetInner()) - assert.IsType(t, &roachpb.QueryIntentRequest{}, ba.Requests[1].GetInner()) - assert.IsType(t, &roachpb.QueryIntentRequest{}, ba.Requests[2].GetInner()) + assert.IsType(t, &kvpb.QueryTxnRequest{}, ba.Requests[0].GetInner()) + assert.IsType(t, &kvpb.QueryIntentRequest{}, ba.Requests[1].GetInner()) + assert.IsType(t, &kvpb.QueryIntentRequest{}, ba.Requests[2].GetInner()) assert.Equal(t, roachpb.Key(txn.Key), ba.Requests[0].GetInner().Header().Key) assert.Equal(t, roachpb.Key("a"), ba.Requests[1].GetInner().Header().Key) assert.Equal(t, roachpb.Key("b"), ba.Requests[2].GetInner().Header().Key) br := ba.CreateReply() - br.Responses[0].GetInner().(*roachpb.QueryTxnResponse).QueriedTxn = txn - br.Responses[1].GetInner().(*roachpb.QueryIntentResponse).FoundIntent = true - br.Responses[2].GetInner().(*roachpb.QueryIntentResponse).FoundIntent = !prevent + br.Responses[0].GetInner().(*kvpb.QueryTxnResponse).QueriedTxn = txn + br.Responses[1].GetInner().(*kvpb.QueryIntentResponse).FoundIntent = true + br.Responses[2].GetInner().(*kvpb.QueryIntentResponse).FoundIntent = !prevent mockSender = kv.SenderFunc(func( - _ context.Context, ba *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, *roachpb.Error) { + _ context.Context, ba *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, *kvpb.Error) { // Recovery Phase. assertMetrics(t, m, metricVals{attemptsPending: 1, attempts: 1}) assert.Equal(t, 1, len(ba.Requests)) - assert.IsType(t, &roachpb.RecoverTxnRequest{}, ba.Requests[0].GetInner()) + assert.IsType(t, &kvpb.RecoverTxnRequest{}, ba.Requests[0].GetInner()) - recTxnReq := ba.Requests[0].GetInner().(*roachpb.RecoverTxnRequest) + recTxnReq := ba.Requests[0].GetInner().(*kvpb.RecoverTxnRequest) assert.Equal(t, roachpb.Key(txn.Key), recTxnReq.Key) assert.Equal(t, txn.TxnMeta, recTxnReq.Txn) assert.Equal(t, !prevent, recTxnReq.ImplicitlyCommitted) br2 := ba.CreateReply() - recTxnResp := br2.Responses[0].GetInner().(*roachpb.RecoverTxnResponse) + recTxnResp := br2.Responses[0].GetInner().(*kvpb.RecoverTxnResponse) recTxnResp.RecoveredTxn = txn if !prevent { recTxnResp.RecoveredTxn.Status = roachpb.COMMITTED @@ -137,7 +138,7 @@ func TestResolveIndeterminateCommit(t *testing.T) { }) assertMetrics(t, m, metricVals{}) - iceErr := roachpb.NewIndeterminateCommitError(txn) + iceErr := kvpb.NewIndeterminateCommitError(txn) resTxn, err := m.ResolveIndeterminateCommit(context.Background(), iceErr) assert.NotNil(t, resTxn) assert.Nil(t, err) @@ -267,15 +268,15 @@ func TestResolveIndeterminateCommitTxnChanges(t *testing.T) { for _, c := range testCases { t.Run(c.name, func(t *testing.T) { mockSender = kv.SenderFunc(func( - _ context.Context, ba *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, *roachpb.Error) { + _ context.Context, ba *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, *kvpb.Error) { // Probing Phase. assertMetrics(t, m, expMetrics.merge(metricVals{attemptsPending: 1, attempts: 1})) assert.Equal(t, 3, len(ba.Requests)) - assert.IsType(t, &roachpb.QueryTxnRequest{}, ba.Requests[0].GetInner()) - assert.IsType(t, &roachpb.QueryIntentRequest{}, ba.Requests[1].GetInner()) - assert.IsType(t, &roachpb.QueryIntentRequest{}, ba.Requests[2].GetInner()) + assert.IsType(t, &kvpb.QueryTxnRequest{}, ba.Requests[0].GetInner()) + assert.IsType(t, &kvpb.QueryIntentRequest{}, ba.Requests[1].GetInner()) + assert.IsType(t, &kvpb.QueryIntentRequest{}, ba.Requests[2].GetInner()) assert.Equal(t, roachpb.Key(txn.Key), ba.Requests[0].GetInner().Header().Key) assert.Equal(t, roachpb.Key("a"), ba.Requests[1].GetInner().Header().Key) @@ -283,36 +284,36 @@ func TestResolveIndeterminateCommitTxnChanges(t *testing.T) { br := ba.CreateReply() if c.duringProbing { - br.Responses[0].GetInner().(*roachpb.QueryTxnResponse).QueriedTxn = c.changedTxn + br.Responses[0].GetInner().(*kvpb.QueryTxnResponse).QueriedTxn = c.changedTxn } else { - br.Responses[0].GetInner().(*roachpb.QueryTxnResponse).QueriedTxn = txn + br.Responses[0].GetInner().(*kvpb.QueryTxnResponse).QueriedTxn = txn } - br.Responses[1].GetInner().(*roachpb.QueryIntentResponse).FoundIntent = true - br.Responses[2].GetInner().(*roachpb.QueryIntentResponse).FoundIntent = false + br.Responses[1].GetInner().(*kvpb.QueryIntentResponse).FoundIntent = true + br.Responses[2].GetInner().(*kvpb.QueryIntentResponse).FoundIntent = false mockSender = kv.SenderFunc(func( - _ context.Context, ba *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, *roachpb.Error) { + _ context.Context, ba *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, *kvpb.Error) { // Recovery Phase. assert.False(t, c.duringProbing, "the recovery phase should not be run") assertMetrics(t, m, expMetrics.merge(metricVals{attemptsPending: 1, attempts: 1})) assert.Equal(t, 1, len(ba.Requests)) - assert.IsType(t, &roachpb.RecoverTxnRequest{}, ba.Requests[0].GetInner()) + assert.IsType(t, &kvpb.RecoverTxnRequest{}, ba.Requests[0].GetInner()) - recTxnReq := ba.Requests[0].GetInner().(*roachpb.RecoverTxnRequest) + recTxnReq := ba.Requests[0].GetInner().(*kvpb.RecoverTxnRequest) assert.Equal(t, roachpb.Key(txn.Key), recTxnReq.Key) assert.Equal(t, txn.TxnMeta, recTxnReq.Txn) assert.Equal(t, false, recTxnReq.ImplicitlyCommitted) br2 := ba.CreateReply() - br2.Responses[0].GetInner().(*roachpb.RecoverTxnResponse).RecoveredTxn = c.changedTxn + br2.Responses[0].GetInner().(*kvpb.RecoverTxnResponse).RecoveredTxn = c.changedTxn return br2, nil }) return br, nil }) - iceErr := roachpb.NewIndeterminateCommitError(txn) + iceErr := kvpb.NewIndeterminateCommitError(txn) resTxn, err := m.ResolveIndeterminateCommit(context.Background(), iceErr) assert.NotNil(t, resTxn) assert.Equal(t, c.changedTxn, *resTxn) @@ -341,25 +342,25 @@ func TestResolveIndeterminateCommitTxnWithoutInFlightWrites(t *testing.T) { txn := makeStagingTransaction(clock) mockSender = kv.SenderFunc(func( - _ context.Context, ba *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, *roachpb.Error) { + _ context.Context, ba *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, *kvpb.Error) { // Recovery Phase. Probing phase skipped. assert.Equal(t, 1, len(ba.Requests)) - assert.IsType(t, &roachpb.RecoverTxnRequest{}, ba.Requests[0].GetInner()) + assert.IsType(t, &kvpb.RecoverTxnRequest{}, ba.Requests[0].GetInner()) - recTxnReq := ba.Requests[0].GetInner().(*roachpb.RecoverTxnRequest) + recTxnReq := ba.Requests[0].GetInner().(*kvpb.RecoverTxnRequest) assert.Equal(t, roachpb.Key(txn.Key), recTxnReq.Key) assert.Equal(t, txn.TxnMeta, recTxnReq.Txn) assert.Equal(t, true, recTxnReq.ImplicitlyCommitted) br := ba.CreateReply() - recTxnResp := br.Responses[0].GetInner().(*roachpb.RecoverTxnResponse) + recTxnResp := br.Responses[0].GetInner().(*kvpb.RecoverTxnResponse) recTxnResp.RecoveredTxn = txn recTxnResp.RecoveredTxn.Status = roachpb.COMMITTED return br, nil }) - iceErr := roachpb.NewIndeterminateCommitError(txn) + iceErr := kvpb.NewIndeterminateCommitError(txn) resTxn, err := m.ResolveIndeterminateCommit(context.Background(), iceErr) assert.NotNil(t, resTxn) assert.Equal(t, roachpb.COMMITTED, resTxn.Status) diff --git a/pkg/kv/kvserver/txnwait/BUILD.bazel b/pkg/kv/kvserver/txnwait/BUILD.bazel index 1f8ede16d7d9..759db05b7a4b 100644 --- a/pkg/kv/kvserver/txnwait/BUILD.bazel +++ b/pkg/kv/kvserver/txnwait/BUILD.bazel @@ -12,6 +12,7 @@ go_library( deps = [ "//pkg/base", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/kvserverbase", "//pkg/roachpb", "//pkg/storage/enginepb", @@ -35,6 +36,7 @@ go_test( embed = [":txnwait"], deps = [ "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/storage/enginepb", "//pkg/util/hlc", diff --git a/pkg/kv/kvserver/txnwait/queue.go b/pkg/kv/kvserver/txnwait/queue.go index b09558d3604c..0d2267073300 100644 --- a/pkg/kv/kvserver/txnwait/queue.go +++ b/pkg/kv/kvserver/txnwait/queue.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" @@ -68,11 +69,11 @@ func TestingOverrideTxnLivenessThreshold(t time.Duration) func() { // proceed without queueing. This is true for pushes which are neither // ABORT nor TIMESTAMP, but also for ABORT and TIMESTAMP pushes where // the pushee has min priority or pusher has max priority. -func ShouldPushImmediately(req *roachpb.PushTxnRequest) bool { +func ShouldPushImmediately(req *kvpb.PushTxnRequest) bool { if req.Force { return true } - if !(req.PushType == roachpb.PUSH_ABORT || req.PushType == roachpb.PUSH_TIMESTAMP) { + if !(req.PushType == kvpb.PUSH_ABORT || req.PushType == kvpb.PUSH_TIMESTAMP) { return true } if CanPushWithPriority(req.PusherTxn.Priority, req.PusheeTxn.Priority) { @@ -101,9 +102,9 @@ func CanPushWithPriority(pusher, pushee enginepb.TxnPriority) bool { // isPushed returns whether the PushTxn request has already been // fulfilled by the current transaction state. This may be true // for transactions with pushed timestamps. -func isPushed(req *roachpb.PushTxnRequest, txn *roachpb.Transaction) bool { +func isPushed(req *kvpb.PushTxnRequest, txn *roachpb.Transaction) bool { return txn.Status.IsFinalized() || - (req.PushType == roachpb.PUSH_TIMESTAMP && req.PushTo.LessEq(txn.WriteTimestamp)) + (req.PushType == kvpb.PUSH_TIMESTAMP && req.PushTo.LessEq(txn.WriteTimestamp)) } // TxnExpiration computes the timestamp after which the transaction will be @@ -120,8 +121,8 @@ func IsExpired(now hlc.Timestamp, txn *roachpb.Transaction) bool { // createPushTxnResponse returns a PushTxnResponse struct with a // copy of the supplied transaction. It is necessary to fully copy // each field in the transaction to avoid race conditions. -func createPushTxnResponse(txn *roachpb.Transaction) *roachpb.PushTxnResponse { - return &roachpb.PushTxnResponse{PusheeTxn: *txn} +func createPushTxnResponse(txn *roachpb.Transaction) *kvpb.PushTxnResponse { + return &kvpb.PushTxnResponse{PusheeTxn: *txn} } // A waitingPush represents a PushTxn command that is waiting on the @@ -129,7 +130,7 @@ func createPushTxnResponse(txn *roachpb.Transaction) *roachpb.PushTxnResponse { // set of all txns which are waiting on this txn in order to detect // dependency cycles. type waitingPush struct { - req *roachpb.PushTxnRequest + req *kvpb.PushTxnRequest // pending channel receives updated, pushed txn or nil if queue is cleared. pending chan *roachpb.Transaction mu struct { @@ -207,7 +208,7 @@ type Config struct { // TestingKnobs represents testing knobs for a Queue. type TestingKnobs struct { // OnTxnWaitEnqueue is called when a would-be pusher joins a wait queue. - OnPusherBlocked func(ctx context.Context, push *roachpb.PushTxnRequest) + OnPusherBlocked func(ctx context.Context, push *kvpb.PushTxnRequest) // OnTxnUpdate is called by Queue.UpdateTxn. OnTxnUpdate func(ctx context.Context, txn *roachpb.Transaction) } @@ -416,7 +417,7 @@ func (q *Queue) GetDependents(txnID uuid.UUID) []uuid.UUID { // the QueryTxnRequest has had its status or priority updated // or whether the known set of dependent transactions has // changed. -func (q *Queue) isTxnUpdated(pending *pendingTxn, req *roachpb.QueryTxnRequest) bool { +func (q *Queue) isTxnUpdated(pending *pendingTxn, req *kvpb.QueryTxnRequest) bool { // First check whether txn status or priority has changed. txn := pending.getTxn() if txn.Status.IsFinalized() || txn.Priority > req.Txn.Priority { @@ -452,8 +453,8 @@ func (q *Queue) releaseWaitingQueriesLocked(ctx context.Context, txnID uuid.UUID // If the transaction is successfully pushed while this method is waiting, // the first return value is a non-nil PushTxnResponse object. func (q *Queue) MaybeWaitForPush( - ctx context.Context, req *roachpb.PushTxnRequest, -) (*roachpb.PushTxnResponse, *roachpb.Error) { + ctx context.Context, req *kvpb.PushTxnRequest, +) (*kvpb.PushTxnResponse, *kvpb.Error) { if ShouldPushImmediately(req) { return nil, nil } @@ -518,8 +519,8 @@ func (q *Queue) MaybeWaitForPush( req.PusheeTxn.ID.Short(), waitingPushesCount, ) - var res *roachpb.PushTxnResponse - var err *roachpb.Error + var res *kvpb.PushTxnResponse + var err *kvpb.Error labels := pprof.Labels("pushee", req.PusheeTxn.ID.String(), "pusher", pusherStr) pprof.Do(ctx, labels, func(ctx context.Context) { res, err = q.waitForPush(ctx, req, push, pending) @@ -528,13 +529,13 @@ func (q *Queue) MaybeWaitForPush( } func (q *Queue) waitForPush( - ctx context.Context, req *roachpb.PushTxnRequest, push *waitingPush, pending *pendingTxn, -) (*roachpb.PushTxnResponse, *roachpb.Error) { + ctx context.Context, req *kvpb.PushTxnRequest, push *waitingPush, pending *pendingTxn, +) (*kvpb.PushTxnResponse, *kvpb.Error) { // Wait for any updates to the pusher txn to be notified when // status, priority, or dependents (for deadlock detection) have // changed. var queryPusherCh <-chan *roachpb.Transaction // accepts updates to the pusher txn - var queryPusherErrCh <-chan *roachpb.Error // accepts errors querying the pusher txn + var queryPusherErrCh <-chan *kvpb.Error // accepts errors querying the pusher txn var readyCh chan struct{} // signaled when pusher txn should be queried // Query the pusher if it's a valid read-write transaction. @@ -596,7 +597,7 @@ func (q *Queue) waitForPush( case <-ctx.Done(): // Caller has given up. log.VEvent(ctx, 2, "pusher giving up due to context cancellation") - return nil, roachpb.NewError(ctx.Err()) + return nil, kvpb.NewError(ctx.Err()) case <-q.cfg.Stopper.ShouldQuiesce(): // Let the push out so that they can be sent looking elsewhere. return nil, nil @@ -675,14 +676,14 @@ func (q *Queue) waitForPush( switch updatedPusher.Status { case roachpb.COMMITTED: log.VEventf(ctx, 1, "pusher committed: %v", updatedPusher) - return nil, roachpb.NewErrorWithTxn(roachpb.NewTransactionStatusError( - roachpb.TransactionStatusError_REASON_TXN_COMMITTED, + return nil, kvpb.NewErrorWithTxn(kvpb.NewTransactionStatusError( + kvpb.TransactionStatusError_REASON_TXN_COMMITTED, "already committed"), updatedPusher) case roachpb.ABORTED: log.VEventf(ctx, 1, "pusher aborted: %v", updatedPusher) - return nil, roachpb.NewErrorWithTxn( - roachpb.NewTransactionAbortedError(roachpb.ABORT_REASON_PUSHER_ABORTED), updatedPusher) + return nil, kvpb.NewErrorWithTxn( + kvpb.NewTransactionAbortedError(kvpb.ABORT_REASON_PUSHER_ABORTED), updatedPusher) } log.VEventf(ctx, 2, "pusher was updated: %v", updatedPusher) if updatedPusher.Priority > pusherPriority { @@ -745,9 +746,7 @@ func (q *Queue) waitForPush( // request hasn't specified WaitForUpdate, return immediately. If // there is a queue, enqueue this request as a waiter and enter a // select loop waiting for any updates to the target transaction. -func (q *Queue) MaybeWaitForQuery( - ctx context.Context, req *roachpb.QueryTxnRequest, -) *roachpb.Error { +func (q *Queue) MaybeWaitForQuery(ctx context.Context, req *kvpb.QueryTxnRequest) *kvpb.Error { if !req.WaitForUpdate { return nil } @@ -815,7 +814,7 @@ func (q *Queue) MaybeWaitForQuery( select { case <-ctx.Done(): // Caller has given up. - return roachpb.NewError(ctx.Err()) + return kvpb.NewError(ctx.Err()) case <-maxWaitCh: return nil case <-query.pending: @@ -839,9 +838,9 @@ func (q *Queue) MaybeWaitForQuery( // invocation of QueryTxn in order to avoid busy querying. func (q *Queue) startQueryPusherTxn( ctx context.Context, push *waitingPush, readyCh <-chan struct{}, -) (<-chan *roachpb.Transaction, <-chan *roachpb.Error) { +) (<-chan *roachpb.Transaction, <-chan *kvpb.Error) { ch := make(chan *roachpb.Transaction, 1) - errCh := make(chan *roachpb.Error, 1) + errCh := make(chan *kvpb.Error, 1) push.mu.Lock() var waitingTxns []uuid.UUID if push.mu.dependents != nil { @@ -859,7 +858,7 @@ func (q *Queue) startQueryPusherTxn( // We use a backoff/retry here in case the pusher transaction // doesn't yet exist. for r := retry.StartWithCtx(ctx, base.DefaultRetryOptions()); r.Next(); { - var pErr *roachpb.Error + var pErr *kvpb.Error var updatedPusher *roachpb.Transaction updatedPusher, waitingTxns, pErr = q.queryTxnStatus( ctx, pusher.TxnMeta, true, waitingTxns, @@ -899,16 +898,16 @@ func (q *Queue) startQueryPusherTxn( // push waiter requires another query of the pusher txn. select { case <-ctx.Done(): - errCh <- roachpb.NewError(ctx.Err()) + errCh <- kvpb.NewError(ctx.Err()) return case <-readyCh: } // Reset the retry to query again immediately. r.Reset() } - errCh <- roachpb.NewError(ctx.Err()) + errCh <- kvpb.NewError(ctx.Err()) }); err != nil { - errCh <- roachpb.NewError(err) + errCh <- kvpb.NewError(err) } return ch, errCh } @@ -924,11 +923,11 @@ func (q *Queue) startQueryPusherTxn( // the list of transactions which are waiting on the updated txn. func (q *Queue) queryTxnStatus( ctx context.Context, txnMeta enginepb.TxnMeta, wait bool, dependents []uuid.UUID, -) (*roachpb.Transaction, []uuid.UUID, *roachpb.Error) { +) (*roachpb.Transaction, []uuid.UUID, *kvpb.Error) { b := &kv.Batch{} b.Header.Timestamp = q.cfg.Clock.Now() - b.AddRawRequest(&roachpb.QueryTxnRequest{ - RequestHeader: roachpb.RequestHeader{ + b.AddRawRequest(&kvpb.QueryTxnRequest{ + RequestHeader: kvpb.RequestHeader{ Key: txnMeta.Key, }, Txn: txnMeta, @@ -957,10 +956,10 @@ func (q *Queue) queryTxnStatus( // // so something is sketchy here, but it should all resolve nicely when we // don't use store.db for these internal requests any more. - return nil, nil, roachpb.NewError(err) + return nil, nil, kvpb.NewError(err) } br := b.RawResponse() - resp := br.Responses[0].GetInner().(*roachpb.QueryTxnResponse) + resp := br.Responses[0].GetInner().(*kvpb.QueryTxnResponse) // ID can be nil if no HeartbeatTxn has been sent yet and we're talking to a // 2.1 node. // TODO(nvanbenschoten): Remove this in 2.3. @@ -975,12 +974,12 @@ func (q *Queue) queryTxnStatus( // the pushee. This mechanism can be used to break deadlocks between conflicting // transactions. func (q *Queue) forcePushAbort( - ctx context.Context, req *roachpb.PushTxnRequest, -) (*roachpb.PushTxnResponse, *roachpb.Error) { + ctx context.Context, req *kvpb.PushTxnRequest, +) (*kvpb.PushTxnResponse, *kvpb.Error) { log.VEventf(ctx, 1, "force pushing %v to break deadlock", req.PusheeTxn.ID) forcePush := *req forcePush.Force = true - forcePush.PushType = roachpb.PUSH_ABORT + forcePush.PushType = kvpb.PUSH_ABORT b := &kv.Batch{} b.Header.Timestamp = q.cfg.Clock.Now() b.Header.Timestamp.Forward(req.PushTo) diff --git a/pkg/kv/kvserver/txnwait/queue_test.go b/pkg/kv/kvserver/txnwait/queue_test.go index 1c80eb8c4e81..1b02dd713429 100644 --- a/pkg/kv/kvserver/txnwait/queue_test.go +++ b/pkg/kv/kvserver/txnwait/queue_test.go @@ -19,6 +19,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -38,112 +39,112 @@ func TestShouldPushImmediately(t *testing.T) { mid2 := enginepb.TxnPriority(2) testCases := []struct { force bool - typ roachpb.PushTxnType + typ kvpb.PushTxnType pusherPri enginepb.TxnPriority pusheePri enginepb.TxnPriority shouldPush bool }{ - {false, roachpb.PUSH_ABORT, min, min, false}, - {false, roachpb.PUSH_ABORT, min, mid1, false}, - {false, roachpb.PUSH_ABORT, min, mid2, false}, - {false, roachpb.PUSH_ABORT, min, max, false}, - {false, roachpb.PUSH_ABORT, mid1, min, true}, - {false, roachpb.PUSH_ABORT, mid1, mid1, false}, - {false, roachpb.PUSH_ABORT, mid1, mid2, false}, - {false, roachpb.PUSH_ABORT, mid1, max, false}, - {false, roachpb.PUSH_ABORT, mid2, min, true}, - {false, roachpb.PUSH_ABORT, mid2, mid1, false}, - {false, roachpb.PUSH_ABORT, mid2, mid2, false}, - {false, roachpb.PUSH_ABORT, mid2, max, false}, - {false, roachpb.PUSH_ABORT, max, min, true}, - {false, roachpb.PUSH_ABORT, max, mid1, true}, - {false, roachpb.PUSH_ABORT, max, mid2, true}, - {false, roachpb.PUSH_ABORT, max, max, false}, - {false, roachpb.PUSH_TIMESTAMP, min, min, false}, - {false, roachpb.PUSH_TIMESTAMP, min, mid1, false}, - {false, roachpb.PUSH_TIMESTAMP, min, mid2, false}, - {false, roachpb.PUSH_TIMESTAMP, min, max, false}, - {false, roachpb.PUSH_TIMESTAMP, mid1, min, true}, - {false, roachpb.PUSH_TIMESTAMP, mid1, mid1, false}, - {false, roachpb.PUSH_TIMESTAMP, mid1, mid2, false}, - {false, roachpb.PUSH_TIMESTAMP, mid1, max, false}, - {false, roachpb.PUSH_TIMESTAMP, mid2, min, true}, - {false, roachpb.PUSH_TIMESTAMP, mid2, mid1, false}, - {false, roachpb.PUSH_TIMESTAMP, mid2, mid2, false}, - {false, roachpb.PUSH_TIMESTAMP, mid2, max, false}, - {false, roachpb.PUSH_TIMESTAMP, max, min, true}, - {false, roachpb.PUSH_TIMESTAMP, max, mid1, true}, - {false, roachpb.PUSH_TIMESTAMP, max, mid2, true}, - {false, roachpb.PUSH_TIMESTAMP, max, max, false}, - {false, roachpb.PUSH_TOUCH, min, min, true}, - {false, roachpb.PUSH_TOUCH, min, mid1, true}, - {false, roachpb.PUSH_TOUCH, min, mid2, true}, - {false, roachpb.PUSH_TOUCH, min, max, true}, - {false, roachpb.PUSH_TOUCH, mid1, min, true}, - {false, roachpb.PUSH_TOUCH, mid1, mid1, true}, - {false, roachpb.PUSH_TOUCH, mid1, mid2, true}, - {false, roachpb.PUSH_TOUCH, mid1, max, true}, - {false, roachpb.PUSH_TOUCH, mid2, min, true}, - {false, roachpb.PUSH_TOUCH, mid2, mid1, true}, - {false, roachpb.PUSH_TOUCH, mid2, mid2, true}, - {false, roachpb.PUSH_TOUCH, mid2, max, true}, - {false, roachpb.PUSH_TOUCH, max, min, true}, - {false, roachpb.PUSH_TOUCH, max, mid1, true}, - {false, roachpb.PUSH_TOUCH, max, mid2, true}, - {false, roachpb.PUSH_TOUCH, max, max, true}, + {false, kvpb.PUSH_ABORT, min, min, false}, + {false, kvpb.PUSH_ABORT, min, mid1, false}, + {false, kvpb.PUSH_ABORT, min, mid2, false}, + {false, kvpb.PUSH_ABORT, min, max, false}, + {false, kvpb.PUSH_ABORT, mid1, min, true}, + {false, kvpb.PUSH_ABORT, mid1, mid1, false}, + {false, kvpb.PUSH_ABORT, mid1, mid2, false}, + {false, kvpb.PUSH_ABORT, mid1, max, false}, + {false, kvpb.PUSH_ABORT, mid2, min, true}, + {false, kvpb.PUSH_ABORT, mid2, mid1, false}, + {false, kvpb.PUSH_ABORT, mid2, mid2, false}, + {false, kvpb.PUSH_ABORT, mid2, max, false}, + {false, kvpb.PUSH_ABORT, max, min, true}, + {false, kvpb.PUSH_ABORT, max, mid1, true}, + {false, kvpb.PUSH_ABORT, max, mid2, true}, + {false, kvpb.PUSH_ABORT, max, max, false}, + {false, kvpb.PUSH_TIMESTAMP, min, min, false}, + {false, kvpb.PUSH_TIMESTAMP, min, mid1, false}, + {false, kvpb.PUSH_TIMESTAMP, min, mid2, false}, + {false, kvpb.PUSH_TIMESTAMP, min, max, false}, + {false, kvpb.PUSH_TIMESTAMP, mid1, min, true}, + {false, kvpb.PUSH_TIMESTAMP, mid1, mid1, false}, + {false, kvpb.PUSH_TIMESTAMP, mid1, mid2, false}, + {false, kvpb.PUSH_TIMESTAMP, mid1, max, false}, + {false, kvpb.PUSH_TIMESTAMP, mid2, min, true}, + {false, kvpb.PUSH_TIMESTAMP, mid2, mid1, false}, + {false, kvpb.PUSH_TIMESTAMP, mid2, mid2, false}, + {false, kvpb.PUSH_TIMESTAMP, mid2, max, false}, + {false, kvpb.PUSH_TIMESTAMP, max, min, true}, + {false, kvpb.PUSH_TIMESTAMP, max, mid1, true}, + {false, kvpb.PUSH_TIMESTAMP, max, mid2, true}, + {false, kvpb.PUSH_TIMESTAMP, max, max, false}, + {false, kvpb.PUSH_TOUCH, min, min, true}, + {false, kvpb.PUSH_TOUCH, min, mid1, true}, + {false, kvpb.PUSH_TOUCH, min, mid2, true}, + {false, kvpb.PUSH_TOUCH, min, max, true}, + {false, kvpb.PUSH_TOUCH, mid1, min, true}, + {false, kvpb.PUSH_TOUCH, mid1, mid1, true}, + {false, kvpb.PUSH_TOUCH, mid1, mid2, true}, + {false, kvpb.PUSH_TOUCH, mid1, max, true}, + {false, kvpb.PUSH_TOUCH, mid2, min, true}, + {false, kvpb.PUSH_TOUCH, mid2, mid1, true}, + {false, kvpb.PUSH_TOUCH, mid2, mid2, true}, + {false, kvpb.PUSH_TOUCH, mid2, max, true}, + {false, kvpb.PUSH_TOUCH, max, min, true}, + {false, kvpb.PUSH_TOUCH, max, mid1, true}, + {false, kvpb.PUSH_TOUCH, max, mid2, true}, + {false, kvpb.PUSH_TOUCH, max, max, true}, // Force pushes always push immediately. - {true, roachpb.PUSH_ABORT, min, min, true}, - {true, roachpb.PUSH_ABORT, min, mid1, true}, - {true, roachpb.PUSH_ABORT, min, mid2, true}, - {true, roachpb.PUSH_ABORT, min, max, true}, - {true, roachpb.PUSH_ABORT, mid1, min, true}, - {true, roachpb.PUSH_ABORT, mid1, mid1, true}, - {true, roachpb.PUSH_ABORT, mid1, mid2, true}, - {true, roachpb.PUSH_ABORT, mid1, max, true}, - {true, roachpb.PUSH_ABORT, mid2, min, true}, - {true, roachpb.PUSH_ABORT, mid2, mid1, true}, - {true, roachpb.PUSH_ABORT, mid2, mid2, true}, - {true, roachpb.PUSH_ABORT, mid2, max, true}, - {true, roachpb.PUSH_ABORT, max, min, true}, - {true, roachpb.PUSH_ABORT, max, mid1, true}, - {true, roachpb.PUSH_ABORT, max, mid2, true}, - {true, roachpb.PUSH_ABORT, max, max, true}, - {true, roachpb.PUSH_TIMESTAMP, min, min, true}, - {true, roachpb.PUSH_TIMESTAMP, min, mid1, true}, - {true, roachpb.PUSH_TIMESTAMP, min, mid2, true}, - {true, roachpb.PUSH_TIMESTAMP, min, max, true}, - {true, roachpb.PUSH_TIMESTAMP, mid1, min, true}, - {true, roachpb.PUSH_TIMESTAMP, mid1, mid1, true}, - {true, roachpb.PUSH_TIMESTAMP, mid1, mid2, true}, - {true, roachpb.PUSH_TIMESTAMP, mid1, max, true}, - {true, roachpb.PUSH_TIMESTAMP, mid2, min, true}, - {true, roachpb.PUSH_TIMESTAMP, mid2, mid1, true}, - {true, roachpb.PUSH_TIMESTAMP, mid2, mid2, true}, - {true, roachpb.PUSH_TIMESTAMP, mid2, max, true}, - {true, roachpb.PUSH_TIMESTAMP, max, min, true}, - {true, roachpb.PUSH_TIMESTAMP, max, mid1, true}, - {true, roachpb.PUSH_TIMESTAMP, max, mid2, true}, - {true, roachpb.PUSH_TIMESTAMP, max, max, true}, - {true, roachpb.PUSH_TOUCH, min, min, true}, - {true, roachpb.PUSH_TOUCH, min, mid1, true}, - {true, roachpb.PUSH_TOUCH, min, mid2, true}, - {true, roachpb.PUSH_TOUCH, min, max, true}, - {true, roachpb.PUSH_TOUCH, mid1, min, true}, - {true, roachpb.PUSH_TOUCH, mid1, mid1, true}, - {true, roachpb.PUSH_TOUCH, mid1, mid2, true}, - {true, roachpb.PUSH_TOUCH, mid1, max, true}, - {true, roachpb.PUSH_TOUCH, mid2, min, true}, - {true, roachpb.PUSH_TOUCH, mid2, mid1, true}, - {true, roachpb.PUSH_TOUCH, mid2, mid2, true}, - {true, roachpb.PUSH_TOUCH, mid2, max, true}, - {true, roachpb.PUSH_TOUCH, max, min, true}, - {true, roachpb.PUSH_TOUCH, max, mid1, true}, - {true, roachpb.PUSH_TOUCH, max, mid2, true}, - {true, roachpb.PUSH_TOUCH, max, max, true}, + {true, kvpb.PUSH_ABORT, min, min, true}, + {true, kvpb.PUSH_ABORT, min, mid1, true}, + {true, kvpb.PUSH_ABORT, min, mid2, true}, + {true, kvpb.PUSH_ABORT, min, max, true}, + {true, kvpb.PUSH_ABORT, mid1, min, true}, + {true, kvpb.PUSH_ABORT, mid1, mid1, true}, + {true, kvpb.PUSH_ABORT, mid1, mid2, true}, + {true, kvpb.PUSH_ABORT, mid1, max, true}, + {true, kvpb.PUSH_ABORT, mid2, min, true}, + {true, kvpb.PUSH_ABORT, mid2, mid1, true}, + {true, kvpb.PUSH_ABORT, mid2, mid2, true}, + {true, kvpb.PUSH_ABORT, mid2, max, true}, + {true, kvpb.PUSH_ABORT, max, min, true}, + {true, kvpb.PUSH_ABORT, max, mid1, true}, + {true, kvpb.PUSH_ABORT, max, mid2, true}, + {true, kvpb.PUSH_ABORT, max, max, true}, + {true, kvpb.PUSH_TIMESTAMP, min, min, true}, + {true, kvpb.PUSH_TIMESTAMP, min, mid1, true}, + {true, kvpb.PUSH_TIMESTAMP, min, mid2, true}, + {true, kvpb.PUSH_TIMESTAMP, min, max, true}, + {true, kvpb.PUSH_TIMESTAMP, mid1, min, true}, + {true, kvpb.PUSH_TIMESTAMP, mid1, mid1, true}, + {true, kvpb.PUSH_TIMESTAMP, mid1, mid2, true}, + {true, kvpb.PUSH_TIMESTAMP, mid1, max, true}, + {true, kvpb.PUSH_TIMESTAMP, mid2, min, true}, + {true, kvpb.PUSH_TIMESTAMP, mid2, mid1, true}, + {true, kvpb.PUSH_TIMESTAMP, mid2, mid2, true}, + {true, kvpb.PUSH_TIMESTAMP, mid2, max, true}, + {true, kvpb.PUSH_TIMESTAMP, max, min, true}, + {true, kvpb.PUSH_TIMESTAMP, max, mid1, true}, + {true, kvpb.PUSH_TIMESTAMP, max, mid2, true}, + {true, kvpb.PUSH_TIMESTAMP, max, max, true}, + {true, kvpb.PUSH_TOUCH, min, min, true}, + {true, kvpb.PUSH_TOUCH, min, mid1, true}, + {true, kvpb.PUSH_TOUCH, min, mid2, true}, + {true, kvpb.PUSH_TOUCH, min, max, true}, + {true, kvpb.PUSH_TOUCH, mid1, min, true}, + {true, kvpb.PUSH_TOUCH, mid1, mid1, true}, + {true, kvpb.PUSH_TOUCH, mid1, mid2, true}, + {true, kvpb.PUSH_TOUCH, mid1, max, true}, + {true, kvpb.PUSH_TOUCH, mid2, min, true}, + {true, kvpb.PUSH_TOUCH, mid2, mid1, true}, + {true, kvpb.PUSH_TOUCH, mid2, mid2, true}, + {true, kvpb.PUSH_TOUCH, mid2, max, true}, + {true, kvpb.PUSH_TOUCH, max, min, true}, + {true, kvpb.PUSH_TOUCH, max, mid1, true}, + {true, kvpb.PUSH_TOUCH, max, mid2, true}, + {true, kvpb.PUSH_TOUCH, max, max, true}, } for _, test := range testCases { t.Run("", func(t *testing.T) { - req := roachpb.PushTxnRequest{ + req := kvpb.PushTxnRequest{ Force: test.force, PushType: test.typ, PusherTxn: roachpb.Transaction{ @@ -206,30 +207,30 @@ func makeTS(w int64, l int32) hlc.Timestamp { func TestIsPushed(t *testing.T) { defer leaktest.AfterTest(t)() testCases := []struct { - typ roachpb.PushTxnType + typ kvpb.PushTxnType pushTo hlc.Timestamp txnStatus roachpb.TransactionStatus txnTimestamp hlc.Timestamp isPushed bool }{ - {roachpb.PUSH_ABORT, hlc.Timestamp{}, roachpb.PENDING, hlc.Timestamp{}, false}, - {roachpb.PUSH_ABORT, hlc.Timestamp{}, roachpb.STAGING, hlc.Timestamp{}, false}, - {roachpb.PUSH_ABORT, hlc.Timestamp{}, roachpb.ABORTED, hlc.Timestamp{}, true}, - {roachpb.PUSH_ABORT, hlc.Timestamp{}, roachpb.COMMITTED, hlc.Timestamp{}, true}, - {roachpb.PUSH_TIMESTAMP, makeTS(10, 1), roachpb.PENDING, hlc.Timestamp{}, false}, - {roachpb.PUSH_TIMESTAMP, makeTS(10, 1), roachpb.STAGING, hlc.Timestamp{}, false}, - {roachpb.PUSH_TIMESTAMP, makeTS(10, 1), roachpb.ABORTED, hlc.Timestamp{}, true}, - {roachpb.PUSH_TIMESTAMP, makeTS(10, 1), roachpb.COMMITTED, hlc.Timestamp{}, true}, - {roachpb.PUSH_TIMESTAMP, makeTS(10, 1), roachpb.PENDING, makeTS(10, 0), false}, - {roachpb.PUSH_TIMESTAMP, makeTS(10, 1), roachpb.PENDING, makeTS(10, 1), true}, - {roachpb.PUSH_TIMESTAMP, makeTS(10, 1), roachpb.PENDING, makeTS(10, 2), true}, - {roachpb.PUSH_TIMESTAMP, makeTS(10, 1), roachpb.STAGING, makeTS(10, 0), false}, - {roachpb.PUSH_TIMESTAMP, makeTS(10, 1), roachpb.STAGING, makeTS(10, 1), true}, - {roachpb.PUSH_TIMESTAMP, makeTS(10, 1), roachpb.STAGING, makeTS(10, 2), true}, + {kvpb.PUSH_ABORT, hlc.Timestamp{}, roachpb.PENDING, hlc.Timestamp{}, false}, + {kvpb.PUSH_ABORT, hlc.Timestamp{}, roachpb.STAGING, hlc.Timestamp{}, false}, + {kvpb.PUSH_ABORT, hlc.Timestamp{}, roachpb.ABORTED, hlc.Timestamp{}, true}, + {kvpb.PUSH_ABORT, hlc.Timestamp{}, roachpb.COMMITTED, hlc.Timestamp{}, true}, + {kvpb.PUSH_TIMESTAMP, makeTS(10, 1), roachpb.PENDING, hlc.Timestamp{}, false}, + {kvpb.PUSH_TIMESTAMP, makeTS(10, 1), roachpb.STAGING, hlc.Timestamp{}, false}, + {kvpb.PUSH_TIMESTAMP, makeTS(10, 1), roachpb.ABORTED, hlc.Timestamp{}, true}, + {kvpb.PUSH_TIMESTAMP, makeTS(10, 1), roachpb.COMMITTED, hlc.Timestamp{}, true}, + {kvpb.PUSH_TIMESTAMP, makeTS(10, 1), roachpb.PENDING, makeTS(10, 0), false}, + {kvpb.PUSH_TIMESTAMP, makeTS(10, 1), roachpb.PENDING, makeTS(10, 1), true}, + {kvpb.PUSH_TIMESTAMP, makeTS(10, 1), roachpb.PENDING, makeTS(10, 2), true}, + {kvpb.PUSH_TIMESTAMP, makeTS(10, 1), roachpb.STAGING, makeTS(10, 0), false}, + {kvpb.PUSH_TIMESTAMP, makeTS(10, 1), roachpb.STAGING, makeTS(10, 1), true}, + {kvpb.PUSH_TIMESTAMP, makeTS(10, 1), roachpb.STAGING, makeTS(10, 2), true}, } for _, test := range testCases { t.Run("", func(t *testing.T) { - req := roachpb.PushTxnRequest{ + req := kvpb.PushTxnRequest{ PushType: test.typ, PushTo: test.pushTo, } @@ -270,8 +271,8 @@ func TestMaybeWaitForPushWithContextCancellation(t *testing.T) { var mockSender kv.SenderFunc cfg := makeConfig(func( - ctx context.Context, ba *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, *kvpb.Error) { return mockSender(ctx, ba) }, stopper) q := NewQueue(cfg) @@ -283,18 +284,18 @@ func TestMaybeWaitForPushWithContextCancellation(t *testing.T) { // Mock out responses to any QueryTxn requests. mockSender = func( - ctx context.Context, ba *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, *kvpb.Error) { br := ba.CreateReply() - resp := br.Responses[0].GetInner().(*roachpb.QueryTxnResponse) + resp := br.Responses[0].GetInner().(*kvpb.QueryTxnResponse) resp.QueriedTxn = txn return br, nil } ctx, cancel := context.WithCancel(context.Background()) - waitingRes := make(chan *roachpb.Error) + waitingRes := make(chan *kvpb.Error) go func() { - req := roachpb.PushTxnRequest{PusheeTxn: txn.TxnMeta, PushType: roachpb.PUSH_ABORT} + req := kvpb.PushTxnRequest{PusheeTxn: txn.TxnMeta, PushType: kvpb.PUSH_ABORT} _, err := q.MaybeWaitForPush(ctx, &req) waitingRes <- err }() @@ -328,9 +329,9 @@ func TestMaybeWaitForQueryWithContextCancellation(t *testing.T) { q.Enable(1 /* leaseSeq */) ctx, cancel := context.WithCancel(context.Background()) - waitingRes := make(chan *roachpb.Error) + waitingRes := make(chan *kvpb.Error) go func() { - req := &roachpb.QueryTxnRequest{WaitForUpdate: true} + req := &kvpb.QueryTxnRequest{WaitForUpdate: true} waitingRes <- q.MaybeWaitForQuery(ctx, req) }() @@ -357,8 +358,8 @@ func TestPushersReleasedAfterAnyQueryTxnFindsAbortedTxn(t *testing.T) { defer stopper.Stop(context.Background()) var mockSender kv.SenderFunc cfg := makeConfig(func( - ctx context.Context, ba *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, *kvpb.Error) { return mockSender(ctx, ba) }, stopper) q := NewQueue(cfg) @@ -375,10 +376,10 @@ func TestPushersReleasedAfterAnyQueryTxnFindsAbortedTxn(t *testing.T) { const numPushees = 3 var queryTxnCount int32 mockSender = func( - ctx context.Context, ba *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, *kvpb.Error) { br := ba.CreateReply() - resp := br.Responses[0].GetInner().(*roachpb.QueryTxnResponse) + resp := br.Responses[0].GetInner().(*kvpb.QueryTxnResponse) resp.QueriedTxn = txn if atomic.AddInt32(&queryTxnCount, 1) == numPushees { // Only the last pusher's query observes an ABORTED transaction. As @@ -398,7 +399,7 @@ func TestPushersReleasedAfterAnyQueryTxnFindsAbortedTxn(t *testing.T) { go func() { defer wg.Done() ctx := context.Background() - req := roachpb.PushTxnRequest{PusheeTxn: txn.TxnMeta, PushType: roachpb.PUSH_ABORT} + req := kvpb.PushTxnRequest{PusheeTxn: txn.TxnMeta, PushType: kvpb.PUSH_ABORT} res, err := q.MaybeWaitForPush(ctx, &req) require.Nil(t, err) require.NotNil(t, res) diff --git a/pkg/kv/kvserver/uncertainty/BUILD.bazel b/pkg/kv/kvserver/uncertainty/BUILD.bazel index 944e32bdfc6c..260b9292b968 100644 --- a/pkg/kv/kvserver/uncertainty/BUILD.bazel +++ b/pkg/kv/kvserver/uncertainty/BUILD.bazel @@ -11,6 +11,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/uncertainty", visibility = ["//visibility:public"], deps = [ + "//pkg/kv/kvpb", "//pkg/kv/kvserver/kvserverpb", "//pkg/roachpb", "//pkg/util/hlc", @@ -26,6 +27,7 @@ go_test( args = ["-test.timeout=295s"], embed = [":uncertainty"], deps = [ + "//pkg/kv/kvpb", "//pkg/kv/kvserver/kvserverpb", "//pkg/roachpb", "//pkg/util/hlc", diff --git a/pkg/kv/kvserver/uncertainty/compute.go b/pkg/kv/kvserver/uncertainty/compute.go index 5b8b003548ab..36b67356dc5e 100644 --- a/pkg/kv/kvserver/uncertainty/compute.go +++ b/pkg/kv/kvserver/uncertainty/compute.go @@ -13,6 +13,7 @@ package uncertainty import ( "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/roachpb" ) @@ -71,7 +72,7 @@ import ( // is guaranteed to be greater than any write which occurred on the // right-hand side. func ComputeInterval( - h *roachpb.Header, status kvserverpb.LeaseStatus, maxOffset time.Duration, + h *kvpb.Header, status kvserverpb.LeaseStatus, maxOffset time.Duration, ) Interval { if h.Txn != nil { return computeIntervalForTxn(h.Txn, status) @@ -121,9 +122,9 @@ func computeIntervalForTxn(txn *roachpb.Transaction, status kvserverpb.LeaseStat } func computeIntervalForNonTxn( - h *roachpb.Header, status kvserverpb.LeaseStatus, maxOffset time.Duration, + h *kvpb.Header, status kvserverpb.LeaseStatus, maxOffset time.Duration, ) Interval { - if h.TimestampFromServerClock == nil || h.ReadConsistency != roachpb.CONSISTENT { + if h.TimestampFromServerClock == nil || h.ReadConsistency != kvpb.CONSISTENT { // Non-transactional requests with client-provided timestamps do not // guarantee linearizability. Neither do entirely inconsistent requests. // As a result, they do not have uncertainty intervals. diff --git a/pkg/kv/kvserver/uncertainty/compute_test.go b/pkg/kv/kvserver/uncertainty/compute_test.go index ec8db2243661..a29f8caffcee 100644 --- a/pkg/kv/kvserver/uncertainty/compute_test.go +++ b/pkg/kv/kvserver/uncertainty/compute_test.go @@ -13,6 +13,7 @@ package uncertainty import ( "testing" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -162,7 +163,7 @@ func TestComputeInterval(t *testing.T) { } for _, test := range testCases { t.Run(test.name, func(t *testing.T) { - var h roachpb.Header + var h kvpb.Header h.Txn = test.txn h.TimestampFromServerClock = test.tsFromServerClock require.Equal(t, test.exp, ComputeInterval(&h, test.lease, maxOffset)) diff --git a/pkg/kv/kvserver/uncertainty/doc.go b/pkg/kv/kvserver/uncertainty/doc.go index 7ce5d93248dd..ff244d845113 100644 --- a/pkg/kv/kvserver/uncertainty/doc.go +++ b/pkg/kv/kvserver/uncertainty/doc.go @@ -21,6 +21,7 @@ package uncertainty import ( + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" ) @@ -118,7 +119,7 @@ var D3 = Interval{} // guaranteed to observe any value written by any other transaction with a // happened-before relation to it, which is paramount to ensure single-key // linearizability and avoid stale reads. -var D4 = roachpb.ReadWithinUncertaintyIntervalError{} +var D4 = kvpb.ReadWithinUncertaintyIntervalError{} // D5 ———————————————————————————————————————————————— // @@ -371,7 +372,7 @@ var D7 = roachpb.Transaction{}.ObservedTimestamps // single-range, those that hit uncertainty errors can always retry on the // server, so these errors never bubble up to the client that initiated the // request. -var D8 = roachpb.Header{}.TimestampFromServerClock +var D8 = kvpb.Header{}.TimestampFromServerClock // D9 ———————————————————————————————————————————————— // diff --git a/pkg/kv/mock_transactional_sender.go b/pkg/kv/mock_transactional_sender.go index 340d9f38e2cd..2d9d6f3ece3d 100644 --- a/pkg/kv/mock_transactional_sender.go +++ b/pkg/kv/mock_transactional_sender.go @@ -13,6 +13,7 @@ package kv import ( "context" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -22,8 +23,8 @@ import ( // MockTransactionalSender allows a function to be used as a TxnSender. type MockTransactionalSender struct { senderFunc func( - context.Context, *roachpb.Transaction, *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, *roachpb.Error) + context.Context, *roachpb.Transaction, *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, *kvpb.Error) txn roachpb.Transaction } @@ -31,8 +32,8 @@ type MockTransactionalSender struct { // The passed in txn is cloned. func NewMockTransactionalSender( f func( - context.Context, *roachpb.Transaction, *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, *roachpb.Error), + context.Context, *roachpb.Transaction, *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, *kvpb.Error), txn *roachpb.Transaction, ) *MockTransactionalSender { return &MockTransactionalSender{senderFunc: f, txn: *txn} @@ -40,8 +41,8 @@ func NewMockTransactionalSender( // Send is part of the TxnSender interface. func (m *MockTransactionalSender) Send( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, *kvpb.Error) { return m.senderFunc(ctx, &m.txn, ba) } @@ -170,8 +171,8 @@ func (m *MockTransactionalSender) Active() bool { // UpdateStateOnRemoteRetryableErr is part of the TxnSender interface. func (m *MockTransactionalSender) UpdateStateOnRemoteRetryableErr( - ctx context.Context, pErr *roachpb.Error, -) *roachpb.Error { + ctx context.Context, pErr *kvpb.Error, +) *kvpb.Error { panic("unimplemented") } @@ -182,7 +183,7 @@ func (m *MockTransactionalSender) DisablePipelining() error { return nil } func (m *MockTransactionalSender) PrepareRetryableError( ctx context.Context, msg redact.RedactableString, ) error { - return roachpb.NewTransactionRetryWithProtoRefreshError(msg, m.txn.ID, *m.txn.Clone()) + return kvpb.NewTransactionRetryWithProtoRefreshError(msg, m.txn.ID, *m.txn.Clone()) } // Step is part of the TxnSender interface. @@ -220,7 +221,7 @@ func (m *MockTransactionalSender) DeferCommitWait(ctx context.Context) func(cont // GetTxnRetryableErr is part of the TxnSender interface. func (m *MockTransactionalSender) GetTxnRetryableErr( ctx context.Context, -) *roachpb.TransactionRetryWithProtoRefreshError { +) *kvpb.TransactionRetryWithProtoRefreshError { return nil } @@ -240,8 +241,8 @@ func (m *MockTransactionalSender) HasPerformedWrites() bool { // MockTxnSenderFactory is a TxnSenderFactory producing MockTxnSenders. type MockTxnSenderFactory struct { - senderFunc func(context.Context, *roachpb.Transaction, *roachpb.BatchRequest) ( - *roachpb.BatchResponse, *roachpb.Error) + senderFunc func(context.Context, *roachpb.Transaction, *kvpb.BatchRequest) ( + *kvpb.BatchResponse, *kvpb.Error) nonTxnSenderFunc Sender } @@ -252,8 +253,8 @@ var _ TxnSenderFactory = MockTxnSenderFactory{} // function is responsible for putting the txn inside the batch, if needed. func MakeMockTxnSenderFactory( senderFunc func( - context.Context, *roachpb.Transaction, *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, *roachpb.Error), + context.Context, *roachpb.Transaction, *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, *kvpb.Error), ) MockTxnSenderFactory { return MockTxnSenderFactory{ senderFunc: senderFunc, @@ -265,8 +266,8 @@ func MakeMockTxnSenderFactory( // requests. func MakeMockTxnSenderFactoryWithNonTxnSender( senderFunc func( - context.Context, *roachpb.Transaction, *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, *roachpb.Error), + context.Context, *roachpb.Transaction, *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, *kvpb.Error), nonTxnSenderFunc SenderFunc, ) MockTxnSenderFactory { return MockTxnSenderFactory{ diff --git a/pkg/kv/range_lookup.go b/pkg/kv/range_lookup.go index c77bc2d1132c..c045f0369030 100644 --- a/pkg/kv/range_lookup.go +++ b/pkg/kv/range_lookup.go @@ -15,6 +15,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/retry" @@ -166,7 +167,7 @@ func RangeLookup( ctx context.Context, sender Sender, key roachpb.Key, - rc roachpb.ReadConsistencyType, + rc kvpb.ReadConsistencyType, prefetchNum int64, prefetchReverse bool, ) (rs, preRs []roachpb.RangeDescriptor, err error) { @@ -250,7 +251,7 @@ func RangeLookup( // If we're doing an inconsistent scan and do not find any matching // descriptors, return to the caller so that it can retry by reading // from the leaseholder. - if rc == roachpb.INCONSISTENT { + if rc == kvpb.INCONSISTENT { return nil, nil, nil } log.Warningf(ctx, "range lookup of key %s found only non-matching ranges %v; retrying", @@ -268,7 +269,7 @@ func lookupRangeFwdScan( ctx context.Context, sender Sender, key roachpb.RKey, - rc roachpb.ReadConsistencyType, + rc kvpb.ReadConsistencyType, prefetchNum int64, prefetchReverse bool, ) (rs, preRs []roachpb.RangeDescriptor, err error) { @@ -288,12 +289,12 @@ func lookupRangeFwdScan( return nil, nil, errors.Wrap(err, "could not create scan bounds for range lookup") } - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.ReadConsistency = rc // If the caller is asking for a potentially stale result, we want to route // the request to the nearest replica rather than the leaseholder. - if rc == roachpb.INCONSISTENT { - ba.RoutingPolicy = roachpb.RoutingPolicy_NEAREST + if rc == kvpb.INCONSISTENT { + ba.RoutingPolicy = kvpb.RoutingPolicy_NEAREST } if prefetchReverse { // Even if we're prefetching in the reverse direction, we still scan @@ -320,8 +321,8 @@ func lookupRangeFwdScan( } else { ba.MaxSpanRequestKeys = prefetchNum + 1 } - ba.Add(&roachpb.ScanRequest{ - RequestHeader: roachpb.RequestHeaderFromSpan(bounds.AsRawSpanWithNoLocals()), + ba.Add(&kvpb.ScanRequest{ + RequestHeader: kvpb.RequestHeaderFromSpan(bounds.AsRawSpanWithNoLocals()), }) if !TestingIsRangeLookup(ba) { log.Fatalf(ctx, "BatchRequest %v not detectable as RangeLookup", ba) @@ -331,7 +332,7 @@ func lookupRangeFwdScan( if pErr != nil { return nil, nil, pErr.GoError() } - scanRes := br.Responses[0].GetInner().(*roachpb.ScanResponse) + scanRes := br.Responses[0].GetInner().(*kvpb.ScanResponse) descs, err := kvsToRangeDescriptors(scanRes.Rows) if err != nil { @@ -361,7 +362,7 @@ func lookupRangeRevScan( ctx context.Context, sender Sender, key roachpb.RKey, - rc roachpb.ReadConsistencyType, + rc kvpb.ReadConsistencyType, prefetchNum int64, prefetchReverse bool, fwdDescs, fwdIntentDescs []roachpb.RangeDescriptor, @@ -386,16 +387,16 @@ func lookupRangeRevScan( return nil, nil, errors.Wrap(err, "could not create scan bounds for reverse range lookup") } - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.ReadConsistency = rc // If the caller is asking for a potentially stale result, we want to route // the request to the nearest replica rather than the leaseholder. - if rc == roachpb.INCONSISTENT { - ba.RoutingPolicy = roachpb.RoutingPolicy_NEAREST + if rc == kvpb.INCONSISTENT { + ba.RoutingPolicy = kvpb.RoutingPolicy_NEAREST } ba.MaxSpanRequestKeys = maxKeys - ba.Add(&roachpb.ReverseScanRequest{ - RequestHeader: roachpb.RequestHeaderFromSpan(revBounds.AsRawSpanWithNoLocals()), + ba.Add(&kvpb.ReverseScanRequest{ + RequestHeader: kvpb.RequestHeaderFromSpan(revBounds.AsRawSpanWithNoLocals()), }) if !TestingIsRangeLookup(ba) { log.Fatalf(ctx, "BatchRequest %v not detectable as RangeLookup", ba) @@ -405,7 +406,7 @@ func lookupRangeRevScan( if pErr != nil { return nil, nil, pErr.GoError() } - revScanRes := br.Responses[0].GetInner().(*roachpb.ReverseScanResponse) + revScanRes := br.Responses[0].GetInner().(*kvpb.ReverseScanResponse) revDescs, err := kvsToRangeDescriptors(revScanRes.Rows) if err != nil { @@ -453,7 +454,7 @@ func kvsToRangeDescriptors(kvs []roachpb.KeyValue) ([]roachpb.RangeDescriptor, e // TestingIsRangeLookup returns if the provided BatchRequest looks like a single // RangeLookup scan. It can return false positives and should only be used in // tests. -func TestingIsRangeLookup(ba *roachpb.BatchRequest) bool { +func TestingIsRangeLookup(ba *kvpb.BatchRequest) bool { if ba.IsSingleRequest() { return TestingIsRangeLookupRequest(ba.Requests[0].GetInner()) } @@ -475,10 +476,10 @@ var rangeLookupEndKeyBounds = roachpb.Span{ // TestingIsRangeLookupRequest returns if the provided Request looks like a single // RangeLookup scan. It can return false positives and should only be used in // tests. -func TestingIsRangeLookupRequest(req roachpb.Request) bool { +func TestingIsRangeLookupRequest(req kvpb.Request) bool { switch req.(type) { - case *roachpb.ScanRequest: - case *roachpb.ReverseScanRequest: + case *kvpb.ScanRequest: + case *kvpb.ReverseScanRequest: default: return false } diff --git a/pkg/kv/range_lookup_test.go b/pkg/kv/range_lookup_test.go index 7c678bcfd199..f6469bc26599 100644 --- a/pkg/kv/range_lookup_test.go +++ b/pkg/kv/range_lookup_test.go @@ -16,6 +16,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -50,11 +51,11 @@ func TestRangeLookupRaceSplits(t *testing.T) { } lookupKey := roachpb.Key("k") - assertRangeLookupScan := func(ba *roachpb.BatchRequest) { + assertRangeLookupScan := func(ba *kvpb.BatchRequest) { if len(ba.Requests) != 1 { t.Fatalf("expected single request, found %v", ba) } - scan, ok := ba.Requests[0].GetInner().(*roachpb.ScanRequest) + scan, ok := ba.Requests[0].GetInner().(*kvpb.ScanRequest) if !ok { t.Fatalf("expected single scan request, found %v", ba) } @@ -79,7 +80,7 @@ func TestRangeLookupRaceSplits(t *testing.T) { goodRes := newScanRespFromRangeDescriptors(&desc1AfterSplit) attempt := 0 - sender := SenderFunc(func(_ context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + sender := SenderFunc(func(_ context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { // Increment the attempt counter after each attempt. defer func() { attempt++ @@ -106,7 +107,7 @@ func TestRangeLookupRaceSplits(t *testing.T) { context.Background(), sender, lookupKey, - roachpb.READ_UNCOMMITTED, + kvpb.READ_UNCOMMITTED, 0, /* prefetchNum */ false, /* prefetchReverse */ ) @@ -137,7 +138,7 @@ func TestRangeLookupRaceSplits(t *testing.T) { } attempt := 0 - sender := SenderFunc(func(_ context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + sender := SenderFunc(func(_ context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { // Increment the attempt counter after each attempt. defer func() { attempt++ @@ -157,7 +158,7 @@ func TestRangeLookupRaceSplits(t *testing.T) { context.Background(), sender, lookupKey, - roachpb.READ_UNCOMMITTED, + kvpb.READ_UNCOMMITTED, 0, /* prefetchNum */ false, /* prefetchReverse */ ) @@ -175,9 +176,9 @@ func TestRangeLookupRaceSplits(t *testing.T) { }) } -func newScanRespFromRangeDescriptors(descs ...*roachpb.RangeDescriptor) *roachpb.BatchResponse { - br := &roachpb.BatchResponse{} - r := &roachpb.ScanResponse{} +func newScanRespFromRangeDescriptors(descs ...*roachpb.RangeDescriptor) *kvpb.BatchResponse { + br := &kvpb.BatchResponse{} + r := &kvpb.ScanResponse{} for _, desc := range descs { var kv roachpb.KeyValue if err := kv.Value.SetProto(desc); err != nil { diff --git a/pkg/kv/sender.go b/pkg/kv/sender.go index 4424e5cb8480..dac083a7d3e8 100644 --- a/pkg/kv/sender.go +++ b/pkg/kv/sender.go @@ -13,6 +13,7 @@ package kv import ( "context" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -87,7 +88,7 @@ type Sender interface { // about what the client should update, as opposed to a full txn // that the client is expected to diff with its copy and apply all // the updates. - Send(context.Context, *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) + Send(context.Context, *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) } // TxnSender is the interface used to call into a CockroachDB instance @@ -182,7 +183,7 @@ type TxnSender interface { // UpdateStateOnRemoteRetryableErr updates the txn in response to an // error encountered when running a request through the txn. - UpdateStateOnRemoteRetryableErr(context.Context, *roachpb.Error) *roachpb.Error + UpdateStateOnRemoteRetryableErr(context.Context, *kvpb.Error) *kvpb.Error // DisablePipelining instructs the TxnSender not to pipeline // requests. It should rarely be necessary to call this method. It @@ -322,7 +323,7 @@ type TxnSender interface { // otherwise nil. In this state Send() always fails with the same retryable // error. ClearTxnRetryableErr can be called to clear this error and make // TxnSender usable again. - GetTxnRetryableErr(ctx context.Context) *roachpb.TransactionRetryWithProtoRefreshError + GetTxnRetryableErr(ctx context.Context) *kvpb.TransactionRetryWithProtoRefreshError // ClearTxnRetryableErr clears the retryable error, if any. ClearTxnRetryableErr(ctx context.Context) @@ -395,12 +396,12 @@ type TxnSenderFactory interface { // SenderFunc is an adapter to allow the use of ordinary functions as // Senders. -type SenderFunc func(context.Context, *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) +type SenderFunc func(context.Context, *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) // Send calls f(ctx, c). func (f SenderFunc) Send( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, *kvpb.Error) { return f(ctx, ba) } @@ -435,9 +436,9 @@ func (f NonTransactionalFactoryFunc) NonTransactionalSender() Sender { // returns the unwrapped response or an error. It's valid to pass a // `nil` context; an empty one is used in that case. func SendWrappedWith( - ctx context.Context, sender Sender, h roachpb.Header, args roachpb.Request, -) (roachpb.Response, *roachpb.Error) { - return SendWrappedWithAdmission(ctx, sender, h, roachpb.AdmissionHeader{}, args) + ctx context.Context, sender Sender, h kvpb.Header, args kvpb.Request, +) (kvpb.Response, *kvpb.Error) { + return SendWrappedWithAdmission(ctx, sender, h, kvpb.AdmissionHeader{}, args) } // SendWrappedWithAdmission is a convenience function which wraps the request @@ -445,13 +446,9 @@ func SendWrappedWith( // unwrapped response or an error. It's valid to pass a `nil` context; an // empty one is used in that case. func SendWrappedWithAdmission( - ctx context.Context, - sender Sender, - h roachpb.Header, - ah roachpb.AdmissionHeader, - args roachpb.Request, -) (roachpb.Response, *roachpb.Error) { - ba := &roachpb.BatchRequest{} + ctx context.Context, sender Sender, h kvpb.Header, ah kvpb.AdmissionHeader, args kvpb.Request, +) (kvpb.Response, *kvpb.Error) { + ba := &kvpb.BatchRequest{} ba.Header = h ba.AdmissionHeader = ah ba.Add(args) @@ -471,15 +468,15 @@ func SendWrappedWithAdmission( // TODO(tschottdorf): should move this to testutils and merge with // other helpers which are used, for example, in `storage`. func SendWrapped( - ctx context.Context, sender Sender, args roachpb.Request, -) (roachpb.Response, *roachpb.Error) { - return SendWrappedWith(ctx, sender, roachpb.Header{}, args) + ctx context.Context, sender Sender, args kvpb.Request, +) (kvpb.Response, *kvpb.Error) { + return SendWrappedWith(ctx, sender, kvpb.Header{}, args) } // Wrap returns a Sender which applies the given function before delegating to // the supplied Sender. -func Wrap(sender Sender, f func(*roachpb.BatchRequest) *roachpb.BatchRequest) Sender { - return SenderFunc(func(ctx context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { +func Wrap(sender Sender, f func(*kvpb.BatchRequest) *kvpb.BatchRequest) Sender { + return SenderFunc(func(ctx context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { return sender.Send(ctx, f(ba)) }) } diff --git a/pkg/kv/txn.go b/pkg/kv/txn.go index 537494c45d32..7f0914a5bc33 100644 --- a/pkg/kv/txn.go +++ b/pkg/kv/txn.go @@ -15,6 +15,7 @@ import ( "fmt" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" @@ -93,7 +94,7 @@ type Txn struct { // transaction. Only certain paths initialize this properly, and the // remaining just use the zero value. The set of code paths that initialize // this are expected to expand over time. - admissionHeader roachpb.AdmissionHeader + admissionHeader kvpb.AdmissionHeader } // NewTxn returns a new RootTxn. @@ -118,7 +119,7 @@ type Txn struct { // See also db.NewTxn(). func NewTxn(ctx context.Context, db *DB, gatewayNodeID roachpb.NodeID) *Txn { return NewTxnWithAdmissionControl( - ctx, db, gatewayNodeID, roachpb.AdmissionHeader_OTHER, admissionpb.NormalPri) + ctx, db, gatewayNodeID, kvpb.AdmissionHeader_OTHER, admissionpb.NormalPri) } // NewTxnWithAdmissionControl creates a new transaction with the specified @@ -127,7 +128,7 @@ func NewTxnWithAdmissionControl( ctx context.Context, db *DB, gatewayNodeID roachpb.NodeID, - source roachpb.AdmissionHeader_Source, + source kvpb.AdmissionHeader_Source, priority admissionpb.WorkPriority, ) *Txn { if db == nil { @@ -145,7 +146,7 @@ func NewTxnWithAdmissionControl( int32(db.ctx.NodeID.SQLInstanceID()), ) txn := NewTxnFromProto(ctx, db, gatewayNodeID, now, RootTxn, &kvTxn) - txn.admissionHeader = roachpb.AdmissionHeader{ + txn.admissionHeader = kvpb.AdmissionHeader{ CreateTime: db.clock.PhysicalNow(), Priority: int32(priority), Source: source, @@ -165,7 +166,7 @@ func NewTxnWithSteppingEnabled( qualityOfService sessiondatapb.QoSLevel, ) *Txn { txn := NewTxnWithAdmissionControl(ctx, db, gatewayNodeID, - roachpb.AdmissionHeader_FROM_SQL, admissionpb.WorkPriority(qualityOfService)) + kvpb.AdmissionHeader_FROM_SQL, admissionpb.WorkPriority(qualityOfService)) _ = txn.ConfigureStepping(ctx, SteppingEnabled) return txn } @@ -178,7 +179,7 @@ func NewTxnWithSteppingEnabled( // details. func NewTxnRootKV(ctx context.Context, db *DB, gatewayNodeID roachpb.NodeID) *Txn { return NewTxnWithAdmissionControl( - ctx, db, gatewayNodeID, roachpb.AdmissionHeader_ROOT_KV, admissionpb.NormalPri) + ctx, db, gatewayNodeID, kvpb.AdmissionHeader_ROOT_KV, admissionpb.NormalPri) } // NewTxnFromProto is like NewTxn but assumes the Transaction object is already initialized. @@ -680,7 +681,7 @@ func (txn *Txn) commit(ctx context.Context) error { // will be subject to admission control, and the zero CreateTime will give // it preference within the tenant. et := endTxnReq(true, txn.deadline()) - ba := &roachpb.BatchRequest{Requests: et.unionArr[:]} + ba := &kvpb.BatchRequest{Requests: et.unionArr[:]} _, pErr := txn.Send(ctx, ba) if pErr == nil { for _, t := range txn.commitTriggers { @@ -807,7 +808,7 @@ func (txn *Txn) Rollback(ctx context.Context) error { return txn.rollback(ctx).GoError() } -func (txn *Txn) rollback(ctx context.Context) *roachpb.Error { +func (txn *Txn) rollback(ctx context.Context) *kvpb.Error { log.VEventf(ctx, 2, "rolling back transaction") // If the client has already disconnected, fall back to asynchronous cleanup @@ -819,7 +820,7 @@ func (txn *Txn) rollback(ctx context.Context) *roachpb.Error { // settings, it will be subject to admission control, and the zero // CreateTime will give it preference within the tenant. et := endTxnReq(false, hlc.Timestamp{} /* deadline */) - ba := &roachpb.BatchRequest{Requests: et.unionArr[:]} + ba := &kvpb.BatchRequest{Requests: et.unionArr[:]} _, pErr := txn.Send(ctx, ba) if pErr == nil { return nil @@ -845,12 +846,12 @@ func (txn *Txn) rollback(ctx context.Context) *roachpb.Error { // settings, it will be subject to admission control, and the zero // CreateTime will give it preference within the tenant. et := endTxnReq(false, hlc.Timestamp{} /* deadline */) - ba := &roachpb.BatchRequest{Requests: et.unionArr[:]} + ba := &kvpb.BatchRequest{Requests: et.unionArr[:]} _ = contextutil.RunWithTimeout(ctx, "async txn rollback", asyncRollbackTimeout, func(ctx context.Context) error { if _, pErr := txn.Send(ctx, ba); pErr != nil { - if statusErr, ok := pErr.GetDetail().(*roachpb.TransactionStatusError); ok && - statusErr.Reason == roachpb.TransactionStatusError_REASON_TXN_COMMITTED { + if statusErr, ok := pErr.GetDetail().(*kvpb.TransactionStatusError); ok && + statusErr.Reason == kvpb.TransactionStatusError_REASON_TXN_COMMITTED { // A common cause of these async rollbacks failing is when they're // triggered by a ctx canceled while a commit is in-flight (and it's too // late for it to be canceled), and so the rollback finds the txn to be @@ -864,7 +865,7 @@ func (txn *Txn) rollback(ctx context.Context) *roachpb.Error { }) }); err != nil { cancel() - return roachpb.NewError(err) + return kvpb.NewError(err) } return nil } @@ -881,9 +882,9 @@ func (txn *Txn) AddCommitTrigger(trigger func(ctx context.Context)) { // endTxnReqAlloc is used to batch the heap allocations of an EndTxn request. type endTxnReqAlloc struct { - req roachpb.EndTxnRequest - union roachpb.RequestUnion_EndTxn - unionArr [1]roachpb.RequestUnion + req kvpb.EndTxnRequest + union kvpb.RequestUnion_EndTxn + unionArr [1]kvpb.RequestUnion } func endTxnReq(commit bool, deadline hlc.Timestamp) *endTxnReqAlloc { @@ -933,7 +934,7 @@ func (txn *Txn) exec(ctx context.Context, fn func(context.Context, *Txn) error) err = txn.Commit(ctx) log.Eventf(ctx, "client.Txn did AutoCommit. err: %v", err) if err != nil { - if !errors.HasType(err, (*roachpb.TransactionRetryWithProtoRefreshError)(nil)) { + if !errors.HasType(err, (*kvpb.TransactionRetryWithProtoRefreshError)(nil)) { // We can't retry, so let the caller know we tried to // autocommit. err = &AutoCommitError{cause: err} @@ -944,14 +945,14 @@ func (txn *Txn) exec(ctx context.Context, fn func(context.Context, *Txn) error) var retryable bool if err != nil { - if errors.HasType(err, (*roachpb.UnhandledRetryableError)(nil)) { + if errors.HasType(err, (*kvpb.UnhandledRetryableError)(nil)) { if txn.typ == RootTxn { // We sent transactional requests, so the TxnCoordSender was supposed to // turn retryable errors into TransactionRetryWithProtoRefreshError. Note that this // applies only in the case where this is the root transaction. log.Fatalf(ctx, "unexpected UnhandledRetryableError at the txn.exec() level: %s", err) } - } else if t := (*roachpb.TransactionRetryWithProtoRefreshError)(nil); errors.As(err, &t) { + } else if t := (*kvpb.TransactionRetryWithProtoRefreshError)(nil); errors.As(err, &t) { if !txn.IsRetryableErrMeantForTxn(*t) { // Make sure the txn record that err carries is for this txn. // If it's not, we terminate the "retryable" character of the error. We @@ -998,7 +999,7 @@ func (txn *Txn) PrepareForRetry(ctx context.Context) { // IsRetryableErrMeantForTxn returns true if err is a retryable // error meant to restart this client transaction. func (txn *Txn) IsRetryableErrMeantForTxn( - retryErr roachpb.TransactionRetryWithProtoRefreshError, + retryErr kvpb.TransactionRetryWithProtoRefreshError, ) bool { if txn.typ != RootTxn { panic(errors.AssertionFailedf("IsRetryableErrMeantForTxn() called on leaf txn")) @@ -1026,8 +1027,8 @@ func (txn *Txn) IsRetryableErrMeantForTxn( // commit or clean-up explicitly even when that may not be required // (or even erroneous). Returns (nil, nil) for an empty batch. func (txn *Txn) Send( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, *kvpb.Error) { // Fill in the GatewayNodeID on the batch if the txn knows it. // NOTE(andrei): It seems a bit ugly that we're filling in the batches here as // opposed to the point where the requests are being created, but @@ -1039,7 +1040,7 @@ func (txn *Txn) Send( // Requests with a bounded staleness header should use NegotiateAndSend. if ba.BoundedStaleness != nil { - return nil, roachpb.NewError(errors.AssertionFailedf( + return nil, kvpb.NewError(errors.AssertionFailedf( "bounded staleness header passed to Txn.Send: %s", ba.String())) } @@ -1060,7 +1061,7 @@ func (txn *Txn) Send( return br, nil } - if retryErr, ok := pErr.GetDetail().(*roachpb.TransactionRetryWithProtoRefreshError); ok { + if retryErr, ok := pErr.GetDetail().(*kvpb.TransactionRetryWithProtoRefreshError); ok { if requestTxnID != retryErr.TxnID { // KV should not return errors for transactions other than the one that sent // the request. @@ -1073,7 +1074,7 @@ func (txn *Txn) Send( } func (txn *Txn) handleRetryableErrLocked( - ctx context.Context, retryErr *roachpb.TransactionRetryWithProtoRefreshError, + ctx context.Context, retryErr *kvpb.TransactionRetryWithProtoRefreshError, ) { txn.resetDeadlineLocked() txn.replaceRootSenderIfTxnAbortedLocked(ctx, retryErr, retryErr.TxnID) @@ -1118,13 +1119,13 @@ func (txn *Txn) handleRetryableErrLocked( // and perform the read. Callers can use this flexibility to trade off increased // staleness for reduced latency. func (txn *Txn) NegotiateAndSend( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, *roachpb.Error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, *kvpb.Error) { if err := txn.checkNegotiateAndSendPreconditions(ctx, ba); err != nil { - return nil, roachpb.NewError(err) + return nil, kvpb.NewError(err) } if err := txn.applyDeadlineToBoundedStaleness(ctx, ba.BoundedStaleness); err != nil { - return nil, roachpb.NewError(err) + return nil, kvpb.NewError(err) } // Attempt to hit the server-side negotiation fast-path. This fast-path @@ -1155,14 +1156,14 @@ func (txn *Txn) NegotiateAndSend( // Fix the transaction's timestamp at the result of the server-side // timestamp negotiation. if err := txn.SetFixedTimestamp(ctx, br.Timestamp); err != nil { - return nil, roachpb.NewError(err) + return nil, kvpb.NewError(err) } // Note that we do not need to inform the TxnCoordSender about the // non-transactional reads that we issued on behalf of it. Now that the // transaction's timestamp is fixed, it won't be able to refresh anyway. return br, nil } - if _, ok := pErr.GetDetail().(*roachpb.OpRequiresTxnError); !ok { + if _, ok := pErr.GetDetail().(*kvpb.OpRequiresTxnError); !ok { return nil, pErr } @@ -1175,13 +1176,13 @@ func (txn *Txn) NegotiateAndSend( // // TODO(nvanbenschoten): implement this. #67554. - return nil, roachpb.NewError(unimplemented.NewWithIssue(67554, + return nil, kvpb.NewError(unimplemented.NewWithIssue(67554, "cross-range bounded staleness reads not yet implemented")) } // checks preconditions on BatchRequest and Txn for NegotiateAndSend. func (txn *Txn) checkNegotiateAndSendPreconditions( - ctx context.Context, ba *roachpb.BatchRequest, + ctx context.Context, ba *kvpb.BatchRequest, ) (err error) { assert := func(b bool, s string) { if !b { @@ -1200,7 +1201,7 @@ func (txn *Txn) checkNegotiateAndSendPreconditions( } assert(ba.Timestamp.IsEmpty(), "timestamp must not be set") assert(ba.Txn == nil, "txn must not be set") - assert(ba.ReadConsistency == roachpb.CONSISTENT, "read consistency must be set to CONSISTENT") + assert(ba.ReadConsistency == kvpb.CONSISTENT, "read consistency must be set to CONSISTENT") assert(ba.IsReadOnly(), "batch must be read-only") assert(!ba.IsLocking(), "batch must not be locking") assert(txn.typ == RootTxn, "txn must be root") @@ -1211,7 +1212,7 @@ func (txn *Txn) checkNegotiateAndSendPreconditions( // applyDeadlineToBoundedStaleness modifies the bounded staleness header to // ensure that the negotiated timestamp respects the transaction deadline. func (txn *Txn) applyDeadlineToBoundedStaleness( - ctx context.Context, bs *roachpb.BoundedStalenessHeader, + ctx context.Context, bs *kvpb.BoundedStalenessHeader, ) error { d := txn.deadline() if d.IsEmpty() { @@ -1266,7 +1267,7 @@ func (txn *Txn) GetLeafTxnInputStateOrRejectClient( defer txn.mu.Unlock() tfs, err := txn.mu.sender.GetLeafTxnInputState(ctx, OnlyPending) if err != nil { - var retryErr *roachpb.TransactionRetryWithProtoRefreshError + var retryErr *kvpb.TransactionRetryWithProtoRefreshError if errors.As(err, &retryErr) { txn.handleRetryableErrLocked(ctx, retryErr) } @@ -1315,7 +1316,7 @@ func (txn *Txn) UpdateRootWithLeafFinalState( // UpdateStateOnRemoteRetryableErr updates the txn in response to an error // encountered when running a request through the txn. Returns a // TransactionRetryWithProtoRefreshError on success or another error on failure. -func (txn *Txn) UpdateStateOnRemoteRetryableErr(ctx context.Context, pErr *roachpb.Error) error { +func (txn *Txn) UpdateStateOnRemoteRetryableErr(ctx context.Context, pErr *kvpb.Error) error { if txn.typ != RootTxn { return errors.AssertionFailedf("UpdateStateOnRemoteRetryableErr() called on leaf txn") } @@ -1323,7 +1324,7 @@ func (txn *Txn) UpdateStateOnRemoteRetryableErr(ctx context.Context, pErr *roach txn.mu.Lock() defer txn.mu.Unlock() - if pErr.TransactionRestart() == roachpb.TransactionRestart_NONE { + if pErr.TransactionRestart() == kvpb.TransactionRestart_NONE { log.Fatalf(ctx, "unexpected non-retryable error: %s", pErr) } @@ -1338,7 +1339,7 @@ func (txn *Txn) UpdateStateOnRemoteRetryableErr(ctx context.Context, pErr *roach } pErr = txn.mu.sender.UpdateStateOnRemoteRetryableErr(ctx, pErr) - txn.replaceRootSenderIfTxnAbortedLocked(ctx, pErr.GetDetail().(*roachpb.TransactionRetryWithProtoRefreshError), origTxnID) + txn.replaceRootSenderIfTxnAbortedLocked(ctx, pErr.GetDetail().(*kvpb.TransactionRetryWithProtoRefreshError), origTxnID) return pErr.GoError() } @@ -1350,7 +1351,7 @@ func (txn *Txn) UpdateStateOnRemoteRetryableErr(ctx context.Context, pErr *roach // origTxnID is the id of the txn that generated retryErr. Note that this can be // different from retryErr.Transaction - the latter might be a new transaction. func (txn *Txn) replaceRootSenderIfTxnAbortedLocked( - ctx context.Context, retryErr *roachpb.TransactionRetryWithProtoRefreshError, origTxnID uuid.UUID, + ctx context.Context, retryErr *kvpb.TransactionRetryWithProtoRefreshError, origTxnID uuid.UUID, ) { // The proto inside the error has been prepared for use by the next // transaction attempt. @@ -1570,7 +1571,7 @@ func (txn *Txn) DeferCommitWait(ctx context.Context) func(context.Context) error // AdmissionHeader returns the admission header for work done in the context // of this transaction. -func (txn *Txn) AdmissionHeader() roachpb.AdmissionHeader { +func (txn *Txn) AdmissionHeader() kvpb.AdmissionHeader { h := txn.admissionHeader if txn.mu.sender.IsLocking() { // Assign higher priority to requests by txns that are locking, so that diff --git a/pkg/kv/txn_external_test.go b/pkg/kv/txn_external_test.go index d3bc32770700..fbfeccf4ccf0 100644 --- a/pkg/kv/txn_external_test.go +++ b/pkg/kv/txn_external_test.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -121,34 +122,34 @@ func TestRollbackAfterAmbiguousCommit(t *testing.T) { var key roachpb.Key commitBlocked := make(chan struct{}) onCommitReqFilter := func( - ba *roachpb.BatchRequest, fn func(et *roachpb.EndTxnRequest) *roachpb.Error, - ) *roachpb.Error { + ba *kvpb.BatchRequest, fn func(et *kvpb.EndTxnRequest) *kvpb.Error, + ) *kvpb.Error { if atomic.LoadInt64(&filterSet) == 0 { return nil } - req, ok := ba.GetArg(roachpb.EndTxn) + req, ok := ba.GetArg(kvpb.EndTxn) if !ok { return nil } - et := req.(*roachpb.EndTxnRequest) + et := req.(*kvpb.EndTxnRequest) if et.Key.Equal(key) && et.Commit { return fn(et) } return nil } - blockCommitReqFilter := func(ctx context.Context, ba *roachpb.BatchRequest) *roachpb.Error { - return onCommitReqFilter(ba, func(et *roachpb.EndTxnRequest) *roachpb.Error { + blockCommitReqFilter := func(ctx context.Context, ba *kvpb.BatchRequest) *kvpb.Error { + return onCommitReqFilter(ba, func(et *kvpb.EndTxnRequest) *kvpb.Error { // Inform the test that the commit is blocked. commitBlocked <- struct{}{} // Block until the client interrupts the commit. The client will // cancel its context, at which point gRPC will return an error // to the client and marshall the cancelation to the server. <-ctx.Done() - return roachpb.NewError(ctx.Err()) + return kvpb.NewError(ctx.Err()) }) } - addInFlightWriteToCommitReqFilter := func(ctx context.Context, ba *roachpb.BatchRequest) *roachpb.Error { - return onCommitReqFilter(ba, func(et *roachpb.EndTxnRequest) *roachpb.Error { + addInFlightWriteToCommitReqFilter := func(ctx context.Context, ba *kvpb.BatchRequest) *kvpb.Error { + return onCommitReqFilter(ba, func(et *kvpb.EndTxnRequest) *kvpb.Error { // Add a fake in-flight write. et.InFlightWrites = append(et.InFlightWrites, roachpb.SequencedWrite{ Key: key.Next(), Sequence: et.Sequence, @@ -167,7 +168,7 @@ func TestRollbackAfterAmbiguousCommit(t *testing.T) { // do this either before or after the request completes, depending // on the status that the test wants the txn record to be in when // the rollback is performed. - TestingRequestFilter: func(ctx context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + TestingRequestFilter: func(ctx context.Context, ba *kvpb.BatchRequest) *kvpb.Error { if testCase.txnStatus == roachpb.PENDING { // Block and reject before the request writes the txn record. return blockCommitReqFilter(ctx, ba) @@ -181,7 +182,7 @@ func TestRollbackAfterAmbiguousCommit(t *testing.T) { } return nil }, - TestingResponseFilter: func(ctx context.Context, ba *roachpb.BatchRequest, _ *roachpb.BatchResponse) *roachpb.Error { + TestingResponseFilter: func(ctx context.Context, ba *kvpb.BatchRequest, _ *kvpb.BatchResponse) *kvpb.Error { if testCase.txnStatus != roachpb.PENDING { // Block and reject after the request writes the txn record. return blockCommitReqFilter(ctx, ba) @@ -252,7 +253,7 @@ func TestRollbackAfterAmbiguousCommit(t *testing.T) { cancelCommit() commitErr := <-commitCh - require.IsType(t, &roachpb.AmbiguousResultError{}, commitErr) + require.IsType(t, &kvpb.AmbiguousResultError{}, commitErr) // If the test wants the upcoming rollback to find a COMMITTED record, // we'll perform transaction recovery. This will leave the transaction in @@ -260,8 +261,8 @@ func TestRollbackAfterAmbiguousCommit(t *testing.T) { if testCase.txnStatus == roachpb.COMMITTED && !testCase.txnRecordCleanedUp { // Sanity check - verify that the txn is STAGING. txnProto := txn.TestingCloneTxn() - queryTxn := roachpb.QueryTxnRequest{ - RequestHeader: roachpb.RequestHeader{ + queryTxn := kvpb.QueryTxnRequest{ + RequestHeader: kvpb.RequestHeader{ Key: txnProto.Key, }, Txn: txnProto.TxnMeta, @@ -428,26 +429,26 @@ func testTxnNegotiateAndSendDoesNotBlock(t *testing.T, multiRange, strict, route // error (WriteIntentError) under conditions that would otherwise // cause us to block on an intent. Otherwise, allow the request to be // redirected to the leaseholder and to block on intents. - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} if strict { - ba.BoundedStaleness = &roachpb.BoundedStalenessHeader{ + ba.BoundedStaleness = &kvpb.BoundedStalenessHeader{ MinTimestampBound: minTSBound, MinTimestampBoundStrict: true, } ba.WaitPolicy = lock.WaitPolicy_Error } else { - ba.BoundedStaleness = &roachpb.BoundedStalenessHeader{ + ba.BoundedStaleness = &kvpb.BoundedStalenessHeader{ MinTimestampBound: store.Clock().Now(), MinTimestampBoundStrict: false, } ba.WaitPolicy = lock.WaitPolicy_Block } - ba.RoutingPolicy = roachpb.RoutingPolicy_LEASEHOLDER + ba.RoutingPolicy = kvpb.RoutingPolicy_LEASEHOLDER if routeNearest { - ba.RoutingPolicy = roachpb.RoutingPolicy_NEAREST + ba.RoutingPolicy = kvpb.RoutingPolicy_NEAREST } - ba.Add(&roachpb.ScanRequest{ - RequestHeader: roachpb.RequestHeaderFromSpan(keySpan), + ba.Add(&kvpb.ScanRequest{ + RequestHeader: kvpb.RequestHeaderFromSpan(keySpan), }) // Trace the request so we can determine whether it was served as a diff --git a/pkg/kv/txn_test.go b/pkg/kv/txn_test.go index 50942c7093cc..9a88788cc0ea 100644 --- a/pkg/kv/txn_test.go +++ b/pkg/kv/txn_test.go @@ -17,6 +17,7 @@ import ( "regexp" "testing" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -32,7 +33,7 @@ import ( ) var ( - testPutResp = roachpb.PutResponse{} + testPutResp = kvpb.PutResponse{} ) // An example of verbose tracing being used to dump a trace around a @@ -80,18 +81,18 @@ func TestTxnVerboseTrace(t *testing.T) { } func newTestTxnFactory( - createReply func(*roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error), + createReply func(*kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error), ) TxnSenderFactory { return MakeMockTxnSenderFactory( func( - ctx context.Context, txn *roachpb.Transaction, ba *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, *roachpb.Error) { + ctx context.Context, txn *roachpb.Transaction, ba *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, *kvpb.Error) { if ba.UserPriority == 0 { ba.UserPriority = 1 } - var br *roachpb.BatchResponse - var pErr *roachpb.Error + var br *kvpb.BatchResponse + var pErr *kvpb.Error ba.Txn = txn @@ -106,14 +107,14 @@ func newTestTxnFactory( status := roachpb.PENDING for i, req := range ba.Requests { args := req.GetInner() - if _, ok := args.(*roachpb.PutRequest); ok { + if _, ok := args.(*kvpb.PutRequest); ok { testPutRespCopy := testPutResp union := &br.Responses[i] // avoid operating on copy union.MustSetInner(&testPutRespCopy) } } - if args, ok := ba.GetArg(roachpb.EndTxn); ok { - et := args.(*roachpb.EndTxnRequest) + if args, ok := ba.GetArg(kvpb.EndTxn); ok { + et := args.(*kvpb.EndTxnRequest) if et.Commit { status = roachpb.COMMITTED } else { @@ -142,7 +143,7 @@ func TestInitPut(t *testing.T) { // TODO(vivekmenezes): update test or remove when InitPut is being // considered sufficiently tested and this path exercised. clock := hlc.NewClockForTesting(nil) - db := NewDB(log.MakeTestingAmbientCtxWithNewTracer(), newTestTxnFactory(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + db := NewDB(log.MakeTestingAmbientCtxWithNewTracer(), newTestTxnFactory(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { br := ba.CreateReply() return br, nil }), clock, stopper) @@ -172,17 +173,17 @@ func TestTransactionConfig(t *testing.T) { for _, tc := range []struct { label string txnCreator func(context.Context, func(context.Context, *Txn) error) error - wantAdmissionHeader roachpb.AdmissionHeader_Source + wantAdmissionHeader kvpb.AdmissionHeader_Source }{ { label: "source is other", txnCreator: db.Txn, - wantAdmissionHeader: roachpb.AdmissionHeader_OTHER, + wantAdmissionHeader: kvpb.AdmissionHeader_OTHER, }, { label: "source is root kv", txnCreator: db.TxnRootKV, - wantAdmissionHeader: roachpb.AdmissionHeader_ROOT_KV, + wantAdmissionHeader: kvpb.AdmissionHeader_ROOT_KV, }, } { if err := tc.txnCreator(context.Background(), func(ctx context.Context, txn *Txn) error { @@ -211,7 +212,7 @@ func TestCommitTransactionOnce(t *testing.T) { defer stopper.Stop(ctx) clock := hlc.NewClockForTesting(nil) count := 0 - db := NewDB(log.MakeTestingAmbientCtxWithNewTracer(), newTestTxnFactory(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + db := NewDB(log.MakeTestingAmbientCtxWithNewTracer(), newTestTxnFactory(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { count++ return ba.CreateReply(), nil }), clock, stopper) @@ -236,10 +237,10 @@ func TestAbortMutatingTransaction(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(ctx) clock := hlc.NewClockForTesting(nil) - var calls []roachpb.Method - db := NewDB(log.MakeTestingAmbientCtxWithNewTracer(), newTestTxnFactory(func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + var calls []kvpb.Method + db := NewDB(log.MakeTestingAmbientCtxWithNewTracer(), newTestTxnFactory(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { calls = append(calls, ba.Methods()...) - if et, ok := ba.GetArg(roachpb.EndTxn); ok && et.(*roachpb.EndTxnRequest).Commit { + if et, ok := ba.GetArg(kvpb.EndTxn); ok && et.(*kvpb.EndTxnRequest).Commit { t.Errorf("expected commit to be false") } return ba.CreateReply(), nil @@ -253,7 +254,7 @@ func TestAbortMutatingTransaction(t *testing.T) { }); err == nil { t.Error("expected error on abort") } - expectedCalls := []roachpb.Method{roachpb.Put, roachpb.EndTxn} + expectedCalls := []kvpb.Method{kvpb.Put, kvpb.EndTxn} if !reflect.DeepEqual(expectedCalls, calls) { t.Errorf("expected %s, got %s", expectedCalls, calls) } @@ -273,14 +274,14 @@ func TestRunTransactionRetryOnErrors(t *testing.T) { err error retry bool // Expect retry? }{ - {roachpb.NewReadWithinUncertaintyIntervalError(hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, hlc.Timestamp{}, hlc.ClockTimestamp{}), true}, - {&roachpb.TransactionAbortedError{}, true}, - {&roachpb.TransactionPushError{}, true}, - {&roachpb.TransactionRetryError{}, true}, - {&roachpb.WriteTooOldError{}, true}, - {&roachpb.RangeNotFoundError{}, false}, - {&roachpb.RangeKeyMismatchError{}, false}, - {&roachpb.TransactionStatusError{}, false}, + {kvpb.NewReadWithinUncertaintyIntervalError(hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, hlc.Timestamp{}, hlc.ClockTimestamp{}), true}, + {&kvpb.TransactionAbortedError{}, true}, + {&kvpb.TransactionPushError{}, true}, + {&kvpb.TransactionRetryError{}, true}, + {&kvpb.WriteTooOldError{}, true}, + {&kvpb.RangeNotFoundError{}, false}, + {&kvpb.RangeKeyMismatchError{}, false}, + {&kvpb.TransactionStatusError{}, false}, } for _, test := range testCases { @@ -290,26 +291,26 @@ func TestRunTransactionRetryOnErrors(t *testing.T) { defer stopper.Stop(ctx) count := 0 db := NewDB(log.MakeTestingAmbientCtxWithNewTracer(), newTestTxnFactory( - func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { - if _, ok := ba.GetArg(roachpb.Put); ok { + if _, ok := ba.GetArg(kvpb.Put); ok { count++ if count == 1 { - var pErr *roachpb.Error - if errors.HasType(test.err, (*roachpb.ReadWithinUncertaintyIntervalError)(nil)) { + var pErr *kvpb.Error + if errors.HasType(test.err, (*kvpb.ReadWithinUncertaintyIntervalError)(nil)) { // This error requires an observed timestamp to have been // recorded on the origin node. ba.Txn.UpdateObservedTimestamp(1, hlc.ClockTimestamp{WallTime: 1, Logical: 1}) - pErr = roachpb.NewErrorWithTxn(test.err, ba.Txn) + pErr = kvpb.NewErrorWithTxn(test.err, ba.Txn) pErr.OriginNode = 1 } else { - pErr = roachpb.NewErrorWithTxn(test.err, ba.Txn) + pErr = kvpb.NewErrorWithTxn(test.err, ba.Txn) } - if pErr.TransactionRestart() != roachpb.TransactionRestart_NONE { + if pErr.TransactionRestart() != kvpb.TransactionRestart_NONE { // HACK ALERT: to do without a TxnCoordSender, we jump through // hoops to get the retryable error expected by db.Txn(). - return nil, roachpb.NewError(roachpb.NewTransactionRetryWithProtoRefreshError( + return nil, kvpb.NewError(kvpb.NewTransactionRetryWithProtoRefreshError( "foo", ba.Txn.ID, *ba.Txn)) } return nil, pErr @@ -413,14 +414,14 @@ func TestSetPriority(t *testing.T) { clock := hlc.NewClockForTesting(nil) var expected roachpb.UserPriority db := NewDB(log.MakeTestingAmbientCtxWithNewTracer(), newTestTxnFactory( - func(ba *roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) { if ba.UserPriority != expected { - pErr := roachpb.NewErrorf("Priority not set correctly in the batch! "+ + pErr := kvpb.NewErrorf("Priority not set correctly in the batch! "+ "(expected: %s, value: %s)", expected, ba.UserPriority) return nil, pErr } - br := &roachpb.BatchResponse{} + br := &kvpb.BatchResponse{} br.Txn.Update(ba.Txn) // copy return br, nil }), clock, stopper) @@ -431,7 +432,7 @@ func TestSetPriority(t *testing.T) { if err := txn.SetUserPriority(expected); err != nil { t.Fatal(err) } - if _, pErr := txn.Send(ctx, &roachpb.BatchRequest{}); pErr != nil { + if _, pErr := txn.Send(ctx, &kvpb.BatchRequest{}); pErr != nil { t.Fatal(pErr) } @@ -439,7 +440,7 @@ func TestSetPriority(t *testing.T) { expected = roachpb.UserPriority(-13) txn = NewTxn(ctx, db, 0 /* gatewayNodeID */) txn.TestingSetPriority(13) - if _, pErr := txn.Send(ctx, &roachpb.BatchRequest{}); pErr != nil { + if _, pErr := txn.Send(ctx, &kvpb.BatchRequest{}); pErr != nil { t.Fatal(pErr) } } @@ -464,7 +465,7 @@ func TestWrongTxnRetry(t *testing.T) { t.Fatal(err) } // Simulate an inner txn by generating an error with a bogus txn id. - return roachpb.NewTransactionRetryWithProtoRefreshError("test error", uuid.MakeV4(), roachpb.Transaction{}) + return kvpb.NewTransactionRetryWithProtoRefreshError("test error", uuid.MakeV4(), roachpb.Transaction{}) } if err := db.Txn(context.Background(), txnClosure); !testutils.IsError(err, "test error") { @@ -486,7 +487,7 @@ func TestBatchMixRawRequest(t *testing.T) { db := NewDB(log.MakeTestingAmbientCtxWithNewTracer(), newTestTxnFactory(nil), clock, stopper) b := &Batch{} - b.AddRawRequest(&roachpb.EndTxnRequest{}) + b.AddRawRequest(&kvpb.EndTxnRequest{}) b.Put("x", "y") if err := db.Run(context.Background(), b); !testutils.IsError(err, "non-raw operations") { t.Fatal(err) @@ -502,8 +503,8 @@ func TestUpdateDeadlineMaybe(t *testing.T) { clock := hlc.NewClockForTesting(timeutil.NewManualTime(timeutil.Unix(0, 1))) db := NewDB(log.MakeTestingAmbientCtxWithNewTracer(), MakeMockTxnSenderFactory( - func(context.Context, *roachpb.Transaction, *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, *roachpb.Error) { + func(context.Context, *roachpb.Transaction, *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, *kvpb.Error) { return nil, nil }), clock, stopper) txn := NewTxn(ctx, db, 0 /* gatewayNodeID */) @@ -551,15 +552,15 @@ func TestTxnNegotiateAndSend(t *testing.T) { ts20 := hlc.Timestamp{WallTime: 20} clock := hlc.NewClockForTesting(timeutil.NewManualTime(timeutil.Unix(0, 1))) txnSender := MakeMockTxnSenderFactoryWithNonTxnSender(nil /* senderFunc */, func( - _ context.Context, ba *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, *roachpb.Error) { + _ context.Context, ba *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, *kvpb.Error) { require.NotNil(t, ba.BoundedStaleness) require.Equal(t, ts10, ba.BoundedStaleness.MinTimestampBound) require.False(t, ba.BoundedStaleness.MinTimestampBoundStrict) require.Zero(t, ba.BoundedStaleness.MaxTimestampBound) if !fastPath { - return nil, roachpb.NewError(&roachpb.OpRequiresTxnError{}) + return nil, kvpb.NewError(&kvpb.OpRequiresTxnError{}) } br := ba.CreateReply() br.Timestamp = ts20 @@ -568,12 +569,12 @@ func TestTxnNegotiateAndSend(t *testing.T) { db := NewDB(log.MakeTestingAmbientCtxWithNewTracer(), txnSender, clock, stopper) txn := NewTxn(ctx, db, 0 /* gatewayNodeID */) - ba := &roachpb.BatchRequest{} - ba.BoundedStaleness = &roachpb.BoundedStalenessHeader{ + ba := &kvpb.BatchRequest{} + ba.BoundedStaleness = &kvpb.BoundedStalenessHeader{ MinTimestampBound: ts10, } - ba.RoutingPolicy = roachpb.RoutingPolicy_NEAREST - ba.Add(roachpb.NewGet(roachpb.Key("a"), false)) + ba.RoutingPolicy = kvpb.RoutingPolicy_NEAREST + ba.Add(kvpb.NewGet(roachpb.Key("a"), false)) br, pErr := txn.NegotiateAndSend(ctx, ba) if fastPath { @@ -663,8 +664,8 @@ func TestTxnNegotiateAndSendWithDeadline(t *testing.T) { t.Run(test.name, func(t *testing.T) { clock := hlc.NewClockForTesting(timeutil.NewManualTime(timeutil.Unix(0, 1))) txnSender := MakeMockTxnSenderFactoryWithNonTxnSender(nil /* senderFunc */, func( - _ context.Context, ba *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, *roachpb.Error) { + _ context.Context, ba *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, *kvpb.Error) { require.NotNil(t, ba.BoundedStaleness) require.Equal(t, minTSBound, ba.BoundedStaleness.MinTimestampBound) require.False(t, ba.BoundedStaleness.MinTimestampBoundStrict) @@ -678,13 +679,13 @@ func TestTxnNegotiateAndSendWithDeadline(t *testing.T) { txn := NewTxn(ctx, db, 0 /* gatewayNodeID */) require.NoError(t, txn.UpdateDeadline(ctx, test.txnDeadline)) - ba := &roachpb.BatchRequest{} - ba.BoundedStaleness = &roachpb.BoundedStalenessHeader{ + ba := &kvpb.BatchRequest{} + ba.BoundedStaleness = &kvpb.BoundedStalenessHeader{ MinTimestampBound: minTSBound, MaxTimestampBound: test.maxTSBound, } - ba.RoutingPolicy = roachpb.RoutingPolicy_NEAREST - ba.Add(roachpb.NewGet(roachpb.Key("a"), false)) + ba.RoutingPolicy = kvpb.RoutingPolicy_NEAREST + ba.Add(kvpb.NewGet(roachpb.Key("a"), false)) br, pErr := txn.NegotiateAndSend(ctx, ba) if test.expErr == "" { @@ -720,8 +721,8 @@ func TestTxnNegotiateAndSendWithResumeSpan(t *testing.T) { ts20 := hlc.Timestamp{WallTime: 20} clock := hlc.NewClockForTesting(timeutil.NewManualTime(timeutil.Unix(0, 1))) txnSender := MakeMockTxnSenderFactoryWithNonTxnSender(nil /* senderFunc */, func( - _ context.Context, ba *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, *roachpb.Error) { + _ context.Context, ba *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, *kvpb.Error) { require.NotNil(t, ba.BoundedStaleness) require.Equal(t, ts10, ba.BoundedStaleness.MinTimestampBound) require.False(t, ba.BoundedStaleness.MinTimestampBoundStrict) @@ -729,7 +730,7 @@ func TestTxnNegotiateAndSendWithResumeSpan(t *testing.T) { require.Equal(t, int64(2), ba.MaxSpanRequestKeys) if !fastPath { - return nil, roachpb.NewError(&roachpb.OpRequiresTxnError{}) + return nil, kvpb.NewError(&kvpb.OpRequiresTxnError{}) } br := ba.CreateReply() br.Timestamp = ts20 @@ -742,19 +743,19 @@ func TestTxnNegotiateAndSendWithResumeSpan(t *testing.T) { Key: roachpb.Key("c"), EndKey: roachpb.Key("d"), } - scanResp.ResumeReason = roachpb.RESUME_KEY_LIMIT + scanResp.ResumeReason = kvpb.RESUME_KEY_LIMIT return br, nil }) db := NewDB(log.MakeTestingAmbientCtxWithNewTracer(), txnSender, clock, stopper) txn := NewTxn(ctx, db, 0 /* gatewayNodeID */) - ba := &roachpb.BatchRequest{} - ba.BoundedStaleness = &roachpb.BoundedStalenessHeader{ + ba := &kvpb.BatchRequest{} + ba.BoundedStaleness = &kvpb.BoundedStalenessHeader{ MinTimestampBound: ts10, } - ba.RoutingPolicy = roachpb.RoutingPolicy_NEAREST + ba.RoutingPolicy = kvpb.RoutingPolicy_NEAREST ba.MaxSpanRequestKeys = 2 - ba.Add(roachpb.NewScan(roachpb.Key("a"), roachpb.Key("d"), false /* forUpdate */)) + ba.Add(kvpb.NewScan(roachpb.Key("a"), roachpb.Key("d"), false /* forUpdate */)) br, pErr := txn.NegotiateAndSend(ctx, ba) if fastPath { @@ -771,7 +772,7 @@ func TestTxnNegotiateAndSendWithResumeSpan(t *testing.T) { require.NotNil(t, scanResp.ResumeSpan) require.Equal(t, roachpb.Key("c"), scanResp.ResumeSpan.Key) require.Equal(t, roachpb.Key("d"), scanResp.ResumeSpan.EndKey) - require.Equal(t, roachpb.RESUME_KEY_LIMIT, scanResp.ResumeReason) + require.Equal(t, kvpb.RESUME_KEY_LIMIT, scanResp.ResumeReason) } else { require.Nil(t, br) require.NotNil(t, pErr) diff --git a/pkg/multitenant/BUILD.bazel b/pkg/multitenant/BUILD.bazel index bd57fd4d507d..f048c1cbfbb6 100644 --- a/pkg/multitenant/BUILD.bazel +++ b/pkg/multitenant/BUILD.bazel @@ -13,6 +13,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/base", + "//pkg/kv/kvpb", "//pkg/multitenant/tenantcostmodel", "//pkg/roachpb", "//pkg/settings", diff --git a/pkg/multitenant/mtinfopb/BUILD.bazel b/pkg/multitenant/mtinfopb/BUILD.bazel index 1854e02a7fbe..4e88374d63f7 100644 --- a/pkg/multitenant/mtinfopb/BUILD.bazel +++ b/pkg/multitenant/mtinfopb/BUILD.bazel @@ -18,8 +18,8 @@ proto_library( strip_import_prefix = "/pkg", visibility = ["//visibility:public"], deps = [ + "//pkg/kv/kvpb:kvpb_proto", "//pkg/multitenant/tenantcapabilities/tenantcapabilitiespb:tenantcapabilitiespb_proto", - "//pkg/roachpb:roachpb_proto", "@com_github_gogo_protobuf//gogoproto:gogo_proto", ], ) @@ -31,8 +31,9 @@ go_proto_library( proto = ":mtinfopb_proto", visibility = ["//visibility:public"], deps = [ + "//pkg/kv/kvpb", "//pkg/multitenant/tenantcapabilities/tenantcapabilitiespb", - "//pkg/roachpb", + "//pkg/roachpb", # keep "@com_github_gogo_protobuf//gogoproto", ], ) diff --git a/pkg/multitenant/mtinfopb/info.proto b/pkg/multitenant/mtinfopb/info.proto index 5bbd0607900c..4eda90ca8491 100644 --- a/pkg/multitenant/mtinfopb/info.proto +++ b/pkg/multitenant/mtinfopb/info.proto @@ -13,7 +13,7 @@ package cockroach.multitenant; option go_package = "mtinfopb"; import "gogoproto/gogo.proto"; -import "roachpb/api.proto"; +import "kv/kvpb/api.proto"; import "multitenant/tenantcapabilities/tenantcapabilitiespb/capabilities.proto"; // ProtoInfo represents the metadata for a tenant as diff --git a/pkg/multitenant/tenant_usage.go b/pkg/multitenant/tenant_usage.go index c49a2f90ae71..3cbed6fac7f1 100644 --- a/pkg/multitenant/tenant_usage.go +++ b/pkg/multitenant/tenant_usage.go @@ -14,6 +14,7 @@ import ( "context" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/isql" "github.com/cockroachdb/cockroach/pkg/util/metric" @@ -27,8 +28,8 @@ type TenantUsageServer interface { // service. Used to to service requests coming from tenants (through the // kvtenant.Connector) TokenBucketRequest( - ctx context.Context, tenantID roachpb.TenantID, in *roachpb.TokenBucketRequest, - ) *roachpb.TokenBucketResponse + ctx context.Context, tenantID roachpb.TenantID, in *kvpb.TokenBucketRequest, + ) *kvpb.TokenBucketResponse // ReconfigureTokenBucket updates a tenant's token bucket settings. // diff --git a/pkg/multitenant/tenantcapabilities/BUILD.bazel b/pkg/multitenant/tenantcapabilities/BUILD.bazel index ef9c025ee0b8..016c42198360 100644 --- a/pkg/multitenant/tenantcapabilities/BUILD.bazel +++ b/pkg/multitenant/tenantcapabilities/BUILD.bazel @@ -11,6 +11,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/base", + "//pkg/kv/kvpb", "//pkg/multitenant/tenantcapabilities/tenantcapabilitiespb", "//pkg/roachpb", ], diff --git a/pkg/multitenant/tenantcapabilities/capabilities.go b/pkg/multitenant/tenantcapabilities/capabilities.go index b4cb0812adad..5dcdf9192638 100644 --- a/pkg/multitenant/tenantcapabilities/capabilities.go +++ b/pkg/multitenant/tenantcapabilities/capabilities.go @@ -14,6 +14,7 @@ import ( "context" "fmt" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitiespb" "github.com/cockroachdb/cockroach/pkg/roachpb" ) @@ -45,7 +46,7 @@ type Authorizer interface { // HasCapabilityForBatch returns an error if a tenant, referenced by its ID, // is not allowed to execute the supplied batch request given the capabilities // it possesses. - HasCapabilityForBatch(context.Context, roachpb.TenantID, *roachpb.BatchRequest) error + HasCapabilityForBatch(context.Context, roachpb.TenantID, *kvpb.BatchRequest) error // BindReader is a mechanism by which the caller can bind a Reader[1] to the // Authorizer post-creation. The Authorizer uses the Reader to consult the diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/BUILD.bazel b/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/BUILD.bazel index b029594cada5..c305e9f64e8b 100644 --- a/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/BUILD.bazel +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/BUILD.bazel @@ -10,6 +10,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer", visibility = ["//visibility:public"], deps = [ + "//pkg/kv/kvpb", "//pkg/multitenant/tenantcapabilities", "//pkg/roachpb", "//pkg/settings/cluster", diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/authorizer.go b/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/authorizer.go index 02453f680080..decaeec35169 100644 --- a/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/authorizer.go +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/authorizer.go @@ -13,6 +13,7 @@ package tenantcapabilitiesauthorizer import ( "context" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -48,7 +49,7 @@ func New(settings *cluster.Settings, knobs *tenantcapabilities.TestingKnobs) *Au // HasCapabilityForBatch implements the tenantcapabilities.Authorizer interface. func (a *Authorizer) HasCapabilityForBatch( - ctx context.Context, tenID roachpb.TenantID, ba *roachpb.BatchRequest, + ctx context.Context, tenID roachpb.TenantID, ba *kvpb.BatchRequest, ) error { if tenID.IsSystem() { return nil // the system tenant is allowed to do as it pleases @@ -70,7 +71,7 @@ func (a *Authorizer) HasCapabilityForBatch( for _, ru := range ba.Requests { switch ru.GetInner().(type) { - case *roachpb.AdminSplitRequest: + case *kvpb.AdminSplitRequest: if !cp.CanAdminSplit && !a.knobs.AuthorizerSkipAdminSplitCapabilityChecks { return errors.Newf("tenant %s does not have admin split capability", tenID) } diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/noop.go b/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/noop.go index 7b1f7844462f..c180d65325ce 100644 --- a/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/noop.go +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/noop.go @@ -13,6 +13,7 @@ package tenantcapabilitiesauthorizer import ( "context" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities" "github.com/cockroachdb/cockroach/pkg/roachpb" ) @@ -29,7 +30,7 @@ func NewNoopAuthorizer() *NoopAuthorizer { // HasCapabilityForBatch implements the tenantcapabilities.Authorizer interface. func (n *NoopAuthorizer) HasCapabilityForBatch( - context.Context, roachpb.TenantID, *roachpb.BatchRequest, + context.Context, roachpb.TenantID, *kvpb.BatchRequest, ) error { return nil } diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitiestestutils/BUILD.bazel b/pkg/multitenant/tenantcapabilities/tenantcapabilitiestestutils/BUILD.bazel index 73f61b9e374a..700cbe0c38dd 100644 --- a/pkg/multitenant/tenantcapabilities/tenantcapabilitiestestutils/BUILD.bazel +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitiestestutils/BUILD.bazel @@ -7,6 +7,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitiestestutils", visibility = ["//visibility:public"], deps = [ + "//pkg/kv/kvpb", "//pkg/multitenant/tenantcapabilities", "//pkg/multitenant/tenantcapabilities/tenantcapabilitiespb", "//pkg/roachpb", diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitiestestutils/testutils.go b/pkg/multitenant/tenantcapabilities/tenantcapabilitiestestutils/testutils.go index 578b67ef1435..eca1baaf0e85 100644 --- a/pkg/multitenant/tenantcapabilities/tenantcapabilitiestestutils/testutils.go +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitiestestutils/testutils.go @@ -14,6 +14,7 @@ import ( "strconv" "testing" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitiespb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -27,17 +28,17 @@ import ( // form: // // cmds=(split, scan, cput) -func ParseBatchRequests(t *testing.T, d *datadriven.TestData) (ba roachpb.BatchRequest) { +func ParseBatchRequests(t *testing.T, d *datadriven.TestData) (ba kvpb.BatchRequest) { for _, cmd := range d.CmdArgs { if cmd.Key == "cmds" { for _, z := range cmd.Vals { switch z { case "split": - ba.Add(&roachpb.AdminSplitRequest{}) + ba.Add(&kvpb.AdminSplitRequest{}) case "scan": - ba.Add(&roachpb.ScanRequest{}) + ba.Add(&kvpb.ScanRequest{}) case "cput": - ba.Add(&roachpb.ConditionalPutRequest{}) + ba.Add(&kvpb.ConditionalPutRequest{}) default: t.Fatalf("unsupported request type: %s", z) } diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/BUILD.bazel b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/BUILD.bazel index 016bf17cab35..3913380c8056 100644 --- a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/BUILD.bazel +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/BUILD.bazel @@ -16,6 +16,7 @@ go_library( "//pkg/kv/kvclient/rangefeed", "//pkg/kv/kvclient/rangefeed/rangefeedbuffer", "//pkg/kv/kvclient/rangefeed/rangefeedcache", + "//pkg/kv/kvpb", "//pkg/multitenant/mtinfopb", "//pkg/multitenant/tenantcapabilities", "//pkg/multitenant/tenantcapabilities/tenantcapabilitiespb", diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/decoder.go b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/decoder.go index 705cccd0b45e..194f5c457940 100644 --- a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/decoder.go +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/decoder.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedbuffer" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -95,7 +96,7 @@ func (d *decoder) decode(kv roachpb.KeyValue) (tenantcapabilities.Entry, error) } func (d *decoder) translateEvent( - ctx context.Context, ev *roachpb.RangeFeedValue, + ctx context.Context, ev *kvpb.RangeFeedValue, ) rangefeedbuffer.Event { deleted := !ev.Value.IsPresent() var value roachpb.Value diff --git a/pkg/multitenant/tenantcostmodel/BUILD.bazel b/pkg/multitenant/tenantcostmodel/BUILD.bazel index 5da27ba040c6..9d4fdb795675 100644 --- a/pkg/multitenant/tenantcostmodel/BUILD.bazel +++ b/pkg/multitenant/tenantcostmodel/BUILD.bazel @@ -10,7 +10,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcostmodel", visibility = ["//visibility:public"], deps = [ - "//pkg/roachpb", + "//pkg/kv/kvpb", "//pkg/settings", "//pkg/util/log", "@com_github_cockroachdb_errors//:errors", diff --git a/pkg/multitenant/tenantcostmodel/model.go b/pkg/multitenant/tenantcostmodel/model.go index 19435f1687f0..4e15512eef0e 100644 --- a/pkg/multitenant/tenantcostmodel/model.go +++ b/pkg/multitenant/tenantcostmodel/model.go @@ -10,7 +10,7 @@ package tenantcostmodel -import "github.com/cockroachdb/cockroach/pkg/roachpb" +import "github.com/cockroachdb/cockroach/pkg/kv/kvpb" // RU stands for "Request Unit(s)"; the tenant cost model maps tenant activity // into this abstract unit. @@ -179,7 +179,7 @@ type RequestInfo struct { // MakeRequestInfo extracts the relevant information from a BatchRequest. func MakeRequestInfo( - ba *roachpb.BatchRequest, replicas int, writeMultiplier RUMultiplier, + ba *kvpb.BatchRequest, replicas int, writeMultiplier RUMultiplier, ) RequestInfo { // The cost of read-only batches is captured by MakeResponseInfo. if !ba.IsWrite() { @@ -193,11 +193,11 @@ func MakeRequestInfo( // Only count non-admin requests in the batch that write user data. Other // requests are considered part of the "base" cost of a batch. switch req.(type) { - case *roachpb.PutRequest, *roachpb.ConditionalPutRequest, *roachpb.IncrementRequest, - *roachpb.DeleteRequest, *roachpb.DeleteRangeRequest, *roachpb.ClearRangeRequest, - *roachpb.RevertRangeRequest, *roachpb.InitPutRequest, *roachpb.AddSSTableRequest: + case *kvpb.PutRequest, *kvpb.ConditionalPutRequest, *kvpb.IncrementRequest, + *kvpb.DeleteRequest, *kvpb.DeleteRangeRequest, *kvpb.ClearRangeRequest, + *kvpb.RevertRangeRequest, *kvpb.InitPutRequest, *kvpb.AddSSTableRequest: writeCount++ - if swr, isSizedWrite := req.(roachpb.SizedWriteRequest); isSizedWrite { + if swr, isSizedWrite := req.(kvpb.SizedWriteRequest); isSizedWrite { writeBytes += swr.WriteBytes() } } @@ -268,7 +268,7 @@ type ResponseInfo struct { // MakeResponseInfo extracts the relevant information from a BatchResponse. func MakeResponseInfo( - br *roachpb.BatchResponse, isReadOnly bool, readMultiplier RUMultiplier, + br *kvpb.BatchResponse, isReadOnly bool, readMultiplier RUMultiplier, ) ResponseInfo { // The cost of non read-only batches is captured by MakeRequestInfo. if !isReadOnly { @@ -282,8 +282,8 @@ func MakeResponseInfo( // Only count requests in the batch that read user data. Other requests // are considered part of the "base" cost of a batch. switch resp.(type) { - case *roachpb.GetResponse, *roachpb.ScanResponse, *roachpb.ReverseScanResponse, - *roachpb.ExportResponse: + case *kvpb.GetResponse, *kvpb.ScanResponse, *kvpb.ReverseScanResponse, + *kvpb.ExportResponse: readCount++ readBytes += resp.Header().NumBytes } diff --git a/pkg/repstream/streampb/BUILD.bazel b/pkg/repstream/streampb/BUILD.bazel index 80184f78b2f1..9b56c732c014 100644 --- a/pkg/repstream/streampb/BUILD.bazel +++ b/pkg/repstream/streampb/BUILD.bazel @@ -10,6 +10,7 @@ proto_library( visibility = ["//visibility:public"], deps = [ "//pkg/jobs/jobspb:jobspb_proto", + "//pkg/kv/kvpb:kvpb_proto", "//pkg/roachpb:roachpb_proto", "//pkg/util:util_proto", "//pkg/util/hlc:hlc_proto", @@ -26,6 +27,7 @@ go_proto_library( visibility = ["//visibility:public"], deps = [ "//pkg/jobs/jobspb", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/util", "//pkg/util/hlc", diff --git a/pkg/repstream/streampb/stream.proto b/pkg/repstream/streampb/stream.proto index 1ee9376cf74a..2e9d9c0e02c5 100644 --- a/pkg/repstream/streampb/stream.proto +++ b/pkg/repstream/streampb/stream.proto @@ -14,7 +14,7 @@ package cockroach.repstream.streampb; option go_package = "streampb"; -import "roachpb/api.proto"; +import "kv/kvpb/api.proto"; import "roachpb/data.proto"; import "jobs/jobspb/jobs.proto"; import "roachpb/metadata.proto"; diff --git a/pkg/roachpb/.gitattributes b/pkg/roachpb/.gitattributes index f39d346a703f..29675c0bf166 100644 --- a/pkg/roachpb/.gitattributes +++ b/pkg/roachpb/.gitattributes @@ -1,2 +1 @@ -batch_generated.go -diff *_string.go -diff diff --git a/pkg/roachpb/BUILD.bazel b/pkg/roachpb/BUILD.bazel index 300e22199c7b..1138f9b0b05d 100644 --- a/pkg/roachpb/BUILD.bazel +++ b/pkg/roachpb/BUILD.bazel @@ -2,33 +2,21 @@ load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data") load("@rules_proto//proto:defs.bzl", "proto_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//build:STRINGER.bzl", "stringer") load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") -load(":gen.bzl", "batch_gen") go_library( name = "roachpb", srcs = [ - "ambiguous_result_error.go", - "api.go", - "api_requestheader.go", - "batch.go", "data.go", - "errors.go", "index_usage_stats.go", "internal.go", "merge_spans.go", "metadata.go", "metadata_replicas.go", - "method.go", - "replica_unavailable_error.go", "span_config.go", "span_group.go", "tenant.go", "version.go", - ":gen-batch-generated", # keep - ":gen-errordetailtype-stringer", # keep - ":gen-method-stringer", # keep ], embed = [":roachpb_go_proto"], importpath = "github.com/cockroachdb/cockroach/pkg/roachpb", @@ -37,14 +25,11 @@ go_library( "//pkg/base/serverident", "//pkg/geo/geopb", "//pkg/keysbase", - "//pkg/kv/kvnemesis/kvnemesisutil", "//pkg/kv/kvserver/allocator/load", "//pkg/kv/kvserver/concurrency/lock", "//pkg/storage/enginepb", "//pkg/util", "//pkg/util/bitarray", - "//pkg/util/buildutil", - "//pkg/util/caller", "//pkg/util/duration", "//pkg/util/encoding", "//pkg/util/hlc", @@ -57,10 +42,7 @@ go_library( "//pkg/util/uuid", "@com_github_cockroachdb_apd_v3//:apd", "@com_github_cockroachdb_errors//:errors", - "@com_github_cockroachdb_errors//extgrpc", "@com_github_cockroachdb_redact//:redact", - "@com_github_dustin_go_humanize//:go-humanize", - "@com_github_gogo_protobuf//proto", "@com_github_golang_mock//gomock", # keep "@io_etcd_go_raft_v3//raftpb", "@org_golang_google_grpc//metadata", # keep @@ -71,16 +53,12 @@ go_test( name = "roachpb_test", size = "small", srcs = [ - "api_test.go", - "batch_test.go", "data_test.go", - "errors_test.go", "index_usage_stats_test.go", "main_test.go", "merge_spans_test.go", "metadata_replicas_test.go", "metadata_test.go", - "replica_unavailable_error_test.go", "span_config_conformance_report_test.go", "span_group_test.go", "string_test.go", @@ -88,20 +66,16 @@ go_test( "version_test.go", ], args = ["-test.timeout=55s"], - data = glob(["testdata/**"]), embed = [":roachpb"], tags = ["no-remote"], deps = [ "//pkg/cli/exit", "//pkg/keys", - "//pkg/kv/kvnemesis/kvnemesisutil", "//pkg/kv/kvserver/concurrency/lock", "//pkg/storage/enginepb", - "//pkg/testutils/echotest", "//pkg/testutils/zerofields", "//pkg/util", "//pkg/util/bitarray", - "//pkg/util/buildutil", "//pkg/util/duration", "//pkg/util/encoding", "//pkg/util/hlc", @@ -114,8 +88,6 @@ go_test( "@com_github_cockroachdb_apd_v3//:apd", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_redact//:redact", - "@com_github_gogo_protobuf//proto", - "@com_github_golang_protobuf//proto:go_default_library", "@com_github_kr_pretty//:pretty", "@com_github_stretchr_testify//assert", "@com_github_stretchr_testify//require", @@ -124,17 +96,13 @@ go_test( "@io_etcd_go_raft_v3//quorum", "@io_etcd_go_raft_v3//raftpb", "@io_etcd_go_raft_v3//tracker", - "@org_golang_google_grpc//codes", - "@org_golang_google_grpc//status", ], ) proto_library( name = "roachpb_proto", srcs = [ - "api.proto", "data.proto", - "errors.proto", "index_usage_stats.proto", "internal.proto", "internal_raft.proto", @@ -148,13 +116,10 @@ proto_library( deps = [ "//pkg/kv/kvserver/concurrency/lock:lock_proto", "//pkg/kv/kvserver/readsummary/rspb:rspb_proto", - "//pkg/settings:settings_proto", - "//pkg/sql/catalog/fetchpb:fetchpb_proto", "//pkg/storage/enginepb:enginepb_proto", "//pkg/util:util_proto", "//pkg/util/admission/admissionpb:admissionpb_proto", "//pkg/util/hlc:hlc_proto", - "//pkg/util/tracing/tracingpb:tracingpb_proto", "@com_github_cockroachdb_errors//errorspb:errorspb_proto", "@com_github_gogo_protobuf//gogoproto:gogo_proto", "@com_google_protobuf//:duration_proto", @@ -165,48 +130,23 @@ proto_library( go_proto_library( name = "roachpb_go_proto", - compilers = ["//pkg/cmd/protoc-gen-gogoroach:protoc-gen-gogoroach_grpc_compiler"], + compilers = ["//pkg/cmd/protoc-gen-gogoroach:protoc-gen-gogoroach_compiler"], importpath = "github.com/cockroachdb/cockroach/pkg/roachpb", proto = ":roachpb_proto", visibility = ["//visibility:public"], deps = [ "//pkg/kv/kvserver/concurrency/lock", "//pkg/kv/kvserver/readsummary/rspb", - "//pkg/settings", - "//pkg/sql/catalog/fetchpb", "//pkg/storage/enginepb", "//pkg/util", "//pkg/util/admission/admissionpb", "//pkg/util/hlc", - "//pkg/util/tracing/tracingpb", "@com_github_cockroachdb_errors//errorspb", "@com_github_gogo_protobuf//gogoproto", "@org_golang_google_genproto//googleapis/api/annotations:go_default_library", ], ) -stringer( - name = "gen-method-stringer", - src = "method.go", - typ = "Method", -) - -stringer( - name = "gen-errordetailtype-stringer", - src = "errors.go", - typ = "ErrorDetailType", -) - -batch_gen( - name = "gen-batch-generated", - src = ":roachpb_go_proto", - out = "batch_generated.go", - visibility = [ - ":__pkg__", - "//pkg/gen:__pkg__", - ], -) - disallowed_imports_test( "roachpb", disallow_cdeps = True, diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index 72ae6bf46a68..2325ffb89df4 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -1022,6 +1022,35 @@ func (t *Transaction) AssertInitialized(ctx context.Context) { } } +// UserPriority is a custom type for transaction's user priority. +type UserPriority float64 + +func (up UserPriority) String() string { + switch up { + case MinUserPriority: + return "low" + case UnspecifiedUserPriority, NormalUserPriority: + return "normal" + case MaxUserPriority: + return "high" + default: + return fmt.Sprintf("%g", float64(up)) + } +} + +const ( + // MinUserPriority is the minimum allowed user priority. + MinUserPriority UserPriority = 0.001 + // UnspecifiedUserPriority means NormalUserPriority. + UnspecifiedUserPriority UserPriority = 0 + // NormalUserPriority is set to 1, meaning ops run through the database + // are all given equal weight when a random priority is chosen. This can + // be set specifically via client.NewDBWithPriority(). + NormalUserPriority UserPriority = 1 + // MaxUserPriority is the maximum allowed user priority. + MaxUserPriority UserPriority = 1000 +) + // MakePriority generates a random priority value, biased by the specified // userPriority. If userPriority=100, the random priority will be 100x more // likely to be greater than if userPriority=1. If userPriority = 0.1, the @@ -1423,132 +1452,6 @@ func (tr *TransactionRecord) AsTransaction() Transaction { return t } -// PrepareTransactionForRetry returns a new Transaction to be used for retrying -// the original Transaction. Depending on the error, this might return an -// already-existing Transaction with an incremented epoch, or a completely new -// Transaction. -// -// The caller should generally check that the error was meant for this -// Transaction before calling this. -// -// pri is the priority that should be used when giving the restarted transaction -// the chance to get a higher priority. Not used when the transaction is being -// aborted. -// -// In case retryErr tells us that a new Transaction needs to be created, -// isolation and name help initialize this new transaction. -func PrepareTransactionForRetry( - ctx context.Context, pErr *Error, pri UserPriority, clock *hlc.Clock, -) Transaction { - if pErr.TransactionRestart() == TransactionRestart_NONE { - log.Fatalf(ctx, "invalid retryable err (%T): %s", pErr.GetDetail(), pErr) - } - - if pErr.GetTxn() == nil { - log.Fatalf(ctx, "missing txn for retryable error: %s", pErr) - } - - txn := *pErr.GetTxn() - aborted := false - switch tErr := pErr.GetDetail().(type) { - case *TransactionAbortedError: - // The txn coming with a TransactionAbortedError is not supposed to be used - // for the restart. Instead, a brand new transaction is created. - aborted = true - // TODO(andrei): Should we preserve the ObservedTimestamps across the - // restart? - errTxnPri := txn.Priority - // Start the new transaction at the current time from the local clock. - // The local hlc should have been advanced to at least the error's - // timestamp already. - now := clock.NowAsClockTimestamp() - txn = MakeTransaction( - txn.Name, - nil, // baseKey - // We have errTxnPri, but this wants a UserPriority. So we're going to - // overwrite the priority below. - NormalUserPriority, - now.ToTimestamp(), - clock.MaxOffset().Nanoseconds(), - txn.CoordinatorNodeID, - ) - // Use the priority communicated back by the server. - txn.Priority = errTxnPri - case *ReadWithinUncertaintyIntervalError: - txn.WriteTimestamp.Forward(tErr.RetryTimestamp()) - case *TransactionPushError: - // Increase timestamp if applicable, ensuring that we're just ahead of - // the pushee. - txn.WriteTimestamp.Forward(tErr.PusheeTxn.WriteTimestamp) - txn.UpgradePriority(tErr.PusheeTxn.Priority - 1) - case *TransactionRetryError: - // Transaction.Timestamp has already been forwarded to be ahead of any - // timestamp cache entries or newer versions which caused the restart. - if tErr.Reason == RETRY_SERIALIZABLE { - // For RETRY_SERIALIZABLE case, we want to bump timestamp further than - // timestamp cache. - // This helps transactions that had their commit timestamp fixed (See - // roachpb.Transaction.CommitTimestampFixed for details on when it happens) - // or transactions that hit read-write contention and can't bump - // read timestamp because of later writes. - // Upon retry, we want those transactions to restart on now() instead of - // closed ts to give them some time to complete without a need to refresh - // read spans yet again and possibly fail. - // The tradeoff here is that transactions that failed because they were - // waiting on locks or were slowed down in their first epoch for any other - // reason (e.g. lease transfers, network congestion, node failure, etc.) - // would have a chance to retry and succeed, but transactions that are - // just slow would still retry indefinitely and delay transactions that - // try to write to the keys this transaction reads because reads are not - // in the past anymore. - now := clock.Now() - txn.WriteTimestamp.Forward(now) - } - case *WriteTooOldError: - // Increase the timestamp to the ts at which we've actually written. - txn.WriteTimestamp.Forward(tErr.RetryTimestamp()) - default: - log.Fatalf(ctx, "invalid retryable err (%T): %s", pErr.GetDetail(), pErr) - } - if !aborted { - if txn.Status.IsFinalized() { - log.Fatalf(ctx, "transaction unexpectedly finalized in (%T): %s", pErr.GetDetail(), pErr) - } - txn.Restart(pri, txn.Priority, txn.WriteTimestamp) - } - return txn -} - -// TransactionRefreshTimestamp returns whether the supplied error is a retry -// error that can be discarded if the transaction in the error is refreshed. If -// true, the function returns the timestamp that the Transaction object should -// be refreshed at in order to discard the error and avoid a restart. -func TransactionRefreshTimestamp(pErr *Error) (bool, hlc.Timestamp) { - txn := pErr.GetTxn() - if txn == nil { - return false, hlc.Timestamp{} - } - timestamp := txn.WriteTimestamp - switch err := pErr.GetDetail().(type) { - case *TransactionRetryError: - if err.Reason != RETRY_SERIALIZABLE && err.Reason != RETRY_WRITE_TOO_OLD { - return false, hlc.Timestamp{} - } - case *WriteTooOldError: - // TODO(andrei): Chances of success for on write-too-old conditions might be - // usually small: if our txn previously read the key that generated this - // error, obviously the refresh will fail. It might be worth trying to - // detect these cases and save the futile attempt; we'd need to have access - // to the key that generated the error. - timestamp.Forward(err.RetryTimestamp()) - case *ReadWithinUncertaintyIntervalError: - timestamp.Forward(err.RetryTimestamp()) - default: - return false, hlc.Timestamp{} - } - return true, timestamp -} - // Replicas returns all of the replicas present in the descriptor after this // trigger applies. func (crt ChangeReplicasTrigger) Replicas() []ReplicaDescriptor { diff --git a/pkg/roachpb/data_test.go b/pkg/roachpb/data_test.go index f879c6bdb7ca..5af10d745da0 100644 --- a/pkg/roachpb/data_test.go +++ b/pkg/roachpb/data_test.go @@ -1251,6 +1251,13 @@ func TestSpanCombine(t *testing.T) { // or key range is contained within the span. func TestSpanContains(t *testing.T) { s := Span{Key: []byte("a"), EndKey: []byte("b")} + sp := func(start, end string) Span { + res := Span{Key: Key(start)} + if end != "" { + res.EndKey = Key(end) + } + return res + } testData := []struct { start, end string @@ -1317,8 +1324,8 @@ func TestSpanSplitOnKey(t *testing.T) { // Simple split. { []byte("bb"), - sp("b", "bb"), - sp("bb", "c"), + Span{Key: Key("b"), EndKey: Key("bb")}, + Span{Key: Key("bb"), EndKey: Key("c")}, }, } for testIdx, test := range testData { diff --git a/pkg/roachpb/string_test.go b/pkg/roachpb/string_test.go index 680c7e4e24cc..9a036535d06f 100644 --- a/pkg/roachpb/string_test.go +++ b/pkg/roachpb/string_test.go @@ -11,20 +11,14 @@ package roachpb_test import ( - "context" - "fmt" - "path/filepath" "testing" // Hook up the pretty printer. _ "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" - "github.com/cockroachdb/cockroach/pkg/testutils/echotest" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/uuid" - "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" "github.com/stretchr/testify/require" ) @@ -61,41 +55,6 @@ func TestTransactionString(t *testing.T) { } } -func TestBatchRequestString(t *testing.T) { - ba := roachpb.BatchRequest{} - txn := roachpb.MakeTransaction( - "test", - nil, // baseKey - roachpb.NormalUserPriority, - hlc.Timestamp{}, // now - 0, // maxOffsetNs - 99, // coordinatorNodeID - ) - txn.ID = uuid.NamespaceDNS - ba.Txn = &txn - ba.WaitPolicy = lock.WaitPolicy_Error - ba.CanForwardReadTimestamp = true - ba.BoundedStaleness = &roachpb.BoundedStalenessHeader{ - MinTimestampBound: hlc.Timestamp{WallTime: 1}, - MinTimestampBoundStrict: true, - MaxTimestampBound: hlc.Timestamp{WallTime: 2}, - } - for i := 0; i < 100; i++ { - var ru roachpb.RequestUnion - ru.MustSetInner(&roachpb.GetRequest{}) - ba.Requests = append(ba.Requests, ru) - } - var ru roachpb.RequestUnion - ru.MustSetInner(&roachpb.EndTxnRequest{}) - ba.Requests = append(ba.Requests, ru) - - { - exp := `Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min),... 76 skipped ..., Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), EndTxn(abort) [/Min], [txn: 6ba7b810], [wait-policy: Error], [can-forward-ts], [bounded-staleness, min_ts_bound: 0.000000001,0, min_ts_bound_strict, max_ts_bound: 0.000000002,0]` - act := ba.String() - require.Equal(t, exp, act) - } -} - func TestKeyString(t *testing.T) { require.Equal(t, `/Table/53/42/"=\xbc ⌘"`, @@ -141,33 +100,3 @@ func TestSpansString(t *testing.T) { require.Equal(t, tc.expected, tc.spans.String()) } } - -func TestReplicaUnavailableError(t *testing.T) { - ctx := context.Background() - rDesc := roachpb.ReplicaDescriptor{NodeID: 1, StoreID: 2, ReplicaID: 3} - var set roachpb.ReplicaSet - set.AddReplica(rDesc) - desc := roachpb.NewRangeDescriptor(123, roachpb.RKeyMin, roachpb.RKeyMax, set) - - errSlowProposal := errors.New("slow proposal") - var err = roachpb.NewReplicaUnavailableError(errSlowProposal, desc, rDesc) - err = errors.DecodeError(ctx, errors.EncodeError(ctx, err)) - // Sanity check that Unwrap() was implemented. - require.True(t, errors.Is(err, errSlowProposal), "%+v", err) - require.True(t, errors.HasType(err, (*roachpb.ReplicaUnavailableError)(nil)), "%+v", err) - - s := fmt.Sprintf("%s\n%s", err, redact.Sprint(err)) - echotest.Require(t, s, filepath.Join("testdata", "replica_unavailable_error.txt")) -} - -func TestAmbiguousResultError(t *testing.T) { - ctx := context.Background() - - wrapped := errors.Errorf("boom with a %s", redact.Unsafe("secret")) - var err error = roachpb.NewAmbiguousResultError(wrapped) - err = errors.DecodeError(ctx, errors.EncodeError(ctx, err)) - require.True(t, errors.Is(err, wrapped), "%+v", err) - - s := fmt.Sprintf("%s\n%s", err, redact.Sprint(err)) - echotest.Require(t, s, filepath.Join("testdata", "ambiguous_result_error.txt")) -} diff --git a/pkg/rpc/BUILD.bazel b/pkg/rpc/BUILD.bazel index 884006042a71..69e973b90522 100644 --- a/pkg/rpc/BUILD.bazel +++ b/pkg/rpc/BUILD.bazel @@ -29,6 +29,7 @@ go_library( "//pkg/base", "//pkg/clusterversion", "//pkg/keys", + "//pkg/kv/kvpb", "//pkg/multitenant/tenantcapabilities", "//pkg/roachpb", "//pkg/security", @@ -100,6 +101,7 @@ go_test( "//pkg/base", "//pkg/clusterversion", "//pkg/keys", + "//pkg/kv/kvpb", "//pkg/multitenant/tenantcapabilities", "//pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer", "//pkg/roachpb", diff --git a/pkg/rpc/auth_tenant.go b/pkg/rpc/auth_tenant.go index 0679366e2bda..1eb4c9aca893 100644 --- a/pkg/rpc/auth_tenant.go +++ b/pkg/rpc/auth_tenant.go @@ -15,6 +15,7 @@ import ( "strconv" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/errors" @@ -53,21 +54,21 @@ func (a tenantAuthorizer) authorize( ) error { switch fullMethod { case "/cockroach.roachpb.Internal/Batch": - return a.authBatch(ctx, tenID, req.(*roachpb.BatchRequest)) + return a.authBatch(ctx, tenID, req.(*kvpb.BatchRequest)) case "/cockroach.roachpb.Internal/RangeLookup": - return a.authRangeLookup(tenID, req.(*roachpb.RangeLookupRequest)) + return a.authRangeLookup(tenID, req.(*kvpb.RangeLookupRequest)) case "/cockroach.roachpb.Internal/RangeFeed", "/cockroach.roachpb.Internal/MuxRangeFeed": - return a.authRangeFeed(tenID, req.(*roachpb.RangeFeedRequest)) + return a.authRangeFeed(tenID, req.(*kvpb.RangeFeedRequest)) case "/cockroach.roachpb.Internal/GossipSubscription": - return a.authGossipSubscription(tenID, req.(*roachpb.GossipSubscriptionRequest)) + return a.authGossipSubscription(tenID, req.(*kvpb.GossipSubscriptionRequest)) case "/cockroach.roachpb.Internal/TokenBucket": - return a.authTokenBucket(tenID, req.(*roachpb.TokenBucketRequest)) + return a.authTokenBucket(tenID, req.(*kvpb.TokenBucketRequest)) case "/cockroach.roachpb.Internal/TenantSettings": - return a.authTenantSettings(tenID, req.(*roachpb.TenantSettingsRequest)) + return a.authTenantSettings(tenID, req.(*kvpb.TenantSettingsRequest)) case "/cockroach.rpc.Heartbeat/Ping": return nil // no restriction to usage of this endpoint by tenants @@ -136,7 +137,7 @@ func (a tenantAuthorizer) authorize( return a.authUpdateSpanConfigs(tenID, req.(*roachpb.UpdateSpanConfigsRequest)) case "/cockroach.roachpb.Internal/GetRangeDescriptors": - return a.authGetRangeDescriptors(tenID, req.(*roachpb.GetRangeDescriptorsRequest)) + return a.authGetRangeDescriptors(tenID, req.(*kvpb.GetRangeDescriptorsRequest)) case "/cockroach.server.serverpb.Status/HotRangesV2": return a.authHotRangesV2(tenID) @@ -165,7 +166,7 @@ func checkSpanBounds(rSpan, tenSpan roachpb.RSpan) error { // authBatch authorizes the provided tenant to invoke the Batch RPC with the // provided args. func (a tenantAuthorizer) authBatch( - ctx context.Context, tenID roachpb.TenantID, args *roachpb.BatchRequest, + ctx context.Context, tenID roachpb.TenantID, args *kvpb.BatchRequest, ) error { if err := a.capabilitiesAuthorizer.HasCapabilityForBatch(ctx, tenID, args); err != nil { return authError(err.Error()) @@ -178,35 +179,35 @@ func (a tenantAuthorizer) authBatch( for _, ru := range args.Requests { switch ru.GetInner().(type) { case - *roachpb.AddSSTableRequest, - *roachpb.AdminChangeReplicasRequest, - *roachpb.AdminRelocateRangeRequest, - *roachpb.AdminScatterRequest, - *roachpb.AdminSplitRequest, - *roachpb.AdminTransferLeaseRequest, - *roachpb.AdminUnsplitRequest, - *roachpb.ClearRangeRequest, - *roachpb.ConditionalPutRequest, - *roachpb.DeleteRangeRequest, - *roachpb.DeleteRequest, - *roachpb.EndTxnRequest, - *roachpb.ExportRequest, - *roachpb.GetRequest, - *roachpb.HeartbeatTxnRequest, - *roachpb.IncrementRequest, - *roachpb.InitPutRequest, - *roachpb.IsSpanEmptyRequest, - *roachpb.LeaseInfoRequest, - *roachpb.PutRequest, - *roachpb.QueryIntentRequest, - *roachpb.QueryLocksRequest, - *roachpb.QueryTxnRequest, - *roachpb.RangeStatsRequest, - *roachpb.RefreshRangeRequest, - *roachpb.RefreshRequest, - *roachpb.ReverseScanRequest, - *roachpb.RevertRangeRequest, - *roachpb.ScanRequest: + *kvpb.AddSSTableRequest, + *kvpb.AdminChangeReplicasRequest, + *kvpb.AdminRelocateRangeRequest, + *kvpb.AdminScatterRequest, + *kvpb.AdminSplitRequest, + *kvpb.AdminTransferLeaseRequest, + *kvpb.AdminUnsplitRequest, + *kvpb.ClearRangeRequest, + *kvpb.ConditionalPutRequest, + *kvpb.DeleteRangeRequest, + *kvpb.DeleteRequest, + *kvpb.EndTxnRequest, + *kvpb.ExportRequest, + *kvpb.GetRequest, + *kvpb.HeartbeatTxnRequest, + *kvpb.IncrementRequest, + *kvpb.InitPutRequest, + *kvpb.IsSpanEmptyRequest, + *kvpb.LeaseInfoRequest, + *kvpb.PutRequest, + *kvpb.QueryIntentRequest, + *kvpb.QueryLocksRequest, + *kvpb.QueryTxnRequest, + *kvpb.RangeStatsRequest, + *kvpb.RefreshRangeRequest, + *kvpb.RefreshRequest, + *kvpb.ReverseScanRequest, + *kvpb.RevertRangeRequest, + *kvpb.ScanRequest: continue } return authErrorf("request [%s] not permitted", args.Summary()) @@ -222,7 +223,7 @@ func (a tenantAuthorizer) authBatch( } func (a tenantAuthorizer) authGetRangeDescriptors( - tenID roachpb.TenantID, args *roachpb.GetRangeDescriptorsRequest, + tenID roachpb.TenantID, args *kvpb.GetRangeDescriptorsRequest, ) error { return validateSpan(tenID, args.Span) } @@ -239,7 +240,7 @@ func (a tenantAuthorizer) authSpanStats( // authRangeLookup authorizes the provided tenant to invoke the RangeLookup RPC // with the provided args. func (a tenantAuthorizer) authRangeLookup( - tenID roachpb.TenantID, args *roachpb.RangeLookupRequest, + tenID roachpb.TenantID, args *kvpb.RangeLookupRequest, ) error { tenSpan := tenantPrefix(tenID) if !tenSpan.ContainsKey(args.Key) { @@ -250,9 +251,7 @@ func (a tenantAuthorizer) authRangeLookup( // authRangeFeed authorizes the provided tenant to invoke the RangeFeed RPC with // the provided args. -func (a tenantAuthorizer) authRangeFeed( - tenID roachpb.TenantID, args *roachpb.RangeFeedRequest, -) error { +func (a tenantAuthorizer) authRangeFeed(tenID roachpb.TenantID, args *kvpb.RangeFeedRequest) error { rSpan, err := keys.SpanAddr(args.Span) if err != nil { return authError(err.Error()) @@ -264,7 +263,7 @@ func (a tenantAuthorizer) authRangeFeed( // authGossipSubscription authorizes the provided tenant to invoke the // GossipSubscription RPC with the provided args. func (a tenantAuthorizer) authGossipSubscription( - tenID roachpb.TenantID, args *roachpb.GossipSubscriptionRequest, + tenID roachpb.TenantID, args *kvpb.GossipSubscriptionRequest, ) error { for _, pat := range args.Patterns { allowed := false @@ -314,7 +313,7 @@ func (a tenantAuthorizer) authTenantRanges(tenID roachpb.TenantID) error { // authTokenBucket authorizes the provided tenant to invoke the // TokenBucket RPC with the provided args. func (a tenantAuthorizer) authTokenBucket( - tenID roachpb.TenantID, args *roachpb.TokenBucketRequest, + tenID roachpb.TenantID, args *kvpb.TokenBucketRequest, ) error { if args.TenantID == 0 { return authErrorf("token bucket request with unspecified tenant not permitted") @@ -328,7 +327,7 @@ func (a tenantAuthorizer) authTokenBucket( // authTenantSettings authorizes the provided tenant to invoke the // TenantSettings RPC with the provided args. func (a tenantAuthorizer) authTenantSettings( - tenID roachpb.TenantID, args *roachpb.TenantSettingsRequest, + tenID roachpb.TenantID, args *kvpb.TenantSettingsRequest, ) error { if !args.TenantID.IsSet() { return authErrorf("tenant settings request with unspecified tenant not permitted") diff --git a/pkg/rpc/auth_test.go b/pkg/rpc/auth_test.go index 7be00f3b3ea5..991e9f3edff0 100644 --- a/pkg/rpc/auth_test.go +++ b/pkg/rpc/auth_test.go @@ -20,6 +20,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -225,14 +226,14 @@ func makeSpanShared(t *testing.T, key string, endKey ...string) roachpb.Span { return s } -func makeReqShared(t *testing.T, key string, endKey ...string) roachpb.Request { +func makeReqShared(t *testing.T, key string, endKey ...string) kvpb.Request { s := makeSpanShared(t, key, endKey...) - h := roachpb.RequestHeaderFromSpan(s) - return &roachpb.ScanRequest{RequestHeader: h} + h := kvpb.RequestHeaderFromSpan(s) + return &kvpb.ScanRequest{RequestHeader: h} } -func makeReqs(reqs ...roachpb.Request) []roachpb.RequestUnion { - ru := make([]roachpb.RequestUnion, len(reqs)) +func makeReqs(reqs ...kvpb.Request) []kvpb.RequestUnion { + ru := make([]kvpb.RequestUnion, len(reqs)) for i, r := range reqs { ru[i].MustSetInner(r) } @@ -245,23 +246,23 @@ func TestTenantAuthRequest(t *testing.T) { makeSpan := func(key string, endKey ...string) roachpb.Span { return makeSpanShared(t, key, endKey...) } - makeReq := func(key string, endKey ...string) roachpb.Request { + makeReq := func(key string, endKey ...string) kvpb.Request { return makeReqShared(t, key, endKey...) } - makeDisallowedAdminReq := func(key string) roachpb.Request { + makeDisallowedAdminReq := func(key string) kvpb.Request { s := makeSpanShared(t, key) - h := roachpb.RequestHeader{Key: s.Key} - return &roachpb.AdminMergeRequest{RequestHeader: h} + h := kvpb.RequestHeader{Key: s.Key} + return &kvpb.AdminMergeRequest{RequestHeader: h} } - makeAdminSplitReq := func(key string) roachpb.Request { + makeAdminSplitReq := func(key string) kvpb.Request { s := makeSpan(key) - h := roachpb.RequestHeaderFromSpan(s) - return &roachpb.AdminSplitRequest{RequestHeader: h, SplitKey: s.Key} + h := kvpb.RequestHeaderFromSpan(s) + return &kvpb.AdminSplitRequest{RequestHeader: h, SplitKey: s.Key} } - makeAdminScatterReq := func(key string) roachpb.Request { + makeAdminScatterReq := func(key string) kvpb.Request { s := makeSpan(key) - h := roachpb.RequestHeaderFromSpan(s) - return &roachpb.AdminScatterRequest{RequestHeader: h} + h := kvpb.RequestHeaderFromSpan(s) + return &kvpb.AdminScatterRequest{RequestHeader: h} } makeSystemSpanConfigTarget := func(source, target uint64) roachpb.SpanConfigTarget { return roachpb.SpanConfigTarget{ @@ -297,8 +298,8 @@ func TestTenantAuthRequest(t *testing.T) { return &roachpb.SpanConfigConformanceRequest{Spans: []roachpb.Span{span}} } - makeGetRangeDescriptorsReq := func(span roachpb.Span) *roachpb.GetRangeDescriptorsRequest { - return &roachpb.GetRangeDescriptorsRequest{ + makeGetRangeDescriptorsReq := func(span roachpb.Span) *kvpb.GetRangeDescriptorsRequest { + return &kvpb.GetRangeDescriptorsRequest{ Span: span, } } @@ -310,160 +311,160 @@ func TestTenantAuthRequest(t *testing.T) { }{ "/cockroach.roachpb.Internal/Batch": { { - req: &roachpb.BatchRequest{}, + req: &kvpb.BatchRequest{}, expErr: `requested key span /Max not fully contained in tenant keyspace /Tenant/1{0-1}`, }, { - req: &roachpb.BatchRequest{Requests: makeReqs( + req: &kvpb.BatchRequest{Requests: makeReqs( makeReq("a", "b"), )}, expErr: `requested key span {a-b} not fully contained in tenant keyspace /Tenant/1{0-1}`, }, { - req: &roachpb.BatchRequest{Requests: makeReqs( + req: &kvpb.BatchRequest{Requests: makeReqs( makeReq(prefix(5, "a"), prefix(5, "b")), )}, expErr: `requested key span /Tenant/5{a-b} not fully contained in tenant keyspace /Tenant/1{0-1}`, }, { - req: &roachpb.BatchRequest{Requests: makeReqs( + req: &kvpb.BatchRequest{Requests: makeReqs( makeReq(prefix(10, "a"), prefix(10, "b")), )}, expErr: noError, }, { - req: &roachpb.BatchRequest{Requests: makeReqs( + req: &kvpb.BatchRequest{Requests: makeReqs( makeReq(prefix(50, "a"), prefix(50, "b")), )}, expErr: `requested key span /Tenant/50{a-b} not fully contained in tenant keyspace /Tenant/1{0-1}`, }, { - req: &roachpb.BatchRequest{Requests: makeReqs( + req: &kvpb.BatchRequest{Requests: makeReqs( makeReq("a", "b"), makeReq(prefix(5, "a"), prefix(5, "b")), )}, expErr: `requested key span {a-/Tenant/5b} not fully contained in tenant keyspace /Tenant/1{0-1}`, }, { - req: &roachpb.BatchRequest{Requests: makeReqs( + req: &kvpb.BatchRequest{Requests: makeReqs( makeReq(prefix(5, "a"), prefix(5, "b")), makeReq(prefix(10, "a"), prefix(10, "b")), )}, expErr: `requested key span /Tenant/{5a-10b} not fully contained in tenant keyspace /Tenant/1{0-1}`, }, { - req: &roachpb.BatchRequest{Requests: makeReqs( + req: &kvpb.BatchRequest{Requests: makeReqs( makeReq("a", prefix(10, "b")), )}, expErr: `requested key span {a-/Tenant/10b} not fully contained in tenant keyspace /Tenant/1{0-1}`, }, { - req: &roachpb.BatchRequest{Requests: makeReqs( + req: &kvpb.BatchRequest{Requests: makeReqs( makeReq(prefix(10, "a"), prefix(20, "b")), )}, expErr: `requested key span /Tenant/{10a-20b} not fully contained in tenant keyspace /Tenant/1{0-1}`, }, { - req: &roachpb.BatchRequest{Requests: makeReqs( + req: &kvpb.BatchRequest{Requests: makeReqs( makeDisallowedAdminReq("a"), )}, expErr: `request \[1 AdmMerge\] not permitted`, }, { - req: &roachpb.BatchRequest{Requests: makeReqs( + req: &kvpb.BatchRequest{Requests: makeReqs( makeDisallowedAdminReq(prefix(10, "a")), )}, expErr: `request \[1 AdmMerge\] not permitted`, }, { - req: &roachpb.BatchRequest{Requests: makeReqs( + req: &kvpb.BatchRequest{Requests: makeReqs( makeDisallowedAdminReq(prefix(50, "a")), )}, expErr: `request \[1 AdmMerge\] not permitted`, }, { - req: &roachpb.BatchRequest{Requests: makeReqs( + req: &kvpb.BatchRequest{Requests: makeReqs( makeDisallowedAdminReq(prefix(10, "a")), makeReq(prefix(10, "a"), prefix(10, "b")), )}, expErr: `request \[1 Scan, 1 AdmMerge\] not permitted`, }, { - req: &roachpb.BatchRequest{Requests: makeReqs( + req: &kvpb.BatchRequest{Requests: makeReqs( makeReq(prefix(10, "a"), prefix(10, "b")), makeDisallowedAdminReq(prefix(10, "a")), )}, expErr: `request \[1 Scan, 1 AdmMerge\] not permitted`, }, { - req: &roachpb.BatchRequest{Requests: makeReqs( + req: &kvpb.BatchRequest{Requests: makeReqs( makeAdminSplitReq("a"), )}, expErr: `requested key span a{-\\x00} not fully contained in tenant keyspace /Tenant/1{0-1}`, }, { - req: &roachpb.BatchRequest{Requests: makeReqs( + req: &kvpb.BatchRequest{Requests: makeReqs( makeAdminSplitReq(prefix(10, "a")), )}, expErr: noError, }, { - req: &roachpb.BatchRequest{Requests: makeReqs( + req: &kvpb.BatchRequest{Requests: makeReqs( makeAdminSplitReq(prefix(50, "a")), )}, expErr: `requested key span /Tenant/50a{-\\x00} not fully contained in tenant keyspace /Tenant/1{0-1}`, }, { - req: &roachpb.BatchRequest{Requests: makeReqs( + req: &kvpb.BatchRequest{Requests: makeReqs( makeAdminSplitReq(prefix(10, "a")), makeReq(prefix(10, "a"), prefix(10, "b")), )}, expErr: noError, }, { - req: &roachpb.BatchRequest{Requests: makeReqs( + req: &kvpb.BatchRequest{Requests: makeReqs( makeReq(prefix(10, "a"), prefix(10, "b")), makeAdminSplitReq(prefix(10, "a")), )}, expErr: noError, }, { - req: &roachpb.BatchRequest{Requests: makeReqs( + req: &kvpb.BatchRequest{Requests: makeReqs( makeAdminScatterReq("a"), )}, expErr: `requested key span a{-\\x00} not fully contained in tenant keyspace /Tenant/1{0-1}`, }, { - req: &roachpb.BatchRequest{Requests: makeReqs( + req: &kvpb.BatchRequest{Requests: makeReqs( makeAdminScatterReq(prefix(10, "a")), )}, expErr: noError, }, { - req: &roachpb.BatchRequest{Requests: makeReqs( + req: &kvpb.BatchRequest{Requests: makeReqs( makeAdminScatterReq(prefix(50, "a")), )}, expErr: `requested key span /Tenant/50a{-\\x00} not fully contained in tenant keyspace /Tenant/1{0-1}`, }, { - req: &roachpb.BatchRequest{Requests: makeReqs( + req: &kvpb.BatchRequest{Requests: makeReqs( makeAdminScatterReq(prefix(10, "a")), makeReq(prefix(10, "a"), prefix(10, "b")), )}, expErr: noError, }, { - req: &roachpb.BatchRequest{Requests: makeReqs( + req: &kvpb.BatchRequest{Requests: makeReqs( makeReq(prefix(10, "a"), prefix(10, "b")), makeAdminScatterReq(prefix(10, "a")), )}, expErr: noError, }, { - req: &roachpb.BatchRequest{Requests: makeReqs( - func() roachpb.Request { - h := roachpb.RequestHeaderFromSpan(makeSpan("a")) - return &roachpb.SubsumeRequest{RequestHeader: h} + req: &kvpb.BatchRequest{Requests: makeReqs( + func() kvpb.Request { + h := kvpb.RequestHeaderFromSpan(makeSpan("a")) + return &kvpb.SubsumeRequest{RequestHeader: h} }(), )}, expErr: `request \[1 Subsume\] not permitted`, @@ -471,97 +472,97 @@ func TestTenantAuthRequest(t *testing.T) { }, "/cockroach.roachpb.Internal/RangeLookup": { { - req: &roachpb.RangeLookupRequest{}, + req: &kvpb.RangeLookupRequest{}, expErr: `requested key /Min not fully contained in tenant keyspace /Tenant/1{0-1}`, }, { - req: &roachpb.RangeLookupRequest{Key: roachpb.RKey("a")}, + req: &kvpb.RangeLookupRequest{Key: roachpb.RKey("a")}, expErr: `requested key "a" not fully contained in tenant keyspace /Tenant/1{0-1}`, }, { - req: &roachpb.RangeLookupRequest{Key: roachpb.RKey(prefix(5, "a"))}, + req: &kvpb.RangeLookupRequest{Key: roachpb.RKey(prefix(5, "a"))}, expErr: `requested key /Tenant/5"a" not fully contained in tenant keyspace /Tenant/1{0-1}`, }, { - req: &roachpb.RangeLookupRequest{Key: roachpb.RKey(prefix(10, "a"))}, + req: &kvpb.RangeLookupRequest{Key: roachpb.RKey(prefix(10, "a"))}, expErr: noError, }, { - req: &roachpb.RangeLookupRequest{Key: roachpb.RKey(prefix(50, "a"))}, + req: &kvpb.RangeLookupRequest{Key: roachpb.RKey(prefix(50, "a"))}, expErr: `requested key /Tenant/50"a" not fully contained in tenant keyspace /Tenant/1{0-1}`, }, }, "/cockroach.roachpb.Internal/RangeFeed": { { - req: &roachpb.RangeFeedRequest{}, + req: &kvpb.RangeFeedRequest{}, expErr: `requested key span /Min not fully contained in tenant keyspace /Tenant/1{0-1}`, }, { - req: &roachpb.RangeFeedRequest{Span: makeSpan("a", "b")}, + req: &kvpb.RangeFeedRequest{Span: makeSpan("a", "b")}, expErr: `requested key span {a-b} not fully contained in tenant keyspace /Tenant/1{0-1}`, }, { - req: &roachpb.RangeFeedRequest{Span: makeSpan(prefix(5, "a"), prefix(5, "b"))}, + req: &kvpb.RangeFeedRequest{Span: makeSpan(prefix(5, "a"), prefix(5, "b"))}, expErr: `requested key span /Tenant/5{a-b} not fully contained in tenant keyspace /Tenant/1{0-1}`, }, { - req: &roachpb.RangeFeedRequest{Span: makeSpan(prefix(10, "a"), prefix(10, "b"))}, + req: &kvpb.RangeFeedRequest{Span: makeSpan(prefix(10, "a"), prefix(10, "b"))}, expErr: noError, }, { - req: &roachpb.RangeFeedRequest{Span: makeSpan(prefix(50, "a"), prefix(50, "b"))}, + req: &kvpb.RangeFeedRequest{Span: makeSpan(prefix(50, "a"), prefix(50, "b"))}, expErr: `requested key span /Tenant/50{a-b} not fully contained in tenant keyspace /Tenant/1{0-1}`, }, { - req: &roachpb.RangeFeedRequest{Span: makeSpan("a", prefix(10, "b"))}, + req: &kvpb.RangeFeedRequest{Span: makeSpan("a", prefix(10, "b"))}, expErr: `requested key span {a-/Tenant/10b} not fully contained in tenant keyspace /Tenant/1{0-1}`, }, { - req: &roachpb.RangeFeedRequest{Span: makeSpan(prefix(10, "a"), prefix(20, "b"))}, + req: &kvpb.RangeFeedRequest{Span: makeSpan(prefix(10, "a"), prefix(20, "b"))}, expErr: `requested key span /Tenant/{10a-20b} not fully contained in tenant keyspace /Tenant/1{0-1}`, }, }, "/cockroach.roachpb.Internal/GossipSubscription": { { - req: &roachpb.GossipSubscriptionRequest{}, + req: &kvpb.GossipSubscriptionRequest{}, expErr: noError, }, { - req: &roachpb.GossipSubscriptionRequest{Patterns: []string{"node:.*"}}, + req: &kvpb.GossipSubscriptionRequest{Patterns: []string{"node:.*"}}, expErr: noError, }, { - req: &roachpb.GossipSubscriptionRequest{Patterns: []string{"system-db"}}, + req: &kvpb.GossipSubscriptionRequest{Patterns: []string{"system-db"}}, expErr: noError, }, { - req: &roachpb.GossipSubscriptionRequest{Patterns: []string{"table-stat-added"}}, + req: &kvpb.GossipSubscriptionRequest{Patterns: []string{"table-stat-added"}}, expErr: `requested pattern "table-stat-added" not permitted`, }, { - req: &roachpb.GossipSubscriptionRequest{Patterns: []string{"node:.*", "system-db"}}, + req: &kvpb.GossipSubscriptionRequest{Patterns: []string{"node:.*", "system-db"}}, expErr: noError, }, { - req: &roachpb.GossipSubscriptionRequest{Patterns: []string{"node:.*", "system-db", "table-stat-added"}}, + req: &kvpb.GossipSubscriptionRequest{Patterns: []string{"node:.*", "system-db", "table-stat-added"}}, expErr: `requested pattern "table-stat-added" not permitted`, }, }, "/cockroach.roachpb.Internal/TokenBucket": { { - req: &roachpb.TokenBucketRequest{TenantID: tenID.ToUint64()}, + req: &kvpb.TokenBucketRequest{TenantID: tenID.ToUint64()}, expErr: noError, }, { - req: &roachpb.TokenBucketRequest{TenantID: roachpb.SystemTenantID.ToUint64()}, + req: &kvpb.TokenBucketRequest{TenantID: roachpb.SystemTenantID.ToUint64()}, expErr: `token bucket request for tenant system not permitted`, }, { - req: &roachpb.TokenBucketRequest{TenantID: 13}, + req: &kvpb.TokenBucketRequest{TenantID: 13}, expErr: `token bucket request for tenant 13 not permitted`, }, { - req: &roachpb.TokenBucketRequest{}, + req: &kvpb.TokenBucketRequest{}, expErr: `token bucket request with unspecified tenant not permitted`, }, }, @@ -902,7 +903,7 @@ func TestTenantAuthCapabilityChecks(t *testing.T) { }{ "/cockroach.roachpb.Internal/Batch": { { - req: &roachpb.BatchRequest{Requests: makeReqs( + req: &kvpb.BatchRequest{Requests: makeReqs( makeReqShared(t, prefix(10, "a"), prefix(10, "b")), )}, configureAuthorizer: func(authorizer *mockAuthorizer) { @@ -911,7 +912,7 @@ func TestTenantAuthCapabilityChecks(t *testing.T) { expErr: "", }, { - req: &roachpb.BatchRequest{Requests: makeReqs( + req: &kvpb.BatchRequest{Requests: makeReqs( makeReqShared(t, prefix(10, "a"), prefix(10, "b")), )}, configureAuthorizer: func(authorizer *mockAuthorizer) { @@ -949,7 +950,7 @@ var _ tenantcapabilities.Authorizer = &mockAuthorizer{} // HasCapabilityForBatch implements the tenantcapabilities.Authorizer interface. func (m mockAuthorizer) HasCapabilityForBatch( - context.Context, roachpb.TenantID, *roachpb.BatchRequest, + context.Context, roachpb.TenantID, *kvpb.BatchRequest, ) error { if m.hasCapabilityForBatch { return nil diff --git a/pkg/rpc/codec_test.go b/pkg/rpc/codec_test.go index 3405b175c410..acd18bd640fa 100644 --- a/pkg/rpc/codec_test.go +++ b/pkg/rpc/codec_test.go @@ -16,6 +16,7 @@ import ( "testing" "unicode" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/stretchr/testify/require" @@ -50,15 +51,15 @@ func TestCodecMarshalUnmarshal(t *testing.T) { } }, func() interface{} { return &grpc_health_v1.HealthCheckRequest{} }}, - {"roachpb.GetRequest", + {"kvpb.GetRequest", func() interface{} { - return &roachpb.GetRequest{ - RequestHeader: roachpb.RequestHeader{ + return &kvpb.GetRequest{ + RequestHeader: kvpb.RequestHeader{ Key: roachpb.Key("turtle"), }, } }, - func() interface{} { return &roachpb.GetRequest{} }}, + func() interface{} { return &kvpb.GetRequest{} }}, } { t.Run(test.name, func(t *testing.T) { input := test.filledMsgBuilder() diff --git a/pkg/rpc/context.go b/pkg/rpc/context.go index 44df5e9dba75..c47a3a6f1f68 100644 --- a/pkg/rpc/context.go +++ b/pkg/rpc/context.go @@ -25,6 +25,7 @@ import ( circuit "github.com/cockroachdb/circuitbreaker" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" @@ -731,13 +732,13 @@ func (rpcCtx *Context) GetLocalInternalClientForAddr( return nil } -// internalClientAdapter is an implementation of roachpb.InternalClient that +// internalClientAdapter is an implementation of kvpb.InternalClient that // bypasses gRPC, calling the wrapped local server directly. // // Even though the calls don't go through gRPC, the internalClientAdapter runs // the configured gRPC client-side and server-side interceptors. type internalClientAdapter struct { - server roachpb.InternalServer + server kvpb.InternalServer // clientTenantID is the tenant ID for the client (caller) side // of the call. (The server/callee side is @@ -751,7 +752,7 @@ type internalClientAdapter struct { // batchHandler is the RPC handler for Batch(). This includes both the chain // of client-side and server-side gRPC interceptors, and bottoms out by // calling server.Batch(). - batchHandler func(ctx context.Context, ba *roachpb.BatchRequest, opts ...grpc.CallOption) (*roachpb.BatchResponse, error) + batchHandler func(ctx context.Context, ba *kvpb.BatchRequest, opts ...grpc.CallOption) (*kvpb.BatchResponse, error) // The streaming interceptors. These cannot be chained together at // construction time like the unary interceptors. @@ -772,7 +773,7 @@ var _ RestrictedInternalClient = internalClientAdapter{} // caller and callee use separate tracers, so we can't // use a child tracing span directly. func makeInternalClientAdapter( - server roachpb.InternalServer, + server kvpb.InternalServer, clientTenantID roachpb.TenantID, separateTracers bool, clientUnaryInterceptors []grpc.UnaryClientInterceptor, @@ -798,7 +799,7 @@ func makeInternalClientAdapter( }, serverUnaryInterceptors, func(ctx context.Context, req interface{}) (interface{}, error) { - br, err := server.Batch(ctx, req.(*roachpb.BatchRequest)) + br, err := server.Batch(ctx, req.(*kvpb.BatchRequest)) return br, err }, ) @@ -810,9 +811,9 @@ func makeInternalClientAdapter( func(ctx context.Context, method string, req, reply interface{}, cc *grpc.ClientConn, opts ...grpc.CallOption) error { resp, err := batchServerHandler(ctx, req) if resp != nil { - br := resp.(*roachpb.BatchResponse) + br := resp.(*kvpb.BatchResponse) if br != nil { - *(reply.(*roachpb.BatchResponse)) = *br + *(reply.(*kvpb.BatchResponse)) = *br } } return err @@ -824,11 +825,11 @@ func makeInternalClientAdapter( separateTracers: separateTracers, clientStreamInterceptors: clientStreamInterceptors, serverStreamInterceptors: serverStreamInterceptors, - batchHandler: func(ctx context.Context, ba *roachpb.BatchRequest, opts ...grpc.CallOption) (*roachpb.BatchResponse, error) { + batchHandler: func(ctx context.Context, ba *kvpb.BatchRequest, opts ...grpc.CallOption) (*kvpb.BatchResponse, error) { ba = ba.ShallowCopy() // Mark this as originating locally, which is useful for the decision about // memory allocation tracking. - ba.AdmissionHeader.SourceLocation = roachpb.AdmissionHeader_LOCAL + ba.AdmissionHeader.SourceLocation = kvpb.AdmissionHeader_LOCAL // reply serves to communicate the RPC response from the RPC handler (through // the server interceptors) to the client interceptors. The client // interceptors will have a chance to modify it, and ultimately it will be @@ -837,7 +838,7 @@ func makeInternalClientAdapter( // a result from the next interceptor (and eventually from the server); // instead, the result is allocated by the client. We'll copy the // server-side result into reply in batchHandler(). - reply := new(roachpb.BatchResponse) + reply := new(kvpb.BatchResponse) // Create a new context from the existing one with the "local request" // field set. This tells the handler that this is an in-process request, @@ -997,10 +998,10 @@ func getChainUnaryInvoker( } } -// Batch implements the roachpb.InternalClient interface. +// Batch implements the kvpb.InternalClient interface. func (a internalClientAdapter) Batch( - ctx context.Context, ba *roachpb.BatchRequest, opts ...grpc.CallOption, -) (*roachpb.BatchResponse, error) { + ctx context.Context, ba *kvpb.BatchRequest, opts ...grpc.CallOption, +) (*kvpb.BatchResponse, error) { return a.batchHandler(ctx, ba, opts...) } @@ -1033,8 +1034,8 @@ var muxRangefeedStreamInfo = &grpc.StreamServerInfo{ // RangeFeed implements the RestrictedInternalClient interface. func (a internalClientAdapter) RangeFeed( - ctx context.Context, args *roachpb.RangeFeedRequest, opts ...grpc.CallOption, -) (roachpb.Internal_RangeFeedClient, error) { + ctx context.Context, args *kvpb.RangeFeedRequest, opts ...grpc.CallOption, +) (kvpb.Internal_RangeFeedClient, error) { // RangeFeed is a server-streaming RPC, so we'll use a pipe between the // server-side sender and the client-side receiver. The two ends of this pipe // are wrapped in a client stream (rawClientStream) and a server stream @@ -1065,7 +1066,7 @@ func (a internalClientAdapter) RangeFeed( // -> rawClientStream // -> RPC caller writer, reader := makePipe(func(dst interface{}, src interface{}) { - *dst.(*roachpb.RangeFeedEvent) = *src.(*roachpb.RangeFeedEvent) + *dst.(*kvpb.RangeFeedEvent) = *src.(*kvpb.RangeFeedEvent) }) rawClientStream := &clientStream{ ctx: ctx, @@ -1092,7 +1093,7 @@ func (a internalClientAdapter) RangeFeed( } // Mark this request as originating locally. - args.AdmissionHeader.SourceLocation = roachpb.AdmissionHeader_LOCAL + args.AdmissionHeader.SourceLocation = kvpb.AdmissionHeader_LOCAL // Spawn a goroutine running the server-side handler. This goroutine // communicates with the client stream through rfPipe. @@ -1142,15 +1143,15 @@ func (a internalClientAdapter) RangeFeed( } // rangeFeedClientAdapter adapts an untyped ClientStream to the typed -// roachpb.Internal_RangeFeedClient used by the rangefeed RPC client. +// kvpb.Internal_RangeFeedClient used by the rangefeed RPC client. type rangeFeedClientAdapter struct { grpc.ClientStream } -var _ roachpb.Internal_RangeFeedClient = rangeFeedClientAdapter{} +var _ kvpb.Internal_RangeFeedClient = rangeFeedClientAdapter{} -func (x rangeFeedClientAdapter) Recv() (*roachpb.RangeFeedEvent, error) { - m := new(roachpb.RangeFeedEvent) +func (x rangeFeedClientAdapter) Recv() (*kvpb.RangeFeedEvent, error) { + m := new(kvpb.RangeFeedEvent) if err := x.ClientStream.RecvMsg(m); err != nil { return nil, err } @@ -1163,7 +1164,7 @@ func (x rangeFeedClientAdapter) Recv() (*roachpb.RangeFeedEvent, error) { // uni-directional. This is why this implementation is a bit different. func (a internalClientAdapter) MuxRangeFeed( ctx context.Context, opts ...grpc.CallOption, -) (roachpb.Internal_MuxRangeFeedClient, error) { +) (kvpb.Internal_MuxRangeFeedClient, error) { // MuxRangeFeed is a bi-directional RPC, so we have to deal with two streams: // the client stream and the server stream. The client stream sends // RangeFeedRequests and receives MuxRangeFeedEvents, whereas the server @@ -1189,10 +1190,10 @@ func (a internalClientAdapter) MuxRangeFeed( // -> RPC caller eventWriter, eventReader := makePipe(func(dst interface{}, src interface{}) { - *dst.(*roachpb.MuxRangeFeedEvent) = *src.(*roachpb.MuxRangeFeedEvent) + *dst.(*kvpb.MuxRangeFeedEvent) = *src.(*kvpb.MuxRangeFeedEvent) }) requestWriter, requestReader := makePipe(func(dst interface{}, src interface{}) { - *dst.(*roachpb.RangeFeedRequest) = *src.(*roachpb.RangeFeedRequest) + *dst.(*kvpb.RangeFeedRequest) = *src.(*kvpb.RangeFeedRequest) }) rawClientStream := &clientStream{ ctx: ctx, @@ -1268,16 +1269,16 @@ type muxRangeFeedClientAdapter struct { grpc.ClientStream } -var _ roachpb.Internal_MuxRangeFeedClient = muxRangeFeedClientAdapter{} +var _ kvpb.Internal_MuxRangeFeedClient = muxRangeFeedClientAdapter{} -func (a muxRangeFeedClientAdapter) Send(request *roachpb.RangeFeedRequest) error { +func (a muxRangeFeedClientAdapter) Send(request *kvpb.RangeFeedRequest) error { // Mark this request as originating locally. - request.AdmissionHeader.SourceLocation = roachpb.AdmissionHeader_LOCAL + request.AdmissionHeader.SourceLocation = kvpb.AdmissionHeader_LOCAL return a.SendMsg(request) } -func (a muxRangeFeedClientAdapter) Recv() (*roachpb.MuxRangeFeedEvent, error) { - m := new(roachpb.MuxRangeFeedEvent) +func (a muxRangeFeedClientAdapter) Recv() (*kvpb.MuxRangeFeedEvent, error) { + m := new(kvpb.MuxRangeFeedEvent) if err := a.ClientStream.RecvMsg(m); err != nil { return nil, err } @@ -1288,14 +1289,14 @@ type muxRangeFeedServerAdapter struct { grpc.ServerStream } -var _ roachpb.Internal_MuxRangeFeedServer = muxRangeFeedServerAdapter{} +var _ kvpb.Internal_MuxRangeFeedServer = muxRangeFeedServerAdapter{} -func (a muxRangeFeedServerAdapter) Send(event *roachpb.MuxRangeFeedEvent) error { +func (a muxRangeFeedServerAdapter) Send(event *kvpb.MuxRangeFeedEvent) error { return a.SendMsg(event) } -func (a muxRangeFeedServerAdapter) Recv() (*roachpb.RangeFeedRequest, error) { - m := new(roachpb.RangeFeedRequest) +func (a muxRangeFeedServerAdapter) Recv() (*kvpb.RangeFeedRequest, error) { + m := new(kvpb.RangeFeedRequest) if err := a.RecvMsg(m); err != nil { return nil, err } @@ -1472,17 +1473,17 @@ func (s serverStream) sendError(err error) { var _ grpc.ServerStream = serverStream{} // rangeFeedServerAdapter adapts an untyped ServerStream to the typed -// roachpb.Internal_RangeFeedServer interface, expected by the RangeFeed RPC +// kvpb.Internal_RangeFeedServer interface, expected by the RangeFeed RPC // handler. type rangeFeedServerAdapter struct { grpc.ServerStream } -var _ roachpb.Internal_RangeFeedServer = rangeFeedServerAdapter{} +var _ kvpb.Internal_RangeFeedServer = rangeFeedServerAdapter{} -// roachpb.Internal_RangeFeedServer methods. -func (a rangeFeedServerAdapter) Recv() (*roachpb.RangeFeedEvent, error) { - out := &roachpb.RangeFeedEvent{} +// kvpb.Internal_RangeFeedServer methods. +func (a rangeFeedServerAdapter) Recv() (*kvpb.RangeFeedEvent, error) { + out := &kvpb.RangeFeedEvent{} err := a.RecvMsg(out) if err != nil { return nil, err @@ -1490,8 +1491,8 @@ func (a rangeFeedServerAdapter) Recv() (*roachpb.RangeFeedEvent, error) { return out, nil } -// Send implement the roachpb.Internal_RangeFeedServer interface. -func (a rangeFeedServerAdapter) Send(e *roachpb.RangeFeedEvent) error { +// Send implement the kvpb.Internal_RangeFeedServer interface. +func (a rangeFeedServerAdapter) Send(e *kvpb.RangeFeedEvent) error { return a.ServerStream.SendMsg(e) } @@ -1507,7 +1508,7 @@ func IsLocal(iface RestrictedInternalClient) bool { // serverInterceptors lists the interceptors that will be run on RPCs done // through this local server. func (rpcCtx *Context) SetLocalInternalServer( - internalServer roachpb.InternalServer, + internalServer kvpb.InternalServer, serverInterceptors ServerInterceptorInfo, clientInterceptors ClientInterceptorInfo, ) { @@ -1837,7 +1838,7 @@ type growStackCodec struct { // Unmarshal detects BatchRequests and calls growstack.Grow before calling // through to the underlying codec. func (c growStackCodec) Unmarshal(data []byte, v interface{}) error { - if _, ok := v.(*roachpb.BatchRequest); ok { + if _, ok := v.(*kvpb.BatchRequest); ok { growstack.Grow() } return c.Codec.Unmarshal(data, v) diff --git a/pkg/rpc/context_test.go b/pkg/rpc/context_test.go index 20070aed63b9..c026298e164e 100644 --- a/pkg/rpc/context_test.go +++ b/pkg/rpc/context_test.go @@ -23,6 +23,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -317,31 +318,31 @@ func testClockOffsetInPingRequestInternal(t *testing.T, clientOnly bool) { } } -var _ roachpb.InternalServer = &internalServer{} +var _ kvpb.InternalServer = &internalServer{} type internalServer struct { // rangeFeedEvents are returned on RangeFeed() calls. - rangeFeedEvents []roachpb.RangeFeedEvent - rfServerStream roachpb.Internal_RangeFeedServer - muxRfServerStream roachpb.Internal_MuxRangeFeedServer + rangeFeedEvents []kvpb.RangeFeedEvent + rfServerStream kvpb.Internal_RangeFeedServer + muxRfServerStream kvpb.Internal_MuxRangeFeedServer } type rangefeedEventSink struct { ctx context.Context - stream roachpb.Internal_MuxRangeFeedServer + stream kvpb.Internal_MuxRangeFeedServer } -var _ roachpb.RangeFeedEventSink = (*rangefeedEventSink)(nil) +var _ kvpb.RangeFeedEventSink = (*rangefeedEventSink)(nil) func (s *rangefeedEventSink) Context() context.Context { return s.ctx } -func (s *rangefeedEventSink) Send(event *roachpb.RangeFeedEvent) error { - return s.stream.Send(&roachpb.MuxRangeFeedEvent{RangeFeedEvent: *event}) +func (s *rangefeedEventSink) Send(event *kvpb.RangeFeedEvent) error { + return s.stream.Send(&kvpb.MuxRangeFeedEvent{RangeFeedEvent: *event}) } -func (s *internalServer) MuxRangeFeed(stream roachpb.Internal_MuxRangeFeedServer) error { +func (s *internalServer) MuxRangeFeed(stream kvpb.Internal_MuxRangeFeedServer) error { s.muxRfServerStream = stream _, err := stream.Recv() if err != nil { @@ -351,20 +352,18 @@ func (s *internalServer) MuxRangeFeed(stream roachpb.Internal_MuxRangeFeedServer return s.singleRangeFeed(sink) } -func (*internalServer) Batch( - context.Context, *roachpb.BatchRequest, -) (*roachpb.BatchResponse, error) { +func (*internalServer) Batch(context.Context, *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { return nil, nil } func (*internalServer) RangeLookup( - context.Context, *roachpb.RangeLookupRequest, -) (*roachpb.RangeLookupResponse, error) { + context.Context, *kvpb.RangeLookupRequest, +) (*kvpb.RangeLookupResponse, error) { panic("unimplemented") } func (s *internalServer) RangeFeed( - _ *roachpb.RangeFeedRequest, stream roachpb.Internal_RangeFeedServer, + _ *kvpb.RangeFeedRequest, stream kvpb.Internal_RangeFeedServer, ) error { s.rfServerStream = stream err := s.singleRangeFeed(stream) @@ -374,7 +373,7 @@ func (s *internalServer) RangeFeed( return nil } -func (s *internalServer) singleRangeFeed(sink roachpb.RangeFeedEventSink) error { +func (s *internalServer) singleRangeFeed(sink kvpb.RangeFeedEventSink) error { for _, ev := range s.rangeFeedEvents { evCpy := ev if err := sink.Send(&evCpy); err != nil { @@ -385,26 +384,26 @@ func (s *internalServer) singleRangeFeed(sink roachpb.RangeFeedEventSink) error } func (*internalServer) GossipSubscription( - *roachpb.GossipSubscriptionRequest, roachpb.Internal_GossipSubscriptionServer, + *kvpb.GossipSubscriptionRequest, kvpb.Internal_GossipSubscriptionServer, ) error { panic("unimplemented") } func (*internalServer) ResetQuorum( - context.Context, *roachpb.ResetQuorumRequest, -) (*roachpb.ResetQuorumResponse, error) { + context.Context, *kvpb.ResetQuorumRequest, +) (*kvpb.ResetQuorumResponse, error) { panic("unimplemented") } func (*internalServer) Join( - context.Context, *roachpb.JoinNodeRequest, -) (*roachpb.JoinNodeResponse, error) { + context.Context, *kvpb.JoinNodeRequest, +) (*kvpb.JoinNodeResponse, error) { panic("unimplemented") } func (*internalServer) TokenBucket( - ctx context.Context, in *roachpb.TokenBucketRequest, -) (*roachpb.TokenBucketResponse, error) { + ctx context.Context, in *kvpb.TokenBucketRequest, +) (*kvpb.TokenBucketResponse, error) { panic("unimplemented") } @@ -433,13 +432,13 @@ func (s *internalServer) SpanConfigConformance( } func (*internalServer) TenantSettings( - *roachpb.TenantSettingsRequest, roachpb.Internal_TenantSettingsServer, + *kvpb.TenantSettingsRequest, kvpb.Internal_TenantSettingsServer, ) error { panic("unimplemented") } func (n *internalServer) GetRangeDescriptors( - *roachpb.GetRangeDescriptorsRequest, roachpb.Internal_GetRangeDescriptorsServer, + *kvpb.GetRangeDescriptorsRequest, kvpb.Internal_GetRangeDescriptorsServer, ) error { panic("unimplemented") } @@ -550,7 +549,7 @@ func TestInternalClientAdapterRunsInterceptors(t *testing.T) { for i := 0; i < 2; i++ { clientUnaryInterceptor1Called, clientUnaryInterceptor2Called = false, false serverUnaryInterceptor1Called, serverUnaryInterceptor2Called = false, false - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} _, err := lic.Batch(ctx, ba) require.NoError(t, err) require.True(t, serverUnaryInterceptor1Called) @@ -562,7 +561,7 @@ func TestInternalClientAdapterRunsInterceptors(t *testing.T) { for i := 0; i < 2; i++ { serverStreamInterceptor1Called, serverStreamInterceptor2Called = false, false clientStreamInterceptor1Called, clientStreamInterceptor2Called = false, false - stream, err := lic.RangeFeed(ctx, &roachpb.RangeFeedRequest{}) + stream, err := lic.RangeFeed(ctx, &kvpb.RangeFeedRequest{}) require.NoError(t, err) _, err = stream.Recv() require.ErrorIs(t, err, io.EOF) @@ -610,7 +609,7 @@ func TestInternalClientAdapterWithClientStreamInterceptors(t *testing.T) { return s, nil }) - internal := &internalServer{rangeFeedEvents: []roachpb.RangeFeedEvent{{}, {}}} + internal := &internalServer{rangeFeedEvents: []kvpb.RangeFeedEvent{{}, {}}} serverCtx.SetLocalInternalServer( internal, serverInterceptors, clientInterceptors) @@ -623,14 +622,14 @@ func TestInternalClientAdapterWithClientStreamInterceptors(t *testing.T) { if useMux { stream, err := lic.MuxRangeFeed(ctx) require.NoError(t, err) - require.NoError(t, stream.Send(&roachpb.RangeFeedRequest{})) + require.NoError(t, stream.Send(&kvpb.RangeFeedRequest{})) receiveEvent = func() error { e, err := stream.Recv() _ = e return err } } else { - stream, err := lic.RangeFeed(ctx, &roachpb.RangeFeedRequest{}) + stream, err := lic.RangeFeed(ctx, &kvpb.RangeFeedRequest{}) require.NoError(t, err) receiveEvent = func() error { _, err := stream.Recv() @@ -690,7 +689,7 @@ func TestInternalClientAdapterWithServerStreamInterceptors(t *testing.T) { return handler(srv, serverStream) }) - internal := &internalServer{rangeFeedEvents: []roachpb.RangeFeedEvent{{}, {}}} + internal := &internalServer{rangeFeedEvents: []kvpb.RangeFeedEvent{{}, {}}} serverCtx.SetLocalInternalServer( internal, serverInterceptors, ClientInterceptorInfo{}) @@ -703,13 +702,13 @@ func TestInternalClientAdapterWithServerStreamInterceptors(t *testing.T) { if useMux { stream, err := lic.MuxRangeFeed(ctx) require.NoError(t, err) - require.NoError(t, stream.Send(&roachpb.RangeFeedRequest{})) + require.NoError(t, stream.Send(&kvpb.RangeFeedRequest{})) receiveEvent = func() error { _, err := stream.Recv() return err } } else { - stream, err := lic.RangeFeed(ctx, &roachpb.RangeFeedRequest{}) + stream, err := lic.RangeFeed(ctx, &kvpb.RangeFeedRequest{}) require.NoError(t, err) receiveEvent = func() error { _, err := stream.Recv() @@ -833,7 +832,7 @@ func BenchmarkInternalClientAdapter(b *testing.B) { lic, ok := ic.(internalClientAdapter) require.True(b, ok) require.Equal(b, internal, lic.server) - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} _, err = lic.Batch(ctx, ba) require.NoError(b, err) @@ -1994,11 +1993,11 @@ func TestGRPCDeadlinePropagation(t *testing.T) { s := newTestServer(t, serverCtx, grpc.UnknownServiceHandler( func(srv interface{}, stream grpc.ServerStream) error { serverDeadline, _ = stream.Context().Deadline() - var ba roachpb.BatchRequest + var ba kvpb.BatchRequest if err := stream.RecvMsg(&ba); err != nil { return err } - return stream.SendMsg(&roachpb.BatchResponse{}) + return stream.SendMsg(&kvpb.BatchResponse{}) }, )) RegisterHeartbeatServer(s, serverCtx.NewHeartbeatService()) @@ -2025,8 +2024,8 @@ func TestGRPCDeadlinePropagation(t *testing.T) { const method = "/cockroach.rpc.Testing/Foo" cs, err := defConn.NewStream(ctxWithDeadline, &desc, method) require.Nil(t, err) - require.Nil(t, cs.SendMsg(&roachpb.BatchRequest{})) - var br roachpb.BatchResponse + require.Nil(t, cs.SendMsg(&kvpb.BatchRequest{})) + var br kvpb.BatchResponse require.Nil(t, cs.RecvMsg(&br)) require.Nil(t, cs.CloseSend()) @@ -2356,11 +2355,11 @@ func TestTestingKnobs(t *testing.T) { // a BatchResponse. It will be used both as a unary and stream handler below. s := newTestServer(t, serverCtx, grpc.UnknownServiceHandler( func(srv interface{}, stream grpc.ServerStream) error { - var ba roachpb.BatchRequest + var ba kvpb.BatchRequest if err := stream.RecvMsg(&ba); err != nil { return err } - return stream.SendMsg(&roachpb.BatchResponse{}) + return stream.SendMsg(&kvpb.BatchResponse{}) }, )) RegisterHeartbeatServer(s, &HeartbeatService{ @@ -2426,8 +2425,8 @@ func TestTestingKnobs(t *testing.T) { } cs, err := defConn.NewStream(ctx, &desc, streamMethod) require.Nil(t, err) - require.Nil(t, cs.SendMsg(&roachpb.BatchRequest{})) - var br roachpb.BatchResponse + require.Nil(t, cs.SendMsg(&kvpb.BatchRequest{})) + var br kvpb.BatchResponse require.Nil(t, cs.RecvMsg(&br)) require.Nil(t, cs.CloseSend()) } diff --git a/pkg/rpc/nodedialer/BUILD.bazel b/pkg/rpc/nodedialer/BUILD.bazel index 5e38a95d769f..8d2462648aab 100644 --- a/pkg/rpc/nodedialer/BUILD.bazel +++ b/pkg/rpc/nodedialer/BUILD.bazel @@ -8,6 +8,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/kv/kvbase", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/rpc", "//pkg/util/log", @@ -28,6 +29,7 @@ go_test( embed = [":nodedialer"], deps = [ "//pkg/clusterversion", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/rpc", "//pkg/settings/cluster", diff --git a/pkg/rpc/nodedialer/nodedialer.go b/pkg/rpc/nodedialer/nodedialer.go index d8a409d77662..a69feed722ee 100644 --- a/pkg/rpc/nodedialer/nodedialer.go +++ b/pkg/rpc/nodedialer/nodedialer.go @@ -19,6 +19,7 @@ import ( circuit "github.com/cockroachdb/circuitbreaker" "github.com/cockroachdb/cockroach/pkg/kv/kvbase" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -138,7 +139,7 @@ func (n *Dialer) DialNoBreaker( } // DialInternalClient is a specialization of DialClass for callers that -// want a roachpb.InternalClient. This supports an optimization to bypass the +// want a kvpb.InternalClient. This supports an optimization to bypass the // network for the local node. // // For a more contextualized explanation, see the comment that decorates @@ -167,7 +168,7 @@ func (n *Dialer) DialInternalClient( if err != nil { return nil, err } - return TracingInternalClient{InternalClient: roachpb.NewInternalClient(conn)}, nil + return TracingInternalClient{InternalClient: kvpb.NewInternalClient(conn)}, nil } // dial performs the dialing of the remote connection. If breaker is nil, @@ -305,13 +306,13 @@ func (n *Dialer) Latency(nodeID roachpb.NodeID) (time.Duration, error) { // Note that TracingInternalClient is not used to wrap the internalClientAdapter // - local RPCs don't need this tracing functionality. type TracingInternalClient struct { - roachpb.InternalClient + kvpb.InternalClient } // Batch overrides the Batch RPC client method and fills in tracing information. func (tic TracingInternalClient) Batch( - ctx context.Context, ba *roachpb.BatchRequest, opts ...grpc.CallOption, -) (*roachpb.BatchResponse, error) { + ctx context.Context, ba *kvpb.BatchRequest, opts ...grpc.CallOption, +) (*kvpb.BatchResponse, error) { sp := tracing.SpanFromContext(ctx) if sp != nil && !sp.IsNoop() { ba = ba.ShallowCopy() diff --git a/pkg/rpc/nodedialer/nodedialer_test.go b/pkg/rpc/nodedialer/nodedialer_test.go index f33bbe6f8809..a30f9c394f75 100644 --- a/pkg/rpc/nodedialer/nodedialer_test.go +++ b/pkg/rpc/nodedialer/nodedialer_test.go @@ -21,6 +21,7 @@ import ( circuit "github.com/cockroachdb/circuitbreaker" "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -564,53 +565,49 @@ func (hb *heartbeatService) Ping( }, nil } -var _ roachpb.InternalServer = &internalServer{} +var _ kvpb.InternalServer = &internalServer{} type internalServer struct{} -func (*internalServer) Batch( - context.Context, *roachpb.BatchRequest, -) (*roachpb.BatchResponse, error) { +func (*internalServer) Batch(context.Context, *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { return nil, nil } func (*internalServer) RangeLookup( - context.Context, *roachpb.RangeLookupRequest, -) (*roachpb.RangeLookupResponse, error) { + context.Context, *kvpb.RangeLookupRequest, +) (*kvpb.RangeLookupResponse, error) { panic("unimplemented") } -func (*internalServer) RangeFeed( - *roachpb.RangeFeedRequest, roachpb.Internal_RangeFeedServer, -) error { +func (*internalServer) RangeFeed(*kvpb.RangeFeedRequest, kvpb.Internal_RangeFeedServer) error { panic("unimplemented") } -func (s *internalServer) MuxRangeFeed(server roachpb.Internal_MuxRangeFeedServer) error { +func (s *internalServer) MuxRangeFeed(server kvpb.Internal_MuxRangeFeedServer) error { panic("implement me") } func (*internalServer) GossipSubscription( - *roachpb.GossipSubscriptionRequest, roachpb.Internal_GossipSubscriptionServer, + *kvpb.GossipSubscriptionRequest, kvpb.Internal_GossipSubscriptionServer, ) error { panic("unimplemented") } func (*internalServer) ResetQuorum( - context.Context, *roachpb.ResetQuorumRequest, -) (*roachpb.ResetQuorumResponse, error) { + context.Context, *kvpb.ResetQuorumRequest, +) (*kvpb.ResetQuorumResponse, error) { panic("unimplemented") } func (*internalServer) Join( - context.Context, *roachpb.JoinNodeRequest, -) (*roachpb.JoinNodeResponse, error) { + context.Context, *kvpb.JoinNodeRequest, +) (*kvpb.JoinNodeResponse, error) { panic("unimplemented") } func (*internalServer) TokenBucket( - ctx context.Context, in *roachpb.TokenBucketRequest, -) (*roachpb.TokenBucketResponse, error) { + ctx context.Context, in *kvpb.TokenBucketRequest, +) (*kvpb.TokenBucketResponse, error) { panic("unimplemented") } @@ -639,13 +636,13 @@ func (s *internalServer) SpanConfigConformance( } func (*internalServer) TenantSettings( - *roachpb.TenantSettingsRequest, roachpb.Internal_TenantSettingsServer, + *kvpb.TenantSettingsRequest, kvpb.Internal_TenantSettingsServer, ) error { panic("unimplemented") } func (*internalServer) GetRangeDescriptors( - *roachpb.GetRangeDescriptorsRequest, roachpb.Internal_GetRangeDescriptorsServer, + *kvpb.GetRangeDescriptorsRequest, kvpb.Internal_GetRangeDescriptorsServer, ) error { panic("unimplemented") } diff --git a/pkg/rpc/restricted_internal_client.go b/pkg/rpc/restricted_internal_client.go index ebc91a68a1be..884f608b8d07 100644 --- a/pkg/rpc/restricted_internal_client.go +++ b/pkg/rpc/restricted_internal_client.go @@ -13,11 +13,11 @@ package rpc import ( "context" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "google.golang.org/grpc" ) -// RestrictedInternalClient represents the part of the roachpb.InternalClient +// RestrictedInternalClient represents the part of the kvpb.InternalClient // interface used by the DistSender. Besides the auto-generated gRPC client, // this interface is also implemented by rpc.internalClientAdapter which // bypasses gRPC to call into the local Node. @@ -25,7 +25,7 @@ import ( // For a more contextualized explanation, see the comment that decorates // (*rpc.Context).loopbackDialFn. type RestrictedInternalClient interface { - Batch(ctx context.Context, in *roachpb.BatchRequest, opts ...grpc.CallOption) (*roachpb.BatchResponse, error) - RangeFeed(ctx context.Context, in *roachpb.RangeFeedRequest, opts ...grpc.CallOption) (roachpb.Internal_RangeFeedClient, error) - MuxRangeFeed(ctx context.Context, opts ...grpc.CallOption) (roachpb.Internal_MuxRangeFeedClient, error) + Batch(ctx context.Context, in *kvpb.BatchRequest, opts ...grpc.CallOption) (*kvpb.BatchResponse, error) + RangeFeed(ctx context.Context, in *kvpb.RangeFeedRequest, opts ...grpc.CallOption) (kvpb.Internal_RangeFeedClient, error) + MuxRangeFeed(ctx context.Context, opts ...grpc.CallOption) (kvpb.Internal_MuxRangeFeedClient, error) } diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index 429c8e870d90..20110d123001 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -114,6 +114,7 @@ go_library( "//pkg/kv/kvclient/kvtenant", "//pkg/kv/kvclient/rangefeed", "//pkg/kv/kvclient/rangestats", + "//pkg/kv/kvpb", "//pkg/kv/kvprober", "//pkg/kv/kvserver", "//pkg/kv/kvserver/allocator/allocatorimpl", @@ -441,6 +442,7 @@ go_test( "//pkg/jobs/jobspb", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/kv/kvserver/allocator", "//pkg/kv/kvserver/allocator/allocatorimpl", diff --git a/pkg/server/admin.go b/pkg/server/admin.go index ad808e98e69d..7d2370abf3b3 100644 --- a/pkg/server/admin.go +++ b/pkg/server/admin.go @@ -31,6 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" @@ -3259,8 +3260,8 @@ func (s *systemAdminServer) enqueueRangeLocal( // SendKVBatch proxies the given BatchRequest into KV, returning the // response. It is for use by the CLI `debug send-kv-batch` command. func (s *systemAdminServer) SendKVBatch( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, error) { ctx = s.AnnotateCtx(ctx) // Note: the root user will bypass SQL auth checks, which is useful in case of // a cluster outage. @@ -3297,7 +3298,7 @@ func (s *systemAdminServer) SendKVBatch( // setupSpanForIncomingRPC() call above; from now on the request is traced as // per the span we just created. ba.TraceInfo = nil - var br *roachpb.BatchResponse + var br *kvpb.BatchResponse // NB: wrapped to delay br evaluation to its value when returning. defer func() { var redact redactOpt @@ -3310,7 +3311,7 @@ func (s *systemAdminServer) SendKVBatch( }() br, pErr := s.db.NonTransactionalSender().Send(ctx, ba) if br == nil { - br = &roachpb.BatchResponse{} + br = &kvpb.BatchResponse{} } br.Error = pErr return br, nil diff --git a/pkg/server/authentication_test.go b/pkg/server/authentication_test.go index be8749e33c32..1e80969cefc3 100644 --- a/pkg/server/authentication_test.go +++ b/pkg/server/authentication_test.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/gossip" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" @@ -117,7 +118,7 @@ func TestSSLEnforcement(t *testing.T) { plainHTTPCfg.Insecure = true insecureContext := newRPCContext(plainHTTPCfg) - kvGet := &roachpb.GetRequest{} + kvGet := &kvpb.GetRequest{} kvGet.Key = roachpb.Key("/") for _, tc := range []struct { @@ -758,7 +759,7 @@ func TestGRPCAuthentication(t *testing.T) { return err }}, {"internal", func(ctx context.Context, conn *grpc.ClientConn) error { - _, err := roachpb.NewInternalClient(conn).Batch(ctx, &roachpb.BatchRequest{}) + _, err := kvpb.NewInternalClient(conn).Batch(ctx, &kvpb.BatchRequest{}) return err }}, {"perReplica", func(ctx context.Context, conn *grpc.ClientConn) error { diff --git a/pkg/server/bench_test.go b/pkg/server/bench_test.go index eace721642fe..d5194b7ed3bc 100644 --- a/pkg/server/bench_test.go +++ b/pkg/server/bench_test.go @@ -14,6 +14,7 @@ import ( "context" "testing" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -44,7 +45,7 @@ func BenchmarkSetupSpanForIncomingRPC(b *testing.B) { parentSpan := tr.StartSpan("parent") defer parentSpan.Finish() - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} if tc.traceInfo { ba.TraceInfo = parentSpan.Meta().ToProto() } else if tc.grpcMeta { diff --git a/pkg/server/config.go b/pkg/server/config.go index 0390fb1fc6a7..89cec41a9d3f 100644 --- a/pkg/server/config.go +++ b/pkg/server/config.go @@ -28,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/docs" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" @@ -522,7 +523,7 @@ type SQLConfig struct { // necessary for creating the internalClientAdapter for an in-process tenant // talking to that server. type LocalKVServerInfo struct { - InternalServer roachpb.InternalServer + InternalServer kvpb.InternalServer ServerInterceptors rpc.ServerInterceptorInfo Tracer *tracing.Tracer } diff --git a/pkg/server/import_ts.go b/pkg/server/import_ts.go index f1f8de5e6e01..7099f195586f 100644 --- a/pkg/server/import_ts.go +++ b/pkg/server/import_ts.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/status/statuspb" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" @@ -185,8 +186,8 @@ func maybeImportTS(ctx context.Context, s *Server) (returnErr error) { continue } - p := roachpb.NewPut(v.Key, v.Value) - p.(*roachpb.PutRequest).Inline = true + p := kvpb.NewPut(v.Key, v.Value) + p.(*kvpb.PutRequest).Inline = true batch.AddRawRequest(p) batchSize++ if err := maybeFlush(false /* force */); err != nil { diff --git a/pkg/server/init.go b/pkg/server/init.go index b9bc0fd28322..80f57a8abe0b 100644 --- a/pkg/server/init.go +++ b/pkg/server/init.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvstorage" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" @@ -447,7 +448,7 @@ func (s *initServer) startJoinLoop(ctx context.Context, stopper *stop.Stopper) ( // attemptJoinTo attempts to join to the node running at the given address. func (s *initServer) attemptJoinTo( ctx context.Context, addr string, -) (*roachpb.JoinNodeResponse, error) { +) (*kvpb.JoinNodeResponse, error) { dialOpts, err := s.config.getDialOpts(ctx, addr, rpc.SystemClass) if err != nil { return nil, err @@ -462,11 +463,11 @@ func (s *initServer) attemptJoinTo( }() binaryVersion := s.config.binaryVersion - req := &roachpb.JoinNodeRequest{ + req := &kvpb.JoinNodeRequest{ BinaryVersion: &binaryVersion, } - initClient := roachpb.NewInternalClient(conn) + initClient := kvpb.NewInternalClient(conn) resp, err := initClient.Join(ctx, req) if err != nil { status, ok := grpcstatus.FromError(errors.UnwrapAll(err)) @@ -521,7 +522,7 @@ func (s *initServer) DiskClusterVersion() clusterversion.ClusterVersion { // and persists the appropriate cluster version to disk. After having done so, // it returns an initState that captures the newly initialized store. func (s *initServer) initializeFirstStoreAfterJoin( - ctx context.Context, resp *roachpb.JoinNodeResponse, + ctx context.Context, resp *kvpb.JoinNodeResponse, ) (*initState, error) { // We expect all the stores to be empty at this point, except for // the store cluster version key. Assert so. diff --git a/pkg/server/intent_test.go b/pkg/server/intent_test.go index 8563192f6a27..83712575c7d1 100644 --- a/pkg/server/intent_test.go +++ b/pkg/server/intent_test.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -81,7 +82,7 @@ func TestIntentResolution(t *testing.T) { closer := make(chan struct{}, 2) var done bool storeKnobs.EvalKnobs.TestingEvalFilter = - func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { + func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { mu.Lock() defer mu.Unlock() header := filterArgs.Req.Header() @@ -91,11 +92,11 @@ func TestIntentResolution(t *testing.T) { } var entry string switch arg := filterArgs.Req.(type) { - case *roachpb.ResolveIntentRequest: + case *kvpb.ResolveIntentRequest: if arg.Status == roachpb.COMMITTED { entry = string(header.Key) } - case *roachpb.ResolveIntentRangeRequest: + case *kvpb.ResolveIntentRangeRequest: if arg.Status == roachpb.COMMITTED { entry = fmt.Sprintf("%s-%s", header.Key, header.EndKey) } diff --git a/pkg/server/node.go b/pkg/server/node.go index 34726d2f82fa..2e9dbbbe3c40 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -32,6 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvclient" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvtenant" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvadmission" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvstorage" @@ -173,7 +174,7 @@ type nodeMetrics struct { DiskStalls *metric.Counter BatchCount *metric.Counter - MethodCounts [roachpb.NumMethods]*metric.Counter + MethodCounts [kvpb.NumMethods]*metric.Counter } func makeNodeMetrics(reg *metric.Registry, histogramWindow time.Duration) nodeMetrics { @@ -191,7 +192,7 @@ func makeNodeMetrics(reg *metric.Registry, histogramWindow time.Duration) nodeMe } for i := range nm.MethodCounts { - method := roachpb.Method(i).String() + method := kvpb.Method(i).String() meta := metaInternalBatchRPCMethodCount meta.Name = fmt.Sprintf(meta.Name, strings.ToLower(method)) meta.Help = fmt.Sprintf(meta.Help, method) @@ -205,8 +206,8 @@ func makeNodeMetrics(reg *metric.Registry, histogramWindow time.Duration) nodeMe // callComplete records very high-level metrics about the number of completed // calls and their latency. Currently, this only records statistics at the batch // level; stats on specific lower-level kv operations are not recorded. -func (nm nodeMetrics) callComplete(d time.Duration, pErr *roachpb.Error) { - if pErr != nil && pErr.TransactionRestart() == roachpb.TransactionRestart_NONE { +func (nm nodeMetrics) callComplete(d time.Duration, pErr *kvpb.Error) { + if pErr != nil && pErr.TransactionRestart() == kvpb.TransactionRestart_NONE { nm.Err.Inc(1) } else { nm.Success.Inc(1) @@ -256,13 +257,13 @@ type Node struct { diskStatsMap diskStatsMap - testingErrorEvent func(context.Context, *roachpb.BatchRequest, error) + testingErrorEvent func(context.Context, *kvpb.BatchRequest, error) // Used to collect samples for the key visualizer. spanStatsCollector *spanstatscollector.SpanStatsCollector } -var _ roachpb.InternalServer = &Node{} +var _ kvpb.InternalServer = &Node{} // allocateNodeID increments the node id generator key to allocate // a new, unique node id. @@ -1092,29 +1093,29 @@ func (n *Node) logStructuredEvent(ctx context.Context, event logpb.EventPayload) ) } -// If we receive a (proto-marshaled) roachpb.BatchRequest whose Requests contain +// If we receive a (proto-marshaled) kvpb.BatchRequest whose Requests contain // a message type unknown to this node, we will end up with a zero entry in the // slice. If we don't error out early, this breaks all sorts of assumptions and // usually ends in a panic. -func checkNoUnknownRequest(reqs []roachpb.RequestUnion) *roachpb.UnsupportedRequestError { +func checkNoUnknownRequest(reqs []kvpb.RequestUnion) *kvpb.UnsupportedRequestError { for _, req := range reqs { if req.GetValue() == nil { - return &roachpb.UnsupportedRequestError{} + return &kvpb.UnsupportedRequestError{} } } return nil } func (n *Node) batchInternal( - ctx context.Context, tenID roachpb.TenantID, args *roachpb.BatchRequest, -) (*roachpb.BatchResponse, error) { + ctx context.Context, tenID roachpb.TenantID, args *kvpb.BatchRequest, +) (*kvpb.BatchResponse, error) { if detail := checkNoUnknownRequest(args.Requests); detail != nil { - var br roachpb.BatchResponse - br.Error = roachpb.NewError(detail) + var br kvpb.BatchResponse + br.Error = kvpb.NewError(detail) return &br, nil } - var br *roachpb.BatchResponse + var br *kvpb.BatchResponse var reqSp spanForRequest ctx, reqSp = n.setupSpanForIncomingRPC(ctx, tenID, args) // NB: wrapped to delay br evaluation to its value when returning. @@ -1145,10 +1146,10 @@ func (n *Node) batchInternal( n.storeCfg.KVAdmissionController.AdmittedKVWorkDone(handle, writeBytes) writeBytes.Release() }() - var pErr *roachpb.Error + var pErr *kvpb.Error br, writeBytes, pErr = n.stores.SendWithWriteBytes(ctx, args) if pErr != nil { - br = &roachpb.BatchResponse{} + br = &kvpb.BatchResponse{} if pErr.Index != nil && keyvissettings.Enabled.Get(&n.storeCfg.Settings.SV) { // Tell the SpanStatsCollector about the requests in this BatchRequest, // but stop when we reach the requests that were not attempted @@ -1172,7 +1173,7 @@ func (n *Node) batchInternal( } } if br.Error != nil { - panic(roachpb.ErrorUnexpectedlySet(n.stores, br)) + panic(kvpb.ErrorUnexpectedlySet(n.stores, br)) } n.metrics.callComplete(timeutil.Since(tStart), pErr) br.Error = pErr @@ -1182,7 +1183,7 @@ func (n *Node) batchInternal( // incrementBatchCounters increments counters to track the batch and composite // request methods. -func (n *Node) incrementBatchCounters(ba *roachpb.BatchRequest) { +func (n *Node) incrementBatchCounters(ba *kvpb.BatchRequest) { n.metrics.BatchCount.Inc(1) for _, ru := range ba.Requests { m := ru.GetInner().Method() @@ -1190,10 +1191,8 @@ func (n *Node) incrementBatchCounters(ba *roachpb.BatchRequest) { } } -// Batch implements the roachpb.InternalServer interface. -func (n *Node) Batch( - ctx context.Context, args *roachpb.BatchRequest, -) (*roachpb.BatchResponse, error) { +// Batch implements the kvpb.InternalServer interface. +func (n *Node) Batch(ctx context.Context, args *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { n.incrementBatchCounters(args) // NB: Node.Batch is called directly for "local" calls. We don't want to @@ -1225,14 +1224,14 @@ func (n *Node) Batch( // framework and not from cockroach. if err != nil { if br == nil { - br = &roachpb.BatchResponse{} + br = &kvpb.BatchResponse{} } if br.Error != nil { log.Fatalf( - ctx, "attempting to return both a plain error (%s) and roachpb.Error (%s)", err, br.Error, + ctx, "attempting to return both a plain error (%s) and kvpb.Error (%s)", err, br.Error, ) } - br.Error = roachpb.NewError(err) + br.Error = kvpb.NewError(err) } if buildutil.CrdbTestBuild && br.Error != nil && n.testingErrorEvent != nil { n.testingErrorEvent(ctx, args, errors.DecodeError(ctx, br.Error.EncodedError)) @@ -1259,7 +1258,7 @@ const ( // finish finishes the span. If the span was recording and br is not nil, the // recording is written to br.CollectedSpans. -func (sp *spanForRequest) finish(br *roachpb.BatchResponse, redactOpt redactOpt) { +func (sp *spanForRequest) finish(br *kvpb.BatchResponse, redactOpt redactOpt) { var rec tracingpb.Recording // If we don't have a response, there's nothing to attach a trace to. // Nothing more for us to do. @@ -1301,13 +1300,13 @@ func (sp *spanForRequest) finish(br *roachpb.BatchResponse, redactOpt redactOpt) // in which the response is to serialized. The BatchResponse can // be nil in case no response is to be returned to the rpc caller. func (n *Node) setupSpanForIncomingRPC( - ctx context.Context, tenID roachpb.TenantID, ba *roachpb.BatchRequest, + ctx context.Context, tenID roachpb.TenantID, ba *kvpb.BatchRequest, ) (context.Context, spanForRequest) { return setupSpanForIncomingRPC(ctx, tenID, ba, n.storeCfg.AmbientCtx.Tracer) } func setupSpanForIncomingRPC( - ctx context.Context, tenID roachpb.TenantID, ba *roachpb.BatchRequest, tr *tracing.Tracer, + ctx context.Context, tenID roachpb.TenantID, ba *kvpb.BatchRequest, tr *tracing.Tracer, ) (context.Context, spanForRequest) { var newSpan *tracing.Span remoteParent := !ba.TraceInfo.Empty() @@ -1378,10 +1377,10 @@ func filterRangeLookupResponseForTenant( return truncated } -// RangeLookup implements the roachpb.InternalServer interface. +// RangeLookup implements the kvpb.InternalServer interface. func (n *Node) RangeLookup( - ctx context.Context, req *roachpb.RangeLookupRequest, -) (*roachpb.RangeLookupResponse, error) { + ctx context.Context, req *kvpb.RangeLookupRequest, +) (*kvpb.RangeLookupResponse, error) { ctx = n.storeCfg.AmbientCtx.AnnotateCtx(ctx) // Proxy the RangeLookup through the local DB. Note that this does not use @@ -1399,9 +1398,9 @@ func (n *Node) RangeLookup( req.PrefetchNum, req.PrefetchReverse, ) - resp := new(roachpb.RangeLookupResponse) + resp := new(kvpb.RangeLookupResponse) if err != nil { - resp.Error = roachpb.NewError(err) + resp.Error = kvpb.NewError(err) } else { resp.Descriptors = rs resp.PrefetchedDescriptors = filterRangeLookupResponseForTenant(ctx, preRs) @@ -1409,20 +1408,16 @@ func (n *Node) RangeLookup( return resp, nil } -// RangeFeed implements the roachpb.InternalServer interface. -func (n *Node) RangeFeed( - args *roachpb.RangeFeedRequest, stream roachpb.Internal_RangeFeedServer, -) error { +// RangeFeed implements the kvpb.InternalServer interface. +func (n *Node) RangeFeed(args *kvpb.RangeFeedRequest, stream kvpb.Internal_RangeFeedServer) error { return n.singleRangeFeed(args, stream) } -func (n *Node) singleRangeFeed( - args *roachpb.RangeFeedRequest, stream roachpb.RangeFeedEventSink, -) error { +func (n *Node) singleRangeFeed(args *kvpb.RangeFeedRequest, stream kvpb.RangeFeedEventSink) error { pErr := n.stores.RangeFeed(args, stream) if pErr != nil { - var event roachpb.RangeFeedEvent - event.SetValue(&roachpb.RangeFeedError{ + var event kvpb.RangeFeedEvent + event.SetValue(&kvpb.RangeFeedError{ Error: *pErr, }) return stream.Send(&event) @@ -1445,8 +1440,8 @@ func (s *setRangeIDEventSink) Context() context.Context { return s.ctx } -func (s *setRangeIDEventSink) Send(event *roachpb.RangeFeedEvent) error { - response := &roachpb.MuxRangeFeedEvent{ +func (s *setRangeIDEventSink) Send(event *kvpb.RangeFeedEvent) error { + response := &kvpb.MuxRangeFeedEvent{ RangeFeedEvent: *event, RangeID: s.rangeID, StreamID: s.streamID, @@ -1454,12 +1449,12 @@ func (s *setRangeIDEventSink) Send(event *roachpb.RangeFeedEvent) error { return s.wrapped.Send(response) } -var _ roachpb.RangeFeedEventSink = (*setRangeIDEventSink)(nil) +var _ kvpb.RangeFeedEventSink = (*setRangeIDEventSink)(nil) // lockedMuxStream provides support for concurrent calls to Send. // The underlying MuxRangeFeedServer is not safe for concurrent calls to Send. type lockedMuxStream struct { - wrapped roachpb.Internal_MuxRangeFeedServer + wrapped kvpb.Internal_MuxRangeFeedServer sendMu syncutil.Mutex } @@ -1467,14 +1462,14 @@ func (s *lockedMuxStream) Context() context.Context { return s.wrapped.Context() } -func (s *lockedMuxStream) Send(e *roachpb.MuxRangeFeedEvent) error { +func (s *lockedMuxStream) Send(e *kvpb.MuxRangeFeedEvent) error { s.sendMu.Lock() defer s.sendMu.Unlock() return s.wrapped.Send(e) } -// MuxRangeFeed implements the roachpb.InternalServer interface. -func (n *Node) MuxRangeFeed(stream roachpb.Internal_MuxRangeFeedServer) error { +// MuxRangeFeed implements the kvpb.InternalServer interface. +func (n *Node) MuxRangeFeed(stream kvpb.Internal_MuxRangeFeedServer) error { ctx, cancelFeeds := n.stopper.WithCancelOnQuiesce(stream.Context()) defer cancelFeeds() rfGrp := ctxgroup.WithContext(ctx) @@ -1507,10 +1502,10 @@ func (n *Node) MuxRangeFeed(stream roachpb.Internal_MuxRangeFeedServer) error { } } -// ResetQuorum implements the roachpb.InternalServer interface. +// ResetQuorum implements the kvpb.InternalServer interface. func (n *Node) ResetQuorum( - ctx context.Context, req *roachpb.ResetQuorumRequest, -) (_ *roachpb.ResetQuorumResponse, rErr error) { + ctx context.Context, req *kvpb.ResetQuorumRequest, +) (_ *kvpb.ResetQuorumResponse, rErr error) { // Get range descriptor and save original value of the descriptor for the input range id. var desc roachpb.RangeDescriptor var expValue roachpb.Value @@ -1613,12 +1608,12 @@ func (n *Node) ResetQuorum( } log.Infof(ctx, "sent empty snapshot to %s", toReplicaDescriptor) - return &roachpb.ResetQuorumResponse{}, nil + return &kvpb.ResetQuorumResponse{}, nil } -// GossipSubscription implements the roachpb.InternalServer interface. +// GossipSubscription implements the kvpb.InternalServer interface. func (n *Node) GossipSubscription( - args *roachpb.GossipSubscriptionRequest, stream roachpb.Internal_GossipSubscriptionServer, + args *kvpb.GossipSubscriptionRequest, stream kvpb.Internal_GossipSubscriptionServer, ) error { ctx := n.storeCfg.AmbientCtx.AnnotateCtx(stream.Context()) ctxDone := ctx.Done() @@ -1632,7 +1627,7 @@ func (n *Node) GossipSubscription( // feels fragile, though, especially during the initial information dump. // Instead, we say that if the channel ever blocks for more than some // duration, terminate the subscription. - entC := make(chan *roachpb.GossipSubscriptionEvent, 256) + entC := make(chan *kvpb.GossipSubscriptionEvent, 256) entCClosed := false var callbackMu syncutil.Mutex var systemConfigUpdateCh <-chan struct{} @@ -1662,7 +1657,7 @@ func (n *Node) GossipSubscription( if entCClosed { return } - var event roachpb.GossipSubscriptionEvent + var event kvpb.GossipSubscriptionEvent event.Key = key event.Content = content event.PatternMatched = pattern @@ -1690,10 +1685,10 @@ func (n *Node) GossipSubscription( if isSecondaryTenant { ents = kvtenant.GossipSubscriptionSystemConfigMask.Apply(ents) } - var event roachpb.GossipSubscriptionEvent + var event kvpb.GossipSubscriptionEvent var content roachpb.Value if err := content.SetProto(&ents); err != nil { - event.Error = roachpb.NewError(errors.Wrap(err, "could not marshal system config")) + event.Error = kvpb.NewError(errors.Wrap(err, "could not marshal system config")) } else { event.Key = gossip.KeyDeprecatedSystemConfig event.Content = content @@ -1711,9 +1706,9 @@ func (n *Node) GossipSubscription( if !ok { // The consumer was not keeping up with gossip updates, so its // subscription was terminated to avoid blocking gossip. - err := roachpb.NewErrorf("subscription terminated due to slow consumption") + err := kvpb.NewErrorf("subscription terminated due to slow consumption") log.Warningf(ctx, "%v", err) - e = &roachpb.GossipSubscriptionEvent{Error: err} + e = &kvpb.GossipSubscriptionEvent{Error: err} } if err := stream.Send(e); err != nil { return err @@ -1726,23 +1721,23 @@ func (n *Node) GossipSubscription( } } -// TenantSettings implements the roachpb.InternalServer interface. +// TenantSettings implements the kvpb.InternalServer interface. func (n *Node) TenantSettings( - args *roachpb.TenantSettingsRequest, stream roachpb.Internal_TenantSettingsServer, + args *kvpb.TenantSettingsRequest, stream kvpb.Internal_TenantSettingsServer, ) error { ctx := n.storeCfg.AmbientCtx.AnnotateCtx(stream.Context()) ctxDone := ctx.Done() w := n.tenantSettingsWatcher if err := w.WaitForStart(ctx); err != nil { - return stream.Send(&roachpb.TenantSettingsEvent{ + return stream.Send(&kvpb.TenantSettingsEvent{ Error: errors.EncodeError(ctx, err), }) } - send := func(precedence roachpb.TenantSettingsPrecedence, overrides []roachpb.TenantSetting) error { + send := func(precedence kvpb.TenantSettingsPrecedence, overrides []kvpb.TenantSetting) error { log.VInfof(ctx, 1, "sending precedence %d: %v", precedence, overrides) - return stream.Send(&roachpb.TenantSettingsEvent{ + return stream.Send(&kvpb.TenantSettingsEvent{ Precedence: precedence, Incremental: false, Overrides: overrides, @@ -1750,12 +1745,12 @@ func (n *Node) TenantSettings( } allOverrides, allCh := w.GetAllTenantOverrides() - if err := send(roachpb.AllTenantsOverrides, allOverrides); err != nil { + if err := send(kvpb.AllTenantsOverrides, allOverrides); err != nil { return err } tenantOverrides, tenantCh := w.GetTenantOverrides(args.TenantID) - if err := send(roachpb.SpecificTenantOverrides, tenantOverrides); err != nil { + if err := send(kvpb.SpecificTenantOverrides, tenantOverrides); err != nil { return err } @@ -1764,14 +1759,14 @@ func (n *Node) TenantSettings( case <-allCh: // All-tenant overrides have changed, send them again. allOverrides, allCh = w.GetAllTenantOverrides() - if err := send(roachpb.AllTenantsOverrides, allOverrides); err != nil { + if err := send(kvpb.AllTenantsOverrides, allOverrides); err != nil { return err } case <-tenantCh: // Tenant-specific overrides have changed, send them again. tenantOverrides, tenantCh = w.GetTenantOverrides(args.TenantID) - if err := send(roachpb.SpecificTenantOverrides, tenantOverrides); err != nil { + if err := send(kvpb.SpecificTenantOverrides, tenantOverrides); err != nil { return err } @@ -1784,12 +1779,12 @@ func (n *Node) TenantSettings( } } -// Join implements the roachpb.InternalServer service. This is the +// Join implements the kvpb.InternalServer service. This is the // "connectivity" API; individual CRDB servers are passed in a --join list and // the join targets are addressed through this API. func (n *Node) Join( - ctx context.Context, req *roachpb.JoinNodeRequest, -) (*roachpb.JoinNodeResponse, error) { + ctx context.Context, req *kvpb.JoinNodeRequest, +) (*kvpb.JoinNodeResponse, error) { ctx, span := n.AnnotateCtxWithSpan(ctx, "alloc-{node,store}-id") defer span.Finish() @@ -1822,7 +1817,7 @@ func (n *Node) Join( log.Infof(ctx, "allocated IDs: n%d, s%d", nodeID, storeID) - return &roachpb.JoinNodeResponse{ + return &kvpb.JoinNodeResponse{ ClusterID: n.clusterID.Get().GetBytes(), NodeID: int32(nodeID), StoreID: int32(storeID), @@ -1830,14 +1825,14 @@ func (n *Node) Join( }, nil } -// TokenBucket is part of the roachpb.InternalServer service. +// TokenBucket is part of the kvpb.InternalServer service. func (n *Node) TokenBucket( - ctx context.Context, in *roachpb.TokenBucketRequest, -) (*roachpb.TokenBucketResponse, error) { + ctx context.Context, in *kvpb.TokenBucketRequest, +) (*kvpb.TokenBucketResponse, error) { // Check tenant ID. Note that in production configuration, the tenant ID has // already been checked in the RPC layer (see rpc.tenantAuthorizer). if in.TenantID == 0 || in.TenantID == roachpb.SystemTenantID.ToUint64() { - return &roachpb.TokenBucketResponse{ + return &kvpb.TokenBucketResponse{ Error: errors.EncodeError(ctx, errors.Errorf( "token bucket request with invalid tenant ID %d", in.TenantID, )), @@ -1863,9 +1858,9 @@ type dummyTenantUsageServer struct{} // TokenBucketRequest is defined in the TenantUsageServer interface. func (dummyTenantUsageServer) TokenBucketRequest( - ctx context.Context, tenantID roachpb.TenantID, in *roachpb.TokenBucketRequest, -) *roachpb.TokenBucketResponse { - return &roachpb.TokenBucketResponse{ + ctx context.Context, tenantID roachpb.TenantID, in *kvpb.TokenBucketRequest, +) *kvpb.TokenBucketResponse { + return &kvpb.TokenBucketResponse{ Error: errors.EncodeError(ctx, errors.New("tenant usage requires a CCL binary")), } } @@ -1895,7 +1890,7 @@ var _ metric.Struct = emptyMetricStruct{} func (emptyMetricStruct) MetricStruct() {} -// GetSpanConfigs implements the roachpb.InternalServer interface. +// GetSpanConfigs implements the kvpb.InternalServer interface. func (n *Node) GetSpanConfigs( ctx context.Context, req *roachpb.GetSpanConfigsRequest, ) (*roachpb.GetSpanConfigsResponse, error) { @@ -1913,7 +1908,7 @@ func (n *Node) GetSpanConfigs( }, nil } -// GetAllSystemSpanConfigsThatApply implements the roachpb.InternalServer +// GetAllSystemSpanConfigsThatApply implements the kvpb.InternalServer // interface. func (n *Node) GetAllSystemSpanConfigsThatApply( ctx context.Context, req *roachpb.GetAllSystemSpanConfigsThatApplyRequest, @@ -1928,7 +1923,7 @@ func (n *Node) GetAllSystemSpanConfigsThatApply( }, nil } -// UpdateSpanConfigs implements the roachpb.InternalServer interface. +// UpdateSpanConfigs implements the kvpb.InternalServer interface. func (n *Node) UpdateSpanConfigs( ctx context.Context, req *roachpb.UpdateSpanConfigsRequest, ) (*roachpb.UpdateSpanConfigsResponse, error) { @@ -1950,7 +1945,7 @@ func (n *Node) UpdateSpanConfigs( return &roachpb.UpdateSpanConfigsResponse{}, nil } -// SpanConfigConformance implements the roachpb.InternalServer interface. +// SpanConfigConformance implements the kvpb.InternalServer interface. func (n *Node) SpanConfigConformance( ctx context.Context, req *roachpb.SpanConfigConformanceRequest, ) (*roachpb.SpanConfigConformanceResponse, error) { @@ -1965,9 +1960,9 @@ func (n *Node) SpanConfigConformance( return &roachpb.SpanConfigConformanceResponse{Report: report}, nil } -// GetRangeDescriptors implements the roachpb.InternalServer interface. +// GetRangeDescriptors implements the kvpb.InternalServer interface. func (n *Node) GetRangeDescriptors( - args *roachpb.GetRangeDescriptorsRequest, stream roachpb.Internal_GetRangeDescriptorsServer, + args *kvpb.GetRangeDescriptorsRequest, stream kvpb.Internal_GetRangeDescriptorsServer, ) error { iter, err := n.execCfg.RangeDescIteratorFactory.NewIterator(stream.Context(), args.Span) if err != nil { @@ -1980,7 +1975,7 @@ func (n *Node) GetRangeDescriptors( iter.Next() } - return stream.Send(&roachpb.GetRangeDescriptorsResponse{ + return stream.Send(&kvpb.GetRangeDescriptorsResponse{ RangeDescriptors: rangeDescriptors, }) } diff --git a/pkg/server/node_test.go b/pkg/server/node_test.go index 41b0657c4770..cdb41f04ff9a 100644 --- a/pkg/server/node_test.go +++ b/pkg/server/node_test.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvstorage" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -641,8 +642,8 @@ func TestNodeSendUnknownBatchRequest(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - ba := roachpb.BatchRequest{ - Requests: make([]roachpb.RequestUnion, 1), + ba := kvpb.BatchRequest{ + Requests: make([]kvpb.RequestUnion, 1), } n := &Node{} br, err := n.batchInternal(context.Background(), roachpb.SystemTenantID, &ba) @@ -652,7 +653,7 @@ func TestNodeSendUnknownBatchRequest(t *testing.T) { if br.Error == nil { t.Fatal("no batch error returned") } - if _, ok := br.Error.GetDetail().(*roachpb.UnsupportedRequestError); !ok { + if _, ok := br.Error.GetDetail().(*kvpb.UnsupportedRequestError); !ok { t.Fatalf("expected unsupported request, not %v", br.Error) } } @@ -667,15 +668,15 @@ func TestNodeBatchRequestMetricsInc(t *testing.T) { n := ts.GetNode() bCurr := n.metrics.BatchCount.Count() - getCurr := n.metrics.MethodCounts[roachpb.Get].Count() - putCurr := n.metrics.MethodCounts[roachpb.Put].Count() + getCurr := n.metrics.MethodCounts[kvpb.Get].Count() + putCurr := n.metrics.MethodCounts[kvpb.Put].Count() - var ba roachpb.BatchRequest + var ba kvpb.BatchRequest ba.RangeID = 1 ba.Replica.StoreID = 1 - gr := roachpb.NewGet(roachpb.Key("a"), false) - pr := roachpb.NewPut(gr.Header().Key, roachpb.Value{}) + gr := kvpb.NewGet(roachpb.Key("a"), false) + pr := kvpb.NewPut(gr.Header().Key, roachpb.Value{}) ba.Add(gr, pr) _, _ = n.Batch(context.Background(), &ba) @@ -684,8 +685,8 @@ func TestNodeBatchRequestMetricsInc(t *testing.T) { putCurr++ require.GreaterOrEqual(t, n.metrics.BatchCount.Count(), bCurr) - require.GreaterOrEqual(t, n.metrics.MethodCounts[roachpb.Get].Count(), getCurr) - require.GreaterOrEqual(t, n.metrics.MethodCounts[roachpb.Put].Count(), putCurr) + require.GreaterOrEqual(t, n.metrics.MethodCounts[kvpb.Get].Count(), getCurr) + require.GreaterOrEqual(t, n.metrics.MethodCounts[kvpb.Put].Count(), putCurr) } func TestGetTenantWeights(t *testing.T) { diff --git a/pkg/server/server.go b/pkg/server/server.go index a04b489c3d34..d5382c00f7f5 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -35,6 +35,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangestats" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvprober" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" @@ -814,7 +815,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { spanConfig.kvAccessor, spanConfig.reporter, ) - roachpb.RegisterInternalServer(grpcServer.Server, node) + kvpb.RegisterInternalServer(grpcServer.Server, node) kvserver.RegisterPerReplicaServer(grpcServer.Server, node.perReplicaServer) kvserver.RegisterPerStoreServer(grpcServer.Server, node.perReplicaServer) ctpb.RegisterSideTransportServer(grpcServer.Server, ctReceiver) diff --git a/pkg/server/server_test.go b/pkg/server/server_test.go index eeea6f3e6add..d3bda2ab0748 100644 --- a/pkg/server/server_test.go +++ b/pkg/server/server_test.go @@ -36,6 +36,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -221,8 +222,8 @@ func TestServerStartClock(t *testing.T) { // Run a command so that we are sure to touch the timestamp cache. This is // actually not needed because other commands run during server // initialization, but we cannot guarantee that's going to stay that way. - get := &roachpb.GetRequest{ - RequestHeader: roachpb.RequestHeader{Key: roachpb.Key("a")}, + get := &kvpb.GetRequest{ + RequestHeader: kvpb.RequestHeader{Key: roachpb.Key("a")}, } if _, err := kv.SendWrapped( context.Background(), s.DB().NonTransactionalSender(), get, diff --git a/pkg/server/serverpb/BUILD.bazel b/pkg/server/serverpb/BUILD.bazel index 4d99a58a772b..18cd71c48c79 100644 --- a/pkg/server/serverpb/BUILD.bazel +++ b/pkg/server/serverpb/BUILD.bazel @@ -23,6 +23,7 @@ proto_library( "//pkg/config/zonepb:zonepb_proto", "//pkg/gossip:gossip_proto", "//pkg/jobs/jobspb:jobspb_proto", + "//pkg/kv/kvpb:kvpb_proto", "//pkg/kv/kvserver/kvserverpb:kvserverpb_proto", "//pkg/kv/kvserver/liveness/livenesspb:livenesspb_proto", "//pkg/kv/kvserver/loqrecovery/loqrecoverypb:loqrecoverypb_proto", @@ -63,6 +64,7 @@ go_proto_library( "//pkg/config/zonepb", "//pkg/gossip", "//pkg/jobs/jobspb", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/kvserverpb", "//pkg/kv/kvserver/liveness/livenesspb", "//pkg/kv/kvserver/loqrecovery/loqrecoverypb", diff --git a/pkg/server/serverpb/admin.proto b/pkg/server/serverpb/admin.proto index f6c7c0eda8e5..dd7ef20bbca6 100644 --- a/pkg/server/serverpb/admin.proto +++ b/pkg/server/serverpb/admin.proto @@ -20,7 +20,7 @@ import "storage/enginepb/mvcc.proto"; import "kv/kvserver/liveness/livenesspb/liveness.proto"; import "kv/kvserver/loqrecovery/loqrecoverypb/recovery.proto"; import "kv/kvserver/kvserverpb/range_log.proto"; -import "roachpb/api.proto"; +import "kv/kvpb/api.proto"; import "roachpb/metadata.proto"; import "roachpb/data.proto"; import "ts/catalog/chart_catalog.proto"; diff --git a/pkg/server/settingswatcher/BUILD.bazel b/pkg/server/settingswatcher/BUILD.bazel index 5fd05663112e..5e57b423ccf5 100644 --- a/pkg/server/settingswatcher/BUILD.bazel +++ b/pkg/server/settingswatcher/BUILD.bazel @@ -17,6 +17,7 @@ go_library( "//pkg/kv/kvclient/rangefeed", "//pkg/kv/kvclient/rangefeed/rangefeedbuffer", "//pkg/kv/kvclient/rangefeed/rangefeedcache", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/settings", "//pkg/settings/cluster", @@ -52,6 +53,7 @@ go_test( "//pkg/kv", "//pkg/kv/kvclient/rangefeed", "//pkg/kv/kvclient/rangefeed/rangefeedcache", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/roachpb", "//pkg/security/securityassets", diff --git a/pkg/server/settingswatcher/settings_watcher.go b/pkg/server/settingswatcher/settings_watcher.go index f98df151e706..7f4d5050a0bd 100644 --- a/pkg/server/settingswatcher/settings_watcher.go +++ b/pkg/server/settingswatcher/settings_watcher.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedbuffer" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedcache" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -196,7 +197,7 @@ func (s *SettingsWatcher) Start(ctx context.Context) error { bufferSize, []roachpb.Span{settingsTableSpan}, false, // withPrevValue - func(ctx context.Context, kv *roachpb.RangeFeedValue) rangefeedbuffer.Event { + func(ctx context.Context, kv *kvpb.RangeFeedValue) rangefeedbuffer.Event { return s.handleKV(ctx, kv) }, func(ctx context.Context, update rangefeedcache.Update) { @@ -233,7 +234,7 @@ func (s *SettingsWatcher) Start(ctx context.Context) error { } func (s *SettingsWatcher) handleKV( - ctx context.Context, kv *roachpb.RangeFeedValue, + ctx context.Context, kv *kvpb.RangeFeedValue, ) rangefeedbuffer.Event { var alloc tree.DatumAlloc name, val, tombstone, err := s.dec.DecodeRow(roachpb.KeyValue{ diff --git a/pkg/server/settingswatcher/settings_watcher_external_test.go b/pkg/server/settingswatcher/settings_watcher_external_test.go index bae03e6fc6d7..e8776c5be430 100644 --- a/pkg/server/settingswatcher/settings_watcher_external_test.go +++ b/pkg/server/settingswatcher/settings_watcher_external_test.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedcache" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/settingswatcher" @@ -447,12 +448,12 @@ func TestStaleRowsDoNotCauseSettingsToRegress(t *testing.T) { bogusTenantID := roachpb.MustMakeTenantID(42) bogusCodec := keys.MakeSQLCodec(bogusTenantID) settingsStart := bogusCodec.TablePrefix(keys.SettingsTableID) - interceptedStreamCh := make(chan roachpb.RangeFeedEventSink) + interceptedStreamCh := make(chan kvpb.RangeFeedEventSink) cancelCtx, cancel := context.WithCancel(ctx) s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{ Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ - TestingRangefeedFilter: func(args *roachpb.RangeFeedRequest, stream roachpb.RangeFeedEventSink) *roachpb.Error { + TestingRangefeedFilter: func(args *kvpb.RangeFeedRequest, stream kvpb.RangeFeedEventSink) *kvpb.Error { if !args.Span.ContainsKey(settingsStart) { return nil } @@ -511,13 +512,13 @@ func TestStaleRowsDoNotCauseSettingsToRegress(t *testing.T) { // newRangeFeedEvent creates a RangeFeedEvent for the bogus tenant using a KV // which has a stripped prefix. It also sets the timestamp. - newRangeFeedEvent := func(kv roachpb.KeyValue, ts hlc.Timestamp) *roachpb.RangeFeedEvent { + newRangeFeedEvent := func(kv roachpb.KeyValue, ts hlc.Timestamp) *kvpb.RangeFeedEvent { kv.Key = append(bogusCodec.TenantPrefix(), kv.Key...) kv.Value.Timestamp = ts kv.Value.ClearChecksum() kv.Value.InitChecksum(kv.Key) - return &roachpb.RangeFeedEvent{ - Val: &roachpb.RangeFeedValue{Key: kv.Key, Value: kv.Value}, + return &kvpb.RangeFeedEvent{ + Val: &kvpb.RangeFeedValue{Key: kv.Key, Value: kv.Value}, } } sideSettings := cluster.MakeTestingClusterSettings() diff --git a/pkg/server/status/BUILD.bazel b/pkg/server/status/BUILD.bazel index 1ba9954cb2a9..af001f860c9e 100644 --- a/pkg/server/status/BUILD.bazel +++ b/pkg/server/status/BUILD.bazel @@ -44,6 +44,7 @@ go_library( "//pkg/build", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/liveness", "//pkg/roachpb", "//pkg/rpc", diff --git a/pkg/server/status/recorder.go b/pkg/server/status/recorder.go index d324adfcbf48..5d01268785b5 100644 --- a/pkg/server/status/recorder.go +++ b/pkg/server/status/recorder.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/build" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" @@ -511,7 +512,7 @@ func (mr *MetricsRecorder) WriteNodeStatus( return errors.New("status entry not found, node may have been decommissioned") } err = db.CPutInline(ctx, key, &nodeStatus, entry.Value.TagAndDataBytes()) - if detail := (*roachpb.ConditionFailedError)(nil); errors.As(err, &detail) { + if detail := (*kvpb.ConditionFailedError)(nil); errors.As(err, &detail) { if detail.ActualValue == nil { return errors.New("status entry not found, node may have been decommissioned") } diff --git a/pkg/server/systemconfigwatcher/BUILD.bazel b/pkg/server/systemconfigwatcher/BUILD.bazel index 334910608eda..b033067609f4 100644 --- a/pkg/server/systemconfigwatcher/BUILD.bazel +++ b/pkg/server/systemconfigwatcher/BUILD.bazel @@ -13,6 +13,7 @@ go_library( "//pkg/kv/kvclient/rangefeed", "//pkg/kv/kvclient/rangefeed/rangefeedbuffer", "//pkg/kv/kvclient/rangefeed/rangefeedcache", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/util/hlc", "//pkg/util/stop", diff --git a/pkg/server/systemconfigwatcher/cache.go b/pkg/server/systemconfigwatcher/cache.go index f946fa669f98..595ab4113280 100644 --- a/pkg/server/systemconfigwatcher/cache.go +++ b/pkg/server/systemconfigwatcher/cache.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedbuffer" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedcache" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/stop" @@ -267,9 +268,7 @@ func (c *Cache) setUpdatedConfigLocked(updated *config.SystemConfig, ts hlc.Time } } -func passThroughTranslation( - ctx context.Context, value *roachpb.RangeFeedValue, -) rangefeedbuffer.Event { +func passThroughTranslation(ctx context.Context, value *kvpb.RangeFeedValue) rangefeedbuffer.Event { return value } diff --git a/pkg/server/systemconfigwatcher/systemconfigwatchertest/BUILD.bazel b/pkg/server/systemconfigwatcher/systemconfigwatchertest/BUILD.bazel index 29bcbf565106..3668f9c241e9 100644 --- a/pkg/server/systemconfigwatcher/systemconfigwatchertest/BUILD.bazel +++ b/pkg/server/systemconfigwatcher/systemconfigwatchertest/BUILD.bazel @@ -12,6 +12,7 @@ go_library( "//pkg/keys", "//pkg/kv", "//pkg/kv/kvclient/kvtenant", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/sql", "//pkg/testutils", diff --git a/pkg/server/systemconfigwatcher/systemconfigwatchertest/test_system_config_watcher.go b/pkg/server/systemconfigwatcher/systemconfigwatchertest/test_system_config_watcher.go index c5306620a6dd..06a0e7c77b89 100644 --- a/pkg/server/systemconfigwatcher/systemconfigwatchertest/test_system_config_watcher.go +++ b/pkg/server/systemconfigwatcher/systemconfigwatchertest/test_system_config_watcher.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvtenant" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -131,9 +132,9 @@ func getSystemDescriptorAndZonesSpans( ctx context.Context, t *testing.T, codec keys.SQLCodec, kvDB *kv.DB, ) []roachpb.KeyValue { scanSpanForRows := func(startKey, endKey roachpb.Key) (rows []roachpb.KeyValue) { - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Add( - roachpb.NewScan( + kvpb.NewScan( append(codec.TenantPrefix(), startKey...), append(codec.TenantPrefix(), endKey...), false, // forUpdate diff --git a/pkg/server/tenantsettingswatcher/BUILD.bazel b/pkg/server/tenantsettingswatcher/BUILD.bazel index f48a798ded8c..ec297ba4ebcb 100644 --- a/pkg/server/tenantsettingswatcher/BUILD.bazel +++ b/pkg/server/tenantsettingswatcher/BUILD.bazel @@ -16,6 +16,7 @@ go_library( "//pkg/kv/kvclient/rangefeed", "//pkg/kv/kvclient/rangefeed/rangefeedbuffer", "//pkg/kv/kvclient/rangefeed/rangefeedcache", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/settings", "//pkg/settings/cluster", @@ -46,6 +47,7 @@ go_test( deps = [ "//pkg/base", "//pkg/keys", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/security/securityassets", "//pkg/security/securitytest", diff --git a/pkg/server/tenantsettingswatcher/overrides_store.go b/pkg/server/tenantsettingswatcher/overrides_store.go index 91ac45195607..b7f0ee9922a3 100644 --- a/pkg/server/tenantsettingswatcher/overrides_store.go +++ b/pkg/server/tenantsettingswatcher/overrides_store.go @@ -13,6 +13,7 @@ package tenantsettingswatcher import ( "sort" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -46,14 +47,14 @@ type overridesStore struct { // all-tenant overrides). It is an immutable data structure. type tenantOverrides struct { // overrides, ordered by Name. - overrides []roachpb.TenantSetting + overrides []kvpb.TenantSetting // changeCh is a channel that is closed when the tenant overrides change (in // which case a new tenantOverrides object will contain the updated settings). changeCh chan struct{} } -func newTenantOverrides(overrides []roachpb.TenantSetting) *tenantOverrides { +func newTenantOverrides(overrides []kvpb.TenantSetting) *tenantOverrides { return &tenantOverrides{ overrides: overrides, changeCh: make(chan struct{}), @@ -72,7 +73,7 @@ func (s *overridesStore) Init() { // // This method is called once we complete a full initial scan of the // tenant_setting table. -func (s *overridesStore) SetAll(allOverrides map[roachpb.TenantID][]roachpb.TenantSetting) { +func (s *overridesStore) SetAll(allOverrides map[roachpb.TenantID][]kvpb.TenantSetting) { s.mu.Lock() defer s.mu.Unlock() @@ -123,17 +124,15 @@ func (s *overridesStore) GetTenantOverrides(tenantID roachpb.TenantID) *tenantOv // SetTenantOverride changes an override for the given tenant. If the setting // has an empty value, the existing override is removed; otherwise a new // override is added. -func (s *overridesStore) SetTenantOverride( - tenantID roachpb.TenantID, setting roachpb.TenantSetting, -) { +func (s *overridesStore) SetTenantOverride(tenantID roachpb.TenantID, setting kvpb.TenantSetting) { s.mu.Lock() defer s.mu.Unlock() - var before []roachpb.TenantSetting + var before []kvpb.TenantSetting if existing, ok := s.mu.tenants[tenantID]; ok { before = existing.overrides close(existing.changeCh) } - after := make([]roachpb.TenantSetting, 0, len(before)+1) + after := make([]kvpb.TenantSetting, 0, len(before)+1) // 1. Add all settings up to setting.Name. for len(before) > 0 && before[0].Name < setting.Name { after = append(after, before[0]) diff --git a/pkg/server/tenantsettingswatcher/overrides_store_test.go b/pkg/server/tenantsettingswatcher/overrides_store_test.go index 9e6f4d877b80..51e127189e4b 100644 --- a/pkg/server/tenantsettingswatcher/overrides_store_test.go +++ b/pkg/server/tenantsettingswatcher/overrides_store_test.go @@ -16,6 +16,7 @@ import ( "testing" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -28,8 +29,8 @@ func TestOverridesStore(t *testing.T) { s.Init() t1 := roachpb.MustMakeTenantID(1) t2 := roachpb.MustMakeTenantID(2) - st := func(name, val string) roachpb.TenantSetting { - return roachpb.TenantSetting{ + st := func(name, val string) kvpb.TenantSetting { + return kvpb.TenantSetting{ Name: name, Value: settings.EncodedValue{ Value: val, @@ -56,7 +57,7 @@ func TestOverridesStore(t *testing.T) { } o1 := s.GetTenantOverrides(t1) expect(o1, "") - s.SetAll(map[roachpb.TenantID][]roachpb.TenantSetting{ + s.SetAll(map[roachpb.TenantID][]kvpb.TenantSetting{ t1: {st("a", "aa"), st("b", "bb"), st("d", "dd")}, t2: {st("x", "xx")}, }) diff --git a/pkg/server/tenantsettingswatcher/row_decoder.go b/pkg/server/tenantsettingswatcher/row_decoder.go index 2ed719faedee..9e6a279e1629 100644 --- a/pkg/server/tenantsettingswatcher/row_decoder.go +++ b/pkg/server/tenantsettingswatcher/row_decoder.go @@ -12,6 +12,7 @@ package tenantsettingswatcher import ( "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" @@ -45,22 +46,22 @@ func MakeRowDecoder() RowDecoder { // set. func (d *RowDecoder) DecodeRow( kv roachpb.KeyValue, -) (_ roachpb.TenantID, _ roachpb.TenantSetting, tombstone bool, _ error) { +) (_ roachpb.TenantID, _ kvpb.TenantSetting, tombstone bool, _ error) { // First we need to decode the setting name field from the index key. keyTypes := []*types.T{d.columns[0].GetType(), d.columns[1].GetType()} keyVals := make([]rowenc.EncDatum, 2) _, _, err := rowenc.DecodeIndexKey(keys.SystemSQLCodec, keyTypes, keyVals, nil, kv.Key) if err != nil { - return roachpb.TenantID{}, roachpb.TenantSetting{}, false, errors.Wrap(err, "failed to decode key") + return roachpb.TenantID{}, kvpb.TenantSetting{}, false, errors.Wrap(err, "failed to decode key") } for i := range keyVals { if err := keyVals[i].EnsureDecoded(keyTypes[i], &d.alloc); err != nil { - return roachpb.TenantID{}, roachpb.TenantSetting{}, false, err + return roachpb.TenantID{}, kvpb.TenantSetting{}, false, err } } // We do not use MustMakeTenantID because we want to tolerate the 0 value. tenantID := roachpb.TenantID{InternalValue: uint64(tree.MustBeDInt(keyVals[0].Datum))} - var setting roachpb.TenantSetting + var setting kvpb.TenantSetting setting.Name = string(tree.MustBeDString(keyVals[1].Datum)) if !kv.Value.IsPresent() { return tenantID, setting, true, nil @@ -69,12 +70,12 @@ func (d *RowDecoder) DecodeRow( // The rest of the columns are stored as a family. bytes, err := kv.Value.GetTuple() if err != nil { - return roachpb.TenantID{}, roachpb.TenantSetting{}, false, err + return roachpb.TenantID{}, kvpb.TenantSetting{}, false, err } datums, err := d.decoder.Decode(&d.alloc, bytes) if err != nil { - return roachpb.TenantID{}, roachpb.TenantSetting{}, false, err + return roachpb.TenantID{}, kvpb.TenantSetting{}, false, err } if value := datums[2]; value != tree.DNull { diff --git a/pkg/server/tenantsettingswatcher/watcher.go b/pkg/server/tenantsettingswatcher/watcher.go index 5fa84f73ac00..eaa13cea6962 100644 --- a/pkg/server/tenantsettingswatcher/watcher.go +++ b/pkg/server/tenantsettingswatcher/watcher.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedbuffer" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedcache" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog" @@ -111,9 +112,9 @@ func (w *Watcher) startRangeFeed( ch: make(chan struct{}), } - allOverrides := make(map[roachpb.TenantID][]roachpb.TenantSetting) + allOverrides := make(map[roachpb.TenantID][]kvpb.TenantSetting) - translateEvent := func(ctx context.Context, kv *roachpb.RangeFeedValue) rangefeedbuffer.Event { + translateEvent := func(ctx context.Context, kv *kvpb.RangeFeedValue) rangefeedbuffer.Event { tenantID, setting, tombstone, err := w.dec.DecodeRow(roachpb.KeyValue{ Key: kv.Key, Value: kv.Value, @@ -158,7 +159,7 @@ func (w *Watcher) startRangeFeed( close(initialScan.ch) } else { // The rangefeed will be restarted and will scan the table anew. - allOverrides = make(map[roachpb.TenantID][]roachpb.TenantSetting) + allOverrides = make(map[roachpb.TenantID][]kvpb.TenantSetting) } } @@ -217,7 +218,7 @@ func (w *Watcher) WaitForStart(ctx context.Context) error { // The caller must not modify the returned overrides slice. func (w *Watcher) GetTenantOverrides( tenantID roachpb.TenantID, -) (overrides []roachpb.TenantSetting, changeCh <-chan struct{}) { +) (overrides []kvpb.TenantSetting, changeCh <-chan struct{}) { o := w.store.GetTenantOverrides(tenantID) return o.overrides, o.changeCh } @@ -228,7 +229,7 @@ func (w *Watcher) GetTenantOverrides( // // The caller must not modify the returned overrides slice. func (w *Watcher) GetAllTenantOverrides() ( - overrides []roachpb.TenantSetting, + overrides []kvpb.TenantSetting, changeCh <-chan struct{}, ) { return w.GetTenantOverrides(allTenantOverridesID) diff --git a/pkg/server/tenantsettingswatcher/watcher_test.go b/pkg/server/tenantsettingswatcher/watcher_test.go index a6a26ab13dd5..2ad6864073be 100644 --- a/pkg/server/tenantsettingswatcher/watcher_test.go +++ b/pkg/server/tenantsettingswatcher/watcher_test.go @@ -18,6 +18,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/tenantsettingswatcher" "github.com/cockroachdb/cockroach/pkg/sql" @@ -55,7 +56,7 @@ func TestWatcher(t *testing.T) { err = w.WaitForStart(ctx) require.NoError(t, err) - expect := func(overrides []roachpb.TenantSetting, expected string) { + expect := func(overrides []kvpb.TenantSetting, expected string) { t.Helper() var vals []string for _, s := range overrides { diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index 65fa0df772ab..dddcf7f1310a 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -34,6 +34,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -1416,7 +1417,7 @@ func (ts *TestServer) GetFirstStoreID() roachpb.StoreID { // LookupRange returns the descriptor of the range containing key. func (ts *TestServer) LookupRange(key roachpb.Key) (roachpb.RangeDescriptor, error) { rs, _, err := kv.RangeLookup(context.Background(), ts.DB().NonTransactionalSender(), - key, roachpb.CONSISTENT, 0 /* prefetchNum */, false /* reverse */) + key, kvpb.CONSISTENT, 0 /* prefetchNum */, false /* reverse */) if err != nil { return roachpb.RangeDescriptor{}, errors.Wrapf( err, "%q: lookup range unexpected error", key) @@ -1428,8 +1429,8 @@ func (ts *TestServer) LookupRange(key roachpb.Key) (roachpb.RangeDescriptor, err func (ts *TestServer) MergeRanges(leftKey roachpb.Key) (roachpb.RangeDescriptor, error) { ctx := context.Background() - mergeReq := roachpb.AdminMergeRequest{ - RequestHeader: roachpb.RequestHeader{ + mergeReq := kvpb.AdminMergeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: leftKey, }, } @@ -1454,8 +1455,8 @@ func (ts *TestServer) SplitRangeWithExpiration( splitKey roachpb.Key, expirationTime hlc.Timestamp, ) (roachpb.RangeDescriptor, roachpb.RangeDescriptor, error) { ctx := context.Background() - splitReq := roachpb.AdminSplitRequest{ - RequestHeader: roachpb.RequestHeader{ + splitReq := kvpb.AdminSplitRequest{ + RequestHeader: kvpb.RequestHeader{ Key: splitKey, }, SplitKey: splitKey, @@ -1490,7 +1491,7 @@ func (ts *TestServer) SplitRangeWithExpiration( // i.e. looking up key `c` will match range [a,c), not [c, d). // The result will be the right descriptor, and the first prefetched result will // be the left neighbor, i.e. the resulting left hand side of the split. - rs, more, err := kv.RangeLookup(ctx, txn, splitKey.Next(), roachpb.CONSISTENT, 1, true /* reverse */) + rs, more, err := kv.RangeLookup(ctx, txn, splitKey.Next(), kvpb.CONSISTENT, 1, true /* reverse */) if err != nil { return err } @@ -1568,21 +1569,21 @@ const ( func (ts *TestServer) GetRangeLease( ctx context.Context, key roachpb.Key, queryPolicy LeaseInfoOpt, ) (_ LeaseInfo, now hlc.ClockTimestamp, _ error) { - leaseReq := roachpb.LeaseInfoRequest{ - RequestHeader: roachpb.RequestHeader{ + leaseReq := kvpb.LeaseInfoRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, }, } leaseResp, pErr := kv.SendWrappedWith( ctx, ts.DB().NonTransactionalSender(), - roachpb.Header{ + kvpb.Header{ // INCONSISTENT read with a NEAREST routing policy, since we want to make // sure that the node used to send this is the one that processes the // command, regardless of whether it is the leaseholder, for the hint to // matter. - ReadConsistency: roachpb.INCONSISTENT, - RoutingPolicy: roachpb.RoutingPolicy_NEAREST, + ReadConsistency: kvpb.INCONSISTENT, + RoutingPolicy: kvpb.RoutingPolicy_NEAREST, }, &leaseReq, ) @@ -1590,7 +1591,7 @@ func (ts *TestServer) GetRangeLease( return LeaseInfo{}, hlc.ClockTimestamp{}, pErr.GoError() } // Adapt the LeaseInfoResponse format to LeaseInfo. - resp := leaseResp.(*roachpb.LeaseInfoResponse) + resp := leaseResp.(*kvpb.LeaseInfoResponse) if queryPolicy == QueryLocalNodeOnly && resp.EvaluatedBy != ts.GetFirstStoreID() { // TODO(andrei): Figure out how to deal with nodes with multiple stores. // This API should permit addressing the query to a particular store. @@ -1653,8 +1654,8 @@ func (ts *TestServer) ForceTableGC( } tableID := uint32(*row[0].(*tree.DInt)) tblKey := keys.SystemSQLCodec.TablePrefix(tableID) - gcr := roachpb.GCRequest{ - RequestHeader: roachpb.RequestHeader{ + gcr := kvpb.GCRequest{ + RequestHeader: kvpb.RequestHeader{ Key: tblKey, EndKey: tblKey.PrefixEnd(), }, diff --git a/pkg/spanconfig/spanconfigkvsubscriber/BUILD.bazel b/pkg/spanconfig/spanconfigkvsubscriber/BUILD.bazel index 55ca34d31bbd..bda82ba55260 100644 --- a/pkg/spanconfig/spanconfigkvsubscriber/BUILD.bazel +++ b/pkg/spanconfig/spanconfigkvsubscriber/BUILD.bazel @@ -15,6 +15,7 @@ go_library( "//pkg/kv/kvclient/rangefeed", "//pkg/kv/kvclient/rangefeed/rangefeedbuffer", "//pkg/kv/kvclient/rangefeed/rangefeedcache", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/settings/cluster", "//pkg/spanconfig", diff --git a/pkg/spanconfig/spanconfigkvsubscriber/spanconfigdecoder.go b/pkg/spanconfig/spanconfigkvsubscriber/spanconfigdecoder.go index 20f2548d8335..2bce21321829 100644 --- a/pkg/spanconfig/spanconfigkvsubscriber/spanconfigdecoder.go +++ b/pkg/spanconfig/spanconfigkvsubscriber/spanconfigdecoder.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedbuffer" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/sql/catalog" @@ -91,7 +92,7 @@ func (sd *spanConfigDecoder) decode(kv roachpb.KeyValue) (spanconfig.Record, err } func (sd *spanConfigDecoder) translateEvent( - ctx context.Context, ev *roachpb.RangeFeedValue, + ctx context.Context, ev *kvpb.RangeFeedValue, ) rangefeedbuffer.Event { deleted := !ev.Value.IsPresent() var value roachpb.Value diff --git a/pkg/spanconfig/spanconfigsqlwatcher/BUILD.bazel b/pkg/spanconfig/spanconfigsqlwatcher/BUILD.bazel index c3c1987e27de..24242739cdf3 100644 --- a/pkg/spanconfig/spanconfigsqlwatcher/BUILD.bazel +++ b/pkg/spanconfig/spanconfigsqlwatcher/BUILD.bazel @@ -15,6 +15,7 @@ go_library( "//pkg/keys", "//pkg/kv/kvclient/rangefeed", "//pkg/kv/kvclient/rangefeed/rangefeedbuffer", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/protectedts/ptpb", "//pkg/roachpb", "//pkg/settings/cluster", diff --git a/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher.go b/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher.go index 745117208136..366754546a2b 100644 --- a/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher.go +++ b/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedbuffer" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -193,7 +194,7 @@ func (s *SQLWatcher) watchForDescriptorUpdates( Key: descriptorTableStart, EndKey: descriptorTableStart.PrefixEnd(), } - handleEvent := func(ctx context.Context, ev *roachpb.RangeFeedValue) { + handleEvent := func(ctx context.Context, ev *kvpb.RangeFeedValue) { if !ev.Value.IsPresent() && !ev.PrevValue.IsPresent() { // Event for a tombstone on a tombstone -- nothing for us to do here. return @@ -261,7 +262,7 @@ func (s *SQLWatcher) watchForZoneConfigUpdates( EndKey: zoneTableStart.PrefixEnd(), } decoder := newZonesDecoder(s.codec) - handleEvent := func(ctx context.Context, ev *roachpb.RangeFeedValue) { + handleEvent := func(ctx context.Context, ev *kvpb.RangeFeedValue) { var descID descpb.ID var err error if keys.SystemZonesTableSpan.Key.Equal(ev.Key) { @@ -321,7 +322,7 @@ func (s *SQLWatcher) watchForProtectedTimestampUpdates( } decoder := newProtectedTimestampDecoder() - handleEvent := func(ctx context.Context, ev *roachpb.RangeFeedValue) { + handleEvent := func(ctx context.Context, ev *kvpb.RangeFeedValue) { if !ev.Value.IsPresent() && !ev.PrevValue.IsPresent() { // Event for a tombstone on a tombstone -- nothing for us to do here. return diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index a229dbbd5ab8..94a350979e0b 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -312,6 +312,7 @@ go_library( "//pkg/kv/kvclient/kvtenant", "//pkg/kv/kvclient/rangecache", "//pkg/kv/kvclient/rangefeed", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/concurrency/lock", "//pkg/kv/kvserver/kvserverbase", "//pkg/kv/kvserver/liveness/livenesspb", @@ -710,6 +711,7 @@ go_test( "//pkg/kv/kvclient/kvcoord", "//pkg/kv/kvclient/rangecache", "//pkg/kv/kvclient/rangefeed", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/kv/kvserver/kvserverbase", "//pkg/kv/kvserver/protectedts", diff --git a/pkg/sql/ambiguous_commit_test.go b/pkg/sql/ambiguous_commit_test.go index 4cfcfd7d5b40..41d67be463e3 100644 --- a/pkg/sql/ambiguous_commit_test.go +++ b/pkg/sql/ambiguous_commit_test.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" @@ -39,12 +40,12 @@ import ( type interceptingTransport struct { kvcoord.Transport - sendNext func(context.Context, *roachpb.BatchRequest) (*roachpb.BatchResponse, error) + sendNext func(context.Context, *kvpb.BatchRequest) (*kvpb.BatchResponse, error) } func (t *interceptingTransport) SendNext( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, error) { + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, error) { if fn := t.sendNext; fn != nil { return fn(ctx, ba) } else { @@ -77,9 +78,9 @@ func TestAmbiguousCommit(t *testing.T) { var processed int32 var tableStartKey atomic.Value - translateToRPCError := roachpb.NewError(errors.Errorf("%s: RPC error: success=%t", t.Name(), ambiguousSuccess)) + translateToRPCError := kvpb.NewError(errors.Errorf("%s: RPC error: success=%t", t.Name(), ambiguousSuccess)) - maybeRPCError := func(req *roachpb.ConditionalPutRequest) *roachpb.Error { + maybeRPCError := func(req *kvpb.ConditionalPutRequest) *kvpb.Error { tsk, ok := tableStartKey.Load().(roachpb.Key) if !ok { return nil @@ -100,7 +101,7 @@ func TestAmbiguousCommit(t *testing.T) { transport, err := kvcoord.GRPCTransportFactory(opts, nodeDialer, replicas) return &interceptingTransport{ Transport: transport, - sendNext: func(ctx context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + sendNext: func(ctx context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { if ambiguousSuccess { br, err := transport.SendNext(ctx, ba) // During shutdown, we may get responses that @@ -116,8 +117,8 @@ func TestAmbiguousCommit(t *testing.T) { } return br, err } else { - if req, ok := ba.GetArg(roachpb.ConditionalPut); ok { - if pErr := maybeRPCError(req.(*roachpb.ConditionalPutRequest)); pErr != nil { + if req, ok := ba.GetArg(kvpb.ConditionalPut); ok { + if pErr := maybeRPCError(req.(*kvpb.ConditionalPutRequest)); pErr != nil { // Blackhole the RPC and return an // error to simulate an ambiguous // result. @@ -134,10 +135,10 @@ func TestAmbiguousCommit(t *testing.T) { if ambiguousSuccess { params.Knobs.Store = &kvserver.StoreTestingKnobs{ TestingResponseFilter: func( - ctx context.Context, args *roachpb.BatchRequest, _ *roachpb.BatchResponse, - ) *roachpb.Error { - if req, ok := args.GetArg(roachpb.ConditionalPut); ok { - return maybeRPCError(req.(*roachpb.ConditionalPutRequest)) + ctx context.Context, args *kvpb.BatchRequest, _ *kvpb.BatchResponse, + ) *kvpb.Error { + if req, ok := args.GetArg(kvpb.ConditionalPut); ok { + return maybeRPCError(req.(*kvpb.ConditionalPutRequest)) } return nil }, diff --git a/pkg/sql/as_of_test.go b/pkg/sql/as_of_test.go index d148e419eafa..66d6eac43108 100644 --- a/pkg/sql/as_of_test.go +++ b/pkg/sql/as_of_test.go @@ -20,8 +20,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/tests" @@ -317,28 +317,28 @@ func TestAsOfRetry(t *testing.T) { name: 5, } cleanupFilter := cmdFilters.AppendFilter( - func(args kvserverbase.FilterArgs) *roachpb.Error { + func(args kvserverbase.FilterArgs) *kvpb.Error { magicVals.Lock() defer magicVals.Unlock() switch req := args.Req.(type) { - case *roachpb.GetRequest: + case *kvpb.GetRequest: if kv.TestingIsRangeLookupRequest(req) { return nil } for key, count := range magicVals.restartCounts { if err := checkCorrectTxn(string(req.Key), magicVals, args.Hdr.Txn); err != nil { - return roachpb.NewError(err) + return kvpb.NewError(err) } if count > 0 && bytes.Contains(req.Key, []byte(key)) { magicVals.restartCounts[key]-- - err := roachpb.NewTransactionRetryError( - roachpb.RETRY_REASON_UNKNOWN, "filter err") + err := kvpb.NewTransactionRetryError( + kvpb.RETRY_REASON_UNKNOWN, "filter err") magicVals.failedValues[string(req.Key)] = failureRecord{err, args.Hdr.Txn} txn := args.Hdr.Txn.Clone() txn.WriteTimestamp = txn.WriteTimestamp.Add(0, 1) - return roachpb.NewErrorWithTxn(err, txn) + return kvpb.NewErrorWithTxn(err, txn) } } } diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index 55eb7eaebcae..4f9d2652ff97 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/settings" @@ -2213,7 +2214,7 @@ func (sc *SchemaChanger) backfillIndexes( if err := sc.distIndexBackfill( ctx, version, addingSpans, addedIndexes, writeAtRequestTimestamp, backfill.IndexMutationFilter, fractionScaler, ); err != nil { - if errors.HasType(err, &roachpb.InsufficientSpaceError{}) { + if errors.HasType(err, &kvpb.InsufficientSpaceError{}) { return jobs.MarkPauseRequestError(errors.UnwrapAll(err)) } return err diff --git a/pkg/sql/backfill/BUILD.bazel b/pkg/sql/backfill/BUILD.bazel index 355923f21329..553c7621a1ca 100644 --- a/pkg/sql/backfill/BUILD.bazel +++ b/pkg/sql/backfill/BUILD.bazel @@ -14,6 +14,7 @@ go_library( "//pkg/base", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/settings", "//pkg/sql/catalog", diff --git a/pkg/sql/backfill/mvcc_index_merger.go b/pkg/sql/backfill/mvcc_index_merger.go index 2d82970efed4..00e327f7e92b 100644 --- a/pkg/sql/backfill/mvcc_index_merger.go +++ b/pkg/sql/backfill/mvcc_index_merger.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql/catalog" @@ -264,14 +265,14 @@ func (ibm *IndexBackfillMerger) scan( chunkBytes := indexBackfillMergeBatchBytes.Get(&ibm.evalCtx.Settings.SV) var nextStart roachpb.Key - var br *roachpb.BatchResponse + var br *kvpb.BatchResponse if err := ibm.flowCtx.Cfg.DB.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { if err := txn.KV().SetFixedTimestamp(ctx, readAsOf); err != nil { return err } // For now just grab all of the destination KVs and merge the corresponding entries. log.VInfof(ctx, 2, "scanning batch [%s, %s) at %v to merge", startKey, endKey, readAsOf) - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.TargetBytes = chunkBytes if err := ibm.growBoundAccount(ctx, chunkBytes); err != nil { return errors.Wrap(err, "failed to fetch keys to merge from temp index") @@ -279,14 +280,14 @@ func (ibm *IndexBackfillMerger) scan( defer ibm.shrinkBoundAccount(ctx, chunkBytes) ba.MaxSpanRequestKeys = chunkSize - ba.Add(&roachpb.ScanRequest{ - RequestHeader: roachpb.RequestHeader{ + ba.Add(&kvpb.ScanRequest{ + RequestHeader: kvpb.RequestHeader{ Key: startKey, EndKey: endKey, }, - ScanFormat: roachpb.KEY_VALUES, + ScanFormat: kvpb.KEY_VALUES, }) - var pErr *roachpb.Error + var pErr *kvpb.Error br, pErr = txn.KV().Send(ctx, ba) if pErr != nil { return pErr.GoError() diff --git a/pkg/sql/catalog/descs/BUILD.bazel b/pkg/sql/catalog/descs/BUILD.bazel index c09938ec54c0..45ad5bec3f19 100644 --- a/pkg/sql/catalog/descs/BUILD.bazel +++ b/pkg/sql/catalog/descs/BUILD.bazel @@ -91,8 +91,8 @@ go_test( "//pkg/base", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", - "//pkg/roachpb", "//pkg/security/securityassets", "//pkg/security/securitytest", "//pkg/security/username", diff --git a/pkg/sql/catalog/descs/collection_test.go b/pkg/sql/catalog/descs/collection_test.go index 1ef039c5ffe9..3b2e5cd78dfa 100644 --- a/pkg/sql/catalog/descs/collection_test.go +++ b/pkg/sql/catalog/descs/collection_test.go @@ -23,8 +23,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" @@ -155,15 +155,15 @@ func TestTxnClearsCollectionOnRetry(t *testing.T) { var serverArgs base.TestServerArgs params := base.TestClusterArgs{ServerArgs: serverArgs} params.ServerArgs.Knobs.Store = &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(ctx context.Context, r *roachpb.BatchRequest) *roachpb.Error { + TestingRequestFilter: func(ctx context.Context, r *kvpb.BatchRequest) *kvpb.Error { if r.Txn == nil || r.Txn.Name != txnName { return nil } - if _, ok := r.GetArg(roachpb.EndTxn); ok { + if _, ok := r.GetArg(kvpb.EndTxn); ok { if !haveInjectedRetry { haveInjectedRetry = true // Force a retry error the first time. - return roachpb.NewError(roachpb.NewTransactionRetryError(roachpb.RETRY_REASON_UNKNOWN, "injected error")) + return kvpb.NewError(kvpb.NewTransactionRetryError(kvpb.RETRY_REASON_UNKNOWN, "injected error")) } } return nil diff --git a/pkg/sql/catalog/ingesting/BUILD.bazel b/pkg/sql/catalog/ingesting/BUILD.bazel index 0574c93a186f..4be1d317fef8 100644 --- a/pkg/sql/catalog/ingesting/BUILD.bazel +++ b/pkg/sql/catalog/ingesting/BUILD.bazel @@ -12,6 +12,7 @@ go_library( deps = [ "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/security/username", "//pkg/sql/catalog", diff --git a/pkg/sql/catalog/ingesting/write_descs.go b/pkg/sql/catalog/ingesting/write_descs.go index f3da1865f58c..7c90a4863176 100644 --- a/pkg/sql/catalog/ingesting/write_descs.go +++ b/pkg/sql/catalog/ingesting/write_descs.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql/catalog" @@ -228,7 +229,7 @@ func WriteDescriptors( b.InitPut(kv.Key, &kv.Value, false) } if err := txn.Run(ctx, b); err != nil { - if errors.HasType(err, (*roachpb.ConditionFailedError)(nil)) { + if errors.HasType(err, (*kvpb.ConditionFailedError)(nil)) { return pgerror.Newf(pgcode.DuplicateObject, "table already exists") } return err diff --git a/pkg/sql/catalog/lease/BUILD.bazel b/pkg/sql/catalog/lease/BUILD.bazel index 7885be8f4603..b719478ea754 100644 --- a/pkg/sql/catalog/lease/BUILD.bazel +++ b/pkg/sql/catalog/lease/BUILD.bazel @@ -22,6 +22,7 @@ go_library( "//pkg/keys", "//pkg/kv", "//pkg/kv/kvclient/rangefeed", + "//pkg/kv/kvpb", "//pkg/multitenant", "//pkg/roachpb", "//pkg/settings", @@ -78,6 +79,7 @@ go_test( "//pkg/jobs/jobspb", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/kv/kvserver/kvserverbase", "//pkg/roachpb", diff --git a/pkg/sql/catalog/lease/kv_writer_test.go b/pkg/sql/catalog/lease/kv_writer_test.go index e512905123c2..e9dc8e46a60b 100644 --- a/pkg/sql/catalog/lease/kv_writer_test.go +++ b/pkg/sql/catalog/lease/kv_writer_test.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" @@ -124,14 +125,14 @@ func getRawHistoryKVs( start hlc.Timestamp, codec keys.SQLCodec, ) []roachpb.KeyValue { - var b roachpb.BatchRequest + var b kvpb.BatchRequest b.Header.Timestamp = kvDB.Clock().Now() - b.Add(&roachpb.ExportRequest{ - RequestHeader: roachpb.RequestHeader{ + b.Add(&kvpb.ExportRequest{ + RequestHeader: kvpb.RequestHeader{ Key: codec.TablePrefix(uint32(tabID)), EndKey: codec.TablePrefix(uint32(tabID)).PrefixEnd(), }, - MVCCFilter: roachpb.MVCCFilter_All, + MVCCFilter: kvpb.MVCCFilter_All, StartTime: start, }) br, err := kvDB.NonTransactionalSender().Send(ctx, &b) diff --git a/pkg/sql/catalog/lease/lease.go b/pkg/sql/catalog/lease/lease.go index 323873503ca7..6fcc43436482 100644 --- a/pkg/sql/catalog/lease/lease.go +++ b/pkg/sql/catalog/lease/lease.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -250,18 +251,18 @@ func getDescriptorsFromStoreForInterval( // Create an export request (1 kv call) for all descriptors for given // descriptor ID written during the interval [timestamp, endTimestamp). - batchRequestHeader := roachpb.Header{ + batchRequestHeader := kvpb.Header{ Timestamp: upperBound.Prev(), } descriptorKey := catalogkeys.MakeDescMetadataKey(codec, id) - requestHeader := roachpb.RequestHeader{ + requestHeader := kvpb.RequestHeader{ Key: descriptorKey, EndKey: descriptorKey.PrefixEnd(), } - req := &roachpb.ExportRequest{ + req := &kvpb.ExportRequest{ RequestHeader: requestHeader, StartTime: lowerBound.Prev(), - MVCCFilter: roachpb.MVCCFilter_All, + MVCCFilter: kvpb.MVCCFilter_All, } // Export request returns descriptors in decreasing modification time. @@ -277,7 +278,7 @@ func getDescriptorsFromStoreForInterval( // set as the expiration for the next descriptor to process. Recall we process // descriptors in decreasing modification time. subsequentModificationTime := upperBound - for _, file := range res.(*roachpb.ExportResponse).Files { + for _, file := range res.(*kvpb.ExportResponse).Files { if err := func() error { it, err := kvstorage.NewMemSSTIterator(file.SST, false, /* verify */ kvstorage.IterOptions{ @@ -1150,7 +1151,7 @@ func (m *Manager) watchForUpdates(ctx context.Context, descUpdateCh chan<- *desc EndKey: descriptorTableStart.PrefixEnd(), } handleEvent := func( - ctx context.Context, ev *roachpb.RangeFeedValue, + ctx context.Context, ev *kvpb.RangeFeedValue, ) { if len(ev.Value.RawBytes) == 0 { return diff --git a/pkg/sql/catalog/lease/lease_test.go b/pkg/sql/catalog/lease/lease_test.go index bbfe65b599f8..0cd9fb766a80 100644 --- a/pkg/sql/catalog/lease/lease_test.go +++ b/pkg/sql/catalog/lease/lease_test.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -2776,11 +2777,11 @@ func TestOfflineLeaseRefresh(t *testing.T) { var mu syncutil.RWMutex knobs := &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(ctx context.Context, req *roachpb.BatchRequest) *roachpb.Error { + TestingRequestFilter: func(ctx context.Context, req *kvpb.BatchRequest) *kvpb.Error { mu.RLock() checkRequest := req.Txn != nil && req.Txn.ID.Equal(txnID) mu.RUnlock() - if _, ok := req.GetArg(roachpb.EndTxn); checkRequest && ok { + if _, ok := req.GetArg(kvpb.EndTxn); checkRequest && ok { notify := make(chan struct{}) waitForRqstFilter <- notify <-notify @@ -2902,7 +2903,7 @@ func TestLeaseTxnDeadlineExtension(t *testing.T) { // require the lease to be reacquired. lease.LeaseDuration.Override(ctx, ¶ms.SV, 0) params.Knobs.Store = &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(ctx context.Context, req *roachpb.BatchRequest) *roachpb.Error { + TestingRequestFilter: func(ctx context.Context, req *kvpb.BatchRequest) *kvpb.Error { filterMu.Lock() // Wait for a commit with the txnID, and only allows // it to resume when the channel gets unblocked. @@ -3222,7 +3223,7 @@ func TestAmbiguousResultIsRetried(t *testing.T) { type filter = kvserverbase.ReplicaResponseFilter var f atomic.Value - noop := filter(func(context.Context, *roachpb.BatchRequest, *roachpb.BatchResponse) *roachpb.Error { + noop := filter(func(context.Context, *kvpb.BatchRequest, *kvpb.BatchResponse) *kvpb.Error { return nil }) f.Store(noop) @@ -3230,7 +3231,7 @@ func TestAmbiguousResultIsRetried(t *testing.T) { s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{ Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ - TestingResponseFilter: func(ctx context.Context, request *roachpb.BatchRequest, response *roachpb.BatchResponse) *roachpb.Error { + TestingResponseFilter: func(ctx context.Context, request *kvpb.BatchRequest, response *kvpb.BatchResponse) *kvpb.Error { return f.Load().(filter)(ctx, request, response) }, }, @@ -3249,23 +3250,23 @@ func TestAmbiguousResultIsRetried(t *testing.T) { testCtx, cancel := context.WithCancel(ctx) defer cancel() - errorsAfterEndTxn := make(chan chan *roachpb.Error) - f.Store(filter(func(ctx context.Context, request *roachpb.BatchRequest, response *roachpb.BatchResponse) *roachpb.Error { + errorsAfterEndTxn := make(chan chan *kvpb.Error) + f.Store(filter(func(ctx context.Context, request *kvpb.BatchRequest, response *kvpb.BatchResponse) *kvpb.Error { switch r := request.Requests[0].GetInner().(type) { - case *roachpb.ConditionalPutRequest: + case *kvpb.ConditionalPutRequest: if !bytes.HasPrefix(r.Key, tablePrefix) { return nil } in, _, _, err := keys.DecodeTableIDIndexID(r.Key) if err != nil { - return roachpb.NewError(errors.WithAssertionFailure(err)) + return kvpb.NewError(errors.WithAssertionFailure(err)) } var a tree.DatumAlloc if systemschema.TestSupportMultiRegion() { var err error _, in, err = keyside.Decode(&a, types.Bytes, in, encoding.Ascending) if !assert.NoError(t, err) { - return roachpb.NewError(err) + return kvpb.NewError(err) } } id, _, err := keyside.Decode( @@ -3275,11 +3276,11 @@ func TestAmbiguousResultIsRetried(t *testing.T) { if tree.MustBeDInt(id) == tree.DInt(tableID) { txnID.Store(request.Txn.ID) } - case *roachpb.EndTxnRequest: + case *kvpb.EndTxnRequest: if request.Txn.ID != txnID.Load().(uuid.UUID) { return nil } - errCh := make(chan *roachpb.Error) + errCh := make(chan *kvpb.Error) select { case errorsAfterEndTxn <- errCh: case <-testCtx.Done(): @@ -3298,7 +3299,7 @@ func TestAmbiguousResultIsRetried(t *testing.T) { selectErr <- err }() unblock := <-errorsAfterEndTxn - unblock <- roachpb.NewError(roachpb.NewAmbiguousResultError(errors.New("boom"))) + unblock <- kvpb.NewError(kvpb.NewAmbiguousResultError(errors.New("boom"))) // Make sure we see a retry, then let it succeed. close(<-errorsAfterEndTxn) // Allow anything further to proceed. diff --git a/pkg/sql/catalog/lease/storage.go b/pkg/sql/catalog/lease/storage.go index 16fceab5eac2..e6063bd7a581 100644 --- a/pkg/sql/catalog/lease/storage.go +++ b/pkg/sql/catalog/lease/storage.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/multitenant" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" @@ -175,7 +176,7 @@ func (s storage) acquire( // are propagated up to the caller. for r := retry.StartWithCtx(ctx, retry.Options{}); r.Next(); { err := s.db.KV().Txn(ctx, acquireInTxn) - var pErr *roachpb.AmbiguousResultError + var pErr *kvpb.AmbiguousResultError switch { case errors.As(err, &pErr): log.Infof(ctx, "ambiguous error occurred during lease acquisition for %v, retrying: %v", id, err) diff --git a/pkg/sql/colexecop/BUILD.bazel b/pkg/sql/colexecop/BUILD.bazel index cbdb45f95633..37f0725f6739 100644 --- a/pkg/sql/colexecop/BUILD.bazel +++ b/pkg/sql/colexecop/BUILD.bazel @@ -13,7 +13,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/col/coldata", - "//pkg/roachpb", + "//pkg/kv/kvpb", "//pkg/sql/colexecerror", "//pkg/sql/colmem", "//pkg/sql/execinfra/execopnode", diff --git a/pkg/sql/colexecop/operator.go b/pkg/sql/colexecop/operator.go index 68094182164a..006945a86b42 100644 --- a/pkg/sql/colexecop/operator.go +++ b/pkg/sql/colexecop/operator.go @@ -15,7 +15,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/execinfra/execopnode" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -75,7 +75,7 @@ type KVReader interface { GetBatchRequestsIssued() int64 // GetContentionInfo returns the amount of time KV reads spent // contending. It must be safe for concurrent use. - GetContentionInfo() (time.Duration, []roachpb.ContentionEvent) + GetContentionInfo() (time.Duration, []kvpb.ContentionEvent) // GetScanStats returns statistics about the scan that happened during the // KV reads. It must be safe for concurrent use. GetScanStats() execstats.ScanStats diff --git a/pkg/sql/colfetcher/BUILD.bazel b/pkg/sql/colfetcher/BUILD.bazel index b099f5cb95c5..cae1ff99228c 100644 --- a/pkg/sql/colfetcher/BUILD.bazel +++ b/pkg/sql/colfetcher/BUILD.bazel @@ -22,6 +22,7 @@ go_library( "//pkg/col/typeconv", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/settings", "//pkg/sql/catalog", diff --git a/pkg/sql/colfetcher/colbatch_scan.go b/pkg/sql/colfetcher/colbatch_scan.go index fa9574a6315c..04293fad82ec 100644 --- a/pkg/sql/colfetcher/colbatch_scan.go +++ b/pkg/sql/colfetcher/colbatch_scan.go @@ -16,7 +16,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" @@ -84,7 +84,7 @@ func (s *colBatchScanBase) GetRowsRead() int64 { } // GetContentionInfo is part of the colexecop.KVReader interface. -func (s *colBatchScanBase) GetContentionInfo() (time.Duration, []roachpb.ContentionEvent) { +func (s *colBatchScanBase) GetContentionInfo() (time.Duration, []kvpb.ContentionEvent) { return execstats.GetCumulativeContentionTime(s.Ctx, nil /* recording */) } @@ -125,12 +125,12 @@ func newColBatchScanBase( spec *execinfrapb.TableReaderSpec, post *execinfrapb.PostProcessSpec, typeResolver *descs.DistSQLTypeResolver, -) (*colBatchScanBase, *roachpb.BoundedStalenessHeader, *cFetcherTableArgs, error) { +) (*colBatchScanBase, *kvpb.BoundedStalenessHeader, *cFetcherTableArgs, error) { // NB: we hit this with a zero NodeID (but !ok) with multi-tenancy. if nodeID, ok := flowCtx.NodeID.OptionalNodeID(); nodeID == 0 && ok { return nil, nil, nil, errors.Errorf("attempting to create a ColBatchScan with uninitialized NodeID") } - var bsHeader *roachpb.BoundedStalenessHeader + var bsHeader *kvpb.BoundedStalenessHeader if aost := flowCtx.EvalCtx.AsOfSystemTime; aost != nil && aost.BoundedStaleness { ts := aost.Timestamp // If the descriptor's modification time is after the bounded staleness min bound, @@ -140,7 +140,7 @@ func newColBatchScanBase( if aost.Timestamp.Less(spec.TableDescriptorModificationTime) { ts = spec.TableDescriptorModificationTime } - bsHeader = &roachpb.BoundedStalenessHeader{ + bsHeader = &kvpb.BoundedStalenessHeader{ MinTimestampBound: ts, MinTimestampBoundStrict: aost.NearestOnly, MaxTimestampBound: flowCtx.EvalCtx.AsOfSystemTime.MaxTimestampBound, // may be empty diff --git a/pkg/sql/colfetcher/index_join.go b/pkg/sql/colfetcher/index_join.go index 0f1faf3f3abb..c2af90151725 100644 --- a/pkg/sql/colfetcher/index_join.go +++ b/pkg/sql/colfetcher/index_join.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/typeconv" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" @@ -408,7 +409,7 @@ func (s *ColIndexJoin) GetKVCPUTime() time.Duration { } // GetContentionInfo is part of the colexecop.KVReader interface. -func (s *ColIndexJoin) GetContentionInfo() (time.Duration, []roachpb.ContentionEvent) { +func (s *ColIndexJoin) GetContentionInfo() (time.Duration, []kvpb.ContentionEvent) { return execstats.GetCumulativeContentionTime(s.Ctx, nil /* recording */) } diff --git a/pkg/sql/colflow/colrpc/inbox.go b/pkg/sql/colflow/colrpc/inbox.go index 16321645b698..1937afe66efd 100644 --- a/pkg/sql/colflow/colrpc/inbox.go +++ b/pkg/sql/colflow/colrpc/inbox.go @@ -391,7 +391,7 @@ func (i *Inbox) Next() coldata.Batch { // // We choose this way of handling multiple errors rather // than something like errors.CombineErrors() since we want - // to keep errors unchanged (e.g. roachpb.ErrPriority() will + // to keep errors unchanged (e.g. kvpb.ErrPriority() will // be called on each error in the DistSQLReceiver). i.bufferedMeta = append(i.bufferedMeta, meta) colexecutils.AccountForMetadata(i.allocator, i.bufferedMeta[len(i.bufferedMeta)-1:]) diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index 6d7e14c5ae35..7e0695f7890e 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/multitenant" "github.com/cockroachdb/cockroach/pkg/multitenant/multitenantcpu" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -2722,7 +2723,7 @@ var retriableMinTimestampBoundUnsatisfiableError = errors.Newf( ) func errIsRetriable(err error) bool { - return errors.HasType(err, (*roachpb.TransactionRetryWithProtoRefreshError)(nil)) || + return errors.HasType(err, (*kvpb.TransactionRetryWithProtoRefreshError)(nil)) || scerrors.ConcurrentSchemaChangeDescID(err) != descpb.InvalidID || errors.Is(err, retriableMinTimestampBoundUnsatisfiableError) || errors.Is(err, descidgen.ErrDescIDSequenceMigrationInProgress) || @@ -2737,7 +2738,7 @@ func (ex *connExecutor) makeErrEvent(err error, stmt tree.Statement) (fsm.Event, // MaxTimestampBound set if our MinTimestampBound was bumped up from the // original AS OF SYSTEM TIME timestamp set due to a schema bumping the // timestamp to a higher value. - if minTSErr := (*roachpb.MinTimestampBoundUnsatisfiableError)(nil); errors.As(err, &minTSErr) { + if minTSErr := (*kvpb.MinTimestampBoundUnsatisfiableError)(nil); errors.As(err, &minTSErr) { aost := ex.planner.EvalContext().AsOfSystemTime if aost != nil && aost.BoundedStaleness { if !aost.MaxTimestampBound.IsEmpty() && aost.MaxTimestampBound.LessEq(minTSErr.MinTimestampBound) { @@ -2964,7 +2965,7 @@ func (ex *connExecutor) resetEvalCtx(evalCtx *extendedEvalContext, txn *kv.Txn, // See resetPlanner for more context on setting the maximum timestamp for // AOST read retries. - var minTSErr *roachpb.MinTimestampBoundUnsatisfiableError + var minTSErr *kvpb.MinTimestampBoundUnsatisfiableError if err := ex.state.mu.autoRetryReason; err != nil && errors.As(err, &minTSErr) { evalCtx.AsOfSystemTime.MaxTimestampBound = ex.extraTxnState.descCollection.GetMaxTimestampBound() } else if newTxn { @@ -3019,7 +3020,7 @@ func (ex *connExecutor) resetPlanner( // same minimum timestamp for the AOST read, but set the maximum timestamp // to the point just before our failed read to ensure we don't try to read // data which may be after the schema change when we retry. - var minTSErr *roachpb.MinTimestampBoundUnsatisfiableError + var minTSErr *kvpb.MinTimestampBoundUnsatisfiableError if err := ex.state.mu.autoRetryReason; err != nil && errors.As(err, &minTSErr) { nextMax := minTSErr.MinTimestampBound ex.extraTxnState.descCollection.SetMaxTimestampBound(nextMax) diff --git a/pkg/sql/conn_executor_internal_test.go b/pkg/sql/conn_executor_internal_test.go index 243c898d9846..774da85939ce 100644 --- a/pkg/sql/conn_executor_internal_test.go +++ b/pkg/sql/conn_executor_internal_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -265,8 +266,8 @@ func startConnExecutor( stopper := stop.NewStopper() clock := hlc.NewClockForTesting(nil) factory := kv.MakeMockTxnSenderFactory( - func(context.Context, *roachpb.Transaction, *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, *roachpb.Error) { + func(context.Context, *roachpb.Transaction, *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, *kvpb.Error) { return nil, nil }) db := kv.NewDB(log.MakeTestingAmbientCtxWithNewTracer(), factory, clock, stopper) diff --git a/pkg/sql/conn_executor_test.go b/pkg/sql/conn_executor_test.go index cce78bbf26fa..01f679539cf6 100644 --- a/pkg/sql/conn_executor_test.go +++ b/pkg/sql/conn_executor_test.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -327,12 +328,12 @@ func TestErrorOnRollback(t *testing.T) { params := base.TestServerArgs{ Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ - TestingProposalFilter: func(fArgs kvserverbase.ProposalFilterArgs) *roachpb.Error { + TestingProposalFilter: func(fArgs kvserverbase.ProposalFilterArgs) *kvpb.Error { if !fArgs.Req.IsSingleRequest() { return nil } req := fArgs.Req.Requests[0] - etReq, ok := req.GetInner().(*roachpb.EndTxnRequest) + etReq, ok := req.GetInner().(*kvpb.EndTxnRequest) // We only inject the error once. Turns out that during the // life of the test there's two EndTxns being sent - one is // the direct result of the test's call to tx.Rollback(), @@ -344,7 +345,7 @@ func TestErrorOnRollback(t *testing.T) { atomic.LoadInt64(&injectedErr) == 0 { atomic.StoreInt64(&injectedErr, 1) - return roachpb.NewErrorf("test injected error") + return kvpb.NewErrorf("test injected error") } return nil }, @@ -584,9 +585,9 @@ func TestQueryProgress(t *testing.T) { TableReaderBatchBytesLimit: 1500, }, Store: &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(_ context.Context, req *roachpb.BatchRequest) *roachpb.Error { + TestingRequestFilter: func(_ context.Context, req *kvpb.BatchRequest) *kvpb.Error { if req.IsSingleRequest() { - scan, ok := req.Requests[0].GetInner().(*roachpb.ScanRequest) + scan, ok := req.Requests[0].GetInner().(*kvpb.ScanRequest) if ok && getTableSpan().ContainsKey(scan.Key) && atomic.LoadInt64(&queryRunningAtomic) == 1 { i := atomic.AddInt64(&scannedBatchesAtomic, 1) if i == stallAfterScans { @@ -747,7 +748,7 @@ func TestRetriableErrorDuringPrepare(t *testing.T) { SQLExecutor: &sql.ExecutorTestingKnobs{ BeforePrepare: func(ctx context.Context, stmt string, txn *kv.Txn) error { if strings.Contains(stmt, uniqueString) && atomic.AddInt64(&failed, 1) <= numToFail { - return roachpb.NewTransactionRetryWithProtoRefreshError("boom", + return kvpb.NewTransactionRetryWithProtoRefreshError("boom", txn.ID(), *txn.TestingCloneTxn()) } return nil @@ -795,19 +796,19 @@ func TestRetriableErrorDuringUpgradedTransaction(t *testing.T) { testDB.QueryRow(t, "SELECT 'foo'::regclass::oid").Scan(&fooTableId) // Inject an error that will happen during execution. - filter.setFilter(func(ctx context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + filter.setFilter(func(ctx context.Context, ba *kvpb.BatchRequest) *kvpb.Error { if ba.Txn == nil { return nil } - if req, ok := ba.GetArg(roachpb.ConditionalPut); ok { - put := req.(*roachpb.ConditionalPutRequest) + if req, ok := ba.GetArg(kvpb.ConditionalPut); ok { + put := req.(*kvpb.ConditionalPutRequest) _, tableID, err := keys.SystemSQLCodec.DecodeTablePrefix(put.Key) if err != nil || tableID != fooTableId { return nil } if atomic.AddInt64(&retryCount, 1) <= numToRetry { - return roachpb.NewErrorWithTxn( - roachpb.NewTransactionRetryError(roachpb.RETRY_REASON_UNKNOWN, "injected retry error"), ba.Txn, + return kvpb.NewErrorWithTxn( + kvpb.NewTransactionRetryError(kvpb.RETRY_REASON_UNKNOWN, "injected retry error"), ba.Txn, ) } } @@ -872,19 +873,19 @@ func TestErrorDuringPrepareInExplicitTransactionPropagates(t *testing.T) { require.NoError(t, err) // Inject an error that will happen during planning. - filter.setFilter(func(ctx context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + filter.setFilter(func(ctx context.Context, ba *kvpb.BatchRequest) *kvpb.Error { if ba.Txn == nil { return nil } - if req, ok := ba.GetArg(roachpb.Get); ok { - get := req.(*roachpb.GetRequest) + if req, ok := ba.GetArg(kvpb.Get); ok { + get := req.(*kvpb.GetRequest) _, tableID, err := keys.SystemSQLCodec.DecodeTablePrefix(get.Key) if err != nil || tableID != keys.NamespaceTableID { err = nil return nil } - return roachpb.NewErrorWithTxn( - roachpb.NewTransactionRetryError(roachpb.RETRY_REASON_UNKNOWN, "boom"), ba.Txn) + return kvpb.NewErrorWithTxn( + kvpb.NewTransactionRetryError(kvpb.RETRY_REASON_UNKNOWN, "boom"), ba.Txn) } return nil }) @@ -1131,7 +1132,7 @@ func TestTransactionDeadline(t *testing.T) { // This will be used in the tests for accessing mu. locked := func(f func()) { mu.Lock(); defer mu.Unlock(); f() } // Set up a kvserverbase.ReplicaRequestFilter which will extract the deadline for the test transaction. - checkTransactionDeadlineFilter := func(_ context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + checkTransactionDeadlineFilter := func(_ context.Context, ba *kvpb.BatchRequest) *kvpb.Error { if ba.Txn == nil { return nil } @@ -1142,8 +1143,8 @@ func TestTransactionDeadline(t *testing.T) { return nil } - if args, ok := ba.GetArg(roachpb.EndTxn); ok { - et := args.(*roachpb.EndTxnRequest) + if args, ok := ba.GetArg(kvpb.EndTxn); ok { + et := args.(*kvpb.EndTxnRequest) if et.Deadline.IsEmpty() { return nil } @@ -1799,14 +1800,12 @@ func (f *dynamicRequestFilter) setFilter(filter kvserverbase.ReplicaRequestFilte } // noopRequestFilter is a kvserverbase.ReplicaRequestFilter. -func (f *dynamicRequestFilter) filter( - ctx context.Context, request *roachpb.BatchRequest, -) *roachpb.Error { +func (f *dynamicRequestFilter) filter(ctx context.Context, request *kvpb.BatchRequest) *kvpb.Error { return f.v.Load().(kvserverbase.ReplicaRequestFilter)(ctx, request) } // noopRequestFilter is a kvserverbase.ReplicaRequestFilter that does nothing. -func noopRequestFilter(ctx context.Context, request *roachpb.BatchRequest) *roachpb.Error { +func noopRequestFilter(ctx context.Context, request *kvpb.BatchRequest) *kvpb.Error { return nil } diff --git a/pkg/sql/consistencychecker/BUILD.bazel b/pkg/sql/consistencychecker/BUILD.bazel index c5c08e8836b5..740aa5839a19 100644 --- a/pkg/sql/consistencychecker/BUILD.bazel +++ b/pkg/sql/consistencychecker/BUILD.bazel @@ -8,6 +8,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/roachpb", ], ) diff --git a/pkg/sql/consistencychecker/consistency_checker.go b/pkg/sql/consistencychecker/consistency_checker.go index efa7dadfc92f..c029a6696cde 100644 --- a/pkg/sql/consistencychecker/consistency_checker.go +++ b/pkg/sql/consistencychecker/consistency_checker.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" ) @@ -32,11 +33,11 @@ func NewConsistencyChecker(db *kv.DB) *ConsistencyChecker { // CheckConsistency implements the eval.ConsistencyChecker interface. func (s *ConsistencyChecker) CheckConsistency( - ctx context.Context, from, to roachpb.Key, mode roachpb.ChecksumMode, -) (*roachpb.CheckConsistencyResponse, error) { + ctx context.Context, from, to roachpb.Key, mode kvpb.ChecksumMode, +) (*kvpb.CheckConsistencyResponse, error) { var b kv.Batch - b.AddRawRequest(&roachpb.CheckConsistencyRequest{ - RequestHeader: roachpb.RequestHeader{ + b.AddRawRequest(&kvpb.CheckConsistencyRequest{ + RequestHeader: kvpb.RequestHeader{ Key: from, EndKey: to, }, @@ -48,6 +49,6 @@ func (s *ConsistencyChecker) CheckConsistency( if err := s.db.Run(ctx, &b); err != nil { return nil, err } - resp := b.RawResponse().Responses[0].GetInner().(*roachpb.CheckConsistencyResponse) + resp := b.RawResponse().Responses[0].GetInner().(*kvpb.CheckConsistencyResponse) return resp, nil } diff --git a/pkg/sql/contention/BUILD.bazel b/pkg/sql/contention/BUILD.bazel index e1ff777162b9..60a9c5771983 100644 --- a/pkg/sql/contention/BUILD.bazel +++ b/pkg/sql/contention/BUILD.bazel @@ -15,6 +15,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/keys", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/server/serverpb", "//pkg/settings", @@ -49,6 +50,7 @@ go_test( embed = [":contention"], deps = [ "//pkg/keys", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/settings/cluster", "//pkg/sql/appstatspb", diff --git a/pkg/sql/contention/event_store_test.go b/pkg/sql/contention/event_store_test.go index 6a20a6c4f3b7..47caee63e804 100644 --- a/pkg/sql/contention/event_store_test.go +++ b/pkg/sql/contention/event_store_test.go @@ -20,7 +20,7 @@ import ( "testing" "time" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/appstatspb" "github.com/cockroachdb/cockroach/pkg/sql/contentionpb" @@ -146,7 +146,7 @@ func TestCollectionThreshold(t *testing.T) { input := []contentionpb.ExtendedContentionEvent{ { - BlockingEvent: roachpb.ContentionEvent{ + BlockingEvent: kvpb.ContentionEvent{ TxnMeta: enginepb.TxnMeta{ ID: uuid.FastMakeV4(), }, @@ -154,7 +154,7 @@ func TestCollectionThreshold(t *testing.T) { }, }, { - BlockingEvent: roachpb.ContentionEvent{ + BlockingEvent: kvpb.ContentionEvent{ TxnMeta: enginepb.TxnMeta{ ID: uuid.FastMakeV4(), }, @@ -199,7 +199,7 @@ func BenchmarkEventStoreIntake(b *testing.B) { statusServer := newFakeStatusServerCluster() metrics := NewMetrics() - e := roachpb.ContentionEvent{} + e := kvpb.ContentionEvent{} b.SetBytes(int64(e.Size())) run := func(b *testing.B, store *eventStore, numOfConcurrentWriter int) { diff --git a/pkg/sql/contention/registry.go b/pkg/sql/contention/registry.go index 04735bb9aa4f..5167c810a7ad 100644 --- a/pkg/sql/contention/registry.go +++ b/pkg/sql/contention/registry.go @@ -20,6 +20,7 @@ import ( "github.com/biogo/store/llrb" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -123,7 +124,7 @@ type indexMapValue struct { // newIndexMapValue creates a new indexMapValue for a contention event // initialized with that event's data. -func newIndexMapValue(c roachpb.ContentionEvent) *indexMapValue { +func newIndexMapValue(c kvpb.ContentionEvent) *indexMapValue { txnCache := cache.NewUnorderedCache(txnCacheCfg) txnCache.Add(c.TxnMeta.ID, uint64(1)) keyMap := cache.NewOrderedCache(orderedKeyMapCfg) @@ -137,7 +138,7 @@ func newIndexMapValue(c roachpb.ContentionEvent) *indexMapValue { // addContentionEvent adds the given contention event to previously aggregated // contention data. It assumes that c.Key is a SQL key. -func (v *indexMapValue) addContentionEvent(c roachpb.ContentionEvent) { +func (v *indexMapValue) addContentionEvent(c kvpb.ContentionEvent) { v.numContentionEvents++ v.cumulativeContentionTime += c.Duration var numTimesThisTxnWasEncountered uint64 @@ -208,7 +209,7 @@ type nonSQLKeyMapValue struct { // newNonSQLKeyMapValue creates a new nonSQLKeyMapValue for a contention event // initialized with that event's data. -func newNonSQLKeyMapValue(c roachpb.ContentionEvent) *nonSQLKeyMapValue { +func newNonSQLKeyMapValue(c kvpb.ContentionEvent) *nonSQLKeyMapValue { txnCache := cache.NewUnorderedCache(txnCacheCfg) txnCache.Add(c.TxnMeta.ID, uint64(1)) return &nonSQLKeyMapValue{ @@ -220,7 +221,7 @@ func newNonSQLKeyMapValue(c roachpb.ContentionEvent) *nonSQLKeyMapValue { // addContentionEvent adds the given contention event to previously aggregated // contention data. It assumes that c.Key is a non-SQL key. -func (v *nonSQLKeyMapValue) addContentionEvent(c roachpb.ContentionEvent) { +func (v *nonSQLKeyMapValue) addContentionEvent(c kvpb.ContentionEvent) { v.numContentionEvents++ v.cumulativeContentionTime += c.Duration var numTimesThisTxnWasEncountered uint64 diff --git a/pkg/sql/contention/registry_test.go b/pkg/sql/contention/registry_test.go index e28081d203aa..43ae89b39d08 100644 --- a/pkg/sql/contention/registry_test.go +++ b/pkg/sql/contention/registry_test.go @@ -20,6 +20,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/contention" @@ -148,7 +149,7 @@ func TestRegistry(t *testing.T) { return fmt.Sprintf("could not parse duration %s as int: %v", duration, err) } keyBytes = encoding.EncodeStringAscending(keyBytes, key) - addContentionEvent(registry, roachpb.ContentionEvent{ + addContentionEvent(registry, kvpb.ContentionEvent{ Key: keyBytes, TxnMeta: enginepb.TxnMeta{ ID: contendingTxnID, @@ -182,7 +183,7 @@ func TestRegistryConcurrentAdds(t *testing.T) { for i := 0; i < numGoroutines; i++ { go func() { defer wg.Done() - addContentionEvent(registry, roachpb.ContentionEvent{ + addContentionEvent(registry, kvpb.ContentionEvent{ Key: keys.MakeTableIDIndexID(nil /* key */, 1 /* tableID */, 1 /* indexID */), }) }() @@ -240,7 +241,7 @@ func TestSerializedRegistryInvariants(t *testing.T) { key = keys.MakeTableIDIndexID(key, tableID, indexID) } key = append(key, getKey()...) - addContentionEvent(r, roachpb.ContentionEvent{ + addContentionEvent(r, kvpb.ContentionEvent{ Key: key, TxnMeta: enginepb.TxnMeta{ ID: uuid.MakeV4(), @@ -331,7 +332,7 @@ func TestSerializedRegistryInvariants(t *testing.T) { } } -func addContentionEvent(r *contention.Registry, ev roachpb.ContentionEvent) { +func addContentionEvent(r *contention.Registry, ev kvpb.ContentionEvent) { r.AddContentionEvent(contentionpb.ExtendedContentionEvent{ BlockingEvent: ev, }) diff --git a/pkg/sql/contentionpb/BUILD.bazel b/pkg/sql/contentionpb/BUILD.bazel index 9e187080f142..d77b6d19239f 100644 --- a/pkg/sql/contentionpb/BUILD.bazel +++ b/pkg/sql/contentionpb/BUILD.bazel @@ -23,7 +23,7 @@ proto_library( strip_import_prefix = "/pkg", visibility = ["//visibility:public"], deps = [ - "//pkg/roachpb:roachpb_proto", + "//pkg/kv/kvpb:kvpb_proto", "@com_github_gogo_protobuf//gogoproto:gogo_proto", "@com_google_protobuf//:duration_proto", "@com_google_protobuf//:timestamp_proto", @@ -37,6 +37,7 @@ go_proto_library( proto = ":contentionpb_proto", visibility = ["//visibility:public"], deps = [ + "//pkg/kv/kvpb", "//pkg/roachpb", # keep "//pkg/sql/appstatspb", # keep "//pkg/sql/catalog/descpb", # keep diff --git a/pkg/sql/contentionpb/contention.proto b/pkg/sql/contentionpb/contention.proto index 098d4d0b714e..e609ec5bd251 100644 --- a/pkg/sql/contentionpb/contention.proto +++ b/pkg/sql/contentionpb/contention.proto @@ -12,7 +12,7 @@ syntax = "proto3"; package cockroach.sql.contentionpb; option go_package = "contentionpb"; -import "roachpb/api.proto"; +import "kv/kvpb/api.proto"; import "gogoproto/gogo.proto"; import "google/protobuf/duration.proto"; import "google/protobuf/timestamp.proto"; diff --git a/pkg/sql/copy/BUILD.bazel b/pkg/sql/copy/BUILD.bazel index 085b5b82bff6..47c7ba851093 100644 --- a/pkg/sql/copy/BUILD.bazel +++ b/pkg/sql/copy/BUILD.bazel @@ -14,7 +14,7 @@ go_test( deps = [ "//pkg/base", "//pkg/cli/clisqlclient", - "//pkg/roachpb", + "//pkg/kv/kvpb", "//pkg/security/securityassets", "//pkg/security/securitytest", "//pkg/security/username", diff --git a/pkg/sql/copy/copy_in_test.go b/pkg/sql/copy/copy_in_test.go index 0b02df56873f..bc7f5a53807e 100644 --- a/pkg/sql/copy/copy_in_test.go +++ b/pkg/sql/copy/copy_in_test.go @@ -22,7 +22,7 @@ import ( "testing" "time" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/randgen" @@ -438,7 +438,7 @@ func TestCopyFromRetries(t *testing.T) { retriesEnabled: true, hook: func(attemptNum int) error { if attemptNum == 1 { - return &roachpb.TransactionRetryWithProtoRefreshError{} + return &kvpb.TransactionRetryWithProtoRefreshError{} } return nil }, @@ -450,7 +450,7 @@ func TestCopyFromRetries(t *testing.T) { retriesEnabled: false, hook: func(attemptNum int) error { if attemptNum == 1 { - return &roachpb.TransactionRetryWithProtoRefreshError{} + return &kvpb.TransactionRetryWithProtoRefreshError{} } return nil }, @@ -463,7 +463,7 @@ func TestCopyFromRetries(t *testing.T) { inTxn: true, hook: func(attemptNum int) error { if attemptNum == 1 { - return &roachpb.TransactionRetryWithProtoRefreshError{} + return &kvpb.TransactionRetryWithProtoRefreshError{} } return nil }, @@ -475,7 +475,7 @@ func TestCopyFromRetries(t *testing.T) { retriesEnabled: true, hook: func(attemptNum int) error { if attemptNum%2 == 1 { - return &roachpb.TransactionRetryWithProtoRefreshError{} + return &kvpb.TransactionRetryWithProtoRefreshError{} } return nil }, @@ -486,7 +486,7 @@ func TestCopyFromRetries(t *testing.T) { atomicEnabled: false, retriesEnabled: true, hook: func(attemptNum int) error { - return &roachpb.TransactionRetryWithProtoRefreshError{} + return &kvpb.TransactionRetryWithProtoRefreshError{} }, expectedErr: true, }, diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 12be0226432f..d8ee2dd1e440 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -33,6 +33,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -5869,8 +5870,8 @@ CREATE TABLE crdb_internal.lost_descriptors_with_data ( endPrefix := p.extendedEvalCtx.Codec.TablePrefix(uint32(endID - 1)).PrefixEnd() var b kv.Batch b.Header.MaxSpanRequestKeys = 1 - scanRequest := roachpb.NewScan(startPrefix, endPrefix, false).(*roachpb.ScanRequest) - scanRequest.ScanFormat = roachpb.BATCH_RESPONSE + scanRequest := kvpb.NewScan(startPrefix, endPrefix, false).(*kvpb.ScanRequest) + scanRequest.ScanFormat = kvpb.BATCH_RESPONSE b.AddRawRequest(scanRequest) err = p.execCfg.DB.Run(ctx, &b) if err != nil { @@ -6946,14 +6947,14 @@ func genClusterLocksGenerator( return &nextSpan } - var resp *roachpb.QueryLocksResponse + var resp *kvpb.QueryLocksResponse var locks []roachpb.LockStateInfo var resumeSpan *roachpb.Span fetchLocks := func(key, endKey roachpb.Key) error { b := kv.Batch{} - queryLocksRequest := &roachpb.QueryLocksRequest{ - RequestHeader: roachpb.RequestHeader{ + queryLocksRequest := &kvpb.QueryLocksRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, EndKey: endKey, }, diff --git a/pkg/sql/crdb_internal_test.go b/pkg/sql/crdb_internal_test.go index f7c8c3b0c13a..bb69181517b7 100644 --- a/pkg/sql/crdb_internal_test.go +++ b/pkg/sql/crdb_internal_test.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/keyvisualizer" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" @@ -547,10 +548,10 @@ func TestDistSQLFlowsVirtualTables(t *testing.T) { params := base.TestServerArgs{ Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(_ context.Context, req *roachpb.BatchRequest) *roachpb.Error { + TestingRequestFilter: func(_ context.Context, req *kvpb.BatchRequest) *kvpb.Error { if atomic.LoadInt64(&stallAtomic) == 1 { if req.IsSingleRequest() { - scan, ok := req.Requests[0].GetInner().(*roachpb.ScanRequest) + scan, ok := req.Requests[0].GetInner().(*kvpb.ScanRequest) if ok && getTableSpan().ContainsKey(scan.Key) && atomic.LoadInt64(&queryRunningAtomic) == 1 { t.Logf("stalling on scan at %s and waiting for test to unblock...", scan.Key) <-unblock diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index 3e2d20a10301..ac2c469e0e11 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangecache" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/multitenant" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" @@ -1224,9 +1225,9 @@ func (r *DistSQLReceiver) setErrorWithoutStatusUpdate(err error, willDeferStatus defer r.resultWriterMu.Unlock() // Check if the error we just received should take precedence over a // previous error (if any). - if roachpb.ErrPriority(err) > roachpb.ErrPriority(r.resultWriterMu.row.Err()) { + if kvpb.ErrPriority(err) > kvpb.ErrPriority(r.resultWriterMu.row.Err()) { if r.txn != nil { - if retryErr := (*roachpb.UnhandledRetryableError)(nil); errors.As(err, &retryErr) { + if retryErr := (*kvpb.UnhandledRetryableError)(nil); errors.As(err, &retryErr) { // Update the txn in response to remote errors. In the // non-DistSQL world, the TxnCoordSender handles "unhandled" // retryable errors, but this one is coming from a distributed diff --git a/pkg/sql/distsql_running_test.go b/pkg/sql/distsql_running_test.go index bedd9c3915c7..de4b2400b039 100644 --- a/pkg/sql/distsql_running_test.go +++ b/pkg/sql/distsql_running_test.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql/clusterunique" @@ -224,14 +225,14 @@ func TestDistSQLReceiverErrorRanking(t *testing.T) { &SessionTracing{}, ) - retryErr := roachpb.NewErrorWithTxn( - roachpb.NewTransactionRetryError( - roachpb.RETRY_SERIALIZABLE, "test err"), + retryErr := kvpb.NewErrorWithTxn( + kvpb.NewTransactionRetryError( + kvpb.RETRY_SERIALIZABLE, "test err"), txn.TestingCloneTxn()).GoError() - abortErr := roachpb.NewErrorWithTxn( - roachpb.NewTransactionAbortedError( - roachpb.ABORT_REASON_ABORTED_RECORD_FOUND), + abortErr := kvpb.NewErrorWithTxn( + kvpb.NewTransactionAbortedError( + kvpb.ABORT_REASON_ABORTED_RECORD_FOUND), txn.TestingCloneTxn()).GoError() errs := []struct { diff --git a/pkg/sql/drop_test.go b/pkg/sql/drop_test.go index 60e2a2f4373d..a95d2f81c3ad 100644 --- a/pkg/sql/drop_test.go +++ b/pkg/sql/drop_test.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" @@ -1098,7 +1099,7 @@ WHERE defer filterState.Unlock() return filterState.txnID } - rf.setFilter(func(ctx context.Context, request *roachpb.BatchRequest) *roachpb.Error { + rf.setFilter(func(ctx context.Context, request *kvpb.BatchRequest) *kvpb.Error { if request.Txn == nil || request.Txn.Name != sql.SQLTxnName { return nil } @@ -1107,8 +1108,8 @@ WHERE if filterState.txnID != (uuid.UUID{}) { return nil } - if scanRequest, ok := request.GetArg(roachpb.Scan); ok { - scan := scanRequest.(*roachpb.ScanRequest) + if scanRequest, ok := request.GetArg(kvpb.Scan); ok { + scan := scanRequest.(*kvpb.ScanRequest) if scan.Span().Overlaps(tableSpan) { filterState.txnID = request.Txn.ID } @@ -1137,7 +1138,7 @@ WHERE // fail. We'll want to ensure that we get a retriable error. Use the below // pattern to detect when the user transaction has finished planning and is // now executing: we don't want to inject the error during planning. - rf.setFilter(func(ctx context.Context, request *roachpb.BatchRequest) *roachpb.Error { + rf.setFilter(func(ctx context.Context, request *kvpb.BatchRequest) *kvpb.Error { if request.Txn == nil { return nil } @@ -1151,11 +1152,11 @@ WHERE default: return nil } - if getRequest, ok := request.GetArg(roachpb.Get); ok { - put := getRequest.(*roachpb.GetRequest) + if getRequest, ok := request.GetArg(kvpb.Get); ok { + put := getRequest.(*kvpb.GetRequest) if put.Key.Equal(catalogkeys.MakeDescMetadataKey(keys.SystemSQLCodec, descpb.ID(tableID))) { filterState.txnID = uuid.UUID{} - return roachpb.NewError(roachpb.NewReadWithinUncertaintyIntervalError( + return kvpb.NewError(kvpb.NewReadWithinUncertaintyIntervalError( request.Txn.ReadTimestamp, hlc.ClockTimestamp{}, request.Txn, afterInsert, hlc.ClockTimestamp{})) } } diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index e4d4ad020b97..f7c1283eb200 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -41,6 +41,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvtenant" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangecache" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" "github.com/cockroachdb/cockroach/pkg/multitenant" @@ -1642,7 +1643,7 @@ type BackupRestoreTestingKnobs struct { // RunAfterExportingSpanEntry allows blocking the BACKUP job after a single // span has been exported. - RunAfterExportingSpanEntry func(ctx context.Context, response *roachpb.ExportResponse) + RunAfterExportingSpanEntry func(ctx context.Context, response *kvpb.ExportResponse) // BackupMonitor is used to overwrite the monitor used by backup during // testing. This is typically the bulk mem monitor if not diff --git a/pkg/sql/execinfra/BUILD.bazel b/pkg/sql/execinfra/BUILD.bazel index c2200d88b730..f52b7510f725 100644 --- a/pkg/sql/execinfra/BUILD.bazel +++ b/pkg/sql/execinfra/BUILD.bazel @@ -31,6 +31,7 @@ go_library( "//pkg/kv", "//pkg/kv/kvclient/kvcoord", "//pkg/kv/kvclient/rangecache", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/diskmap", "//pkg/kv/kvserver/kvserverbase", "//pkg/kv/kvserver/protectedts", diff --git a/pkg/sql/execinfra/processorsbase.go b/pkg/sql/execinfra/processorsbase.go index 182cb04d4069..298c0a4a1b68 100644 --- a/pkg/sql/execinfra/processorsbase.go +++ b/pkg/sql/execinfra/processorsbase.go @@ -14,7 +14,7 @@ import ( "context" "math" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" @@ -590,14 +590,14 @@ func (pb *ProcessorBaseNoHelper) DrainHelper() *execinfrapb.ProducerMetadata { // ShouldSwallowReadWithinUncertaintyIntervalError examines meta and returns // true if it should be swallowed and not propagated further. It is the case if -// meta contains roachpb.ReadWithinUncertaintyIntervalError. +// meta contains kvpb.ReadWithinUncertaintyIntervalError. func ShouldSwallowReadWithinUncertaintyIntervalError(meta *execinfrapb.ProducerMetadata) bool { if err := meta.Err; err != nil { // We only look for UnhandledRetryableErrors. Local reads (which would // be transformed by the Root TxnCoordSender into // TransactionRetryWithProtoRefreshErrors) don't have any uncertainty. - if ure := (*roachpb.UnhandledRetryableError)(nil); errors.As(err, &ure) { - if _, uncertain := ure.PErr.GetDetail().(*roachpb.ReadWithinUncertaintyIntervalError); uncertain { + if ure := (*kvpb.UnhandledRetryableError)(nil); errors.As(err, &ure) { + if _, uncertain := ure.PErr.GetDetail().(*kvpb.ReadWithinUncertaintyIntervalError); uncertain { return true } } diff --git a/pkg/sql/execinfrapb/BUILD.bazel b/pkg/sql/execinfrapb/BUILD.bazel index 597af8d5f101..85b8f1de24d7 100644 --- a/pkg/sql/execinfrapb/BUILD.bazel +++ b/pkg/sql/execinfrapb/BUILD.bazel @@ -115,6 +115,7 @@ proto_library( deps = [ "//pkg/cloud/cloudpb:cloudpb_proto", "//pkg/jobs/jobspb:jobspb_proto", + "//pkg/kv/kvpb:kvpb_proto", "//pkg/roachpb:roachpb_proto", "//pkg/sql/catalog/catenumpb:catenumpb_proto", "//pkg/sql/catalog/descpb:descpb_proto", @@ -142,6 +143,7 @@ go_proto_library( deps = [ "//pkg/cloud/cloudpb", "//pkg/jobs/jobspb", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/sql/catalog/catenumpb", "//pkg/sql/catalog/catpb", # keep diff --git a/pkg/sql/execinfrapb/component_stats.proto b/pkg/sql/execinfrapb/component_stats.proto index 4c478aae0bdb..7a473addd72e 100644 --- a/pkg/sql/execinfrapb/component_stats.proto +++ b/pkg/sql/execinfrapb/component_stats.proto @@ -15,7 +15,7 @@ option go_package = "execinfrapb"; import "gogoproto/gogo.proto"; import "util/optional/optional.proto"; -import "roachpb/api.proto"; +import "kv/kvpb/api.proto"; // ComponentID identifies a component in a flow. There are multiple types of // components (e.g. processors, streams); each component of a certain type has diff --git a/pkg/sql/execinfrapb/data.proto b/pkg/sql/execinfrapb/data.proto index 0fea0508220b..d41e5c660046 100644 --- a/pkg/sql/execinfrapb/data.proto +++ b/pkg/sql/execinfrapb/data.proto @@ -19,7 +19,7 @@ syntax = "proto2"; package cockroach.sql.distsqlrun; option go_package = "execinfrapb"; -import "roachpb/api.proto"; +import "kv/kvpb/api.proto"; import "roachpb/data.proto"; import "errorspb/errors.proto"; import "sql/catalog/catenumpb/encoded_datum.proto"; diff --git a/pkg/sql/execinfrapb/processors_bulk_io.proto b/pkg/sql/execinfrapb/processors_bulk_io.proto index 2a340b433448..56973906da59 100644 --- a/pkg/sql/execinfrapb/processors_bulk_io.proto +++ b/pkg/sql/execinfrapb/processors_bulk_io.proto @@ -25,7 +25,7 @@ import "sql/catalog/descpb/structured.proto"; import "util/hlc/timestamp.proto"; import "gogoproto/gogo.proto"; import "roachpb/data.proto"; -import "roachpb/api.proto"; +import "kv/kvpb/api.proto"; import "cloud/cloudpb/external_storage.proto"; // BackfillerSpec is the specification for a "schema change backfiller". diff --git a/pkg/sql/execstats/BUILD.bazel b/pkg/sql/execstats/BUILD.bazel index 0111eb64a2c8..67d10364fa89 100644 --- a/pkg/sql/execstats/BUILD.bazel +++ b/pkg/sql/execstats/BUILD.bazel @@ -12,7 +12,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/base", - "//pkg/roachpb", + "//pkg/kv/kvpb", "//pkg/sql/execinfrapb", "//pkg/util/buildutil", "//pkg/util/optional", @@ -35,7 +35,7 @@ go_test( embed = [":execstats"], deps = [ "//pkg/base", - "//pkg/roachpb", + "//pkg/kv/kvpb", "//pkg/security/securityassets", "//pkg/security/securitytest", "//pkg/server", diff --git a/pkg/sql/execstats/stats.go b/pkg/sql/execstats/stats.go index 81cc5b15fb76..025069371136 100644 --- a/pkg/sql/execstats/stats.go +++ b/pkg/sql/execstats/stats.go @@ -14,7 +14,7 @@ import ( "context" "time" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/util/optional" "github.com/cockroachdb/cockroach/pkg/util/tracing" @@ -37,14 +37,14 @@ func ShouldCollectStats(ctx context.Context, collectStats bool) bool { // the trace are included. func GetCumulativeContentionTime( ctx context.Context, recording tracingpb.Recording, -) (time.Duration, []roachpb.ContentionEvent) { +) (time.Duration, []kvpb.ContentionEvent) { var cumulativeContentionTime time.Duration if recording == nil { recording = tracing.SpanFromContext(ctx).GetConfiguredRecording() } - var contentionEvents []roachpb.ContentionEvent - var ev roachpb.ContentionEvent + var contentionEvents []kvpb.ContentionEvent + var ev kvpb.ContentionEvent for i := range recording { recording[i].Structured(func(any *pbtypes.Any, _ time.Time) { if !pbtypes.Is(any, &ev) { @@ -119,8 +119,8 @@ func GetScanStats(ctx context.Context, recording tracingpb.Recording) (scanStats if recording == nil { recording = tracing.SpanFromContext(ctx).GetRecording(tracingpb.RecordingStructured) } - var ss roachpb.ScanStats - var tc roachpb.TenantConsumption + var ss kvpb.ScanStats + var tc kvpb.TenantConsumption for i := range recording { recording[i].Structured(func(any *pbtypes.Any, _ time.Time) { if pbtypes.Is(any, &ss) { diff --git a/pkg/sql/execstats/traceanalyzer.go b/pkg/sql/execstats/traceanalyzer.go index 6f4d574eb102..738006ab768e 100644 --- a/pkg/sql/execstats/traceanalyzer.go +++ b/pkg/sql/execstats/traceanalyzer.go @@ -14,7 +14,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/util/buildutil" "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" @@ -152,7 +152,7 @@ type QueryLevelStats struct { MvccRangeKeySkippedPoints int64 NetworkMessages int64 ContentionTime time.Duration - ContentionEvents []roachpb.ContentionEvent + ContentionEvents []kvpb.ContentionEvent RUEstimate int64 CPUTime time.Duration } @@ -297,7 +297,7 @@ func (a *TraceAnalyzer) ProcessStats() error { } var errs error - var allContentionEvents []roachpb.ContentionEvent + var allContentionEvents []kvpb.ContentionEvent // Process processorStats. for _, stats := range a.processorStats { if stats == nil { diff --git a/pkg/sql/execstats/traceanalyzer_test.go b/pkg/sql/execstats/traceanalyzer_test.go index 5577fa85a345..11b72a037b77 100644 --- a/pkg/sql/execstats/traceanalyzer_test.go +++ b/pkg/sql/execstats/traceanalyzer_test.go @@ -18,7 +18,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfra/execopnode" @@ -240,7 +240,7 @@ func TestTraceAnalyzerProcessStats(t *testing.T) { } func TestQueryLevelStatsAccumulate(t *testing.T) { - aEvent := roachpb.ContentionEvent{Duration: 7 * time.Second} + aEvent := kvpb.ContentionEvent{Duration: 7 * time.Second} a := execstats.QueryLevelStats{ NetworkBytesSent: 1, MaxMemUsage: 2, @@ -250,7 +250,7 @@ func TestQueryLevelStatsAccumulate(t *testing.T) { KVTime: 5 * time.Second, NetworkMessages: 6, ContentionTime: 7 * time.Second, - ContentionEvents: []roachpb.ContentionEvent{aEvent}, + ContentionEvents: []kvpb.ContentionEvent{aEvent}, MaxDiskUsage: 8, RUEstimate: 9, CPUTime: 10 * time.Second, @@ -268,7 +268,7 @@ func TestQueryLevelStatsAccumulate(t *testing.T) { MvccRangeKeyContainedPoints: 22, MvccRangeKeySkippedPoints: 23, } - bEvent := roachpb.ContentionEvent{Duration: 14 * time.Second} + bEvent := kvpb.ContentionEvent{Duration: 14 * time.Second} b := execstats.QueryLevelStats{ NetworkBytesSent: 8, MaxMemUsage: 9, @@ -278,7 +278,7 @@ func TestQueryLevelStatsAccumulate(t *testing.T) { KVTime: 12 * time.Second, NetworkMessages: 13, ContentionTime: 14 * time.Second, - ContentionEvents: []roachpb.ContentionEvent{bEvent}, + ContentionEvents: []kvpb.ContentionEvent{bEvent}, MaxDiskUsage: 15, RUEstimate: 16, CPUTime: 17 * time.Second, @@ -305,7 +305,7 @@ func TestQueryLevelStatsAccumulate(t *testing.T) { KVTime: 17 * time.Second, NetworkMessages: 19, ContentionTime: 21 * time.Second, - ContentionEvents: []roachpb.ContentionEvent{aEvent, bEvent}, + ContentionEvents: []kvpb.ContentionEvent{aEvent, bEvent}, MaxDiskUsage: 15, RUEstimate: 25, CPUTime: 27 * time.Second, diff --git a/pkg/sql/flowinfra/BUILD.bazel b/pkg/sql/flowinfra/BUILD.bazel index 6662f4ab5898..ef44773a2e7c 100644 --- a/pkg/sql/flowinfra/BUILD.bazel +++ b/pkg/sql/flowinfra/BUILD.bazel @@ -75,6 +75,7 @@ go_test( "//pkg/gossip", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/kv/kvserver/kvserverbase", "//pkg/roachpb", diff --git a/pkg/sql/flowinfra/cluster_test.go b/pkg/sql/flowinfra/cluster_test.go index 55f25345fd2b..eabe12fbb56f 100644 --- a/pkg/sql/flowinfra/cluster_test.go +++ b/pkg/sql/flowinfra/cluster_test.go @@ -22,6 +22,7 @@ import ( _ "github.com/cockroachdb/cockroach/pkg/ccl/kvccl/kvtenantccl" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -528,8 +529,8 @@ func TestDistSQLReadsFillGatewayID(t *testing.T) { UseDatabase: "test", Knobs: base.TestingKnobs{Store: &kvserver.StoreTestingKnobs{ EvalKnobs: kvserverbase.BatchEvalTestingKnobs{ - TestingEvalFilter: func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { - scanReq, ok := filterArgs.Req.(*roachpb.ScanRequest) + TestingEvalFilter: func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { + scanReq, ok := filterArgs.Req.(*kvpb.ScanRequest) if !ok { return nil } @@ -542,7 +543,7 @@ func TestDistSQLReadsFillGatewayID(t *testing.T) { atomic.StoreInt64(&foundReq, 1) if gw := filterArgs.Hdr.GatewayNodeID; gw != expectedGateway { - return roachpb.NewErrorf( + return kvpb.NewErrorf( "expected all scans to have gateway 3, found: %d", gw) } diff --git a/pkg/sql/gcjob/BUILD.bazel b/pkg/sql/gcjob/BUILD.bazel index a3ffd1532e86..112f92de88b5 100644 --- a/pkg/sql/gcjob/BUILD.bazel +++ b/pkg/sql/gcjob/BUILD.bazel @@ -25,6 +25,7 @@ go_library( "//pkg/kv", "//pkg/kv/kvclient", "//pkg/kv/kvclient/kvcoord", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/protectedts", "//pkg/kv/kvserver/protectedts/ptpb", "//pkg/multitenant/mtinfopb", diff --git a/pkg/sql/gcjob/gc_job.go b/pkg/sql/gcjob/gc_job.go index c20ef5de7859..6c30ad8474f5 100644 --- a/pkg/sql/gcjob/gc_job.go +++ b/pkg/sql/gcjob/gc_job.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -408,8 +409,8 @@ func waitForEmptyPrefix( func checkForEmptySpan(ctx context.Context, db *kv.DB, from, to roachpb.Key) (empty bool, _ error) { var ba kv.Batch ba.Header.MaxSpanRequestKeys = 1 - ba.AddRawRequest(&roachpb.IsSpanEmptyRequest{ - RequestHeader: roachpb.RequestHeader{ + ba.AddRawRequest(&kvpb.IsSpanEmptyRequest{ + RequestHeader: kvpb.RequestHeader{ Key: from, EndKey: to, }, }) diff --git a/pkg/sql/gcjob/table_garbage_collection.go b/pkg/sql/gcjob/table_garbage_collection.go index 849dd67d6d9c..2edbf2bdb627 100644 --- a/pkg/sql/gcjob/table_garbage_collection.go +++ b/pkg/sql/gcjob/table_garbage_collection.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/spanconfig" @@ -160,8 +161,8 @@ func clearSpanData( endKey = span.EndKey } var b kv.Batch - b.AddRawRequest(&roachpb.ClearRangeRequest{ - RequestHeader: roachpb.RequestHeader{ + b.AddRawRequest(&kvpb.ClearRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: lastKey.AsRawKey(), EndKey: endKey.AsRawKey(), }, @@ -242,14 +243,14 @@ func deleteAllSpanData( endKey = span.EndKey } var b kv.Batch - b.AdmissionHeader = roachpb.AdmissionHeader{ + b.AdmissionHeader = kvpb.AdmissionHeader{ Priority: int32(admissionpb.BulkNormalPri), CreateTime: timeutil.Now().UnixNano(), - Source: roachpb.AdmissionHeader_FROM_SQL, + Source: kvpb.AdmissionHeader_FROM_SQL, NoMemoryReservedAtSource: true, } - b.AddRawRequest(&roachpb.DeleteRangeRequest{ - RequestHeader: roachpb.RequestHeader{ + b.AddRawRequest(&kvpb.DeleteRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: lastKey.AsRawKey(), EndKey: endKey.AsRawKey(), }, diff --git a/pkg/sql/gcjob_test/BUILD.bazel b/pkg/sql/gcjob_test/BUILD.bazel index af9e229d561a..68f371e18eaa 100644 --- a/pkg/sql/gcjob_test/BUILD.bazel +++ b/pkg/sql/gcjob_test/BUILD.bazel @@ -15,6 +15,7 @@ go_test( "//pkg/jobs/jobspb", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/multitenant/mtinfopb", "//pkg/roachpb", diff --git a/pkg/sql/gcjob_test/gc_job_test.go b/pkg/sql/gcjob_test/gc_job_test.go index c3287225d076..4d9ca4d7a19f 100644 --- a/pkg/sql/gcjob_test/gc_job_test.go +++ b/pkg/sql/gcjob_test/gc_job_test.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -274,16 +275,16 @@ func TestGCJobRetry(t *testing.T) { params := base.TestServerArgs{Settings: cs} params.Knobs.JobsTestingKnobs = jobs.NewTestingKnobsWithShortIntervals() params.Knobs.Store = &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(ctx context.Context, request *roachpb.BatchRequest) *roachpb.Error { - r, ok := request.GetArg(roachpb.DeleteRange) - if !ok || !r.(*roachpb.DeleteRangeRequest).UseRangeTombstone { + TestingRequestFilter: func(ctx context.Context, request *kvpb.BatchRequest) *kvpb.Error { + r, ok := request.GetArg(kvpb.DeleteRange) + if !ok || !r.(*kvpb.DeleteRangeRequest).UseRangeTombstone { return nil } if failed.Load().(bool) { return nil } failed.Store(true) - return roachpb.NewError(&roachpb.BatchTimestampBeforeGCError{ + return kvpb.NewError(&kvpb.BatchTimestampBeforeGCError{ Timestamp: hlc.Timestamp{}, Threshold: hlc.Timestamp{}, }) @@ -583,13 +584,13 @@ func TestDropIndexWithDroppedDescriptor(t *testing.T) { if !beforeDelRange { knobs.Store = &kvserver.StoreTestingKnobs{ TestingRequestFilter: func( - ctx context.Context, request *roachpb.BatchRequest, - ) *roachpb.Error { - req, ok := request.GetArg(roachpb.DeleteRange) + ctx context.Context, request *kvpb.BatchRequest, + ) *kvpb.Error { + req, ok := request.GetArg(kvpb.DeleteRange) if !ok { return nil } - dr := req.(*roachpb.DeleteRangeRequest) + dr := req.(*kvpb.DeleteRangeRequest) if !dr.UseRangeTombstone { return nil } diff --git a/pkg/sql/importer/BUILD.bazel b/pkg/sql/importer/BUILD.bazel index c45fbc5fb208..2a25289cdd50 100644 --- a/pkg/sql/importer/BUILD.bazel +++ b/pkg/sql/importer/BUILD.bazel @@ -44,6 +44,7 @@ go_library( "//pkg/jobs/jobspb", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/kvserverbase", "//pkg/kv/kvserver/protectedts", "//pkg/roachpb", @@ -176,6 +177,7 @@ go_test( "//pkg/keys", "//pkg/keyvisualizer", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/kv/kvserver/kvserverbase", "//pkg/roachpb", diff --git a/pkg/sql/importer/exportcsv_test.go b/pkg/sql/importer/exportcsv_test.go index 63e79794e0de..2ba942473dde 100644 --- a/pkg/sql/importer/exportcsv_test.go +++ b/pkg/sql/importer/exportcsv_test.go @@ -31,8 +31,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descidgen" @@ -643,20 +643,20 @@ func TestProcessorEncountersUncertaintyError(t *testing.T) { Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(_ context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + TestingRequestFilter: func(_ context.Context, ba *kvpb.BatchRequest) *kvpb.Error { if atomic.LoadInt64(&trapRead) == 0 { return nil } // We're going to trap a read for the rows [6,10]. - req, ok := ba.GetArg(roachpb.Scan) + req, ok := ba.GetArg(kvpb.Scan) if !ok { return nil } - key := req.(*roachpb.ScanRequest).Key.String() + key := req.(*kvpb.ScanRequest).Key.String() if strings.Contains(key, "/6") { waitForUnblock() - return roachpb.NewError( - roachpb.NewReadWithinUncertaintyIntervalError( + return kvpb.NewError( + kvpb.NewReadWithinUncertaintyIntervalError( ba.Timestamp, /* readTs */ hlc.ClockTimestamp{}, /* localUncertaintyLimit */ ba.Txn, /* txn */ diff --git a/pkg/sql/importer/import_job.go b/pkg/sql/importer/import_job.go index 5fab109501b9..dac0bc1cee10 100644 --- a/pkg/sql/importer/import_job.go +++ b/pkg/sql/importer/import_job.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/joberror" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" @@ -293,7 +294,7 @@ func (r *importResumer) Resume(ctx context.Context, execCtx interface{}) error { pkIDs := make(map[uint64]struct{}, len(details.Tables)) for _, t := range details.Tables { - pkIDs[roachpb.BulkOpSummaryID(uint64(t.Desc.ID), uint64(t.Desc.PrimaryIndex.ID))] = struct{}{} + pkIDs[kvpb.BulkOpSummaryID(uint64(t.Desc.ID), uint64(t.Desc.PrimaryIndex.ID))] = struct{}{} } r.res.DataSize = res.DataSize for id, count := range res.EntryCounts { @@ -930,7 +931,7 @@ func parseAndCreateBundleTableDescs( // publishTables updates the status of imported tables from OFFLINE to PUBLIC. func (r *importResumer) publishTables( - ctx context.Context, execCfg *sql.ExecutorConfig, res roachpb.BulkOpSummary, + ctx context.Context, execCfg *sql.ExecutorConfig, res kvpb.BulkOpSummary, ) error { details := r.job.Details().(jobspb.ImportDetails) // Tables should only be published once. @@ -1018,13 +1019,13 @@ func (r *importResumer) publishTables( // writeStubStatisticsForImportedTables writes "stub" statistics for new tables // created during an import. func (r *importResumer) writeStubStatisticsForImportedTables( - ctx context.Context, execCfg *sql.ExecutorConfig, res roachpb.BulkOpSummary, + ctx context.Context, execCfg *sql.ExecutorConfig, res kvpb.BulkOpSummary, ) { details := r.job.Details().(jobspb.ImportDetails) for _, tbl := range details.Tables { if tbl.IsNew { desc := tabledesc.NewBuilder(tbl.Desc).BuildImmutableTable() - id := roachpb.BulkOpSummaryID(uint64(desc.GetID()), uint64(desc.GetPrimaryIndexID())) + id := kvpb.BulkOpSummaryID(uint64(desc.GetID()), uint64(desc.GetPrimaryIndexID())) rowCount := uint64(res.EntryCounts[id]) // TODO(michae2): collect distinct and null counts during import. distinctCount := uint64(float64(rowCount) * memo.UnknownDistinctCountRatio) @@ -1234,7 +1235,7 @@ func ingestWithRetry( walltime int64, testingKnobs importTestingKnobs, procsPerNode int, -) (roachpb.BulkOpSummary, error) { +) (kvpb.BulkOpSummary, error) { ctx, sp := tracing.ChildSpan(ctx, "importer.ingestWithRetry") defer sp.Finish() @@ -1249,7 +1250,7 @@ func ingestWithRetry( // We want to retry an import if there are transient failures (i.e. worker // nodes dying), so if we receive a retryable error, re-plan and retry the // import. - var res roachpb.BulkOpSummary + var res kvpb.BulkOpSummary var err error for r := retry.StartWithCtx(ctx, retryOpts); r.Next(); { for { @@ -1264,7 +1265,7 @@ func ingestWithRetry( break } - if errors.HasType(err, &roachpb.InsufficientSpaceError{}) { + if errors.HasType(err, &kvpb.InsufficientSpaceError{}) { return res, jobs.MarkPauseRequestError(errors.UnwrapAll(err)) } @@ -1502,7 +1503,7 @@ func (r *importResumer) dropTables( // admin knob (e.g. ALTER TABLE REVERT TO SYSTEM TIME) if anything goes wrong. ts := hlc.Timestamp{WallTime: details.Walltime}.Prev() if useDeleteRange { - predicates := roachpb.DeleteRangePredicates{StartTime: ts} + predicates := kvpb.DeleteRangePredicates{StartTime: ts} if err := sql.DeleteTableWithPredicate( ctx, execCfg.DB, diff --git a/pkg/sql/importer/import_processor.go b/pkg/sql/importer/import_processor.go index 27cdef863ac0..a01a551fb214 100644 --- a/pkg/sql/importer/import_processor.go +++ b/pkg/sql/importer/import_processor.go @@ -19,6 +19,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" @@ -132,7 +133,7 @@ type readImportDataProcessor struct { seqChunkProvider *row.SeqChunkProvider importErr error - summary *roachpb.BulkOpSummary + summary *kvpb.BulkOpSummary } var ( @@ -219,7 +220,7 @@ func (idp *readImportDataProcessor) Next() (rowenc.EncDatumRow, *execinfrapb.Pro } // Once the import is done, send back to the controller the serialized - // summary of the import operation. For more info see roachpb.BulkOpSummary. + // summary of the import operation. For more info see kvpb.BulkOpSummary. countsBytes, err := protoutil.Marshal(idp.summary) idp.MoveToDraining(err) if err != nil { @@ -359,7 +360,7 @@ func ingestKvs( spec *execinfrapb.ReadImportDataSpec, progCh chan execinfrapb.RemoteProducerMetadata_BulkProcessorProgress, kvCh <-chan row.KVBatch, -) (*roachpb.BulkOpSummary, error) { +) (*kvpb.BulkOpSummary, error) { ctx, span := tracing.ChildSpan(ctx, "import-ingest-kvs") defer span.Finish() @@ -435,13 +436,13 @@ func ingestKvs( bulkSummaryMu := &struct { syncutil.Mutex - summary roachpb.BulkOpSummary + summary kvpb.BulkOpSummary }{} // When the PK adder flushes, everything written has been flushed, so we set // pkFlushedRow to writtenRow. Additionally if the indexAdder is empty then we // can treat it as flushed as well (in case we're not adding anything to it). - pkIndexAdder.SetOnFlush(func(summary roachpb.BulkOpSummary) { + pkIndexAdder.SetOnFlush(func(summary kvpb.BulkOpSummary) { for i, emitted := range writtenRow { atomic.StoreInt64(&pkFlushedRow[i], emitted) bulkSummaryMu.Lock() @@ -454,7 +455,7 @@ func ingestKvs( } } }) - indexAdder.SetOnFlush(func(summary roachpb.BulkOpSummary) { + indexAdder.SetOnFlush(func(summary kvpb.BulkOpSummary) { for i, emitted := range writtenRow { atomic.StoreInt64(&idxFlushedRow[i], emitted) bulkSummaryMu.Lock() diff --git a/pkg/sql/importer/import_processor_planning.go b/pkg/sql/importer/import_processor_planning.go index c88f3afeda3b..645e842bcbf0 100644 --- a/pkg/sql/importer/import_processor_planning.go +++ b/pkg/sql/importer/import_processor_planning.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/settings" @@ -66,7 +67,7 @@ func distImport( walltime int64, testingKnobs importTestingKnobs, procsPerNode int, -) (roachpb.BulkOpSummary, error) { +) (kvpb.BulkOpSummary, error) { ctx, sp := tracing.ChildSpan(ctx, "importer.distImport") defer sp.Finish() @@ -117,7 +118,7 @@ func distImport( p, planCtx, err := makePlan(ctx, dsp) if err != nil { - return roachpb.BulkOpSummary{}, err + return kvpb.BulkOpSummary{}, err } evalCtx := planCtx.ExtendedEvalCtx @@ -126,7 +127,7 @@ func distImport( // data written since the last time we update the job progress. accumulatedBulkSummary := struct { syncutil.Mutex - roachpb.BulkOpSummary + kvpb.BulkOpSummary }{} accumulatedBulkSummary.Lock() accumulatedBulkSummary.BulkOpSummary = getLastImportSummary(job) @@ -151,7 +152,7 @@ func distImport( return 0.0 }, ); err != nil { - return roachpb.BulkOpSummary{}, err + return kvpb.BulkOpSummary{}, err } } @@ -202,9 +203,9 @@ func distImport( return nil } - var res roachpb.BulkOpSummary + var res kvpb.BulkOpSummary rowResultWriter := sql.NewCallbackResultWriter(func(ctx context.Context, row tree.Datums) error { - var counts roachpb.BulkOpSummary + var counts kvpb.BulkOpSummary if err := protoutil.Unmarshal([]byte(*row[0].(*tree.DBytes)), &counts); err != nil { return err } @@ -214,7 +215,7 @@ func distImport( if evalCtx.Codec.ForSystemTenant() { if err := presplitTableBoundaries(ctx, execCtx.ExecCfg(), tables); err != nil { - return roachpb.BulkOpSummary{}, err + return kvpb.BulkOpSummary{}, err } } @@ -276,13 +277,13 @@ func distImport( g.GoCtx(replanChecker) if err := g.Wait(); err != nil { - return roachpb.BulkOpSummary{}, err + return kvpb.BulkOpSummary{}, err } return res, nil } -func getLastImportSummary(job *jobs.Job) roachpb.BulkOpSummary { +func getLastImportSummary(job *jobs.Job) kvpb.BulkOpSummary { progress := job.Progress() importProgress := progress.GetImport() return importProgress.Summary diff --git a/pkg/sql/importer/import_processor_test.go b/pkg/sql/importer/import_processor_test.go index 372030ec9b5d..9a0dfd067723 100644 --- a/pkg/sql/importer/import_processor_test.go +++ b/pkg/sql/importer/import_processor_test.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" @@ -203,7 +204,7 @@ func (r *errorReportingRowReceiver) ProducerDone() {} // A do nothing bulk adder implementation. type doNothingKeyAdder struct { onKeyAdd func(key roachpb.Key) - onFlush func(summary roachpb.BulkOpSummary) + onFlush func(summary kvpb.BulkOpSummary) } var _ kvserverbase.BulkAdder = &doNothingKeyAdder{} @@ -217,16 +218,16 @@ func (a *doNothingKeyAdder) Add(_ context.Context, k roachpb.Key, _ []byte) erro func (a *doNothingKeyAdder) Flush(_ context.Context) error { if a.onFlush != nil { - a.onFlush(roachpb.BulkOpSummary{}) + a.onFlush(kvpb.BulkOpSummary{}) } return nil } -func (*doNothingKeyAdder) IsEmpty() bool { return true } -func (*doNothingKeyAdder) CurrentBufferFill() float32 { return 0 } -func (*doNothingKeyAdder) GetSummary() roachpb.BulkOpSummary { return roachpb.BulkOpSummary{} } -func (*doNothingKeyAdder) Close(_ context.Context) {} -func (a *doNothingKeyAdder) SetOnFlush(f func(_ roachpb.BulkOpSummary)) { a.onFlush = f } +func (*doNothingKeyAdder) IsEmpty() bool { return true } +func (*doNothingKeyAdder) CurrentBufferFill() float32 { return 0 } +func (*doNothingKeyAdder) GetSummary() kvpb.BulkOpSummary { return kvpb.BulkOpSummary{} } +func (*doNothingKeyAdder) Close(_ context.Context) {} +func (a *doNothingKeyAdder) SetOnFlush(f func(_ kvpb.BulkOpSummary)) { a.onFlush = f } var eofOffset int64 = math.MaxInt64 diff --git a/pkg/sql/importer/import_stmt_test.go b/pkg/sql/importer/import_stmt_test.go index 64c3d8043f7f..68cc09ee25dd 100644 --- a/pkg/sql/importer/import_stmt_test.go +++ b/pkg/sql/importer/import_stmt_test.go @@ -40,6 +40,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobstest" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/keyvisualizer" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" @@ -7238,10 +7239,10 @@ func TestUDTChangeDuringImport(t *testing.T) { JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), Store: &kvserver.StoreTestingKnobs{ TestingResponseFilter: jobutils.BulkOpResponseFilter(&allowResponse), - TestingRequestFilter: func(ctx context.Context, br *roachpb.BatchRequest) *roachpb.Error { + TestingRequestFilter: func(ctx context.Context, br *kvpb.BatchRequest) *kvpb.Error { for _, ru := range br.Requests { switch ru.GetInner().(type) { - case *roachpb.AddSSTableRequest: + case *kvpb.AddSSTableRequest: <-requestReceived } } diff --git a/pkg/sql/importer/read_import_base.go b/pkg/sql/importer/read_import_base.go index 3d532188d594..af20d436a336 100644 --- a/pkg/sql/importer/read_import_base.go +++ b/pkg/sql/importer/read_import_base.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cloud" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql/catalog" @@ -53,7 +54,7 @@ func runImport( spec *execinfrapb.ReadImportDataSpec, progCh chan execinfrapb.RemoteProducerMetadata_BulkProcessorProgress, seqChunkProvider *row.SeqChunkProvider, -) (*roachpb.BulkOpSummary, error) { +) (*kvpb.BulkOpSummary, error) { // Used to send ingested import rows to the KV layer. kvCh := make(chan row.KVBatch, 10) @@ -108,7 +109,7 @@ func runImport( // Ingest the KVs that the producer group emitted to the chan and the row result // at the end is one row containing an encoded BulkOpSummary. - var summary *roachpb.BulkOpSummary + var summary *kvpb.BulkOpSummary group.GoCtx(func(ctx context.Context) error { summary, err = ingestKvs(ctx, flowCtx, spec, progCh, kvCh) if err != nil { diff --git a/pkg/sql/insert_fast_path.go b/pkg/sql/insert_fast_path.go index 323fb90259a4..f17c8b107db9 100644 --- a/pkg/sql/insert_fast_path.go +++ b/pkg/sql/insert_fast_path.go @@ -14,6 +14,7 @@ import ( "context" "sync" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" @@ -69,7 +70,7 @@ type insertFastPathRun struct { inputBuf tree.Datums // fkBatch accumulates the FK existence checks. - fkBatch roachpb.BatchRequest + fkBatch kvpb.BatchRequest // fkSpanInfo keeps track of information for each fkBatch.Request entry. fkSpanInfo []insertFastPathFKSpanInfo @@ -188,9 +189,9 @@ func (r *insertFastPathRun) addFKChecks( log.VEventf(ctx, 2, "FKScan %s", span) } reqIdx := len(r.fkBatch.Requests) - r.fkBatch.Requests = append(r.fkBatch.Requests, roachpb.RequestUnion{}) - r.fkBatch.Requests[reqIdx].MustSetInner(&roachpb.ScanRequest{ - RequestHeader: roachpb.RequestHeaderFromSpan(span), + r.fkBatch.Requests = append(r.fkBatch.Requests, kvpb.RequestUnion{}) + r.fkBatch.Requests[reqIdx].MustSetInner(&kvpb.ScanRequest{ + RequestHeader: kvpb.RequestHeaderFromSpan(span), }) r.fkSpanInfo = append(r.fkSpanInfo, insertFastPathFKSpanInfo{ check: c, @@ -216,7 +217,7 @@ func (n *insertFastPathNode) runFKChecks(params runParams) error { } for i := range br.Responses { - resp := br.Responses[i].GetInner().(*roachpb.ScanResponse) + resp := br.Responses[i].GetInner().(*kvpb.ScanResponse) if len(resp.Rows) == 0 { // No results for lookup; generate the violation error. info := n.run.fkSpanInfo[i] @@ -247,7 +248,7 @@ func (n *insertFastPathNode) startExec(params runParams) error { } } maxSpans := len(n.run.fkChecks) * len(n.input) - n.run.fkBatch.Requests = make([]roachpb.RequestUnion, 0, maxSpans) + n.run.fkBatch.Requests = make([]kvpb.RequestUnion, 0, maxSpans) n.run.fkSpanInfo = make([]insertFastPathFKSpanInfo, 0, maxSpans) if len(n.input) > 1 { n.run.fkSpanMap = make(map[string]struct{}, maxSpans) diff --git a/pkg/sql/internal.go b/pkg/sql/internal.go index 1fd57c2fb51f..67eef7d3f1de 100644 --- a/pkg/sql/internal.go +++ b/pkg/sql/internal.go @@ -18,6 +18,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/settings" @@ -1460,7 +1461,7 @@ func (ief *InternalDB) txn( } run = func(ctx context.Context, f kvTxnFunc) error { return db.TxnWithAdmissionControl( - ctx, roachpb.AdmissionHeader_FROM_SQL, priority, steppingMode, f, + ctx, kvpb.AdmissionHeader_FROM_SQL, priority, steppingMode, f, ) } } else if cfg.GetSteppingEnabled() { diff --git a/pkg/sql/metric_test.go b/pkg/sql/metric_test.go index 6c39c8c8e600..71048317238e 100644 --- a/pkg/sql/metric_test.go +++ b/pkg/sql/metric_test.go @@ -17,8 +17,8 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" "github.com/cockroachdb/cockroach/pkg/sql/tests" @@ -192,15 +192,15 @@ func TestAbortCountConflictingWrites(t *testing.T) { // Inject errors on the INSERT below. restarted := false - cmdFilters.AppendFilter(func(args kvserverbase.FilterArgs) *roachpb.Error { + cmdFilters.AppendFilter(func(args kvserverbase.FilterArgs) *kvpb.Error { switch req := args.Req.(type) { // SQL INSERT generates ConditionalPuts for unique indexes (such as the PK). - case *roachpb.ConditionalPutRequest: + case *kvpb.ConditionalPutRequest: if bytes.Contains(req.Value.RawBytes, []byte("marker")) && !restarted { restarted = true - return roachpb.NewErrorWithTxn( - roachpb.NewTransactionAbortedError( - roachpb.ABORT_REASON_ABORTED_RECORD_FOUND), args.Hdr.Txn) + return kvpb.NewErrorWithTxn( + kvpb.NewTransactionAbortedError( + kvpb.ABORT_REASON_ABORTED_RECORD_FOUND), args.Hdr.Txn) } } return nil diff --git a/pkg/sql/mvcc_backfiller_test.go b/pkg/sql/mvcc_backfiller_test.go index a588446852ff..f28fd78d5091 100644 --- a/pkg/sql/mvcc_backfiller_test.go +++ b/pkg/sql/mvcc_backfiller_test.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/keyvisualizer" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/backfill" @@ -86,8 +87,8 @@ func TestIndexBackfillMergeRetry(t *testing.T) { if rand.Intn(2) == 0 { return context.DeadlineExceeded } else { - errAmbiguous := &roachpb.AmbiguousResultError{} - return roachpb.NewError(errAmbiguous).GoError() + errAmbiguous := &kvpb.AmbiguousResultError{} + return kvpb.NewError(errAmbiguous).GoError() } } diff --git a/pkg/sql/opt/norm/BUILD.bazel b/pkg/sql/opt/norm/BUILD.bazel index 6186bce0bf91..68864029a811 100644 --- a/pkg/sql/opt/norm/BUILD.bazel +++ b/pkg/sql/opt/norm/BUILD.bazel @@ -32,7 +32,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/opt/norm", visibility = ["//visibility:public"], deps = [ - "//pkg/roachpb", + "//pkg/kv/kvpb", "//pkg/sql/catalog/colinfo", "//pkg/sql/opt", "//pkg/sql/opt/cat", diff --git a/pkg/sql/opt/norm/fold_constants_funcs.go b/pkg/sql/opt/norm/fold_constants_funcs.go index b530c427c4cf..e36ee1789d3e 100644 --- a/pkg/sql/opt/norm/fold_constants_funcs.go +++ b/pkg/sql/opt/norm/fold_constants_funcs.go @@ -13,7 +13,7 @@ package norm import ( "context" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" "github.com/cockroachdb/cockroach/pkg/sql/opt/memo" @@ -409,7 +409,7 @@ func (c *CustomFuncs) FoldCast(input opt.ScalarExpr, typ *types.T) (_ opt.Scalar // TODO(mgartner): Ideally, casts that can error and cause adverse // side-effects would be marked as volatile so that they are not folded. // That would eliminate the need for this special error handling. - if errors.HasInterface(err, (*roachpb.ErrorDetailInterface)(nil)) { + if errors.HasInterface(err, (*kvpb.ErrorDetailInterface)(nil)) { panic(err) } return nil, false @@ -444,7 +444,7 @@ func (c *CustomFuncs) FoldAssignmentCast( // TODO(mgartner): Ideally, casts that can error and cause adverse // side-effects would be marked as volatile so that they are not folded. // That would eliminate the need for this special error handling. - if errors.HasInterface(err, (*roachpb.ErrorDetailInterface)(nil)) { + if errors.HasInterface(err, (*kvpb.ErrorDetailInterface)(nil)) { panic(err) } return nil, false diff --git a/pkg/sql/pgwire/pgerror/BUILD.bazel b/pkg/sql/pgwire/pgerror/BUILD.bazel index 82026c1bd012..cb9c6b694a6b 100644 --- a/pkg/sql/pgwire/pgerror/BUILD.bazel +++ b/pkg/sql/pgwire/pgerror/BUILD.bazel @@ -44,6 +44,7 @@ go_test( args = ["-test.timeout=55s"], embed = [":pgerror"], deps = [ + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/sql/pgwire/pgcode", "//pkg/util/errorutil/unimplemented", diff --git a/pkg/sql/pgwire/pgerror/errors_test.go b/pkg/sql/pgwire/pgerror/errors_test.go index 452b17e3f9c6..c7a6530e52ae 100644 --- a/pkg/sql/pgwire/pgerror/errors_test.go +++ b/pkg/sql/pgwire/pgerror/errors_test.go @@ -15,7 +15,7 @@ import ( "regexp" "testing" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" ) @@ -56,8 +56,8 @@ func TestPGError(t *testing.T) { } func TestIsSQLRetryableError(t *testing.T) { - errAmbiguous := &roachpb.AmbiguousResultError{} - if !pgerror.IsSQLRetryableError(roachpb.NewError(errAmbiguous).GoError()) { + errAmbiguous := &kvpb.AmbiguousResultError{} + if !pgerror.IsSQLRetryableError(kvpb.NewError(errAmbiguous).GoError()) { t.Fatalf("%s should be a SQLRetryableError", errAmbiguous) } } diff --git a/pkg/sql/pgwire/pgerror/flatten.go b/pkg/sql/pgwire/pgerror/flatten.go index f06ead260c89..9d321d9ee558 100644 --- a/pkg/sql/pgwire/pgerror/flatten.go +++ b/pkg/sql/pgwire/pgerror/flatten.go @@ -93,7 +93,7 @@ func Flatten(err error) *Error { // serializationFailureReasonRegexp captures known failure reasons for // the serialization failure error messages. -// We cannot use roachpb.TransactionRetryReason or roachpb.TransactionAbortedReason +// We cannot use kvpb.TransactionRetryReason or kvpb.TransactionAbortedReason // as this introduces a circular dependency. var serializationFailureReasonRegexp = regexp.MustCompile( `((?:ABORT_|RETRY_)[A-Z_]*|ReadWithinUncertaintyInterval)`, diff --git a/pkg/sql/pgwire/pgerror/flatten_test.go b/pkg/sql/pgwire/pgerror/flatten_test.go index dc99a401adac..3d32933bb4d9 100644 --- a/pkg/sql/pgwire/pgerror/flatten_test.go +++ b/pkg/sql/pgwire/pgerror/flatten_test.go @@ -13,6 +13,7 @@ package pgerror_test import ( "testing" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" @@ -86,14 +87,14 @@ func TestFlatten(t *testing.T) { }, }, { - errors.Wrap(&roachpb.TransactionRetryWithProtoRefreshError{MsgRedactable: "woo"}, ""), + errors.Wrap(&kvpb.TransactionRetryWithProtoRefreshError{MsgRedactable: "woo"}, ""), func(t testutils.T, e *pgerror.Error) { t.CheckRegexpEqual(e.Message, "restart transaction: .* woo") t.CheckEqual(pgcode.MakeCode(e.Code), pgcode.SerializationFailure) }, }, { - errors.Wrap(roachpb.NewAmbiguousResultErrorf("woo"), ""), + errors.Wrap(kvpb.NewAmbiguousResultErrorf("woo"), ""), func(t testutils.T, e *pgerror.Error) { t.CheckRegexpEqual(e.Message, "result is ambiguous.*woo") t.CheckEqual(pgcode.MakeCode(e.Code), pgcode.StatementCompletionUnknown) @@ -101,7 +102,7 @@ func TestFlatten(t *testing.T) { }, { errors.Wrap( - roachpb.NewTransactionRetryWithProtoRefreshError( + kvpb.NewTransactionRetryWithProtoRefreshError( "test", uuid.MakeV4(), roachpb.Transaction{}, @@ -114,8 +115,8 @@ func TestFlatten(t *testing.T) { }, { errors.Wrap( - roachpb.NewTransactionRetryWithProtoRefreshError( - redact.Sprint(roachpb.NewReadWithinUncertaintyIntervalError(hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, hlc.Timestamp{}, hlc.ClockTimestamp{})), + kvpb.NewTransactionRetryWithProtoRefreshError( + redact.Sprint(kvpb.NewReadWithinUncertaintyIntervalError(hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, hlc.Timestamp{}, hlc.ClockTimestamp{})), uuid.MakeV4(), roachpb.Transaction{}, ), @@ -127,8 +128,8 @@ func TestFlatten(t *testing.T) { }, { errors.Wrap( - roachpb.NewTransactionRetryWithProtoRefreshError( - redact.Sprint(roachpb.NewTransactionRetryError(roachpb.RETRY_SERIALIZABLE, "")), + kvpb.NewTransactionRetryWithProtoRefreshError( + redact.Sprint(kvpb.NewTransactionRetryError(kvpb.RETRY_SERIALIZABLE, "")), uuid.MakeV4(), roachpb.Transaction{}, ), @@ -140,8 +141,8 @@ func TestFlatten(t *testing.T) { }, { errors.Wrap( - roachpb.NewTransactionRetryWithProtoRefreshError( - redact.Sprint(roachpb.NewTransactionAbortedError(roachpb.ABORT_REASON_PUSHER_ABORTED)), + kvpb.NewTransactionRetryWithProtoRefreshError( + redact.Sprint(kvpb.NewTransactionAbortedError(kvpb.ABORT_REASON_PUSHER_ABORTED)), uuid.MakeV4(), roachpb.Transaction{}, ), diff --git a/pkg/sql/pgwire/pgerror/pgcode.go b/pkg/sql/pgwire/pgerror/pgcode.go index 01e333e923c3..d3721827d93e 100644 --- a/pkg/sql/pgwire/pgerror/pgcode.go +++ b/pkg/sql/pgwire/pgerror/pgcode.go @@ -113,14 +113,14 @@ func ComputeDefaultCode(err error) pgcode.Code { return pgcode.Code{} } -// ClientVisibleRetryError mirrors roachpb.ClientVisibleRetryError but +// ClientVisibleRetryError mirrors kvpb.ClientVisibleRetryError but // is defined here to avoid an import cycle. type ClientVisibleRetryError interface { ClientVisibleRetryError() } // ClientVisibleAmbiguousError mirrors -// roachpb.ClientVisibleAmbiguousError but is defined here to avoid an +// kvpb.ClientVisibleAmbiguousError but is defined here to avoid an // import cycle. type ClientVisibleAmbiguousError interface { ClientVisibleAmbiguousError() diff --git a/pkg/sql/pgwire/pgerror/wrap_test.go b/pkg/sql/pgwire/pgerror/wrap_test.go index 15c1cc012ddd..e1c2f12c9d89 100644 --- a/pkg/sql/pgwire/pgerror/wrap_test.go +++ b/pkg/sql/pgwire/pgerror/wrap_test.go @@ -13,7 +13,7 @@ package pgerror_test import ( "testing" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/errors" @@ -24,8 +24,8 @@ func TestWrap(t *testing.T) { err error }{ {errors.New("woo")}, - {&roachpb.TransactionRetryWithProtoRefreshError{}}, - {&roachpb.AmbiguousResultError{}}, + {&kvpb.TransactionRetryWithProtoRefreshError{}}, + {&kvpb.AmbiguousResultError{}}, } for i, test := range testData { diff --git a/pkg/sql/physicalplan/BUILD.bazel b/pkg/sql/physicalplan/BUILD.bazel index c00392027ddc..0c1356ad165f 100644 --- a/pkg/sql/physicalplan/BUILD.bazel +++ b/pkg/sql/physicalplan/BUILD.bazel @@ -18,6 +18,7 @@ go_library( "//pkg/keys", "//pkg/kv", "//pkg/kv/kvclient/kvcoord", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/rpc", "//pkg/settings/cluster", diff --git a/pkg/sql/physicalplan/fake_span_resolver.go b/pkg/sql/physicalplan/fake_span_resolver.go index 0da34b939410..08b1cbb36a3d 100644 --- a/pkg/sql/physicalplan/fake_span_resolver.go +++ b/pkg/sql/physicalplan/fake_span_resolver.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/physicalplan/replicaoracle" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -88,7 +89,7 @@ func (fit *fakeSpanResolverIterator) Seek( // read_uncommitted scan outside of the txn to avoid undesired side effects // like breaking tracing and blocking on locks. var b kv.Batch - b.Header.ReadConsistency = roachpb.READ_UNCOMMITTED + b.Header.ReadConsistency = kvpb.READ_UNCOMMITTED if len(span.EndKey) == 0 { // If the EndKey is omitted, then the span represents a point request. // In such case we manually set the EndKey so that the Scan below diff --git a/pkg/sql/relocate_range.go b/pkg/sql/relocate_range.go index 7eabc959e048..7e2a612ca9f9 100644 --- a/pkg/sql/relocate_range.go +++ b/pkg/sql/relocate_range.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/paramparse" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -148,7 +149,7 @@ func (n *relocateRange) relocate(params runParams, rangeDesc roachpb.RangeDescri fromChangeType = roachpb.REMOVE_NON_VOTER } _, err := execCfg.DB.AdminChangeReplicas( - params.ctx, rangeDesc.StartKey, rangeDesc, []roachpb.ReplicationChange{ + params.ctx, rangeDesc.StartKey, rangeDesc, []kvpb.ReplicationChange{ {ChangeType: toChangeType, Target: n.run.toTarget}, {ChangeType: fromChangeType, Target: n.run.fromTarget}, }, diff --git a/pkg/sql/repair.go b/pkg/sql/repair.go index 65a726b473a2..30d1ea2652e6 100644 --- a/pkg/sql/repair.go +++ b/pkg/sql/repair.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" @@ -737,20 +738,20 @@ func (p *planner) ForceDeleteTableData(ctx context.Context, descID int64) error prefix := p.extendedEvalCtx.Codec.TablePrefix(uint32(id)) tableSpan := roachpb.Span{Key: prefix, EndKey: prefix.PrefixEnd()} - requestHeader := roachpb.RequestHeader{ + requestHeader := kvpb.RequestHeader{ Key: tableSpan.Key, EndKey: tableSpan.EndKey, } b := &kv.Batch{} if p.execCfg.Settings.Version.IsActive(ctx, clusterversion.TODODelete_V22_2UseDelRangeInGCJob) && storage.CanUseMVCCRangeTombstones(ctx, p.execCfg.Settings) { - b.AddRawRequest(&roachpb.DeleteRangeRequest{ + b.AddRawRequest(&kvpb.DeleteRangeRequest{ RequestHeader: requestHeader, UseRangeTombstone: true, IdempotentTombstone: true, UpdateRangeDeleteGCHint: true, }) } else { - b.AddRawRequest(&roachpb.ClearRangeRequest{ + b.AddRawRequest(&kvpb.ClearRangeRequest{ RequestHeader: requestHeader, }) } diff --git a/pkg/sql/revert.go b/pkg/sql/revert.go index 3be7aab85b58..446b56a499e6 100644 --- a/pkg/sql/revert.go +++ b/pkg/sql/revert.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql/catalog" @@ -85,8 +86,8 @@ func RevertTables( for len(spans) != 0 { var b kv.Batch for _, span := range spans { - b.AddRawRequest(&roachpb.RevertRangeRequest{ - RequestHeader: roachpb.RequestHeader{ + b.AddRawRequest(&kvpb.RevertRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: span.Key, EndKey: span.EndKey, }, @@ -136,7 +137,7 @@ func DeleteTableWithPredicate( sv *settings.Values, distSender *kvcoord.DistSender, table catalog.TableDescriptor, - predicates roachpb.DeleteRangePredicates, + predicates kvpb.DeleteRangePredicates, batchSize int64, ) error { @@ -175,14 +176,14 @@ func DeleteTableWithPredicate( // returned. resumeCount := 1 for span != nil { - admissionHeader := roachpb.AdmissionHeader{ + admissionHeader := kvpb.AdmissionHeader{ Priority: int32(admissionpb.BulkNormalPri), CreateTime: timeutil.Now().UnixNano(), - Source: roachpb.AdmissionHeader_FROM_SQL, + Source: kvpb.AdmissionHeader_FROM_SQL, NoMemoryReservedAtSource: true, } - delRangeRequest := &roachpb.DeleteRangeRequest{ - RequestHeader: roachpb.RequestHeader{ + delRangeRequest := &kvpb.DeleteRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: span.Key, EndKey: span.EndKey, }, @@ -194,7 +195,7 @@ func DeleteTableWithPredicate( rawResp, err := kv.SendWrappedWithAdmission( ctx, db.NonTransactionalSender(), - roachpb.Header{MaxSpanRequestKeys: batchSize}, + kvpb.Header{MaxSpanRequestKeys: batchSize}, admissionHeader, delRangeRequest) @@ -203,7 +204,7 @@ func DeleteTableWithPredicate( return errors.Wrapf(err.GoError(), "delete range %s - %s", span.Key, span.EndKey) } span = nil - resp := rawResp.(*roachpb.DeleteRangeResponse) + resp := rawResp.(*kvpb.DeleteRangeResponse) if resp.ResumeSpan != nil { if !resp.ResumeSpan.Valid() { return errors.Errorf("invalid resume span: %s", resp.ResumeSpan) diff --git a/pkg/sql/revert_test.go b/pkg/sql/revert_test.go index f28e9d9b3f45..01094847c021 100644 --- a/pkg/sql/revert_test.go +++ b/pkg/sql/revert_test.go @@ -18,7 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/bootstrap" @@ -98,7 +98,7 @@ func TestTableRollback(t *testing.T) { // Delete all keys with values after the targetTime desc := desctestutils.TestingGetPublicTableDescriptor(kv, keys.SystemSQLCodec, "test", "test") - predicates := roachpb.DeleteRangePredicates{StartTime: targetTime} + predicates := kvpb.DeleteRangePredicates{StartTime: targetTime} require.NoError(t, sql.DeleteTableWithPredicate(context.Background(), kv, execCfg.Codec, &s.ClusterSettings().SV, execCfg.DistSender, desc, predicates, 10)) @@ -115,8 +115,8 @@ func TestRevertGCThreshold(t *testing.T) { defer tc.Stopper().Stop(ctx) kvDB := tc.Server(0).DB() - req := &roachpb.RevertRangeRequest{ - RequestHeader: roachpb.RequestHeader{Key: bootstrap.TestingUserTableDataMin(), EndKey: keys.MaxKey}, + req := &kvpb.RevertRangeRequest{ + RequestHeader: kvpb.RequestHeader{Key: bootstrap.TestingUserTableDataMin(), EndKey: keys.MaxKey}, TargetTime: hlc.Timestamp{WallTime: -1}, } _, pErr := kv.SendWrapped(ctx, kvDB.NonTransactionalSender(), req) diff --git a/pkg/sql/row/BUILD.bazel b/pkg/sql/row/BUILD.bazel index 92a88921e3c3..a38dab524196 100644 --- a/pkg/sql/row/BUILD.bazel +++ b/pkg/sql/row/BUILD.bazel @@ -31,6 +31,7 @@ go_library( "//pkg/kv", "//pkg/kv/kvclient/kvcoord", "//pkg/kv/kvclient/kvstreamer", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/concurrency/lock", "//pkg/kv/kvserver/kvserverbase", "//pkg/roachpb", diff --git a/pkg/sql/row/errors.go b/pkg/sql/row/errors.go index 98e5fc20ffde..523b556f4320 100644 --- a/pkg/sql/row/errors.go +++ b/pkg/sql/row/errors.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -35,13 +36,13 @@ import ( func ConvertBatchError(ctx context.Context, tableDesc catalog.TableDescriptor, b *kv.Batch) error { origPErr := b.MustPErr() switch v := origPErr.GetDetail().(type) { - case *roachpb.MinTimestampBoundUnsatisfiableError: + case *kvpb.MinTimestampBoundUnsatisfiableError: return pgerror.WithCandidateCode( origPErr.GoError(), pgcode.UnsatisfiableBoundedStaleness, ) - case *roachpb.ConditionFailedError: + case *kvpb.ConditionFailedError: if origPErr.Index == nil { break } @@ -56,7 +57,7 @@ func ConvertBatchError(ctx context.Context, tableDesc catalog.TableDescriptor, b key := result.Rows[0].Key return NewUniquenessConstraintViolationError(ctx, tableDesc, key, v.ActualValue) - case *roachpb.WriteIntentError: + case *kvpb.WriteIntentError: key := v.Intents[0].Key decodeKeyFn := func() (tableName string, indexName string, colNames []string, values []string, err error) { codec, index, err := decodeKeyCodecAndIndex(tableDesc, key) @@ -80,8 +81,8 @@ func ConvertBatchError(ctx context.Context, tableDesc catalog.TableDescriptor, b // key-value fetch to a user friendly SQL error. func ConvertFetchError(spec *fetchpb.IndexFetchSpec, err error) error { var errs struct { - wi *roachpb.WriteIntentError - bs *roachpb.MinTimestampBoundUnsatisfiableError + wi *kvpb.WriteIntentError + bs *kvpb.MinTimestampBoundUnsatisfiableError } switch { case errors.As(err, &errs.wi): @@ -161,11 +162,11 @@ func decodeKeyValsUsingSpec( // acquire a lock. It uses an IndexFetchSpec for the corresponding index (the // fetch columns in the spec are not used). func newLockNotAvailableError( - reason roachpb.WriteIntentError_Reason, + reason kvpb.WriteIntentError_Reason, decodeKeyFn func() (tableName string, indexName string, colNames []string, values []string, err error), ) error { baseMsg := "could not obtain lock on row" - if reason == roachpb.WriteIntentError_REASON_LOCK_TIMEOUT { + if reason == kvpb.WriteIntentError_REASON_LOCK_TIMEOUT { baseMsg = "canceling statement due to lock timeout on row" } tableName, indexName, colNames, values, err := decodeKeyFn() diff --git a/pkg/sql/row/expr_walker.go b/pkg/sql/row/expr_walker.go index 5c40c851ccbe..a89202767907 100644 --- a/pkg/sql/row/expr_walker.go +++ b/pkg/sql/row/expr_walker.go @@ -18,7 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" @@ -391,7 +391,7 @@ func incrementSequenceByVal( seqValueKey := codec.SequenceKey(uint32(descriptor.GetID())) val, err = kv.IncrementValRetryable(ctx, db, seqValueKey, incrementBy) if err != nil { - if errors.HasType(err, (*roachpb.IntegerOverflowError)(nil)) { + if errors.HasType(err, (*kvpb.IntegerOverflowError)(nil)) { return 0, boundsExceededError(descriptor) } return 0, err diff --git a/pkg/sql/row/fetcher.go b/pkg/sql/row/fetcher.go index 8c224ed7a1b6..a164bf2462ec 100644 --- a/pkg/sql/row/fetcher.go +++ b/pkg/sql/row/fetcher.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catenumpb" @@ -564,7 +565,7 @@ func (rf *Fetcher) StartInconsistentScan( log.Infof(ctx, "starting inconsistent scan at timestamp %v", txnTimestamp) } - sendFn := func(ctx context.Context, ba *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { + sendFn := func(ctx context.Context, ba *kvpb.BatchRequest) (*kvpb.BatchResponse, error) { if now := timeutil.Now(); now.Sub(txnTimestamp.GoTime()) >= maxTimestampAge { // Time to bump the transaction. First commit the old one (should be a no-op). if err := txn.Commit(ctx); err != nil { diff --git a/pkg/sql/row/kv_batch_fetcher.go b/pkg/sql/row/kv_batch_fetcher.go index b7796f65c508..53e527ee523b 100644 --- a/pkg/sql/row/kv_batch_fetcher.go +++ b/pkg/sql/row/kv_batch_fetcher.go @@ -17,6 +17,7 @@ import ( "unsafe" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" @@ -57,8 +58,8 @@ var defaultKVBatchSize = rowinfra.KeyLimit(util.ConstantWithMetamorphicTestValue // sendFunc is the function used to execute a KV batch; normally // wraps (*client.Txn).Send. type sendFunc func( - ctx context.Context, ba *roachpb.BatchRequest, -) (*roachpb.BatchResponse, error) + ctx context.Context, ba *kvpb.BatchRequest, +) (*kvpb.BatchResponse, error) // identifiableSpans is a helper for keeping track of the roachpb.Spans with the // corresponding spanIDs (when necessary). @@ -141,7 +142,7 @@ type txnKVFetcher struct { // scanFormat indicates the scan format that should be used for Scans and // ReverseScans. With COL_BATCH_RESPONSE scan format, indexFetchSpec must be // set. - scanFormat roachpb.ScanFormat + scanFormat kvpb.ScanFormat indexFetchSpec *fetchpb.IndexFetchSpec reverse bool @@ -159,9 +160,9 @@ type txnKVFetcher struct { // least once. alreadyFetched bool batchIdx int - reqsScratch []roachpb.RequestUnion + reqsScratch []kvpb.RequestUnion - responses []roachpb.ResponseUnion + responses []kvpb.ResponseUnion remainingBatches [][]byte // getResponseScratch is reused to return the result of Get requests. @@ -179,7 +180,7 @@ type txnKVFetcher struct { forceProductionKVBatchSize bool // For request and response admission control. - requestAdmissionHeader roachpb.AdmissionHeader + requestAdmissionHeader kvpb.AdmissionHeader responseAdmissionQ *admission.WorkQueue } @@ -238,8 +239,8 @@ func (f *txnKVFetcher) getBatchKeyLimitForIdx(batchIdx int) rowinfra.KeyLimit { func makeTxnKVFetcherDefaultSendFunc(txn *kv.Txn, batchRequestsIssued *int64) sendFunc { return func( ctx context.Context, - ba *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, error) { + ba *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, error) { res, err := txn.Send(ctx, ba) if err != nil { return nil, err.GoError() @@ -258,7 +259,7 @@ type newTxnKVFetcherArgs struct { acc *mon.BoundAccount forceProductionKVBatchSize bool batchRequestsIssued *int64 - requestAdmissionHeader roachpb.AdmissionHeader + requestAdmissionHeader kvpb.AdmissionHeader responseAdmissionQ *admission.WorkQueue } @@ -271,7 +272,7 @@ func newTxnKVFetcherInternal(args newTxnKVFetcherArgs) *txnKVFetcher { f := &txnKVFetcher{ sendFn: args.sendFn, // Default to BATCH_RESPONSE. The caller will override if needed. - scanFormat: roachpb.BATCH_RESPONSE, + scanFormat: kvpb.BATCH_RESPONSE, reverse: args.reverse, lockStrength: GetKeyLockingStrength(args.lockStrength), lockWaitPolicy: getWaitPolicy(args.lockWaitPolicy), @@ -409,13 +410,13 @@ func (f *txnKVFetcher) SetupNextFetch( // fetch retrieves spans from the kv layer. func (f *txnKVFetcher) fetch(ctx context.Context) error { - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Header.WaitPolicy = f.lockWaitPolicy ba.Header.LockTimeout = f.lockTimeout ba.Header.TargetBytes = int64(f.batchBytesLimit) ba.Header.MaxSpanRequestKeys = int64(f.getBatchKeyLimit()) if buildutil.CrdbTestBuild { - if f.scanFormat == roachpb.COL_BATCH_RESPONSE && f.indexFetchSpec == nil { + if f.scanFormat == kvpb.COL_BATCH_RESPONSE && f.indexFetchSpec == nil { return errors.AssertionFailedf("IndexFetchSpec not provided with COL_BATCH_RESPONSE scan format") } } @@ -511,7 +512,7 @@ func (f *txnKVFetcher) fetch(ctx context.Context) error { // alive. f.reqsScratch = ba.Requests for i := range f.reqsScratch { - f.reqsScratch[i] = roachpb.RequestUnion{} + f.reqsScratch[i] = kvpb.RequestUnion{} } if monitoring { reqsScratchMemUsage := requestUnionOverhead * int64(cap(f.reqsScratch)) @@ -562,7 +563,7 @@ func (f *txnKVFetcher) nextBatch(ctx context.Context) (resp KVBatchFetcherRespon // and process it. for len(f.responses) > 0 { reply := f.responses[0].GetInner() - f.responses[0] = roachpb.ResponseUnion{} + f.responses[0] = kvpb.ResponseUnion{} f.responses = f.responses[1:] // Get the original span right away since we might overwrite it with the // resume span below. @@ -590,7 +591,7 @@ func (f *txnKVFetcher) nextBatch(ctx context.Context) (resp KVBatchFetcherRespon } switch t := reply.(type) { - case *roachpb.ScanResponse: + case *kvpb.ScanResponse: if len(t.BatchResponses) > 0 { ret.BatchResponse, f.remainingBatches = popBatch(t.BatchResponses) } @@ -607,7 +608,7 @@ func (f *txnKVFetcher) nextBatch(ctx context.Context) (resp KVBatchFetcherRespon // Note that ret.BatchResponse might be nil when the ScanResponse is // empty, and the caller (the KVFetcher) will skip over it. return ret, nil - case *roachpb.ReverseScanResponse: + case *kvpb.ReverseScanResponse: if len(t.BatchResponses) > 0 { ret.BatchResponse, f.remainingBatches = popBatch(t.BatchResponses) } @@ -624,7 +625,7 @@ func (f *txnKVFetcher) nextBatch(ctx context.Context) (resp KVBatchFetcherRespon // Note that ret.BatchResponse might be nil when the ScanResponse is // empty, and the caller (the KVFetcher) will skip over it. return ret, nil - case *roachpb.GetResponse: + case *kvpb.GetResponse: if t.IntentValue != nil { return KVBatchFetcherResponse{}, errors.AssertionFailedf("unexpectedly got an IntentValue back from a SQL GetRequest %v", *t.IntentValue) } @@ -682,7 +683,7 @@ func (f *txnKVFetcher) Close(ctx context.Context) { f.reset(ctx) } -const requestUnionOverhead = int64(unsafe.Sizeof(roachpb.RequestUnion{})) +const requestUnionOverhead = int64(unsafe.Sizeof(kvpb.RequestUnion{})) // spansToRequests converts the provided spans to the corresponding requests. If // a span doesn't have the EndKey set, then a Get request is used for it; @@ -693,16 +694,16 @@ const requestUnionOverhead = int64(unsafe.Sizeof(roachpb.RequestUnion{})) // if not, a new slice is allocated. func spansToRequests( spans roachpb.Spans, - scanFormat roachpb.ScanFormat, + scanFormat kvpb.ScanFormat, reverse bool, keyLocking lock.Strength, - reqsScratch []roachpb.RequestUnion, -) []roachpb.RequestUnion { - var reqs []roachpb.RequestUnion + reqsScratch []kvpb.RequestUnion, +) []kvpb.RequestUnion { + var reqs []kvpb.RequestUnion if cap(reqsScratch) >= len(spans) { reqs = reqsScratch[:len(spans)] } else { - reqs = make([]roachpb.RequestUnion, len(spans)) + reqs = make([]kvpb.RequestUnion, len(spans)) } // Detect the number of gets vs scans, so we can batch allocate all of the // requests precisely. @@ -713,16 +714,16 @@ func spansToRequests( } } gets := make([]struct { - req roachpb.GetRequest - union roachpb.RequestUnion_Get + req kvpb.GetRequest + union kvpb.RequestUnion_Get }, nGets) // curGet is incremented each time we fill in a GetRequest. curGet := 0 if reverse { scans := make([]struct { - req roachpb.ReverseScanRequest - union roachpb.RequestUnion_ReverseScan + req kvpb.ReverseScanRequest + union kvpb.RequestUnion_ReverseScan }, len(spans)-nGets) for i := range spans { if spans[i].EndKey == nil { @@ -744,8 +745,8 @@ func spansToRequests( } } else { scans := make([]struct { - req roachpb.ScanRequest - union roachpb.RequestUnion_Scan + req kvpb.ScanRequest + union kvpb.RequestUnion_Scan }, len(spans)-nGets) for i := range spans { if spans[i].EndKey == nil { diff --git a/pkg/sql/row/kv_batch_streamer.go b/pkg/sql/row/kv_batch_streamer.go index cfe0dd42b858..7a17c660a814 100644 --- a/pkg/sql/row/kv_batch_streamer.go +++ b/pkg/sql/row/kv_batch_streamer.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvstreamer" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -31,7 +32,7 @@ type txnKVStreamer struct { spans roachpb.Spans spanIDs []int - reqsScratch []roachpb.RequestUnion + reqsScratch []kvpb.RequestUnion acc *mon.BoundAccount @@ -93,10 +94,10 @@ func (f *txnKVStreamer) SetupNextFetch( // about the slice reuse. reqsScratch := f.reqsScratch[:cap(f.reqsScratch)] for i := len(spans); i < len(reqsScratch); i++ { - reqsScratch[i] = roachpb.RequestUnion{} + reqsScratch[i] = kvpb.RequestUnion{} } // TODO(yuzefovich): consider supporting COL_BATCH_RESPONSE scan format. - reqs := spansToRequests(spans, roachpb.BATCH_RESPONSE, false /* reverse */, f.keyLocking, reqsScratch) + reqs := spansToRequests(spans, kvpb.BATCH_RESPONSE, false /* reverse */, f.keyLocking, reqsScratch) if err := f.streamer.Enqueue(ctx, reqs); err != nil { return err } diff --git a/pkg/sql/row/kv_fetcher.go b/pkg/sql/row/kv_fetcher.go index 3db892530386..5dfe7a4e740f 100644 --- a/pkg/sql/row/kv_fetcher.go +++ b/pkg/sql/row/kv_fetcher.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvstreamer" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -51,7 +52,7 @@ var _ storage.NextKVer = &KVFetcher{} // as there is no concurrency). func newTxnKVFetcher( txn *kv.Txn, - bsHeader *roachpb.BoundedStalenessHeader, + bsHeader *kvpb.BoundedStalenessHeader, reverse bool, lockStrength descpb.ScanLockingStrength, lockWaitPolicy descpb.ScanLockingWaitPolicy, @@ -66,9 +67,9 @@ func newTxnKVFetcher( sendFn = makeTxnKVFetcherDefaultSendFunc(txn, &batchRequestsIssued) } else { negotiated := false - sendFn = func(ctx context.Context, ba *roachpb.BatchRequest) (br *roachpb.BatchResponse, _ error) { - ba.RoutingPolicy = roachpb.RoutingPolicy_NEAREST - var pErr *roachpb.Error + sendFn = func(ctx context.Context, ba *kvpb.BatchRequest) (br *kvpb.BatchResponse, _ error) { + ba.RoutingPolicy = kvpb.RoutingPolicy_NEAREST + var pErr *kvpb.Error // Only use NegotiateAndSend if we have not yet negotiated a timestamp. // If we have, fallback to Send which will already have the timestamp // fixed. @@ -117,7 +118,7 @@ func newTxnKVFetcher( // as there is no concurrency). func NewDirectKVBatchFetcher( txn *kv.Txn, - bsHeader *roachpb.BoundedStalenessHeader, + bsHeader *kvpb.BoundedStalenessHeader, spec *fetchpb.IndexFetchSpec, reverse bool, lockStrength descpb.ScanLockingStrength, @@ -130,7 +131,7 @@ func NewDirectKVBatchFetcher( txn, bsHeader, reverse, lockStrength, lockWaitPolicy, lockTimeout, acc, forceProductionKVBatchSize, ) - f.scanFormat = roachpb.COL_BATCH_RESPONSE + f.scanFormat = kvpb.COL_BATCH_RESPONSE f.indexFetchSpec = spec return f } @@ -143,7 +144,7 @@ func NewDirectKVBatchFetcher( // as there is no concurrency). func NewKVFetcher( txn *kv.Txn, - bsHeader *roachpb.BoundedStalenessHeader, + bsHeader *kvpb.BoundedStalenessHeader, reverse bool, lockStrength descpb.ScanLockingStrength, lockWaitPolicy descpb.ScanLockingWaitPolicy, diff --git a/pkg/sql/rowcontainer/BUILD.bazel b/pkg/sql/rowcontainer/BUILD.bazel index 02f5a9beff20..4ca3e0b8f09b 100644 --- a/pkg/sql/rowcontainer/BUILD.bazel +++ b/pkg/sql/rowcontainer/BUILD.bazel @@ -15,6 +15,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/kv/kvclient/kvstreamer", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/diskmap", "//pkg/roachpb", "//pkg/sql/catalog/catenumpb", @@ -56,6 +57,7 @@ go_test( deps = [ "//pkg/base", "//pkg/kv/kvclient/kvstreamer", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/diskmap", "//pkg/roachpb", "//pkg/settings/cluster", diff --git a/pkg/sql/rowcontainer/kvstreamer_result_disk_buffer.go b/pkg/sql/rowcontainer/kvstreamer_result_disk_buffer.go index 9c7ba12eeb94..0f98d5fdd2b7 100644 --- a/pkg/sql/rowcontainer/kvstreamer_result_disk_buffer.go +++ b/pkg/sql/rowcontainer/kvstreamer_result_disk_buffer.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvstreamer" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/diskmap" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" @@ -223,7 +224,7 @@ func deserialize(r *kvstreamer.Result, row rowenc.EncDatumRow, alloc *tree.Datum } } if isGet := tree.MustBeDBool(row[isGetIdx].Datum); isGet { - r.GetResp = &roachpb.GetResponse{} + r.GetResp = &kvpb.GetResponse{} if row[getRawBytesIdx].Datum != tree.DNull { r.GetResp.Value = &roachpb.Value{ RawBytes: []byte(tree.MustBeDBytes(row[getRawBytesIdx].Datum)), @@ -235,7 +236,7 @@ func deserialize(r *kvstreamer.Result, row rowenc.EncDatumRow, alloc *tree.Datum } } } else { - r.ScanResp = &roachpb.ScanResponse{} + r.ScanResp = &kvpb.ScanResponse{} batchResponses := tree.MustBeDArray(row[scanBatchResponsesIdx].Datum) r.ScanResp.BatchResponses = make([][]byte, batchResponses.Len()) for i := range batchResponses.Array { diff --git a/pkg/sql/rowcontainer/kvstreamer_result_disk_buffer_test.go b/pkg/sql/rowcontainer/kvstreamer_result_disk_buffer_test.go index c6a9d1d61ce2..7176e68f917a 100644 --- a/pkg/sql/rowcontainer/kvstreamer_result_disk_buffer_test.go +++ b/pkg/sql/rowcontainer/kvstreamer_result_disk_buffer_test.go @@ -15,6 +15,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvstreamer" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -62,7 +63,7 @@ func TestRoundTripResult(t *testing.T) { func makeResultWithGetResp(rng *rand.Rand, empty bool) kvstreamer.Result { var r kvstreamer.Result - r.GetResp = &roachpb.GetResponse{} + r.GetResp = &kvpb.GetResponse{} if !empty { rawBytes := make([]byte, rng.Intn(20)+1) rng.Read(rawBytes) @@ -87,7 +88,7 @@ func makeResultWithScanResp(rng *rand.Rand) kvstreamer.Result { rng.Read(batchResponse) batchResponses[i] = batchResponse } - r.ScanResp = &roachpb.ScanResponse{ + r.ScanResp = &kvpb.ScanResponse{ BatchResponses: batchResponses, } return r diff --git a/pkg/sql/rowexec/BUILD.bazel b/pkg/sql/rowexec/BUILD.bazel index 868b3ea3dd9f..aa8de28a9670 100644 --- a/pkg/sql/rowexec/BUILD.bazel +++ b/pkg/sql/rowexec/BUILD.bazel @@ -48,6 +48,7 @@ go_library( "//pkg/keys", "//pkg/kv", "//pkg/kv/kvclient/kvstreamer", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/kvserverbase", "//pkg/roachpb", "//pkg/server/telemetry", @@ -157,6 +158,7 @@ go_test( "//pkg/kv", "//pkg/kv/kvclient/kvcoord", "//pkg/kv/kvclient/rangecache", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/roachpb", "//pkg/security/securityassets", diff --git a/pkg/sql/rowexec/bulk_row_writer.go b/pkg/sql/rowexec/bulk_row_writer.go index eacd8194be20..76a0402f6525 100644 --- a/pkg/sql/rowexec/bulk_row_writer.go +++ b/pkg/sql/rowexec/bulk_row_writer.go @@ -14,6 +14,7 @@ import ( "context" "sync/atomic" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" @@ -42,7 +43,7 @@ type bulkRowWriter struct { spec execinfrapb.BulkRowWriterSpec input execinfra.RowSource output execinfra.RowReceiver - summary roachpb.BulkOpSummary + summary kvpb.BulkOpSummary } var _ execinfra.Processor = &bulkRowWriter{} diff --git a/pkg/sql/rowexec/indexbackfiller.go b/pkg/sql/rowexec/indexbackfiller.go index 5c72927c3e42..832f1dfc89d6 100644 --- a/pkg/sql/rowexec/indexbackfiller.go +++ b/pkg/sql/rowexec/indexbackfiller.go @@ -14,6 +14,7 @@ import ( "context" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" @@ -208,7 +209,7 @@ func (ib *indexBackfiller) ingestIndexEntries( // When the bulk adder flushes, the spans which were previously marked as // "added" can now be considered "completed", and be sent back to the // coordinator node as part of the next progress report. - adder.SetOnFlush(func(_ roachpb.BulkOpSummary) { + adder.SetOnFlush(func(_ kvpb.BulkOpSummary) { mu.Lock() defer mu.Unlock() mu.completedSpans = append(mu.completedSpans, mu.addedSpans...) diff --git a/pkg/sql/rowexec/processors_test.go b/pkg/sql/rowexec/processors_test.go index 6844e0b9f1e7..1a4f828f9150 100644 --- a/pkg/sql/rowexec/processors_test.go +++ b/pkg/sql/rowexec/processors_test.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" @@ -464,25 +465,25 @@ func TestDrainingProcessorSwallowsUncertaintyError(t *testing.T) { 0: { Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(_ context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + TestingRequestFilter: func(_ context.Context, ba *kvpb.BatchRequest) *kvpb.Error { if atomic.LoadInt64(&trapRead) == 0 { return nil } // We're going to trap a read for the rows [1,5]. - req, ok := ba.GetArg(roachpb.Scan) + req, ok := ba.GetArg(kvpb.Scan) if !ok { return nil } - key := req.(*roachpb.ScanRequest).Key.String() - endKey := req.(*roachpb.ScanRequest).EndKey.String() + key := req.(*kvpb.ScanRequest).Key.String() + endKey := req.(*kvpb.ScanRequest).EndKey.String() if strings.Contains(key, "/1") && strings.Contains(endKey, "/6") { blockedRead.Lock() for !blockedRead.shouldUnblock { blockedRead.unblockCond.Wait() } blockedRead.Unlock() - return roachpb.NewError( - roachpb.NewReadWithinUncertaintyIntervalError( + return kvpb.NewError( + kvpb.NewReadWithinUncertaintyIntervalError( ba.Timestamp, /* readTs */ hlc.ClockTimestamp{}, /* localUncertaintyLimit */ ba.Txn, /* txn */ @@ -640,7 +641,7 @@ func TestUncertaintyErrorIsReturned(t *testing.T) { testClusterArgs.ServerArgsPerNode[node] = base.TestServerArgs{ Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(_ context.Context, ba *roachpb.BatchRequest) *roachpb.Error { + TestingRequestFilter: func(_ context.Context, ba *kvpb.BatchRequest) *kvpb.Error { if atomic.LoadInt64(&trapRead) == 0 { return nil } @@ -652,15 +653,15 @@ func TestUncertaintyErrorIsReturned(t *testing.T) { return nil } - req, ok := ba.GetArg(roachpb.Scan) + req, ok := ba.GetArg(kvpb.Scan) if !ok { return nil } - if !bytes.HasPrefix(req.(*roachpb.ScanRequest).Key, keyPrefix) { + if !bytes.HasPrefix(req.(*kvpb.ScanRequest).Key, keyPrefix) { return nil } - return roachpb.NewError( - roachpb.NewReadWithinUncertaintyIntervalError( + return kvpb.NewError( + kvpb.NewReadWithinUncertaintyIntervalError( ba.Timestamp, hlc.ClockTimestamp{}, ba.Txn, diff --git a/pkg/sql/run_control_test.go b/pkg/sql/run_control_test.go index 319b8f18ac4c..34aaea12f5f2 100644 --- a/pkg/sql/run_control_test.go +++ b/pkg/sql/run_control_test.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql" @@ -861,10 +862,10 @@ func TestTenantStatementTimeoutAdmissionQueueCancelation(t *testing.T) { // client goroutine finish. wg.Add(numBlockers + 1) - matchBatch := func(ctx context.Context, req *roachpb.BatchRequest) bool { + matchBatch := func(ctx context.Context, req *kvpb.BatchRequest) bool { tid, ok := roachpb.ClientTenantFromContext(ctx) if ok && tid == tenantID && len(req.Requests) > 0 { - scan, ok := req.Requests[0].GetInner().(*roachpb.ScanRequest) + scan, ok := req.Requests[0].GetInner().(*kvpb.ScanRequest) if ok && tableSpan.ContainsKey(scan.Key) { return true } @@ -882,7 +883,7 @@ func TestTenantStatementTimeoutAdmissionQueueCancelation(t *testing.T) { TestingDisableSkipEnforcement: true, }, Store: &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(ctx context.Context, req *roachpb.BatchRequest) *roachpb.Error { + TestingRequestFilter: func(ctx context.Context, req *kvpb.BatchRequest) *kvpb.Error { if matchBatch(ctx, req) { // Notify we're blocking. unblockClientCh <- struct{}{} @@ -890,9 +891,9 @@ func TestTenantStatementTimeoutAdmissionQueueCancelation(t *testing.T) { } return nil }, - TestingResponseErrorEvent: func(ctx context.Context, req *roachpb.BatchRequest, err error) { + TestingResponseErrorEvent: func(ctx context.Context, req *kvpb.BatchRequest, err error) { if matchBatch(ctx, req) { - scan, ok := req.Requests[0].GetInner().(*roachpb.ScanRequest) + scan, ok := req.Requests[0].GetInner().(*kvpb.ScanRequest) if ok && tableSpan.ContainsKey(scan.Key) { cancel() wg.Done() diff --git a/pkg/sql/scatter.go b/pkg/sql/scatter.go index f5d51f9aef40..b32565232cf2 100644 --- a/pkg/sql/scatter.go +++ b/pkg/sql/scatter.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/privilege" @@ -134,15 +135,15 @@ type scatterRun struct { } func (n *scatterNode) startExec(params runParams) error { - req := &roachpb.AdminScatterRequest{ - RequestHeader: roachpb.RequestHeader{Key: n.run.span.Key, EndKey: n.run.span.EndKey}, + req := &kvpb.AdminScatterRequest{ + RequestHeader: kvpb.RequestHeader{Key: n.run.span.Key, EndKey: n.run.span.EndKey}, RandomizeLeases: true, } res, pErr := kv.SendWrapped(params.ctx, params.ExecCfg().DB.NonTransactionalSender(), req) if pErr != nil { return pErr.GoError() } - scatterRes := res.(*roachpb.AdminScatterResponse) + scatterRes := res.(*kvpb.AdminScatterResponse) n.run.rangeIdx = -1 n.run.ranges = make([]roachpb.Span, len(scatterRes.RangeInfos)) for i, rangeInfo := range scatterRes.RangeInfos { diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index 947a30d75b7f..49fafd022ff6 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangecache" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/server/telemetry" @@ -185,7 +186,7 @@ func IsPermanentSchemaChangeError(err error) bool { // Ignore error thrown because of a read at a very old timestamp. // The Backfill will grab a new timestamp to read at for the rest // of the backfill. - if errors.HasType(err, (*roachpb.BatchTimestampBeforeGCError)(nil)) { + if errors.HasType(err, (*kvpb.BatchTimestampBeforeGCError)(nil)) { return false } @@ -308,11 +309,11 @@ func (sc *SchemaChanger) backfillQueryIntoTable( } defer localPlanner.curPlan.close(ctx) - res := roachpb.BulkOpSummary{} + res := kvpb.BulkOpSummary{} rw := NewCallbackResultWriter(func(ctx context.Context, row tree.Datums) error { // TODO(adityamaru): Use the BulkOpSummary for either telemetry or to // return to user. - var counts roachpb.BulkOpSummary + var counts kvpb.BulkOpSummary if err := protoutil.Unmarshal([]byte(*row[0].(*tree.DBytes)), &counts); err != nil { return err } diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go index 60049832e9a9..19ea0e5a0afb 100644 --- a/pkg/sql/schema_changer_test.go +++ b/pkg/sql/schema_changer_test.go @@ -31,6 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -1335,8 +1336,8 @@ func TestSchemaChangeRetry(t *testing.T) { if rand.Intn(2) == 0 { return context.DeadlineExceeded } else { - errAmbiguous := &roachpb.AmbiguousResultError{} - return roachpb.NewError(errAmbiguous).GoError() + errAmbiguous := &kvpb.AmbiguousResultError{} + return kvpb.NewError(errAmbiguous).GoError() } } if sp.Key != nil && seenSpan.Key != nil { @@ -1457,8 +1458,8 @@ func TestSchemaChangeRetryOnVersionChange(t *testing.T) { // version and retry the backfill. Since, the new index backfiller // does not repeat this DistSQL setup step unless retried, we must // force a retry. - errAmbiguous := &roachpb.AmbiguousResultError{} - return roachpb.NewError(errAmbiguous).GoError() + errAmbiguous := &kvpb.AmbiguousResultError{} + return kvpb.NewError(errAmbiguous).GoError() } if seenSpan.Key != nil { if !seenSpan.EndKey.Equal(sp.EndKey) { @@ -4534,8 +4535,8 @@ func TestIndexBackfillAfterGC(t *testing.T) { return nil } gcAt = tc.Server(0).Clock().Now() - gcr := roachpb.GCRequest{ - RequestHeader: roachpb.RequestHeaderFromSpan(sp), + gcr := kvpb.GCRequest{ + RequestHeader: kvpb.RequestHeaderFromSpan(sp), Threshold: gcAt, } _, err := kv.SendWrapped(ctx, tc.Server(0).DistSenderI().(*kvcoord.DistSender), &gcr) diff --git a/pkg/sql/schemachanger/scrun/BUILD.bazel b/pkg/sql/schemachanger/scrun/BUILD.bazel index 7eaa43929b0b..2c3c86153ecc 100644 --- a/pkg/sql/schemachanger/scrun/BUILD.bazel +++ b/pkg/sql/schemachanger/scrun/BUILD.bazel @@ -13,7 +13,7 @@ go_library( "//pkg/clusterversion", "//pkg/jobs", "//pkg/jobs/jobspb", - "//pkg/roachpb", + "//pkg/kv/kvpb", "//pkg/settings", "//pkg/settings/cluster", "//pkg/sql/catalog", diff --git a/pkg/sql/schemachanger/scrun/scrun.go b/pkg/sql/schemachanger/scrun/scrun.go index 88cfbdd38203..62e19688567f 100644 --- a/pkg/sql/schemachanger/scrun/scrun.go +++ b/pkg/sql/schemachanger/scrun/scrun.go @@ -17,7 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -188,7 +188,7 @@ func executeStage( if err := scexec.ExecuteStage(ctx, deps, stage.Phase, stage.Ops()); err != nil { // Don't go through the effort to wrap the error if it's a retry or it's a // cancelation. - if !errors.HasType(err, (*roachpb.TransactionRetryWithProtoRefreshError)(nil)) && + if !errors.HasType(err, (*kvpb.TransactionRetryWithProtoRefreshError)(nil)) && !errors.Is(err, context.Canceled) && !scerrors.HasSchemaChangerUserError(err) { err = p.DecorateErrorWithPlanDetails(err) diff --git a/pkg/sql/schemachanger/sctest/BUILD.bazel b/pkg/sql/schemachanger/sctest/BUILD.bazel index 7df4b3f08a5c..f61688f065a2 100644 --- a/pkg/sql/schemachanger/sctest/BUILD.bazel +++ b/pkg/sql/schemachanger/sctest/BUILD.bazel @@ -17,7 +17,7 @@ go_library( "//pkg/jobs/jobspb", "//pkg/keys", "//pkg/kv", - "//pkg/roachpb", + "//pkg/kv/kvpb", "//pkg/security/username", "//pkg/server", "//pkg/sql", diff --git a/pkg/sql/schemachanger/sctest/cumulative.go b/pkg/sql/schemachanger/sctest/cumulative.go index b73d4a0463f6..b1edb7f099f0 100644 --- a/pkg/sql/schemachanger/sctest/cumulative.go +++ b/pkg/sql/schemachanger/sctest/cumulative.go @@ -29,7 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/corpus" @@ -1482,7 +1482,7 @@ func ValidateMixedVersionElements(t *testing.T, path string, newCluster NewMixed jobPauseResumeChannel <- p.JobID <-waitForPause pauseComplete = true - return roachpb.NewTransactionRetryError(roachpb.RETRY_REASON_UNKNOWN, "test") + return kvpb.NewTransactionRetryError(kvpb.RETRY_REASON_UNKNOWN, "test") } return nil }, diff --git a/pkg/sql/sem/builtins/BUILD.bazel b/pkg/sql/sem/builtins/BUILD.bazel index ef4f0c161a07..07a9f086a962 100644 --- a/pkg/sql/sem/builtins/BUILD.bazel +++ b/pkg/sql/sem/builtins/BUILD.bazel @@ -47,6 +47,7 @@ go_library( "//pkg/keys", "//pkg/kv", "//pkg/kv/kvclient", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/concurrency/lock", "//pkg/kv/kvserver/kvserverbase", "//pkg/repstream/streampb", @@ -171,6 +172,7 @@ go_test( "//pkg/base", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/roachpb", "//pkg/security/securityassets", diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index cf223461b476..08f89d47ca95 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -41,6 +41,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -5360,8 +5361,8 @@ DO NOT USE -- USE 'CREATE TENANT' INSTEAD`, Fn: func(ctx context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) { key := []byte(tree.MustBeDBytes(args[0])) b := &kv.Batch{} - b.AddRawRequest(&roachpb.LeaseInfoRequest{ - RequestHeader: roachpb.RequestHeader{ + b.AddRawRequest(&kvpb.LeaseInfoRequest{ + RequestHeader: kvpb.RequestHeader{ Key: key, }, }) @@ -5372,7 +5373,7 @@ DO NOT USE -- USE 'CREATE TENANT' INSTEAD`, if err := evalCtx.Txn.Run(ctx, b); err != nil { return nil, pgerror.Wrap(err, pgcode.InvalidParameterValue, "error fetching leaseholder") } - resp := b.RawResponse().Responses[0].GetInner().(*roachpb.LeaseInfoResponse) + resp := b.RawResponse().Responses[0].GetInner().(*kvpb.LeaseInfoResponse) return tree.NewDInt(tree.DInt(resp.Lease.Replica.StoreID)), nil }, @@ -7307,7 +7308,7 @@ run from. One of 'mvccGC', 'merge', 'split', 'replicate', 'replicaGC', return nil } - if errors.HasType(err, (*roachpb.RangeNotFoundError)(nil)) { + if errors.HasType(err, (*kvpb.RangeNotFoundError)(nil)) { return nil } return err @@ -7359,7 +7360,7 @@ store housing the range on the node it's run from. One of 'mvccGC', 'merge', 'sp return nil } - if errors.HasType(err, (*roachpb.RangeNotFoundError)(nil)) { + if errors.HasType(err, (*kvpb.RangeNotFoundError)(nil)) { return nil } return err @@ -7579,7 +7580,7 @@ expires until the statement bundle is collected`, if evalCtx.AsOfSystemTime != nil { endTime = evalCtx.AsOfSystemTime.Timestamp } - header := roachpb.Header{ + header := kvpb.Header{ Timestamp: endTime, // We set WaitPolicy to Error, so that the export will return an error // to us instead of a blocking wait if it hits any other txns. @@ -7592,23 +7593,23 @@ expires until the statement bundle is collected`, startTime := args[1].(*tree.DTimestampTZ).Time startTimestamp := hlc.Timestamp{WallTime: startTime.UnixNano()} allRevisions := *args[2].(*tree.DBool) - filter := roachpb.MVCCFilter_Latest + filter := kvpb.MVCCFilter_Latest if allRevisions { - filter = roachpb.MVCCFilter_All + filter = kvpb.MVCCFilter_All } - req := &roachpb.ExportRequest{ - RequestHeader: roachpb.RequestHeader{Key: startKey, EndKey: endKey}, + req := &kvpb.ExportRequest{ + RequestHeader: kvpb.RequestHeader{Key: startKey, EndKey: endKey}, StartTime: startTimestamp, MVCCFilter: filter, ExportFingerprint: true, } - admissionHeader := roachpb.AdmissionHeader{ + admissionHeader := kvpb.AdmissionHeader{ Priority: int32(admissionpb.BulkNormalPri), CreateTime: timeutil.Now().UnixNano(), - Source: roachpb.AdmissionHeader_FROM_SQL, + Source: kvpb.AdmissionHeader_FROM_SQL, NoMemoryReservedAtSource: true, } - todo := make(chan *roachpb.ExportRequest, 1) + todo := make(chan *kvpb.ExportRequest, 1) todo <- req ctxDone := ctx.Done() var fingerprint uint64 @@ -7620,8 +7621,8 @@ expires until the statement bundle is collected`, case <-ctxDone: return nil, ctx.Err() case req := <-todo: - var rawResp roachpb.Response - var pErr *roachpb.Error + var rawResp kvpb.Response + var pErr *kvpb.Error exportRequestErr := contextutil.RunWithTimeout(ctx, fmt.Sprintf("ExportRequest fingerprint for span %s", roachpb.Span{Key: startKey, EndKey: endKey}), 5*time.Minute, func(ctx context.Context) error { @@ -7636,7 +7637,7 @@ expires until the statement bundle is collected`, return nil, exportRequestErr } - resp := rawResp.(*roachpb.ExportResponse) + resp := rawResp.(*kvpb.ExportResponse) for _, file := range resp.Files { fingerprint = fingerprint ^ file.Fingerprint @@ -7652,7 +7653,7 @@ expires until the statement bundle is collected`, } resumeReq := req - resumeReq.RequestHeader = roachpb.RequestHeaderFromSpan(*resp.ResumeSpan) + resumeReq.RequestHeader = kvpb.RequestHeaderFromSpan(*resp.ResumeSpan) todo <- resumeReq } default: diff --git a/pkg/sql/sem/builtins/fingerprint_builtin_test.go b/pkg/sql/sem/builtins/fingerprint_builtin_test.go index 07ea1cdbd76e..d30d0179a61b 100644 --- a/pkg/sql/sem/builtins/fingerprint_builtin_test.go +++ b/pkg/sql/sem/builtins/fingerprint_builtin_test.go @@ -17,6 +17,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server" @@ -45,12 +46,12 @@ func TestFingerprint(t *testing.T) { serv, sqlDB, db := serverutils.StartServer(t, base.TestServerArgs{ Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ - TestingResponseFilter: func(ctx context.Context, ba *roachpb.BatchRequest, br *roachpb.BatchResponse) *roachpb.Error { + TestingResponseFilter: func(ctx context.Context, ba *kvpb.BatchRequest, br *kvpb.BatchResponse) *kvpb.Error { mu.Lock() defer mu.Unlock() for i, ru := range br.Responses { - if _, ok := ba.Requests[i].GetInner().(*roachpb.ExportRequest); ok { - exportResponse := ru.GetInner().(*roachpb.ExportResponse) + if _, ok := ba.Requests[i].GetInner().(*kvpb.ExportRequest); ok { + exportResponse := ru.GetInner().(*kvpb.ExportResponse) numExportResponses++ numSSTsInExportResponses += len(exportResponse.Files) } diff --git a/pkg/sql/sem/builtins/generator_builtins.go b/pkg/sql/sem/builtins/generator_builtins.go index 6aaaaa2f940b..961a25224bf0 100644 --- a/pkg/sql/sem/builtins/generator_builtins.go +++ b/pkg/sql/sem/builtins/generator_builtins.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/lexbase" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" @@ -1900,14 +1901,14 @@ type checkConsistencyGenerator struct { txn *kv.Txn // to load range descriptors consistencyChecker eval.ConsistencyCheckRunner from, to roachpb.Key - mode roachpb.ChecksumMode + mode kvpb.ChecksumMode // The descriptors for which we haven't yet emitted rows. Rows are consumed // from this field and produce one (or more, in the case of splits not reflected // in the descriptor) rows in `next`. descs []roachpb.RangeDescriptor // The current row, emitted by Values(). - cur roachpb.CheckConsistencyResponse_Result + cur kvpb.CheckConsistencyResponse_Result // The time it took to produce the current row, i.e. how long it took to run // the consistency check that produced the row. When a consistency check // produces more than one row (i.e. after a split), all of the duration will @@ -1916,7 +1917,7 @@ type checkConsistencyGenerator struct { // next are the potentially prefetched subsequent rows. This is usually empty // (as one consistency check produces one result which immediately moves to // `cur`) except when a descriptor we use doesn't reflect subsequent splits. - next []roachpb.CheckConsistencyResponse_Result + next []kvpb.CheckConsistencyResponse_Result } var _ eval.ValueGenerator = &checkConsistencyGenerator{} @@ -1961,9 +1962,9 @@ func makeCheckConsistencyGenerator( return nil, errors.New("start key must be less than end key") } - mode := roachpb.ChecksumMode_CHECK_FULL + mode := kvpb.ChecksumMode_CHECK_FULL if statsOnly := bool(*args[0].(*tree.DBool)); statsOnly { - mode = roachpb.ChecksumMode_CHECK_STATS + mode = kvpb.ChecksumMode_CHECK_STATS } if evalCtx.ConsistencyChecker == nil { @@ -2043,10 +2044,10 @@ func (c *checkConsistencyGenerator) maybeRefillRows(ctx context.Context) time.Du ctx, desc.StartKey.AsRawKey(), desc.EndKey.AsRawKey(), c.mode, ) if err != nil { - resp = &roachpb.CheckConsistencyResponse{Result: []roachpb.CheckConsistencyResponse_Result{{ + resp = &kvpb.CheckConsistencyResponse{Result: []kvpb.CheckConsistencyResponse_Result{{ RangeID: desc.RangeID, StartKey: desc.StartKey, - Status: roachpb.CheckConsistencyResponse_RANGE_INDETERMINATE, + Status: kvpb.CheckConsistencyResponse_RANGE_INDETERMINATE, Detail: err.Error(), }}} } @@ -2176,15 +2177,15 @@ func (sp *spanKeyIterator) Next(ctx context.Context) (bool, error) { func (sp *spanKeyIterator) scan( ctx context.Context, startKey roachpb.Key, endKey roachpb.Key, ) error { - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.TargetBytes = spanKeyIteratorChunkBytes ba.MaxSpanRequestKeys = spanKeyIteratorChunkKeys - ba.Add(&roachpb.ScanRequest{ - RequestHeader: roachpb.RequestHeader{ + ba.Add(&kvpb.ScanRequest{ + RequestHeader: kvpb.RequestHeader{ Key: startKey, EndKey: endKey, }, - ScanFormat: roachpb.KEY_VALUES, + ScanFormat: kvpb.KEY_VALUES, }) br, pErr := sp.txn.Send(ctx, ba) if pErr != nil { diff --git a/pkg/sql/sem/eval/BUILD.bazel b/pkg/sql/sem/eval/BUILD.bazel index 8ead406fbdab..4545552c9310 100644 --- a/pkg/sql/sem/eval/BUILD.bazel +++ b/pkg/sql/sem/eval/BUILD.bazel @@ -43,6 +43,7 @@ go_library( "//pkg/jobs/jobspb", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/kvserverbase", "//pkg/repstream/streampb", "//pkg/roachpb", @@ -113,6 +114,7 @@ go_test( embed = [":eval"], deps = [ "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/settings/cluster", "//pkg/sql/faketreeeval", diff --git a/pkg/sql/sem/eval/context.go b/pkg/sql/sem/eval/context.go index 0e82194d635a..5b6f8e69d3f9 100644 --- a/pkg/sql/sem/eval/context.go +++ b/pkg/sql/sem/eval/context.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/repstream/streampb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -272,7 +273,7 @@ type DescIDGenerator interface { type RangeStatsFetcher interface { // RangeStats fetches the stats for the ranges which contain the passed keys. - RangeStats(ctx context.Context, keys ...roachpb.Key) ([]*roachpb.RangeStatsResponse, error) + RangeStats(ctx context.Context, keys ...roachpb.Key) ([]*kvpb.RangeStatsResponse, error) } var _ tree.ParseContext = &Context{} @@ -281,8 +282,8 @@ var _ tree.ParseContext = &Context{} // crdb_internal.check_consistency. type ConsistencyCheckRunner interface { CheckConsistency( - ctx context.Context, from, to roachpb.Key, mode roachpb.ChecksumMode, - ) (*roachpb.CheckConsistencyResponse, error) + ctx context.Context, from, to roachpb.Key, mode kvpb.ChecksumMode, + ) (*kvpb.CheckConsistencyResponse, error) } // RangeProber is an interface embedded in eval.Context used by diff --git a/pkg/sql/sem/eval/timeconv_test.go b/pkg/sql/sem/eval/timeconv_test.go index 048b41df1324..8ab1d9e434c9 100644 --- a/pkg/sql/sem/eval/timeconv_test.go +++ b/pkg/sql/sem/eval/timeconv_test.go @@ -15,6 +15,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" _ "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" @@ -47,8 +48,8 @@ func TestClusterTimestampConversion(t *testing.T) { clock := hlc.NewClockForTesting(nil) senderFactory := kv.MakeMockTxnSenderFactory( - func(context.Context, *roachpb.Transaction, *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, *roachpb.Error) { + func(context.Context, *roachpb.Transaction, *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, *kvpb.Error) { panic("unused") }) db := kv.NewDB(log.MakeTestingAmbientCtxWithNewTracer(), senderFactory, clock, stopper) diff --git a/pkg/sql/sequence.go b/pkg/sql/sequence.go index a261c41742bb..5a94eb87ac2b 100644 --- a/pkg/sql/sequence.go +++ b/pkg/sql/sequence.go @@ -16,7 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" @@ -176,7 +176,7 @@ func (p *planner) incrementSequenceUsingCache( } if err != nil { - if errors.HasType(err, (*roachpb.IntegerOverflowError)(nil)) { + if errors.HasType(err, (*kvpb.IntegerOverflowError)(nil)) { return 0, 0, 0, boundsExceededError(descriptor) } return 0, 0, 0, err diff --git a/pkg/sql/sqlinstance/instancestorage/BUILD.bazel b/pkg/sql/sqlinstance/instancestorage/BUILD.bazel index e2a2d036d767..3ba2536d3c67 100644 --- a/pkg/sql/sqlinstance/instancestorage/BUILD.bazel +++ b/pkg/sql/sqlinstance/instancestorage/BUILD.bazel @@ -16,6 +16,7 @@ go_library( "//pkg/keys", "//pkg/kv", "//pkg/kv/kvclient/rangefeed", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/concurrency/lock", "//pkg/multitenant", "//pkg/roachpb", diff --git a/pkg/sql/sqlinstance/instancestorage/instancereader.go b/pkg/sql/sqlinstance/instancestorage/instancereader.go index abe0f6ffeb14..4b377d47cdfb 100644 --- a/pkg/sql/sqlinstance/instancestorage/instancereader.go +++ b/pkg/sql/sqlinstance/instancestorage/instancereader.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" @@ -168,7 +169,7 @@ func (r *Reader) GetAllInstancesUsingTxn( func (r *Reader) startRangeFeed(ctx context.Context) { updateCacheFn := func( - ctx context.Context, keyVal *roachpb.RangeFeedValue, + ctx context.Context, keyVal *kvpb.RangeFeedValue, ) { instance, err := r.rowcodec.decodeRow(keyVal.Key, &keyVal.Value) if err != nil { diff --git a/pkg/sql/sqlliveness/slstorage/BUILD.bazel b/pkg/sql/sqlliveness/slstorage/BUILD.bazel index 1c3921e11230..6307f99bd18d 100644 --- a/pkg/sql/sqlliveness/slstorage/BUILD.bazel +++ b/pkg/sql/sqlliveness/slstorage/BUILD.bazel @@ -53,6 +53,7 @@ go_test( deps = [ "//pkg/base", "//pkg/keys", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/roachpb", "//pkg/security/securityassets", diff --git a/pkg/sql/sqlliveness/slstorage/slstorage_test.go b/pkg/sql/sqlliveness/slstorage/slstorage_test.go index 90f40ac4e88b..c8ffdddeb5fe 100644 --- a/pkg/sql/sqlliveness/slstorage/slstorage_test.go +++ b/pkg/sql/sqlliveness/slstorage/slstorage_test.go @@ -23,8 +23,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" @@ -485,13 +485,13 @@ func TestConcurrentAccessSynchronization(t *testing.T) { } func testConcurrentAccessSynchronization(t *testing.T) { ctx := context.Background() - type filterFunc = func(ctx context.Context, request *roachpb.BatchRequest) *roachpb.Error + type filterFunc = func(ctx context.Context, request *kvpb.BatchRequest) *kvpb.Error var requestFilter atomic.Value requestFilter.Store(filterFunc(nil)) s, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{ Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(ctx context.Context, request *roachpb.BatchRequest) *roachpb.Error { + TestingRequestFilter: func(ctx context.Context, request *kvpb.BatchRequest) *kvpb.Error { if f := requestFilter.Load().(filterFunc); f != nil { return f(ctx, request) } @@ -533,19 +533,19 @@ func testConcurrentAccessSynchronization(t *testing.T) { }) } unblock := func() { close(blockChannel.Load().(chan struct{})) } - requestFilter.Store(func(ctx context.Context, request *roachpb.BatchRequest) *roachpb.Error { - getRequest, ok := request.GetArg(roachpb.Get) + requestFilter.Store(func(ctx context.Context, request *kvpb.BatchRequest) *kvpb.Error { + getRequest, ok := request.GetArg(kvpb.Get) if !ok { return nil } - get := getRequest.(*roachpb.GetRequest) + get := getRequest.(*kvpb.GetRequest) if !bytes.HasPrefix(get.Key, prefix) { return nil } atomic.AddInt64(&blocked, 1) defer atomic.AddInt64(&blocked, -1) <-blockChannel.Load().(chan struct{}) - return roachpb.NewError(ctx.Err()) + return kvpb.NewError(ctx.Err()) }) t.Run("CachedReader does not block", func(t *testing.T) { @@ -688,15 +688,15 @@ func TestDeleteMidUpdateFails(t *testing.T) { } func testDeleteMidUpdateFails(t *testing.T) { ctx := context.Background() - type filterFunc = func(context.Context, *roachpb.BatchRequest, *roachpb.BatchResponse) *roachpb.Error + type filterFunc = func(context.Context, *kvpb.BatchRequest, *kvpb.BatchResponse) *kvpb.Error var respFilter atomic.Value respFilter.Store(filterFunc(nil)) s, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{ Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ TestingResponseFilter: func( - ctx context.Context, request *roachpb.BatchRequest, resp *roachpb.BatchResponse, - ) *roachpb.Error { + ctx context.Context, request *kvpb.BatchRequest, resp *kvpb.BatchResponse, + ) *kvpb.Error { if f := respFilter.Load().(filterFunc); f != nil { return f(ctx, request, resp) } @@ -726,10 +726,10 @@ func testDeleteMidUpdateFails(t *testing.T) { // to perform an update after the get has evaluated. getChan := make(chan chan struct{}) respFilter.Store(func( - ctx context.Context, request *roachpb.BatchRequest, _ *roachpb.BatchResponse, - ) *roachpb.Error { - if get, ok := request.GetArg(roachpb.Get); !ok || !bytes.HasPrefix( - get.(*roachpb.GetRequest).Key, + ctx context.Context, request *kvpb.BatchRequest, _ *kvpb.BatchResponse, + ) *kvpb.Error { + if get, ok := request.GetArg(kvpb.Get); !ok || !bytes.HasPrefix( + get.(*kvpb.GetRequest).Key, keys.SystemSQLCodec.TablePrefix(uint32(tableID)), ) { return nil diff --git a/pkg/sql/sqlstats/persistedsqlstats/BUILD.bazel b/pkg/sql/sqlstats/persistedsqlstats/BUILD.bazel index e2f6742e2f05..df13b9e5e71a 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/BUILD.bazel +++ b/pkg/sql/sqlstats/persistedsqlstats/BUILD.bazel @@ -75,6 +75,7 @@ go_test( "//pkg/jobs", "//pkg/jobs/jobspb", "//pkg/jobs/jobstest", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/roachpb", "//pkg/scheduledjobs", diff --git a/pkg/sql/sqlstats/persistedsqlstats/compaction_test.go b/pkg/sql/sqlstats/persistedsqlstats/compaction_test.go index 0705f7ea3565..713c47028461 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/compaction_test.go +++ b/pkg/sql/sqlstats/persistedsqlstats/compaction_test.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql" @@ -467,14 +468,14 @@ func (k *kvScanInterceptor) disable() { atomic.StoreInt32(&k.enabled, 0) } -func (k *kvScanInterceptor) intercept(_ context.Context, ba *roachpb.BatchRequest) *roachpb.Error { +func (k *kvScanInterceptor) intercept(_ context.Context, ba *kvpb.BatchRequest) *kvpb.Error { if atomic.LoadInt32(&k.enabled) == 0 { return nil } - if req, ok := ba.GetArg(roachpb.Scan); ok { - _, tableID, _ := encoding.DecodeUvarintAscending(req.(*roachpb.ScanRequest).Key) + if req, ok := ba.GetArg(kvpb.Scan); ok { + _, tableID, _ := encoding.DecodeUvarintAscending(req.(*kvpb.ScanRequest).Key) if tableID == stmtStatsTableID || tableID == txnStatsTableID { - prettyKey := roachpb.PrettyPrintKey([]encoding.Direction{}, req.(*roachpb.ScanRequest).Key) + prettyKey := roachpb.PrettyPrintKey([]encoding.Direction{}, req.(*kvpb.ScanRequest).Key) keyMatchedWideScan := kvReqWideScanStartKeyPattern.MatchString(prettyKey) diff --git a/pkg/sql/stats/BUILD.bazel b/pkg/sql/stats/BUILD.bazel index a2eb553e2162..0eea98df5e84 100644 --- a/pkg/sql/stats/BUILD.bazel +++ b/pkg/sql/stats/BUILD.bazel @@ -27,6 +27,7 @@ go_library( "//pkg/jobs/jobspb", "//pkg/keys", "//pkg/kv/kvclient/rangefeed", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/settings", "//pkg/settings/cluster", @@ -90,9 +91,9 @@ go_test( "//pkg/keys", "//pkg/kv", "//pkg/kv/kvclient/rangefeed", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/kv/kvserver/kvserverbase", - "//pkg/roachpb", "//pkg/security/securityassets", "//pkg/security/securitytest", "//pkg/server", diff --git a/pkg/sql/stats/create_stats_job_test.go b/pkg/sql/stats/create_stats_job_test.go index e64bfe88c57c..50637512904c 100644 --- a/pkg/sql/stats/create_stats_job_test.go +++ b/pkg/sql/stats/create_stats_job_test.go @@ -20,9 +20,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" @@ -487,9 +487,9 @@ func createStatsRequestFilter( ) (kvserverbase.ReplicaRequestFilter, func(descpb.ID)) { var tableToBlock atomic.Value tableToBlock.Store(descpb.InvalidID) - return func(ctx context.Context, ba *roachpb.BatchRequest) *roachpb.Error { - if req, ok := ba.GetArg(roachpb.Scan); ok { - _, tableID, _ := encoding.DecodeUvarintAscending(req.(*roachpb.ScanRequest).Key) + return func(ctx context.Context, ba *kvpb.BatchRequest) *kvpb.Error { + if req, ok := ba.GetArg(kvpb.Scan); ok { + _, tableID, _ := encoding.DecodeUvarintAscending(req.(*kvpb.ScanRequest).Key) // Ensure that the tableID is what we expect it to be. if tableID > 0 && descpb.ID(tableID) == tableToBlock.Load().(descpb.ID) { // Read from the channel twice to allow jobutils.RunJob to complete diff --git a/pkg/sql/stats/stats_cache.go b/pkg/sql/stats/stats_cache.go index 2d6f73d4d681..2e3bdacbdfb3 100644 --- a/pkg/sql/stats/stats_cache.go +++ b/pkg/sql/stats/stats_cache.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog" @@ -142,7 +143,7 @@ func (sc *TableStatisticsCache) Start( var lastTableID descpb.ID var lastTS hlc.Timestamp - handleEvent := func(ctx context.Context, kv *roachpb.RangeFeedValue) { + handleEvent := func(ctx context.Context, kv *kvpb.RangeFeedValue) { tableID, err := decodeTableStatisticsKV(codec, kv, &sc.datumAlloc) if err != nil { log.Warningf(ctx, "failed to decode table statistics row %v: %v", kv.Key, err) @@ -179,7 +180,7 @@ func (sc *TableStatisticsCache) Start( // decodeTableStatisticsKV decodes the table ID from a range feed event on // system.table_statistics. func decodeTableStatisticsKV( - codec keys.SQLCodec, kv *roachpb.RangeFeedValue, da *tree.DatumAlloc, + codec keys.SQLCodec, kv *kvpb.RangeFeedValue, da *tree.DatumAlloc, ) (tableDesc descpb.ID, err error) { // The primary key of table_statistics is (tableID INT, statisticID INT). types := []*types.T{types.Int, types.Int} diff --git a/pkg/sql/stmtdiagnostics/BUILD.bazel b/pkg/sql/stmtdiagnostics/BUILD.bazel index c02afda348ea..ecbcf0ee8a00 100644 --- a/pkg/sql/stmtdiagnostics/BUILD.bazel +++ b/pkg/sql/stmtdiagnostics/BUILD.bazel @@ -38,6 +38,7 @@ go_test( deps = [ "//pkg/base", "//pkg/keys", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/roachpb", "//pkg/security/securityassets", diff --git a/pkg/sql/stmtdiagnostics/statement_diagnostics_test.go b/pkg/sql/stmtdiagnostics/statement_diagnostics_test.go index 6169ebc330d9..c345bd6f2c08 100644 --- a/pkg/sql/stmtdiagnostics/statement_diagnostics_test.go +++ b/pkg/sql/stmtdiagnostics/statement_diagnostics_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -583,7 +584,7 @@ func TestChangePollInterval(t *testing.T) { Settings: settings, Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(ctx context.Context, request *roachpb.BatchRequest) *roachpb.Error { + TestingRequestFilter: func(ctx context.Context, request *kvpb.BatchRequest) *kvpb.Error { if request.Txn == nil { return nil } diff --git a/pkg/sql/tenant_creation.go b/pkg/sql/tenant_creation.go index 442b08f90197..c2ddb4c46252 100644 --- a/pkg/sql/tenant_creation.go +++ b/pkg/sql/tenant_creation.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" @@ -196,7 +197,7 @@ func (p *planner) createTenantInternal( b.CPut(kv.Key, &kv.Value, nil) } if err := p.Txn().Run(ctx, b); err != nil { - if errors.HasType(err, (*roachpb.ConditionFailedError)(nil)) { + if errors.HasType(err, (*kvpb.ConditionFailedError)(nil)) { return tid, errors.Wrap(err, "programming error: "+ "tenant already exists but was not in system.tenants table") } diff --git a/pkg/sql/tenant_gc.go b/pkg/sql/tenant_gc.go index 8a423ede9c14..ffc61ef137ac 100644 --- a/pkg/sql/tenant_gc.go +++ b/pkg/sql/tenant_gc.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/spanconfig" @@ -119,8 +120,8 @@ func clearTenant(ctx context.Context, execCfg *ExecutorConfig, info *mtinfopb.Te // ClearRange cannot be run in a transaction, so create a non-transactional // batch to send the request. b := &kv.Batch{} - b.AddRawRequest(&roachpb.ClearRangeRequest{ - RequestHeader: roachpb.RequestHeader{Key: prefix, EndKey: prefixEnd}, + b.AddRawRequest(&kvpb.ClearRangeRequest{ + RequestHeader: kvpb.RequestHeader{Key: prefix, EndKey: prefixEnd}, }) return errors.Wrapf(execCfg.DB.Run(ctx, b), "clearing tenant %d data", info.ID) diff --git a/pkg/sql/tests/BUILD.bazel b/pkg/sql/tests/BUILD.bazel index 0251e135258d..a634cd8c7a8f 100644 --- a/pkg/sql/tests/BUILD.bazel +++ b/pkg/sql/tests/BUILD.bazel @@ -13,6 +13,7 @@ go_library( deps = [ "//pkg/base", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/kv/kvserver/kvserverbase", "//pkg/roachpb", diff --git a/pkg/sql/tests/command_filters.go b/pkg/sql/tests/command_filters.go index fe1e17d7ad3f..ab035671df59 100644 --- a/pkg/sql/tests/command_filters.go +++ b/pkg/sql/tests/command_filters.go @@ -11,8 +11,8 @@ package tests import ( + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils/storageutils" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/errors" @@ -37,7 +37,7 @@ type CommandFilters struct { // RunFilters executes the registered filters, stopping at the first one // that returns an error. -func (c *CommandFilters) RunFilters(args kvserverbase.FilterArgs) *roachpb.Error { +func (c *CommandFilters) RunFilters(args kvserverbase.FilterArgs) *kvpb.Error { c.RLock() defer c.RUnlock() @@ -47,7 +47,7 @@ func (c *CommandFilters) RunFilters(args kvserverbase.FilterArgs) *roachpb.Error return c.runFiltersInternal(args) } -func (c *CommandFilters) runFiltersInternal(args kvserverbase.FilterArgs) *roachpb.Error { +func (c *CommandFilters) runFiltersInternal(args kvserverbase.FilterArgs) *kvpb.Error { for _, f := range c.filters { if pErr := f.filter(args); pErr != nil { return pErr diff --git a/pkg/sql/truncate.go b/pkg/sql/truncate.go index 1b367458b95e..48a6a736ff62 100644 --- a/pkg/sql/truncate.go +++ b/pkg/sql/truncate.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" @@ -499,8 +500,8 @@ func (p *planner) copySplitPointsToNewIndexes( expirationTime += jitter log.Infof(ctx, "truncate sending split request for key %s", sp) - b.AddRawRequest(&roachpb.AdminSplitRequest{ - RequestHeader: roachpb.RequestHeader{ + b.AddRawRequest(&kvpb.AdminSplitRequest{ + RequestHeader: kvpb.RequestHeader{ Key: sp, }, SplitKey: sp, @@ -514,10 +515,10 @@ func (p *planner) copySplitPointsToNewIndexes( // Now scatter the ranges, after we've finished splitting them. b = kv.Batch{} - b.AddRawRequest(&roachpb.AdminScatterRequest{ + b.AddRawRequest(&kvpb.AdminScatterRequest{ // Scatter all of the data between the start key of the first new index, and // the PrefixEnd of the last new index. - RequestHeader: roachpb.RequestHeader{ + RequestHeader: kvpb.RequestHeader{ Key: execCfg.Codec.IndexPrefix(uint32(tableID), uint32(newIndexIDs[0])), EndKey: execCfg.Codec.IndexPrefix(uint32(tableID), uint32(newIndexIDs[len(newIndexIDs)-1])).PrefixEnd(), }, diff --git a/pkg/sql/txn_restart_test.go b/pkg/sql/txn_restart_test.go index 4fe0a9062b5c..5dc0eb1c7c63 100644 --- a/pkg/sql/txn_restart_test.go +++ b/pkg/sql/txn_restart_test.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -86,7 +87,7 @@ func checkCorrectTxn(value string, magicVals *filterVals, txn *roachpb.Transacti if !found { return nil } - if errors.HasType(failureRec.err, (*roachpb.TransactionAbortedError)(nil)) { + if errors.HasType(failureRec.err, (*kvpb.TransactionAbortedError)(nil)) { // The previous txn should have been aborted, so check that we're running // in a new one. if failureRec.txn.ID == txn.ID { @@ -115,24 +116,22 @@ type injectionApproaches []injectionApproach func (ia injectionApproaches) Len() int { return len(ia) } func (ia injectionApproaches) Swap(i, j int) { ia[i], ia[j] = ia[j], ia[i] } -func injectErrors( - req roachpb.Request, hdr roachpb.Header, magicVals *filterVals, verifyTxn bool, -) error { +func injectErrors(req kvpb.Request, hdr kvpb.Header, magicVals *filterVals, verifyTxn bool) error { magicVals.Lock() defer magicVals.Unlock() switch req := req.(type) { - case *roachpb.ConditionalPutRequest: + case *kvpb.ConditionalPutRequest: // Create a list of each injection approach and shuffle the order of // injection for some additional randomness. injections := injectionApproaches{ {counts: magicVals.restartCounts, errFn: func() error { // Note we use a retry error that cannot be automatically retried // by the transaction coord sender. - return roachpb.NewTransactionRetryError(roachpb.RETRY_REASON_UNKNOWN, "injected err") + return kvpb.NewTransactionRetryError(kvpb.RETRY_REASON_UNKNOWN, "injected err") }}, {counts: magicVals.abortCounts, errFn: func() error { - return roachpb.NewTransactionAbortedError(roachpb.ABORT_REASON_ABORTED_RECORD_FOUND) + return kvpb.NewTransactionAbortedError(kvpb.ABORT_REASON_ABORTED_RECORD_FOUND) }}, } shuffle.Shuffle(injections) @@ -495,9 +494,9 @@ func TestTxnAutoRetry(t *testing.T) { "boulanger": 2, } cleanupFilter := cmdFilters.AppendFilter( - func(args kvserverbase.FilterArgs) *roachpb.Error { + func(args kvserverbase.FilterArgs) *kvpb.Error { if err := injectErrors(args.Req, args.Hdr, magicVals, true /* verifyTxn */); err != nil { - return roachpb.NewErrorWithTxn(err, args.Hdr.Txn) + return kvpb.NewErrorWithTxn(err, args.Hdr.Txn) } return nil }, false) @@ -599,9 +598,9 @@ INSERT INTO t.public.test(k, v, t) VALUES (6, 'laureal', cluster_logical_timesta "hooly": 2, } cleanupFilter = cmdFilters.AppendFilter( - func(args kvserverbase.FilterArgs) *roachpb.Error { + func(args kvserverbase.FilterArgs) *kvpb.Error { if err := injectErrors(args.Req, args.Hdr, magicVals, true /* verifyTxn */); err != nil { - return roachpb.NewErrorWithTxn(err, args.Hdr.Txn) + return kvpb.NewErrorWithTxn(err, args.Hdr.Txn) } return nil }, false) @@ -809,9 +808,9 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v TEXT); t.Fatal(err) } cleanupFilter := cmdFilters.AppendFilter( - func(args kvserverbase.FilterArgs) *roachpb.Error { + func(args kvserverbase.FilterArgs) *kvpb.Error { if err := injectErrors(args.Req, args.Hdr, tc.magicVals, true /* verifyTxn */); err != nil { - return roachpb.NewErrorWithTxn(err, args.Hdr.Txn) + return kvpb.NewErrorWithTxn(err, args.Hdr.Txn) } return nil }, false) @@ -1112,11 +1111,11 @@ func TestNonRetryableError(t *testing.T) { testKey := []byte("test_key") hitError := false cleanupFilter := cmdFilters.AppendFilter( - func(args kvserverbase.FilterArgs) *roachpb.Error { - if req, ok := args.Req.(*roachpb.GetRequest); ok { + func(args kvserverbase.FilterArgs) *kvpb.Error { + if req, ok := args.Req.(*kvpb.GetRequest); ok { if bytes.Contains(req.Key, testKey) && !kv.TestingIsRangeLookupRequest(req) { hitError = true - return roachpb.NewErrorWithTxn(fmt.Errorf("testError"), args.Hdr.Txn) + return kvpb.NewErrorWithTxn(fmt.Errorf("testError"), args.Hdr.Txn) } } return nil @@ -1158,8 +1157,8 @@ func TestReacquireLeaseOnRestart(t *testing.T) { var s serverutils.TestServerInterface var clockUpdate, restartDone int32 testingResponseFilter := func( - ctx context.Context, ba *roachpb.BatchRequest, br *roachpb.BatchResponse, - ) *roachpb.Error { + ctx context.Context, ba *kvpb.BatchRequest, br *kvpb.BatchResponse, + ) *kvpb.Error { for _, ru := range ba.Requests { if req := ru.GetGet(); req != nil { if bytes.Contains(req.Key, testKey) && !kv.TestingIsRangeLookupRequest(req) { @@ -1183,7 +1182,7 @@ func TestReacquireLeaseOnRestart(t *testing.T) { txn.ResetObservedTimestamps() now := s.Clock().NowAsClockTimestamp() txn.UpdateObservedTimestamp(s.(*server.TestServer).Gossip().NodeID.Get(), now) - return roachpb.NewErrorWithTxn(roachpb.NewReadWithinUncertaintyIntervalError(now.ToTimestamp(), now, txn, now.ToTimestamp(), now), txn) + return kvpb.NewErrorWithTxn(kvpb.NewReadWithinUncertaintyIntervalError(now.ToTimestamp(), now, txn, now.ToTimestamp(), now), txn) } } } @@ -1254,12 +1253,12 @@ func TestFlushUncommitedDescriptorCacheOnRestart(t *testing.T) { var restartDone int32 cleanupFilter := cmdFilters.AppendFilter( - func(args kvserverbase.FilterArgs) *roachpb.Error { + func(args kvserverbase.FilterArgs) *kvpb.Error { if atomic.LoadInt32(&restartDone) > 0 { return nil } - if req, ok := args.Req.(*roachpb.GetRequest); ok { + if req, ok := args.Req.(*kvpb.GetRequest); ok { if bytes.Contains(req.Key, testKey) && !kv.TestingIsRangeLookupRequest(req) { atomic.AddInt32(&restartDone, 1) // Return ReadWithinUncertaintyIntervalError. @@ -1267,7 +1266,7 @@ func TestFlushUncommitedDescriptorCacheOnRestart(t *testing.T) { txn.ResetObservedTimestamps() now := s.Clock().NowAsClockTimestamp() txn.UpdateObservedTimestamp(s.(*server.TestServer).Gossip().NodeID.Get(), now) - return roachpb.NewErrorWithTxn(roachpb.NewReadWithinUncertaintyIntervalError(now.ToTimestamp(), now, txn, now.ToTimestamp(), now), txn) + return kvpb.NewErrorWithTxn(kvpb.NewReadWithinUncertaintyIntervalError(now.ToTimestamp(), now, txn, now.ToTimestamp(), now), txn) } } return nil @@ -1346,11 +1345,11 @@ func TestDistSQLRetryableError(t *testing.T) { Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ EvalKnobs: kvserverbase.BatchEvalTestingKnobs{ - TestingEvalFilter: func(fArgs kvserverbase.FilterArgs) *roachpb.Error { - _, ok := fArgs.Req.(*roachpb.ScanRequest) + TestingEvalFilter: func(fArgs kvserverbase.FilterArgs) *kvpb.Error { + _, ok := fArgs.Req.(*kvpb.ScanRequest) if ok && fArgs.Req.Header().Key.Equal(targetKey) && fArgs.Hdr.Txn.Epoch == 0 { restarted = true - err := roachpb.NewReadWithinUncertaintyIntervalError( + err := kvpb.NewReadWithinUncertaintyIntervalError( fArgs.Hdr.Timestamp, /* readTS */ hlc.ClockTimestamp{}, nil, @@ -1358,7 +1357,7 @@ func TestDistSQLRetryableError(t *testing.T) { hlc.ClockTimestamp{}) errTxn := fArgs.Hdr.Txn.Clone() errTxn.UpdateObservedTimestamp(roachpb.NodeID(2), hlc.ClockTimestamp{}) - pErr := roachpb.NewErrorWithTxn(err, errTxn) + pErr := kvpb.NewErrorWithTxn(err, errTxn) pErr.OriginNode = 2 return pErr } @@ -1637,10 +1636,10 @@ func TestTxnAutoRetryReasonAvailable(t *testing.T) { retriedStmtKey := []byte("test_key") cleanupFilter := cmdFilters.AppendFilter( - func(args kvserverbase.FilterArgs) *roachpb.Error { - if req, ok := args.Req.(*roachpb.GetRequest); ok { + func(args kvserverbase.FilterArgs) *kvpb.Error { + if req, ok := args.Req.(*kvpb.GetRequest); ok { if bytes.Contains(req.Key, retriedStmtKey) && retryCount < numRetries { - return roachpb.NewErrorWithTxn(roachpb.NewTransactionRetryError(roachpb.RETRY_REASON_UNKNOWN, + return kvpb.NewErrorWithTxn(kvpb.NewTransactionRetryError(kvpb.RETRY_REASON_UNKNOWN, redact.Sprintf("injected err %d", retryCount+1)), args.Hdr.Txn) } } diff --git a/pkg/sql/txn_state_test.go b/pkg/sql/txn_state_test.go index 03f0e35b73e4..24c63e535030 100644 --- a/pkg/sql/txn_state_test.go +++ b/pkg/sql/txn_state_test.go @@ -17,6 +17,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -52,8 +53,8 @@ type testContext struct { func makeTestContext(stopper *stop.Stopper) testContext { clock := hlc.NewClockForTesting(timeutil.NewManualTime(timeutil.Unix(0, 123))) factory := kv.MakeMockTxnSenderFactory( - func(context.Context, *roachpb.Transaction, *roachpb.BatchRequest, - ) (*roachpb.BatchResponse, *roachpb.Error) { + func(context.Context, *roachpb.Transaction, *kvpb.BatchRequest, + ) (*kvpb.BatchResponse, *kvpb.Error) { return nil, nil }) diff --git a/pkg/sql/upsert_test.go b/pkg/sql/upsert_test.go index e3aae8e71aec..0b9c9f5f8c87 100644 --- a/pkg/sql/upsert_test.go +++ b/pkg/sql/upsert_test.go @@ -17,6 +17,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -45,14 +46,14 @@ func TestUpsertFastPath(t *testing.T) { var gets uint64 var scans uint64 var endTxn uint64 - filter := func(filterArgs kvserverbase.FilterArgs) *roachpb.Error { + filter := func(filterArgs kvserverbase.FilterArgs) *kvpb.Error { if bytes.Compare(filterArgs.Req.Header().Key, bootstrap.TestingUserTableDataMin()) >= 0 { switch filterArgs.Req.Method() { - case roachpb.Scan: + case kvpb.Scan: atomic.AddUint64(&scans, 1) - case roachpb.Get: + case kvpb.Get: atomic.AddUint64(&gets, 1) - case roachpb.EndTxn: + case kvpb.EndTxn: if filterArgs.Hdr.Txn.Status == roachpb.STAGING { // Ignore async explicit commits. return nil diff --git a/pkg/sql/user_test.go b/pkg/sql/user_test.go index 6a13e511a382..51b20d615f56 100644 --- a/pkg/sql/user_test.go +++ b/pkg/sql/user_test.go @@ -18,8 +18,8 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql/sqltestutils" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -59,7 +59,7 @@ func TestGetUserTimeout(t *testing.T) { close(closedCh) unavailableCh.Store(closedCh) knobs := &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(ctx context.Context, _ *roachpb.BatchRequest) *roachpb.Error { + TestingRequestFilter: func(ctx context.Context, _ *kvpb.BatchRequest) *kvpb.Error { select { case <-unavailableCh.Load().(chan struct{}): case <-ctx.Done(): diff --git a/pkg/storage/BUILD.bazel b/pkg/storage/BUILD.bazel index 8e80e545e2e6..916b6c7490ab 100644 --- a/pkg/storage/BUILD.bazel +++ b/pkg/storage/BUILD.bazel @@ -52,6 +52,7 @@ go_library( "//pkg/clusterversion", "//pkg/keys", "//pkg/kv/kvnemesis/kvnemesisutil", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/concurrency/lock", "//pkg/kv/kvserver/diskmap", "//pkg/kv/kvserver/uncertainty", @@ -145,6 +146,7 @@ go_test( "//pkg/base", "//pkg/clusterversion", "//pkg/keys", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/concurrency/lock", "//pkg/kv/kvserver/diskmap", "//pkg/kv/kvserver/spanset", diff --git a/pkg/storage/bench_test.go b/pkg/storage/bench_test.go index 9ae6d381b8ab..f2c33ec9a76d 100644 --- a/pkg/storage/bench_test.go +++ b/pkg/storage/bench_test.go @@ -23,6 +23,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog" @@ -1385,12 +1386,12 @@ func runMVCCGarbageCollect( return ts.Add(int64(wallTime), int32(logical)) } - setup := func() (gcKeys []roachpb.GCRequest_GCKey) { + setup := func() (gcKeys []kvpb.GCRequest_GCKey) { batch := eng.NewBatch() pointKeys := make([]roachpb.Key, opts.numKeys) for i := 0; i < opts.numKeys; i++ { pointKeys[i] = randutil.RandBytes(rng, opts.keyBytes) - gcKeys = append(gcKeys, roachpb.GCRequest_GCKey{ + gcKeys = append(gcKeys, kvpb.GCRequest_GCKey{ Timestamp: pointKeyTs(opts.deleteVersions - 1), Key: pointKeys[i], }) diff --git a/pkg/storage/enginepb/mvcc3.proto b/pkg/storage/enginepb/mvcc3.proto index 99c201416d68..4e3b2f385afe 100644 --- a/pkg/storage/enginepb/mvcc3.proto +++ b/pkg/storage/enginepb/mvcc3.proto @@ -150,7 +150,7 @@ message IgnoredSeqNumRange { // change, need to update mvccGetWithValueHeader. // // NB: ensure that this struct stays in sync with MVCCValueHeader{Pure,CrdbTest}. -// See roachpb.RequestHeader for details on how they help us keep the KVNemesisSeq +// See kvpb.RequestHeader for details on how they help us keep the KVNemesisSeq // field out of production code. message MVCCValueHeader { option (gogoproto.equal) = true; diff --git a/pkg/storage/metamorphic/BUILD.bazel b/pkg/storage/metamorphic/BUILD.bazel index b674ce8eeada..fc28c84a8b1f 100644 --- a/pkg/storage/metamorphic/BUILD.bazel +++ b/pkg/storage/metamorphic/BUILD.bazel @@ -16,6 +16,7 @@ go_library( deps = [ "//pkg/base", "//pkg/keys", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/settings/cluster", "//pkg/storage", diff --git a/pkg/storage/metamorphic/operations.go b/pkg/storage/metamorphic/operations.go index 2c99168bb000..75a5b26db568 100644 --- a/pkg/storage/metamorphic/operations.go +++ b/pkg/storage/metamorphic/operations.go @@ -20,6 +20,7 @@ import ( "strings" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" @@ -207,7 +208,7 @@ func (m mvccPutOp) run(ctx context.Context) string { err := storage.MVCCPut(ctx, writer, nil, m.key, txn.ReadTimestamp, hlc.ClockTimestamp{}, m.value, txn) if err != nil { - if writeTooOldErr := (*roachpb.WriteTooOldError)(nil); errors.As(err, &writeTooOldErr) { + if writeTooOldErr := (*kvpb.WriteTooOldError)(nil); errors.As(err, &writeTooOldErr) { txn.WriteTimestamp.Forward(writeTooOldErr.ActualTimestamp) // Update the txn's lock spans to account for this intent being written. addKeyToLockSpans(txn, m.key) @@ -237,7 +238,7 @@ func (m mvccCPutOp) run(ctx context.Context) string { err := storage.MVCCConditionalPut(ctx, writer, nil, m.key, txn.ReadTimestamp, hlc.ClockTimestamp{}, m.value, m.expVal, true, txn) if err != nil { - if writeTooOldErr := (*roachpb.WriteTooOldError)(nil); errors.As(err, &writeTooOldErr) { + if writeTooOldErr := (*kvpb.WriteTooOldError)(nil); errors.As(err, &writeTooOldErr) { txn.WriteTimestamp.Forward(writeTooOldErr.ActualTimestamp) // Update the txn's lock spans to account for this intent being written. addKeyToLockSpans(txn, m.key) @@ -265,7 +266,7 @@ func (m mvccInitPutOp) run(ctx context.Context) string { err := storage.MVCCInitPut(ctx, writer, nil, m.key, txn.ReadTimestamp, hlc.ClockTimestamp{}, m.value, false, txn) if err != nil { - if writeTooOldErr := (*roachpb.WriteTooOldError)(nil); errors.As(err, &writeTooOldErr) { + if writeTooOldErr := (*kvpb.WriteTooOldError)(nil); errors.As(err, &writeTooOldErr) { txn.WriteTimestamp.Forward(writeTooOldErr.ActualTimestamp) // Update the txn's lock spans to account for this intent being written. addKeyToLockSpans(txn, m.key) @@ -377,7 +378,7 @@ func (m mvccDeleteOp) run(ctx context.Context) string { _, err := storage.MVCCDelete(ctx, writer, nil, m.key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn) if err != nil { - if writeTooOldErr := (*roachpb.WriteTooOldError)(nil); errors.As(err, &writeTooOldErr) { + if writeTooOldErr := (*kvpb.WriteTooOldError)(nil); errors.As(err, &writeTooOldErr) { txn.WriteTimestamp.Forward(writeTooOldErr.ActualTimestamp) // Update the txn's lock spans to account for this intent being written. addKeyToLockSpans(txn, m.key) diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index e563c85353f1..b1973155166f 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/uncertainty" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -924,10 +925,10 @@ type MVCCGetResult struct { // consistent mode, an intent will generate a WriteIntentError with the // intent embedded within and the intent parameter will be nil. Intent *roachpb.Intent - // See the documentation for roachpb.ResponseHeader for information on + // See the documentation for kvpb.ResponseHeader for information on // these parameters. ResumeSpan *roachpb.Span - ResumeReason roachpb.ResumeReason + ResumeReason kvpb.ResumeReason ResumeNextBytes int64 NumKeys int64 NumBytes int64 @@ -1059,9 +1060,9 @@ func MVCCGetWithValueHeader( // ResumeSpan for this GetRequest. result.ResumeSpan = &roachpb.Span{Key: key} if opts.MaxKeys < 0 { - result.ResumeReason = roachpb.RESUME_KEY_LIMIT + result.ResumeReason = kvpb.RESUME_KEY_LIMIT } else if opts.TargetBytes < 0 { - result.ResumeReason = roachpb.RESUME_BYTE_LIMIT + result.ResumeReason = kvpb.RESUME_BYTE_LIMIT } return result, enginepb.MVCCValueHeader{}, nil } @@ -1080,7 +1081,7 @@ func MVCCGetWithValueHeader( numBytes := int64(len(val.RawBytes)) if opts.TargetBytes > 0 && opts.AllowEmpty && numBytes > opts.TargetBytes { result.ResumeSpan = &roachpb.Span{Key: key} - result.ResumeReason = roachpb.RESUME_BYTE_LIMIT + result.ResumeReason = kvpb.RESUME_BYTE_LIMIT result.ResumeNextBytes = numBytes return result, enginepb.MVCCValueHeader{}, nil } @@ -1159,7 +1160,7 @@ func mvccGetWithValueHeader( return optionalValue{}, nil, enginepb.MVCCValueHeader{}, err } if opts.errOnIntents() && len(intents) > 0 { - return optionalValue{}, nil, enginepb.MVCCValueHeader{}, &roachpb.WriteIntentError{Intents: intents} + return optionalValue{}, nil, enginepb.MVCCValueHeader{}, &kvpb.WriteIntentError{Intents: intents} } if len(intents) > 1 { @@ -1847,7 +1848,7 @@ func mvccPutInternal( if txn == nil || meta.Txn.ID != txn.ID { // The current Put operation does not come from the same // transaction. - return false, &roachpb.WriteIntentError{Intents: []roachpb.Intent{ + return false, &kvpb.WriteIntentError{Intents: []roachpb.Intent{ roachpb.MakeIntent(meta.Txn, key), }} } else if txn.Epoch < meta.Txn.Epoch { @@ -2067,7 +2068,7 @@ func mvccPutInternal( // instead of allowing their transactions to continue and be retried // before committing. writeTimestamp.Forward(metaTimestamp.Next()) - maybeTooOldErr = roachpb.NewWriteTooOldError(readTimestamp, writeTimestamp, key) + maybeTooOldErr = kvpb.NewWriteTooOldError(readTimestamp, writeTimestamp, key) // If we're in a transaction, always get the value at the orig // timestamp. Outside of a transaction, the read timestamp advances // to the the latest value's timestamp + 1 as well. The new @@ -2257,7 +2258,7 @@ func MVCCIncrement( if willOverflow(int64Val, inc) { // Return the old value, since we've failed to modify it. newInt64Val = int64Val - return roachpb.Value{}, &roachpb.IntegerOverflowError{ + return roachpb.Value{}, &kvpb.IntegerOverflowError{ Key: key, CurrentValue: int64Val, IncrementValue: inc, @@ -2367,12 +2368,12 @@ func mvccConditionalPutUsingIter( valueFn := func(existVal optionalValue) (roachpb.Value, error) { if expValPresent, existValPresent := len(expBytes) != 0, existVal.IsPresent(); expValPresent && existValPresent { if !bytes.Equal(expBytes, existVal.TagAndDataBytes()) { - return roachpb.Value{}, &roachpb.ConditionFailedError{ + return roachpb.Value{}, &kvpb.ConditionFailedError{ ActualValue: existVal.ToPointer(), } } } else if expValPresent != existValPresent && (existValPresent || !bool(allowNoExisting)) { - return roachpb.Value{}, &roachpb.ConditionFailedError{ + return roachpb.Value{}, &kvpb.ConditionFailedError{ ActualValue: existVal.ToPointer(), } } @@ -2449,13 +2450,13 @@ func mvccInitPutUsingIter( valueFn := func(existVal optionalValue) (roachpb.Value, error) { if failOnTombstones && existVal.IsTombstone() { // We found a tombstone and failOnTombstones is true: fail. - return roachpb.Value{}, &roachpb.ConditionFailedError{ + return roachpb.Value{}, &kvpb.ConditionFailedError{ ActualValue: existVal.ToPointer(), } } if existVal.IsPresent() && !existVal.EqualTagAndData(value) { // The existing value does not match the supplied value. - return roachpb.Value{}, &roachpb.ConditionFailedError{ + return roachpb.Value{}, &kvpb.ConditionFailedError{ ActualValue: existVal.ToPointer(), } } @@ -3065,7 +3066,7 @@ func MVCCPredicateDeleteRange( endTime hlc.Timestamp, localTimestamp hlc.ClockTimestamp, leftPeekBound, rightPeekBound roachpb.Key, - predicates roachpb.DeleteRangePredicates, + predicates kvpb.DeleteRangePredicates, maxBatchSize, maxBatchByteSize int64, rangeTombstoneThreshold int64, maxIntents int64, @@ -3101,7 +3102,7 @@ func MVCCPredicateDeleteRange( if intents, err := ScanIntents(ctx, rw, startKey, endKey, maxIntents, 0); err != nil { return nil, err } else if len(intents) > 0 { - return nil, &roachpb.WriteIntentError{Intents: intents} + return nil, &kvpb.WriteIntentError{Intents: intents} } // continueRun returns three bools: the first is true if the current run @@ -3126,7 +3127,7 @@ func MVCCPredicateDeleteRange( if hasRangeKey { newestRangeKey := rangeKeys.Newest() if endTime.LessEq(rangeKeys.Newest()) { - return false, false, false, roachpb.NewWriteTooOldError( + return false, false, false, kvpb.NewWriteTooOldError( endTime, newestRangeKey.Next(), k.Key.Clone()) } if !hasPointKey { @@ -3142,7 +3143,7 @@ func MVCCPredicateDeleteRange( // At this point, there exists a point key that shadows all range keys, // if they exist. if endTime.LessEq(k.Timestamp) { - return false, false, false, roachpb.NewWriteTooOldError(endTime, k.Timestamp.Next(), + return false, false, false, kvpb.NewWriteTooOldError(endTime, k.Timestamp.Next(), k.Key.Clone()) } _, isTombstone, err := iter.MVCCValueLenAndIsTombstone() @@ -3390,7 +3391,7 @@ func MVCCDeleteRangeUsingTombstone( if intents, err := ScanIntents(ctx, rw, startKey, endKey, maxIntents, 0); err != nil { return err } else if len(intents) > 0 { - return &roachpb.WriteIntentError{Intents: intents} + return &kvpb.WriteIntentError{Intents: intents} } // If requested, check if there are any point keys/tombstones in the span that @@ -3414,7 +3415,7 @@ func MVCCDeleteRangeUsingTombstone( if hasPoint, _ := iter.HasPointAndRange(); hasPoint { return false, nil } else if newest := iter.RangeKeys().Newest(); timestamp.LessEq(newest) { - return false, roachpb.NewWriteTooOldError(timestamp, newest.Next(), iter.RangeBounds().Key) + return false, kvpb.NewWriteTooOldError(timestamp, newest.Next(), iter.RangeBounds().Key) } } return true, nil @@ -3439,7 +3440,7 @@ func MVCCDeleteRangeUsingTombstone( return err } else if ok { key := iter.UnsafeKey() - return roachpb.NewWriteTooOldError(timestamp, key.Timestamp.Next(), key.Key) + return kvpb.NewWriteTooOldError(timestamp, key.Timestamp.Next(), key.Key) } return nil }(); err != nil { @@ -3477,7 +3478,7 @@ func MVCCDeleteRangeUsingTombstone( if hasRange { rangeKeys := iter.RangeKeys() if timestamp.LessEq(rangeKeys.Newest()) { - return roachpb.NewWriteTooOldError(timestamp, rangeKeys.Newest().Next(), + return kvpb.NewWriteTooOldError(timestamp, rangeKeys.Newest().Next(), rangeKeys.Bounds.Key) } @@ -3508,7 +3509,7 @@ func MVCCDeleteRangeUsingTombstone( // Process point key. key := iter.UnsafeKey() if timestamp.LessEq(key.Timestamp) { - return roachpb.NewWriteTooOldError(timestamp, key.Timestamp.Next(), key.Key) + return kvpb.NewWriteTooOldError(timestamp, key.Timestamp.Next(), key.Key) } if key.Timestamp.IsEmpty() { return errors.Errorf("can't write range tombstone across inline key %s", key) @@ -3626,7 +3627,7 @@ func recordIteratorStats(ctx context.Context, iter MVCCIterator) { seeks := stats.ReverseSeekCount[pebble.InterfaceCall] + stats.ForwardSeekCount[pebble.InterfaceCall] internalSteps := stats.ReverseStepCount[pebble.InternalIterCall] + stats.ForwardStepCount[pebble.InternalIterCall] internalSeeks := stats.ReverseSeekCount[pebble.InternalIterCall] + stats.ForwardSeekCount[pebble.InternalIterCall] - sp.RecordStructured(&roachpb.ScanStats{ + sp.RecordStructured(&kvpb.ScanStats{ NumInterfaceSeeks: uint64(seeks), NumInternalSeeks: uint64(internalSeeks), NumInterfaceSteps: uint64(steps), @@ -3670,13 +3671,13 @@ func mvccScanInit( if opts.MaxKeys < 0 { return false, nil, MVCCScanResult{ ResumeSpan: &roachpb.Span{Key: key, EndKey: endKey}, - ResumeReason: roachpb.RESUME_KEY_LIMIT, + ResumeReason: kvpb.RESUME_KEY_LIMIT, }, nil } if opts.TargetBytes < 0 { return false, nil, MVCCScanResult{ ResumeSpan: &roachpb.Span{Key: key, EndKey: endKey}, - ResumeReason: roachpb.RESUME_BYTE_LIMIT, + ResumeReason: kvpb.RESUME_BYTE_LIMIT, }, nil } @@ -3755,7 +3756,7 @@ func finalizeScanResult( } if errOnIntents && len(res.Intents) > 0 { - return &roachpb.WriteIntentError{Intents: res.Intents} + return &kvpb.WriteIntentError{Intents: res.Intents} } return nil } @@ -3913,7 +3914,7 @@ type MVCCScanResult struct { NumBytes int64 ResumeSpan *roachpb.Span - ResumeReason roachpb.ResumeReason + ResumeReason kvpb.ResumeReason ResumeNextBytes int64 // populated if TargetBytes != 0, size of next resume kv Intents []roachpb.Intent } @@ -4924,20 +4925,15 @@ func MVCCResolveWriteIntentRange( ms *enginepb.MVCCStats, intent roachpb.LockUpdate, opts MVCCResolveWriteIntentRangeOptions, -) ( - numKeys, numBytes int64, - resumeSpan *roachpb.Span, - resumeReason roachpb.ResumeReason, - err error, -) { +) (numKeys, numBytes int64, resumeSpan *roachpb.Span, resumeReason kvpb.ResumeReason, err error) { keysExceeded := opts.MaxKeys < 0 bytesExceeded := opts.TargetBytes < 0 if keysExceeded || bytesExceeded { resumeSpan := intent.Span // don't inline or `intent` would escape to heap if keysExceeded { - resumeReason = roachpb.RESUME_KEY_LIMIT + resumeReason = kvpb.RESUME_KEY_LIMIT } else if bytesExceeded { - resumeReason = roachpb.RESUME_BYTE_LIMIT + resumeReason = kvpb.RESUME_BYTE_LIMIT } return 0, 0, &resumeSpan, resumeReason, nil } @@ -4986,9 +4982,9 @@ func MVCCResolveWriteIntentRange( bytesExceeded = opts.TargetBytes > 0 && numBytes >= opts.TargetBytes if keysExceeded || bytesExceeded { if keysExceeded { - resumeReason = roachpb.RESUME_KEY_LIMIT + resumeReason = kvpb.RESUME_KEY_LIMIT } else if bytesExceeded { - resumeReason = roachpb.RESUME_BYTE_LIMIT + resumeReason = kvpb.RESUME_BYTE_LIMIT } // We could also compute a tighter nextKey here if we wanted to. return numKeys, numBytes, &roachpb.Span{Key: lastResolvedKey.Next(), EndKey: intentEndKey}, resumeReason, nil @@ -5044,7 +5040,7 @@ func MVCCGarbageCollect( ctx context.Context, rw ReadWriter, ms *enginepb.MVCCStats, - keys []roachpb.GCRequest_GCKey, + keys []kvpb.GCRequest_GCKey, timestamp hlc.Timestamp, ) error { @@ -6190,7 +6186,7 @@ func MVCCIsSpanEmpty( // allocations. func MVCCExportFingerprint( ctx context.Context, cs *cluster.Settings, reader Reader, opts MVCCExportOptions, dest io.Writer, -) (roachpb.BulkOpSummary, MVCCKey, uint64, bool, error) { +) (kvpb.BulkOpSummary, MVCCKey, uint64, bool, error) { ctx, span := tracing.ChildSpan(ctx, "storage.MVCCExportFingerprint") defer span.Finish() @@ -6200,12 +6196,12 @@ func MVCCExportFingerprint( summary, resumeKey, err := mvccExportToWriter(ctx, reader, opts, &fingerprintWriter) if err != nil { - return roachpb.BulkOpSummary{}, MVCCKey{}, 0, false, err + return kvpb.BulkOpSummary{}, MVCCKey{}, 0, false, err } fingerprint, err := fingerprintWriter.Finish() if err != nil { - return roachpb.BulkOpSummary{}, MVCCKey{}, 0, false, err + return kvpb.BulkOpSummary{}, MVCCKey{}, 0, false, err } hasRangeKeys := fingerprintWriter.sstWriter.DataSize != 0 @@ -6217,7 +6213,7 @@ func MVCCExportFingerprint( // details. func MVCCExportToSST( ctx context.Context, cs *cluster.Settings, reader Reader, opts MVCCExportOptions, dest io.Writer, -) (roachpb.BulkOpSummary, MVCCKey, error) { +) (kvpb.BulkOpSummary, MVCCKey, error) { ctx, span := tracing.ChildSpan(ctx, "storage.MVCCExportToSST") defer span.Finish() sstWriter := MakeBackupSSTWriter(ctx, cs, dest) @@ -6225,7 +6221,7 @@ func MVCCExportToSST( summary, resumeKey, err := mvccExportToWriter(ctx, reader, opts, &sstWriter) if err != nil { - return roachpb.BulkOpSummary{}, MVCCKey{}, err + return kvpb.BulkOpSummary{}, MVCCKey{}, err } if summary.DataSize == 0 { @@ -6236,7 +6232,7 @@ func MVCCExportToSST( // iteration may have been halted because of resource // limitations before any keys were added to the // returned SST. - return roachpb.BulkOpSummary{}, resumeKey, nil + return kvpb.BulkOpSummary{}, resumeKey, nil } return summary, resumeKey, sstWriter.Finish() @@ -6299,7 +6295,7 @@ type ExportWriter interface { // responsibility of the caller to Finish() / Close() the passed in writer. func mvccExportToWriter( ctx context.Context, reader Reader, opts MVCCExportOptions, writer ExportWriter, -) (roachpb.BulkOpSummary, MVCCKey, error) { +) (kvpb.BulkOpSummary, MVCCKey, error) { // If we're not exporting all revisions then we can mask point keys below any // MVCC range tombstones, since we don't care about them. var rangeKeyMasking hlc.Timestamp @@ -6383,7 +6379,7 @@ func mvccExportToWriter( iter.SeekGE(opts.StartKey) for { if ok, err := iter.Valid(); err != nil { - return roachpb.BulkOpSummary{}, MVCCKey{}, err + return kvpb.BulkOpSummary{}, MVCCKey{}, err } else if !ok { break } else if iter.NumCollectedIntents() > 0 { @@ -6434,13 +6430,13 @@ func mvccExportToWriter( mvccValue, err = decodeExtendedMVCCValue(v.Value) } if err != nil { - return roachpb.BulkOpSummary{}, MVCCKey{}, errors.Wrapf(err, + return kvpb.BulkOpSummary{}, MVCCKey{}, errors.Wrapf(err, "decoding mvcc value %s", v.Value) } // Export only the inner roachpb.Value, not the MVCCValue header. rawValue := mvccValue.Value.RawBytes if err := writer.PutRawMVCCRangeKey(rangeKeys.AsRangeKey(v), rawValue); err != nil { - return roachpb.BulkOpSummary{}, MVCCKey{}, err + return kvpb.BulkOpSummary{}, MVCCKey{}, err } } rows.BulkOpSummary.DataSize += rangeKeysSize @@ -6485,7 +6481,7 @@ func mvccExportToWriter( break } if reachedMaxSize { - return roachpb.BulkOpSummary{}, MVCCKey{}, &ExceedMaxSizeError{ + return kvpb.BulkOpSummary{}, MVCCKey{}, &ExceedMaxSizeError{ reached: newSize, maxSize: opts.MaxSize} } } @@ -6501,7 +6497,7 @@ func mvccExportToWriter( // Process point keys. unsafeValue, err := iter.UnsafeValue() if err != nil { - return roachpb.BulkOpSummary{}, MVCCKey{}, err + return kvpb.BulkOpSummary{}, MVCCKey{}, err } skip := false if unsafeKey.IsValue() { @@ -6510,7 +6506,7 @@ func mvccExportToWriter( mvccValue, err = decodeExtendedMVCCValue(unsafeValue) } if err != nil { - return roachpb.BulkOpSummary{}, MVCCKey{}, errors.Wrapf(err, "decoding mvcc value %s", unsafeKey) + return kvpb.BulkOpSummary{}, MVCCKey{}, errors.Wrapf(err, "decoding mvcc value %s", unsafeKey) } // Export only the inner roachpb.Value, not the MVCCValue header. @@ -6523,7 +6519,7 @@ func mvccExportToWriter( if !skip { if err := rows.Count(unsafeKey.Key); err != nil { - return roachpb.BulkOpSummary{}, MVCCKey{}, errors.Wrapf(err, "decoding %s", unsafeKey) + return kvpb.BulkOpSummary{}, MVCCKey{}, errors.Wrapf(err, "decoding %s", unsafeKey) } curSize := rows.BulkOpSummary.DataSize curSizeWithRangeKeys := curSize + maxRangeKeysSizeIfTruncated(unsafeKey.Key) @@ -6532,7 +6528,7 @@ func mvccExportToWriter( kvSize := int64(len(unsafeKey.Key) + len(unsafeValue)) if curSize == 0 && opts.MaxSize > 0 && kvSize > int64(opts.MaxSize) { // This single key exceeds the MaxSize. Even if we paginate below, this will still fail. - return roachpb.BulkOpSummary{}, MVCCKey{}, &ExceedMaxSizeError{reached: kvSize, maxSize: opts.MaxSize} + return kvpb.BulkOpSummary{}, MVCCKey{}, &ExceedMaxSizeError{reached: kvSize, maxSize: opts.MaxSize} } newSize := curSize + kvSize newSizeWithRangeKeys := curSizeWithRangeKeys + kvSize @@ -6551,18 +6547,18 @@ func mvccExportToWriter( break } if reachedMaxSize { - return roachpb.BulkOpSummary{}, MVCCKey{}, &ExceedMaxSizeError{ + return kvpb.BulkOpSummary{}, MVCCKey{}, &ExceedMaxSizeError{ reached: newSizeWithRangeKeys, maxSize: opts.MaxSize} } if unsafeKey.Timestamp.IsEmpty() { // This should never be an intent since the incremental iterator returns // an error when encountering intents. if err := writer.PutUnversioned(unsafeKey.Key, unsafeValue); err != nil { - return roachpb.BulkOpSummary{}, MVCCKey{}, errors.Wrapf(err, "adding key %s", unsafeKey) + return kvpb.BulkOpSummary{}, MVCCKey{}, errors.Wrapf(err, "adding key %s", unsafeKey) } } else { if err := writer.PutRawMVCC(unsafeKey, unsafeValue); err != nil { - return roachpb.BulkOpSummary{}, MVCCKey{}, errors.Wrapf(err, "adding key %s", unsafeKey) + return kvpb.BulkOpSummary{}, MVCCKey{}, errors.Wrapf(err, "adding key %s", unsafeKey) } } rows.BulkOpSummary.DataSize = newSize @@ -6589,7 +6585,7 @@ func mvccExportToWriter( } } err := iter.TryGetIntentError() - return roachpb.BulkOpSummary{}, MVCCKey{}, err + return kvpb.BulkOpSummary{}, MVCCKey{}, err } // Flush any pending buffered range keys, truncated to the resume key (if @@ -6614,13 +6610,13 @@ func mvccExportToWriter( mvccValue, err = decodeExtendedMVCCValue(v.Value) } if err != nil { - return roachpb.BulkOpSummary{}, MVCCKey{}, errors.Wrapf(err, + return kvpb.BulkOpSummary{}, MVCCKey{}, errors.Wrapf(err, "decoding mvcc value %s", v.Value) } // Export only the inner roachpb.Value, not the MVCCValue header. rawValue := mvccValue.Value.RawBytes if err := writer.PutRawMVCCRangeKey(rangeKeys.AsRangeKey(v), rawValue); err != nil { - return roachpb.BulkOpSummary{}, MVCCKey{}, err + return kvpb.BulkOpSummary{}, MVCCKey{}, err } } rows.BulkOpSummary.DataSize += rangeKeysSize diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index e597c211b71d..e06a204eb040 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -21,6 +21,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/uncertainty" @@ -1106,7 +1107,7 @@ func cmdDelete(e *evalCtx) error { resolve, resolveStatus := e.getResolve() return e.withWriter("del", func(rw storage.ReadWriter) error { foundKey, err := storage.MVCCDelete(e.ctx, rw, e.ms, key, ts, localTs, txn) - if err == nil || errors.HasType(err, &roachpb.WriteTooOldError{}) { + if err == nil || errors.HasType(err, &kvpb.WriteTooOldError{}) { // We want to output foundKey even if a WriteTooOldError is returned, // since the error may be swallowed/deferred during evaluation. e.results.buf.Printf("del: %v: found key %v\n", key, foundKey) @@ -1195,7 +1196,7 @@ func cmdDeleteRangePredicate(e *evalCtx) error { if e.hasArg("maxBytes") { e.scanArg("maxBytes", &maxBytes) } - predicates := roachpb.DeleteRangePredicates{ + predicates := kvpb.DeleteRangePredicates{ StartTime: e.getTsWithName("startTime"), } rangeThreshold := 64 @@ -1384,7 +1385,7 @@ func cmdExport(e *evalCtx) error { sstFile := &storage.MemFile{} - var summary roachpb.BulkOpSummary + var summary kvpb.BulkOpSummary var resume storage.MVCCKey var fingerprint uint64 var hasRangeKeys bool @@ -2253,7 +2254,7 @@ func (e *evalCtx) withWriter(cmd string, fn func(_ storage.ReadWriter) error) er if batch != nil { // WriteTooOldError is sometimes expected to leave behind a provisional // value at a higher timestamp. We commit this for parity with the engine. - if err == nil || errors.HasType(err, &roachpb.WriteTooOldError{}) { + if err == nil || errors.HasType(err, &kvpb.WriteTooOldError{}) { if err := batch.Commit(true); err != nil { return err } diff --git a/pkg/storage/mvcc_incremental_iterator.go b/pkg/storage/mvcc_incremental_iterator.go index 1d9049d250c8..778c15781e24 100644 --- a/pkg/storage/mvcc_incremental_iterator.go +++ b/pkg/storage/mvcc_incremental_iterator.go @@ -13,6 +13,7 @@ package storage import ( "fmt" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util" @@ -434,7 +435,7 @@ func (i *MVCCIncrementalIterator) updateMeta() error { if i.startTime.Less(metaTimestamp) && metaTimestamp.LessEq(i.endTime) { switch i.intentPolicy { case MVCCIncrementalIterIntentPolicyError: - i.err = &roachpb.WriteIntentError{ + i.err = &kvpb.WriteIntentError{ Intents: []roachpb.Intent{ roachpb.MakeIntent(i.meta.Txn, i.iter.Key().Key), }, @@ -760,14 +761,14 @@ func (i *MVCCIncrementalIterator) NumCollectedIntents() int { return len(i.intents) } -// TryGetIntentError returns roachpb.WriteIntentError if intents were encountered +// TryGetIntentError returns kvpb.WriteIntentError if intents were encountered // during iteration and intent aggregation is enabled. Otherwise function -// returns nil. roachpb.WriteIntentError will contain all encountered intents. +// returns nil. kvpb.WriteIntentError will contain all encountered intents. func (i *MVCCIncrementalIterator) TryGetIntentError() error { if len(i.intents) == 0 { return nil } - return &roachpb.WriteIntentError{ + return &kvpb.WriteIntentError{ Intents: i.intents, } } diff --git a/pkg/storage/mvcc_incremental_iterator_test.go b/pkg/storage/mvcc_incremental_iterator_test.go index 4e47183e81c1..c4a9b7ef2522 100644 --- a/pkg/storage/mvcc_incremental_iterator_test.go +++ b/pkg/storage/mvcc_incremental_iterator_test.go @@ -20,6 +20,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" @@ -121,7 +122,7 @@ func assertExpectErr( } _, err := iter.Valid() - if intentErr := (*roachpb.WriteIntentError)(nil); errors.As(err, &intentErr) { + if intentErr := (*kvpb.WriteIntentError)(nil); errors.As(err, &intentErr) { if !expectedIntent.Key.Equal(intentErr.Intents[0].Key) { t.Fatalf("Expected intent key %v, but got %v", expectedIntent.Key, intentErr.Intents[0].Key) } @@ -162,7 +163,7 @@ func assertExpectErrs( t.Fatalf("Expected %d intents but found %d", len(expectedIntents), iter.NumCollectedIntents()) } err := iter.TryGetIntentError() - if intentErr := (*roachpb.WriteIntentError)(nil); errors.As(err, &intentErr) { + if intentErr := (*kvpb.WriteIntentError)(nil); errors.As(err, &intentErr) { for i := range expectedIntents { if !expectedIntents[i].Key.Equal(intentErr.Intents[i].Key) { t.Fatalf("%d intent key: got %v, expected %v", i, intentErr.Intents[i].Key, expectedIntents[i].Key) @@ -172,7 +173,7 @@ func assertExpectErrs( } } } else { - t.Fatalf("Expected roachpb.WriteIntentError, found %T", err) + t.Fatalf("Expected kvpb.WriteIntentError, found %T", err) } } @@ -200,7 +201,7 @@ func assertExportedErrs( }, sstFile) require.Error(t, err) - if intentErr := (*roachpb.WriteIntentError)(nil); errors.As(err, &intentErr) { + if intentErr := (*kvpb.WriteIntentError)(nil); errors.As(err, &intentErr) { for i := range expectedIntents { if !expectedIntents[i].Key.Equal(intentErr.Intents[i].Key) { t.Fatalf("%d intent key: got %v, expected %v", i, intentErr.Intents[i].Key, expectedIntents[i].Key) @@ -210,7 +211,7 @@ func assertExportedErrs( } } } else { - t.Fatalf("Expected roachpb.WriteIntentError, found %T", err) + t.Fatalf("Expected kvpb.WriteIntentError, found %T", err) } } @@ -787,7 +788,7 @@ func TestMVCCIncrementalIteratorIntentPolicy(t *testing.T) { kv2_2_2 := makeKVT(testKey2, testValue2, ts2) txn, intent2_2_2 := makeKVTxn(testKey2, ts2) - intentErr := &roachpb.WriteIntentError{Intents: []roachpb.Intent{intent2_2_2}} + intentErr := &kvpb.WriteIntentError{Intents: []roachpb.Intent{intent2_2_2}} e := NewDefaultInMemForTesting() defer e.Close() @@ -1401,7 +1402,7 @@ func TestMVCCIncrementalIteratorIntentStraddlesSStables(t *testing.T) { for it.SeekGE(MVCCKey{Key: keys.LocalMax}); ; it.Next() { ok, err := it.Valid() if err != nil { - if errors.HasType(err, (*roachpb.WriteIntentError)(nil)) { + if errors.HasType(err, (*kvpb.WriteIntentError)(nil)) { // This is the write intent error we were expecting. return } diff --git a/pkg/storage/mvcc_stats_test.go b/pkg/storage/mvcc_stats_test.go index 66da5df50f8e..0a4e1b509865 100644 --- a/pkg/storage/mvcc_stats_test.go +++ b/pkg/storage/mvcc_stats_test.go @@ -18,6 +18,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -995,7 +996,7 @@ func TestMVCCStatsDelDelGC(t *testing.T) { ctx, engine, aggMS, - []roachpb.GCRequest_GCKey{{ + []kvpb.GCRequest_GCKey{{ Key: key, Timestamp: ts2, }}, @@ -1192,7 +1193,7 @@ func TestMVCCStatsPutWaitDeleteGC(t *testing.T) { assertEq(t, engine, "after delete", aggMS, &expMS) - if err := MVCCGarbageCollect(ctx, engine, aggMS, []roachpb.GCRequest_GCKey{{ + if err := MVCCGarbageCollect(ctx, engine, aggMS, []kvpb.GCRequest_GCKey{{ Key: key, Timestamp: ts1, }}, ts2); err != nil { @@ -1665,7 +1666,7 @@ func TestMVCCStatsRandomized(t *testing.T) { mvccRangeDel := !s.isLocalKey && s.Txn == nil && s.rng.Intn(2) == 0 var mvccRangeDelKey, mvccRangeDelEndKey roachpb.Key - var predicates roachpb.DeleteRangePredicates + var predicates kvpb.DeleteRangePredicates if mvccRangeDel { mvccRangeDelKey = keys.LocalMax mvccRangeDelEndKey = roachpb.KeyMax @@ -1700,7 +1701,7 @@ func TestMVCCStatsRandomized(t *testing.T) { _, _, _, err = MVCCDeleteRange( ctx, s.batch, s.MSDelta, keySpan.Key, keySpan.EndKey, max, s.TS, hlc.ClockTimestamp{}, s.Txn, returnKeys, ) - } else if predicates == (roachpb.DeleteRangePredicates{}) { + } else if predicates == (kvpb.DeleteRangePredicates{}) { desc = fmt.Sprintf("mvccDeleteRangeUsingTombstone=%s", roachpb.Span{Key: mvccRangeDelKey, EndKey: mvccRangeDelEndKey}) const idempotent = false @@ -1798,7 +1799,7 @@ func TestMVCCStatsRandomized(t *testing.T) { ctx, s.batch, s.MSDelta, - []roachpb.GCRequest_GCKey{{ + []kvpb.GCRequest_GCKey{{ Key: s.key, Timestamp: gcTS, }}, diff --git a/pkg/storage/mvcc_test.go b/pkg/storage/mvcc_test.go index 821c4e4b87d1..2f202c82936a 100644 --- a/pkg/storage/mvcc_test.go +++ b/pkg/storage/mvcc_test.go @@ -25,6 +25,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" @@ -619,7 +620,7 @@ func TestMVCCScanWriteIntentError(t *testing.T) { t.Run(scan.name, func(t *testing.T) { res, err := MVCCScan(ctx, engine, testKey1, testKey6.Next(), hlc.Timestamp{WallTime: 1}, MVCCScanOptions{Inconsistent: !scan.consistent, Txn: scan.txn, MaxIntents: 2}) - var wiErr *roachpb.WriteIntentError + var wiErr *kvpb.WriteIntentError _ = errors.As(err, &wiErr) if (err == nil) != (wiErr == nil) { t.Errorf("unexpected error: %+v", err) @@ -746,7 +747,7 @@ func TestMVCCGetProtoInconsistent(t *testing.T) { Txn: txn1, }); err == nil { t.Error("expected an error getting inconsistently in txn") - } else if errors.HasType(err, (*roachpb.WriteIntentError)(nil)) { + } else if errors.HasType(err, (*kvpb.WriteIntentError)(nil)) { t.Error("expected non-WriteIntentError with inconsistent read in txn") } @@ -1697,7 +1698,7 @@ func TestMVCCDeleteRangeOldTimestamp(t *testing.T) { require.Nil(t, resume) require.Equal(t, int64(0), keyCount) require.NotNil(t, err) - require.IsType(t, (*roachpb.WriteTooOldError)(nil), err) + require.IsType(t, (*kvpb.WriteTooOldError)(nil), err) // Delete at the same time as the tombstone. Should return a WriteTooOld error. b = engine.NewBatch() @@ -1708,7 +1709,7 @@ func TestMVCCDeleteRangeOldTimestamp(t *testing.T) { require.Nil(t, resume) require.Equal(t, int64(0), keyCount) require.NotNil(t, err) - require.IsType(t, (*roachpb.WriteTooOldError)(nil), err) + require.IsType(t, (*kvpb.WriteTooOldError)(nil), err) // Delete at a time after the tombstone. Should succeed and should not // include the tombstone in the returned keys. @@ -2219,7 +2220,7 @@ func TestMVCCInitPut(t *testing.T) { // Reinserting the value fails if we fail on tombstones. err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 4}, hlc.ClockTimestamp{}, value1, true, nil) - if e := (*roachpb.ConditionFailedError)(nil); errors.As(err, &e) { + if e := (*kvpb.ConditionFailedError)(nil); errors.As(err, &e) { if !bytes.Equal(e.ActualValue.RawBytes, nil) { t.Fatalf("the value %s in get result is not a tombstone", e.ActualValue.RawBytes) } @@ -2237,7 +2238,7 @@ func TestMVCCInitPut(t *testing.T) { // A repeat of the command with a different value will fail. err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 6}, hlc.ClockTimestamp{}, value2, false, nil) - if e := (*roachpb.ConditionFailedError)(nil); errors.As(err, &e) { + if e := (*kvpb.ConditionFailedError)(nil); errors.As(err, &e) { if !bytes.Equal(e.ActualValue.RawBytes, value1.RawBytes) { t.Fatalf("the value %s in get result does not match the value %s in request", e.ActualValue.RawBytes, value1.RawBytes) @@ -2325,7 +2326,7 @@ func TestMVCCInitPutWithTxn(t *testing.T) { // Write value4 with an old timestamp without txn...should get an error. err = MVCCInitPut(ctx, engine, nil, testKey1, clock.Now(), hlc.ClockTimestamp{}, value4, false, nil) - if e := (*roachpb.ConditionFailedError)(nil); errors.As(err, &e) { + if e := (*kvpb.ConditionFailedError)(nil); errors.As(err, &e) { if !bytes.Equal(e.ActualValue.RawBytes, value2.RawBytes) { t.Fatalf("the value %s in get result does not match the value %s in request", e.ActualValue.RawBytes, value2.RawBytes) @@ -2653,7 +2654,7 @@ func TestMVCCResolveNewerIntent(t *testing.T) { // Now, put down an intent which should return a write too old error // (but will still write the intent at tx1Commit.Timestamp+1. err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1) - if !errors.HasType(err, (*roachpb.WriteTooOldError)(nil)) { + if !errors.HasType(err, (*kvpb.WriteTooOldError)(nil)) { t.Fatalf("expected write too old error; got %s", err) } @@ -2714,7 +2715,7 @@ func TestMVCCResolveIntentTxnTimestampMismatch(t *testing.T) { {hlc.MaxTimestamp, true}, } { _, err := MVCCGet(ctx, engine, testKey1, test.Timestamp, MVCCGetOptions{}) - if errors.HasType(err, (*roachpb.WriteIntentError)(nil)) != test.found { + if errors.HasType(err, (*kvpb.WriteIntentError)(nil)) != test.found { t.Fatalf("%d: expected write intent error: %t, got %v", i, test.found, err) } } @@ -2747,7 +2748,7 @@ func TestMVCCConditionalPutOldTimestamp(t *testing.T) { if err == nil { t.Errorf("unexpected success on conditional put") } - if !errors.HasType(err, (*roachpb.ConditionFailedError)(nil)) { + if !errors.HasType(err, (*kvpb.ConditionFailedError)(nil)) { t.Errorf("unexpected error on conditional put: %+v", err) } @@ -2757,7 +2758,7 @@ func TestMVCCConditionalPutOldTimestamp(t *testing.T) { if err == nil { t.Errorf("unexpected success on conditional put") } - if !errors.HasType(err, (*roachpb.WriteTooOldError)(nil)) { + if !errors.HasType(err, (*kvpb.WriteTooOldError)(nil)) { t.Errorf("unexpected error on conditional put: %+v", err) } // Verify new value was actually written at (3, 1). @@ -2792,7 +2793,7 @@ func TestMVCCMultiplePutOldTimestamp(t *testing.T) { txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) txn.Sequence++ err = MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn) - if !errors.HasType(err, (*roachpb.WriteTooOldError)(nil)) { + if !errors.HasType(err, (*kvpb.WriteTooOldError)(nil)) { t.Errorf("expected WriteTooOldError on Put; got %v", err) } // Verify new value was actually written at (3, 1). @@ -2868,7 +2869,7 @@ func TestMVCCPutOldOrigTimestampNewCommitTimestamp(t *testing.T) { // Verify that the Put returned a WriteTooOld with the ActualTime set to the // transactions provisional commit timestamp. expTS := txn.WriteTimestamp - if wtoErr := (*roachpb.WriteTooOldError)(nil); !errors.As(err, &wtoErr) || wtoErr.ActualTimestamp != expTS { + if wtoErr := (*kvpb.WriteTooOldError)(nil); !errors.As(err, &wtoErr) || wtoErr.ActualTimestamp != expTS { t.Fatalf("expected WriteTooOldError with actual time = %s; got %s", expTS, wtoErr) } @@ -3016,7 +3017,7 @@ func TestMVCCWriteWithDiffTimestampsAndEpochs(t *testing.T) { // Now try writing an earlier value without a txn--should get WriteTooOldError. err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value4, nil) - if wtoErr := (*roachpb.WriteTooOldError)(nil); !errors.As(err, &wtoErr) { + if wtoErr := (*kvpb.WriteTooOldError)(nil); !errors.As(err, &wtoErr) { t.Fatal("unexpected success") } else if wtoErr.ActualTimestamp != expTS { t.Fatalf("expected write too old error with actual ts %s; got %s", expTS, wtoErr.ActualTimestamp) @@ -3030,7 +3031,7 @@ func TestMVCCWriteWithDiffTimestampsAndEpochs(t *testing.T) { // Now write an intent with exactly the same timestamp--ties also get WriteTooOldError. err = MVCCPut(ctx, engine, nil, testKey1, txn2.ReadTimestamp, hlc.ClockTimestamp{}, value5, txn2) intentTS := expTS.Next() - if wtoErr := (*roachpb.WriteTooOldError)(nil); !errors.As(err, &wtoErr) { + if wtoErr := (*kvpb.WriteTooOldError)(nil); !errors.As(err, &wtoErr) { t.Fatal("unexpected success") } else if wtoErr.ActualTimestamp != intentTS { t.Fatalf("expected write too old error with actual ts %s; got %s", intentTS, wtoErr.ActualTimestamp) @@ -3103,7 +3104,7 @@ func TestMVCCGetWithDiffEpochs(t *testing.T) { if test.expErr { if err == nil { t.Errorf("test %d: unexpected success", i) - } else if !errors.HasType(err, (*roachpb.WriteIntentError)(nil)) { + } else if !errors.HasType(err, (*kvpb.WriteIntentError)(nil)) { t.Errorf("test %d: expected write intent error; got %v", i, err) } } else if err != nil || valueRes.Value == nil || !bytes.Equal(test.expValue.RawBytes, valueRes.Value.RawBytes) { @@ -3142,7 +3143,7 @@ func TestMVCCGetWithDiffEpochsAndTimestamps(t *testing.T) { txn1ts.WriteTimestamp = hlc.Timestamp{WallTime: 4} // Expected to hit WriteTooOld error but to still lay down intent. err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn1ts) - if wtoErr := (*roachpb.WriteTooOldError)(nil); !errors.As(err, &wtoErr) { + if wtoErr := (*kvpb.WriteTooOldError)(nil); !errors.As(err, &wtoErr) { t.Fatalf("unexpectedly not WriteTooOld: %+v", err) } else if expTS, actTS := txn1ts.WriteTimestamp, wtoErr.ActualTimestamp; expTS != actTS { t.Fatalf("expected write too old error with actual ts %s; got %s", expTS, actTS) @@ -4600,7 +4601,7 @@ func TestMVCCGarbageCollect(t *testing.T) { } gcTime := ts5 - gcKeys := []roachpb.GCRequest_GCKey{ + gcKeys := []kvpb.GCRequest_GCKey{ {Key: roachpb.Key("a"), Timestamp: ts1}, {Key: roachpb.Key("a-del"), Timestamp: ts2}, {Key: roachpb.Key("b"), Timestamp: ts1}, @@ -4711,7 +4712,7 @@ func TestMVCCGarbageCollectNonDeleted(t *testing.T) { t.Fatal(err) } } - keys := []roachpb.GCRequest_GCKey{ + keys := []kvpb.GCRequest_GCKey{ {Key: test.key, Timestamp: ts2}, } err := MVCCGarbageCollect(ctx, engine, nil, keys, ts2) @@ -4748,7 +4749,7 @@ func TestMVCCGarbageCollectIntent(t *testing.T) { if _, err := MVCCDelete(ctx, engine, nil, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { t.Fatal(err) } - keys := []roachpb.GCRequest_GCKey{ + keys := []kvpb.GCRequest_GCKey{ {Key: key, Timestamp: ts2}, } if err := MVCCGarbageCollect(ctx, engine, nil, keys, ts2); err == nil { @@ -4770,7 +4771,7 @@ func TestMVCCGarbageCollectPanicsWithMixOfLocalAndGlobalKeys(t *testing.T) { require.Panics(t, func() { ts := hlc.Timestamp{WallTime: 1e9} k := roachpb.Key("a") - keys := []roachpb.GCRequest_GCKey{ + keys := []kvpb.GCRequest_GCKey{ {Key: k, Timestamp: ts}, {Key: keys.RangeDescriptorKey(roachpb.RKey(k))}, } @@ -4842,10 +4843,10 @@ func TestMVCCGarbageCollectUsesSeekLTAppropriately(t *testing.T) { } } - var keys []roachpb.GCRequest_GCKey + var keys []kvpb.GCRequest_GCKey var expectedSeekLTs int for _, key := range tc.keys { - keys = append(keys, roachpb.GCRequest_GCKey{ + keys = append(keys, kvpb.GCRequest_GCKey{ Key: roachpb.Key(key.key), Timestamp: toHLC(key.gcTimestamp), }) @@ -5063,7 +5064,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { name string // Note that range test data should be in ascending order (valid writes). before rangeTestData - request []roachpb.GCRequest_GCRangeKey + request []kvpb.GCRequest_GCRangeKey // Note that expectations should be in timestamp descending order // (forward iteration). after []MVCCRangeKey @@ -5077,7 +5078,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { before: rangeTestData{ rng(keyA, keyD, ts3), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyA, EndKey: keyD, Timestamp: ts3}, }, after: []MVCCRangeKey{}, @@ -5088,7 +5089,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { rng(keyA, keyD, ts2), rng(keyB, keyC, ts4), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyA, EndKey: keyD, Timestamp: ts2}, }, after: []MVCCRangeKey{ @@ -5101,7 +5102,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { rng(keyA, keyB, ts2), rng(keyC, keyD, ts2), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyA, EndKey: keyD, Timestamp: ts2}, }, after: []MVCCRangeKey{}, @@ -5112,7 +5113,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { rng(keyA, keyB, ts2), rng(keyC, keyD, ts2), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyB, EndKey: keyC, Timestamp: ts2}, }, after: []MVCCRangeKey{ @@ -5125,7 +5126,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { before: rangeTestData{ rng(keyB, keyC, ts2), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyA, EndKey: keyD, Timestamp: ts2}, }, after: []MVCCRangeKey{}, @@ -5135,7 +5136,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { before: rangeTestData{ rng(keyB, keyD, ts2), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyA, EndKey: keyC, Timestamp: ts2}, }, after: []MVCCRangeKey{ @@ -5147,7 +5148,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { before: rangeTestData{ rng(keyA, keyC, ts2), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyB, EndKey: keyD, Timestamp: ts2}, }, after: []MVCCRangeKey{ @@ -5159,7 +5160,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { before: rangeTestData{ rng(keyA, keyD, ts2), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyB, EndKey: keyC, Timestamp: ts2}, }, after: []MVCCRangeKey{ @@ -5172,7 +5173,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { before: rangeTestData{ rng(keyA, keyD, ts2), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyA, EndKey: keyB, Timestamp: ts2}, }, after: []MVCCRangeKey{ @@ -5184,7 +5185,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { before: rangeTestData{ rng(keyA, keyB, ts2), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyA, EndKey: keyD, Timestamp: ts2}, }, after: []MVCCRangeKey{}, @@ -5194,7 +5195,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { before: rangeTestData{ rng(keyA, keyD, ts2), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyB, EndKey: keyD, Timestamp: ts2}, }, after: []MVCCRangeKey{ @@ -5206,7 +5207,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { before: rangeTestData{ rng(keyB, keyD, ts2), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyA, EndKey: keyD, Timestamp: ts2}, }, after: []MVCCRangeKey{}, @@ -5217,7 +5218,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { rng(keyB, keyD, ts2), pt(keyA, ts4), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyC, EndKey: keyD, Timestamp: ts2}, }, after: []MVCCRangeKey{ @@ -5230,7 +5231,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { rng(keyA, keyD, ts2), pt(keyA, ts4), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyB, EndKey: keyD, Timestamp: ts2}, }, after: []MVCCRangeKey{ @@ -5243,7 +5244,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { rng(keyA, keyD, ts2), pt(keyB, ts4), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyC, EndKey: keyD, Timestamp: ts2}, }, after: []MVCCRangeKey{ @@ -5256,7 +5257,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { rng(keyA, keyD, ts2), pt(keyB, ts4), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyB, EndKey: keyD, Timestamp: ts2}, }, after: []MVCCRangeKey{ @@ -5269,7 +5270,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { rng(keyC, keyD, ts2), pt(keyA, ts4), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyB, EndKey: keyD, Timestamp: ts2}, }, after: []MVCCRangeKey{}, @@ -5280,7 +5281,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { rng(keyB, keyD, ts2), pt(keyA, ts4), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyA, EndKey: keyD, Timestamp: ts2}, }, after: []MVCCRangeKey{}, @@ -5291,7 +5292,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { rng(keyC, keyD, ts2), pt(keyB, ts4), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyA, EndKey: keyD, Timestamp: ts2}, }, after: []MVCCRangeKey{}, @@ -5302,7 +5303,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { rng(keyB, keyD, ts2), pt(keyB, ts4), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyA, EndKey: keyD, Timestamp: ts2}, }, after: []MVCCRangeKey{}, @@ -5313,7 +5314,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { rng(keyA, keyD, ts2), txn(pt(keyA, ts4)), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyA, EndKey: keyD, Timestamp: ts2}, }, after: []MVCCRangeKey{}, @@ -5324,7 +5325,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { rng(keyB, keyC, ts2), rng(keyA, keyD, ts4), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyA, EndKey: keyD, Timestamp: ts4}, }, after: []MVCCRangeKey{}, @@ -5335,7 +5336,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { pt(keyA, ts2), rng(keyB, keyC, ts3), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyB, EndKey: keyC, Timestamp: ts3}, }, after: []MVCCRangeKey{}, @@ -5346,7 +5347,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { pt(keyC, ts2), rng(keyB, keyC, ts3), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyB, EndKey: keyC, Timestamp: ts3}, }, after: []MVCCRangeKey{}, @@ -5357,7 +5358,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { rng(keyA, keyD, ts1), rng(keyA, keyD, ts3), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyB, EndKey: keyC, Timestamp: ts1}, }, after: []MVCCRangeKey{ @@ -5374,7 +5375,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { rng(keyB, keyC, ts1), rng(keyA, keyD, ts3), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyB, EndKey: keyC, Timestamp: ts1}, }, after: []MVCCRangeKey{ @@ -5387,7 +5388,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { rng(keyB, keyC, ts1), rng(keyA, keyC, ts3), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyB, EndKey: keyC, Timestamp: ts1}, }, after: []MVCCRangeKey{ @@ -5400,7 +5401,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { rng(keyA, keyB, ts1), rng(keyA, keyD, ts3), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyA, EndKey: keyB, Timestamp: ts1}, }, after: []MVCCRangeKey{ @@ -5415,7 +5416,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { rng(keyA, keyF, ts3), rng(keyA, keyF, ts4), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyB, EndKey: keyC, Timestamp: ts1}, {StartKey: keyD, EndKey: keyE, Timestamp: ts2}, }, @@ -5431,7 +5432,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { rng(keyB, keyD, ts2), rng(keyA, keyE, ts3), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyB, EndKey: keyC, Timestamp: ts2}, {StartKey: keyC, EndKey: keyD, Timestamp: ts2}, }, @@ -5446,7 +5447,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { // Tombstone spanning multiple ranges. rng(keyA, keyD, ts4), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyB, EndKey: keyC, Timestamp: ts1}, }, after: []MVCCRangeKey{ @@ -5511,7 +5512,7 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { } func rangesFromRequests( - rangeStart, rangeEnd roachpb.Key, rangeKeys []roachpb.GCRequest_GCRangeKey, + rangeStart, rangeEnd roachpb.Key, rangeKeys []kvpb.GCRequest_GCRangeKey, ) []CollectableGCRangeKey { collectableKeys := make([]CollectableGCRangeKey, len(rangeKeys)) for i, rk := range rangeKeys { @@ -5566,7 +5567,7 @@ func TestMVCCGarbageCollectRangesFailures(t *testing.T) { testData := []struct { name string before rangeTestData - request []roachpb.GCRequest_GCRangeKey + request []kvpb.GCRequest_GCRangeKey error string }{ { @@ -5574,7 +5575,7 @@ func TestMVCCGarbageCollectRangesFailures(t *testing.T) { before: rangeTestData{ rng(keyA, keyD, ts3), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyA, EndKey: keyC, Timestamp: ts3}, {StartKey: keyB, EndKey: keyD, Timestamp: ts3}, }, @@ -5586,7 +5587,7 @@ func TestMVCCGarbageCollectRangesFailures(t *testing.T) { pt(keyB, ts2), rng(keyA, keyD, ts3), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyA, EndKey: keyD, Timestamp: ts3}, }, error: "attempt to delete range tombstone .* hiding key at .*", @@ -5602,7 +5603,7 @@ func TestMVCCGarbageCollectRangesFailures(t *testing.T) { rng(keyA, keyD, ts2), txn(pt(keyB, ts3)), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyA, EndKey: keyD, Timestamp: ts4}, }, error: "attempt to delete range tombstone .* hiding key at .*", @@ -5619,7 +5620,7 @@ func TestMVCCGarbageCollectRangesFailures(t *testing.T) { pt(keyB, ts7), pt(keyB, ts8), }, - request: []roachpb.GCRequest_GCRangeKey{ + request: []kvpb.GCRequest_GCRangeKey{ {StartKey: keyA, EndKey: keyD, Timestamp: ts2}, }, error: "attempt to delete range tombstone .* hiding key at .*", @@ -5671,8 +5672,8 @@ func TestMVCCGarbageCollectClearRange(t *testing.T) { tsGC := mkTs(5) tsMax := mkTs(9) - mkGCReq := func(start roachpb.Key, end roachpb.Key) roachpb.GCRequest_GCClearRange { - return roachpb.GCRequest_GCClearRange{ + mkGCReq := func(start roachpb.Key, end roachpb.Key) kvpb.GCRequest_GCClearRange { + return kvpb.GCRequest_GCClearRange{ StartKey: start, EndKey: end, } @@ -5734,8 +5735,8 @@ func TestMVCCGarbageCollectClearRangeInlinedValue(t *testing.T) { tsGC := mkTs(5) - mkGCReq := func(start roachpb.Key, end roachpb.Key) roachpb.GCRequest_GCClearRange { - return roachpb.GCRequest_GCClearRange{ + mkGCReq := func(start roachpb.Key, end roachpb.Key) kvpb.GCRequest_GCClearRange { + return kvpb.GCRequest_GCClearRange{ StartKey: start, EndKey: end, } @@ -6303,7 +6304,7 @@ func TestMVCCExportToSSTFailureIntentBatching(t *testing.T) { require.NoError(t, err) } else { require.Error(t, err) - e := (*roachpb.WriteIntentError)(nil) + e := (*kvpb.WriteIntentError)(nil) if !errors.As(err, &e) { require.Fail(t, "Expected WriteIntentFailure, got %T", err) } @@ -6440,7 +6441,7 @@ func TestMVCCExportFingerprint(t *testing.T) { ctx := context.Background() st := cluster.MakeTestingClusterSettings() - fingerprint := func(opts MVCCExportOptions, engine Engine) (uint64, []byte, roachpb.BulkOpSummary, MVCCKey) { + fingerprint := func(opts MVCCExportOptions, engine Engine) (uint64, []byte, kvpb.BulkOpSummary, MVCCKey) { dest := &MemFile{} var err error res, resumeKey, fingerprint, hasRangeKeys, err := MVCCExportFingerprint( diff --git a/pkg/storage/pebble_mvcc_scanner.go b/pkg/storage/pebble_mvcc_scanner.go index a304dc44afcc..59c70a11cf16 100644 --- a/pkg/storage/pebble_mvcc_scanner.go +++ b/pkg/storage/pebble_mvcc_scanner.go @@ -18,6 +18,7 @@ import ( "sync" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/uncertainty" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -414,7 +415,7 @@ type pebbleMVCCScanner struct { maxIntents int64 // Resume fields describe the resume span to return. resumeReason must be set // to a non-zero value to return a resume span, the others are optional. - resumeReason roachpb.ResumeReason + resumeReason kvpb.ResumeReason resumeKey roachpb.Key // if unset, falls back to p.advanceKey() resumeNextBytes int64 // set when targetBytes is exceeded // Transaction epoch and sequence number. @@ -613,7 +614,7 @@ func (p *pebbleMVCCScanner) advance() bool { // resume span, resume reason, and for targetBytes the size of the next result. func (p *pebbleMVCCScanner) scan( ctx context.Context, -) (*roachpb.Span, roachpb.ResumeReason, int64, error) { +) (*roachpb.Span, kvpb.ResumeReason, int64, error) { if p.wholeRows && !p.results.(*pebbleResults).lastOffsetsEnabled { return nil, 0, 0, errors.AssertionFailedf("cannot use wholeRows without trackLastOffsets") } @@ -632,7 +633,7 @@ func (p *pebbleMVCCScanner) scan( // afterScan checks whether some limit was exceeded during the scan, and if so, // it returns a resume span, resume reason, and for targetBytes the size of the // next result. -func (p *pebbleMVCCScanner) afterScan() (*roachpb.Span, roachpb.ResumeReason, int64, error) { +func (p *pebbleMVCCScanner) afterScan() (*roachpb.Span, kvpb.ResumeReason, int64, error) { p.maybeFailOnMoreRecent() if p.err != nil { @@ -735,7 +736,7 @@ func (p *pebbleMVCCScanner) maybeFailOnMoreRecent() { } // The txn can't write at the existing timestamp, so we provide the error // with the timestamp immediately after it. - p.err = roachpb.NewWriteTooOldError(p.ts, p.mostRecentTS.Next(), p.mostRecentKey) + p.err = kvpb.NewWriteTooOldError(p.ts, p.mostRecentTS.Next(), p.mostRecentKey) p.results.clear() p.intents.Reset() } @@ -745,7 +746,7 @@ func (p *pebbleMVCCScanner) maybeFailOnMoreRecent() { func (p *pebbleMVCCScanner) uncertaintyError( valueTs hlc.Timestamp, localTs hlc.ClockTimestamp, ) (ok bool) { - p.err = roachpb.NewReadWithinUncertaintyIntervalError( + p.err = kvpb.NewReadWithinUncertaintyIntervalError( p.ts, p.uncertainty.LocalLimit, p.txn, valueTs, localTs) p.results.clear() p.intents.Reset() @@ -971,7 +972,7 @@ func (p *pebbleMVCCScanner) getOne(ctx context.Context) (ok, added bool) { } // Limit number of intents returned in write intent error. if p.maxIntents > 0 && int64(p.intents.Count()) >= p.maxIntents { - p.resumeReason = roachpb.RESUME_INTENT_LIMIT + p.resumeReason = kvpb.RESUME_INTENT_LIMIT return false, false } return true /* ok */, false @@ -1210,11 +1211,11 @@ func (p *pebbleMVCCScanner) add( // Check if adding the key would exceed a limit. if p.targetBytes > 0 && numBytes+numBytesInc > p.targetBytes { - p.resumeReason = roachpb.RESUME_BYTE_LIMIT + p.resumeReason = kvpb.RESUME_BYTE_LIMIT p.resumeNextBytes = numBytesInc } else if p.maxKeys > 0 && numKeys >= p.maxKeys { - p.resumeReason = roachpb.RESUME_KEY_LIMIT + p.resumeReason = kvpb.RESUME_KEY_LIMIT } var mustPutKey bool @@ -1277,7 +1278,7 @@ func (p *pebbleMVCCScanner) add( // the row may have been omitted (if they are all NULL values) -- to find // out, we must continue scanning to the next key and handle it above. if !p.wholeRows || p.results.lastRowHasFinalColumnFamily(p.reverse) { - p.resumeReason = roachpb.RESUME_KEY_LIMIT + p.resumeReason = kvpb.RESUME_KEY_LIMIT return false /* ok */, true /* added */ } } diff --git a/pkg/storage/row_counter.go b/pkg/storage/row_counter.go index 814c337a6259..7b6c5c2a0b40 100644 --- a/pkg/storage/row_counter.go +++ b/pkg/storage/row_counter.go @@ -14,6 +14,7 @@ import ( "bytes" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" ) @@ -21,7 +22,7 @@ import ( // that is is shown via `Count`. Note: the `DataSize` field of the BulkOpSummary // is *not* populated by this and should be set separately. type RowCounter struct { - roachpb.BulkOpSummary + kvpb.BulkOpSummary prev roachpb.Key } @@ -63,7 +64,7 @@ func (r *RowCounter) Count(key roachpb.Key) error { if r.EntryCounts == nil { r.EntryCounts = make(map[uint64]int64) } - r.EntryCounts[roachpb.BulkOpSummaryID(uint64(tableID), uint64(indexID))]++ + r.EntryCounts[kvpb.BulkOpSummaryID(uint64(tableID), uint64(indexID))]++ if indexID == 1 { r.DeprecatedRows++ diff --git a/pkg/storage/sst.go b/pkg/storage/sst.go index d2ba10c94eab..b0fbf1103880 100644 --- a/pkg/storage/sst.go +++ b/pkg/storage/sst.go @@ -15,6 +15,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" @@ -265,7 +266,7 @@ func CheckSSTConflicts( // of scans. intents = append(intents, roachpb.MakeIntent(mvccMeta.Txn, extIter.Key().Key)) if int64(len(intents)) >= maxIntents { - return &roachpb.WriteIntentError{Intents: intents} + return &kvpb.WriteIntentError{Intents: intents} } return nil } @@ -336,7 +337,7 @@ func CheckSSTConflicts( // do if AddSSTable had SSTTimestampToRequestTimestamp set, but AddSSTable // cannot be used in transactions so we don't need to check. if sstKey.Timestamp.LessEq(extKey.Timestamp) { - return roachpb.NewWriteTooOldError( + return kvpb.NewWriteTooOldError( sstKey.Timestamp, extKey.Timestamp.Next(), sstKey.Key) } @@ -460,7 +461,7 @@ func CheckSSTConflicts( if sstTombstone.Timestamp.Less(extRangeKeys.Versions[0].Timestamp) { // Conflict. We can't slide an MVCC range tombstone below an // existing MVCC range tombstone in the engine. - return enginepb.MVCCStats{}, roachpb.NewWriteTooOldError( + return enginepb.MVCCStats{}, kvpb.NewWriteTooOldError( sstTombstone.Timestamp, extRangeKeys.Versions[0].Timestamp.Next(), sstRangeKeys.Bounds.Key) } if !extRangeKeys.Versions[0].Timestamp.Less(sstTombstone.Timestamp) { @@ -485,7 +486,7 @@ func CheckSSTConflicts( if extRangeKeys.Covers(sstKey) { // A range tombstone in the engine deletes this SST key. Return // a WriteTooOldError. - return enginepb.MVCCStats{}, roachpb.NewWriteTooOldError( + return enginepb.MVCCStats{}, kvpb.NewWriteTooOldError( sstKey.Timestamp, extRangeKeys.Versions[0].Timestamp.Next(), sstKey.Key) } } @@ -511,7 +512,7 @@ func CheckSSTConflicts( } intents = append(intents, roachpb.MakeIntent(mvccMeta.Txn, extIter.Key().Key)) if int64(len(intents)) >= maxIntents { - return statsDiff, &roachpb.WriteIntentError{Intents: intents} + return statsDiff, &kvpb.WriteIntentError{Intents: intents} } extIter.Next() continue @@ -519,7 +520,7 @@ func CheckSSTConflicts( if sstBottomTombstone.Timestamp.LessEq(extKey.Timestamp) { // Conflict. - return enginepb.MVCCStats{}, roachpb.NewWriteTooOldError( + return enginepb.MVCCStats{}, kvpb.NewWriteTooOldError( sstBottomTombstone.Timestamp, extKey.Timestamp.Next(), sstRangeKeys.Bounds.Key) } if sstRangeKeys.Covers(extKey) { @@ -1139,7 +1140,7 @@ func CheckSSTConflicts( return enginepb.MVCCStats{}, sstErr } if len(intents) > 0 { - return enginepb.MVCCStats{}, &roachpb.WriteIntentError{Intents: intents} + return enginepb.MVCCStats{}, &kvpb.WriteIntentError{Intents: intents} } return statsDiff, nil diff --git a/pkg/storage/sst_test.go b/pkg/storage/sst_test.go index cbb440eaae5a..541afbbcc3fd 100644 --- a/pkg/storage/sst_test.go +++ b/pkg/storage/sst_test.go @@ -17,6 +17,7 @@ import ( "math/rand" "testing" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" @@ -92,7 +93,7 @@ func TestCheckSSTConflictsMaxIntents(t *testing.T) { _, err := CheckSSTConflicts(ctx, sstFile.Bytes(), engine, startKey, endKey, startKey.Key, endKey.Key.Next(), false /*disallowShadowing*/, hlc.Timestamp{} /*disallowShadowingBelow*/, hlc.Timestamp{} /* sstReqTS */, tc.maxIntents, usePrefixSeek) require.Error(t, err) - writeIntentErr := &roachpb.WriteIntentError{} + writeIntentErr := &kvpb.WriteIntentError{} require.ErrorAs(t, err, &writeIntentErr) actual := []string{} diff --git a/pkg/storage/testdata/mvcc_histories/conditional_put b/pkg/storage/testdata/mvcc_histories/conditional_put index 1b01884e32c5..fc3c9b076a7a 100644 --- a/pkg/storage/testdata/mvcc_histories/conditional_put +++ b/pkg/storage/testdata/mvcc_histories/conditional_put @@ -3,7 +3,7 @@ cput k=k v=v cond=v2 ts=123 ---- >> at end: -error: (*roachpb.ConditionFailedError:) unexpected value: +error: (*kvpb.ConditionFailedError:) unexpected value: # Verify the difference between missing value and empty value. @@ -12,7 +12,7 @@ cput k=k v=v cond= ts=123,1 ---- >> at end: -error: (*roachpb.ConditionFailedError:) unexpected value: +error: (*kvpb.ConditionFailedError:) unexpected value: # Do a conditional put with expectation that the value is completely missing; will succeed. @@ -32,7 +32,7 @@ cput k=k v=v ts=123,3 ---- >> at end: data: "k"/123.000000000,2 -> /BYTES/v -error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003v" timestamp: +error: (*kvpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003v" timestamp: # Conditional put expecting wrong value2, will fail. @@ -41,7 +41,7 @@ cput k=k v=v cond=v2 ts=123,4 ---- >> at end: data: "k"/123.000000000,2 -> /BYTES/v -error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003v" timestamp: +error: (*kvpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003v" timestamp: # Move to an empty value. Will succeed. @@ -64,7 +64,7 @@ cput k=k2 v=v2 cond=v ts=123,6 >> at end: data: "k"/123.000000000,5 -> /BYTES/ data: "k"/123.000000000,2 -> /BYTES/v -error: (*roachpb.ConditionFailedError:) unexpected value: +error: (*kvpb.ConditionFailedError:) unexpected value: # Move key2 (which does not yet exist) to from value1 to value2, but # allowing for it not existing. @@ -89,7 +89,7 @@ cput k=k2 v= cond=v allow_missing ts=123,8 data: "k"/123.000000000,5 -> /BYTES/ data: "k"/123.000000000,2 -> /BYTES/v data: "k2"/123.000000000,7 -> /BYTES/v2 -error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003v2" timestamp: +error: (*kvpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003v2" timestamp: # Try to move key2 (which has value2) from value2 to empty. Expect success. diff --git a/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_enable_separated b/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_enable_separated index 4b7bebf88ef5..179d69362d5e 100644 --- a/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_enable_separated +++ b/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_enable_separated @@ -74,7 +74,7 @@ cput k=k v=v4 cond=v3 ts=123 >> at end: data: "k"/124.000000000,1 -> /BYTES/v4 data: "k"/124.000000000,0 -> {localTs=123.000000000,0}/BYTES/v3 -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k" at timestamp 123.000000000,0 too old; wrote at 124.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k" at timestamp 123.000000000,0 too old; wrote at 124.000000000,1 # Reset for next test diff --git a/pkg/storage/testdata/mvcc_histories/conditional_put_write_too_old b/pkg/storage/testdata/mvcc_histories/conditional_put_write_too_old index 97bac0fa6dea..ca164c3e26a9 100644 --- a/pkg/storage/testdata/mvcc_histories/conditional_put_write_too_old +++ b/pkg/storage/testdata/mvcc_histories/conditional_put_write_too_old @@ -16,7 +16,7 @@ cput ts=1 k=k v=v2 ---- >> at end: data: "k"/10.000000000,0 -> /BYTES/v1 -error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003v1" timestamp: +error: (*kvpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003v1" timestamp: # Now do a non-transactional put @t=1 with expectation of value1; will "succeed" @t=10,1 with WriteTooOld. run error @@ -25,7 +25,7 @@ cput ts=1 k=k v=v2 cond=v1 >> at end: data: "k"/10.000000000,1 -> /BYTES/v2 data: "k"/10.000000000,0 -> /BYTES/v1 -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k" at timestamp 1.000000000,0 too old; wrote at 10.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k" at timestamp 1.000000000,0 too old; wrote at 10.000000000,1 # Try a transactional put @t=1 with expectation of value2; should fail. run error @@ -37,7 +37,7 @@ with t=a txn: "a" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=1.000000000,0 wto=false gul=0,0 data: "k"/10.000000000,1 -> /BYTES/v2 data: "k"/10.000000000,0 -> /BYTES/v1 -error: (*roachpb.ConditionFailedError:) unexpected value: +error: (*kvpb.ConditionFailedError:) unexpected value: # Now do a transactional put @t=1 with expectation of nil; will "succeed" @t=10,2 with WriteTooOld. run error @@ -49,4 +49,4 @@ meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000, data: "k"/10.000000000,2 -> /BYTES/v3 data: "k"/10.000000000,1 -> /BYTES/v2 data: "k"/10.000000000,0 -> /BYTES/v1 -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k" at timestamp 1.000000000,0 too old; wrote at 10.000000000,2 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k" at timestamp 1.000000000,0 too old; wrote at 10.000000000,2 diff --git a/pkg/storage/testdata/mvcc_histories/delete_range b/pkg/storage/testdata/mvcc_histories/delete_range index 5687f627ea46..02bcfb17fbb7 100644 --- a/pkg/storage/testdata/mvcc_histories/delete_range +++ b/pkg/storage/testdata/mvcc_histories/delete_range @@ -137,7 +137,7 @@ data: "c/123"/47.000000000,0 -> / data: "c/123"/44.000000000,0 -> /BYTES/abc data: "d"/44.000000000,0 -> /BYTES/abc data: "d/123"/44.000000000,0 -> /BYTES/abc -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "c" at timestamp 46.000000000,0 too old; wrote at 47.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "c" at timestamp 46.000000000,0 too old; wrote at 47.000000000,1 run ok txn_remove t=A diff --git a/pkg/storage/testdata/mvcc_histories/delete_range_predicate b/pkg/storage/testdata/mvcc_histories/delete_range_predicate index fab880c3ae7b..f1f432f672a0 100644 --- a/pkg/storage/testdata/mvcc_histories/delete_range_predicate +++ b/pkg/storage/testdata/mvcc_histories/delete_range_predicate @@ -94,7 +94,7 @@ data: "h"/4.000000000,0 -> /BYTES/h4 meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "i"/7.000000000,0 -> /BYTES/i7 stats: key_count=8 key_bytes=160 val_count=12 val_bytes=111 range_key_count=2 range_key_bytes=27 range_val_count=2 live_count=3 live_bytes=111 gc_bytes_age=17863 intent_count=1 intent_bytes=19 separated_intent_count=1 intent_age=93 -error: (*roachpb.WriteIntentError:) conflicting intents on "i" +error: (*kvpb.WriteIntentError:) conflicting intents on "i" run stats error del_range_pred k=i end=+i ts=6 startTime=1 @@ -118,7 +118,7 @@ data: "h"/4.000000000,0 -> /BYTES/h4 meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "i"/7.000000000,0 -> /BYTES/i7 stats: key_count=8 key_bytes=160 val_count=12 val_bytes=111 range_key_count=2 range_key_bytes=27 range_val_count=2 live_count=3 live_bytes=111 gc_bytes_age=17863 intent_count=1 intent_bytes=19 separated_intent_count=1 intent_age=93 -error: (*roachpb.WriteIntentError:) conflicting intents on "i" +error: (*kvpb.WriteIntentError:) conflicting intents on "i" # error encountering point key at d5. # a tombstone should not get written at c5 or e5, since @@ -149,7 +149,7 @@ data: "h"/4.000000000,0 -> /BYTES/h4 meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "i"/7.000000000,0 -> /BYTES/i7 stats: key_count=9 key_bytes=174 val_count=13 val_bytes=118 range_key_count=2 range_key_bytes=27 range_val_count=2 live_count=4 live_bytes=132 gc_bytes_age=17863 intent_count=1 intent_bytes=19 separated_intent_count=1 intent_age=93 -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "d" at timestamp 5.000000000,0 too old; wrote at 5.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "d" at timestamp 5.000000000,0 too old; wrote at 5.000000000,1 # error encountering range key at k4. # a tombstones should not get written to j4 or q4 since @@ -185,7 +185,7 @@ data: "i"/7.000000000,0 -> /BYTES/i7 data: "j"/2.000000000,0 -> /BYTES/j2 data: "q"/2.000000000,0 -> /BYTES/q2 stats: key_count=11 key_bytes=202 val_count=15 val_bytes=132 range_key_count=2 range_key_bytes=27 range_val_count=2 live_count=6 live_bytes=174 gc_bytes_age=17863 intent_count=1 intent_bytes=19 separated_intent_count=1 intent_age=93 -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k" at timestamp 4.000000000,0 too old; wrote at 4.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k" at timestamp 4.000000000,0 too old; wrote at 4.000000000,1 # At this point the keyspace looks like this: # 7 [i7] diff --git a/pkg/storage/testdata/mvcc_histories/deletes b/pkg/storage/testdata/mvcc_histories/deletes index ef8725fa5160..7ca162509756 100644 --- a/pkg/storage/testdata/mvcc_histories/deletes +++ b/pkg/storage/testdata/mvcc_histories/deletes @@ -105,7 +105,7 @@ data: "a"/47.000000000,0 -> / data: "a"/46.000000000,0 -> /BYTES/abc data: "a"/44.000000000,0 -> / data: "b"/49.000000000,0 -> / -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "a" at timestamp 43.000000000,0 too old; wrote at 50.000000000,0 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "a" at timestamp 43.000000000,0 too old; wrote at 50.000000000,0 run ok with t=A @@ -135,7 +135,7 @@ data: "a"/47.000000000,0 -> / data: "a"/46.000000000,0 -> /BYTES/abc data: "a"/44.000000000,0 -> / data: "b"/49.000000000,0 -> / -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "a" at timestamp 44.000000000,0 too old; wrote at 50.000000000,0 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "a" at timestamp 44.000000000,0 too old; wrote at 50.000000000,0 run ok with t=A @@ -165,7 +165,7 @@ data: "a"/47.000000000,0 -> / data: "a"/46.000000000,0 -> /BYTES/abc data: "a"/44.000000000,0 -> / data: "b"/49.000000000,0 -> / -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "a" at timestamp 46.000000000,0 too old; wrote at 50.000000000,0 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "a" at timestamp 46.000000000,0 too old; wrote at 50.000000000,0 run ok with t=A @@ -195,7 +195,7 @@ data: "a"/47.000000000,0 -> / data: "a"/46.000000000,0 -> /BYTES/abc data: "a"/44.000000000,0 -> / data: "b"/49.000000000,0 -> / -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "a" at timestamp 47.000000000,0 too old; wrote at 50.000000000,0 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "a" at timestamp 47.000000000,0 too old; wrote at 50.000000000,0 run ok with t=A diff --git a/pkg/storage/testdata/mvcc_histories/export b/pkg/storage/testdata/mvcc_histories/export index bcd3b52224c7..94b81d85033d 100644 --- a/pkg/storage/testdata/mvcc_histories/export +++ b/pkg/storage/testdata/mvcc_histories/export @@ -81,17 +81,17 @@ data: "o"/7.000000000,0 -> /BYTES/n7 run error export k=a end=z ---- -error: (*roachpb.WriteIntentError:) conflicting intents on "a" +error: (*kvpb.WriteIntentError:) conflicting intents on "a" run error export k=a end=z maxIntents=100 ---- -error: (*roachpb.WriteIntentError:) conflicting intents on "a", "d", "j", "l", "o" +error: (*kvpb.WriteIntentError:) conflicting intents on "a", "d", "j", "l", "o" run error export k=a end=z maxIntents=3 ---- -error: (*roachpb.WriteIntentError:) conflicting intents on "a", "d", "j" +error: (*kvpb.WriteIntentError:) conflicting intents on "a", "d", "j" # Export the entire dataset below the intents, with full revision history. run ok diff --git a/pkg/storage/testdata/mvcc_histories/export_fingerprint b/pkg/storage/testdata/mvcc_histories/export_fingerprint index 2defbbd54799..da75ba80788b 100644 --- a/pkg/storage/testdata/mvcc_histories/export_fingerprint +++ b/pkg/storage/testdata/mvcc_histories/export_fingerprint @@ -81,17 +81,17 @@ data: "o"/7.000000000,0 -> /BYTES/n7 run error export fingerprint k=a end=z ---- -error: (*roachpb.WriteIntentError:) conflicting intents on "a" +error: (*kvpb.WriteIntentError:) conflicting intents on "a" run error export fingerprint k=a end=z maxIntents=100 ---- -error: (*roachpb.WriteIntentError:) conflicting intents on "a", "d", "j", "l", "o" +error: (*kvpb.WriteIntentError:) conflicting intents on "a", "d", "j", "l", "o" run error export fingerprint k=a end=z maxIntents=3 ---- -error: (*roachpb.WriteIntentError:) conflicting intents on "a", "d", "j" +error: (*kvpb.WriteIntentError:) conflicting intents on "a", "d", "j" # Export the entire dataset below the intents, with full revision history. run ok diff --git a/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_cput b/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_cput index 148217a6da73..922a60d95740 100644 --- a/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_cput +++ b/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_cput @@ -32,7 +32,7 @@ cput t=A k=k cond=a v=c meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true data: "k"/11.000000000,0 -> /BYTES/a data: "k"/1.000000000,0 -> /BYTES/first -error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003first" timestamp: +error: (*kvpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003first" timestamp: # Condition succeeds to find the original value. @@ -78,7 +78,7 @@ cput t=B k=k cond=b v=c meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false data: "k"/11.000000000,0 -> /BYTES/b data: "k"/1.000000000,0 -> /BYTES/first -error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003a" timestamp:<> +error: (*kvpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003a" timestamp:<> # However it succeeds to find the write before that. @@ -126,7 +126,7 @@ cput t=C k=k cond=c v=d meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "k"/11.000000000,0 -> /BYTES/c data: "k"/1.000000000,0 -> /BYTES/first -error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003a" timestamp:<> +error: (*kvpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003a" timestamp:<> run error cput t=C k=k cond=b v=d @@ -135,7 +135,7 @@ cput t=C k=k cond=b v=d meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "k"/11.000000000,0 -> /BYTES/c data: "k"/1.000000000,0 -> /BYTES/first -error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003a" timestamp:<> +error: (*kvpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003a" timestamp:<> # However it succeeds to find the write before that. @@ -182,7 +182,7 @@ cput t=D k=k cond=a v=c meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false data: "k"/11.000000000,0 -> /BYTES/b data: "k"/1.000000000,0 -> /BYTES/first -error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003first" timestamp: +error: (*kvpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003first" timestamp: run error cput t=D k=k cond=b v=c @@ -191,7 +191,7 @@ cput t=D k=k cond=b v=c meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false data: "k"/11.000000000,0 -> /BYTES/b data: "k"/1.000000000,0 -> /BYTES/first -error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003first" timestamp: +error: (*kvpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003first" timestamp: # However it succeeds to find the write before that. diff --git a/pkg/storage/testdata/mvcc_histories/increment b/pkg/storage/testdata/mvcc_histories/increment index e28fde3d7d84..e522821ebaba 100644 --- a/pkg/storage/testdata/mvcc_histories/increment +++ b/pkg/storage/testdata/mvcc_histories/increment @@ -73,7 +73,7 @@ data: "k"/0,1 -> /INT/2 data: "r"/3.000000000,1 -> /INT/3 data: "r"/3.000000000,0 -> /INT/2 data: "r"/1.000000000,0 -> /INT/1 -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "r" at timestamp 2.000000000,0 too old; wrote at 3.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "r" at timestamp 2.000000000,0 too old; wrote at 3.000000000,1 # Ditto with transactional. run error @@ -90,4 +90,4 @@ data: "r"/3.000000000,2 -> /INT/2 data: "r"/3.000000000,1 -> /INT/3 data: "r"/3.000000000,0 -> /INT/2 data: "r"/1.000000000,0 -> /INT/1 -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "r" at timestamp 2.000000000,0 too old; wrote at 3.000000000,2 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "r" at timestamp 2.000000000,0 too old; wrote at 3.000000000,2 diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_conflicts b/pkg/storage/testdata/mvcc_histories/range_tombstone_conflicts index 8c8ccdd6dcfb..0a976c1a1e20 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_conflicts +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_conflicts @@ -121,7 +121,7 @@ data: "g"/1.000000000,0 -> /BYTES/g1 data: "i"/1.000000000,0 -> /INT/1 data: "j"/1.000000000,0 -> /INT/1 stats: key_count=6 key_bytes=108 val_count=8 val_bytes=94 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=1 live_bytes=68 gc_bytes_age=16400 intent_count=1 intent_bytes=18 separated_intent_count=1 intent_age=93 -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "c" at timestamp 5.000000000,0 too old; wrote at 5.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "c" at timestamp 5.000000000,0 too old; wrote at 5.000000000,1 # Point key below range tombstones should error, but is written anyway at a # higher timestamp. Stats are updated correctly, even when there are @@ -145,7 +145,7 @@ data: "g"/1.000000000,0 -> /BYTES/g1 data: "i"/1.000000000,0 -> /INT/1 data: "j"/1.000000000,0 -> /INT/1 stats: key_count=7 key_bytes=122 val_count=9 val_bytes=101 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=2 live_bytes=89 gc_bytes_age=16400 intent_count=1 intent_bytes=18 separated_intent_count=1 intent_age=93 -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "c" at timestamp 3.000000000,0 too old; wrote at 5.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "c" at timestamp 3.000000000,0 too old; wrote at 5.000000000,1 run stats error put k=d ts=3 v=d3 @@ -167,7 +167,7 @@ data: "g"/1.000000000,0 -> /BYTES/g1 data: "i"/1.000000000,0 -> /INT/1 data: "j"/1.000000000,0 -> /INT/1 stats: key_count=7 key_bytes=134 val_count=10 val_bytes=108 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=3 live_bytes=110 gc_bytes_age=16206 intent_count=1 intent_bytes=18 separated_intent_count=1 intent_age=93 -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "d" at timestamp 3.000000000,0 too old; wrote at 5.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "d" at timestamp 3.000000000,0 too old; wrote at 5.000000000,1 run stats error put k=e ts=3 v=e3 @@ -190,7 +190,7 @@ data: "g"/1.000000000,0 -> /BYTES/g1 data: "i"/1.000000000,0 -> /INT/1 data: "j"/1.000000000,0 -> /INT/1 stats: key_count=7 key_bytes=146 val_count=11 val_bytes=115 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=4 live_bytes=131 gc_bytes_age=16010 intent_count=1 intent_bytes=18 separated_intent_count=1 intent_age=93 -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "e" at timestamp 3.000000000,0 too old; wrote at 5.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "e" at timestamp 3.000000000,0 too old; wrote at 5.000000000,1 # CPuts expecting a value covered by a range tombstone should error. run stats error @@ -214,7 +214,7 @@ data: "g"/1.000000000,0 -> /BYTES/g1 data: "i"/1.000000000,0 -> /INT/1 data: "j"/1.000000000,0 -> /INT/1 stats: key_count=7 key_bytes=146 val_count=11 val_bytes=115 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=4 live_bytes=131 gc_bytes_age=16010 intent_count=1 intent_bytes=18 separated_intent_count=1 intent_age=93 -error: (*roachpb.ConditionFailedError:) unexpected value: timestamp: +error: (*kvpb.ConditionFailedError:) unexpected value: timestamp: # A CPut replay of an intent expecting a value covered by a range tombstone # should error because of the range tombstone covering it. @@ -240,7 +240,7 @@ data: "g"/1.000000000,0 -> /BYTES/g1 data: "i"/1.000000000,0 -> /INT/1 data: "j"/1.000000000,0 -> /INT/1 stats: key_count=7 key_bytes=146 val_count=11 val_bytes=115 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=4 live_bytes=131 gc_bytes_age=16010 intent_count=1 intent_bytes=18 separated_intent_count=1 intent_age=93 -error: (*roachpb.ConditionFailedError:) unexpected value: timestamp: +error: (*kvpb.ConditionFailedError:) unexpected value: timestamp: # A CPut replacing an existing but ignored intent expecting a value covered # by a range tombstone should error because of the range tombstone covering it. @@ -269,7 +269,7 @@ data: "g"/1.000000000,0 -> /BYTES/g1 data: "i"/1.000000000,0 -> /INT/1 data: "j"/1.000000000,0 -> /INT/1 stats: key_count=7 key_bytes=146 val_count=11 val_bytes=115 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=4 live_bytes=131 gc_bytes_age=16010 intent_count=1 intent_bytes=18 separated_intent_count=1 intent_age=93 -error: (*roachpb.ConditionFailedError:) unexpected value: timestamp: +error: (*kvpb.ConditionFailedError:) unexpected value: timestamp: # An InitPut with failOnTombstones above a range tombstone should error. run stats error @@ -293,7 +293,7 @@ data: "g"/1.000000000,0 -> /BYTES/g1 data: "i"/1.000000000,0 -> /INT/1 data: "j"/1.000000000,0 -> /INT/1 stats: key_count=7 key_bytes=146 val_count=11 val_bytes=115 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=4 live_bytes=131 gc_bytes_age=16010 intent_count=1 intent_bytes=18 separated_intent_count=1 intent_age=93 -error: (*roachpb.ConditionFailedError:) unexpected value: timestamp: +error: (*kvpb.ConditionFailedError:) unexpected value: timestamp: # An InitPut with a different value as an existing key should succeed when there's # a range tombstone covering the existing value. @@ -345,7 +345,7 @@ data: "i"/5.000000000,1 -> /INT/1 data: "i"/1.000000000,0 -> /INT/1 data: "j"/1.000000000,0 -> /INT/1 stats: key_count=7 key_bytes=170 val_count=13 val_bytes=128 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=6 live_bytes=172 gc_bytes_age=15622 intent_count=1 intent_bytes=18 separated_intent_count=1 intent_age=93 -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "i" at timestamp 2.000000000,0 too old; wrote at 5.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "i" at timestamp 2.000000000,0 too old; wrote at 5.000000000,1 # An increment above a range tombstone should reset to 1. run stats ok diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_gets b/pkg/storage/testdata/mvcc_histories/range_tombstone_gets index 3787307fe94a..c77d8cfe7559 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_gets +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_gets @@ -246,25 +246,25 @@ run error get k=c ts=1 failOnMoreRecent ---- get: "c" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "c" at timestamp 1.000000000,0 too old; wrote at 4.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "c" at timestamp 1.000000000,0 too old; wrote at 4.000000000,1 run error get k=c ts=2 failOnMoreRecent ---- get: "c" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "c" at timestamp 2.000000000,0 too old; wrote at 4.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "c" at timestamp 2.000000000,0 too old; wrote at 4.000000000,1 run error get k=c ts=3 failOnMoreRecent ---- get: "c" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "c" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "c" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 run error get k=c ts=4 failOnMoreRecent ---- get: "c" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "c" at timestamp 4.000000000,0 too old; wrote at 4.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "c" at timestamp 4.000000000,0 too old; wrote at 4.000000000,1 run ok get k=c ts=5 failOnMoreRecent @@ -276,32 +276,32 @@ run error get k=e ts=3 failOnMoreRecent ---- get: "e" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "e" +error: (*kvpb.WriteIntentError:) conflicting intents on "e" run error get k=e ts=4 failOnMoreRecent ---- get: "e" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "e" +error: (*kvpb.WriteIntentError:) conflicting intents on "e" run error get k=e ts=5 failOnMoreRecent ---- get: "e" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "e" +error: (*kvpb.WriteIntentError:) conflicting intents on "e" # failOnMoreRecent: g run error get k=g ts=3 failOnMoreRecent ---- get: "g" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "g" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "g" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 run error get k=g ts=4 failOnMoreRecent ---- get: "g" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "g" at timestamp 4.000000000,0 too old; wrote at 4.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "g" at timestamp 4.000000000,0 too old; wrote at 4.000000000,1 run ok get k=g ts=5 failOnMoreRecent @@ -319,7 +319,7 @@ run error get k=b ts=1 globalUncertaintyLimit=2 ---- get: "b" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 1.000000000,0 encountered previous write with future timestamp 2.000000000,0 within uncertainty interval `t <= (local=0,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 1.000000000,0 encountered previous write with future timestamp 2.000000000,0 within uncertainty interval `t <= (local=0,0, global=0,0)`; observed timestamps: [] # globalUncertaintyLimit: d run ok @@ -331,7 +331,7 @@ run error get k=d ts=1 globalUncertaintyLimit=2 ---- get: "d" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 1.000000000,0 encountered previous write with future timestamp 2.000000000,0 within uncertainty interval `t <= (local=0,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 1.000000000,0 encountered previous write with future timestamp 2.000000000,0 within uncertainty interval `t <= (local=0,0, global=0,0)`; observed timestamps: [] # globalUncertaintyLimit: g run ok @@ -348,7 +348,7 @@ run error get k=g ts=1 globalUncertaintyLimit=4 ---- get: "g" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 1.000000000,0 encountered previous write with future timestamp 4.000000000,0 within uncertainty interval `t <= (local=0,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 1.000000000,0 encountered previous write with future timestamp 4.000000000,0 within uncertainty interval `t <= (local=0,0, global=0,0)`; observed timestamps: [] run ok get k=g ts=4 globalUncertaintyLimit=5 @@ -370,7 +370,7 @@ run error get k=h ts=2 globalUncertaintyLimit=4 ---- get: "h" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 2.000000000,0 encountered previous write with future timestamp 4.000000000,0 within uncertainty interval `t <= (local=0,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 2.000000000,0 encountered previous write with future timestamp 4.000000000,0 within uncertainty interval `t <= (local=0,0, global=0,0)`; observed timestamps: [] # Test local timestamp uncertainty for [j-l)@4 with localTs=3. Normally, # globalUncertaintyLimit=4 would error. However, localUncertaintyLimit<4 @@ -380,7 +380,7 @@ run error get k=k ts=1 globalUncertaintyLimit=4 ---- get: "k" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 1.000000000,0 encountered previous write with future timestamp 4.000000000,0 (local=3.000000000,0) within uncertainty interval `t <= (local=0,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 1.000000000,0 encountered previous write with future timestamp 4.000000000,0 (local=3.000000000,0) within uncertainty interval `t <= (local=0,0, global=0,0)`; observed timestamps: [] run ok get k=k ts=1 globalUncertaintyLimit=4 localUncertaintyLimit=1 @@ -396,4 +396,4 @@ run error get k=k ts=1 globalUncertaintyLimit=4 localUncertaintyLimit=3 ---- get: "k" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 1.000000000,0 encountered previous write with future timestamp 4.000000000,0 (local=3.000000000,0) within uncertainty interval `t <= (local=3.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 1.000000000,0 encountered previous write with future timestamp 4.000000000,0 (local=3.000000000,0) within uncertainty interval `t <= (local=3.000000000,0, global=0,0)`; observed timestamps: [] diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_iter_incremental b/pkg/storage/testdata/mvcc_histories/range_tombstone_iter_incremental index 9843aa830673..3b308c9714ac 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_iter_incremental +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_iter_incremental @@ -923,7 +923,7 @@ iter_scan ---- iter_seek_ge: err=conflicting intents on "a" iter_scan: err=conflicting intents on "a" -error: (*roachpb.WriteIntentError:) conflicting intents on "a" +error: (*kvpb.WriteIntentError:) conflicting intents on "a" run error iter_new_incremental types=pointsAndRanges k=a end=z intents=aggregate @@ -959,7 +959,7 @@ iter_scan: {m-n}/[3.000000000,0={localTs=2.000000000,0}/] ! iter_scan: "m"/8.000000000,0=/BYTES/m8 {m-n}/[3.000000000,0={localTs=2.000000000,0}/] iter_scan: "o"/7.000000000,0=/BYTES/o7 ! iter_scan: . -error: (*roachpb.WriteIntentError:) conflicting intents on "a", "d", "j", "l", "m", "o" +error: (*kvpb.WriteIntentError:) conflicting intents on "a", "d", "j", "l", "m", "o" run error iter_new_incremental types=pointsAndRanges k=a end=z endTs=7 intents=aggregate @@ -993,7 +993,7 @@ iter_scan: "l"/7.000000000,0=/BYTES/l7 iter_scan: {m-n}/[3.000000000,0={localTs=2.000000000,0}/] ! iter_scan: "o"/7.000000000,0=/BYTES/o7 ! iter_scan: . -error: (*roachpb.WriteIntentError:) conflicting intents on "a", "j", "l", "o" +error: (*kvpb.WriteIntentError:) conflicting intents on "a", "j", "l", "o" run ok iter_new_incremental types=pointsAndRanges k=a end=z endTs=6 intents=aggregate @@ -1061,7 +1061,7 @@ iter_next_ignoring_time ---- iter_seek_ge: {c-d}/[5.000000000,0=/ 3.000000000,0=/] ! iter_next_ignoring_time: err=conflicting intents on "d" -error: (*roachpb.WriteIntentError:) conflicting intents on "d" +error: (*kvpb.WriteIntentError:) conflicting intents on "d" run ok iter_new_incremental types=pointsAndRanges k=a end=z startTs=2 endTs=4 intents=error @@ -1078,7 +1078,7 @@ iter_next_key_ignoring_time ---- iter_seek_ge: {c-d}/[5.000000000,0=/ 3.000000000,0=/] ! iter_next_key_ignoring_time: err=conflicting intents on "d" -error: (*roachpb.WriteIntentError:) conflicting intents on "d" +error: (*kvpb.WriteIntentError:) conflicting intents on "d" # rangesOnly doesn't care about intents. run ok diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_scans b/pkg/storage/testdata/mvcc_histories/range_tombstone_scans index c01ac2dedd6a..bd0a7fec0589 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_scans +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_scans @@ -383,13 +383,13 @@ run error scan k=a end=d ts=3 failOnMoreRecent ---- scan: "a"-"d" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "a" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "a" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 run error scan k=a end=d ts=4 failOnMoreRecent ---- scan: "a"-"d" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "c" at timestamp 4.000000000,0 too old; wrote at 4.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "c" at timestamp 4.000000000,0 too old; wrote at 4.000000000,1 run ok scan k=a end=d ts=5 failOnMoreRecent @@ -401,25 +401,25 @@ run error scan k=c end=d ts=1 failOnMoreRecent ---- scan: "c"-"d" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "c" at timestamp 1.000000000,0 too old; wrote at 4.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "c" at timestamp 1.000000000,0 too old; wrote at 4.000000000,1 run error scan k=c end=d ts=2 failOnMoreRecent ---- scan: "c"-"d" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "c" at timestamp 2.000000000,0 too old; wrote at 4.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "c" at timestamp 2.000000000,0 too old; wrote at 4.000000000,1 run error scan k=c end=d ts=3 failOnMoreRecent ---- scan: "c"-"d" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "c" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "c" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 run error scan k=c end=d ts=4 failOnMoreRecent ---- scan: "c"-"d" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "c" at timestamp 4.000000000,0 too old; wrote at 4.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "c" at timestamp 4.000000000,0 too old; wrote at 4.000000000,1 run ok scan k=c end=d ts=5 failOnMoreRecent @@ -431,32 +431,32 @@ run error scan k=e end=f ts=3 failOnMoreRecent ---- scan: "e"-"f" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "e" +error: (*kvpb.WriteIntentError:) conflicting intents on "e" run error scan k=e end=f ts=4 failOnMoreRecent ---- scan: "e"-"f" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "e" +error: (*kvpb.WriteIntentError:) conflicting intents on "e" run error scan k=e end=f ts=5 failOnMoreRecent ---- scan: "e"-"f" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "e" +error: (*kvpb.WriteIntentError:) conflicting intents on "e" # failOnMoreRecent: g-h run error scan k=g end=h ts=3 failOnMoreRecent ---- scan: "g"-"h" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "g" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "g" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 run error scan k=g end=h ts=4 failOnMoreRecent ---- scan: "g"-"h" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "g" at timestamp 4.000000000,0 too old; wrote at 4.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "g" at timestamp 4.000000000,0 too old; wrote at 4.000000000,1 run ok scan k=g end=h ts=5 failOnMoreRecent @@ -474,7 +474,7 @@ run error scan k=b end=d ts=3 globalUncertaintyLimit=4 ---- scan: "b"-"d" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 3.000000000,0 encountered previous write with future timestamp 4.000000000,0 within uncertainty interval `t <= (local=0,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 3.000000000,0 encountered previous write with future timestamp 4.000000000,0 within uncertainty interval `t <= (local=0,0, global=0,0)`; observed timestamps: [] run ok scan k=b end=d ts=4 globalUncertaintyLimit=5 @@ -496,7 +496,7 @@ run error scan k=g end=h ts=1 globalUncertaintyLimit=4 ---- scan: "g"-"h" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 1.000000000,0 encountered previous write with future timestamp 4.000000000,0 within uncertainty interval `t <= (local=0,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 1.000000000,0 encountered previous write with future timestamp 4.000000000,0 within uncertainty interval `t <= (local=0,0, global=0,0)`; observed timestamps: [] run ok scan k=g end=h ts=4 globalUncertaintyLimit=5 @@ -518,7 +518,7 @@ run error scan k=h end=i ts=2 globalUncertaintyLimit=4 ---- scan: "h"-"i" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 2.000000000,0 encountered previous write with future timestamp 4.000000000,0 within uncertainty interval `t <= (local=0,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 2.000000000,0 encountered previous write with future timestamp 4.000000000,0 within uncertainty interval `t <= (local=0,0, global=0,0)`; observed timestamps: [] # Test local timestamp uncertainty for [j-l)@4 with localTs=3. Normally, # globalUncertaintyLimit=4 would error. However, localUncertaintyLimit<4 @@ -528,7 +528,7 @@ run error scan k=j end=l ts=1 globalUncertaintyLimit=4 ---- scan: "j"-"l" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 1.000000000,0 encountered previous write with future timestamp 4.000000000,0 (local=3.000000000,0) within uncertainty interval `t <= (local=0,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 1.000000000,0 encountered previous write with future timestamp 4.000000000,0 (local=3.000000000,0) within uncertainty interval `t <= (local=0,0, global=0,0)`; observed timestamps: [] run ok scan k=j end=l ts=1 globalUncertaintyLimit=4 localUncertaintyLimit=1 @@ -544,4 +544,4 @@ run error scan k=j end=l ts=1 globalUncertaintyLimit=4 localUncertaintyLimit=3 ---- scan: "j"-"l" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 1.000000000,0 encountered previous write with future timestamp 4.000000000,0 (local=3.000000000,0) within uncertainty interval `t <= (local=3.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 1.000000000,0 encountered previous write with future timestamp 4.000000000,0 (local=3.000000000,0) within uncertainty interval `t <= (local=3.000000000,0, global=0,0)`; observed timestamps: [] diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_writes b/pkg/storage/testdata/mvcc_histories/range_tombstone_writes index bc3c4b651493..91a912fb42df 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_writes +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_writes @@ -122,7 +122,7 @@ data: "g"/2.000000000,0 -> /BYTES/g2 meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "i"/7.000000000,0 -> /BYTES/i7 -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "a" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "a" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 run error del_range_ts k=a end=b ts=4 @@ -141,7 +141,7 @@ data: "g"/2.000000000,0 -> /BYTES/g2 meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "i"/7.000000000,0 -> /BYTES/i7 -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "a" at timestamp 4.000000000,0 too old; wrote at 4.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "a" at timestamp 4.000000000,0 too old; wrote at 4.000000000,1 run error del_range_ts k=e end=g ts=3 @@ -160,7 +160,7 @@ data: "g"/2.000000000,0 -> /BYTES/g2 meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "i"/7.000000000,0 -> /BYTES/i7 -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "f" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "f" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 # Writing at or below existing range tombstones should return a WriteTooOldError, # regardless of how they overlap. @@ -181,7 +181,7 @@ data: "g"/2.000000000,0 -> /BYTES/g2 meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "i"/7.000000000,0 -> /BYTES/i7 -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 run error del_range_ts k=k end=p ts=4 @@ -200,7 +200,7 @@ data: "g"/2.000000000,0 -> /BYTES/g2 meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "i"/7.000000000,0 -> /BYTES/i7 -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k" at timestamp 4.000000000,0 too old; wrote at 4.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k" at timestamp 4.000000000,0 too old; wrote at 4.000000000,1 run error del_range_ts k=j end=m ts=3 @@ -219,7 +219,7 @@ data: "g"/2.000000000,0 -> /BYTES/g2 meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "i"/7.000000000,0 -> /BYTES/i7 -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 run error del_range_ts k=o end=q ts=3 @@ -238,7 +238,7 @@ data: "g"/2.000000000,0 -> /BYTES/g2 meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "i"/7.000000000,0 -> /BYTES/i7 -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "o" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "o" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 run error del_range_ts k=j end=q ts=3 @@ -257,7 +257,7 @@ data: "g"/2.000000000,0 -> /BYTES/g2 meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "i"/7.000000000,0 -> /BYTES/i7 -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 run error del_range_ts k=k end=n ts=3 @@ -276,7 +276,7 @@ data: "g"/2.000000000,0 -> /BYTES/g2 meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "i"/7.000000000,0 -> /BYTES/i7 -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 # Writing below intents should return a WriteIntentError, both when above and # below the intent timestamp and any existing values. @@ -297,7 +297,7 @@ data: "g"/2.000000000,0 -> /BYTES/g2 meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "i"/7.000000000,0 -> /BYTES/i7 -error: (*roachpb.WriteIntentError:) conflicting intents on "d" +error: (*kvpb.WriteIntentError:) conflicting intents on "d" run error del_range_ts k=d end=e ts=5 @@ -316,7 +316,7 @@ data: "g"/2.000000000,0 -> /BYTES/g2 meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "i"/7.000000000,0 -> /BYTES/i7 -error: (*roachpb.WriteIntentError:) conflicting intents on "d" +error: (*kvpb.WriteIntentError:) conflicting intents on "d" run error del_range_ts k=i end=j ts=5 @@ -335,7 +335,7 @@ data: "g"/2.000000000,0 -> /BYTES/g2 meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "i"/7.000000000,0 -> /BYTES/i7 -error: (*roachpb.WriteIntentError:) conflicting intents on "i" +error: (*kvpb.WriteIntentError:) conflicting intents on "i" run error del_range_ts k=i end=j ts=7 @@ -354,7 +354,7 @@ data: "g"/2.000000000,0 -> /BYTES/g2 meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "i"/7.000000000,0 -> /BYTES/i7 -error: (*roachpb.WriteIntentError:) conflicting intents on "i" +error: (*kvpb.WriteIntentError:) conflicting intents on "i" run error del_range_ts k=i end=j ts=10 @@ -373,7 +373,7 @@ data: "g"/2.000000000,0 -> /BYTES/g2 meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "i"/7.000000000,0 -> /BYTES/i7 -error: (*roachpb.WriteIntentError:) conflicting intents on "i" +error: (*kvpb.WriteIntentError:) conflicting intents on "i" # Writing above an inline value should error. We disable passing covered MVCC # stats in metamorphic tests because it changes the error message. diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_writes_idempotent b/pkg/storage/testdata/mvcc_histories/range_tombstone_writes_idempotent index 545c704bb640..04d456602384 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_writes_idempotent +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_writes_idempotent @@ -70,7 +70,7 @@ data: "k"/3.000000000,0 -> /BYTES/k3 data: "m"/1.000000000,0 -> /BYTES/m1 data: "n"/3.000000000,0 -> / meta: "p"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/p0 mergeTs= txnDidNotUpdateMeta=false -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "e" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "e" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 run error del_range_ts k=f end=g ts=3 idempotent @@ -91,7 +91,7 @@ data: "k"/3.000000000,0 -> /BYTES/k3 data: "m"/1.000000000,0 -> /BYTES/m1 data: "n"/3.000000000,0 -> / meta: "p"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/p0 mergeTs= txnDidNotUpdateMeta=false -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "f" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "f" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 run error del_range_ts k=l end=m ts=3 idempotent @@ -112,7 +112,7 @@ data: "k"/3.000000000,0 -> /BYTES/k3 data: "m"/1.000000000,0 -> /BYTES/m1 data: "n"/3.000000000,0 -> / meta: "p"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/p0 mergeTs= txnDidNotUpdateMeta=false -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "l" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "l" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 # Writing at the timestamp of a point or range key errors. run error @@ -134,7 +134,7 @@ data: "k"/3.000000000,0 -> /BYTES/k3 data: "m"/1.000000000,0 -> /BYTES/m1 data: "n"/3.000000000,0 -> / meta: "p"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/p0 mergeTs= txnDidNotUpdateMeta=false -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "e" at timestamp 4.000000000,0 too old; wrote at 4.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "e" at timestamp 4.000000000,0 too old; wrote at 4.000000000,1 run error del_range_ts k=l end=m ts=4 idempotent @@ -155,7 +155,7 @@ data: "k"/3.000000000,0 -> /BYTES/k3 data: "m"/1.000000000,0 -> /BYTES/m1 data: "n"/3.000000000,0 -> / meta: "p"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/p0 mergeTs= txnDidNotUpdateMeta=false -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "l" at timestamp 4.000000000,0 too old; wrote at 4.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "l" at timestamp 4.000000000,0 too old; wrote at 4.000000000,1 # Writing below intents error. run error @@ -177,7 +177,7 @@ data: "k"/3.000000000,0 -> /BYTES/k3 data: "m"/1.000000000,0 -> /BYTES/m1 data: "n"/3.000000000,0 -> / meta: "p"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/p0 mergeTs= txnDidNotUpdateMeta=false -error: (*roachpb.WriteIntentError:) conflicting intents on "i" +error: (*kvpb.WriteIntentError:) conflicting intents on "i" # Idempotent writes: exact, left, right, subset. run stats ok diff --git a/pkg/storage/testdata/mvcc_histories/read_fail_on_more_recent b/pkg/storage/testdata/mvcc_histories/read_fail_on_more_recent index f1286da73ab4..0a0945ac6bba 100644 --- a/pkg/storage/testdata/mvcc_histories/read_fail_on_more_recent +++ b/pkg/storage/testdata/mvcc_histories/read_fail_on_more_recent @@ -37,7 +37,7 @@ run error get k=k1 ts=9,0 failOnMoreRecent ---- get: "k1" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 9.000000000,0 too old; wrote at 10.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 9.000000000,0 too old; wrote at 10.000000000,1 run ok get k=k1 ts=10,0 @@ -48,7 +48,7 @@ run error get k=k1 ts=10,0 failOnMoreRecent ---- get: "k1" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 10.000000000,0 too old; wrote at 10.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 10.000000000,0 too old; wrote at 10.000000000,1 run ok get k=k1 ts=11,0 @@ -69,7 +69,7 @@ run error scan k=k1 end=k2 ts=9,0 failOnMoreRecent ---- scan: "k1"-"k2" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 9.000000000,0 too old; wrote at 10.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 9.000000000,0 too old; wrote at 10.000000000,1 run ok scan k=k1 end=k2 ts=10,0 @@ -80,7 +80,7 @@ run error scan k=k1 end=k2 ts=10,0 failOnMoreRecent ---- scan: "k1"-"k2" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 10.000000000,0 too old; wrote at 10.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 10.000000000,0 too old; wrote at 10.000000000,1 run ok scan k=k1 end=k2 ts=11,0 @@ -101,31 +101,31 @@ run error get k=k2 ts=9,0 failOnMoreRecent ---- get: "k2" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k2" +error: (*kvpb.WriteIntentError:) conflicting intents on "k2" run error get k=k2 ts=10,0 ---- get: "k2" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k2" +error: (*kvpb.WriteIntentError:) conflicting intents on "k2" run error get k=k2 ts=10,0 failOnMoreRecent ---- get: "k2" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k2" +error: (*kvpb.WriteIntentError:) conflicting intents on "k2" run error get k=k2 ts=11,0 ---- get: "k2" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k2" +error: (*kvpb.WriteIntentError:) conflicting intents on "k2" run error get k=k2 ts=11,0 failOnMoreRecent ---- get: "k2" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k2" +error: (*kvpb.WriteIntentError:) conflicting intents on "k2" run ok scan k=k2 end=k3 ts=9,0 @@ -136,31 +136,31 @@ run error scan k=k2 end=k3 ts=9,0 failOnMoreRecent ---- scan: "k2"-"k3" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k2" +error: (*kvpb.WriteIntentError:) conflicting intents on "k2" run error scan k=k2 end=k3 ts=10,0 ---- scan: "k2"-"k3" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k2" +error: (*kvpb.WriteIntentError:) conflicting intents on "k2" run error scan k=k2 end=k3 ts=10,0 failOnMoreRecent ---- scan: "k2"-"k3" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k2" +error: (*kvpb.WriteIntentError:) conflicting intents on "k2" run error scan k=k2 end=k3 ts=11,0 ---- scan: "k2"-"k3" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k2" +error: (*kvpb.WriteIntentError:) conflicting intents on "k2" run error scan k=k2 end=k3 ts=11,0 failOnMoreRecent ---- scan: "k2"-"k3" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k2" +error: (*kvpb.WriteIntentError:) conflicting intents on "k2" # More test cases: # @@ -180,31 +180,31 @@ run error scan k=k1 end=k3 ts=9,0 failOnMoreRecent ---- scan: "k1"-"k3" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 9.000000000,0 too old; wrote at 10.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 9.000000000,0 too old; wrote at 10.000000000,1 run error scan k=k1 end=k3 ts=10,0 ---- scan: "k1"-"k3" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k2" +error: (*kvpb.WriteIntentError:) conflicting intents on "k2" run error scan k=k1 end=k3 ts=10,0 failOnMoreRecent ---- scan: "k1"-"k3" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 10.000000000,0 too old; wrote at 10.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 10.000000000,0 too old; wrote at 10.000000000,1 run error scan k=k1 end=k3 ts=11,0 ---- scan: "k1"-"k3" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k2" +error: (*kvpb.WriteIntentError:) conflicting intents on "k2" run error scan k=k1 end=k3 ts=11,0 failOnMoreRecent ---- scan: "k1"-"k3" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k2" +error: (*kvpb.WriteIntentError:) conflicting intents on "k2" # The failOnMoreRecent and inconsistent options cannot be used together. @@ -241,16 +241,16 @@ run error scan k=a end=b_next ts=9,0 failOnMoreRecent ---- scan: "a"-"b_next" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "a" at timestamp 9.000000000,0 too old; wrote at 13.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "a" at timestamp 9.000000000,0 too old; wrote at 13.000000000,1 run error scan k=a end=c_next ts=9,0 failOnMoreRecent ---- scan: "a"-"c_next" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "a" at timestamp 9.000000000,0 too old; wrote at 13.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "a" at timestamp 9.000000000,0 too old; wrote at 13.000000000,1 run error scan k=b end=c_next ts=9,0 failOnMoreRecent ---- scan: "b"-"c_next" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "b" at timestamp 9.000000000,0 too old; wrote at 13.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "b" at timestamp 9.000000000,0 too old; wrote at 13.000000000,1 diff --git a/pkg/storage/testdata/mvcc_histories/skip_locked b/pkg/storage/testdata/mvcc_histories/skip_locked index d310a97c3579..50934432059b 100644 --- a/pkg/storage/testdata/mvcc_histories/skip_locked +++ b/pkg/storage/testdata/mvcc_histories/skip_locked @@ -620,7 +620,7 @@ run error get ts=10 k=k1 skipLocked failOnMoreRecent ---- get: "k1" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 10.000000000,0 too old; wrote at 11.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 10.000000000,0 too old; wrote at 11.000000000,1 run ok get ts=10 k=k2 skipLocked failOnMoreRecent @@ -644,25 +644,25 @@ run error get ts=10 k=k5 skipLocked failOnMoreRecent ---- get: "k5" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 10.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 10.000000000,0 too old; wrote at 17.000000000,1 run error scan ts=10 k=k1 end=k6 skipLocked failOnMoreRecent ---- scan: "k1"-"k6" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 10.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 10.000000000,0 too old; wrote at 17.000000000,1 run error scan ts=10 k=k1 end=k6 reverse skipLocked failOnMoreRecent ---- scan: "k1"-"k6" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 10.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 10.000000000,0 too old; wrote at 17.000000000,1 run error get ts=11 k=k1 skipLocked failOnMoreRecent ---- get: "k1" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 11.000000000,0 too old; wrote at 11.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 11.000000000,0 too old; wrote at 11.000000000,1 run ok get ts=11 k=k2 skipLocked failOnMoreRecent @@ -686,19 +686,19 @@ run error get ts=11 k=k5 skipLocked failOnMoreRecent ---- get: "k5" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 11.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 11.000000000,0 too old; wrote at 17.000000000,1 run error scan ts=11 k=k1 end=k6 skipLocked failOnMoreRecent ---- scan: "k1"-"k6" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 11.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k1" at timestamp 11.000000000,0 too old; wrote at 17.000000000,1 run error scan ts=11 k=k1 end=k6 reverse skipLocked failOnMoreRecent ---- scan: "k1"-"k6" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 11.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 11.000000000,0 too old; wrote at 17.000000000,1 run ok get ts=12 k=k1 skipLocked failOnMoreRecent @@ -727,19 +727,19 @@ run error get ts=12 k=k5 skipLocked failOnMoreRecent ---- get: "k5" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 12.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 12.000000000,0 too old; wrote at 17.000000000,1 run error scan ts=12 k=k1 end=k6 skipLocked failOnMoreRecent ---- scan: "k1"-"k6" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 12.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 12.000000000,0 too old; wrote at 17.000000000,1 run error scan ts=12 k=k1 end=k6 reverse skipLocked failOnMoreRecent ---- scan: "k1"-"k6" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 12.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 12.000000000,0 too old; wrote at 17.000000000,1 run ok get ts=12 t=A k=k1 skipLocked failOnMoreRecent @@ -768,19 +768,19 @@ run error get ts=12 t=A k=k5 skipLocked failOnMoreRecent ---- get: "k5" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 12.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 12.000000000,0 too old; wrote at 17.000000000,1 run error scan ts=12 t=A k=k1 end=k6 skipLocked failOnMoreRecent ---- scan: "k1"-"k6" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 12.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 12.000000000,0 too old; wrote at 17.000000000,1 run error scan ts=12 t=A k=k1 end=k6 reverse skipLocked failOnMoreRecent ---- scan: "k1"-"k6" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 12.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 12.000000000,0 too old; wrote at 17.000000000,1 run ok get ts=13 k=k1 skipLocked failOnMoreRecent @@ -809,19 +809,19 @@ run error get ts=13 k=k5 skipLocked failOnMoreRecent ---- get: "k5" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 13.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 13.000000000,0 too old; wrote at 17.000000000,1 run error scan ts=13 k=k1 end=k6 skipLocked failOnMoreRecent ---- scan: "k1"-"k6" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 13.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 13.000000000,0 too old; wrote at 17.000000000,1 run error scan ts=13 k=k1 end=k6 reverse skipLocked failOnMoreRecent ---- scan: "k1"-"k6" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 13.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 13.000000000,0 too old; wrote at 17.000000000,1 run ok get ts=13 t=B k=k1 skipLocked failOnMoreRecent @@ -849,19 +849,19 @@ run error get ts=13 t=B k=k5 skipLocked failOnMoreRecent ---- get: "k5" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 13.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 13.000000000,0 too old; wrote at 17.000000000,1 run error scan ts=13 t=B k=k1 end=k6 skipLocked failOnMoreRecent ---- scan: "k1"-"k6" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 13.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 13.000000000,0 too old; wrote at 17.000000000,1 run error scan ts=13 t=B k=k1 end=k6 reverse skipLocked failOnMoreRecent ---- scan: "k1"-"k6" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 13.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 13.000000000,0 too old; wrote at 17.000000000,1 run ok get ts=14 k=k1 skipLocked failOnMoreRecent @@ -890,19 +890,19 @@ run error get ts=14 k=k5 skipLocked failOnMoreRecent ---- get: "k5" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 14.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 14.000000000,0 too old; wrote at 17.000000000,1 run error scan ts=14 k=k1 end=k6 skipLocked failOnMoreRecent ---- scan: "k1"-"k6" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 14.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 14.000000000,0 too old; wrote at 17.000000000,1 run error scan ts=14 k=k1 end=k6 reverse skipLocked failOnMoreRecent ---- scan: "k1"-"k6" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 14.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 14.000000000,0 too old; wrote at 17.000000000,1 run ok get ts=14 t=C k=k1 skipLocked failOnMoreRecent @@ -930,19 +930,19 @@ run error get ts=14 t=C k=k5 skipLocked failOnMoreRecent ---- get: "k5" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 14.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 14.000000000,0 too old; wrote at 17.000000000,1 run error scan ts=14 t=C k=k1 end=k6 skipLocked failOnMoreRecent ---- scan: "k1"-"k6" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 14.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 14.000000000,0 too old; wrote at 17.000000000,1 run error scan ts=14 t=C k=k1 end=k6 reverse skipLocked failOnMoreRecent ---- scan: "k1"-"k6" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 14.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 14.000000000,0 too old; wrote at 17.000000000,1 run ok get ts=15 k=k1 skipLocked failOnMoreRecent @@ -971,19 +971,19 @@ run error get ts=15 k=k5 skipLocked failOnMoreRecent ---- get: "k5" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 15.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 15.000000000,0 too old; wrote at 17.000000000,1 run error scan ts=15 k=k1 end=k6 skipLocked failOnMoreRecent ---- scan: "k1"-"k6" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 15.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 15.000000000,0 too old; wrote at 17.000000000,1 run error scan ts=15 k=k1 end=k6 reverse skipLocked failOnMoreRecent ---- scan: "k1"-"k6" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 15.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 15.000000000,0 too old; wrote at 17.000000000,1 run ok get ts=15 t=D k=k1 skipLocked failOnMoreRecent @@ -1012,19 +1012,19 @@ run error get ts=15 t=D k=k5 skipLocked failOnMoreRecent ---- get: "k5" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 15.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 15.000000000,0 too old; wrote at 17.000000000,1 run error scan ts=15 t=D k=k1 end=k6 skipLocked failOnMoreRecent ---- scan: "k1"-"k6" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 15.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 15.000000000,0 too old; wrote at 17.000000000,1 run error scan ts=15 t=D k=k1 end=k6 reverse skipLocked failOnMoreRecent ---- scan: "k1"-"k6" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 15.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 15.000000000,0 too old; wrote at 17.000000000,1 run ok get ts=16 k=k1 skipLocked failOnMoreRecent @@ -1053,19 +1053,19 @@ run error get ts=16 k=k5 skipLocked failOnMoreRecent ---- get: "k5" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 16.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 16.000000000,0 too old; wrote at 17.000000000,1 run error scan ts=16 k=k1 end=k6 skipLocked failOnMoreRecent ---- scan: "k1"-"k6" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 16.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 16.000000000,0 too old; wrote at 17.000000000,1 run error scan ts=16 k=k1 end=k6 reverse skipLocked failOnMoreRecent ---- scan: "k1"-"k6" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 16.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 16.000000000,0 too old; wrote at 17.000000000,1 run ok get ts=16 t=E k=k1 skipLocked failOnMoreRecent @@ -1093,19 +1093,19 @@ run error get ts=16 t=E k=k5 skipLocked failOnMoreRecent ---- get: "k5" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 16.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 16.000000000,0 too old; wrote at 17.000000000,1 run error scan ts=16 t=E k=k1 end=k6 skipLocked failOnMoreRecent ---- scan: "k1"-"k6" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 16.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 16.000000000,0 too old; wrote at 17.000000000,1 run error scan ts=16 t=E k=k1 end=k6 reverse skipLocked failOnMoreRecent ---- scan: "k1"-"k6" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 16.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 16.000000000,0 too old; wrote at 17.000000000,1 run ok get ts=17 k=k1 skipLocked failOnMoreRecent @@ -1134,19 +1134,19 @@ run error get ts=17 k=k5 skipLocked failOnMoreRecent ---- get: "k5" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 17.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 17.000000000,0 too old; wrote at 17.000000000,1 run error scan ts=17 k=k1 end=k6 skipLocked failOnMoreRecent ---- scan: "k1"-"k6" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 17.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 17.000000000,0 too old; wrote at 17.000000000,1 run error scan ts=17 k=k1 end=k6 reverse skipLocked failOnMoreRecent ---- scan: "k1"-"k6" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 17.000000000,0 too old; wrote at 17.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k5" at timestamp 17.000000000,0 too old; wrote at 17.000000000,1 run ok get ts=18 k=k1 skipLocked failOnMoreRecent diff --git a/pkg/storage/testdata/mvcc_histories/uncertainty_interval b/pkg/storage/testdata/mvcc_histories/uncertainty_interval index b0bbaa64706f..02d158327b25 100644 --- a/pkg/storage/testdata/mvcc_histories/uncertainty_interval +++ b/pkg/storage/testdata/mvcc_histories/uncertainty_interval @@ -77,25 +77,25 @@ run error get t=txn2 k=k1 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=0,0, global=10.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=0,0, global=10.000000000,0)`; observed timestamps: [] run error scan t=txn2 k=k1 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=0,0, global=10.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=0,0, global=10.000000000,0)`; observed timestamps: [] run error get t=txn2 k=k2 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=0,0, global=10.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=0,0, global=10.000000000,0)`; observed timestamps: [] run error scan t=txn2 k=k2 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=0,0, global=10.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=0,0, global=10.000000000,0)`; observed timestamps: [] run ok @@ -108,25 +108,25 @@ run error get t=txn3 k=k1 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=0,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=0,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn3 k=k1 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=0,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=0,0, global=15.000000000,0)`; observed timestamps: [] run error get t=txn3 k=k2 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=0,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=0,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn3 k=k2 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=0,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=0,0, global=15.000000000,0)`; observed timestamps: [] run ok @@ -139,25 +139,25 @@ run error get t=txn4 k=k1 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn4 k=k1 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn4 k=k2 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn4 k=k2 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=20.000000000,0)`; observed timestamps: [] run ok @@ -170,25 +170,25 @@ run error get t=txn5 k=k1 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k1 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn5 k=k2 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k2 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -255,25 +255,25 @@ run error get t=txn8 k=k1 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k1 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k2 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k2 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=20.000000000,0)`; observed timestamps: [] run ok @@ -286,25 +286,25 @@ run error get t=txn9 k=k1 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k1 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k2 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k2 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -344,25 +344,25 @@ run error get t=txn11 k=k1 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn11 k=k1 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn11 k=k2 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn11 k=k2 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=20.000000000,0)`; observed timestamps: [] run ok @@ -375,25 +375,25 @@ run error get t=txn12 k=k1 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn12 k=k1 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn12 k=k2 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn12 k=k2 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=0,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -416,13 +416,13 @@ run error get t=txn13 k=k2 ---- get: "k2" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k2" +error: (*kvpb.WriteIntentError:) conflicting intents on "k2" run error scan t=txn13 k=k2 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k2" +error: (*kvpb.WriteIntentError:) conflicting intents on "k2" run ok @@ -445,13 +445,13 @@ run error get t=txn14 k=k2 ---- get: "k2" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k2" +error: (*kvpb.WriteIntentError:) conflicting intents on "k2" run error scan t=txn14 k=k2 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k2" +error: (*kvpb.WriteIntentError:) conflicting intents on "k2" run ok @@ -474,10 +474,10 @@ run error get t=txn15 k=k2 ---- get: "k2" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k2" +error: (*kvpb.WriteIntentError:) conflicting intents on "k2" run error scan t=txn15 k=k2 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k2" +error: (*kvpb.WriteIntentError:) conflicting intents on "k2" diff --git a/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit b/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit index 2a89f3dfd6bd..fd80e9e311b5 100644 --- a/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit +++ b/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit @@ -204,13 +204,13 @@ run error get t=txn1 k=k2 localUncertaintyLimit=5,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run error scan t=txn1 k=k2 localUncertaintyLimit=5,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run ok get t=txn1 k=k3 localUncertaintyLimit=5,0 @@ -226,13 +226,13 @@ run error get t=txn1 k=k4 localUncertaintyLimit=5,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run error scan t=txn1 k=k4 localUncertaintyLimit=5,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run ok get t=txn1 k=k5 localUncertaintyLimit=5,0 @@ -248,13 +248,13 @@ run error get t=txn1 k=k6 localUncertaintyLimit=5,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run error scan t=txn1 k=k6 localUncertaintyLimit=5,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run ok get t=txn1 k=k7 localUncertaintyLimit=5,0 @@ -270,13 +270,13 @@ run error get t=txn1 k=k8 localUncertaintyLimit=5,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run error scan t=txn1 k=k8 localUncertaintyLimit=5,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run ok @@ -299,13 +299,13 @@ run error get t=txn2 k=k2 localUncertaintyLimit=5,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn2 k=k2 localUncertaintyLimit=5,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run ok get t=txn2 k=k3 localUncertaintyLimit=5,0 @@ -321,13 +321,13 @@ run error get t=txn2 k=k4 localUncertaintyLimit=5,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn2 k=k4 localUncertaintyLimit=5,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run ok get t=txn2 k=k5 localUncertaintyLimit=5,0 @@ -343,13 +343,13 @@ run error get t=txn2 k=k6 localUncertaintyLimit=5,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn2 k=k6 localUncertaintyLimit=5,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run ok get t=txn2 k=k7 localUncertaintyLimit=5,0 @@ -365,13 +365,13 @@ run error get t=txn2 k=k8 localUncertaintyLimit=5,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn2 k=k8 localUncertaintyLimit=5,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run ok @@ -394,13 +394,13 @@ run error get t=txn3 k=k2 localUncertaintyLimit=5,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn3 k=k2 localUncertaintyLimit=5,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok get t=txn3 k=k3 localUncertaintyLimit=5,0 @@ -416,13 +416,13 @@ run error get t=txn3 k=k4 localUncertaintyLimit=5,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn3 k=k4 localUncertaintyLimit=5,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok get t=txn3 k=k5 localUncertaintyLimit=5,0 @@ -438,13 +438,13 @@ run error get t=txn3 k=k6 localUncertaintyLimit=5,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn3 k=k6 localUncertaintyLimit=5,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok get t=txn3 k=k7 localUncertaintyLimit=5,0 @@ -460,13 +460,13 @@ run error get t=txn3 k=k8 localUncertaintyLimit=5,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn3 k=k8 localUncertaintyLimit=5,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok @@ -489,13 +489,13 @@ run error get t=txn4 k=k2 localUncertaintyLimit=5,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn4 k=k2 localUncertaintyLimit=5,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok get t=txn4 k=k3 localUncertaintyLimit=5,0 @@ -511,13 +511,13 @@ run error get t=txn4 k=k4 localUncertaintyLimit=5,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn4 k=k4 localUncertaintyLimit=5,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok get t=txn4 k=k5 localUncertaintyLimit=5,0 @@ -533,13 +533,13 @@ run error get t=txn4 k=k6 localUncertaintyLimit=5,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn4 k=k6 localUncertaintyLimit=5,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok get t=txn4 k=k7 localUncertaintyLimit=5,0 @@ -555,13 +555,13 @@ run error get t=txn4 k=k8 localUncertaintyLimit=5,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn4 k=k8 localUncertaintyLimit=5,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -574,97 +574,97 @@ run error get t=txn5 k=k1 localUncertaintyLimit=10,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k1 localUncertaintyLimit=10,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error get t=txn5 k=k2 localUncertaintyLimit=10,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k2 localUncertaintyLimit=10,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error get t=txn5 k=k3 localUncertaintyLimit=10,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k3 localUncertaintyLimit=10,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error get t=txn5 k=k4 localUncertaintyLimit=10,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k4 localUncertaintyLimit=10,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error get t=txn5 k=k5 localUncertaintyLimit=10,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k5 localUncertaintyLimit=10,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error get t=txn5 k=k6 localUncertaintyLimit=10,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k6 localUncertaintyLimit=10,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error get t=txn5 k=k7 localUncertaintyLimit=10,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k7 localUncertaintyLimit=10,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error get t=txn5 k=k8 localUncertaintyLimit=10,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k8 localUncertaintyLimit=10,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run ok @@ -677,97 +677,97 @@ run error get t=txn6 k=k1 localUncertaintyLimit=10,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k1 localUncertaintyLimit=10,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k2 localUncertaintyLimit=10,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k2 localUncertaintyLimit=10,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k3 localUncertaintyLimit=10,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k3 localUncertaintyLimit=10,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k4 localUncertaintyLimit=10,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k4 localUncertaintyLimit=10,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k5 localUncertaintyLimit=10,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k5 localUncertaintyLimit=10,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k6 localUncertaintyLimit=10,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k6 localUncertaintyLimit=10,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k7 localUncertaintyLimit=10,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k7 localUncertaintyLimit=10,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k8 localUncertaintyLimit=10,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k8 localUncertaintyLimit=10,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok @@ -780,97 +780,97 @@ run error get t=txn7 k=k1 localUncertaintyLimit=10,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k1 localUncertaintyLimit=10,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k2 localUncertaintyLimit=10,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k2 localUncertaintyLimit=10,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k3 localUncertaintyLimit=10,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k3 localUncertaintyLimit=10,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k4 localUncertaintyLimit=10,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k4 localUncertaintyLimit=10,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k5 localUncertaintyLimit=10,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k5 localUncertaintyLimit=10,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k6 localUncertaintyLimit=10,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k6 localUncertaintyLimit=10,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k7 localUncertaintyLimit=10,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k7 localUncertaintyLimit=10,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k8 localUncertaintyLimit=10,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k8 localUncertaintyLimit=10,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -883,97 +883,97 @@ run error get t=txn8 k=k1 localUncertaintyLimit=15,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k1 localUncertaintyLimit=15,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k2 localUncertaintyLimit=15,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k2 localUncertaintyLimit=15,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k3 localUncertaintyLimit=15,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k3 localUncertaintyLimit=15,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k4 localUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k4 localUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k5 localUncertaintyLimit=15,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k5 localUncertaintyLimit=15,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k6 localUncertaintyLimit=15,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k6 localUncertaintyLimit=15,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k7 localUncertaintyLimit=15,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k7 localUncertaintyLimit=15,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k8 localUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k8 localUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok @@ -986,97 +986,97 @@ run error get t=txn9 k=k1 localUncertaintyLimit=15,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k1 localUncertaintyLimit=15,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k2 localUncertaintyLimit=15,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k2 localUncertaintyLimit=15,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k3 localUncertaintyLimit=15,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k3 localUncertaintyLimit=15,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k4 localUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k4 localUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k5 localUncertaintyLimit=15,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k5 localUncertaintyLimit=15,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k6 localUncertaintyLimit=15,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k6 localUncertaintyLimit=15,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k7 localUncertaintyLimit=15,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k7 localUncertaintyLimit=15,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k8 localUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k8 localUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -1089,97 +1089,97 @@ run error get t=txn10 k=k1 localUncertaintyLimit=20,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k1 localUncertaintyLimit=20,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn10 k=k2 localUncertaintyLimit=20,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k2 localUncertaintyLimit=20,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn10 k=k3 localUncertaintyLimit=20,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k3 localUncertaintyLimit=20,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn10 k=k4 localUncertaintyLimit=20,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k4 localUncertaintyLimit=20,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn10 k=k5 localUncertaintyLimit=20,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k5 localUncertaintyLimit=20,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn10 k=k6 localUncertaintyLimit=20,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k6 localUncertaintyLimit=20,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn10 k=k7 localUncertaintyLimit=20,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k7 localUncertaintyLimit=20,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn10 k=k8 localUncertaintyLimit=20,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k8 localUncertaintyLimit=20,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -1299,25 +1299,25 @@ run error get t=txn12 k=k3 localUncertaintyLimit=10,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn12 k=k3 localUncertaintyLimit=10,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn12 k=k4 localUncertaintyLimit=10,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn12 k=k4 localUncertaintyLimit=10,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok get t=txn12 k=k5 localUncertaintyLimit=10,0 @@ -1343,25 +1343,25 @@ run error get t=txn12 k=k7 localUncertaintyLimit=10,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn12 k=k7 localUncertaintyLimit=10,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn12 k=k8 localUncertaintyLimit=10,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn12 k=k8 localUncertaintyLimit=10,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok @@ -1394,25 +1394,25 @@ run error get t=txn13 k=k3 localUncertaintyLimit=10,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn13 k=k3 localUncertaintyLimit=10,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn13 k=k4 localUncertaintyLimit=10,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn13 k=k4 localUncertaintyLimit=10,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok get t=txn13 k=k5 localUncertaintyLimit=10,0 @@ -1438,25 +1438,25 @@ run error get t=txn13 k=k7 localUncertaintyLimit=10,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn13 k=k7 localUncertaintyLimit=10,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn13 k=k8 localUncertaintyLimit=10,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn13 k=k8 localUncertaintyLimit=10,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -1489,25 +1489,25 @@ run error get t=txn14 k=k3 localUncertaintyLimit=15,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn14 k=k3 localUncertaintyLimit=15,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn14 k=k4 localUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn14 k=k4 localUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok get t=txn14 k=k5 localUncertaintyLimit=15,0 @@ -1533,25 +1533,25 @@ run error get t=txn14 k=k7 localUncertaintyLimit=15,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn14 k=k7 localUncertaintyLimit=15,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn14 k=k8 localUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn14 k=k8 localUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok @@ -1584,25 +1584,25 @@ run error get t=txn15 k=k3 localUncertaintyLimit=15,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn15 k=k3 localUncertaintyLimit=15,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn15 k=k4 localUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn15 k=k4 localUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok get t=txn15 k=k5 localUncertaintyLimit=15,0 @@ -1628,25 +1628,25 @@ run error get t=txn15 k=k7 localUncertaintyLimit=15,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn15 k=k7 localUncertaintyLimit=15,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn15 k=k8 localUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn15 k=k8 localUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -1659,97 +1659,97 @@ run error get t=txn16 k=k1 localUncertaintyLimit=20,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k1 localUncertaintyLimit=20,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn16 k=k2 localUncertaintyLimit=20,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k2 localUncertaintyLimit=20,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn16 k=k3 localUncertaintyLimit=20,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k3 localUncertaintyLimit=20,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn16 k=k4 localUncertaintyLimit=20,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k4 localUncertaintyLimit=20,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn16 k=k5 localUncertaintyLimit=20,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k5 localUncertaintyLimit=20,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn16 k=k6 localUncertaintyLimit=20,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k6 localUncertaintyLimit=20,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn16 k=k7 localUncertaintyLimit=20,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k7 localUncertaintyLimit=20,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn16 k=k8 localUncertaintyLimit=20,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k8 localUncertaintyLimit=20,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -1782,25 +1782,25 @@ run error get t=txn17 k=k3 localUncertaintyLimit=15,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn17 k=k3 localUncertaintyLimit=15,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn17 k=k4 localUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn17 k=k4 localUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok get t=txn17 k=k5 localUncertaintyLimit=15,0 @@ -1826,25 +1826,25 @@ run error get t=txn17 k=k7 localUncertaintyLimit=15,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn17 k=k7 localUncertaintyLimit=15,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn17 k=k8 localUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn17 k=k8 localUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok @@ -1877,25 +1877,25 @@ run error get t=txn18 k=k3 localUncertaintyLimit=15,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn18 k=k3 localUncertaintyLimit=15,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn18 k=k4 localUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn18 k=k4 localUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok get t=txn18 k=k5 localUncertaintyLimit=15,0 @@ -1921,25 +1921,25 @@ run error get t=txn18 k=k7 localUncertaintyLimit=15,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn18 k=k7 localUncertaintyLimit=15,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn18 k=k8 localUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn18 k=k8 localUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -1952,97 +1952,97 @@ run error get t=txn19 k=k1 localUncertaintyLimit=20,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k1 localUncertaintyLimit=20,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn19 k=k2 localUncertaintyLimit=20,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k2 localUncertaintyLimit=20,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn19 k=k3 localUncertaintyLimit=20,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k3 localUncertaintyLimit=20,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn19 k=k4 localUncertaintyLimit=20,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k4 localUncertaintyLimit=20,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn19 k=k5 localUncertaintyLimit=20,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k5 localUncertaintyLimit=20,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn19 k=k6 localUncertaintyLimit=20,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k6 localUncertaintyLimit=20,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn19 k=k7 localUncertaintyLimit=20,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k7 localUncertaintyLimit=20,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn19 k=k8 localUncertaintyLimit=20,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k8 localUncertaintyLimit=20,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -2095,49 +2095,49 @@ run error get t=txn20 k=k5 localUncertaintyLimit=20,0 ---- get: "k5" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k5" +error: (*kvpb.WriteIntentError:) conflicting intents on "k5" run error scan t=txn20 k=k5 localUncertaintyLimit=20,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k5" +error: (*kvpb.WriteIntentError:) conflicting intents on "k5" run error get t=txn20 k=k6 localUncertaintyLimit=20,0 ---- get: "k6" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k6" +error: (*kvpb.WriteIntentError:) conflicting intents on "k6" run error scan t=txn20 k=k6 localUncertaintyLimit=20,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k6" +error: (*kvpb.WriteIntentError:) conflicting intents on "k6" run error get t=txn20 k=k7 localUncertaintyLimit=20,0 ---- get: "k7" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k7" +error: (*kvpb.WriteIntentError:) conflicting intents on "k7" run error scan t=txn20 k=k7 localUncertaintyLimit=20,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k7" +error: (*kvpb.WriteIntentError:) conflicting intents on "k7" run error get t=txn20 k=k8 localUncertaintyLimit=20,0 ---- get: "k8" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k8" +error: (*kvpb.WriteIntentError:) conflicting intents on "k8" run error scan t=txn20 k=k8 localUncertaintyLimit=20,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k8" +error: (*kvpb.WriteIntentError:) conflicting intents on "k8" # A subset of the previous test cases, but with non-transactional reads: # @@ -2246,13 +2246,13 @@ run error get k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 @@ -2268,13 +2268,13 @@ run error get k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 @@ -2290,13 +2290,13 @@ run error get k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 @@ -2312,13 +2312,13 @@ run error get k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k1 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 @@ -2334,13 +2334,13 @@ run error get k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 @@ -2356,13 +2356,13 @@ run error get k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 @@ -2378,13 +2378,13 @@ run error get k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 @@ -2400,301 +2400,301 @@ run error get k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 (local=5.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k1 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k1 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k2 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k2 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k3 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k3 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k4 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k5 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k5 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k6 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k6 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k7 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k7 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k8 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k1 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 @@ -2800,25 +2800,25 @@ run error get k=k3 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k3 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k4 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k5 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 @@ -2844,121 +2844,121 @@ run error get k=k7 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k7 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k8 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k1 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k1 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k2 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k2 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k3 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k3 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k4 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k5 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k5 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k6 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k6 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k7 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k7 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k8 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 (local=10.000000000,0) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k1 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 @@ -3004,46 +3004,46 @@ run error get k=k5 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k5" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k5" +error: (*kvpb.WriteIntentError:) conflicting intents on "k5" run error scan k=k5 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k5" +error: (*kvpb.WriteIntentError:) conflicting intents on "k5" run error get k=k6 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k6" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k6" +error: (*kvpb.WriteIntentError:) conflicting intents on "k6" run error scan k=k6 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k6" +error: (*kvpb.WriteIntentError:) conflicting intents on "k6" run error get k=k7 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k7" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k7" +error: (*kvpb.WriteIntentError:) conflicting intents on "k7" run error scan k=k7 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k7" +error: (*kvpb.WriteIntentError:) conflicting intents on "k7" run error get k=k8 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k8" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k8" +error: (*kvpb.WriteIntentError:) conflicting intents on "k8" run error scan k=k8 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k8" +error: (*kvpb.WriteIntentError:) conflicting intents on "k8" diff --git a/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit_and_synthetic_timestamps b/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit_and_synthetic_timestamps index 69d74bf72835..14b682b3db0b 100644 --- a/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit_and_synthetic_timestamps +++ b/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit_and_synthetic_timestamps @@ -204,13 +204,13 @@ run error get t=txn1 k=k2 localUncertaintyLimit=5,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run error scan t=txn1 k=k2 localUncertaintyLimit=5,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run ok get t=txn1 k=k3 localUncertaintyLimit=5,0 @@ -226,13 +226,13 @@ run error get t=txn1 k=k4 localUncertaintyLimit=5,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run error scan t=txn1 k=k4 localUncertaintyLimit=5,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run ok get t=txn1 k=k5 localUncertaintyLimit=5,0 @@ -248,13 +248,13 @@ run error get t=txn1 k=k6 localUncertaintyLimit=5,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run error scan t=txn1 k=k6 localUncertaintyLimit=5,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run ok get t=txn1 k=k7 localUncertaintyLimit=5,0 @@ -270,13 +270,13 @@ run error get t=txn1 k=k8 localUncertaintyLimit=5,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run error scan t=txn1 k=k8 localUncertaintyLimit=5,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run ok @@ -299,13 +299,13 @@ run error get t=txn2 k=k2 localUncertaintyLimit=5,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn2 k=k2 localUncertaintyLimit=5,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run ok get t=txn2 k=k3 localUncertaintyLimit=5,0 @@ -321,13 +321,13 @@ run error get t=txn2 k=k4 localUncertaintyLimit=5,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn2 k=k4 localUncertaintyLimit=5,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run ok get t=txn2 k=k5 localUncertaintyLimit=5,0 @@ -343,13 +343,13 @@ run error get t=txn2 k=k6 localUncertaintyLimit=5,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn2 k=k6 localUncertaintyLimit=5,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run ok get t=txn2 k=k7 localUncertaintyLimit=5,0 @@ -365,13 +365,13 @@ run error get t=txn2 k=k8 localUncertaintyLimit=5,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn2 k=k8 localUncertaintyLimit=5,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run ok @@ -394,37 +394,37 @@ run error get t=txn3 k=k2 localUncertaintyLimit=5,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn3 k=k2 localUncertaintyLimit=5,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn3 k=k3 localUncertaintyLimit=5,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn3 k=k3 localUncertaintyLimit=5,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn3 k=k4 localUncertaintyLimit=5,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn3 k=k4 localUncertaintyLimit=5,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok get t=txn3 k=k5 localUncertaintyLimit=5,0 @@ -440,37 +440,37 @@ run error get t=txn3 k=k6 localUncertaintyLimit=5,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn3 k=k6 localUncertaintyLimit=5,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn3 k=k7 localUncertaintyLimit=5,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn3 k=k7 localUncertaintyLimit=5,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn3 k=k8 localUncertaintyLimit=5,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn3 k=k8 localUncertaintyLimit=5,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok @@ -493,37 +493,37 @@ run error get t=txn4 k=k2 localUncertaintyLimit=5,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn4 k=k2 localUncertaintyLimit=5,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn4 k=k3 localUncertaintyLimit=5,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn4 k=k3 localUncertaintyLimit=5,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn4 k=k4 localUncertaintyLimit=5,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn4 k=k4 localUncertaintyLimit=5,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok get t=txn4 k=k5 localUncertaintyLimit=5,0 @@ -539,37 +539,37 @@ run error get t=txn4 k=k6 localUncertaintyLimit=5,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn4 k=k6 localUncertaintyLimit=5,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn4 k=k7 localUncertaintyLimit=5,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn4 k=k7 localUncertaintyLimit=5,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn4 k=k8 localUncertaintyLimit=5,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn4 k=k8 localUncertaintyLimit=5,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -582,97 +582,97 @@ run error get t=txn5 k=k1 localUncertaintyLimit=10,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k1 localUncertaintyLimit=10,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error get t=txn5 k=k2 localUncertaintyLimit=10,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k2 localUncertaintyLimit=10,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error get t=txn5 k=k3 localUncertaintyLimit=10,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k3 localUncertaintyLimit=10,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error get t=txn5 k=k4 localUncertaintyLimit=10,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k4 localUncertaintyLimit=10,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error get t=txn5 k=k5 localUncertaintyLimit=10,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k5 localUncertaintyLimit=10,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error get t=txn5 k=k6 localUncertaintyLimit=10,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k6 localUncertaintyLimit=10,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error get t=txn5 k=k7 localUncertaintyLimit=10,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k7 localUncertaintyLimit=10,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error get t=txn5 k=k8 localUncertaintyLimit=10,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k8 localUncertaintyLimit=10,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run ok @@ -685,97 +685,97 @@ run error get t=txn6 k=k1 localUncertaintyLimit=10,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k1 localUncertaintyLimit=10,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k2 localUncertaintyLimit=10,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k2 localUncertaintyLimit=10,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k3 localUncertaintyLimit=10,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k3 localUncertaintyLimit=10,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k4 localUncertaintyLimit=10,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k4 localUncertaintyLimit=10,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k5 localUncertaintyLimit=10,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k5 localUncertaintyLimit=10,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k6 localUncertaintyLimit=10,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k6 localUncertaintyLimit=10,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k7 localUncertaintyLimit=10,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k7 localUncertaintyLimit=10,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k8 localUncertaintyLimit=10,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k8 localUncertaintyLimit=10,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok @@ -788,97 +788,97 @@ run error get t=txn7 k=k1 localUncertaintyLimit=10,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k1 localUncertaintyLimit=10,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k2 localUncertaintyLimit=10,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k2 localUncertaintyLimit=10,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k3 localUncertaintyLimit=10,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k3 localUncertaintyLimit=10,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k4 localUncertaintyLimit=10,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k4 localUncertaintyLimit=10,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k5 localUncertaintyLimit=10,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k5 localUncertaintyLimit=10,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k6 localUncertaintyLimit=10,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k6 localUncertaintyLimit=10,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k7 localUncertaintyLimit=10,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k7 localUncertaintyLimit=10,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k8 localUncertaintyLimit=10,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k8 localUncertaintyLimit=10,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -891,97 +891,97 @@ run error get t=txn8 k=k1 localUncertaintyLimit=15,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k1 localUncertaintyLimit=15,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k2 localUncertaintyLimit=15,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k2 localUncertaintyLimit=15,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k3 localUncertaintyLimit=15,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k3 localUncertaintyLimit=15,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k4 localUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k4 localUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k5 localUncertaintyLimit=15,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k5 localUncertaintyLimit=15,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k6 localUncertaintyLimit=15,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k6 localUncertaintyLimit=15,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k7 localUncertaintyLimit=15,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k7 localUncertaintyLimit=15,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k8 localUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k8 localUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok @@ -994,97 +994,97 @@ run error get t=txn9 k=k1 localUncertaintyLimit=15,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k1 localUncertaintyLimit=15,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k2 localUncertaintyLimit=15,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k2 localUncertaintyLimit=15,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k3 localUncertaintyLimit=15,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k3 localUncertaintyLimit=15,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k4 localUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k4 localUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k5 localUncertaintyLimit=15,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k5 localUncertaintyLimit=15,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k6 localUncertaintyLimit=15,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k6 localUncertaintyLimit=15,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k7 localUncertaintyLimit=15,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k7 localUncertaintyLimit=15,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k8 localUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k8 localUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -1097,97 +1097,97 @@ run error get t=txn10 k=k1 localUncertaintyLimit=20,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k1 localUncertaintyLimit=20,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn10 k=k2 localUncertaintyLimit=20,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k2 localUncertaintyLimit=20,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn10 k=k3 localUncertaintyLimit=20,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k3 localUncertaintyLimit=20,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn10 k=k4 localUncertaintyLimit=20,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k4 localUncertaintyLimit=20,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn10 k=k5 localUncertaintyLimit=20,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k5 localUncertaintyLimit=20,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn10 k=k6 localUncertaintyLimit=20,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k6 localUncertaintyLimit=20,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn10 k=k7 localUncertaintyLimit=20,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k7 localUncertaintyLimit=20,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn10 k=k8 localUncertaintyLimit=20,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k8 localUncertaintyLimit=20,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -1307,25 +1307,25 @@ run error get t=txn12 k=k3 localUncertaintyLimit=10,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn12 k=k3 localUncertaintyLimit=10,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn12 k=k4 localUncertaintyLimit=10,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn12 k=k4 localUncertaintyLimit=10,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok get t=txn12 k=k5 localUncertaintyLimit=10,0 @@ -1351,25 +1351,25 @@ run error get t=txn12 k=k7 localUncertaintyLimit=10,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn12 k=k7 localUncertaintyLimit=10,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn12 k=k8 localUncertaintyLimit=10,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn12 k=k8 localUncertaintyLimit=10,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok @@ -1402,25 +1402,25 @@ run error get t=txn13 k=k3 localUncertaintyLimit=10,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn13 k=k3 localUncertaintyLimit=10,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn13 k=k4 localUncertaintyLimit=10,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn13 k=k4 localUncertaintyLimit=10,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok get t=txn13 k=k5 localUncertaintyLimit=10,0 @@ -1446,25 +1446,25 @@ run error get t=txn13 k=k7 localUncertaintyLimit=10,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn13 k=k7 localUncertaintyLimit=10,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn13 k=k8 localUncertaintyLimit=10,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn13 k=k8 localUncertaintyLimit=10,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -1497,25 +1497,25 @@ run error get t=txn14 k=k3 localUncertaintyLimit=15,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn14 k=k3 localUncertaintyLimit=15,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn14 k=k4 localUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn14 k=k4 localUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok get t=txn14 k=k5 localUncertaintyLimit=15,0 @@ -1541,25 +1541,25 @@ run error get t=txn14 k=k7 localUncertaintyLimit=15,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn14 k=k7 localUncertaintyLimit=15,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn14 k=k8 localUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn14 k=k8 localUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok @@ -1592,25 +1592,25 @@ run error get t=txn15 k=k3 localUncertaintyLimit=15,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn15 k=k3 localUncertaintyLimit=15,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn15 k=k4 localUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn15 k=k4 localUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok get t=txn15 k=k5 localUncertaintyLimit=15,0 @@ -1636,25 +1636,25 @@ run error get t=txn15 k=k7 localUncertaintyLimit=15,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn15 k=k7 localUncertaintyLimit=15,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn15 k=k8 localUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn15 k=k8 localUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -1667,97 +1667,97 @@ run error get t=txn16 k=k1 localUncertaintyLimit=20,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k1 localUncertaintyLimit=20,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn16 k=k2 localUncertaintyLimit=20,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k2 localUncertaintyLimit=20,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn16 k=k3 localUncertaintyLimit=20,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k3 localUncertaintyLimit=20,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn16 k=k4 localUncertaintyLimit=20,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k4 localUncertaintyLimit=20,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn16 k=k5 localUncertaintyLimit=20,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k5 localUncertaintyLimit=20,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn16 k=k6 localUncertaintyLimit=20,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k6 localUncertaintyLimit=20,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn16 k=k7 localUncertaintyLimit=20,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k7 localUncertaintyLimit=20,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn16 k=k8 localUncertaintyLimit=20,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k8 localUncertaintyLimit=20,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -1790,25 +1790,25 @@ run error get t=txn17 k=k3 localUncertaintyLimit=15,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn17 k=k3 localUncertaintyLimit=15,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn17 k=k4 localUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn17 k=k4 localUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok get t=txn17 k=k5 localUncertaintyLimit=15,0 @@ -1834,25 +1834,25 @@ run error get t=txn17 k=k7 localUncertaintyLimit=15,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn17 k=k7 localUncertaintyLimit=15,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn17 k=k8 localUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn17 k=k8 localUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok @@ -1885,25 +1885,25 @@ run error get t=txn18 k=k3 localUncertaintyLimit=15,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn18 k=k3 localUncertaintyLimit=15,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn18 k=k4 localUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn18 k=k4 localUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok get t=txn18 k=k5 localUncertaintyLimit=15,0 @@ -1929,25 +1929,25 @@ run error get t=txn18 k=k7 localUncertaintyLimit=15,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn18 k=k7 localUncertaintyLimit=15,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn18 k=k8 localUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn18 k=k8 localUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -1960,97 +1960,97 @@ run error get t=txn19 k=k1 localUncertaintyLimit=20,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k1 localUncertaintyLimit=20,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn19 k=k2 localUncertaintyLimit=20,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k2 localUncertaintyLimit=20,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn19 k=k3 localUncertaintyLimit=20,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k3 localUncertaintyLimit=20,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn19 k=k4 localUncertaintyLimit=20,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k4 localUncertaintyLimit=20,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn19 k=k5 localUncertaintyLimit=20,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k5 localUncertaintyLimit=20,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn19 k=k6 localUncertaintyLimit=20,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k6 localUncertaintyLimit=20,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn19 k=k7 localUncertaintyLimit=20,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k7 localUncertaintyLimit=20,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn19 k=k8 localUncertaintyLimit=20,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k8 localUncertaintyLimit=20,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -2103,49 +2103,49 @@ run error get t=txn20 k=k5 localUncertaintyLimit=20,0 ---- get: "k5" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k5" +error: (*kvpb.WriteIntentError:) conflicting intents on "k5" run error scan t=txn20 k=k5 localUncertaintyLimit=20,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k5" +error: (*kvpb.WriteIntentError:) conflicting intents on "k5" run error get t=txn20 k=k6 localUncertaintyLimit=20,0 ---- get: "k6" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k6" +error: (*kvpb.WriteIntentError:) conflicting intents on "k6" run error scan t=txn20 k=k6 localUncertaintyLimit=20,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k6" +error: (*kvpb.WriteIntentError:) conflicting intents on "k6" run error get t=txn20 k=k7 localUncertaintyLimit=20,0 ---- get: "k7" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k7" +error: (*kvpb.WriteIntentError:) conflicting intents on "k7" run error scan t=txn20 k=k7 localUncertaintyLimit=20,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k7" +error: (*kvpb.WriteIntentError:) conflicting intents on "k7" run error get t=txn20 k=k8 localUncertaintyLimit=20,0 ---- get: "k8" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k8" +error: (*kvpb.WriteIntentError:) conflicting intents on "k8" run error scan t=txn20 k=k8 localUncertaintyLimit=20,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k8" +error: (*kvpb.WriteIntentError:) conflicting intents on "k8" # A subset of the previous test cases, but with non-transactional reads: # @@ -2254,13 +2254,13 @@ run error get k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 @@ -2276,13 +2276,13 @@ run error get k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 @@ -2298,13 +2298,13 @@ run error get k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 @@ -2320,13 +2320,13 @@ run error get k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k1 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 @@ -2342,37 +2342,37 @@ run error get k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 @@ -2388,325 +2388,325 @@ run error get k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k1 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k1 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k2 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k2 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k3 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k3 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k4 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k5 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k5 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k6 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k6 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k7 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k7 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k8 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k1 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 @@ -2812,25 +2812,25 @@ run error get k=k3 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k3 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k4 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k5 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 @@ -2856,121 +2856,121 @@ run error get k=k7 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k7 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k8 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k1 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k1 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k2 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k2 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k3 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k3 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k4 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k5 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k5 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k6 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k6 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k7 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k7 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k8 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? (local=0,1) within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k1 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 @@ -3016,46 +3016,46 @@ run error get k=k5 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k5" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k5" +error: (*kvpb.WriteIntentError:) conflicting intents on "k5" run error scan k=k5 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k5" +error: (*kvpb.WriteIntentError:) conflicting intents on "k5" run error get k=k6 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k6" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k6" +error: (*kvpb.WriteIntentError:) conflicting intents on "k6" run error scan k=k6 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k6" +error: (*kvpb.WriteIntentError:) conflicting intents on "k6" run error get k=k7 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k7" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k7" +error: (*kvpb.WriteIntentError:) conflicting intents on "k7" run error scan k=k7 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k7" +error: (*kvpb.WriteIntentError:) conflicting intents on "k7" run error get k=k8 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k8" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k8" +error: (*kvpb.WriteIntentError:) conflicting intents on "k8" run error scan k=k8 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k8" +error: (*kvpb.WriteIntentError:) conflicting intents on "k8" diff --git a/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit_disable_local_timestamps b/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit_disable_local_timestamps index e5def62954ea..034cdde49526 100644 --- a/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit_disable_local_timestamps +++ b/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit_disable_local_timestamps @@ -542,97 +542,97 @@ run error get t=txn5 k=k1 localUncertaintyLimit=10,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k1 localUncertaintyLimit=10,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error get t=txn5 k=k2 localUncertaintyLimit=10,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k2 localUncertaintyLimit=10,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error get t=txn5 k=k3 localUncertaintyLimit=10,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k3 localUncertaintyLimit=10,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error get t=txn5 k=k4 localUncertaintyLimit=10,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k4 localUncertaintyLimit=10,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error get t=txn5 k=k5 localUncertaintyLimit=10,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k5 localUncertaintyLimit=10,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error get t=txn5 k=k6 localUncertaintyLimit=10,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k6 localUncertaintyLimit=10,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error get t=txn5 k=k7 localUncertaintyLimit=10,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k7 localUncertaintyLimit=10,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error get t=txn5 k=k8 localUncertaintyLimit=10,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k8 localUncertaintyLimit=10,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run ok @@ -645,97 +645,97 @@ run error get t=txn6 k=k1 localUncertaintyLimit=10,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k1 localUncertaintyLimit=10,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k2 localUncertaintyLimit=10,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k2 localUncertaintyLimit=10,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k3 localUncertaintyLimit=10,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k3 localUncertaintyLimit=10,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k4 localUncertaintyLimit=10,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k4 localUncertaintyLimit=10,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k5 localUncertaintyLimit=10,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k5 localUncertaintyLimit=10,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k6 localUncertaintyLimit=10,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k6 localUncertaintyLimit=10,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k7 localUncertaintyLimit=10,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k7 localUncertaintyLimit=10,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k8 localUncertaintyLimit=10,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k8 localUncertaintyLimit=10,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok @@ -748,97 +748,97 @@ run error get t=txn7 k=k1 localUncertaintyLimit=10,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k1 localUncertaintyLimit=10,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k2 localUncertaintyLimit=10,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k2 localUncertaintyLimit=10,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k3 localUncertaintyLimit=10,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k3 localUncertaintyLimit=10,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k4 localUncertaintyLimit=10,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k4 localUncertaintyLimit=10,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k5 localUncertaintyLimit=10,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k5 localUncertaintyLimit=10,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k6 localUncertaintyLimit=10,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k6 localUncertaintyLimit=10,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k7 localUncertaintyLimit=10,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k7 localUncertaintyLimit=10,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k8 localUncertaintyLimit=10,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k8 localUncertaintyLimit=10,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -851,97 +851,97 @@ run error get t=txn8 k=k1 localUncertaintyLimit=15,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k1 localUncertaintyLimit=15,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k2 localUncertaintyLimit=15,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k2 localUncertaintyLimit=15,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k3 localUncertaintyLimit=15,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k3 localUncertaintyLimit=15,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k4 localUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k4 localUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k5 localUncertaintyLimit=15,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k5 localUncertaintyLimit=15,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k6 localUncertaintyLimit=15,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k6 localUncertaintyLimit=15,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k7 localUncertaintyLimit=15,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k7 localUncertaintyLimit=15,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k8 localUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k8 localUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok @@ -954,97 +954,97 @@ run error get t=txn9 k=k1 localUncertaintyLimit=15,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k1 localUncertaintyLimit=15,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k2 localUncertaintyLimit=15,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k2 localUncertaintyLimit=15,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k3 localUncertaintyLimit=15,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k3 localUncertaintyLimit=15,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k4 localUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k4 localUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k5 localUncertaintyLimit=15,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k5 localUncertaintyLimit=15,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k6 localUncertaintyLimit=15,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k6 localUncertaintyLimit=15,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k7 localUncertaintyLimit=15,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k7 localUncertaintyLimit=15,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k8 localUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k8 localUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -1057,97 +1057,97 @@ run error get t=txn10 k=k1 localUncertaintyLimit=20,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k1 localUncertaintyLimit=20,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn10 k=k2 localUncertaintyLimit=20,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k2 localUncertaintyLimit=20,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn10 k=k3 localUncertaintyLimit=20,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k3 localUncertaintyLimit=20,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn10 k=k4 localUncertaintyLimit=20,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k4 localUncertaintyLimit=20,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn10 k=k5 localUncertaintyLimit=20,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k5 localUncertaintyLimit=20,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn10 k=k6 localUncertaintyLimit=20,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k6 localUncertaintyLimit=20,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn10 k=k7 localUncertaintyLimit=20,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k7 localUncertaintyLimit=20,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn10 k=k8 localUncertaintyLimit=20,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k8 localUncertaintyLimit=20,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -1595,97 +1595,97 @@ run error get t=txn16 k=k1 localUncertaintyLimit=20,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k1 localUncertaintyLimit=20,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn16 k=k2 localUncertaintyLimit=20,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k2 localUncertaintyLimit=20,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn16 k=k3 localUncertaintyLimit=20,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k3 localUncertaintyLimit=20,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn16 k=k4 localUncertaintyLimit=20,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k4 localUncertaintyLimit=20,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn16 k=k5 localUncertaintyLimit=20,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k5 localUncertaintyLimit=20,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn16 k=k6 localUncertaintyLimit=20,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k6 localUncertaintyLimit=20,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn16 k=k7 localUncertaintyLimit=20,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k7 localUncertaintyLimit=20,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn16 k=k8 localUncertaintyLimit=20,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k8 localUncertaintyLimit=20,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -1872,97 +1872,97 @@ run error get t=txn19 k=k1 localUncertaintyLimit=20,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k1 localUncertaintyLimit=20,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn19 k=k2 localUncertaintyLimit=20,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k2 localUncertaintyLimit=20,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn19 k=k3 localUncertaintyLimit=20,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k3 localUncertaintyLimit=20,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn19 k=k4 localUncertaintyLimit=20,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k4 localUncertaintyLimit=20,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn19 k=k5 localUncertaintyLimit=20,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k5 localUncertaintyLimit=20,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn19 k=k6 localUncertaintyLimit=20,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k6 localUncertaintyLimit=20,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn19 k=k7 localUncertaintyLimit=20,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k7 localUncertaintyLimit=20,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn19 k=k8 localUncertaintyLimit=20,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k8 localUncertaintyLimit=20,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -2015,49 +2015,49 @@ run error get t=txn20 k=k5 localUncertaintyLimit=20,0 ---- get: "k5" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k5" +error: (*kvpb.WriteIntentError:) conflicting intents on "k5" run error scan t=txn20 k=k5 localUncertaintyLimit=20,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k5" +error: (*kvpb.WriteIntentError:) conflicting intents on "k5" run error get t=txn20 k=k6 localUncertaintyLimit=20,0 ---- get: "k6" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k6" +error: (*kvpb.WriteIntentError:) conflicting intents on "k6" run error scan t=txn20 k=k6 localUncertaintyLimit=20,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k6" +error: (*kvpb.WriteIntentError:) conflicting intents on "k6" run error get t=txn20 k=k7 localUncertaintyLimit=20,0 ---- get: "k7" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k7" +error: (*kvpb.WriteIntentError:) conflicting intents on "k7" run error scan t=txn20 k=k7 localUncertaintyLimit=20,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k7" +error: (*kvpb.WriteIntentError:) conflicting intents on "k7" run error get t=txn20 k=k8 localUncertaintyLimit=20,0 ---- get: "k8" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k8" +error: (*kvpb.WriteIntentError:) conflicting intents on "k8" run error scan t=txn20 k=k8 localUncertaintyLimit=20,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k8" +error: (*kvpb.WriteIntentError:) conflicting intents on "k8" # A subset of the previous test cases, but with non-transactional reads: # @@ -2316,289 +2316,289 @@ run error get k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k1 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k1 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k2 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k2 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k3 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k3 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k4 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k5 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k5 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k6 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k6 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k7 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k7 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k8 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k1 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 @@ -2764,97 +2764,97 @@ run error get k=k1 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k1 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k2 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k2 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k3 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k3 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k4 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k5 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k5" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k5 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k6 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k6 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k7 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k7 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k8 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k1 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 @@ -2900,46 +2900,46 @@ run error get k=k5 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k5" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k5" +error: (*kvpb.WriteIntentError:) conflicting intents on "k5" run error scan k=k5 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k5"-"k5\x00" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k5" +error: (*kvpb.WriteIntentError:) conflicting intents on "k5" run error get k=k6 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k6" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k6" +error: (*kvpb.WriteIntentError:) conflicting intents on "k6" run error scan k=k6 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k6" +error: (*kvpb.WriteIntentError:) conflicting intents on "k6" run error get k=k7 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k7" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k7" +error: (*kvpb.WriteIntentError:) conflicting intents on "k7" run error scan k=k7 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k7" +error: (*kvpb.WriteIntentError:) conflicting intents on "k7" run error get k=k8 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k8" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k8" +error: (*kvpb.WriteIntentError:) conflicting intents on "k8" run error scan k=k8 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.WriteIntentError:) conflicting intents on "k8" +error: (*kvpb.WriteIntentError:) conflicting intents on "k8" diff --git a/pkg/storage/testdata/mvcc_histories/update_existing_key_diff_txn b/pkg/storage/testdata/mvcc_histories/update_existing_key_diff_txn index bd3a23258c29..0cfaaabf9302 100644 --- a/pkg/storage/testdata/mvcc_histories/update_existing_key_diff_txn +++ b/pkg/storage/testdata/mvcc_histories/update_existing_key_diff_txn @@ -13,7 +13,7 @@ with t=B txn: "B" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=44.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=44.000000000,0 wto=false gul=0,0 meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=33.000000000,0 min=0,0 seq=0} ts=33.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "a"/33.000000000,0 -> /BYTES/xyz -error: (*roachpb.WriteIntentError:) conflicting intents on "a" +error: (*kvpb.WriteIntentError:) conflicting intents on "a" ## It's possible to observe the intent with an inconsistent read. diff --git a/pkg/storage/testdata/mvcc_histories/update_existing_key_old_version b/pkg/storage/testdata/mvcc_histories/update_existing_key_old_version index 78da368792e8..a74cdabf3136 100644 --- a/pkg/storage/testdata/mvcc_histories/update_existing_key_old_version +++ b/pkg/storage/testdata/mvcc_histories/update_existing_key_old_version @@ -14,7 +14,7 @@ put k=k v=v2 ts=0,1 >> at end: data: "k"/1.000000000,2 -> /BYTES/v2 data: "k"/1.000000000,1 -> /BYTES/v -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k" at timestamp 0,1 too old; wrote at 1.000000000,2 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k" at timestamp 0,1 too old; wrote at 1.000000000,2 # Earlier logical time. @@ -25,4 +25,4 @@ put k=k v=v2 ts=1,0 data: "k"/1.000000000,3 -> /BYTES/v2 data: "k"/1.000000000,2 -> /BYTES/v2 data: "k"/1.000000000,1 -> /BYTES/v -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k" at timestamp 1.000000000,0 too old; wrote at 1.000000000,3 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "k" at timestamp 1.000000000,0 too old; wrote at 1.000000000,3 diff --git a/pkg/storage/testdata/mvcc_histories/write_too_old b/pkg/storage/testdata/mvcc_histories/write_too_old index c2dbd69193c0..dd34061bd7d7 100644 --- a/pkg/storage/testdata/mvcc_histories/write_too_old +++ b/pkg/storage/testdata/mvcc_histories/write_too_old @@ -25,7 +25,7 @@ txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=33.000000000,0 min=0 meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=44.000000000,1 min=0,0 seq=0} ts=44.000000000,1 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true data: "a"/44.000000000,1 -> / data: "a"/44.000000000,0 -> /BYTES/abc -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "a" at timestamp 33.000000000,0 too old; wrote at 44.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "a" at timestamp 33.000000000,0 too old; wrote at 44.000000000,1 run ok resolve_intent t=A k=a status=ABORTED @@ -48,7 +48,7 @@ txn: "B" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=33.000000000,0 min=0 meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=44.000000000,1 min=0,0 seq=0} ts=44.000000000,1 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "a"/44.000000000,1 -> /BYTES/def data: "a"/44.000000000,0 -> /BYTES/abc -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "a" at timestamp 33.000000000,0 too old; wrote at 44.000000000,1 +error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "a" at timestamp 33.000000000,0 too old; wrote at 44.000000000,1 run ok resolve_intent t=B k=a status=ABORTED diff --git a/pkg/testutils/BUILD.bazel b/pkg/testutils/BUILD.bazel index c0ef2503a527..a98f5cc7464e 100644 --- a/pkg/testutils/BUILD.bazel +++ b/pkg/testutils/BUILD.bazel @@ -24,7 +24,7 @@ go_library( deps = [ "//pkg/base", "//pkg/build/bazel", - "//pkg/roachpb", + "//pkg/kv/kvpb", "//pkg/security/certnames", "//pkg/security/username", "//pkg/sql/pgwire/pgerror", diff --git a/pkg/testutils/error.go b/pkg/testutils/error.go index e2dff22b1e32..73c586edfdc8 100644 --- a/pkg/testutils/error.go +++ b/pkg/testutils/error.go @@ -13,7 +13,7 @@ package testutils import ( "regexp" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" ) @@ -36,7 +36,7 @@ func IsError(err error, re string) bool { // IsPError returns true if pErr's message matches the supplied regex. // An empty regex is interpreted to mean that a nil error is expected. -func IsPError(pErr *roachpb.Error, re string) bool { +func IsPError(pErr *kvpb.Error, re string) bool { if pErr == nil && re == "" { return true } diff --git a/pkg/testutils/jobutils/BUILD.bazel b/pkg/testutils/jobutils/BUILD.bazel index 644136734ec6..be6e3ee4d57b 100644 --- a/pkg/testutils/jobutils/BUILD.bazel +++ b/pkg/testutils/jobutils/BUILD.bazel @@ -9,8 +9,8 @@ go_library( deps = [ "//pkg/jobs", "//pkg/jobs/jobspb", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/kvserverbase", - "//pkg/roachpb", "//pkg/security/username", "//pkg/sql/catalog/descpb", "//pkg/testutils", diff --git a/pkg/testutils/jobutils/jobs_verification.go b/pkg/testutils/jobutils/jobs_verification.go index 73381be8cfa2..aed22d7a6e87 100644 --- a/pkg/testutils/jobutils/jobs_verification.go +++ b/pkg/testutils/jobutils/jobs_verification.go @@ -20,8 +20,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -140,10 +140,10 @@ func RunJob( // related to bulk IO/backup/restore/import: Export, Import and AddSSTable. See // discussion on RunJob for where this might be useful. func BulkOpResponseFilter(allowProgressIota *chan struct{}) kvserverbase.ReplicaResponseFilter { - return func(_ context.Context, ba *roachpb.BatchRequest, br *roachpb.BatchResponse) *roachpb.Error { + return func(_ context.Context, ba *kvpb.BatchRequest, br *kvpb.BatchResponse) *kvpb.Error { for _, ru := range br.Responses { switch ru.GetInner().(type) { - case *roachpb.ExportResponse, *roachpb.AddSSTableResponse: + case *kvpb.ExportResponse, *kvpb.AddSSTableResponse: <-*allowProgressIota } } diff --git a/pkg/testutils/kvclientutils/BUILD.bazel b/pkg/testutils/kvclientutils/BUILD.bazel index 6c211912a213..55febfc7a5d3 100644 --- a/pkg/testutils/kvclientutils/BUILD.bazel +++ b/pkg/testutils/kvclientutils/BUILD.bazel @@ -11,6 +11,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/util/hlc", "//pkg/util/tracing", diff --git a/pkg/testutils/kvclientutils/txn_recovery.go b/pkg/testutils/kvclientutils/txn_recovery.go index 1b97035f103c..cdc6e3ee4862 100644 --- a/pkg/testutils/kvclientutils/txn_recovery.go +++ b/pkg/testutils/kvclientutils/txn_recovery.go @@ -15,6 +15,7 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/tracing" @@ -62,18 +63,18 @@ func CheckPushResult( expResolution ExpectedTxnResolution, pushExpectation PushExpectation, ) error { - pushReq := roachpb.PushTxnRequest{ - RequestHeader: roachpb.RequestHeader{ + pushReq := kvpb.PushTxnRequest{ + RequestHeader: kvpb.RequestHeader{ Key: txn.Key, }, PusheeTxn: txn.TxnMeta, PushTo: hlc.Timestamp{}, - PushType: roachpb.PUSH_ABORT, + PushType: kvpb.PUSH_ABORT, // We're going to Force the push in order to not wait for the pushee to // expire. Force: true, } - ba := &roachpb.BatchRequest{} + ba := &kvpb.BatchRequest{} ba.Add(&pushReq) recCtx, collectRecAndFinish := tracing.ContextWithRecordingSpan(ctx, tr, "test trace") diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index 8e094916c2f7..512f445dfa8d 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -1118,7 +1118,7 @@ func TestLint(t *testing.T) { ":!rpc/codec.go", ":!rpc/codec_test.go", ":!settings/settings_test.go", - ":!roachpb/api_requestheader.go", + ":!kv/kvpb/api_requestheader.go", ":!storage/mvcc_value.go", ":!storage/enginepb/mvcc3_valueheader.go", ":!sql/types/types_jsonpb.go", @@ -1243,8 +1243,8 @@ func TestLint(t *testing.T) { ":!ccl/changefeedccl/changefeedbase/errors.go", ":!kv/kvclient/kvcoord/lock_spans_over_budget_error.go", ":!spanconfig/errors.go", - ":!roachpb/replica_unavailable_error.go", - ":!roachpb/ambiguous_result_error.go", + ":!kv/kvpb/replica_unavailable_error.go", + ":!kv/kvpb/ambiguous_result_error.go", ":!sql/flowinfra/flow_registry.go", ":!sql/pgwire/pgerror/constraint_name.go", ":!sql/pgwire/pgerror/severity.go", @@ -1298,7 +1298,7 @@ func TestLint(t *testing.T) { ":!bench/cmd", ":!sql/opt/optgen", ":!sql/colexec/execgen", - ":!roachpb/gen/main.go", + ":!kv/kvpb/gen/main.go", ) if err != nil { t.Fatal(err) @@ -1564,7 +1564,7 @@ func TestLint(t *testing.T) { stream.GrepNot(`cockroachdb/cockroach/pkg/rpc: github\.com/golang/protobuf/proto$`), stream.GrepNot(`cockroachdb/cockroach/pkg/sql/lexbase/allkeywords: log$`), stream.GrepNot(`cockroachdb/cockroach/pkg/util/timeutil/gen: log$`), - stream.GrepNot(`cockroachdb/cockroach/pkg/roachpb/gen: log$`), + stream.GrepNot(`cockroachdb/cockroach/pkg/kv/kvpb/gen: log$`), stream.GrepNot(`cockroachdb/cockroach/pkg/util/log/gen: log$`), stream.GrepNot(`cockroach/pkg/util/uuid: github\.com/satori/go\.uuid$`), ), func(s string) { @@ -1610,6 +1610,7 @@ func TestLint(t *testing.T) { forbiddenImports := map[string]struct{}{ "github.com/cockroachdb/pebble": {}, "github.com/cockroachdb/cockroach/pkg/cli": {}, + "github.com/cockroachdb/cockroach/pkg/kv/kvpb": {}, "github.com/cockroachdb/cockroach/pkg/kv/kvserver": {}, "github.com/cockroachdb/cockroach/pkg/roachpb": {}, "github.com/cockroachdb/cockroach/pkg/server": {}, @@ -1761,8 +1762,8 @@ func TestLint(t *testing.T) { stream.GrepNot(`pkg/.*.go:.* func .*\.Cause is unused`), // Using deprecated WireLength call. stream.GrepNot(`pkg/rpc/stats_handler.go:.*v.WireLength is deprecated: This field is never set.*`), - // roachpb/api.go needs v1 Protobuf reflection - stream.GrepNot(`pkg/roachpb/api_test.go:.*"github.com/golang/protobuf/proto" is deprecated: Use the "google.golang.org/protobuf/proto" package instead.`), + // kv/kvpb/api.go needs v1 Protobuf reflection + stream.GrepNot(`pkg/kv/kvpb/api_test.go:.*"github.com/golang/protobuf/proto" is deprecated: Use the "google.golang.org/protobuf/proto" package instead.`), // rpc/codec.go imports the same proto package that grpc-go imports (as of crdb@dd87d1145 and grpc-go@7b167fd6). stream.GrepNot(`pkg/rpc/codec.go:.*"github.com/golang/protobuf/proto" is deprecated: Use the "google.golang.org/protobuf/proto" package instead.`), // goschedstats contains partial copies of go runtime structures, with @@ -2040,6 +2041,7 @@ func TestLint(t *testing.T) { "../../col/colserde", "../../keys", "../../kv/kvclient/rangecache", + "../../kv/kvpb", "../../roachpb", "../../sql/catalog/descs", "../../sql/colcontainer", @@ -2272,13 +2274,13 @@ func TestLint(t *testing.T) { // for efficient hashing. stream.GrepNot(`pkg/sql/colexec/colexechash/hash.go:[0-9:]+: possible misuse of unsafe.Pointer`), stream.GrepNot(`^#`), // comment line - // Roachpb's own error package takes ownership of error unwraps + // kvpb's own error package takes ownership of error unwraps // (by enforcing that errors can never been wrapped under a - // roachpb.Error, which is an inconvenient limitation but it is + // kvpb.Error, which is an inconvenient limitation but it is // what it is). Once this code is simplified to use generalized // error encode/decode, it can be dropped from the linter // exception as well. - stream.GrepNot(`pkg/roachpb/errors\.go:.*invalid direct cast on error object`), + stream.GrepNot(`pkg/kv/kvpb/errors\.go:.*invalid direct cast on error object`), // Cast in decode handler. stream.GrepNot(`pkg/sql/pgwire/pgerror/constraint_name\.go:.*invalid direct cast on error object`), // Cast in decode handler. diff --git a/pkg/testutils/lint/passes/errwrap/functions.go b/pkg/testutils/lint/passes/errwrap/functions.go index fe4597356fb0..29d7b64a3099 100644 --- a/pkg/testutils/lint/passes/errwrap/functions.go +++ b/pkg/testutils/lint/passes/errwrap/functions.go @@ -59,7 +59,7 @@ var ErrorFnFormatStringIndex = map[string]int{ "github.com/cockroachdb/cockroach/pkg/security.makeErrorf": 1, - "github.com/cockroachdb/cockroach/pkg/roachpb.NewErrorf": 0, + "github.com/cockroachdb/cockroach/pkg/kv/kvpb.NewErrorf": 0, "github.com/cockroachdb/cockroach/pkg/sql/importer.makeRowErr": 3, "github.com/cockroachdb/cockroach/pkg/sql/importer.wrapRowErr": 4, diff --git a/pkg/testutils/lint/passes/fmtsafe/functions.go b/pkg/testutils/lint/passes/fmtsafe/functions.go index 9c878a24013a..07260086d6d0 100644 --- a/pkg/testutils/lint/passes/fmtsafe/functions.go +++ b/pkg/testutils/lint/passes/fmtsafe/functions.go @@ -73,7 +73,7 @@ var requireConstFmt = map[string]bool{ "github.com/cockroachdb/cockroach/pkg/util/log/logcrash.ReportOrPanic": true, - "github.com/cockroachdb/cockroach/pkg/roachpb.NewAmbiguousResultErrorf": true, + "github.com/cockroachdb/cockroach/pkg/kv/kvpb.NewAmbiguousResultErrorf": true, "(*github.com/cockroachdb/cockroach/pkg/util/tracing.Span).Recordf": true, "(*github.com/cockroachdb/cockroach/pkg/util/tracing.spanInner).Recordf": true, @@ -157,7 +157,7 @@ var requireConstFmt = map[string]bool{ "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis.l": true, "(*github.com/cockroachdb/cockroach/pkg/kv/kvnemesis.logLogger).Logf": true, - "(github.com/cockroachdb/cockroach/pkg/roachpb.TestPrinter).Printf": true, + "(github.com/cockroachdb/cockroach/pkg/kv/kvpb.TestPrinter).Printf": true, // Error things are populated in the init() message. } diff --git a/pkg/testutils/lint/passes/redactcheck/redactcheck.go b/pkg/testutils/lint/passes/redactcheck/redactcheck.go index d9c7c4465f03..d194ab9ac9d8 100644 --- a/pkg/testutils/lint/passes/redactcheck/redactcheck.go +++ b/pkg/testutils/lint/passes/redactcheck/redactcheck.go @@ -23,7 +23,7 @@ import ( ) // Analyzer is an analysis.Analyzer that checks for unused or discarded -// roachpb.Error objects from function calls. +// kvpb.Error objects from function calls. var Analyzer = &analysis.Analyzer{ Name: "redactcheck", Doc: "checks registered redact-safe types against an allow-list", @@ -73,6 +73,9 @@ func runAnalyzer(pass *analysis.Pass) (interface{}, error) { "sz": {}, "timing": {}, }, + "github.com/cockroachdb/cockroach/pkg/kv/kvpb": { + "Method": {}, + }, "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb": { "LAI": {}, "SeqNum": {}, @@ -91,7 +94,6 @@ func runAnalyzer(pass *analysis.Pass) (interface{}, error) { }, "github.com/cockroachdb/cockroach/pkg/roachpb": { "LeaseSequence": {}, - "Method": {}, "NodeID": {}, "RangeGeneration": {}, "RangeID": {}, diff --git a/pkg/testutils/lint/passes/returncheck/returncheck.go b/pkg/testutils/lint/passes/returncheck/returncheck.go index f84de32d918d..4efac610e693 100644 --- a/pkg/testutils/lint/passes/returncheck/returncheck.go +++ b/pkg/testutils/lint/passes/returncheck/returncheck.go @@ -9,7 +9,7 @@ // licenses/APL.txt. // Package returncheck defines an Analyzer that detects unused or -// discarded roachpb.Error objects. +// discarded kvpb.Error objects. package returncheck import ( @@ -22,10 +22,10 @@ import ( ) // Analyzer is an analysis.Analyzer that checks for unused or discarded -// roachpb.Error objects from function calls. +// kvpb.Error objects from function calls. var Analyzer = &analysis.Analyzer{ Name: "returncheck", - Doc: "`returncheck` : `roachpb.Error` :: `errcheck` : (stdlib)`error`", + Doc: "`returncheck` : `kvpb.Error` :: `errcheck` : (stdlib)`error`", Requires: []*analysis.Analyzer{inspect.Analyzer}, Run: runAnalyzer, } @@ -73,7 +73,7 @@ func runAnalyzer(pass *analysis.Pass) (interface{}, error) { // recordUnchecked records an error if a given calls has an unchecked // return. If pos is not a negative value and the call returns a // tuple, check if the return value at the specified position is of type -// roachpb.Error. +// kvpb.Error. func recordUnchecked(pass *analysis.Pass, call *ast.CallExpr, pos int) { isTarget := false switch t := pass.TypesInfo.Types[call].Type.(type) { @@ -96,10 +96,10 @@ func recordUnchecked(pass *analysis.Pass, call *ast.CallExpr, pos int) { } if isTarget { - pass.Reportf(call.Pos(), "unchecked roachpb.Error value") + pass.Reportf(call.Pos(), "unchecked kvpb.Error value") } } func isTargetType(t types.Type) bool { - return t.String() == "github.com/cockroachdb/cockroach/pkg/roachpb.Error" + return t.String() == "github.com/cockroachdb/cockroach/pkg/kv/kvpb.Error" } diff --git a/pkg/testutils/storageutils/BUILD.bazel b/pkg/testutils/storageutils/BUILD.bazel index 97a7245be102..cecd0a4da470 100644 --- a/pkg/testutils/storageutils/BUILD.bazel +++ b/pkg/testutils/storageutils/BUILD.bazel @@ -15,6 +15,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/keys", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/kvserverbase", "//pkg/roachpb", "//pkg/settings/cluster", diff --git a/pkg/testutils/storageutils/mocking.go b/pkg/testutils/storageutils/mocking.go index 0b39886bc3de..52ab1012b147 100644 --- a/pkg/testutils/storageutils/mocking.go +++ b/pkg/testutils/storageutils/mocking.go @@ -14,8 +14,8 @@ import ( "context" "fmt" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/syncutil/singleflight" ) @@ -35,7 +35,7 @@ func (r raftCmdIDAndIndex) String() string { type ReplayProtectionFilterWrapper struct { syncutil.Mutex inFlight *singleflight.Group - processedCommands map[raftCmdIDAndIndex]*roachpb.Error + processedCommands map[raftCmdIDAndIndex]*kvpb.Error filter kvserverbase.ReplicaCommandFilter } @@ -46,14 +46,14 @@ func WrapFilterForReplayProtection( ) kvserverbase.ReplicaCommandFilter { wrapper := ReplayProtectionFilterWrapper{ inFlight: singleflight.NewGroup("replay-protection", "key"), - processedCommands: make(map[raftCmdIDAndIndex]*roachpb.Error), + processedCommands: make(map[raftCmdIDAndIndex]*kvpb.Error), filter: filter, } return wrapper.run } // Errors are mutated on the Send path, so we must always return copies. -func shallowCloneErrorWithTxn(pErr *roachpb.Error) *roachpb.Error { +func shallowCloneErrorWithTxn(pErr *kvpb.Error) *kvpb.Error { if pErr != nil { pErrCopy := *pErr pErrCopy.SetTxn(pErrCopy.GetTxn()) @@ -64,7 +64,7 @@ func shallowCloneErrorWithTxn(pErr *roachpb.Error) *roachpb.Error { } // run executes the wrapped filter. -func (c *ReplayProtectionFilterWrapper) run(args kvserverbase.FilterArgs) *roachpb.Error { +func (c *ReplayProtectionFilterWrapper) run(args kvserverbase.FilterArgs) *kvpb.Error { if !args.InRaftCmd() { return c.filter(args) } @@ -96,5 +96,5 @@ func (c *ReplayProtectionFilterWrapper) run(args kvserverbase.FilterArgs) *roach c.Unlock() res := future.WaitForResult(args.Ctx) - return shallowCloneErrorWithTxn(res.Val.(*roachpb.Error)) + return shallowCloneErrorWithTxn(res.Val.(*kvpb.Error)) } diff --git a/pkg/testutils/testcluster/BUILD.bazel b/pkg/testutils/testcluster/BUILD.bazel index 9f005c946fba..d8c18f6572ac 100644 --- a/pkg/testutils/testcluster/BUILD.bazel +++ b/pkg/testutils/testcluster/BUILD.bazel @@ -10,6 +10,7 @@ go_library( "//pkg/base", "//pkg/gossip", "//pkg/keys", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/kv/kvserver/liveness/livenesspb", "//pkg/roachpb", @@ -51,6 +52,7 @@ go_test( "//pkg/base", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/rpc", "//pkg/security/securityassets", diff --git a/pkg/testutils/testcluster/testcluster.go b/pkg/testutils/testcluster/testcluster.go index fb9bc303e56c..5a9f428b567b 100644 --- a/pkg/testutils/testcluster/testcluster.go +++ b/pkg/testutils/testcluster/testcluster.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -740,7 +741,7 @@ func (tc *TestCluster) changeReplicas( } var err error desc, err = tc.Servers[0].DB().AdminChangeReplicas( - ctx, startKey.AsRawKey(), beforeDesc, roachpb.MakeReplicationChanges(changeType, targets...), + ctx, startKey.AsRawKey(), beforeDesc, kvpb.MakeReplicationChanges(changeType, targets...), ) if kvserver.IsRetriableReplicationChangeError(err) { tc.t.Logf("encountered retriable replication change error: %v", err) @@ -962,7 +963,7 @@ func (tc *TestCluster) SwapVoterWithNonVoter( ); err != nil { return nil, errors.Wrap(err, "range descriptor lookup error") } - changes := []roachpb.ReplicationChange{ + changes := []kvpb.ReplicationChange{ {ChangeType: roachpb.ADD_VOTER, Target: nonVoterTarget}, {ChangeType: roachpb.REMOVE_NON_VOTER, Target: nonVoterTarget}, {ChangeType: roachpb.ADD_NON_VOTER, Target: voterTarget}, @@ -1001,7 +1002,7 @@ func (tc *TestCluster) RebalanceVoter( ); err != nil { return nil, errors.Wrap(err, "range descriptor lookup error") } - changes := []roachpb.ReplicationChange{ + changes := []kvpb.ReplicationChange{ {ChangeType: roachpb.REMOVE_VOTER, Target: src}, {ChangeType: roachpb.ADD_VOTER, Target: dest}, } @@ -1152,7 +1153,7 @@ func (tc *TestCluster) MoveRangeLeaseNonCooperatively( ls, err := r.TestingAcquireLease(ctx) if err != nil { log.Infof(ctx, "TestingAcquireLease failed: %s", err) - if lErr := (*roachpb.NotLeaseHolderError)(nil); errors.As(err, &lErr) && lErr.Lease != nil { + if lErr := (*kvpb.NotLeaseHolderError)(nil); errors.As(err, &lErr) && lErr.Lease != nil { newLease = lErr.Lease } else { return err diff --git a/pkg/testutils/testcluster/testcluster_test.go b/pkg/testutils/testcluster/testcluster_test.go index d96b5b9f0332..3c303d894dda 100644 --- a/pkg/testutils/testcluster/testcluster_test.go +++ b/pkg/testutils/testcluster/testcluster_test.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/server" @@ -307,8 +308,8 @@ func TestRestart(t *testing.T) { ids[i] = tc.Target(i) } - incArgs := &roachpb.IncrementRequest{ - RequestHeader: roachpb.RequestHeader{ + incArgs := &kvpb.IncrementRequest{ + RequestHeader: kvpb.RequestHeader{ Key: roachpb.Key("b"), }, Increment: 9, diff --git a/pkg/ts/BUILD.bazel b/pkg/ts/BUILD.bazel index 6d469bfd4ce1..8269d03e603d 100644 --- a/pkg/ts/BUILD.bazel +++ b/pkg/ts/BUILD.bazel @@ -24,6 +24,7 @@ go_library( "//pkg/keys", "//pkg/kv", "//pkg/kv/kvclient/kvtenant", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/kv/kvserver/allocator/storepool", "//pkg/roachpb", @@ -74,6 +75,7 @@ go_test( "//pkg/keys", "//pkg/kv", "//pkg/kv/kvclient/kvcoord", + "//pkg/kv/kvpb", "//pkg/kv/kvserver", "//pkg/roachpb", "//pkg/rpc", diff --git a/pkg/ts/db.go b/pkg/ts/db.go index b812e4136560..2bc4abacc7dd 100644 --- a/pkg/ts/db.go +++ b/pkg/ts/db.go @@ -16,6 +16,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -307,8 +308,8 @@ func (db *DB) tryStoreRollup(ctx context.Context, r Resolution, data []rollupDat func (db *DB) storeKvs(ctx context.Context, kvs []roachpb.KeyValue) error { b := &kv.Batch{} for _, kv := range kvs { - b.AddRawRequest(&roachpb.MergeRequest{ - RequestHeader: roachpb.RequestHeader{ + b.AddRawRequest(&kvpb.MergeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: kv.Key, }, Value: kv.Value, diff --git a/pkg/ts/pruning.go b/pkg/ts/pruning.go index 198929bf4d51..5efbd87671bd 100644 --- a/pkg/ts/pruning.go +++ b/pkg/ts/pruning.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -135,8 +136,8 @@ func (tsdb *DB) pruneTimeSeries( end = start.PrefixEnd() } - b.AddRawRequest(&roachpb.DeleteRangeRequest{ - RequestHeader: roachpb.RequestHeader{ + b.AddRawRequest(&kvpb.DeleteRangeRequest{ + RequestHeader: kvpb.RequestHeader{ Key: start, EndKey: end, }, diff --git a/pkg/ts/server.go b/pkg/ts/server.go index d216a5fbfbd3..81d3631bfe12 100644 --- a/pkg/ts/server.go +++ b/pkg/ts/server.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvtenant" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/ts/catalog" @@ -476,7 +477,7 @@ func dumpTimeseriesAllSources( for span != nil { b := &kv.Batch{} - scan := roachpb.NewScan(span.Key, span.EndKey, false /* forUpdate */) + scan := kvpb.NewScan(span.Key, span.EndKey, false /* forUpdate */) b.AddRawRequest(scan) b.Header.MaxSpanRequestKeys = dumpBatchSize err := db.Run(ctx, b) diff --git a/pkg/ts/server_test.go b/pkg/ts/server_test.go index 238753dbf6a8..e7f066acb387 100644 --- a/pkg/ts/server_test.go +++ b/pkg/ts/server_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" @@ -492,8 +493,8 @@ func TestServerDump(t *testing.T) { var b kv.Batch for _, kv := range kvs { - p := roachpb.NewPut(kv.Key, kv.Value) - p.(*roachpb.PutRequest).Inline = true + p := kvpb.NewPut(kv.Key, kv.Value) + p.(*kvpb.PutRequest).Inline = true b.AddRawRequest(p) } // Write and check multiple times, to make sure there aren't any issues diff --git a/pkg/util/admission/admission.go b/pkg/util/admission/admission.go index e8db8e8be5ae..5e7ffa4b108f 100644 --- a/pkg/util/admission/admission.go +++ b/pkg/util/admission/admission.go @@ -110,7 +110,7 @@ // registry.AddMetricStruct(metrics[i]) // } // kvQueue := coord.GetWorkQueue(admission.KVWork) -// // Pass kvQueue to server.Node that implements roachpb.InternalServer. +// // Pass kvQueue to server.Node that implements kvpb.InternalServer. // ... // // Do similar things with the other WorkQueues. // diff --git a/pkg/util/hlc/doc.go b/pkg/util/hlc/doc.go index 2326c7db4adb..ae297170a016 100644 --- a/pkg/util/hlc/doc.go +++ b/pkg/util/hlc/doc.go @@ -42,16 +42,16 @@ between nodes in a cluster: API will attach HLC clock readings on requests and responses (successes and errors). - Ref: (roachpb.Header).Timestamp. - Ref: (roachpb.BatchResponse_Header).Now. - Ref: (roachpb.Error).Now. + Ref: (kvpb.Header).Timestamp. + Ref: (kvpb.BatchResponse_Header).Now. + Ref: (kvpb.Error).Now. - DistSQL flows (unidirectional): leaves of a DistSQL flow will pass clock readings back to the root of the flow. Currently, this only takes place on errors, and relates to the "Transaction retry errors" interaction detailed below. - Ref: (roachpb.Error).Now. + Ref: (kvpb.Error).Now. Capturing causal relationships between events on different nodes is critical for enforcing invariants within CockroachDB. What follows is an enumeration of each diff --git a/pkg/util/protoutil/BUILD.bazel b/pkg/util/protoutil/BUILD.bazel index fe8dec1bbadb..d686fe5a25b0 100644 --- a/pkg/util/protoutil/BUILD.bazel +++ b/pkg/util/protoutil/BUILD.bazel @@ -32,6 +32,7 @@ go_test( ":protoutil", "//pkg/config/zonepb", "//pkg/gossip", + "//pkg/kv/kvpb", "//pkg/kv/kvserver/kvserverpb", "//pkg/kv/kvserver/protectedts/ptpb", "//pkg/roachpb", diff --git a/pkg/util/protoutil/clone_test.go b/pkg/util/protoutil/clone_test.go index 77674bb82602..6b38cb6c48a4 100644 --- a/pkg/util/protoutil/clone_test.go +++ b/pkg/util/protoutil/clone_test.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -37,7 +38,7 @@ func TestCloneProto(t *testing.T) { {&roachpb.StoreIdent{}, true}, {&enginepb.TxnMeta{}, true}, {&roachpb.Transaction{}, true}, - {&roachpb.Error{}, true}, + {&kvpb.Error{}, true}, {&protoutil.RecursiveAndUncloneable{}, true}, // Cloneable types. This includes all types for which a diff --git a/pkg/util/stop/BUILD.bazel b/pkg/util/stop/BUILD.bazel index 71792af1e075..11ccee60623f 100644 --- a/pkg/util/stop/BUILD.bazel +++ b/pkg/util/stop/BUILD.bazel @@ -7,7 +7,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/util/stop", visibility = ["//visibility:public"], deps = [ - "//pkg/roachpb", + "//pkg/kv/kvpb", "//pkg/util/leaktest", "//pkg/util/log", "//pkg/util/log/logcrash", @@ -28,7 +28,7 @@ go_test( args = ["-test.timeout=55s"], deps = [ ":stop", - "//pkg/roachpb", + "//pkg/kv/kvpb", "//pkg/testutils", "//pkg/util/leaktest", "//pkg/util/log", diff --git a/pkg/util/stop/stopper.go b/pkg/util/stop/stopper.go index 870219716a35..058b68528204 100644 --- a/pkg/util/stop/stopper.go +++ b/pkg/util/stop/stopper.go @@ -20,7 +20,7 @@ import ( "testing" "time" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" @@ -40,7 +40,7 @@ var ErrThrottled = errors.New("throttled on async limiting semaphore") // ErrUnavailable indicates that the server is quiescing and is unable to // process new work. -var ErrUnavailable = &roachpb.NodeUnavailableError{} +var ErrUnavailable = &kvpb.NodeUnavailableError{} func register(s *Stopper) { trackedStoppers.Lock() diff --git a/pkg/util/stop/stopper_test.go b/pkg/util/stop/stopper_test.go index fb0a731a22c6..beddf17a7033 100644 --- a/pkg/util/stop/stopper_test.go +++ b/pkg/util/stop/stopper_test.go @@ -19,7 +19,7 @@ import ( "testing" "time" - "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -182,7 +182,7 @@ func TestStopperQuiesce(t *testing.T) { // Wait until Quiesce() is called. <-qc err := thisStopper.RunTask(ctx, "inner", func(context.Context) {}) - if !errors.HasType(err, (*roachpb.NodeUnavailableError)(nil)) { + if !errors.HasType(err, (*kvpb.NodeUnavailableError)(nil)) { t.Error(err) } // Make the stoppers call Stop().