From 15636bc42d2af87b8a55d54f738a435a09745991 Mon Sep 17 00:00:00 2001 From: Andrei Matei Date: Tue, 18 May 2021 15:59:37 -0400 Subject: [PATCH] closedts: delete old closedts mechanism Closes #61989 Closes #60682 Closes #59448 --- docs/generated/redact_safe.md | 2 +- pkg/clusterversion/cockroach_versions.go | 7 - pkg/clusterversion/key_string.go | 53 +- pkg/kv/kvserver/BUILD.bazel | 3 - pkg/kv/kvserver/batcheval/BUILD.bazel | 2 - .../batcheval/cmd_query_resolved_timestamp.go | 2 +- pkg/kv/kvserver/batcheval/cmd_subsume.go | 54 +- pkg/kv/kvserver/batcheval/eval_context.go | 11 +- .../kvserver/client_closed_timestamp_test.go | 145 ---- pkg/kv/kvserver/client_merge_test.go | 4 +- pkg/kv/kvserver/closed_timestamp_test.go | 3 +- pkg/kv/kvserver/closedts/BUILD.bazel | 2 - pkg/kv/kvserver/closedts/closedts.go | 201 ----- .../kvserver/closedts/container/BUILD.bazel | 49 -- .../kvserver/closedts/container/container.go | 153 ---- .../closedts/container/container_test.go | 373 --------- pkg/kv/kvserver/closedts/container/noop.go | 86 -- pkg/kv/kvserver/closedts/ctpb/BUILD.bazel | 12 +- pkg/kv/kvserver/closedts/ctpb/client.go | 23 - pkg/kv/kvserver/closedts/ctpb/entry.go | 58 -- pkg/kv/kvserver/closedts/ctpb/entry.pb.go | 762 ------------------ pkg/kv/kvserver/closedts/ctpb/entry.proto | 61 -- pkg/kv/kvserver/closedts/ctpb/server.go | 40 - pkg/kv/kvserver/closedts/ctpb/service.go | 7 + pkg/kv/kvserver/closedts/ctpb/service.pb.go | 184 +---- pkg/kv/kvserver/closedts/ctpb/service.proto | 5 - pkg/kv/kvserver/closedts/minprop/BUILD.bazel | 41 - pkg/kv/kvserver/closedts/minprop/doc.go | 24 - pkg/kv/kvserver/closedts/minprop/doc_test.go | 225 ------ pkg/kv/kvserver/closedts/minprop/tracker.go | 395 --------- .../kvserver/closedts/minprop/tracker_test.go | 537 ------------ pkg/kv/kvserver/closedts/provider/BUILD.bazel | 45 -- pkg/kv/kvserver/closedts/provider/provider.go | 380 --------- .../closedts/provider/provider_test.go | 352 -------- .../closedts/provider/testutils/BUILD.bazel | 19 - .../closedts/provider/testutils/clock.go | 58 -- .../closedts/provider/testutils/storage.go | 87 -- .../closedts/sidetransport/BUILD.bazel | 1 - .../kvserver/closedts/sidetransport/sender.go | 9 - pkg/kv/kvserver/closedts/storage/BUILD.bazel | 35 - pkg/kv/kvserver/closedts/storage/storage.go | 170 ---- .../kvserver/closedts/storage/storage_mem.go | 189 ----- .../kvserver/closedts/storage/storage_test.go | 467 ----------- .../kvserver/closedts/transport/BUILD.bazel | 49 -- pkg/kv/kvserver/closedts/transport/clients.go | 179 ---- pkg/kv/kvserver/closedts/transport/server.go | 100 --- .../closedts/transport/testutils/BUILD.bazel | 14 - .../transport/testutils/chan_dialer.go | 146 ---- .../closedts/transport/transport_test.go | 199 ----- .../closedts/transport/transport_util_test.go | 104 --- pkg/kv/kvserver/helpers_test.go | 14 +- pkg/kv/kvserver/kvserverpb/state.pb.go | 487 ++--------- pkg/kv/kvserver/kvserverpb/state.proto | 23 +- pkg/kv/kvserver/liveness/BUILD.bazel | 2 - pkg/kv/kvserver/liveness/liveness.go | 21 - pkg/kv/kvserver/metrics.go | 6 +- pkg/kv/kvserver/replica.go | 23 +- pkg/kv/kvserver/replica_application_result.go | 11 +- .../replica_application_state_machine.go | 6 +- pkg/kv/kvserver/replica_closedts.go | 42 - .../replica_closedts_internal_test.go | 6 +- pkg/kv/kvserver/replica_eval_context_span.go | 13 +- pkg/kv/kvserver/replica_follower_read.go | 69 +- pkg/kv/kvserver/replica_proposal.go | 5 - pkg/kv/kvserver/replica_proposal_buf.go | 17 - pkg/kv/kvserver/replica_proposal_buf_test.go | 4 - pkg/kv/kvserver/replica_raftstorage.go | 7 - pkg/kv/kvserver/replica_rangefeed.go | 41 +- pkg/kv/kvserver/replica_rangefeed_test.go | 108 --- pkg/kv/kvserver/replica_tscache.go | 2 +- pkg/kv/kvserver/replica_write.go | 21 +- pkg/kv/kvserver/store.go | 78 +- pkg/kv/kvserver/store_raft.go | 2 - pkg/kv/kvserver/store_split.go | 2 +- pkg/rpc/nodedialer/BUILD.bazel | 2 - pkg/rpc/nodedialer/nodedialer.go | 23 - pkg/server/BUILD.bazel | 1 - pkg/server/authentication_test.go | 4 +- pkg/server/node.go | 3 - pkg/server/server.go | 29 - 80 files changed, 205 insertions(+), 6994 deletions(-) delete mode 100644 pkg/kv/kvserver/client_closed_timestamp_test.go delete mode 100644 pkg/kv/kvserver/closedts/closedts.go delete mode 100644 pkg/kv/kvserver/closedts/container/container.go delete mode 100644 pkg/kv/kvserver/closedts/container/container_test.go delete mode 100644 pkg/kv/kvserver/closedts/container/noop.go delete mode 100644 pkg/kv/kvserver/closedts/ctpb/client.go delete mode 100644 pkg/kv/kvserver/closedts/ctpb/entry.go delete mode 100644 pkg/kv/kvserver/closedts/ctpb/entry.pb.go delete mode 100644 pkg/kv/kvserver/closedts/ctpb/entry.proto delete mode 100644 pkg/kv/kvserver/closedts/ctpb/server.go delete mode 100644 pkg/kv/kvserver/closedts/minprop/doc.go delete mode 100644 pkg/kv/kvserver/closedts/minprop/doc_test.go delete mode 100644 pkg/kv/kvserver/closedts/minprop/tracker.go delete mode 100644 pkg/kv/kvserver/closedts/minprop/tracker_test.go delete mode 100644 pkg/kv/kvserver/closedts/provider/provider.go delete mode 100644 pkg/kv/kvserver/closedts/provider/provider_test.go delete mode 100644 pkg/kv/kvserver/closedts/provider/testutils/clock.go delete mode 100644 pkg/kv/kvserver/closedts/provider/testutils/storage.go delete mode 100644 pkg/kv/kvserver/closedts/storage/storage.go delete mode 100644 pkg/kv/kvserver/closedts/storage/storage_mem.go delete mode 100644 pkg/kv/kvserver/closedts/storage/storage_test.go delete mode 100644 pkg/kv/kvserver/closedts/transport/clients.go delete mode 100644 pkg/kv/kvserver/closedts/transport/server.go delete mode 100644 pkg/kv/kvserver/closedts/transport/testutils/chan_dialer.go delete mode 100644 pkg/kv/kvserver/closedts/transport/transport_test.go delete mode 100644 pkg/kv/kvserver/closedts/transport/transport_util_test.go diff --git a/docs/generated/redact_safe.md b/docs/generated/redact_safe.md index f729c69f01f5..761858c34676 100644 --- a/docs/generated/redact_safe.md +++ b/docs/generated/redact_safe.md @@ -4,7 +4,7 @@ File | Type --|-- pkg/cli/exit/exit.go | `Code` pkg/jobs/jobspb/wrap.go | `Type` -pkg/kv/kvserver/closedts/ctpb/entry.go | `LAI` +pkg/kv/kvserver/closedts/ctpb/service.go | `LAI` pkg/kv/kvserver/concurrency/lock/locking.go | `WaitPolicy` pkg/kv/kvserver/raft.go | `SnapshotRequest_Type` pkg/roachpb/data.go | `LeaseSequence` diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index e5e0413b00e8..5ec28fbd1658 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -222,9 +222,6 @@ const ( // are propagated across RPC boundaries independently of their verbosity setting. // This requires a version gate this violates implicit assumptions in v20.2. TracingVerbosityIndependentSemantics - // ClosedTimestampsRaftTransport enables the Raft transport for closed - // timestamps and disables the previous per-node transport. - ClosedTimestampsRaftTransport // PriorReadSummaries introduces support for the use of read summary objects // to ship information about reads on a range through lease changes and // range merges. @@ -384,10 +381,6 @@ var versionsSingleton = keyedVersions{ Key: TracingVerbosityIndependentSemantics, Version: roachpb.Version{Major: 20, Minor: 2, Internal: 28}, }, - { - Key: ClosedTimestampsRaftTransport, - Version: roachpb.Version{Major: 20, Minor: 2, Internal: 36}, - }, { Key: PriorReadSummaries, Version: roachpb.Version{Major: 20, Minor: 2, Internal: 44}, diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index dda8790961d4..3de27ddb6590 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -24,36 +24,35 @@ func _() { _ = x[PostTruncatedAndRangeAppliedStateMigration-13] _ = x[SeparatedIntents-14] _ = x[TracingVerbosityIndependentSemantics-15] - _ = x[ClosedTimestampsRaftTransport-16] - _ = x[PriorReadSummaries-17] - _ = x[NonVotingReplicas-18] - _ = x[V21_1-19] - _ = x[Start21_1PLUS-20] - _ = x[Start21_2-21] - _ = x[JoinTokensTable-22] - _ = x[AcquisitionTypeInLeaseHistory-23] - _ = x[SerializeViewUDTs-24] - _ = x[ExpressionIndexes-25] - _ = x[DeleteDeprecatedNamespaceTableDescriptorMigration-26] - _ = x[FixDescriptors-27] - _ = x[SQLStatsTable-28] - _ = x[DatabaseRoleSettings-29] - _ = x[TenantUsageTable-30] - _ = x[SQLInstancesTable-31] - _ = x[NewRetryableRangefeedErrors-32] - _ = x[AlterSystemWebSessionsCreateIndexes-33] - _ = x[SeparatedIntentsMigration-34] - _ = x[PostSeparatedIntentsMigration-35] - _ = x[RetryJobsWithExponentialBackoff-36] - _ = x[RecordsBasedRegistry-37] - _ = x[AutoSpanConfigReconciliationJob-38] - _ = x[PreventNewInterleavedTables-39] - _ = x[EnsureNoInterleavedTables-40] + _ = x[PriorReadSummaries-16] + _ = x[NonVotingReplicas-17] + _ = x[V21_1-18] + _ = x[Start21_1PLUS-19] + _ = x[Start21_2-20] + _ = x[JoinTokensTable-21] + _ = x[AcquisitionTypeInLeaseHistory-22] + _ = x[SerializeViewUDTs-23] + _ = x[ExpressionIndexes-24] + _ = x[DeleteDeprecatedNamespaceTableDescriptorMigration-25] + _ = x[FixDescriptors-26] + _ = x[SQLStatsTable-27] + _ = x[DatabaseRoleSettings-28] + _ = x[TenantUsageTable-29] + _ = x[SQLInstancesTable-30] + _ = x[NewRetryableRangefeedErrors-31] + _ = x[AlterSystemWebSessionsCreateIndexes-32] + _ = x[SeparatedIntentsMigration-33] + _ = x[PostSeparatedIntentsMigration-34] + _ = x[RetryJobsWithExponentialBackoff-35] + _ = x[RecordsBasedRegistry-36] + _ = x[AutoSpanConfigReconciliationJob-37] + _ = x[PreventNewInterleavedTables-38] + _ = x[EnsureNoInterleavedTables-39] } -const _Key_name = "Start20_2NodeMembershipStatusMinPasswordLengthAbortSpanBytesCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1CPutInlineReplicaVersionsreplacedTruncatedAndRangeAppliedStateMigrationreplacedPostTruncatedAndRangeAppliedStateMigrationTruncatedAndRangeAppliedStateMigrationPostTruncatedAndRangeAppliedStateMigrationSeparatedIntentsTracingVerbosityIndependentSemanticsClosedTimestampsRaftTransportPriorReadSummariesNonVotingReplicasV21_1Start21_1PLUSStart21_2JoinTokensTableAcquisitionTypeInLeaseHistorySerializeViewUDTsExpressionIndexesDeleteDeprecatedNamespaceTableDescriptorMigrationFixDescriptorsSQLStatsTableDatabaseRoleSettingsTenantUsageTableSQLInstancesTableNewRetryableRangefeedErrorsAlterSystemWebSessionsCreateIndexesSeparatedIntentsMigrationPostSeparatedIntentsMigrationRetryJobsWithExponentialBackoffRecordsBasedRegistryAutoSpanConfigReconciliationJobPreventNewInterleavedTablesEnsureNoInterleavedTables" +const _Key_name = "Start20_2NodeMembershipStatusMinPasswordLengthAbortSpanBytesCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1CPutInlineReplicaVersionsreplacedTruncatedAndRangeAppliedStateMigrationreplacedPostTruncatedAndRangeAppliedStateMigrationTruncatedAndRangeAppliedStateMigrationPostTruncatedAndRangeAppliedStateMigrationSeparatedIntentsTracingVerbosityIndependentSemanticsPriorReadSummariesNonVotingReplicasV21_1Start21_1PLUSStart21_2JoinTokensTableAcquisitionTypeInLeaseHistorySerializeViewUDTsExpressionIndexesDeleteDeprecatedNamespaceTableDescriptorMigrationFixDescriptorsSQLStatsTableDatabaseRoleSettingsTenantUsageTableSQLInstancesTableNewRetryableRangefeedErrorsAlterSystemWebSessionsCreateIndexesSeparatedIntentsMigrationPostSeparatedIntentsMigrationRetryJobsWithExponentialBackoffRecordsBasedRegistryAutoSpanConfigReconciliationJobPreventNewInterleavedTablesEnsureNoInterleavedTables" -var _Key_index = [...]uint16{0, 9, 29, 46, 60, 80, 92, 97, 106, 116, 131, 177, 227, 265, 307, 323, 359, 388, 406, 423, 428, 441, 450, 465, 494, 511, 528, 577, 591, 604, 624, 640, 657, 684, 719, 744, 773, 804, 824, 855, 882, 907} +var _Key_index = [...]uint16{0, 9, 29, 46, 60, 80, 92, 97, 106, 116, 131, 177, 227, 265, 307, 323, 359, 377, 394, 399, 412, 421, 436, 465, 482, 499, 548, 562, 575, 595, 611, 628, 655, 690, 715, 744, 775, 795, 826, 853, 878} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index b4041fa543ca..84ceb060f6cf 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -114,10 +114,8 @@ go_library( "//pkg/kv/kvserver/batcheval", "//pkg/kv/kvserver/batcheval/result", "//pkg/kv/kvserver/closedts", - "//pkg/kv/kvserver/closedts/container", "//pkg/kv/kvserver/closedts/ctpb", "//pkg/kv/kvserver/closedts/sidetransport", - "//pkg/kv/kvserver/closedts/storage", "//pkg/kv/kvserver/closedts/tracker", "//pkg/kv/kvserver/concurrency", "//pkg/kv/kvserver/constraint", @@ -206,7 +204,6 @@ go_test( "batch_spanset_test.go", "below_raft_protos_test.go", "client_atomic_membership_change_test.go", - "client_closed_timestamp_test.go", "client_lease_test.go", "client_merge_test.go", "client_metrics_test.go", diff --git a/pkg/kv/kvserver/batcheval/BUILD.bazel b/pkg/kv/kvserver/batcheval/BUILD.bazel index 5e5cc68f1933..4350cd56525f 100644 --- a/pkg/kv/kvserver/batcheval/BUILD.bazel +++ b/pkg/kv/kvserver/batcheval/BUILD.bazel @@ -56,8 +56,6 @@ go_library( "//pkg/keys", "//pkg/kv/kvserver/abortspan", "//pkg/kv/kvserver/batcheval/result", - "//pkg/kv/kvserver/closedts", - "//pkg/kv/kvserver/closedts/ctpb", "//pkg/kv/kvserver/concurrency", "//pkg/kv/kvserver/concurrency/lock", "//pkg/kv/kvserver/gc", diff --git a/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp.go b/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp.go index 5146af00a290..fbc2818ac282 100644 --- a/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp.go +++ b/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp.go @@ -54,7 +54,7 @@ func QueryResolvedTimestamp( // because QueryResolvedTimestamp requests are often run without acquiring // latches (see roachpb.INCONSISTENT) and often also on follower replicas, // so latches won't help them to synchronize with writes. - closedTS := cArgs.EvalCtx.GetClosedTimestampV2(ctx) + closedTS := cArgs.EvalCtx.GetClosedTimestamp(ctx) // Compute the minimum timestamp of any intent in the request's key span, // which may span the entire range, but does not need to. diff --git a/pkg/kv/kvserver/batcheval/cmd_subsume.go b/pkg/kv/kvserver/batcheval/cmd_subsume.go index 0a8a84a8ea4d..43f4251849c4 100644 --- a/pkg/kv/kvserver/batcheval/cmd_subsume.go +++ b/pkg/kv/kvserver/batcheval/cmd_subsume.go @@ -16,7 +16,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -125,50 +124,10 @@ func Subsume( return result.Result{}, errors.Wrap(err, "watching for merge during subsume") } - // We prevent followers of the RHS from being able to serve follower reads on - // timestamps that fall in the timestamp window representing the range's - // subsumed state (i.e. between the subsumption time (FreezeStart) and the - // timestamp at which the merge transaction commits or aborts), by requiring - // follower replicas to catch up to an MLAI that succeeds the range's current - // LeaseAppliedIndex (note that we're tracking lai + 1 below instead of lai). - // In case the merge successfully commits, this MLAI will never be caught up - // to since the RHS will be destroyed. In case the merge aborts, this ensures - // that the followers can only activate the newer closed timestamps once they - // catch up to the LAI associated with the merge abort. We need to do this - // because the closed timestamps that are broadcast by RHS in this subsumed - // state are not going to be reflected in the timestamp cache of the LHS range - // after the merge, which can cause a serializability violation. - // - // Note that we are essentially lying to the closed timestamp tracker here in - // order to achieve the effect of unactionable closed timestamp updates until - // the merge concludes. Tracking lai + 1 here ensures that the follower - // replicas need to catch up to at least that index before they are able to - // activate _any of the closed timestamps from this point onwards_. In other - // words, we will never publish a closed timestamp update for this range below - // this lai, regardless of whether a different proposal untracks a lower lai - // at any point in the future. - // - // NB: The above statement relies on the invariant that the LAI that follows a - // Subsume request will be applied only after the merge aborts. More - // specifically, this means that no intervening request can bump the LAI of - // range while it is subsumed. This invariant is upheld because the only Raft - // proposals allowed after a range has been subsumed are lease requests, which - // do not bump the LAI. In case there is lease transfer on this range while it - // is subsumed, we ensure that the initial MLAI update broadcast by the new - // leaseholder respects the invariant in question, in much the same way we do - // here. Take a look at `EmitMLAI()` in replica_closedts.go for more details. - // - // TODO(nvanbenschoten): remove this in v21.2 when the rest of the v1 closed - // timestamp system disappears. - _, untrack := cArgs.EvalCtx.GetTracker().Track(ctx) - lease, _ := cArgs.EvalCtx.GetLease() - lai := cArgs.EvalCtx.GetLeaseAppliedIndex() - untrack(ctx, ctpb.Epoch(lease.Epoch), desc.RangeID, ctpb.LAI(lai+1)) - // Now that the range is frozen, collect some information to ship to the LHS // leaseholder through the merge trigger. reply.MVCCStats = cArgs.EvalCtx.GetMVCCStats() - reply.LeaseAppliedIndex = lai + reply.LeaseAppliedIndex = cArgs.EvalCtx.GetLeaseAppliedIndex() reply.FreezeStart = cArgs.EvalCtx.Clock().NowAsClockTimestamp() // Collect a read summary from the RHS leaseholder to ship to the LHS @@ -188,16 +147,7 @@ func Subsume( // think about. priorReadSum.Merge(rspb.FromTimestamp(reply.FreezeStart.ToTimestamp())) reply.ReadSummary = &priorReadSum - // NOTE FOR v21.1: GetClosedTimestampV2 might return an empty timestamp if - // the Raft-based closed timestamp transport hasn't been enabled yet. That's - // OK because, if the new transport is not enabled, then ranges with leading - // closed timestamps can't exist yet, and so the closed timestamp must be - // below the FreezeStart. The FreezeStart is used by Store.MergeRange to - // bump the RHS' ts cache if LHS/RHS leases are not collocated. The case - // when the leases are collocated also works out because then the closed - // timestamp (according to the old mechanism) is the same for both ranges - // being merged. - reply.ClosedTimestamp = cArgs.EvalCtx.GetClosedTimestampV2(ctx) + reply.ClosedTimestamp = cArgs.EvalCtx.GetClosedTimestamp(ctx) return result.Result{}, nil } diff --git a/pkg/kv/kvserver/batcheval/eval_context.go b/pkg/kv/kvserver/batcheval/eval_context.go index bedb8e254c9e..8e17a018359e 100644 --- a/pkg/kv/kvserver/batcheval/eval_context.go +++ b/pkg/kv/kvserver/batcheval/eval_context.go @@ -16,7 +16,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/cloud" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/abortspan" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb" @@ -63,7 +62,6 @@ type EvalContext interface { GetFirstIndex() (uint64, error) GetTerm(uint64) (uint64, error) GetLeaseAppliedIndex() uint64 - GetTracker() closedts.TrackerI Desc() *roachpb.RangeDescriptor ContainsKey(key roachpb.Key) bool @@ -109,11 +107,11 @@ type EvalContext interface { // requests on the range. GetCurrentReadSummary(ctx context.Context) rspb.ReadSummary - // GetClosedTimestampV2 returns the current closed timestamp on the range. + // GetClosedTimestamp returns the current closed timestamp on the range. // It is expected that a caller will have performed some action (either // calling RevokeLease or WatchForMerge) to freeze further progression of // the closed timestamp before calling this method. - GetClosedTimestampV2(ctx context.Context) hlc.Timestamp + GetClosedTimestamp(ctx context.Context) hlc.Timestamp GetExternalStorage(ctx context.Context, dest roachpb.ExternalStorage) (cloud.ExternalStorage, error) GetExternalStorageFromURI(ctx context.Context, uri string, user security.SQLUsername) (cloud.ExternalStorage, @@ -208,9 +206,6 @@ func (m *mockEvalCtxImpl) GetTerm(uint64) (uint64, error) { func (m *mockEvalCtxImpl) GetLeaseAppliedIndex() uint64 { panic("unimplemented") } -func (m *mockEvalCtxImpl) GetTracker() closedts.TrackerI { - panic("unimplemented") -} func (m *mockEvalCtxImpl) Desc() *roachpb.RangeDescriptor { return m.MockEvalCtx.Desc } @@ -246,7 +241,7 @@ func (m *mockEvalCtxImpl) GetRangeInfo(ctx context.Context) roachpb.RangeInfo { func (m *mockEvalCtxImpl) GetCurrentReadSummary(ctx context.Context) rspb.ReadSummary { return m.CurrentReadSummary } -func (m *mockEvalCtxImpl) GetClosedTimestampV2(ctx context.Context) hlc.Timestamp { +func (m *mockEvalCtxImpl) GetClosedTimestamp(ctx context.Context) hlc.Timestamp { return m.ClosedTimestamp } func (m *mockEvalCtxImpl) GetExternalStorage( diff --git a/pkg/kv/kvserver/client_closed_timestamp_test.go b/pkg/kv/kvserver/client_closed_timestamp_test.go deleted file mode 100644 index 2ff9e711cf0b..000000000000 --- a/pkg/kv/kvserver/client_closed_timestamp_test.go +++ /dev/null @@ -1,145 +0,0 @@ -// Copyright 2020 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 kvserver_test - -import ( - "context" - "testing" - "time" - - "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/testutils" - "github.com/cockroachdb/cockroach/pkg/testutils/skip" - "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" - "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" - "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/errors" - "github.com/stretchr/testify/require" -) - -// TestTimestampsCanBeClosedWhenRequestsAreSentToNonLeaseHolders ensures that -// the errant closed timestamp requests sent to non-leaseholder nodes do not -// prevent future closed timestamps from being created if that node later -// becomes the leaseholder. See #48553 for more details. -func TestClosedTimestampWorksWhenRequestsAreSentToNonLeaseHolders(t *testing.T) { - defer leaktest.AfterTest(t)() - skip.WithIssue(t, 60682, "flaky test") - defer log.Scope(t).Close(t) - - ctx := context.Background() - // Set an incredibly long timeout so we don't need to risk node liveness - // failures and subsequent unexpected lease transfers under extreme stress. - serverArgs := base.TestServerArgs{ - RaftConfig: base.RaftConfig{RaftElectionTimeoutTicks: 1000}, - } - tc := testcluster.StartTestCluster(t, 2, base.TestClusterArgs{ - ServerArgs: serverArgs, - ReplicationMode: base.ReplicationManual, - }) - defer tc.Stopper().Stop(ctx) - - // We want to ensure that node 3 has a high epoch and then we want to - // make it the leaseholder of range and then we want to tickle requesting an - // MLAI from node 1. Then make node 1 the leaseholder and ensure that it - // can still close timestamps. - db1 := tc.Server(0).DB() - sqlRunner := sqlutils.MakeSQLRunner(tc.ServerConn(0)) - - // Set a very short closed timestamp target duration so that we don't need to - // wait long for the closed timestamp machinery to propagate information. - const closeInterval = 10 * time.Millisecond - sqlRunner.Exec(t, "SET CLUSTER SETTING kv.closed_timestamp.target_duration = '"+ - closeInterval.String()+"'") - sqlRunner.Exec(t, "SET CLUSTER SETTING kv.closed_timestamp.side_transport_interval = '"+ - closeInterval.String()+"'") - - // To make node3 have a large epoch, synthesize a liveness record for with - // epoch 1000 before starting the node. - require.NoError(t, db1.Put(ctx, keys.NodeLivenessKey(3), - &livenesspb.Liveness{ - NodeID: 3, - Epoch: 1000, - Expiration: hlc.LegacyTimestamp{WallTime: 1}, - })) - tc.AddAndStartServer(t, serverArgs) - - // Create our scratch range and up-replicate it. - k := tc.ScratchRange(t) - _, err := tc.AddVoters(k, tc.Target(1), tc.Target(2)) - require.NoError(t, err) - require.NoError(t, tc.WaitForVoters(k, tc.Target(1), tc.Target(2))) - - // Wrap transferring the lease to deal with errors due to initial node - // liveness for n3. We could probably alternatively wait for n3 to be live but - // that felt like more work at the time and this works. - transferLease := func(desc *roachpb.RangeDescriptor, target roachpb.ReplicationTarget) { - testutils.SucceedsSoon(t, func() error { - return tc.TransferRangeLease(*desc, target) - }) - } - - // transferLeaseAndWaitForClosed will transfer the lease to the serverIdx - // specified. It will ensure that the lease transfer happens and then will - // call afterLease. It will then wait until at the closed timestamp moves - // forward a few intervals. - transferLeaseAndWaitForClosed := func(serverIdx int, afterLease func()) { - _, repl := getFirstStoreReplica(t, tc.Server(serverIdx), k) - target := tc.Target(serverIdx) - transferLease(repl.Desc(), target) - testutils.SucceedsSoon(t, func() error { - if !repl.OwnsValidLease(ctx, db1.Clock().NowAsClockTimestamp()) { - return errors.Errorf("don't yet have the lease") - } - return nil - }) - if afterLease != nil { - afterLease() - } - nowClosed, ok := repl.MaxClosed(ctx) - require.True(t, ok) - lease, _ := repl.GetLease() - if lease.Replica.NodeID != target.NodeID { - t.Fatalf("lease was unexpectedly transferred away which should" + - " not happen given the very long timeouts") - } - const closedMultiple = 5 - targetClosed := nowClosed.Add(closedMultiple*closeInterval.Nanoseconds(), 0) - testutils.SucceedsSoon(t, func() error { - curLease, _ := repl.GetLease() - if !lease.Equivalent(curLease) { - t.Fatalf("lease was unexpectedly transferred away which should" + - " not happen given the very long timeouts") - } - closed, ok := repl.MaxClosed(ctx) - require.True(t, ok) - if closed.Less(targetClosed) { - return errors.Errorf("closed timestamp %v not yet after target %v", closed, targetClosed) - } - return nil - }) - } - - // Our new server should have a liveness epoch of 1000. - s3, repl3 := getFirstStoreReplica(t, tc.Server(2), k) - transferLeaseAndWaitForClosed(2, func() { - s3.RequestClosedTimestamp(1, repl3.RangeID) - }) - - // At this point we expect there's a high chance that the request made its - // way to n1. Now we're going to transfer the lease to n1 and make sure that - // the closed timestamp advances. - transferLeaseAndWaitForClosed(0, nil) -} diff --git a/pkg/kv/kvserver/client_merge_test.go b/pkg/kv/kvserver/client_merge_test.go index 8bca610e6b80..9083a9b8a00a 100644 --- a/pkg/kv/kvserver/client_merge_test.go +++ b/pkg/kv/kvserver/client_merge_test.go @@ -2144,8 +2144,8 @@ func TestStoreRangeMergeLHSLeaseTransfersAfterFreezeTime(t *testing.T) { } return nil }) - lhsClosedTS, ok := lhsLeaseholder.MaxClosed(ctx) - require.True(t, ok) + lhsClosedTS := lhsLeaseholder.MaxClosed(ctx) + require.NotEmpty(t, lhsClosedTS) // Finally, allow the merge to complete. It should complete successfully. close(finishSubsume) diff --git a/pkg/kv/kvserver/closed_timestamp_test.go b/pkg/kv/kvserver/closed_timestamp_test.go index c19578097be1..f5adc2842cd9 100644 --- a/pkg/kv/kvserver/closed_timestamp_test.go +++ b/pkg/kv/kvserver/closed_timestamp_test.go @@ -738,8 +738,7 @@ func TestClosedTimestampFrozenAfterSubsumption(t *testing.T) { require.NoError(t, err) r, err := store.GetReplica(rightDesc.RangeID) require.NoError(t, err) - maxClosed, ok := r.MaxClosed(ctx) - require.True(t, ok) + maxClosed := r.MaxClosed(ctx) // Note that maxClosed would not necessarily be below the freeze start if // this was a LEAD_FOR_GLOBAL_READS range. assert.True(t, maxClosed.LessEq(freezeStartTimestamp), diff --git a/pkg/kv/kvserver/closedts/BUILD.bazel b/pkg/kv/kvserver/closedts/BUILD.bazel index 1c52142d0487..9216354f40a1 100644 --- a/pkg/kv/kvserver/closedts/BUILD.bazel +++ b/pkg/kv/kvserver/closedts/BUILD.bazel @@ -3,14 +3,12 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "closedts", srcs = [ - "closedts.go", "policy.go", "setting.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts", visibility = ["//visibility:public"], deps = [ - "//pkg/kv/kvserver/closedts/ctpb", "//pkg/roachpb:with-mocks", "//pkg/settings", "//pkg/util/hlc", diff --git a/pkg/kv/kvserver/closedts/closedts.go b/pkg/kv/kvserver/closedts/closedts.go deleted file mode 100644 index db73913973bf..000000000000 --- a/pkg/kv/kvserver/closedts/closedts.go +++ /dev/null @@ -1,201 +0,0 @@ -// Copyright 2018 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 closedts houses the interfaces and basic definitions used by the -// various components of the closed timestamp subsystems. -// -// The following diagram illustrates how these components fit together. In -// running operation, the components are grouped in a container.Container -// (intended as a pass-around per-instance Singleton). -// Replicas proposing commands talk to the Tracker; replicas trying to serve -// follower reads talk to the Provider, which receives closed timestamp updates -// for the local node and its peers. -// -// Node 1 | Node 2 -// | -// +---------+ Close +-----------+ | +-----------+ -// | Tracker |<--------| | | | | -// +-----+---+ | +-------+ | | | +-------+ | CanServe -// ^ | |Storage| | | | |Storage| |<---------+ -// | | --------+ | | | +-------+ | | -// |Track | | | | | +----+----+ -// | | Provider | | | Provider | | Follower| -// | +-----------+ | +-----------+ | Replica | -// | ^ ^ +----+----+ -// | |Subscribe |Notify | -// | | | | -// +---------+ | Request | | -// |Proposing| Refresh +---+----+ <------ +---+-----+ Request | -// | Replica |<--------| Server | | Clients |<----------+ -// +---------+ +--------+ ------> +---------+ EnsureClient -// CT -package closedts - -import ( - "context" - - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" -) - -// IssueTrackingRemovalOfOldClosedTimestampsCode is the Github issue tracking -// the deletion of the "old" closed timestamps code (i.e. everything around -// here) in 21.2, now that 21.1 has a new Raft-based closed-timestamps -// mechanism. The old mechanism is disabled when the cluster version is -// sufficiently high, and all the tests failing because of it are skipped with -// this issue. -const IssueTrackingRemovalOfOldClosedTimestampsCode = 61299 - -// ReleaseFunc is a closure returned from Track which is used to record the -// LeaseAppliedIndex (LAI) given to a tracked proposal. The supplied epoch must -// match that of the lease under which the proposal was proposed. -type ReleaseFunc func(context.Context, ctpb.Epoch, roachpb.RangeID, ctpb.LAI) - -// TrackerI is part of the machinery enabling follower reads, that is, consistent -// reads served by replicas not holding the lease (for the requested timestamp). -// This data structure keeps tabs on ongoing command evaluations (which it -// forces to successively higher timestamps) and provides closed timestamp -// updates along with a map delta of minimum Lease Applied Indexes a replica -// wishing to serve a follower read must reach in order to do so correctly. -// -// See https://github.com/cockroachdb/cockroach/pull/26362 for more information. -// -// The methods exposed on Tracker are safe for concurrent use. -type TrackerI interface { - Close(next hlc.Timestamp, expCurEpoch ctpb.Epoch) (hlc.Timestamp, map[roachpb.RangeID]ctpb.LAI, bool) - Track(ctx context.Context) (hlc.Timestamp, ReleaseFunc) - FailedCloseAttempts() int64 -} - -// A Storage holds the closed timestamps and associated MLAIs for each node. It -// additionally provides historical information about past state that it -// "compacts" regularly, and which can be introspected via the VisitAscending -// method. -// -// The data in a Storage is ephemeral, i.e. is lost during process restarts. -// Introducing a persistent storage will require some design work to make -// sure a) that the records in the storage are certifiably up to date (they -// won't be naturally, unless we add a synchronous write to each proposal) -// and b) that the proposal at each MLAI has actually been proposed. It's -// unlikely that we'll ever find it useful to introduce persistence here -// (though we want to persist historical information for recovery after -// permanent loss of quorum, but there we only need some consistent on- -// disk state; we don't need to bootstrap it into a new consistent state -// that can be updated incrementally). -type Storage interface { - // VisitAscending visits the historical states contained within the Storage - // in ascending closed timestamp order. Each state (Entry) is full, i.e. - // non-incremental. The iteration stops when all states have been visited - // or the visitor returns true. - VisitAscending(roachpb.NodeID, func(ctpb.Entry) (done bool)) - // VisitDescending visits the historical states contained within the Storage - // in descending closed timestamp order. Each state (Entry) is full, i.e. - // non-incremental. The iteration stops when all states have been visited - // or the visitor returns true. - VisitDescending(roachpb.NodeID, func(ctpb.Entry) (done bool)) - // Add merges the given Entry into the state for the given NodeID. The first - // Entry passed in for any given Entry.Epoch must have Entry.Full set. - Add(roachpb.NodeID, ctpb.Entry) - // Clear removes all closed timestamp information from the Storage. It can - // be used to simulate the loss of information caused by a process restart. - Clear() -} - -// A Notifyee is a sink for closed timestamp updates. -type Notifyee interface { - // Notify returns a channel into which updates are written. - // - // In practice, the Notifyee will be a Provider. - Notify(roachpb.NodeID) chan<- ctpb.Entry -} - -// A Producer is a source of closed timestamp updates about the local node. -type Producer interface { - // The Subscribe method blocks and, until the context cancels, writes a - // stream of updates to the provided channel the aggregate of which is - // guaranteed to represent a valid (i.e. gapless) state. - Subscribe(context.Context, chan<- ctpb.Entry) -} - -// Provider is the central coordinator in the closed timestamp subsystem and the -// gatekeeper for the closed timestamp state for both local and remote nodes, -// which it handles in a symmetric fashion. It has the following tasks: -// -// 1. it accepts subscriptions for closed timestamp updates sourced from the -// local node. Upon accepting a subscription, the subscriber first receives -// the aggregate closed timestamp snapshot of the local node and then periodic -// updates. -// 2. it periodically closes out timestamps on the local node and passes the -// resulting entries to all of its subscribers. -// 3. it accepts notifications from other nodes, passing these updates through -// to its local storage, so that -// 4. the CanServe method determines via the underlying storage whether a -// given read can be satisfied via follower reads. -// 5. the MaxClosed method determines via the underlying storage what the maximum -// closed timestamp is for the specified LAI. -// TODO(tschottdorf): This is already adding some cruft to this nice interface. -// CanServe and MaxClosed are almost identical. -// -// Note that a Provider has no duty to immediately persist the local closed -// timestamps to the underlying storage. -type Provider interface { - Producer - Notifyee - Start() - MaxClosed(roachpb.NodeID, roachpb.RangeID, ctpb.Epoch, ctpb.LAI) hlc.Timestamp -} - -// A ClientRegistry is the client component of the follower reads subsystem. It -// contacts other nodes and requests a continuous stream of closed timestamp -// updates which it relays to the Provider. -type ClientRegistry interface { - // Request asynchronously notifies the given node that an update should be - // emitted for the given range. - Request(roachpb.NodeID, roachpb.RangeID) - // EnsureClient instructs the registry to (asynchronously) request a stream - // of closed timestamp updates from the given node. - EnsureClient(roachpb.NodeID) -} - -// CloseFn is periodically called by Producers to close out new timestamps. -// Outside of tests, it corresponds to (*Tracker).Close; see there for a -// detailed description of the semantics. The final returned boolean indicates -// whether tracked epoch matched the expCurEpoch and that returned information -// may be used. -type CloseFn func(next hlc.Timestamp, expCurEpoch ctpb.Epoch) (hlc.Timestamp, map[roachpb.RangeID]ctpb.LAI, bool) - -// AsCloseFn uses the TrackerI as a CloseFn. -func AsCloseFn(t TrackerI) CloseFn { - return func(next hlc.Timestamp, expCurEpoch ctpb.Epoch) (hlc.Timestamp, map[roachpb.RangeID]ctpb.LAI, bool) { - return t.Close(next, expCurEpoch) - } -} - -// LiveClockFn supplies a current HLC timestamp from the local node with the -// extra constraints that the local node is live for the returned timestamp at -// the given epoch. The NodeID is passed in to make this method easier to define -// before the NodeID is known. -type LiveClockFn func(roachpb.NodeID) (liveNow hlc.Timestamp, liveEpoch ctpb.Epoch, _ error) - -// RefreshFn is called by the Producer when it is asked to manually create (and -// emit) an update for a number of its replicas. The closed timestamp subsystem -// intentionally knows as little about the outside world as possible, and this -// function, injected from the outside, provides the minimal glue. Its job is -// to register a proposal for the current lease applied indexes of the replicas -// with the Tracker, so that updates for them are emitted soon thereafter. -type RefreshFn func(...roachpb.RangeID) - -// A Dialer opens closed timestamp connections to receive updates from remote -// nodes. -type Dialer interface { - Dial(context.Context, roachpb.NodeID) (ctpb.Client, error) - Ready(roachpb.NodeID) bool // if false, Dial is likely to fail -} diff --git a/pkg/kv/kvserver/closedts/container/BUILD.bazel b/pkg/kv/kvserver/closedts/container/BUILD.bazel index 5bc884b341ea..e69de29bb2d1 100644 --- a/pkg/kv/kvserver/closedts/container/BUILD.bazel +++ b/pkg/kv/kvserver/closedts/container/BUILD.bazel @@ -1,49 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") - -go_library( - name = "container", - srcs = [ - "container.go", - "noop.go", - ], - importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/container", - visibility = ["//visibility:public"], - deps = [ - "//pkg/kv/kvserver/closedts", - "//pkg/kv/kvserver/closedts/ctpb", - "//pkg/kv/kvserver/closedts/minprop", - "//pkg/kv/kvserver/closedts/provider", - "//pkg/kv/kvserver/closedts/storage", - "//pkg/kv/kvserver/closedts/transport", - "//pkg/roachpb:with-mocks", - "//pkg/settings/cluster", - "//pkg/util/hlc", - "//pkg/util/stop", - "@com_github_cockroachdb_errors//:errors", - "@org_golang_google_grpc//:go_default_library", - ], -) - -go_test( - name = "container_test", - size = "small", - srcs = ["container_test.go"], - deps = [ - ":container", - "//pkg/kv/kvserver/closedts", - "//pkg/kv/kvserver/closedts/ctpb", - "//pkg/kv/kvserver/closedts/provider/testutils", - "//pkg/kv/kvserver/closedts/transport/testutils", - "//pkg/roachpb:with-mocks", - "//pkg/settings/cluster", - "//pkg/testutils", - "//pkg/testutils/skip", - "//pkg/util/hlc", - "//pkg/util/leaktest", - "//pkg/util/stop", - "//pkg/util/syncutil", - "@com_github_cockroachdb_errors//:errors", - "@com_github_kr_pretty//:pretty", - "@com_github_stretchr_testify//require", - ], -) diff --git a/pkg/kv/kvserver/closedts/container/container.go b/pkg/kv/kvserver/closedts/container/container.go deleted file mode 100644 index e26c8ed27c33..000000000000 --- a/pkg/kv/kvserver/closedts/container/container.go +++ /dev/null @@ -1,153 +0,0 @@ -// Copyright 2018 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 container - -import ( - "sync/atomic" - "time" - - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/minprop" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/provider" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/storage" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/transport" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/util/stop" - "github.com/cockroachdb/errors" - "google.golang.org/grpc" -) - -// Config is a container that holds references to all of the components required -// to set up a full closed timestamp subsystem. -type Config struct { - Settings *cluster.Settings - Stopper *stop.Stopper - Clock closedts.LiveClockFn - Refresh closedts.RefreshFn - Dialer closedts.Dialer -} - -// A Container is a full closed timestamp subsystem along with the Config it was -// created from. -type Container struct { - Config - // Initialized on Start(). - Tracker closedts.TrackerI - Storage closedts.Storage - Provider closedts.Provider - Server ctpb.Server - Clients closedts.ClientRegistry - - nodeID roachpb.NodeID - delayedServer *delayedServer - noop bool // if true, is NoopContainer -} - -const ( - // For each node, keep two historical buckets (i.e. one recent one, and one that - // lagging followers can still satisfy some reads from). - storageBucketNum = 2 - // StorageBucketScale determines the (exponential) spacing of storage buckets. - // For example, a scale of 5s means that the second bucket will attempt to hold - // a closed timestamp 5s in the past from the first, and the third 5*5=25s from - // the first, etc. - // - // TODO(tschottdorf): it's straightforward to make this dynamic. It should track - // the interval at which timestamps are closed out, ideally being a little shorter. - // The effect of that would be that the most recent closed timestamp and the previous - // one can be queried against separately. - StorageBucketScale = 10 * time.Second -) - -// NewContainer initializes a Container from the given Config. The Container -// will need to be started separately, and will only be populated during Start(). -// -// However, its RegisterClosedTimestampServer method can only be called before -// the Container is started. -func NewContainer(cfg Config) *Container { - return &Container{ - Config: cfg, - } -} - -type delayedServer struct { - active int32 // atomic - s ctpb.Server -} - -func (s *delayedServer) Start() { - atomic.StoreInt32(&s.active, 1) -} - -func (s delayedServer) Get(client ctpb.ClosedTimestamp_GetServer) error { - if atomic.LoadInt32(&s.active) == 0 { - return errors.New("not available yet") - } - return s.s.Get(client) -} - -// RegisterClosedTimestampServer registers the Server contained in the container -// with gRPC. -func (c *Container) RegisterClosedTimestampServer(s *grpc.Server) { - c.delayedServer = &delayedServer{} - ctpb.RegisterClosedTimestampServer(s, c.delayedServer) -} - -// Start starts the Container. The Stopper used to create the Container is in -// charge of stopping it. -func (c *Container) Start(nodeID roachpb.NodeID) { - cfg := c.Config - - if c.noop { - return - } - - storage := storage.NewMultiStorage(func() storage.SingleStorage { - return storage.NewMemStorage(StorageBucketScale, storageBucketNum) - }) - - tracker := minprop.NewTracker() - - pConf := provider.Config{ - NodeID: nodeID, - Settings: cfg.Settings, - Stopper: cfg.Stopper, - Storage: storage, - Clock: cfg.Clock, - Close: closedts.AsCloseFn(tracker), - } - - provider := provider.NewProvider(&pConf) - - server := transport.NewServer(cfg.Stopper, provider, cfg.Refresh) - - rConf := transport.Config{ - NodeID: nodeID, - Settings: cfg.Settings, - Stopper: cfg.Stopper, - Dialer: cfg.Dialer, - Sink: provider, - } - - c.nodeID = nodeID - c.Storage = storage - c.Tracker = tracker - c.Server = server - c.Clients = transport.NewClients(rConf) - c.Provider = provider - c.Provider.Start() - if c.delayedServer != nil { - c.delayedServer.s = server - c.delayedServer.Start() - } -} diff --git a/pkg/kv/kvserver/closedts/container/container_test.go b/pkg/kv/kvserver/closedts/container/container_test.go deleted file mode 100644 index 0b120ea9225a..000000000000 --- a/pkg/kv/kvserver/closedts/container/container_test.go +++ /dev/null @@ -1,373 +0,0 @@ -// Copyright 2018 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 container_test // intentionally test from external package - -import ( - "context" - "reflect" - "sync" - "testing" - "time" - - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/container" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" - providertestutils "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/provider/testutils" - transporttestutils "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/transport/testutils" - "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/skip" - "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/cockroachdb/cockroach/pkg/util/stop" - "github.com/cockroachdb/cockroach/pkg/util/syncutil" - "github.com/cockroachdb/errors" - "github.com/kr/pretty" - "github.com/stretchr/testify/require" -) - -type LateBoundDialer struct { - Wrapped *transporttestutils.ChanDialer -} - -func (d *LateBoundDialer) Dial(ctx context.Context, nodeID roachpb.NodeID) (ctpb.Client, error) { - return d.Wrapped.Dial(ctx, nodeID) -} - -func (d *LateBoundDialer) Ready(nodeID roachpb.NodeID) bool { - return d.Wrapped.Ready(nodeID) -} - -type TestContainer struct { - *container.Container - NodeID roachpb.NodeID - Refreshed struct { - syncutil.Mutex - RangeIDs []roachpb.RangeID - } - Dialer *LateBoundDialer - TestClock *providertestutils.TestClock -} - -func prepareContainer() *TestContainer { - stopper := stop.NewStopper() - - tc := &TestContainer{} - - tc.TestClock = providertestutils.NewTestClock(stopper) - - var wg sync.WaitGroup - wg.Add(1) - refresh := func(requested ...roachpb.RangeID) { - tc.Refreshed.Lock() - tc.Refreshed.RangeIDs = append(tc.Refreshed.RangeIDs, requested...) - tc.Refreshed.Unlock() - } - - st := cluster.MakeTestingClusterSettings() - - // Set the target duration to a second and the close fraction so small - // that the Provider will essentially close in a hot loop. In this test - // we'll block in the clock to pace the Provider's closer loop. - closedts.TargetDuration.Override(context.Background(), &st.SV, time.Second) - closedts.CloseFraction.Override(context.Background(), &st.SV, 1e-9) - - // We perform a little dance with the Dialer. It needs to be hooked up to the - // Server, but that's only created in NewContainer. The Dialer isn't used until - // that point, so we just create it a little later. - tc.Dialer = &LateBoundDialer{} - - cfg := container.Config{ - Settings: st, - Stopper: stopper, - Clock: tc.TestClock.LiveNow, - Refresh: refresh, - Dialer: tc.Dialer, - } - - tc.Container = container.NewContainer(cfg) - return tc -} - -func setupTwoNodeTest() (_ *TestContainer, _ *TestContainer, shutdown func()) { - c1 := prepareContainer() - c2 := prepareContainer() - - c1.NodeID = roachpb.NodeID(1) - c2.NodeID = roachpb.NodeID(2) - - c1.Start(c1.NodeID) - c2.Start(c2.NodeID) - - // Link the containers. - c1.Dialer.Wrapped = transporttestutils.NewChanDialer(c1.Stopper, c2.Server) - c2.Dialer.Wrapped = transporttestutils.NewChanDialer(c2.Stopper, c1.Server) - - return c1, c2, func() { - // Oh, the joy of multiple stoppers. - var wg sync.WaitGroup - wg.Add(2) - go func() { - defer wg.Done() - c1.Stopper.Stop(context.Background()) - }() - go func() { - defer wg.Done() - c2.Stopper.Stop(context.Background()) - }() - wg.Wait() - } -} - -func TestTwoNodes(t *testing.T) { - defer leaktest.AfterTest(t)() - skip.WithIssue(t, closedts.IssueTrackingRemovalOfOldClosedTimestampsCode) - - ctx := context.Background() - - c1, c2, shutdown := setupTwoNodeTest() - defer shutdown() - defer func() { - t.Logf("n1 -> n2: %s", pretty.Sprint(c1.Dialer.Wrapped.Transcript(c2.NodeID))) - t.Logf("n2 -> n1: %s", pretty.Sprint(c2.Dialer.Wrapped.Transcript(c1.NodeID))) - }() - const ( - ep0 ctpb.Epoch = iota - ep1 - ep2 - ) - // Initially, can't serve random things for either n1 or n2. - require.True(t, c1.Container.Provider.MaxClosed( - c1.NodeID, roachpb.RangeID(5), ep0, ctpb.LAI(0)).IsEmpty(), - ) - require.True(t, c1.Container.Provider.MaxClosed( - c2.NodeID, roachpb.RangeID(5), ep0, ctpb.LAI(0)).IsEmpty(), - ) - - // Track and release a command. - ts, release := c1.Tracker.Track(ctx) - release(ctx, ep1, roachpb.RangeID(17), ctpb.LAI(12)) - - // The command is forced above ts=0.2. This is just an artifact of how the - // Tracker is implemented - it closes out 0.1 first, so it begins by forcing - // commands just above that. - require.Equal(t, hlc.Timestamp{Logical: 2}, ts) - - // The clock gives a timestamp to the Provider, which should close out the - // current timestamp and set up 2E9-1E9=1E9 as the next one it wants to close. - // We do this twice (for the same timestamp) to make sure that the Provider - // not only read the tick, but also processed it. Otherwise, it becomes hard - // to write the remainder of the test because the commands we track below may - // fall into either case, and may be forced above the old or new timestamp. - for i := 0; i < 2; i++ { - c1.TestClock.Tick(hlc.Timestamp{WallTime: 2e9}, ep1, nil) - } - - // The Tracker still won't let us serve anything, even though it has closed out - // 0.1 - this is because it has no information about any ranges at that timestamp. - // (Note that the Tracker may not have processed the closing yet, so if there were - // a bug here, this test would fail flakily - that's ok). - require.True(t, c1.Container.Provider.MaxClosed( - c1.NodeID, roachpb.RangeID(17), ep1, ctpb.LAI(12)).IsEmpty(), - ) - - // Two more commands come in. - ts, release = c1.Tracker.Track(ctx) - release(ctx, ep1, roachpb.RangeID(17), ctpb.LAI(16)) - require.Equal(t, hlc.Timestamp{WallTime: 1e9, Logical: 1}, ts) - - ts, release = c1.Tracker.Track(ctx) - release(ctx, ep1, roachpb.RangeID(8), ctpb.LAI(88)) - require.Equal(t, hlc.Timestamp{WallTime: 1e9, Logical: 1}, ts) - - // Now another tick. Shortly after it, we should be able to serve below 1E9, and 2E9 should - // be the next planned closed timestamp (though we can only verify the former). - c1.TestClock.Tick(hlc.Timestamp{WallTime: 3e9}, ep1, nil) - - testutils.SucceedsSoon(t, func() error { - if c1.Container.Provider.MaxClosed( - c1.NodeID, roachpb.RangeID(17), ep1, ctpb.LAI(12), - ).Less(hlc.Timestamp{WallTime: 1e9}) { - return errors.New("still can't serve") - } - return nil - }) - - // Shouldn't be able to serve the same thing if we haven't caught up yet. - require.False(t, !c1.Container.Provider.MaxClosed( - c1.NodeID, roachpb.RangeID(17), ep1, ctpb.LAI(11), - ).Less(hlc.Timestamp{WallTime: 1e9})) - - // Shouldn't be able to serve at a higher timestamp. - require.False(t, !c1.Container.Provider.MaxClosed( - c1.NodeID, roachpb.RangeID(17), ep1, ctpb.LAI(12), - ).Less(hlc.Timestamp{WallTime: 1e9, Logical: 1})) - - // Now things get a little more interesting. Tell node2 to get a stream of - // information from node1. We do this via Request, which as a side effect lets - // us ascertain that this request makes it to n1. - c2.Clients.Request(roachpb.NodeID(1), roachpb.RangeID(18)) - testutils.SucceedsSoon(t, func() error { - exp := []roachpb.RangeID{18} - c1.Refreshed.Lock() - defer c1.Refreshed.Unlock() - if !reflect.DeepEqual(exp, c1.Refreshed.RangeIDs) { - return errors.Errorf("still waiting for %v: currently %v", exp, c1.Refreshed.RangeIDs) - } - return nil - }) - - // And n2 should soon also be able to serve follower reads for a range lead by - // n1 when it has caught up. - testutils.SucceedsSoon(t, func() error { - if c2.Container.Provider.MaxClosed( - c1.NodeID, roachpb.RangeID(17), ep1, ctpb.LAI(12), - ).Less(hlc.Timestamp{WallTime: 1e9}) { - return errors.New("n2 still can't serve") - } - return nil - }) - - // Remember the other proposals we tracked above on n1: (r17, 16) and (r8, 88). Feeding another - // timestamp to n1, we should see them closed out at t=2E9, and both n1 and n2 should automatically - // be able to serve them soon thereafter. - c1.TestClock.Tick(hlc.Timestamp{WallTime: 4e9}, ep1, nil) - - checkEpoch1Reads := func(ts hlc.Timestamp) { - t.Helper() - for i, c := range []*TestContainer{c1, c2} { - for _, tuple := range []struct { - roachpb.RangeID - ctpb.LAI - }{ - {17, 16}, - {8, 88}, - } { - testutils.SucceedsSoon(t, func() error { - t.Helper() - if c.Container.Provider.MaxClosed( - c1.NodeID, tuple.RangeID, ep1, tuple.LAI, - ).Less(ts) { - return errors.Errorf("n%d still can't serve (r%d,%d) @ %s", i+1, tuple.RangeID, tuple.LAI, ts) - } - return nil - }) - // Still can't serve when not caught up. - require.False(t, !c.Container.Provider.MaxClosed( - c1.NodeID, tuple.RangeID, ep1, tuple.LAI-1, - ).Less(ts)) - // Can serve when more than caught up. - require.True(t, !c.Container.Provider.MaxClosed( - c1.NodeID, tuple.RangeID, ep1, tuple.LAI+1, - ).Less(ts)) - // Can't serve when in different epoch, no matter larger or smaller. - require.False(t, !c.Container.Provider.MaxClosed( - c1.NodeID, tuple.RangeID, ep0, tuple.LAI, - ).Less(ts)) - require.False(t, !c.Container.Provider.MaxClosed( - c1.NodeID, tuple.RangeID, ep2, tuple.LAI, - ).Less(ts)) - } - } - } - checkEpoch1Reads(hlc.Timestamp{WallTime: 2e9}) - - // Tick again in epoch 1 and ensure that reads at t=3E9 can be safely served. - // 3E9 gets closed out under the first epoch in this tick with 4E9 as the - // timestamp to be closed next due to the 1s target interval. - c1.TestClock.Tick(hlc.Timestamp{WallTime: 5e9}, ep1, nil) - checkEpoch1Reads(hlc.Timestamp{WallTime: 3e9}) - - // Uh-oh! n1 must've missed a heartbeat. The epoch goes up by one. This means - // that soon (after the next tick) timestamps should be closed out under the - // the epoch. The timestamp at which this happens is doctored to make sure the - // Storage holds on to the past information, because we want to end-to-end test - // that this all works out. Consequently we try Tick at the rotation interval - // plus the target duration next (so that the next closed timestamp is the - // rotation interval). - c1.TestClock.Tick(hlc.Timestamp{WallTime: int64(container.StorageBucketScale) + 5e9}, ep2, nil) - - // Previously valid reads should remain valid. - checkEpoch1Reads(hlc.Timestamp{WallTime: 2e9}) - checkEpoch1Reads(hlc.Timestamp{WallTime: 3e9}) - - // After the above tick makes it to the tracker, commands get forced above - // the next closed timestamp (from the tick above) minus target interval. - // The SucceedsSoon is to ensure that the above tick in ep2 has made it to the tracker. - testutils.SucceedsSoon(t, func() error { - ts, release = c1.Tracker.Track(ctx) - release(ctx, ep2, roachpb.RangeID(123), ctpb.LAI(456)) - if !(&hlc.Timestamp{WallTime: int64(container.StorageBucketScale) + 4e9, Logical: 1}).Equal(ts) { - return errors.Errorf("command still not forced above %v", ts) - } - return nil - }) - - // Previously valid reads should remain valid. - checkEpoch1Reads(hlc.Timestamp{WallTime: 2e9}) - checkEpoch1Reads(hlc.Timestamp{WallTime: 3e9}) - - // With the next tick, epoch two fully goes into effect (as the first epoch two - // timestamp gets closed out). We do this twice to make sure it's processed before - // the test proceeds. - c1.TestClock.Tick(hlc.Timestamp{WallTime: int64(container.StorageBucketScale) + 6e9}, ep2, nil) - - // Previously valid reads should remain valid. Note that this is because the - // storage keeps historical data, and we've fine tuned the epoch flip so that - // it happens after the epoch 1 information rotates into another bucket and - // thus is preserved. If the epoch changed at a smaller timestamp, that - // would've wiped out the first epoch's information. - // - // TODO(tschottdorf): we could make the storage smarter so that it forces a - // rotation when the epoch changes, at the expense of pushing out historical - // information earlier. Frequent epoch changes could lead to very little - // historical information in the storage. Probably better not to risk that. - checkEpoch1Reads(hlc.Timestamp{WallTime: 2e9}) - checkEpoch1Reads(hlc.Timestamp{WallTime: 3e9}) - - // Another second, another tick. Now the proposal tracked during epoch 2 should - // be readable from followers (as `scale+5E9` gets closed out). - c1.TestClock.Tick(hlc.Timestamp{WallTime: int64(container.StorageBucketScale) + 7e9}, ep2, nil) - for i, c := range []*TestContainer{c1, c2} { - rangeID := roachpb.RangeID(123) - lai := ctpb.LAI(456) - epoch := ep2 - ts := hlc.Timestamp{WallTime: int64(container.StorageBucketScale) + 5e9} - - testutils.SucceedsSoon(t, func() error { - if c.Container.Provider.MaxClosed( - c1.NodeID, rangeID, epoch, lai, - ).Less(ts) { - return errors.Errorf("n%d still can't serve (r%d,%d) @ %s", i+1, rangeID, lai, ts) - } - return nil - }) - - // Still can't serve when not caught up. - require.False(t, !c.Container.Provider.MaxClosed( - c1.NodeID, rangeID, epoch, lai-1, - ).Less(ts)) - - // Can serve when more than caught up. - require.True(t, !c.Container.Provider.MaxClosed( - c1.NodeID, rangeID, epoch, lai+1, - ).Less(ts)) - - // Can't serve when in different epoch, no matter larger or smaller. - require.False(t, !c.Container.Provider.MaxClosed( - c1.NodeID, rangeID, epoch-1, lai, - ).Less(ts)) - require.False(t, !c.Container.Provider.MaxClosed( - c1.NodeID, rangeID, epoch+1, lai, - ).Less(ts)) - } -} diff --git a/pkg/kv/kvserver/closedts/container/noop.go b/pkg/kv/kvserver/closedts/container/noop.go deleted file mode 100644 index abeec20f4872..000000000000 --- a/pkg/kv/kvserver/closedts/container/noop.go +++ /dev/null @@ -1,86 +0,0 @@ -// Copyright 2018 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 container - -import ( - "context" - - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/errors" -) - -type noopEverything struct{} - -// NoopContainer returns a Container for which all parts of the subsystem are -// mocked out. This is for usage in testing where there just needs to be a -// structure that stays out of the way. -// -// The returned container will behave correctly. It will never allow any time- -// stamps to be closed out, so it never makes any promises; it doesn't use any -// locking and it does not consume any (nontrivial) resources. -func NoopContainer() *Container { - return &Container{ - Config: Config{ - Settings: cluster.MakeTestingClusterSettings(), - Stopper: nil, - Clock: func(roachpb.NodeID) (hlc.Timestamp, ctpb.Epoch, error) { - return hlc.Timestamp{}, 0, errors.New("closed timestamps disabled for testing") - }, - Refresh: func(...roachpb.RangeID) {}, - Dialer: noopEverything{}, - }, - Tracker: noopEverything{}, - Storage: noopEverything{}, - Provider: noopEverything{}, - Server: noopEverything{}, - Clients: noopEverything{}, - noop: true, - } -} - -func (noopEverything) Get(client ctpb.InboundClient) error { - return errors.New("closed timestamps disabled") -} -func (noopEverything) Close( - next hlc.Timestamp, expCurEpoch ctpb.Epoch, -) (hlc.Timestamp, map[roachpb.RangeID]ctpb.LAI, bool) { - return hlc.Timestamp{}, nil, false -} -func (noopEverything) Track(ctx context.Context) (hlc.Timestamp, closedts.ReleaseFunc) { - return hlc.Timestamp{}, func(context.Context, ctpb.Epoch, roachpb.RangeID, ctpb.LAI) {} -} -func (noopEverything) FailedCloseAttempts() int64 { - return 0 -} -func (noopEverything) VisitAscending(roachpb.NodeID, func(ctpb.Entry) (done bool)) {} -func (noopEverything) VisitDescending(roachpb.NodeID, func(ctpb.Entry) (done bool)) {} -func (noopEverything) Add(roachpb.NodeID, ctpb.Entry) {} -func (noopEverything) Clear() {} -func (noopEverything) Notify(roachpb.NodeID) chan<- ctpb.Entry { - return nil // will explode when used, but nobody would use this -} -func (noopEverything) Subscribe(context.Context, chan<- ctpb.Entry) {} -func (noopEverything) Start() {} -func (noopEverything) MaxClosed( - roachpb.NodeID, roachpb.RangeID, ctpb.Epoch, ctpb.LAI, -) hlc.Timestamp { - return hlc.Timestamp{} -} -func (noopEverything) Request(roachpb.NodeID, roachpb.RangeID) {} -func (noopEverything) EnsureClient(roachpb.NodeID) {} -func (noopEverything) Dial(context.Context, roachpb.NodeID) (ctpb.Client, error) { - return nil, errors.New("closed timestamps disabled") -} -func (noopEverything) Ready(roachpb.NodeID) bool { return false } diff --git a/pkg/kv/kvserver/closedts/ctpb/BUILD.bazel b/pkg/kv/kvserver/closedts/ctpb/BUILD.bazel index 79121879c480..e31e2fc0c9e1 100644 --- a/pkg/kv/kvserver/closedts/ctpb/BUILD.bazel +++ b/pkg/kv/kvserver/closedts/ctpb/BUILD.bazel @@ -4,12 +4,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "ctpb", - srcs = [ - "client.go", - "entry.go", - "server.go", - "service.go", - ], + srcs = ["service.go"], embed = [":ctpb_go_proto"], importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb", visibility = ["//visibility:public"], @@ -21,10 +16,7 @@ go_library( proto_library( name = "ctpb_proto", - srcs = [ - "entry.proto", - "service.proto", - ], + srcs = ["service.proto"], strip_import_prefix = "/pkg", visibility = ["//visibility:public"], deps = [ diff --git a/pkg/kv/kvserver/closedts/ctpb/client.go b/pkg/kv/kvserver/closedts/ctpb/client.go deleted file mode 100644 index 4677f0273134..000000000000 --- a/pkg/kv/kvserver/closedts/ctpb/client.go +++ /dev/null @@ -1,23 +0,0 @@ -// Copyright 2018 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 ctpb - -import "context" - -// Client is the interface for closed timestamp update clients. -type Client interface { - Send(*Reaction) error - Recv() (*Entry, error) - CloseSend() error - Context() context.Context -} - -var _ Client = ClosedTimestamp_GetClient(nil) diff --git a/pkg/kv/kvserver/closedts/ctpb/entry.go b/pkg/kv/kvserver/closedts/ctpb/entry.go deleted file mode 100644 index 9b1d8a376272..000000000000 --- a/pkg/kv/kvserver/closedts/ctpb/entry.go +++ /dev/null @@ -1,58 +0,0 @@ -// Copyright 2018 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 ctpb - -import ( - "fmt" - "sort" - "strings" - - "github.com/cockroachdb/cockroach/pkg/roachpb" -) - -// Epoch is an int64 with its own type to avoid mix-ups in positional arguments. -type Epoch int64 - -// LAI is an int64 denoting a lease applied index with its own type to avoid -// mix-ups in positional arguments. -type LAI int64 - -// SafeValue implements the redact.SafeValue interface. -func (LAI) SafeValue() {} - -// String formats Entry for human consumption as well as testing (by avoiding -// randomness in the output caused by map iteraton order). -func (e Entry) String() string { - rangeIDs := make([]roachpb.RangeID, 0, len(e.MLAI)) - for k := range e.MLAI { - rangeIDs = append(rangeIDs, k) - } - - sort.Slice(rangeIDs, func(i, j int) bool { - a, b := rangeIDs[i], rangeIDs[j] - if a == b { - return e.MLAI[a] < e.MLAI[b] - } - return a < b - }) - sl := make([]string, 0, len(rangeIDs)) - for _, rangeID := range rangeIDs { - sl = append(sl, fmt.Sprintf("r%d: %d", rangeID, e.MLAI[rangeID])) - } - if len(sl) == 0 { - sl = []string{"(empty)"} - } - return fmt.Sprintf("CT: %s @ Epoch %d\nFull: %t\nMLAI: %s\n", e.ClosedTimestamp, e.Epoch, e.Full, strings.Join(sl, ", ")) -} - -func (r Reaction) String() string { - return fmt.Sprintf("Refresh: %v", r.Requested) -} diff --git a/pkg/kv/kvserver/closedts/ctpb/entry.pb.go b/pkg/kv/kvserver/closedts/ctpb/entry.pb.go deleted file mode 100644 index ee7169ce0afa..000000000000 --- a/pkg/kv/kvserver/closedts/ctpb/entry.pb.go +++ /dev/null @@ -1,762 +0,0 @@ -// Code generated by protoc-gen-gogo. DO NOT EDIT. -// source: kv/kvserver/closedts/ctpb/entry.proto - -package ctpb - -import ( - fmt "fmt" - github_com_cockroachdb_cockroach_pkg_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" - hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" - _ "github.com/gogo/protobuf/gogoproto" - proto "github.com/gogo/protobuf/proto" - github_com_gogo_protobuf_sortkeys "github.com/gogo/protobuf/sortkeys" - _ "google.golang.org/genproto/googleapis/api/annotations" - io "io" - math "math" - math_bits "math/bits" -) - -// Reference imports to suppress errors if they are not otherwise used. -var _ = proto.Marshal -var _ = fmt.Errorf -var _ = math.Inf - -// This is a compile-time assertion to ensure that this generated file -// is compatible with the proto package it is being compiled against. -// A compilation error at this line likely means your copy of the -// proto package needs to be updated. -const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package - -// An Entry is a closed timestamp update. It consists of a closed timestamp -// (i.e. a timestamp at or below which the origin node guarantees no more new -// writes are going to be permitted), an associated epoch in which the origin -// node promises it was live (for the closed timestamp), a map of minimum lease -// applied indexes (which have to be caught up to before being allowed to use -// the closed timestamp) as well as an indicator of whether this update supplies -// a full initial state or an increment to be merged into a previous state. In -// practice, the first Entry received for each epoch is full, while the remainder -// are incremental. An incremental update represents the implicit promise that -// the state accumulated since the last full Entry is the true full state. -type Entry struct { - Epoch Epoch `protobuf:"varint,1,opt,name=epoch,proto3,casttype=Epoch" json:"epoch,omitempty"` - ClosedTimestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=closed_timestamp,json=closedTimestamp,proto3" json:"closed_timestamp"` - MLAI map[github_com_cockroachdb_cockroach_pkg_roachpb.RangeID]LAI `protobuf:"bytes,3,rep,name=mlai,proto3,castkey=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID,castvalue=LAI" json:"mlai,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` - // Full is true if the emitter promises that any future write to any range - // mentioned in this Entry will be reflected in a subsequent Entry before any - // stale follower reads are possible. For example, if range 1 is assigned an - // MLAI of 12 in this Entry and isn't mentioned in the five subsequent - // entries, the recipient may behave as if the MLAI of 12 were repeated across - // all of these entries. - // - // In practice, a Full message is received when a stream of Entries is first - // established (or the Epoch changes), and all other updates are incremental - // (i.e. not Full). - Full bool `protobuf:"varint,4,opt,name=full,proto3" json:"full,omitempty"` -} - -func (m *Entry) Reset() { *m = Entry{} } -func (*Entry) ProtoMessage() {} -func (*Entry) Descriptor() ([]byte, []int) { - return fileDescriptor_db146746651382e6, []int{0} -} -func (m *Entry) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *Entry) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - b = b[:cap(b)] - n, err := m.MarshalToSizedBuffer(b) - if err != nil { - return nil, err - } - return b[:n], nil -} -func (m *Entry) XXX_Merge(src proto.Message) { - xxx_messageInfo_Entry.Merge(m, src) -} -func (m *Entry) XXX_Size() int { - return m.Size() -} -func (m *Entry) XXX_DiscardUnknown() { - xxx_messageInfo_Entry.DiscardUnknown(m) -} - -var xxx_messageInfo_Entry proto.InternalMessageInfo - -// Reactions flow in the direction opposite to Entries and request for ranges to -// be included in the next Entry. Under rare circumstances, ranges may be omitted -// from closed timestamp updates, and so serving follower reads from them would -// fail. The Reaction mechanism serves to explicitly request the missing information -// when that happens. -type Reaction struct { - Requested []github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,1,rep,packed,name=Requested,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"Requested,omitempty"` -} - -func (m *Reaction) Reset() { *m = Reaction{} } -func (*Reaction) ProtoMessage() {} -func (*Reaction) Descriptor() ([]byte, []int) { - return fileDescriptor_db146746651382e6, []int{1} -} -func (m *Reaction) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *Reaction) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - b = b[:cap(b)] - n, err := m.MarshalToSizedBuffer(b) - if err != nil { - return nil, err - } - return b[:n], nil -} -func (m *Reaction) XXX_Merge(src proto.Message) { - xxx_messageInfo_Reaction.Merge(m, src) -} -func (m *Reaction) XXX_Size() int { - return m.Size() -} -func (m *Reaction) XXX_DiscardUnknown() { - xxx_messageInfo_Reaction.DiscardUnknown(m) -} - -var xxx_messageInfo_Reaction proto.InternalMessageInfo - -func init() { - proto.RegisterType((*Entry)(nil), "cockroach.kv.kvserver.ctupdate.Entry") - proto.RegisterMapType((map[github_com_cockroachdb_cockroach_pkg_roachpb.RangeID]LAI)(nil), "cockroach.kv.kvserver.ctupdate.Entry.MlaiEntry") - proto.RegisterType((*Reaction)(nil), "cockroach.kv.kvserver.ctupdate.Reaction") -} - -func init() { - proto.RegisterFile("kv/kvserver/closedts/ctpb/entry.proto", fileDescriptor_db146746651382e6) -} - -var fileDescriptor_db146746651382e6 = []byte{ - // 442 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x92, 0x3f, 0x6f, 0xd4, 0x30, - 0x18, 0xc6, 0xe3, 0x4b, 0x82, 0x7a, 0xee, 0x40, 0x15, 0x75, 0x88, 0x4e, 0xe0, 0x44, 0x95, 0x40, - 0x99, 0x6c, 0xa9, 0x20, 0x51, 0x75, 0x6b, 0x44, 0x85, 0x4e, 0x6a, 0x19, 0x2c, 0xc4, 0xc0, 0x82, - 0x1c, 0x9f, 0x49, 0xa2, 0xf8, 0xe2, 0x90, 0x38, 0x91, 0xba, 0x32, 0x22, 0x06, 0x46, 0x46, 0x3e, - 0xce, 0x8d, 0x1d, 0x2b, 0x86, 0x2b, 0xe4, 0xbe, 0x45, 0x27, 0x14, 0x87, 0xde, 0x6d, 0x0c, 0xdd, - 0x7e, 0x79, 0xff, 0x3d, 0x4f, 0xde, 0xd7, 0xf0, 0x59, 0xd1, 0x91, 0xa2, 0x6b, 0x44, 0xdd, 0x89, - 0x9a, 0x70, 0xa9, 0x1a, 0xb1, 0xd0, 0x0d, 0xe1, 0xba, 0x4a, 0x88, 0x28, 0x75, 0x7d, 0x85, 0xab, - 0x5a, 0x69, 0xe5, 0x21, 0xae, 0x78, 0x51, 0x2b, 0xc6, 0x33, 0x5c, 0x74, 0xf8, 0xbe, 0x01, 0x73, - 0xdd, 0x56, 0x0b, 0xa6, 0xc5, 0xec, 0x30, 0x55, 0xa9, 0x32, 0xa5, 0x64, 0xa0, 0xb1, 0x6b, 0xf6, - 0x24, 0x55, 0x2a, 0x95, 0x82, 0xb0, 0x2a, 0x27, 0xac, 0x2c, 0x95, 0x66, 0x3a, 0x57, 0x65, 0xf3, - 0x2f, 0xeb, 0xb7, 0x3a, 0x97, 0x24, 0x93, 0x9c, 0xe8, 0x7c, 0x29, 0x1a, 0xcd, 0x96, 0xd5, 0x98, - 0x39, 0xfa, 0x35, 0x81, 0xee, 0xf9, 0xa0, 0xee, 0x05, 0xd0, 0x15, 0x95, 0xe2, 0x99, 0x0f, 0x42, - 0x10, 0xd9, 0xf1, 0xf4, 0x6e, 0x1d, 0xb8, 0xe7, 0x43, 0x80, 0x8e, 0x71, 0xef, 0x2d, 0x3c, 0x18, - 0x5d, 0x7f, 0xdc, 0x0e, 0xf1, 0x27, 0x21, 0x88, 0xf6, 0x8f, 0x9f, 0xe2, 0x9d, 0xe7, 0x41, 0x09, - 0x67, 0x92, 0xe3, 0x77, 0xf7, 0x45, 0xb1, 0xb3, 0x5a, 0x07, 0x16, 0x7d, 0x3c, 0x36, 0x6f, 0xc3, - 0xde, 0x37, 0x00, 0x9d, 0xa5, 0x64, 0xb9, 0x6f, 0x87, 0x76, 0xb4, 0x7f, 0x4c, 0xf0, 0xff, 0x7f, - 0x1c, 0x1b, 0x9b, 0xf8, 0x52, 0xb2, 0xdc, 0x50, 0xfc, 0xa6, 0x5f, 0x07, 0xce, 0xe5, 0xc5, 0xd9, - 0xfc, 0xcb, 0x6d, 0xf0, 0x32, 0xcd, 0x75, 0xd6, 0x26, 0x98, 0xab, 0x25, 0xd9, 0x8e, 0x59, 0x24, - 0x3b, 0x26, 0x55, 0x91, 0x12, 0x43, 0x55, 0x82, 0x29, 0x2b, 0x53, 0x31, 0x7f, 0xfd, 0xf5, 0x36, - 0xb0, 0x2f, 0xce, 0xe6, 0xd4, 0xb8, 0xf0, 0x3c, 0xe8, 0x7c, 0x6a, 0xa5, 0xf4, 0x9d, 0x10, 0x44, - 0x7b, 0xd4, 0xf0, 0xec, 0x15, 0x9c, 0x6e, 0xf5, 0xbc, 0x03, 0x68, 0x17, 0xe2, 0xca, 0xac, 0xc7, - 0xa5, 0x03, 0x7a, 0x87, 0xd0, 0xed, 0x98, 0x6c, 0x85, 0x59, 0x83, 0x4d, 0xc7, 0x8f, 0xd3, 0xc9, - 0x09, 0x38, 0x75, 0x7e, 0xfc, 0x0c, 0xac, 0xa3, 0x0c, 0xee, 0x51, 0xc1, 0xf8, 0x70, 0x09, 0xef, - 0x3d, 0x9c, 0x52, 0xf1, 0xb9, 0x15, 0x8d, 0x16, 0x0b, 0x1f, 0x84, 0x76, 0xe4, 0xc6, 0x27, 0x77, - 0xeb, 0x87, 0x19, 0xa7, 0xbb, 0x51, 0xa3, 0x52, 0xfc, 0x7c, 0xf5, 0x07, 0x59, 0xab, 0x1e, 0x81, - 0xeb, 0x1e, 0x81, 0x9b, 0x1e, 0x81, 0xdf, 0x3d, 0x02, 0xdf, 0x37, 0xc8, 0xba, 0xde, 0x20, 0xeb, - 0x66, 0x83, 0xac, 0x0f, 0xce, 0xf0, 0xd0, 0x92, 0x47, 0xe6, 0xea, 0x2f, 0xfe, 0x06, 0x00, 0x00, - 0xff, 0xff, 0x39, 0x68, 0xb2, 0x7f, 0x8c, 0x02, 0x00, 0x00, -} - -func (m *Entry) Marshal() (dAtA []byte, err error) { - size := m.Size() - dAtA = make([]byte, size) - n, err := m.MarshalToSizedBuffer(dAtA[:size]) - if err != nil { - return nil, err - } - return dAtA[:n], nil -} - -func (m *Entry) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *Entry) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if m.Full { - i-- - if m.Full { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i-- - dAtA[i] = 0x20 - } - if len(m.MLAI) > 0 { - keysForMLAI := make([]int32, 0, len(m.MLAI)) - for k := range m.MLAI { - keysForMLAI = append(keysForMLAI, int32(k)) - } - github_com_gogo_protobuf_sortkeys.Int32s(keysForMLAI) - for iNdEx := len(keysForMLAI) - 1; iNdEx >= 0; iNdEx-- { - v := m.MLAI[github_com_cockroachdb_cockroach_pkg_roachpb.RangeID(keysForMLAI[iNdEx])] - baseI := i - i = encodeVarintEntry(dAtA, i, uint64(v)) - i-- - dAtA[i] = 0x10 - i = encodeVarintEntry(dAtA, i, uint64(keysForMLAI[iNdEx])) - i-- - dAtA[i] = 0x8 - i = encodeVarintEntry(dAtA, i, uint64(baseI-i)) - i-- - dAtA[i] = 0x1a - } - } - { - size, err := m.ClosedTimestamp.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintEntry(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x12 - if m.Epoch != 0 { - i = encodeVarintEntry(dAtA, i, uint64(m.Epoch)) - i-- - dAtA[i] = 0x8 - } - return len(dAtA) - i, nil -} - -func (m *Reaction) Marshal() (dAtA []byte, err error) { - size := m.Size() - dAtA = make([]byte, size) - n, err := m.MarshalToSizedBuffer(dAtA[:size]) - if err != nil { - return nil, err - } - return dAtA[:n], nil -} - -func (m *Reaction) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *Reaction) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if len(m.Requested) > 0 { - dAtA3 := make([]byte, len(m.Requested)*10) - var j2 int - for _, num1 := range m.Requested { - num := uint64(num1) - for num >= 1<<7 { - dAtA3[j2] = uint8(uint64(num)&0x7f | 0x80) - num >>= 7 - j2++ - } - dAtA3[j2] = uint8(num) - j2++ - } - i -= j2 - copy(dAtA[i:], dAtA3[:j2]) - i = encodeVarintEntry(dAtA, i, uint64(j2)) - i-- - dAtA[i] = 0xa - } - return len(dAtA) - i, nil -} - -func encodeVarintEntry(dAtA []byte, offset int, v uint64) int { - offset -= sovEntry(v) - base := offset - for v >= 1<<7 { - dAtA[offset] = uint8(v&0x7f | 0x80) - v >>= 7 - offset++ - } - dAtA[offset] = uint8(v) - return base -} -func (m *Entry) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if m.Epoch != 0 { - n += 1 + sovEntry(uint64(m.Epoch)) - } - l = m.ClosedTimestamp.Size() - n += 1 + l + sovEntry(uint64(l)) - if len(m.MLAI) > 0 { - for k, v := range m.MLAI { - _ = k - _ = v - mapEntrySize := 1 + sovEntry(uint64(k)) + 1 + sovEntry(uint64(v)) - n += mapEntrySize + 1 + sovEntry(uint64(mapEntrySize)) - } - } - if m.Full { - n += 2 - } - return n -} - -func (m *Reaction) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if len(m.Requested) > 0 { - l = 0 - for _, e := range m.Requested { - l += sovEntry(uint64(e)) - } - n += 1 + sovEntry(uint64(l)) + l - } - return n -} - -func sovEntry(x uint64) (n int) { - return (math_bits.Len64(x|1) + 6) / 7 -} -func sozEntry(x uint64) (n int) { - return sovEntry(uint64((x << 1) ^ uint64((int64(x) >> 63)))) -} -func (m *Entry) Unmarshal(dAtA []byte) error { - l := len(dAtA) - iNdEx := 0 - for iNdEx < l { - preIndex := iNdEx - var wire uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowEntry - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: Entry: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: Entry: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) - } - m.Epoch = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowEntry - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.Epoch |= Epoch(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ClosedTimestamp", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowEntry - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthEntry - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthEntry - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if err := m.ClosedTimestamp.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field MLAI", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowEntry - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthEntry - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthEntry - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if m.MLAI == nil { - m.MLAI = make(map[github_com_cockroachdb_cockroach_pkg_roachpb.RangeID]LAI) - } - var mapkey int32 - var mapvalue int64 - for iNdEx < postIndex { - entryPreIndex := iNdEx - var wire uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowEntry - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - fieldNum := int32(wire >> 3) - if fieldNum == 1 { - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowEntry - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - mapkey |= int32(b&0x7F) << shift - if b < 0x80 { - break - } - } - } else if fieldNum == 2 { - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowEntry - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - mapvalue |= int64(b&0x7F) << shift - if b < 0x80 { - break - } - } - } else { - iNdEx = entryPreIndex - skippy, err := skipEntry(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthEntry - } - if (iNdEx + skippy) > postIndex { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - m.MLAI[github_com_cockroachdb_cockroach_pkg_roachpb.RangeID(mapkey)] = ((LAI)(mapvalue)) - iNdEx = postIndex - case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Full", wireType) - } - var v int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowEntry - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.Full = bool(v != 0) - default: - iNdEx = preIndex - skippy, err := skipEntry(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthEntry - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *Reaction) Unmarshal(dAtA []byte) error { - l := len(dAtA) - iNdEx := 0 - for iNdEx < l { - preIndex := iNdEx - var wire uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowEntry - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: Reaction: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: Reaction: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType == 0 { - var v github_com_cockroachdb_cockroach_pkg_roachpb.RangeID - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowEntry - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_roachpb.RangeID(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.Requested = append(m.Requested, v) - } else if wireType == 2 { - var packedLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowEntry - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - packedLen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if packedLen < 0 { - return ErrInvalidLengthEntry - } - postIndex := iNdEx + packedLen - if postIndex < 0 { - return ErrInvalidLengthEntry - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - var elementCount int - var count int - for _, integer := range dAtA[iNdEx:postIndex] { - if integer < 128 { - count++ - } - } - elementCount = count - if elementCount != 0 && len(m.Requested) == 0 { - m.Requested = make([]github_com_cockroachdb_cockroach_pkg_roachpb.RangeID, 0, elementCount) - } - for iNdEx < postIndex { - var v github_com_cockroachdb_cockroach_pkg_roachpb.RangeID - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowEntry - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_roachpb.RangeID(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.Requested = append(m.Requested, v) - } - } else { - return fmt.Errorf("proto: wrong wireType = %d for field Requested", wireType) - } - default: - iNdEx = preIndex - skippy, err := skipEntry(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthEntry - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func skipEntry(dAtA []byte) (n int, err error) { - l := len(dAtA) - iNdEx := 0 - depth := 0 - for iNdEx < l { - var wire uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return 0, ErrIntOverflowEntry - } - if iNdEx >= l { - return 0, io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - wire |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - wireType := int(wire & 0x7) - switch wireType { - case 0: - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return 0, ErrIntOverflowEntry - } - if iNdEx >= l { - return 0, io.ErrUnexpectedEOF - } - iNdEx++ - if dAtA[iNdEx-1] < 0x80 { - break - } - } - case 1: - iNdEx += 8 - case 2: - var length int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return 0, ErrIntOverflowEntry - } - if iNdEx >= l { - return 0, io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - length |= (int(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - if length < 0 { - return 0, ErrInvalidLengthEntry - } - iNdEx += length - case 3: - depth++ - case 4: - if depth == 0 { - return 0, ErrUnexpectedEndOfGroupEntry - } - depth-- - case 5: - iNdEx += 4 - default: - return 0, fmt.Errorf("proto: illegal wireType %d", wireType) - } - if iNdEx < 0 { - return 0, ErrInvalidLengthEntry - } - if depth == 0 { - return iNdEx, nil - } - } - return 0, io.ErrUnexpectedEOF -} - -var ( - ErrInvalidLengthEntry = fmt.Errorf("proto: negative length found during unmarshaling") - ErrIntOverflowEntry = fmt.Errorf("proto: integer overflow") - ErrUnexpectedEndOfGroupEntry = fmt.Errorf("proto: unexpected end of group") -) diff --git a/pkg/kv/kvserver/closedts/ctpb/entry.proto b/pkg/kv/kvserver/closedts/ctpb/entry.proto deleted file mode 100644 index 47335f424626..000000000000 --- a/pkg/kv/kvserver/closedts/ctpb/entry.proto +++ /dev/null @@ -1,61 +0,0 @@ -// Copyright 2018 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. - -syntax = "proto3"; -package cockroach.kv.kvserver.ctupdate; -option go_package = "ctpb"; - -import "gogoproto/gogo.proto"; -import "google/api/annotations.proto"; - -import "util/hlc/timestamp.proto"; - -// An Entry is a closed timestamp update. It consists of a closed timestamp -// (i.e. a timestamp at or below which the origin node guarantees no more new -// writes are going to be permitted), an associated epoch in which the origin -// node promises it was live (for the closed timestamp), a map of minimum lease -// applied indexes (which have to be caught up to before being allowed to use -// the closed timestamp) as well as an indicator of whether this update supplies -// a full initial state or an increment to be merged into a previous state. In -// practice, the first Entry received for each epoch is full, while the remainder -// are incremental. An incremental update represents the implicit promise that -// the state accumulated since the last full Entry is the true full state. -message Entry { - option (gogoproto.goproto_stringer) = false; - - int64 epoch = 1 [(gogoproto.casttype) = "Epoch"]; - util.hlc.Timestamp closed_timestamp = 2 [(gogoproto.nullable) = false]; - map mlai = 3 [(gogoproto.castkey) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID", - (gogoproto.castvalue) = "LAI", - (gogoproto.customname) = "MLAI"]; - // Full is true if the emitter promises that any future write to any range - // mentioned in this Entry will be reflected in a subsequent Entry before any - // stale follower reads are possible. For example, if range 1 is assigned an - // MLAI of 12 in this Entry and isn't mentioned in the five subsequent - // entries, the recipient may behave as if the MLAI of 12 were repeated across - // all of these entries. - // - // In practice, a Full message is received when a stream of Entries is first - // established (or the Epoch changes), and all other updates are incremental - // (i.e. not Full). - bool full = 4; -} - -// Reactions flow in the direction opposite to Entries and request for ranges to -// be included in the next Entry. Under rare circumstances, ranges may be omitted -// from closed timestamp updates, and so serving follower reads from them would -// fail. The Reaction mechanism serves to explicitly request the missing information -// when that happens. -message Reaction { - option (gogoproto.goproto_stringer) = false; - - repeated int32 Requested = 1 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID"]; -} - diff --git a/pkg/kv/kvserver/closedts/ctpb/server.go b/pkg/kv/kvserver/closedts/ctpb/server.go deleted file mode 100644 index 4b39d87e979a..000000000000 --- a/pkg/kv/kvserver/closedts/ctpb/server.go +++ /dev/null @@ -1,40 +0,0 @@ -// Copyright 2018 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 ctpb - -import "context" - -// InboundClient is an interface that narrows ClosedTimestamp_GetServer down to what's -// actually required. -type InboundClient interface { - Send(*Entry) error - Recv() (*Reaction, error) - Context() context.Context -} - -// Server is the interface implemented by types that want to serve incoming -// closed timestamp update streams. -type Server interface { - Get(InboundClient) error -} - -// ServerShim is a wrapper around Server that provides the wider interface that -// gRPC expects. -type ServerShim struct{ Server } - -var _ ClosedTimestampServer = (*ServerShim)(nil) - -// Get implements ClosedTimestampServer by passing through to the wrapped Server. -func (s ServerShim) Get(client ClosedTimestamp_GetServer) error { - return s.Server.Get(client) -} - -var _ InboundClient = ClosedTimestamp_GetServer(nil) diff --git a/pkg/kv/kvserver/closedts/ctpb/service.go b/pkg/kv/kvserver/closedts/ctpb/service.go index 81435113cd22..5591a6a71875 100644 --- a/pkg/kv/kvserver/closedts/ctpb/service.go +++ b/pkg/kv/kvserver/closedts/ctpb/service.go @@ -23,6 +23,13 @@ import ( // SeqNum identifies a ctpb.Update. type SeqNum int64 +// LAI is an int64 denoting a lease applied index with its own type to avoid +// mix-ups in positional arguments. +type LAI int64 + +// SafeValue implements the redact.SafeValue interface. +func (LAI) SafeValue() {} + func (m *Update) String() string { sb := &strings.Builder{} fmt.Fprintf(sb, "Seq num: %d, sending node: n%d, snapshot: %t, size: %d bytes", diff --git a/pkg/kv/kvserver/closedts/ctpb/service.pb.go b/pkg/kv/kvserver/closedts/ctpb/service.pb.go index 8cabdbf05d0a..ce2ebea7fc6e 100644 --- a/pkg/kv/kvserver/closedts/ctpb/service.pb.go +++ b/pkg/kv/kvserver/closedts/ctpb/service.pb.go @@ -230,47 +230,45 @@ func init() { } var fileDescriptor_b0044b737e1a7700 = []byte{ - // 638 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x54, 0xc1, 0x4f, 0xdb, 0x3e, - 0x14, 0x4e, 0x7e, 0x29, 0x69, 0xe5, 0xea, 0x07, 0xcc, 0xda, 0x21, 0x8a, 0xb6, 0xa4, 0x62, 0x82, - 0xe5, 0xb2, 0x78, 0x2a, 0x3b, 0x4c, 0xbb, 0x51, 0x86, 0x10, 0xd2, 0xc4, 0x50, 0x60, 0x17, 0x34, - 0xad, 0x72, 0x63, 0x2b, 0x8d, 0x9a, 0xc6, 0x21, 0x76, 0x22, 0xf1, 0x5f, 0xec, 0xb8, 0xe3, 0x0e, - 0xfb, 0x63, 0xd0, 0x4e, 0x9c, 0x26, 0x4e, 0xd1, 0x16, 0xfe, 0x8b, 0x9e, 0xa6, 0xc4, 0xa1, 0x2b, - 0x48, 0x1b, 0x20, 0x4e, 0x79, 0x7e, 0xce, 0xfb, 0xfc, 0x7d, 0x9f, 0x9f, 0x1f, 0x78, 0x3e, 0xc9, - 0xd1, 0x24, 0xe7, 0x34, 0xcd, 0x69, 0x8a, 0xfc, 0x88, 0x71, 0x4a, 0x04, 0x47, 0xbe, 0x48, 0x46, - 0xa8, 0x4a, 0x86, 0x3e, 0x75, 0x93, 0x94, 0x09, 0x06, 0x2d, 0x9f, 0xf9, 0x93, 0x94, 0x61, 0x7f, - 0xec, 0x4e, 0x72, 0xf7, 0xaa, 0xc4, 0xf5, 0x45, 0x96, 0x10, 0x2c, 0xa8, 0xb9, 0xfe, 0x77, 0x20, - 0x1a, 0x8b, 0xf4, 0x54, 0xc2, 0x98, 0xb0, 0x86, 0x48, 0x46, 0x88, 0x60, 0x81, 0x9b, 0x9c, 0x91, - 0x89, 0x30, 0x42, 0xe3, 0xc8, 0x47, 0x22, 0x9c, 0x52, 0x2e, 0xf0, 0x34, 0x69, 0x76, 0x1e, 0x07, - 0x2c, 0x60, 0x75, 0x88, 0xaa, 0xa8, 0xc9, 0x3e, 0x09, 0x18, 0x0b, 0x22, 0x8a, 0x70, 0x12, 0x22, - 0x1c, 0xc7, 0x4c, 0x60, 0x11, 0xb2, 0x98, 0xcb, 0xdd, 0xb5, 0xef, 0x3a, 0xd0, 0x3f, 0xd4, 0x9c, - 0xe0, 0x31, 0x68, 0xc7, 0x8c, 0xd0, 0x61, 0x48, 0x0c, 0xb5, 0xa7, 0x3a, 0x4b, 0x83, 0xad, 0xb2, - 0xb0, 0xf5, 0x7d, 0x46, 0xe8, 0xde, 0xdb, 0x59, 0x61, 0x6f, 0x06, 0xa1, 0x18, 0x67, 0x23, 0xd7, - 0x67, 0x53, 0x34, 0x57, 0x47, 0x46, 0x7f, 0x62, 0x94, 0x4c, 0x02, 0xd4, 0x10, 0x76, 0x65, 0x99, - 0xa7, 0x57, 0x88, 0x7b, 0x04, 0x3e, 0x03, 0x6d, 0x4e, 0x4f, 0x86, 0x71, 0x36, 0x35, 0xfe, 0xeb, - 0xa9, 0x8e, 0x36, 0x00, 0xb3, 0xc2, 0xd6, 0x0f, 0xe9, 0xc9, 0x7e, 0x36, 0xf5, 0x74, 0x5e, 0x7f, - 0xa1, 0x09, 0x3a, 0x3c, 0xc6, 0x09, 0x1f, 0x33, 0x61, 0x68, 0x3d, 0xd5, 0xe9, 0x78, 0xf3, 0x35, - 0xa4, 0xe0, 0x91, 0xb4, 0x69, 0x38, 0x57, 0xcd, 0x8d, 0x56, 0x4f, 0x73, 0xba, 0xfd, 0xbe, 0xfb, - 0x6f, 0xb3, 0x5d, 0xa9, 0xcf, 0xdd, 0x4d, 0x59, 0x96, 0xc8, 0x78, 0xd0, 0x3a, 0x2b, 0x6c, 0xc5, - 0x5b, 0x95, 0x90, 0x47, 0x73, 0x44, 0xe8, 0x81, 0x76, 0x4a, 0xa7, 0x2c, 0xa7, 0xc4, 0x58, 0xea, - 0x69, 0xce, 0xd2, 0xe0, 0xf5, 0xac, 0xb0, 0x5f, 0xdd, 0x4b, 0xb9, 0x87, 0xe3, 0xa0, 0x92, 0x7e, - 0x05, 0x04, 0x47, 0x60, 0x15, 0x13, 0x42, 0xc9, 0x90, 0xa5, 0x43, 0xc9, 0x88, 0x18, 0xfa, 0xbd, - 0x98, 0xd7, 0x90, 0xd7, 0x98, 0x2f, 0xd7, 0x88, 0xef, 0x53, 0x99, 0x24, 0xe6, 0x37, 0x15, 0x74, - 0x17, 0xf4, 0xc1, 0x1d, 0xa0, 0x27, 0x2c, 0x0a, 0xfd, 0xd3, 0xfa, 0x2a, 0x97, 0xfb, 0x2f, 0x16, - 0x4e, 0xba, 0x46, 0x74, 0xfb, 0xba, 0x03, 0x07, 0x75, 0x91, 0xd7, 0x14, 0xc3, 0x7d, 0xb0, 0x7a, - 0xd3, 0xf5, 0xfa, 0xfe, 0xba, 0xfd, 0xa7, 0x0b, 0x80, 0x55, 0x43, 0xba, 0xe3, 0xc8, 0x77, 0xe7, - 0x30, 0x0d, 0xcb, 0x95, 0x1b, 0xfe, 0x9a, 0x3f, 0x54, 0xd0, 0x5d, 0x10, 0x03, 0x3f, 0x81, 0x4e, - 0x5a, 0x2d, 0xaf, 0x7a, 0xae, 0x35, 0xd8, 0x2e, 0x0b, 0xbb, 0xdd, 0x58, 0xf8, 0x00, 0xeb, 0xeb, - 0x80, 0xc0, 0x1e, 0xd0, 0x22, 0x1c, 0x36, 0x2d, 0xb7, 0x5c, 0x16, 0xb6, 0xf6, 0x6e, 0x6b, 0x6f, - 0x26, 0x3f, 0x5e, 0xb5, 0xb5, 0x60, 0x94, 0xf6, 0x00, 0xa3, 0xde, 0xb4, 0xbe, 0x7c, 0xb5, 0x95, - 0x35, 0x00, 0x3a, 0x1e, 0xe5, 0x09, 0x8b, 0x39, 0xed, 0x33, 0xb0, 0x72, 0xa3, 0x04, 0x7e, 0x04, - 0xda, 0x2e, 0x15, 0xd0, 0xb9, 0xed, 0xd6, 0x3d, 0x8a, 0xfd, 0xea, 0x8d, 0x9a, 0xeb, 0xb7, 0xfd, - 0xb9, 0x53, 0xcd, 0x8a, 0x35, 0xc5, 0x51, 0x5f, 0xaa, 0xfd, 0x1c, 0xfc, 0x7f, 0x18, 0x12, 0x7a, - 0x94, 0xe2, 0x98, 0x27, 0x2c, 0xad, 0x9e, 0x4c, 0xf7, 0x20, 0xe3, 0x63, 0x69, 0x35, 0x87, 0x1b, - 0x77, 0x6b, 0x36, 0xf3, 0x0e, 0xf4, 0xa4, 0x44, 0x79, 0xee, 0x60, 0xe3, 0xec, 0x97, 0xa5, 0x9c, - 0x95, 0x96, 0x7a, 0x5e, 0x5a, 0xea, 0x45, 0x69, 0xa9, 0x3f, 0x4b, 0x4b, 0xfd, 0x7c, 0x69, 0x29, - 0xe7, 0x97, 0x96, 0x72, 0x71, 0x69, 0x29, 0xc7, 0xad, 0x6a, 0xae, 0x8d, 0xf4, 0x7a, 0xe0, 0x6c, - 0xfe, 0x0e, 0x00, 0x00, 0xff, 0xff, 0x63, 0x05, 0x7d, 0x2c, 0x44, 0x05, 0x00, 0x00, + // 599 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x54, 0x4f, 0x6f, 0xd3, 0x4e, + 0x10, 0xf5, 0xfe, 0x9c, 0x3a, 0xd1, 0x46, 0xbf, 0x52, 0x56, 0x1c, 0xac, 0x08, 0xec, 0xa8, 0x48, + 0xc5, 0x17, 0xbc, 0x28, 0xe5, 0x80, 0xb8, 0x35, 0x05, 0xa1, 0x4a, 0xa8, 0x54, 0x6e, 0xb9, 0xf4, + 0x40, 0xb4, 0xf1, 0xae, 0x1c, 0x2b, 0xb6, 0x77, 0xeb, 0x5d, 0x5b, 0xe2, 0x5b, 0x70, 0xe4, 0xc8, + 0x81, 0x0f, 0x53, 0x71, 0xea, 0x09, 0xf5, 0x64, 0x81, 0xfb, 0x2d, 0x72, 0x42, 0xfe, 0x93, 0x90, + 0xf6, 0x00, 0x54, 0x3d, 0xed, 0xec, 0x4e, 0xe6, 0xcd, 0x7b, 0x6f, 0x26, 0x86, 0x4f, 0xe6, 0x39, + 0x9e, 0xe7, 0x92, 0xa5, 0x39, 0x4b, 0xb1, 0x1f, 0x71, 0xc9, 0xa8, 0x92, 0xd8, 0x57, 0x62, 0x8a, + 0xab, 0xc7, 0xd0, 0x67, 0xae, 0x48, 0xb9, 0xe2, 0xc8, 0xf2, 0xb9, 0x3f, 0x4f, 0x39, 0xf1, 0x67, + 0xee, 0x3c, 0x77, 0x97, 0x25, 0xae, 0xaf, 0x32, 0x41, 0x89, 0x62, 0x03, 0x54, 0xe7, 0xc4, 0x14, + 0x53, 0xa2, 0x48, 0x53, 0x33, 0x30, 0x33, 0x15, 0x46, 0x78, 0x16, 0xf9, 0x58, 0x85, 0x31, 0x93, + 0x8a, 0xc4, 0xa2, 0xcd, 0x3c, 0x08, 0x78, 0xc0, 0xeb, 0x10, 0x57, 0x51, 0xfb, 0xfa, 0x30, 0xe0, + 0x3c, 0x88, 0x18, 0x26, 0x22, 0xc4, 0x24, 0x49, 0xb8, 0x22, 0x2a, 0xe4, 0x89, 0x6c, 0xb2, 0xdb, + 0xdf, 0x0c, 0x68, 0xbc, 0xaf, 0x9b, 0xa1, 0x53, 0xd8, 0x4d, 0x38, 0x65, 0x93, 0x90, 0x9a, 0x60, + 0x08, 0x9c, 0x8d, 0xf1, 0x5e, 0x59, 0xd8, 0xc6, 0x21, 0xa7, 0xec, 0xe0, 0xd5, 0xa2, 0xb0, 0x77, + 0x83, 0x50, 0xcd, 0xb2, 0xa9, 0xeb, 0xf3, 0x18, 0xaf, 0x68, 0xd3, 0xe9, 0xef, 0x18, 0x8b, 0x79, + 0x80, 0x5b, 0xc2, 0x6e, 0x53, 0xe6, 0x19, 0x15, 0xe2, 0x01, 0x45, 0x8f, 0x61, 0x57, 0xb2, 0xb3, + 0x49, 0x92, 0xc5, 0xe6, 0x7f, 0x43, 0xe0, 0xe8, 0x63, 0xb8, 0x28, 0x6c, 0xe3, 0x98, 0x9d, 0x1d, + 0x66, 0xb1, 0x67, 0xc8, 0xfa, 0x44, 0x03, 0xd8, 0x93, 0x09, 0x11, 0x72, 0xc6, 0x95, 0xa9, 0x0f, + 0x81, 0xd3, 0xf3, 0x56, 0x77, 0xc4, 0xe0, 0xfd, 0xc6, 0xc8, 0xc9, 0x4a, 0xb5, 0x34, 0x3b, 0x43, + 0xdd, 0xe9, 0x8f, 0x46, 0xee, 0x9f, 0x5d, 0x74, 0x1b, 0x7d, 0xee, 0x9b, 0x94, 0x67, 0xa2, 0x89, + 0xc7, 0x9d, 0xf3, 0xc2, 0xd6, 0xbc, 0xad, 0x06, 0xf2, 0x64, 0x85, 0x88, 0x3c, 0xd8, 0x4d, 0x59, + 0xcc, 0x73, 0x46, 0xcd, 0x8d, 0xa1, 0xee, 0x6c, 0x8c, 0x5f, 0x2c, 0x0a, 0xfb, 0xf9, 0xad, 0x94, + 0x7b, 0x24, 0x09, 0x2a, 0xe9, 0x4b, 0x20, 0x34, 0x85, 0x5b, 0x84, 0x52, 0x46, 0x27, 0x3c, 0x9d, + 0x34, 0x8c, 0xa8, 0x69, 0xdc, 0x8a, 0x79, 0x0d, 0x79, 0x8d, 0xf9, 0x66, 0x8d, 0xf8, 0x2e, 0x6d, + 0x1e, 0xe9, 0xe0, 0x2b, 0x80, 0xfd, 0x35, 0x7d, 0xe8, 0x35, 0x34, 0x04, 0x8f, 0x42, 0xff, 0x63, + 0x3d, 0xca, 0xcd, 0xd1, 0xd3, 0xb5, 0x4e, 0xd7, 0x88, 0xee, 0x5f, 0x77, 0xe0, 0xa8, 0x2e, 0xf2, + 0xda, 0x62, 0x74, 0x08, 0xb7, 0x6e, 0xba, 0x5e, 0xcf, 0xaf, 0x3f, 0x7a, 0xb4, 0x06, 0x58, 0x2d, + 0xa4, 0x3b, 0x8b, 0x7c, 0x77, 0x05, 0xd3, 0xb2, 0xbc, 0x77, 0xc3, 0xdf, 0xc1, 0x77, 0x00, 0xfb, + 0x6b, 0x62, 0xd0, 0x07, 0xd8, 0x4b, 0xab, 0xeb, 0x72, 0xe7, 0x3a, 0xe3, 0xfd, 0xb2, 0xb0, 0xbb, + 0xad, 0x85, 0x77, 0xb0, 0xbe, 0x0e, 0x28, 0x1a, 0x42, 0x3d, 0x22, 0x61, 0xbb, 0x72, 0x9b, 0x65, + 0x61, 0xeb, 0x6f, 0xf7, 0x0e, 0x16, 0xcd, 0xe1, 0x55, 0xa9, 0x35, 0xa3, 0xf4, 0x3b, 0x18, 0xf5, + 0xb2, 0xf3, 0xf9, 0x8b, 0xad, 0x6d, 0x43, 0xd8, 0xf3, 0x98, 0x14, 0x3c, 0x91, 0x6c, 0x94, 0xc3, + 0xff, 0x8f, 0x43, 0xca, 0x4e, 0x52, 0x92, 0x48, 0xc1, 0xd3, 0x6a, 0x83, 0xfb, 0x47, 0x99, 0x9c, + 0x35, 0xca, 0x25, 0xda, 0xf9, 0xb7, 0xd9, 0x0f, 0x9c, 0xbf, 0xfd, 0x6e, 0xd9, 0x71, 0x5b, 0x73, + 0xc0, 0x33, 0x30, 0xde, 0x39, 0xff, 0x69, 0x69, 0xe7, 0xa5, 0x05, 0x2e, 0x4a, 0x0b, 0x5c, 0x96, + 0x16, 0xf8, 0x51, 0x5a, 0xe0, 0xd3, 0x95, 0xa5, 0x5d, 0x5c, 0x59, 0xda, 0xe5, 0x95, 0xa5, 0x9d, + 0x76, 0xaa, 0x0f, 0xd1, 0xd4, 0xa8, 0xff, 0xff, 0xbb, 0xbf, 0x02, 0x00, 0x00, 0xff, 0xff, 0xbe, + 0xe6, 0x5c, 0x73, 0xac, 0x04, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -281,110 +279,6 @@ var _ grpc.ClientConn // is compatible with the grpc package it is being compiled against. const _ = grpc.SupportPackageIsVersion4 -// ClosedTimestampClient is the client API for ClosedTimestamp service. -// -// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. -type ClosedTimestampClient interface { - Get(ctx context.Context, opts ...grpc.CallOption) (ClosedTimestamp_GetClient, error) -} - -type closedTimestampClient struct { - cc *grpc.ClientConn -} - -func NewClosedTimestampClient(cc *grpc.ClientConn) ClosedTimestampClient { - return &closedTimestampClient{cc} -} - -func (c *closedTimestampClient) Get(ctx context.Context, opts ...grpc.CallOption) (ClosedTimestamp_GetClient, error) { - stream, err := c.cc.NewStream(ctx, &_ClosedTimestamp_serviceDesc.Streams[0], "/cockroach.kv.kvserver.ctupdate.ClosedTimestamp/Get", opts...) - if err != nil { - return nil, err - } - x := &closedTimestampGetClient{stream} - return x, nil -} - -type ClosedTimestamp_GetClient interface { - Send(*Reaction) error - Recv() (*Entry, error) - grpc.ClientStream -} - -type closedTimestampGetClient struct { - grpc.ClientStream -} - -func (x *closedTimestampGetClient) Send(m *Reaction) error { - return x.ClientStream.SendMsg(m) -} - -func (x *closedTimestampGetClient) Recv() (*Entry, error) { - m := new(Entry) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - -// ClosedTimestampServer is the server API for ClosedTimestamp service. -type ClosedTimestampServer interface { - Get(ClosedTimestamp_GetServer) error -} - -// UnimplementedClosedTimestampServer can be embedded to have forward compatible implementations. -type UnimplementedClosedTimestampServer struct { -} - -func (*UnimplementedClosedTimestampServer) Get(srv ClosedTimestamp_GetServer) error { - return status.Errorf(codes.Unimplemented, "method Get not implemented") -} - -func RegisterClosedTimestampServer(s *grpc.Server, srv ClosedTimestampServer) { - s.RegisterService(&_ClosedTimestamp_serviceDesc, srv) -} - -func _ClosedTimestamp_Get_Handler(srv interface{}, stream grpc.ServerStream) error { - return srv.(ClosedTimestampServer).Get(&closedTimestampGetServer{stream}) -} - -type ClosedTimestamp_GetServer interface { - Send(*Entry) error - Recv() (*Reaction, error) - grpc.ServerStream -} - -type closedTimestampGetServer struct { - grpc.ServerStream -} - -func (x *closedTimestampGetServer) Send(m *Entry) error { - return x.ServerStream.SendMsg(m) -} - -func (x *closedTimestampGetServer) Recv() (*Reaction, error) { - m := new(Reaction) - if err := x.ServerStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - -var _ClosedTimestamp_serviceDesc = grpc.ServiceDesc{ - ServiceName: "cockroach.kv.kvserver.ctupdate.ClosedTimestamp", - HandlerType: (*ClosedTimestampServer)(nil), - Methods: []grpc.MethodDesc{}, - Streams: []grpc.StreamDesc{ - { - StreamName: "Get", - Handler: _ClosedTimestamp_Get_Handler, - ServerStreams: true, - ClientStreams: true, - }, - }, - Metadata: "kv/kvserver/closedts/ctpb/service.proto", -} - // SideTransportClient is the client API for SideTransport service. // // For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. diff --git a/pkg/kv/kvserver/closedts/ctpb/service.proto b/pkg/kv/kvserver/closedts/ctpb/service.proto index 7061834b9daf..7dcc66cfd5b1 100644 --- a/pkg/kv/kvserver/closedts/ctpb/service.proto +++ b/pkg/kv/kvserver/closedts/ctpb/service.proto @@ -12,17 +12,12 @@ syntax = "proto3"; package cockroach.kv.kvserver.ctupdate; option go_package = "ctpb"; -import "kv/kvserver/closedts/ctpb/entry.proto"; import "roachpb/data.proto"; import "util/hlc/timestamp.proto"; import "gogoproto/gogo.proto"; import "google/api/annotations.proto"; -service ClosedTimestamp { - rpc Get(stream Reaction) returns (stream Entry) { } -} - // Update contains information about (the advancement of) closed timestamps for // ranges with leases on the sender node. Updates are of two types: snapshots // and incrementals. Snapshots are stand-alone messages, explicitly containing diff --git a/pkg/kv/kvserver/closedts/minprop/BUILD.bazel b/pkg/kv/kvserver/closedts/minprop/BUILD.bazel index 926d88c55fb4..e69de29bb2d1 100644 --- a/pkg/kv/kvserver/closedts/minprop/BUILD.bazel +++ b/pkg/kv/kvserver/closedts/minprop/BUILD.bazel @@ -1,41 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") - -go_library( - name = "minprop", - srcs = [ - "doc.go", - "tracker.go", - ], - importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/minprop", - visibility = ["//visibility:public"], - deps = [ - "//pkg/kv/kvserver/closedts", - "//pkg/kv/kvserver/closedts/ctpb", - "//pkg/roachpb:with-mocks", - "//pkg/util/hlc", - "//pkg/util/log", - "//pkg/util/syncutil", - ], -) - -go_test( - name = "minprop_test", - size = "small", - srcs = [ - "doc_test.go", - "tracker_test.go", - ], - embed = [":minprop"], - deps = [ - "//pkg/cli/exit", - "//pkg/kv/kvserver/closedts/ctpb", - "//pkg/roachpb:with-mocks", - "//pkg/util/hlc", - "//pkg/util/log", - "//pkg/util/syncutil", - "@com_github_cockroachdb_errors//:errors", - "@com_github_kr_pretty//:pretty", - "@com_github_stretchr_testify//assert", - "@org_golang_x_sync//errgroup", - ], -) diff --git a/pkg/kv/kvserver/closedts/minprop/doc.go b/pkg/kv/kvserver/closedts/minprop/doc.go deleted file mode 100644 index 112e96a55750..000000000000 --- a/pkg/kv/kvserver/closedts/minprop/doc.go +++ /dev/null @@ -1,24 +0,0 @@ -// Copyright 2018 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 minprop exports a main data structure, Tracker, which checkpoints -// closed timestamps and associated Raft Lease Applied indexes positions for -// which (under additional conditions) it is legal to serve follower reads. It -// does so by maintaining a 'next' timestamp above which new command evaluations -// are forced, and by tracking when all in-flight evaluations below this -// timestamp have completed (at which point a call to the Close method succeeds: -// 'next' becomes closed, and a new 'next' is initialized with a future -// timestamp). -// -// In-flight command evaluations are tracked via the Track method which acquires -// a reference with the tracker, returns a minimum timestamp to be used for the -// proposal evaluation, and provides a closure that releases the reference with -// a lease applied index used for the proposal. -package minprop diff --git a/pkg/kv/kvserver/closedts/minprop/doc_test.go b/pkg/kv/kvserver/closedts/minprop/doc_test.go deleted file mode 100644 index d51d77eb563c..000000000000 --- a/pkg/kv/kvserver/closedts/minprop/doc_test.go +++ /dev/null @@ -1,225 +0,0 @@ -// Copyright 2018 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 minprop - -import ( - "context" - "fmt" - "sort" - "strings" - - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" -) - -func Example() { - ctx := context.Background() - - tracker := NewTracker() - const ep1 ctpb.Epoch = 1 - fmt.Println("The newly initialized tracker has a zero closed timestamp:") - fmt.Println(tracker) - - fmt.Println("A first command arrives on range 12 (though the range isn't known yet to the Tracker).") - ts, done1 := tracker.Track(ctx) - fmt.Println("All commands initially start out on the right. The command has its timestamp forwarded to", ts, ".") - fmt.Println(tracker) - - fmt.Println("Two more commands arrive, on r1 and r12.") - _, done2 := tracker.Track(ctx) - _, done3 := tracker.Track(ctx) - fmt.Println(tracker) - - fmt.Println("The command on r1 finishes evaluating at Lease Applied Index 10 and lets the Tracker know.") - done2(ctx, ep1, 1, 10) - fmt.Println(tracker) - - fmt.Println("The command on r12 also finishes quickly, at LAI 77.") - done3(ctx, ep1, 12, 77) - fmt.Println(tracker) - - fmt.Println("The system closes out a timestamp (registering 1000 as the next timestamp to close out).") - closed1, mlai1, _ := tracker.Close(hlc.Timestamp{WallTime: 1e9}, ep1) - fmt.Println("No problem: nothing is tracked on the left side; returns:", closed1, "and", mlaiString(mlai1)) - fmt.Println("Note how the items on the right have moved to the left, as they are relevant for the") - fmt.Println("next call to Close.") - fmt.Println(tracker) - - fmt.Println("Nothing happens for a while until the system tries to close out the next timestamp.") - fmt.Println("However, the very first proposal is still tracked and blocks progress.") - closed2, mlai2, _ := tracker.Close(hlc.Timestamp{WallTime: 2e9}, ep1) - fmt.Println("The call returns a no-op in the form", closed2, mlaiString(mlai2), ".") - fmt.Println(tracker) - - ts4, done4 := tracker.Track(ctx) - fmt.Println("A new command gets tracked on r12 (and is forwarded to", ts4, "(if necessary).") - fmt.Println("It terminates quickly, leaving an MLAI entry of 78 behind.") - done4(ctx, ep1, 12, 78) - fmt.Println(tracker) - - fmt.Println("Finally! The slow evaluation finishes and the command gets proposed at index 79.") - fmt.Println("Note that the right now tracks a smaller value of 78. Consumers have to keep the") - fmt.Println("maximum they've seen.") - done1(ctx, ep1, 12, 79) - fmt.Println(tracker) - - closed3, mlai3, _ := tracker.Close(hlc.Timestamp{WallTime: 3e9}, ep1) - fmt.Println("The next call to Close() is successful and returns:", closed3, "and", mlaiString(mlai3)) - fmt.Println(tracker) - - // Output: - // The newly initialized tracker has a zero closed timestamp: - // - // closed=0,0 - // | next=0,1 - // | left | right - // | 0 # 0 - // | 1 e 1 - // v v - // ---------------------------------------------------------> time - // - // A first command arrives on range 12 (though the range isn't known yet to the Tracker). - // All commands initially start out on the right. The command has its timestamp forwarded to 0,2 . - // - // closed=0,0 - // | next=0,1 - // | left | right - // | 0 # 1 - // | 1 e 1 - // v v - // ---------------------------------------------------------> time - // - // Two more commands arrive, on r1 and r12. - // - // closed=0,0 - // | next=0,1 - // | left | right - // | 0 # 3 - // | 1 e 1 - // v v - // ---------------------------------------------------------> time - // - // The command on r1 finishes evaluating at Lease Applied Index 10 and lets the Tracker know. - // - // closed=0,0 - // | next=0,1 - // | left | right - // | 0 # 2 - // | 1 e 1 - // | @ 10 (r1) - // v v - // ---------------------------------------------------------> time - // - // The command on r12 also finishes quickly, at LAI 77. - // - // closed=0,0 - // | next=0,1 - // | left | right - // | 0 # 1 - // | 1 e 1 - // | @ 10 (r1) - // | @ 77 (r12) - // v v - // ---------------------------------------------------------> time - // - // The system closes out a timestamp (registering 1000 as the next timestamp to close out). - // No problem: nothing is tracked on the left side; returns: 0,1 and map[] - // Note how the items on the right have moved to the left, as they are relevant for the - // next call to Close. - // - // closed=0,1 - // | next=1.000000000,0 - // | left | right - // | 1 # 0 - // | 1 e 1 - // | 10 @ (r1) - // | 77 @ (r12) - // v v - // ---------------------------------------------------------> time - // - // Nothing happens for a while until the system tries to close out the next timestamp. - // However, the very first proposal is still tracked and blocks progress. - // The call returns a no-op in the form 0,1 map[] . - // - // closed=0,1 - // | next=1.000000000,0 - // | left | right - // | 1 # 0 - // | 1 e 1 - // | 10 @ (r1) - // | 77 @ (r12) - // v v - // ---------------------------------------------------------> time - // - // A new command gets tracked on r12 (and is forwarded to 1.000000000,1 (if necessary). - // It terminates quickly, leaving an MLAI entry of 78 behind. - // - // closed=0,1 - // | next=1.000000000,0 - // | left | right - // | 1 # 0 - // | 1 e 1 - // | 10 @ (r1) - // | 77 @ (r12) - // | @ 78 (r12) - // v v - // ---------------------------------------------------------> time - // - // Finally! The slow evaluation finishes and the command gets proposed at index 79. - // Note that the right now tracks a smaller value of 78. Consumers have to keep the - // maximum they've seen. - // - // closed=0,1 - // | next=1.000000000,0 - // | left | right - // | 0 # 0 - // | 1 e 1 - // | 10 @ (r1) - // | @ 78 (r12) - // | 79 @ (r12) - // v v - // ---------------------------------------------------------> time - // - // The next call to Close() is successful and returns: 1.000000000,0 and map[1:10 12:79] - // - // closed=1.000000000,0 - // | next=3.000000000,0 - // | left | right - // | 0 # 0 - // | 1 e 1 - // | 78 @ (r12) - // v v - // ---------------------------------------------------------> time -} - -// mlaiString converts an mlai map into a string. Avoids randomized ordering of -// map elements in string output. -func mlaiString(mlai map[roachpb.RangeID]ctpb.LAI) string { - var rangeIDs []roachpb.RangeID - for rangeID := range mlai { - rangeIDs = append(rangeIDs, rangeID) - } - sort.Slice(rangeIDs, func(i, j int) bool { - return rangeIDs[i] < rangeIDs[j] - }) - - var sb strings.Builder - sb.WriteString("map[") - for i, rangeID := range rangeIDs { - if i > 0 { - sb.WriteString(" ") - } - fmt.Fprintf(&sb, "%d:%d", rangeID, mlai[rangeID]) - } - sb.WriteString("]") - return sb.String() -} diff --git a/pkg/kv/kvserver/closedts/minprop/tracker.go b/pkg/kv/kvserver/closedts/minprop/tracker.go deleted file mode 100644 index df5cd39ca368..000000000000 --- a/pkg/kv/kvserver/closedts/minprop/tracker.go +++ /dev/null @@ -1,395 +0,0 @@ -// Copyright 2018 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 minprop - -import ( - "context" - "fmt" - "sort" - - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/cockroach/pkg/util/syncutil" -) - -// Tracker implements TrackerI. -type Tracker struct { - mu struct { - syncutil.Mutex - // closed is the most recently closed timestamp. - closed hlc.Timestamp - closedEpoch ctpb.Epoch - - // The variables below track required information for the next closed - // timestamp and beyond. First, `next` is the timestamp that will be - // closed out next (i.e. will replace `closed`). - // - // "left" and "right" refers to how the timestamps at which the - // associated command evaluations take place relate to `next`. - // `left`-tracked proposals are taken into account for the next closed - // timestamp, i.e. they could mutate at timestamps <= `next`. `right` - // proposals affect only MVCC timestamps > `next` and thus will become - // relevant only after `next` has been closed out, at which point the - // "right" set will replace the "left". - // - // closed next - // | left | right - // | | - // | | - // v v - //---------------------------------------------------------> time - // - // A replica wishing to serve a follower read will first have to catch - // up to a lease applied index that is guaranteed to include all writes - // affecting the closed timestamp or below. When `next` is closed out, - // the set of relevant Lease Applied Indexes will be stored in `leftMLAI`. - // - // This is augmented by reference counts for the proposals currently in - // the process of evaluating. `next` can only be closed out once - // `leftRef` has been drained (i.e. has dropped to zero); new proposals - // are always forced above `next` and consequently count towards - // `rightRef`. - // - // Epochs track the highest liveness epoch observed for any released - // proposals. Tracking a max epoch allows the MPT to provide some MLAI - // information about the current epoch when calls to Close straddle multiple - // different epochs. Before epoch tracking was added the client of the MPT - // was forced to assume that the MLAI information from the current call to - // Close corresponded to the highest known epoch as of the previous call to - // Close. This is problematic in cases where an epoch change leads to a - // lease change for an otherwise quiescent range. If this mechanism were - // not in place then the client would never learn about an MLAI for the - // current epoch. Clients provide their view of the current epoch to calls - // to Close which use this information to determine whether the current - // state should be moved and whether the caller can make use of the - // currently tracked data. Each side tracks data which corresponds exactly - // to the side's epoch value. Releasing a proposal into the tracker at a - // later epoch than is currently tracked will result in the current data - // corresponding to the prior epoch to be evicted. - - next hlc.Timestamp - leftMLAI, rightMLAI map[roachpb.RangeID]ctpb.LAI - leftRef, rightRef int - leftEpoch, rightEpoch ctpb.Epoch - // failedCloseAttempts keeps track of the number of attempts by the tracker - // that failed to close a timestamp due to an epoch mismatch or pending - // evaluations. - failedCloseAttempts int64 - } -} - -var _ closedts.TrackerI = (*Tracker)(nil) - -// NewTracker returns a Tracker initialized to a closed timestamp of zero and -// a next closed timestamp of one logical tick past zero. -func NewTracker() *Tracker { - t := &Tracker{} - const initialEpoch = 1 - t.mu.closedEpoch = initialEpoch - t.mu.leftEpoch = initialEpoch - t.mu.rightEpoch = initialEpoch - t.mu.next = hlc.Timestamp{Logical: 1} - t.mu.leftMLAI = map[roachpb.RangeID]ctpb.LAI{} - t.mu.rightMLAI = map[roachpb.RangeID]ctpb.LAI{} - return t -} - -// String prints a string representation of the Tracker's state. -func (t *Tracker) String() string { - t.mu.Lock() - defer t.mu.Unlock() - closed, next := t.mu.closed, t.mu.next - leftRef, rightRef := t.mu.leftRef, t.mu.rightRef - leftEpoch, rightEpoch := t.mu.leftEpoch, t.mu.rightEpoch - - type item struct { - rangeID roachpb.RangeID - mlai ctpb.LAI - left bool - } - - var lais []item - for rangeID, mlai := range t.mu.leftMLAI { - lais = append(lais, item{rangeID, mlai, true}) - } - for rangeID, mlai := range t.mu.rightMLAI { - lais = append(lais, item{rangeID, mlai, false}) - } - - sort.Slice(lais, func(i, j int) bool { - if lais[i].rangeID != lais[j].rangeID { - return lais[i].rangeID < lais[j].rangeID - } - return lais[i].mlai < lais[j].mlai - }) - - var lines string - for _, item := range lais { - var format string - if !item.left { - format = ` | @ %-2d (r%d) -` - } else { - format = ` | %11d @ (r%d) -` - } - lines += fmt.Sprintf(format, item.mlai, item.rangeID) - } - - return fmt.Sprintf(` - closed=%s - | next=%s - | left | right - | %3d # %d - | %3d e %d -`+lines+ - ` v v ----------------------------------------------------------> time -`, - closed, next, leftRef, rightRef, leftEpoch, rightEpoch, - ) -} - -// Close attempts to close out the current candidate timestamp (replacing it -// with the provided one). This is possible only if tracked proposals that were -// evaluating when Close was previously called have since completed. On success, -// all subsequent proposals will be forced to evaluate strictly above the -// provided timestamp, and the timestamp previously passed to Close is returned -// as a closed timestamp along with a map of minimum Lease Applied Indexes -// reflecting the updates for the past period. On failure, the previous closed -// timestamp is returned along with a nil map (which can be treated by callers -// like a successful call that happens to not return any new information). -// Similarly, failure to provide a timestamp strictly larger than that to be -// closed out next results in the same "idempotent" return values. -// -// Callers additionally provide the current expected epoch value, the liveness -// epoch at which the caller intends to advertise this closed timestamp. The -// caller must know that it is live at a timestamp greater than or equal to the -// timestamp which the tracker will close. For correctness purposes this will -// be the case if the caller knows that it is live at next and calls to Close() -// pass monontic calues for next. If the current expected epoch is older than -// the currently tracked data then the timestamp will fail to be closed. If the -// expected epoch value is older than the epoch tracked on the left but -// corresponds to the epoch of the previous successful close then the previous -// closed timestamp is returned along with a nil map. This situation is just -// like the unsuccessful close scenario due to unreleased proposals. This -// behavior enables the caller to successfully obtain the tracked data at the -// newer epoch in a later query after its epoch has updated. If the caller's -// expected epoch is even older than the previously returned epoch then zero -// values are returned. If the caller's expected epoch is newer than that of -// tracked data the state of the tracker is progressed but zero values are -// returned. -func (t *Tracker) Close( - next hlc.Timestamp, expCurEpoch ctpb.Epoch, -) (ts hlc.Timestamp, mlai map[roachpb.RangeID]ctpb.LAI, ok bool) { - t.mu.Lock() - defer t.mu.Unlock() - defer func() { - if mlai == nil { - // Record if our attempt to close a timestamp fails. - t.mu.failedCloseAttempts++ - } - }() - - if log.V(3) { - log.Infof(context.TODO(), - "close: leftRef=%d (ep: %d) rightRef=%d (ep: %d) next=%s closed=%s@ (ep: %d) new=%s (ep: %d)", - t.mu.leftRef, t.mu.leftEpoch, t.mu.rightRef, t.mu.rightEpoch, t.mu.next, - t.mu.closed, t.mu.closedEpoch, next, expCurEpoch) - } - - // Make sure to not let `t.mu.next` regress, or we'll accept proposals - // that violate earlier closed timestamps. (And if it stayed the same - // the logic in the closure returned from Track would fall apart). - canClose := t.mu.leftRef == 0 && t.mu.next.Less(next) - - // NB: the expected closed epoch may not match the epoch for the timestamp we - // are currently closing. If the expected closed epoch is earlier than the - // epoch tracked on the left then the caller likely read its liveness just - // before an epoch change and we should not move the tracker state as the - // caller will likely visit again with the new epoch and would like the - // tracked information. If the expCurEpoch is greater than or equal to the - // current epoch, proceed with closing out the current timestamp, deferring - // the decision regarding whether to return the updated state based on epoch - // until after updating the data. - if canClose && t.mu.leftEpoch <= expCurEpoch { - // NB: if rightRef is also zero, then nothing is in flight right now and - // we could theoretically close out `next`. However, we'd also have to - // merge the left and right MLAI maps, and would force followers to - // catch up to more commands much more rapidly than can be expected of - // them. If we want to make use of this optimization, we should emit - // two closed timestamp updates for this case. - t.mu.closed = t.mu.next - t.mu.closedEpoch = t.mu.leftEpoch - mlai = t.mu.leftMLAI - - // NB: if the expCurEpoch is after the epoch tracked on the right, we'll - // never be able to use that information so clear it. The below logic is - // not required for correctness but adds an invariant that after a call to - // Close with a give expCurEpoch no state corresponding to an earlier epoch - // will be tracked on either side. Without this logic, subsequent proposals - // or Close calls at the later epoch would lead to this data being - // discarded at that point. - if t.mu.rightEpoch < expCurEpoch { - t.mu.rightEpoch = expCurEpoch - clearMLAIMap(t.mu.rightMLAI) - } - - // `next` moves forward to the provided timestamp, and picks up the - // right refcount and MLAIs (so that it is now responsible for tracking - // everything that's in-flight). - t.mu.leftMLAI = t.mu.rightMLAI - t.mu.leftRef = t.mu.rightRef - t.mu.leftEpoch = t.mu.rightEpoch - t.mu.rightMLAI = map[roachpb.RangeID]ctpb.LAI{} - t.mu.rightRef = 0 - - t.mu.next = next - } - - if t.mu.closedEpoch != expCurEpoch { - return hlc.Timestamp{}, nil, false - } - return t.mu.closed, mlai, true -} - -// Track is called before evaluating a proposal. It returns the minimum -// timestamp at which the proposal can be evaluated (i.e. the request timestamp -// needs to be forwarded if necessary), and acquires a reference with the -// Tracker. This reference is released by calling the returned closure either -// a) before proposing the command, supplying the Lease Applied Index at which -// the proposal will be carried out, or -// b) with zero arguments if the command won't end up being proposed (i.e. hit -// an error during evaluation). -// -// The ReleaseFunc is not thread safe. For convenience, it may be called with -// zero arguments once after a regular call. -func (t *Tracker) Track(ctx context.Context) (hlc.Timestamp, closedts.ReleaseFunc) { - shouldLog := log.V(3) - - t.mu.Lock() - minProp := t.mu.next.Next() - t.mu.rightRef++ - t.mu.Unlock() - - if shouldLog { - log.Infof(ctx, "track: proposal on the right at minProp %s", minProp) - } - - var calls int - release := func(ctx context.Context, epoch ctpb.Epoch, rangeID roachpb.RangeID, lai ctpb.LAI) { - calls++ - if calls != 1 { - if lai != 0 || rangeID != 0 || calls > 2 { - log.Fatalf(ctx, "command released %d times, this time with arguments (%d, %d)", - log.Safe(calls), log.Safe(rangeID), log.Safe(lai)) - } - return - } - t.release(ctx, minProp, epoch, rangeID, lai, shouldLog) - } - - return minProp, release -} - -// FailedCloseAttempts returns the numbers of attempts by the tracker that failed to -// close a timestamp due to an epoch mismatch or pending evaluations. -func (t *Tracker) FailedCloseAttempts() int64 { - t.mu.Lock() - defer t.mu.Unlock() - return t.mu.failedCloseAttempts -} - -// release is the business logic to release properly account for the release of -// a tracked proposal. It is called from the ReleaseFunc closure returned from -// Track. -func (t *Tracker) release( - ctx context.Context, - minProp hlc.Timestamp, - epoch ctpb.Epoch, - rangeID roachpb.RangeID, - lai ctpb.LAI, - shouldLog bool, -) { - t.mu.Lock() - defer t.mu.Unlock() - var left bool - if minProp == t.mu.closed.Next() { - left = true - } else if minProp == t.mu.next.Next() { - left = false - } else { - log.Fatalf(ctx, "min proposal %s not tracked under closed (%s) or next (%s) timestamp", minProp, t.mu.closed, t.mu.next) - } - // If the update is from the left side, clear all existing MLAIs from the left - // to uphold the invariant that all tracked MLAIs belong to the same (and - // largest seen) epoch. It would not violate correctness to clear the data on - // the left even if the proposal being released is tracked on the right; it is - // likely that the next call to close will observe the later epoch and thus - // not read this data but the code chooses to retain it. - if left && epoch > t.mu.leftEpoch { - t.mu.leftEpoch = epoch - clearMLAIMap(t.mu.leftMLAI) - } - // The right side is bumped and cleared when the epoch increases without - // taking into account which side the current proposal is tracked under - // because bumping the left side implies that the information from the right - // side will never be retrieved by the client (as epochs only ever go up and - // the current left will be emitted before the current right side). - if epoch > t.mu.rightEpoch { - t.mu.rightEpoch = epoch - clearMLAIMap(t.mu.rightMLAI) - } - if left { - releaseProposal(ctx, "left", shouldLog, minProp, rangeID, lai, - &t.mu.leftRef, t.mu.leftMLAI, t.mu.leftEpoch != epoch) - } else { - releaseProposal(ctx, "right", shouldLog, minProp, rangeID, lai, - &t.mu.rightRef, t.mu.rightMLAI, t.mu.rightEpoch != epoch) - } -} - -func clearMLAIMap(m map[roachpb.RangeID]ctpb.LAI) { - for rangeID := range m { - delete(m, rangeID) - } -} - -func releaseProposal( - ctx context.Context, - side string, - shouldLog bool, - minProp hlc.Timestamp, - rangeID roachpb.RangeID, - lai ctpb.LAI, - refs *int, - mlaiMap map[roachpb.RangeID]ctpb.LAI, - fromPreviousEpoch bool, -) { - if shouldLog { - log.Infof(ctx, "release: minprop %s on r%d@%d tracked on the %s", minProp, rangeID, lai, side) - } - *refs-- - if *refs < 0 { - log.Fatalf(ctx, "min proposal %s ref count < 0", side) - } - if rangeID == 0 { - return - } - if !fromPreviousEpoch { - if curLAI, found := mlaiMap[rangeID]; !found || curLAI < lai { - mlaiMap[rangeID] = lai - } - } -} diff --git a/pkg/kv/kvserver/closedts/minprop/tracker_test.go b/pkg/kv/kvserver/closedts/minprop/tracker_test.go deleted file mode 100644 index bc11241e4490..000000000000 --- a/pkg/kv/kvserver/closedts/minprop/tracker_test.go +++ /dev/null @@ -1,537 +0,0 @@ -// Copyright 2018 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 minprop - -import ( - "context" - "fmt" - "runtime" - "sync/atomic" - "testing" - - "github.com/cockroachdb/cockroach/pkg/cli/exit" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/cockroach/pkg/util/syncutil" - "github.com/cockroachdb/errors" - "github.com/kr/pretty" - "github.com/stretchr/testify/assert" - "golang.org/x/sync/errgroup" -) - -const ( - _ ctpb.Epoch = iota - ep1 - ep2 - ep3 -) - -func TestTrackerClosure(t *testing.T) { - ctx := context.Background() - tracker := NewTracker() - _, done := tracker.Track(ctx) - - done(ctx, ep1, 100, 200) - done(ctx, ep1, 0, 0) -} - -func ExampleTracker_Close() { - ctx := context.Background() - tracker := NewTracker() - _, slow := tracker.Track(ctx) - _, _, _ = tracker.Close(hlc.Timestamp{WallTime: 1e9}, ep1) - _, fast := tracker.Track(ctx) - - fmt.Println("Slow proposal finishes at LAI 2") - slow(ctx, ep1, 99, 2) - closed, m, ok := tracker.Close(hlc.Timestamp{WallTime: 2e9}, ep1) - fmt.Println("Closed:", closed, m, ok) - - fmt.Println("Fast proposal finishes at LAI 1") - fast(ctx, ep1, 99, 1) - fmt.Println(tracker) - closed, m, ok = tracker.Close(hlc.Timestamp{WallTime: 3e9}, ep1) - fmt.Println("Closed:", closed, m, ok) - fmt.Println("Note how the MLAI has 'regressed' from 2 to 1. The consumer") - fmt.Println("needs to track the maximum over all deltas received.") - - // Output: - // Slow proposal finishes at LAI 2 - // Closed: 1.000000000,0 map[99:2] true - // Fast proposal finishes at LAI 1 - // - // closed=1.000000000,0 - // | next=2.000000000,0 - // | left | right - // | 0 # 0 - // | 1 e 1 - // | 1 @ (r99) - // v v - // ---------------------------------------------------------> time - // - // Closed: 2.000000000,0 map[99:1] true - // Note how the MLAI has 'regressed' from 2 to 1. The consumer - // needs to track the maximum over all deltas received. -} - -func TestTrackerDoubleRelease(t *testing.T) { - var exited bool - log.SetExitFunc(true /* hideStack */, func(exit.Code) { exited = true }) - defer log.ResetExitFunc() - - ctx := context.Background() - tracker := NewTracker() - - _, release := tracker.Track(ctx) - release(ctx, ep1, 0, 0) - release(ctx, ep1, 4, 10) - - if !exited { - t.Fatal("expected fatal error") - } -} - -func TestTrackerReleaseZero(t *testing.T) { - ctx := context.Background() - tracker := NewTracker() - trackedTs1, release1 := tracker.Track(ctx) - trackedTs2, release2 := tracker.Track(ctx) - release2(ctx, ep1, 2, 0) - leftTs, _, _ := tracker.Close(trackedTs2, ep1) - leftTs.Logical += 2 - release1(ctx, ep1, 1, 0) - closedTs, mlais, ok := tracker.Close(leftTs, ep1) - if !ok { - t.Fatalf("expected closed to succeed") - } else if closedTs != trackedTs1 { - t.Fatalf("expected to have closed %v, got %v %v", trackedTs1, closedTs, mlais) - } else if mlai1, found := mlais[1]; !found { - t.Fatalf("expected to find mlai for range 1") - } else if mlai1 != 0 { - t.Fatalf("expected to find zero mlai for range 1, got %v", mlai1) - } else if mlai2, found := mlais[2]; !found { - t.Fatalf("expected to find mlai for range 2") - } else if mlai2 != 0 { - t.Fatalf("expected to find zero mlai for range 2, got %v", mlai2) - } -} - -type modelClient struct { - lai map[roachpb.RangeID]*int64 // read-only map, values accessed atomically - mu struct { - syncutil.Mutex - closed []hlc.Timestamp // closed timestamps - released []map[roachpb.RangeID]ctpb.LAI // known released LAIs, rotated on Close - m map[roachpb.RangeID]ctpb.LAI // max over all maps returned from Close() - } -} - -// Operate a Tracker concurrently and verify that closed timestamps don't regress -// and that the emitted MLAIs are not obviously inconsistent with commands we know -// finished. -func TestTrackerConcurrentUse(t *testing.T) { - ctx := context.Background() - tracker := NewTracker() - - const ( - numCmds = 1000 // operations to carry out in total - closeEvery = 20 // turn every i'th operation into a Close - numRanges = 5 - ) - - var mc modelClient - mc.mu.m = map[roachpb.RangeID]ctpb.LAI{} - mc.mu.closed = make([]hlc.Timestamp, 1) - mc.mu.released = []map[roachpb.RangeID]ctpb.LAI{{}, {}, {}} - - mc.lai = map[roachpb.RangeID]*int64{} - for i := roachpb.RangeID(1); i <= numRanges; i++ { - mc.lai[i] = new(int64) - } - - get := func(i int) (roachpb.RangeID, ctpb.LAI) { - rangeID := roachpb.RangeID(1 + (i % numRanges)) - return rangeID, ctpb.LAI(atomic.AddInt64(mc.lai[rangeID], 1)) - } - - // It becomes a lot more complicated to collect the released indexes - // correctly when multiple calls to Close are in-flight at any given time. - // The intended use case is for Close to be called from a single goroutine, - // so the test specializes to that situation. - // - // NB: The `mc.mu` sections are intentionally kept small to allow for more - // interleaving between tracked commands and close operations. - var closeMU syncutil.Mutex - close := func(newNext hlc.Timestamp) error { - closeMU.Lock() - defer closeMU.Unlock() - mc.mu.Lock() - // Note last closed timestamp. - prevClosed := mc.mu.closed[len(mc.mu.closed)-1] - - mc.mu.Unlock() - - t.Log("before closing:", tracker) - // Ignore epoch mismatches which may occur before any values have been - // released from the tracker. - closed, m, _ := tracker.Close(newNext, ep1) - if closed.Less(prevClosed) { - return errors.Errorf("closed timestamp regressed from %s to %s", prevClosed, closed) - } else if prevClosed == closed && len(m) != 0 { - return errors.Errorf("closed timestamp %s not incremented, but MLAIs %v emitted", prevClosed, m) - } - - mc.mu.Lock() - defer mc.mu.Unlock() - - if closed != prevClosed { - // The released bucket is rotated after each call to Close (we can't - // really do it before because we only want to rotate when a new - // closed timestamp was established). - // - // Taking into account the call to Close we just performed, the - // - current bucket contains: commands that could be on the left - // (expected) or the right: A command could start after our call to - // Close but make it into the pre-rotation bucket. - // - previous bucket contains commands that could be on the left - // or emitted - // - bucket before that contains commands that definitely must have - // been emitted. - // - // So we check the latter bucket. Trying to close the synchronization - // gap would allow checking the middle bucket instead, but this would - // weaken the test overall. - released := mc.mu.released[len(mc.mu.released)-3] - // Rotate released commands bucket. - mc.mu.released = append(mc.mu.released, map[roachpb.RangeID]ctpb.LAI{}) - - for rangeID, mlai := range m { - // Intuitively you expect mc.mu.m[rangeID] < mlai, but this - // doesn't always hold. A slow proposal could get assigned a - // higher lease index on the left side than a "newer" - // proposal on the right. The client really has to track the - // maximum. - // - if mc.mu.m[rangeID] < mlai { - mc.mu.m[rangeID] = mlai - } - - if trackedMLAI, rMLAI := mc.mu.m[rangeID], released[rangeID]; rMLAI > trackedMLAI { - return errors.Errorf( - "incorrect MLAI %d for r%d does not reflect %d:\nemitted: %+v\n%s\nreleased: %s\naggregate: %s", - trackedMLAI, rangeID, rMLAI, m, tracker, pretty.Sprint(mc.mu.released), pretty.Sprint(mc.mu.m), - ) - } - } - } - - // Store latest closed timestamp. - mc.mu.closed = append(mc.mu.closed, closed) - return nil - } - - newNext := func(i int) hlc.Timestamp { - return hlc.Timestamp{WallTime: int64(i) * 1e9} - } - - run := func(i int) error { - if i%closeEvery == 1 { - return close(newNext(i)) - } - - mc.mu.Lock() - prevClosed := mc.mu.closed[len(mc.mu.closed)-1] - mc.mu.Unlock() - - ts, done := tracker.Track(ctx) - if ts.Less(prevClosed) { - return errors.Errorf("%d: proposal forwarded to %s, but closed %s", i, ts, prevClosed) - } - - runtime.Gosched() - - var rangeID roachpb.RangeID - var lai ctpb.LAI - switch i % 3 { - case 0: - // Successful evaluation. - rangeID, lai = get(i) - done(ctx, ep1, rangeID, lai) - case 1: - // Successful evaluation followed by deferred zero call. - rangeID, lai = get(i) - done(ctx, ep1, rangeID, lai) - done(ctx, ep1, 0, 0) - case 2: - // Failed evaluation. Burns a LAI. - done(ctx, ep1, 0, 0) - default: - panic("the impossible happened") - } - - mc.mu.Lock() - if lai != 0 { - mc.mu.released[len(mc.mu.released)-1][rangeID] = lai - } - mc.mu.Unlock() - - return nil - } - - var g errgroup.Group - for i := 0; i < numCmds; i++ { - i := i - g.Go(func() error { - return run(i) - }) - } - - if err := g.Wait(); err != nil { - t.Fatal(err) - } - - // We'd like to at least assert something about the MLAIs below, namely that - // the final view of the client state is equivalent to the MLAIs that were - // actually used by the proposals. To get there, we need to close out twice: - // once to flush the right side to the left, and another time to force it - // to be output. - for i := 0; i < 2; i++ { - if err := close(newNext(numCmds + i)); err != nil { - t.Fatal(err) - } - } - - t.Log(tracker) - - for rangeID, addr := range mc.lai { - assignedMLAI := ctpb.LAI(atomic.LoadInt64(addr)) - mlai := mc.mu.m[rangeID] - - if assignedMLAI > mlai { - t.Errorf("r%d: assigned %d, but only %d reflected in final MLAI map", rangeID, assignedMLAI, mlai) - } - } -} - -// ExampleTracker_EpochChanges tests the interactions between epoch values -// passed to Close and epoch values of proposals being tracked. -func ExampleTracker_Close_epochChange() { - ts1 := hlc.Timestamp{WallTime: 1e9} - ts2 := hlc.Timestamp{WallTime: 2e9} - ts3 := hlc.Timestamp{WallTime: 3e9} - - ctx := context.Background() - tracker := NewTracker() - fmt.Println("The newly initialized tracker has a zero closed timestamp:") - fmt.Println(tracker) - - fmt.Println("A first command arrives on range 1 (though the range isn't known yet to the Tracker).") - ts, r1e1lai1 := tracker.Track(ctx) - fmt.Println("All commands initially start out on the right. The command has its timestamp forwarded to", ts, ".") - fmt.Println("The command finished quickly and is released in epoch 1.") - r1e1lai1(ctx, ep1, 1, 1) - fmt.Println(tracker) - - fmt.Println("Another proposal arrives on range 2 but does not complete before the next call to Close().") - _, r2e2lai1 := tracker.Track(ctx) - fmt.Println(tracker) - - fmt.Println("The system closes out a timestamp expecting liveness epoch 2 (registering", ts1, "as the next", - "timestamp to close out).") - closed, mlai, ok := tracker.Close(ts1, ep2) - fmt.Println("The Close() call fails due to the liveness epoch mismatch between", - "the expected current epoch and the tracked data, returning", closed, mlai, ok) - fmt.Println("The Close() call evicts the tracked range 1 LAI.") - fmt.Println(tracker) - - fmt.Println("The proposal on range 2 is released in epoch 2.") - r2e2lai1(ctx, ep2, 2, 1) - fmt.Println(tracker) - - fmt.Println("Another proposal arrives on range 1 and quickly finishes with", - "LAI 2 but is still in epoch 1 and is not tracked.") - _, r1e1lai2 := tracker.Track(ctx) - r1e1lai2(ctx, ep1, 2, 2) - fmt.Println("Meanwhile a proposal arrives on range 2 and quickly finishes with", - "LAI 2 in epoch 2.") - _, r2e2lai2 := tracker.Track(ctx) - r2e2lai2(ctx, ep2, 2, 2) - fmt.Println(tracker) - - fmt.Println("A new proposal arrives on range 1 and quickly finishes with LAI 2 in epoch 3.") - fmt.Println("This new epoch evicts the data on the right side corresponding to epoch 2.") - _, r1e3lai2 := tracker.Track(ctx) - r1e3lai2(ctx, ep3, 1, 2) - fmt.Println(tracker) - - closed, mlai, ok = tracker.Close(ts2, ep2) - fmt.Println("The next call to Close() occurs in epoch 2 and successfully returns:", closed, mlai, ok) - closed, mlai, ok = tracker.Close(ts3, ep2) - fmt.Println("Subsequent calls to Close() at later times but still in epoch 2 do not move the tracker state.") - fmt.Println("They return the previous closed timestamp with an empty mlai map:", closed, mlai, ok, ".") - fmt.Println("Data corresponding to epoch 3 is retained.") - fmt.Println(tracker) - closed, mlai, ok = tracker.Close(ts3, ep3) - fmt.Println("The next call to Close() occurs in epoch 3 and successfully returns:", closed, mlai, ok, ".") - - // Output: - // The newly initialized tracker has a zero closed timestamp: - // - // closed=0,0 - // | next=0,1 - // | left | right - // | 0 # 0 - // | 1 e 1 - // v v - // ---------------------------------------------------------> time - // - // A first command arrives on range 1 (though the range isn't known yet to the Tracker). - // All commands initially start out on the right. The command has its timestamp forwarded to 0,2 . - // The command finished quickly and is released in epoch 1. - // - // closed=0,0 - // | next=0,1 - // | left | right - // | 0 # 0 - // | 1 e 1 - // | @ 1 (r1) - // v v - // ---------------------------------------------------------> time - // - // Another proposal arrives on range 2 but does not complete before the next call to Close(). - // - // closed=0,0 - // | next=0,1 - // | left | right - // | 0 # 1 - // | 1 e 1 - // | @ 1 (r1) - // v v - // ---------------------------------------------------------> time - // - // The system closes out a timestamp expecting liveness epoch 2 (registering 1.000000000,0 as the next timestamp to close out). - // The Close() call fails due to the liveness epoch mismatch between the expected current epoch and the tracked data, returning 0,0 map[] false - // The Close() call evicts the tracked range 1 LAI. - // - // closed=0,1 - // | next=1.000000000,0 - // | left | right - // | 1 # 0 - // | 2 e 2 - // v v - // ---------------------------------------------------------> time - // - // The proposal on range 2 is released in epoch 2. - // - // closed=0,1 - // | next=1.000000000,0 - // | left | right - // | 0 # 0 - // | 2 e 2 - // | 1 @ (r2) - // v v - // ---------------------------------------------------------> time - // - // Another proposal arrives on range 1 and quickly finishes with LAI 2 but is still in epoch 1 and is not tracked. - // Meanwhile a proposal arrives on range 2 and quickly finishes with LAI 2 in epoch 2. - // - // closed=0,1 - // | next=1.000000000,0 - // | left | right - // | 0 # 0 - // | 2 e 2 - // | 1 @ (r2) - // | @ 2 (r2) - // v v - // ---------------------------------------------------------> time - // - // A new proposal arrives on range 1 and quickly finishes with LAI 2 in epoch 3. - // This new epoch evicts the data on the right side corresponding to epoch 2. - // - // closed=0,1 - // | next=1.000000000,0 - // | left | right - // | 0 # 0 - // | 2 e 3 - // | @ 2 (r1) - // | 1 @ (r2) - // v v - // ---------------------------------------------------------> time - // - // The next call to Close() occurs in epoch 2 and successfully returns: 1.000000000,0 map[2:1] true - // Subsequent calls to Close() at later times but still in epoch 2 do not move the tracker state. - // They return the previous closed timestamp with an empty mlai map: 1.000000000,0 map[] true . - // Data corresponding to epoch 3 is retained. - // - // closed=1.000000000,0 - // | next=2.000000000,0 - // | left | right - // | 0 # 0 - // | 3 e 3 - // | 2 @ (r1) - // v v - // ---------------------------------------------------------> time - // - // The next call to Close() occurs in epoch 3 and successfully returns: 2.000000000,0 map[1:2] true . -} - -// TestTrackerMultipleEpochsReleased tests that when proposals submitted between -// calls to Close span multiple epochs, only data for the highest epoch are -// retained and reported. -func TestTrackerMultipleEpochsReleased(t *testing.T) { - ts0 := hlc.Timestamp{Logical: 1} - ts1 := hlc.Timestamp{WallTime: 1e9} - ts2 := hlc.Timestamp{WallTime: 2e9} - ts3 := hlc.Timestamp{WallTime: 3e9} - - ctx := context.Background() - tracker := NewTracker() - - // Track and release a proposal on range 1 in ep1. - _, r1e1lai1 := tracker.Track(ctx) - r1e1lai1(ctx, ep1, 1, 1) - // Begin tracking a proposal on range 2 which won't be released until after - // the next call to Close. - _, r2e2lai1 := tracker.Track(ctx) - // Close the current left side and assert that the tracker reports an empty - // MLAI map in epoch 1 for the initial timestamp value. - assertClosed(tracker.Close(ts1, ep1))(t, ts0, mlais{}, true) - // Track and release another proposal on range 1 in epoch 1 with LAI 2. - // This proposal is on the right side. - _, r1e1lai2 := tracker.Track(ctx) - r1e1lai2(ctx, ep1, 1, 2) - // Release the proposal for range 2 in epoch 2 which should be on the left - // side. This release call will invalidate the LAI for range 1 that was - // recorded in epoch 1 both on the left and right side. - r2e2lai1(ctx, ep2, 2, 1) - // Close the current left side and assert that the tracker value on the - // range 1 epoch 1 value from the first interval is not present. - assertClosed(tracker.Close(ts2, ep2))(t, ts1, mlais{2: 1}, true) - assertClosed(tracker.Close(ts2, ep2))(t, ts1, nil, true) - assertClosed(tracker.Close(ts3, ep2))(t, ts2, mlais{}, true) -} - -type mlais = map[roachpb.RangeID]ctpb.LAI - -func assertClosed( - ts hlc.Timestamp, m mlais, ok bool, -) func(t *testing.T, expTs hlc.Timestamp, expM mlais, expOk bool) { - return func( - t *testing.T, expTs hlc.Timestamp, expM mlais, expOk bool, - ) { - t.Helper() - assert.Equal(t, expOk, ok) - assert.Equal(t, expTs, ts) - assert.EqualValues(t, expM, m) - } -} diff --git a/pkg/kv/kvserver/closedts/provider/BUILD.bazel b/pkg/kv/kvserver/closedts/provider/BUILD.bazel index 314bfce365d9..e69de29bb2d1 100644 --- a/pkg/kv/kvserver/closedts/provider/BUILD.bazel +++ b/pkg/kv/kvserver/closedts/provider/BUILD.bazel @@ -1,45 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") - -go_library( - name = "provider", - srcs = ["provider.go"], - importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/provider", - visibility = ["//visibility:public"], - deps = [ - "//pkg/clusterversion", - "//pkg/kv/kvserver/closedts", - "//pkg/kv/kvserver/closedts/ctpb", - "//pkg/roachpb:with-mocks", - "//pkg/settings/cluster", - "//pkg/util/hlc", - "//pkg/util/log", - "//pkg/util/stop", - "//pkg/util/syncutil", - "//pkg/util/timeutil", - "@com_github_cockroachdb_logtags//:logtags", - ], -) - -go_test( - name = "provider_test", - size = "small", - srcs = ["provider_test.go"], - deps = [ - ":provider", - "//pkg/kv/kvserver/closedts", - "//pkg/kv/kvserver/closedts/ctpb", - "//pkg/kv/kvserver/closedts/provider/testutils", - "//pkg/roachpb:with-mocks", - "//pkg/settings/cluster", - "//pkg/testutils", - "//pkg/testutils/skip", - "//pkg/util/hlc", - "//pkg/util/leaktest", - "//pkg/util/log", - "//pkg/util/stop", - "@com_github_cockroachdb_errors//:errors", - "@com_github_cockroachdb_logtags//:logtags", - "@com_github_stretchr_testify//require", - "@org_golang_x_sync//errgroup", - ], -) diff --git a/pkg/kv/kvserver/closedts/provider/provider.go b/pkg/kv/kvserver/closedts/provider/provider.go deleted file mode 100644 index 00bdfa671d8a..000000000000 --- a/pkg/kv/kvserver/closedts/provider/provider.go +++ /dev/null @@ -1,380 +0,0 @@ -// Copyright 2018 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 provider - -import ( - "context" - "math" - "sync" - "time" - - "github.com/cockroachdb/cockroach/pkg/clusterversion" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/cockroach/pkg/util/stop" - "github.com/cockroachdb/cockroach/pkg/util/syncutil" - "github.com/cockroachdb/cockroach/pkg/util/timeutil" - "github.com/cockroachdb/logtags" -) - -// Config holds the information necessary to create a Provider. -type Config struct { - // NodeID is the ID of the node on which the Provider is housed. - NodeID roachpb.NodeID - Settings *cluster.Settings - Stopper *stop.Stopper - Storage closedts.Storage - Clock closedts.LiveClockFn - Close closedts.CloseFn -} - -type subscriber struct { - ch chan<- ctpb.Entry - queue []ctpb.Entry -} - -// Provider implements closedts.Provider. It orchestrates the flow of closed -// timestamps and lets callers check whether they can serve reads. -type Provider struct { - cfg *Config - - mu struct { - syncutil.RWMutex - *sync.Cond // on RWMutex.RLocker() - // The current subscribers. The goroutine associated to each - // subscriber uses the RLock to mutate its slot. Thus, when - // accessing this slice for any other reason, the write lock - // needs to be acquired. - subscribers []*subscriber - draining bool // tell subscribers to terminate - } - - everyClockLog log.EveryN -} - -var _ closedts.Provider = (*Provider)(nil) - -// NewProvider initializes a Provider, that has yet to be started. -func NewProvider(cfg *Config) *Provider { - p := &Provider{ - cfg: cfg, - everyClockLog: log.Every(time.Minute), - } - p.mu.Cond = sync.NewCond(p.mu.RLocker()) - return p -} - -// Start implements closedts.Provider. -func (p *Provider) Start() { - if err := p.cfg.Stopper.RunAsyncTask( - logtags.AddTag(context.Background(), "ct-closer", nil), "ct-closer", p.runCloser, - ); err != nil { - p.drain() - } -} - -func (p *Provider) drain() { - p.mu.Lock() - p.mu.draining = true - p.mu.Unlock() - for { - p.mu.Broadcast() - p.mu.Lock() - done := true - for _, sub := range p.mu.subscribers { - done = done && sub == nil - } - p.mu.Unlock() - - if done { - return - } - } -} - -func (p *Provider) runCloser(ctx context.Context) { - // The loop below signals the subscribers, so when it exits it needs to do - // extra work to help the subscribers terminate. - defer p.drain() - - if p.cfg.NodeID == 0 { - // This Provider is likely misconfigured. - panic("can't use NodeID zero") - } - ch := p.Notify(p.cfg.NodeID) - defer close(ch) - - confCh := make(chan struct{}, 1) - confChanged := func(ctx context.Context) { - select { - case confCh <- struct{}{}: - default: - } - } - closedts.TargetDuration.SetOnChange(&p.cfg.Settings.SV, confChanged) - // Track whether we've ever been live to avoid logging warnings about not - // being live during node startup. - var everBeenLive bool - t := timeutil.NewTimer() - defer t.Stop() - for { - // If the "new" closed timestamps mechanism is enabled, we inhibit this old one. - if p.cfg.Settings.Version.IsActive(ctx, clusterversion.ClosedTimestampsRaftTransport) { - log.Infof(ctx, "disabling legacy closed-timestamp mechanism; the new one is enabled") - break - } - - closeFraction := closedts.CloseFraction.Get(&p.cfg.Settings.SV) - targetDuration := float64(closedts.TargetDuration.Get(&p.cfg.Settings.SV)) - if targetDuration > 0 { - t.Reset(time.Duration(closeFraction * targetDuration)) - } else { - // Disable closing when the target duration is non-positive. - t.Stop() - t = timeutil.NewTimer() - } - select { - case <-p.cfg.Stopper.ShouldQuiesce(): - return - case <-ctx.Done(): - return - case <-t.C: - t.Read = true - case <-confCh: - // Loop around to use the updated timer. - continue - } - - next, liveAtEpoch, err := p.cfg.Clock(p.cfg.NodeID) - next = next.Add(-int64(targetDuration), 0) - if err != nil { - if everBeenLive && p.everyClockLog.ShouldLog() { - log.Warningf(ctx, "unable to move closed timestamp forward: %+v", err) - } - // Broadcast even if nothing new was queued, so that the subscribers - // loop to check their client's context. - p.mu.Broadcast() - } else { - everBeenLive = true - // Close may fail if the data being closed does not correspond to the - // current liveAtEpoch. - closed, m, ok := p.cfg.Close(next, liveAtEpoch) - if !ok { - if log.V(1) { - log.Infof(ctx, "failed to close %v due to liveness epoch mismatch at %v", - next, liveAtEpoch) - } - continue - } - if log.V(1) { - log.Infof(ctx, "closed ts=%s with %+v, next closed timestamp should be %s", - closed, m, next) - } - entry := ctpb.Entry{ - Epoch: liveAtEpoch, - ClosedTimestamp: closed, - MLAI: m, - } - - // Simulate a subscription to the local node, so that the new information - // is added to the storage (and thus becomes available to future subscribers - // as well, not only to existing ones). The other end of the chan will Broadcast(). - // - // TODO(tschottdorf): the transport should ignore connection requests from - // the node to itself. Those connections would pointlessly loop this around - // once more. - select { - case ch <- entry: - case <-p.cfg.Stopper.ShouldQuiesce(): - return - } - } - } -} - -// Notify implements closedts.Notifyee. It passes the incoming stream of Entries -// to the local Storage. -func (p *Provider) Notify(nodeID roachpb.NodeID) chan<- ctpb.Entry { - ch := make(chan ctpb.Entry) - - _ = p.cfg.Stopper.RunAsyncTask(context.Background(), "provider-notify", func(ctx context.Context) { - handle := func(entry ctpb.Entry) { - p.cfg.Storage.Add(nodeID, entry) - } - // Special-case data about the origin node, which folks can subscribe to. - // This is easily generalized to also allow subscriptions for data that - // originated on other nodes, but this doesn't seem necessary right now. - if nodeID == p.cfg.NodeID { - handle = func(entry ctpb.Entry) { - // Add to the Storage first. - p.cfg.Storage.Add(nodeID, entry) - // Notify existing subscribers. - p.mu.Lock() - for _, sub := range p.mu.subscribers { - if sub == nil { - continue - } - sub.queue = append(sub.queue, entry) - } - p.mu.Unlock() - // Wake up all clients. - p.mu.Broadcast() - } - } - for { - select { - case entry, ok := <-ch: - if !ok { - return - } - handle(entry) - case <-p.cfg.Stopper.ShouldQuiesce(): - return - } - } - }) - - return ch -} - -// Subscribe implements closedts.Producer. It produces a stream of Entries -// pertaining to the local Node. -// -// TODO(tschottdorf): consider not forcing the caller to launch the goroutine. -func (p *Provider) Subscribe(ctx context.Context, ch chan<- ctpb.Entry) { - var i int - sub := &subscriber{ch, nil} - p.mu.Lock() - for i = 0; i < len(p.mu.subscribers); i++ { - if p.mu.subscribers[i] == nil { - p.mu.subscribers[i] = sub - break - } - } - if i == len(p.mu.subscribers) { - p.mu.subscribers = append(p.mu.subscribers, sub) - } - draining := p.mu.draining - p.mu.Unlock() - - defer func() { - p.mu.Lock() - p.mu.subscribers[i] = nil - p.mu.Unlock() - close(ch) - }() - - if draining { - return - } - - if log.V(1) { - log.Infof(ctx, "new subscriber (slot %d) connected", i) - } - - // The subscription is already active, so any storage snapshot from now on is - // going to fully catch up the subscriber without a gap. - { - var entries []ctpb.Entry - - p.cfg.Storage.VisitAscending(p.cfg.NodeID, func(entry ctpb.Entry) (done bool) { - // Don't block in this method. - entries = append(entries, entry) - return false // not done - }) - - for _, entry := range entries { - select { - case ch <- entry: - case <-p.cfg.Stopper.ShouldQuiesce(): - return - case <-ctx.Done(): - return - } - } - } - - for { - p.mu.RLock() - var done bool - for len(p.mu.subscribers[i].queue) == 0 { - if ctx.Err() != nil || p.mu.draining { - done = true - break - } - p.mu.Wait() - } - var queue []ctpb.Entry - // When only readers are around (as they are now), we can actually - // mutate our slot because that's all the others do as well. - queue, p.mu.subscribers[i].queue = p.mu.subscribers[i].queue, nil - p.mu.RUnlock() - - if done { - return - } - - shouldLog := log.V(1) - var n int - minMLAI := ctpb.LAI(math.MaxInt64) - var minRangeID, maxRangeID roachpb.RangeID - var maxMLAI ctpb.LAI - - for _, entry := range queue { - if shouldLog { - n += len(entry.MLAI) - for rangeID, mlai := range entry.MLAI { - if mlai < minMLAI { - minMLAI = mlai - minRangeID = rangeID - } - if mlai > maxMLAI { - maxMLAI = mlai - maxRangeID = rangeID - } - } - } - - select { - case ch <- entry: - case <-p.cfg.Stopper.ShouldQuiesce(): - return - case <-ctx.Done(): - return - } - } - if shouldLog { - log.Infof(ctx, "sent %d closed timestamp entries to client %d (%d range updates total, min/max mlai: %d@r%d / %d@r%d)", len(queue), i, n, minMLAI, minRangeID, maxMLAI, maxRangeID) - } - } -} - -// MaxClosed implements closedts.Provider. -func (p *Provider) MaxClosed( - nodeID roachpb.NodeID, rangeID roachpb.RangeID, epoch ctpb.Epoch, lai ctpb.LAI, -) hlc.Timestamp { - var maxTS hlc.Timestamp - p.cfg.Storage.VisitDescending(nodeID, func(entry ctpb.Entry) (done bool) { - if mlai, found := entry.MLAI[rangeID]; found { - if entry.Epoch == epoch && mlai <= lai { - maxTS = entry.ClosedTimestamp - return true - } - } - return false - }) - - return maxTS -} diff --git a/pkg/kv/kvserver/closedts/provider/provider_test.go b/pkg/kv/kvserver/closedts/provider/provider_test.go deleted file mode 100644 index fde9caa3cdf7..000000000000 --- a/pkg/kv/kvserver/closedts/provider/provider_test.go +++ /dev/null @@ -1,352 +0,0 @@ -// Copyright 2018 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 provider_test - -import ( - "context" - "reflect" - "sync" - "sync/atomic" - "testing" - "time" - - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/provider" - providertestutils "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/provider/testutils" - "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/skip" - "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/cockroach/pkg/util/stop" - "github.com/cockroachdb/errors" - "github.com/cockroachdb/logtags" - "github.com/stretchr/testify/require" - "golang.org/x/sync/errgroup" -) - -func TestProviderSubscribeNotify(t *testing.T) { - defer leaktest.AfterTest(t)() - skip.WithIssue(t, closedts.IssueTrackingRemovalOfOldClosedTimestampsCode) - - ctx := context.Background() - stopper := stop.NewStopper() - defer stopper.Stop(ctx) - - st := cluster.MakeTestingClusterSettings() - // We'll only unleash the closer loop when the test is basically done, and - // once we do that we want it to run aggressively. - // Testing that the closer loop works as advertised is left to another test. - closedts.TargetDuration.Override(ctx, &st.SV, time.Millisecond) - closedts.CloseFraction.Override(ctx, &st.SV, 1.0) - - storage := &providertestutils.TestStorage{} - unblockClockCh := make(chan struct{}) - cfg := &provider.Config{ - NodeID: 2, // note that we're not using 1, just for kicks - Settings: st, - Stopper: stopper, - Storage: storage, - Clock: func(roachpb.NodeID) (hlc.Timestamp, ctpb.Epoch, error) { - select { - case <-stopper.ShouldQuiesce(): - return hlc.Timestamp{}, 0, errors.New("stopping") - case <-unblockClockCh: - } - return hlc.Timestamp{}, ctpb.Epoch(1), errors.New("injected clock error") - }, - Close: func(next hlc.Timestamp, expCurEpoch ctpb.Epoch) (hlc.Timestamp, map[roachpb.RangeID]ctpb.LAI, bool) { - panic("should never be called") - }, - } - - p := provider.NewProvider(cfg) - p.Start() - - // We won't touch n1 in this test, so this entry should never pop up. - unseenEntry := ctpb.Entry{ - ClosedTimestamp: hlc.Timestamp{WallTime: 456}, - Epoch: 17, - } - cfg.Storage.Add(1, unseenEntry) - - entryAt := func(i int) ctpb.Entry { - return ctpb.Entry{ - ClosedTimestamp: hlc.Timestamp{WallTime: int64(i) * 1e9}, - Epoch: ctpb.Epoch(i), - MLAI: map[roachpb.RangeID]ctpb.LAI{ - roachpb.RangeID(i): ctpb.LAI(10 * i), - }, - } - } - - const numEntries = 10 // must be even - var entries []ctpb.Entry - for i := 0; i < numEntries; i++ { - entries = append(entries, entryAt(i)) - } - - var readerSeq int32 // atomically - reader := func() error { - i := atomic.AddInt32(&readerSeq, 1) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - ctx = logtags.AddTag(ctx, "reader", int(i)) - - log.Infof(ctx, "starting") - defer log.Infof(ctx, "done") - - ch := make(chan ctpb.Entry) - _ = stopper.RunAsyncTask(ctx, "subscribe", func(ctx context.Context) { - p.Subscribe(ctx, ch) - }) - - var sl []ctpb.Entry // for debug purposes only - // Read entries off the subscription. We check two invariants: - // 1. we see each Entry (identified via its Epoch) at least twice - // (morally exactly twice, but the Provider gives a weaker guarantee) - // 2. An Entry can only be observed after the previous Entry has been seen - // at least once. That is, to see Epoch X, we need to have seen Epoch X-1. - // - // These could be sharpened somewhat, but only at a distinct loss of clarity - // in this part of the test. - // - // Examples, writing only the Epoch (which uniquely identifies the Entry in this test): - // OK: - // - 1 2 3 1 2 3 - // First writer sends everything before second writer. - // - 1 1 2 3 1 2 3 - // First writer sends everything before second, but first 1 gets duplicated by Provider. - // - 1 2 3 1 2 3 1 2 3 - // Same as last, but whole thing gets duplicated. - // - 1 2 3 2 3 1 2 3 - // Only 2 3 gets duplicated. - // Not OK: - // - 1 1 2 3 3 - // Two seen only once. - // - 1 3 2 1 2 3 - // Three observed before two. - m := map[ctpb.Epoch]int{-1: 2} // pretend we've seen Epoch -1 twice, streamlines code below - expM := map[ctpb.Epoch]int{-1: 2} - for _, entry := range entries { - expM[entry.Epoch] = 2 - } - for { - select { - case <-time.After(10 * time.Second): - return errors.Errorf("nothing emitted after %v", sl) - case entry, ok := <-ch: // implies runtime.Gosched - if !ok { - if ctx.Err() != nil { - // Expected, we must've canceled the context below earlier, which means the - // checks were successful. - return nil - } - return errors.New("sender closed channel before reader canceled their context") - } - sl = append(sl, entry) - log.Infof(ctx, "got %d entries now,latest: %+v", len(sl), entry) - diagErr := errors.Errorf("saw: %v", sl) - prevEpo := entry.Epoch - 1 - if m[prevEpo] < 1 { - return errors.Wrapf( - diagErr, - "entry for epoch %d received before a matching entry for immediately preceding epoch %d", - entry.Epoch, prevEpo, - ) - } - m[entry.Epoch]++ - if m[entry.Epoch] > 2 { - m[entry.Epoch] = 2 - } - - if reflect.DeepEqual(expM, m) && ctx.Err() == nil { - log.Info(ctx, "canceling subscription") - cancel() - // As a little gotcha, we need to work around the implementation a tiny bit. - // The provider uses a sync.Cond to notify clients and it is likely waiting - // for new activity for our subscription. Thus, it's not going to notice - // that this client is going away; it would notice if the Provider's closer - // did its job (we've blocked it so far) because that periodically wakes - // up all clients, rain or shine. So we unblock it now; the Clock is set up - // to return errors, so as a nice little benefit we verify that even in that - // case the subscription does get woken up. - close(unblockClockCh) - } - } - } - } - - // Add some entries via Notify, and race them with various subscriptions. Note - // that in reality, we have only a single notification going on for the local node - // (run by a Provider goroutine). But the data that comes in from other nodes uses - // the same mechanism, and it's nice to get coverage for it. In particular, during - // reconnections, you could imagine two notification streams for the same NodeID to - // be active in parallel. - var g errgroup.Group - for i := range []struct{}{{}, {}} { // twice - i := i // goroutine copy - g.Go(func() error { - ctx := logtags.AddTag(context.Background(), "writer", i) - log.Info(ctx, "starting") - defer log.Info(ctx, "done") - nCh := p.Notify(roachpb.NodeID(2)) - defer close(nCh) - for _, entry := range entries { - nCh <- entry // implies runtime.Gosched - log.Infof(ctx, "wrote %s", entry) - } - return nil - }) - } - - for i := 0; i < 1; i++ { // HACK - g.Go(reader) - } - if err := g.Wait(); err != nil { - t.Fatal(err) - } - - testutils.SucceedsSoon(t, func() error { - snap := storage.Snapshot() - require.Equal(t, 2, len(snap)) // definitely should have records about two nodes - require.Equal(t, 1, len(snap[1])) // one persisted entry for n1 - // Morally this is true immediately, but consider that the goroutine consuming - // from the writer threads above may have read the entry but not put it into - // the Storage yet. The reader threads would usually remove this race, but - // they can be satisfied early by a duplicate that is emitted during the - // switchover from storage to subscription. - if exp, act := 2*numEntries, len(snap[2]); exp < act { - t.Fatalf("got %d entries in storage, expected no more than %d", act, exp) - } else if exp > act { - return errors.Errorf("storage has %d entries, need %d", exp, act) - } - return nil - }) -} - -// TestProviderSubscribeConcurrent prevents regression of a bug that improperly -// handled concurrent subscriptions. -func TestProviderSubscribeConcurrent(t *testing.T) { - defer leaktest.AfterTest(t)() - ctx := context.Background() - - st := cluster.MakeTestingClusterSettings() - closedts.TargetDuration.Override(ctx, &st.SV, time.Millisecond) - closedts.CloseFraction.Override(ctx, &st.SV, 1.0) - - stopper := stop.NewStopper() - storage := &providertestutils.TestStorage{} - - var ts int64 // atomic - cfg := &provider.Config{ - NodeID: 1, - Settings: st, - Stopper: stopper, - Storage: storage, - Clock: func(roachpb.NodeID) (hlc.Timestamp, ctpb.Epoch, error) { - return hlc.Timestamp{}, 1, nil - }, - Close: func(next hlc.Timestamp, expCurEpoch ctpb.Epoch) (hlc.Timestamp, map[roachpb.RangeID]ctpb.LAI, bool) { - return hlc.Timestamp{ - WallTime: atomic.AddInt64(&ts, 1), - }, map[roachpb.RangeID]ctpb.LAI{ - 1: ctpb.LAI(atomic.LoadInt64(&ts)), - }, true - }, - } - - p := provider.NewProvider(cfg) - p.Start() - - ctx, cancel := context.WithTimeout(context.Background(), 5*time.Millisecond) - defer cancel() - cancel = func() {} - const n = 10 - var wg sync.WaitGroup - wg.Add(n) - for i := 0; i < n; i++ { - go func() { - defer wg.Done() - ch := make(chan ctpb.Entry, 3) - p.Subscribe(ctx, ch) - // Read from channel until stopper stops Provider (and in turn Provider - // closes channel). - for range ch { - } - }() - } - stopper.Stop(context.Background()) - wg.Wait() -} - -// TestProviderTargetDurationSetting ensures that setting the target duration to -// zero disables closing the timestamp and that setting it back to a positive -// value re-enables it. -func TestProviderTargetDurationSetting(t *testing.T) { - defer leaktest.AfterTest(t)() - skip.WithIssue(t, closedts.IssueTrackingRemovalOfOldClosedTimestampsCode) - ctx := context.Background() - - st := cluster.MakeTestingClusterSettings() - closedts.TargetDuration.Override(ctx, &st.SV, time.Millisecond) - closedts.CloseFraction.Override(ctx, &st.SV, 1.0) - - stopper := stop.NewStopper() - storage := &providertestutils.TestStorage{} - defer stopper.Stop(context.Background()) - - var ts int64 // atomic - var called int - calledCh := make(chan struct{}) - cfg := &provider.Config{ - NodeID: 1, - Settings: st, - Stopper: stopper, - Storage: storage, - Clock: func(roachpb.NodeID) (hlc.Timestamp, ctpb.Epoch, error) { - return hlc.Timestamp{}, 1, nil - }, - Close: func(next hlc.Timestamp, expCurEpoch ctpb.Epoch) (hlc.Timestamp, map[roachpb.RangeID]ctpb.LAI, bool) { - if called++; called == 1 { - closedts.TargetDuration.Override(ctx, &st.SV, 0) - } - select { - case calledCh <- struct{}{}: - case <-stopper.ShouldQuiesce(): - } - return hlc.Timestamp{ - WallTime: atomic.AddInt64(&ts, 1), - }, map[roachpb.RangeID]ctpb.LAI{ - 1: ctpb.LAI(atomic.LoadInt64(&ts)), - }, true - }, - } - - p := provider.NewProvider(cfg) - p.Start() - - // Get called once. While it's being called, we set the target duration to 0, - // disabling the updates. We wait someTime and ensure we don't get called - // again. Then we re-enable the setting and ensure we do get called. - <-calledCh - const someTime = 10 * time.Millisecond - select { - case <-calledCh: - t.Fatal("expected no updates to be sent") - case <-time.After(someTime): - } - closedts.TargetDuration.Override(ctx, &st.SV, time.Millisecond) - <-calledCh -} diff --git a/pkg/kv/kvserver/closedts/provider/testutils/BUILD.bazel b/pkg/kv/kvserver/closedts/provider/testutils/BUILD.bazel index c2ab5bc7b9c0..e69de29bb2d1 100644 --- a/pkg/kv/kvserver/closedts/provider/testutils/BUILD.bazel +++ b/pkg/kv/kvserver/closedts/provider/testutils/BUILD.bazel @@ -1,19 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") - -go_library( - name = "testutils", - srcs = [ - "clock.go", - "storage.go", - ], - importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/provider/testutils", - visibility = ["//visibility:public"], - deps = [ - "//pkg/kv/kvserver/closedts/ctpb", - "//pkg/roachpb:with-mocks", - "//pkg/util/hlc", - "//pkg/util/stop", - "//pkg/util/syncutil", - "@com_github_cockroachdb_errors//:errors", - ], -) diff --git a/pkg/kv/kvserver/closedts/provider/testutils/clock.go b/pkg/kv/kvserver/closedts/provider/testutils/clock.go deleted file mode 100644 index 21176fdadb05..000000000000 --- a/pkg/kv/kvserver/closedts/provider/testutils/clock.go +++ /dev/null @@ -1,58 +0,0 @@ -// Copyright 2018 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 testutils - -import ( - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/cockroach/pkg/util/stop" - "github.com/cockroachdb/errors" -) - -// A TestClock provides a blocking LiveClockFn that can be triggered -// at will. -type TestClock struct { - stopper *stop.Stopper - ch chan tick -} - -// NewTestClock sets up a test clock that returns errors once the -// Stopper starts quiescing. -func NewTestClock(stopper *stop.Stopper) *TestClock { - t := &TestClock{ - stopper: stopper, - ch: make(chan tick), - } - return t -} - -type tick struct { - liveNow hlc.Timestamp - liveEpoch ctpb.Epoch - err error -} - -// Tick is called by tests to manually emit a single clock tick. The tick -// will only returned to a single caller of LiveNow(). -func (c *TestClock) Tick(liveNow hlc.Timestamp, liveEpoch ctpb.Epoch, err error) { - c.ch <- tick{liveNow, liveEpoch, err} -} - -// LiveNow implements closedts.LiveClockFn. -func (c *TestClock) LiveNow(roachpb.NodeID) (liveNow hlc.Timestamp, liveEpoch ctpb.Epoch, _ error) { - select { - case r := <-c.ch: - return r.liveNow, r.liveEpoch, r.err - case <-c.stopper.ShouldQuiesce(): - return hlc.Timestamp{}, 0, errors.New("quiescing") - } -} diff --git a/pkg/kv/kvserver/closedts/provider/testutils/storage.go b/pkg/kv/kvserver/closedts/provider/testutils/storage.go deleted file mode 100644 index c5ac0ccd9e98..000000000000 --- a/pkg/kv/kvserver/closedts/provider/testutils/storage.go +++ /dev/null @@ -1,87 +0,0 @@ -// Copyright 2018 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 testutils - -import ( - "sort" - - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/util/syncutil" -) - -// TestStorage is a storage backed by a map[NodeID]Entry. -type TestStorage struct { - mu syncutil.Mutex - m map[roachpb.NodeID][]ctpb.Entry -} - -// VisitAscending implements closedts.Storage. -func (s *TestStorage) VisitAscending(nodeID roachpb.NodeID, f func(ctpb.Entry) (done bool)) { - s.mu.Lock() - defer s.mu.Unlock() - - for _, entry := range s.m[nodeID] { - if f(entry) { - break - } - } -} - -// VisitDescending implements closedts.Storage. -func (s *TestStorage) VisitDescending(nodeID roachpb.NodeID, f func(entry ctpb.Entry) (done bool)) { - s.mu.Lock() - defer s.mu.Unlock() - - for i := len(s.m[nodeID]) - 1; i >= 0; i-- { - if f(s.m[nodeID][i]) { - break - } - } -} - -// Add implements closedts.Storage. -func (s *TestStorage) Add(nodeID roachpb.NodeID, entry ctpb.Entry) { - s.mu.Lock() - defer s.mu.Unlock() - - if s.m == nil { - s.m = map[roachpb.NodeID][]ctpb.Entry{} - } - - s.m[nodeID] = append(s.m[nodeID], entry) - sort.Slice(s.m[nodeID], func(i, j int) bool { - e1, e2 := s.m[nodeID][i], s.m[nodeID][j] - if e1.ClosedTimestamp == e2.ClosedTimestamp { - return e1.Epoch < e2.Epoch - } - return e1.ClosedTimestamp.Less(e2.ClosedTimestamp) - }) -} - -// Clear implements closedts.Storage. -func (s *TestStorage) Clear() { - s.mu.Lock() - defer s.mu.Unlock() - s.m = nil -} - -// Snapshot returns a copy of the data contain within the TestStorage. -func (s *TestStorage) Snapshot() map[roachpb.NodeID][]ctpb.Entry { - s.mu.Lock() - defer s.mu.Unlock() - - m := map[roachpb.NodeID][]ctpb.Entry{} - for nodeID, entries := range s.m { - m[nodeID] = append([]ctpb.Entry(nil), entries...) - } - return m -} diff --git a/pkg/kv/kvserver/closedts/sidetransport/BUILD.bazel b/pkg/kv/kvserver/closedts/sidetransport/BUILD.bazel index 3f7cad1ae09e..8788cff8d3fe 100644 --- a/pkg/kv/kvserver/closedts/sidetransport/BUILD.bazel +++ b/pkg/kv/kvserver/closedts/sidetransport/BUILD.bazel @@ -13,7 +13,6 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/base", - "//pkg/clusterversion", "//pkg/kv/kvserver/closedts", "//pkg/kv/kvserver/closedts/ctpb", "//pkg/roachpb:with-mocks", diff --git a/pkg/kv/kvserver/closedts/sidetransport/sender.go b/pkg/kv/kvserver/closedts/sidetransport/sender.go index 85b14296f0e3..00536eb51a45 100644 --- a/pkg/kv/kvserver/closedts/sidetransport/sender.go +++ b/pkg/kv/kvserver/closedts/sidetransport/sender.go @@ -20,7 +20,6 @@ import ( "sync/atomic" "time" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -221,8 +220,6 @@ func newSenderWithConnFactory( // This is not know at construction time. func (s *Sender) Run(ctx context.Context, nodeID roachpb.NodeID) { s.nodeID = nodeID - waitForUpgrade := !s.st.Version.IsActive(ctx, clusterversion.ClosedTimestampsRaftTransport) - confCh := make(chan struct{}, 1) confChanged := func(ctx context.Context) { select { @@ -253,12 +250,6 @@ func (s *Sender) Run(ctx context.Context, nodeID roachpb.NodeID) { select { case <-timer.C: timer.Read = true - if waitForUpgrade && !s.st.Version.IsActive(ctx, clusterversion.ClosedTimestampsRaftTransport) { - continue - } else if waitForUpgrade { - waitForUpgrade = false - log.Infof(ctx, "closed-timestamps v2 mechanism enabled by cluster version upgrade") - } s.publish(ctx) case <-confCh: // Loop around to use the updated timer. diff --git a/pkg/kv/kvserver/closedts/storage/BUILD.bazel b/pkg/kv/kvserver/closedts/storage/BUILD.bazel index 7aecfb713cd3..e69de29bb2d1 100644 --- a/pkg/kv/kvserver/closedts/storage/BUILD.bazel +++ b/pkg/kv/kvserver/closedts/storage/BUILD.bazel @@ -1,35 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") - -go_library( - name = "storage", - srcs = [ - "storage.go", - "storage_mem.go", - ], - importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/storage", - visibility = ["//visibility:public"], - deps = [ - "//pkg/kv/kvserver/closedts", - "//pkg/kv/kvserver/closedts/ctpb", - "//pkg/roachpb:with-mocks", - "//pkg/util/syncutil", - "@com_github_olekukonko_tablewriter//:tablewriter", - ], -) - -go_test( - name = "storage_test", - size = "small", - srcs = ["storage_test.go"], - embed = [":storage"], - deps = [ - "//pkg/kv/kvserver/closedts/ctpb", - "//pkg/roachpb:with-mocks", - "//pkg/util/hlc", - "//pkg/util/leaktest", - "//pkg/util/randutil", - "//pkg/util/timeutil", - "@com_github_cockroachdb_errors//:errors", - "@org_golang_x_sync//errgroup", - ], -) diff --git a/pkg/kv/kvserver/closedts/storage/storage.go b/pkg/kv/kvserver/closedts/storage/storage.go deleted file mode 100644 index 1f59e8dfd7c3..000000000000 --- a/pkg/kv/kvserver/closedts/storage/storage.go +++ /dev/null @@ -1,170 +0,0 @@ -// Copyright 2018 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 storage - -import ( - "bytes" - "fmt" - "sort" - "unsafe" - - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/util/syncutil" -) - -// SingleStorage stores and manages closed timestamp updates originating from a -// single source (i.e. node). A SingleStorage internally maintains multiple -// buckets for historical closed timestamp information. The reason for this is -// twofold: -// -// 1. The most recent closed timestamp update is also the hardest to prove a -// read for, since it comes with larger minimum lease applied indexes. In -// situations in which followers are lagging behind with their command -// application, this could lead to a runaway scenario, in which a closed -// timestamp update can never be used until it is replaced by a new one, which -// in turn also will never be used, etc. Instead, a SingleStorage keeps some -// amount of history and upstream systems can try to prove a follower read using -// an older closed timestamp instead. -// -// 2. Follower reads can be used to implement recovery of a consistent -// cluster-wide snapshot after catastrophic loss of quorum. To do this, the -// mechanism must locate at least one replica of every range in the cluster, and -// for each range find the largest possible timestamp at which follower reads -// are possible among the surviving replicas. Of all these per-range timestamps, -// the smallest can be used to read from all ranges, resulting in a consistent -// snapshot. This makes it crucial that every replica can serve at least some -// follower reads, even when regularly outpaced by the closed timestamp -// frontier. Emitted MLAIs may never even be proposed to Raft in the event of -// an ill-timed crash, and so historic information is invaluable. -// -// TODO(tschottdorf): revisit whether this shouldn't be a concrete impl instead, -// with only the buckets abstracted out. -type SingleStorage interface { - fmt.Stringer - // VisitAscending walks through the buckets of the storage in ascending - // closed timestamp order, until the closure returns true (or all buckets - // have been visited). - VisitAscending(func(ctpb.Entry) (done bool)) - // VisitDescending walks through the buckets of the storage in descending - // closed timestamp order, until the closure returns true (or all buckets - // have been visited). - VisitDescending(func(ctpb.Entry) (done bool)) - // Add adds a new Entry to this storage. The entry is added to the most - // recent bucket and remaining buckets are rotated as indicated by their age - // relative to the newly added Entry. - Add(ctpb.Entry) - // Clear removes all Entries from this storage. - Clear() -} - -type entry struct { - SingleStorage -} - -// MultiStorage implements the closedts.Storage interface. -type MultiStorage struct { - // constructor creates a SingleStorage whenever one is initialized for a new - // NodeID. - constructor func() SingleStorage - // TODO(tschottdorf): clean up storages that haven't been used for extended - // periods of time. - m syncutil.IntMap -} - -var _ closedts.Storage = (*MultiStorage)(nil) - -// NewMultiStorage sets up a MultiStorage which uses the given factory method -// for setting up the SingleStorage used for each individual NodeID for which -// operations are received. -func NewMultiStorage(constructor func() SingleStorage) *MultiStorage { - return &MultiStorage{constructor: constructor} -} - -func (ms *MultiStorage) getOrCreate(nodeID roachpb.NodeID) SingleStorage { - key := int64(nodeID) - p, found := ms.m.Load(key) - if found { - // Fast path that avoids calling f(). - return (*entry)(p).SingleStorage - } - - ss := ms.constructor() - p, _ = ms.m.LoadOrStore(key, unsafe.Pointer(&entry{ss})) - return (*entry)(p).SingleStorage -} - -// VisitAscending implements closedts.Storage. -func (ms *MultiStorage) VisitAscending(nodeID roachpb.NodeID, f func(ctpb.Entry) (done bool)) { - ss := ms.getOrCreate(nodeID) - ss.VisitAscending(f) -} - -// VisitDescending implements closedts.Storage. -func (ms *MultiStorage) VisitDescending(nodeID roachpb.NodeID, f func(ctpb.Entry) (done bool)) { - ss := ms.getOrCreate(nodeID) - ss.VisitDescending(f) -} - -// Add implements closedts.Storage. -func (ms *MultiStorage) Add(nodeID roachpb.NodeID, entry ctpb.Entry) { - ss := ms.getOrCreate(nodeID) - ss.Add(entry) -} - -// Clear implements closedts.Storage. -func (ms *MultiStorage) Clear() { - ms.m.Range(func(_ int64, p unsafe.Pointer) bool { - (*entry)(p).SingleStorage.Clear() - return true // continue - }) -} - -// String prints a tabular rundown of the contents of the MultiStorage. -func (ms *MultiStorage) String() string { - return ms.StringForNodes() -} - -// StringForNodes is like String, but restricted to the supplied NodeIDs. -// If none are specified, is equivalent to String(). -func (ms *MultiStorage) StringForNodes(nodes ...roachpb.NodeID) string { - type tuple struct { - roachpb.NodeID - SingleStorage - } - - var shouldPrint map[roachpb.NodeID]struct{} - if len(nodes) > 0 { - shouldPrint = make(map[roachpb.NodeID]struct{}, len(nodes)) - for _, nodeID := range nodes { - shouldPrint[nodeID] = struct{}{} - } - } - - var sl []tuple - ms.m.Range(func(k int64, p unsafe.Pointer) bool { - nodeID := roachpb.NodeID(k) - if _, ok := shouldPrint[nodeID]; ok || len(shouldPrint) == 0 { - sl = append(sl, tuple{nodeID, (*entry)(p).SingleStorage}) - } - return true // want more - }) - sort.Slice(sl, func(i, j int) bool { - return sl[i].NodeID < sl[j].NodeID - }) - var buf bytes.Buffer - for i := range sl { - buf.WriteString(fmt.Sprintf("***** n%d *****\n", sl[i].NodeID)) - buf.WriteString(sl[i].SingleStorage.String()) - } - return buf.String() -} diff --git a/pkg/kv/kvserver/closedts/storage/storage_mem.go b/pkg/kv/kvserver/closedts/storage/storage_mem.go deleted file mode 100644 index 50caaa2c84dc..000000000000 --- a/pkg/kv/kvserver/closedts/storage/storage_mem.go +++ /dev/null @@ -1,189 +0,0 @@ -// Copyright 2018 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 storage - -import ( - "bytes" - "fmt" - "sort" - "strconv" - "time" - - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/util/syncutil" - "github.com/olekukonko/tablewriter" -) - -type memStorage struct { - mu struct { - syncutil.RWMutex - buckets []ctpb.Entry - scale time.Duration - } -} - -var _ SingleStorage = (*memStorage)(nil) - -// NewMemStorage initializes a SingleStorage backed by an in-memory slice that -// represents the given number of buckets, where the i-th bucket holds a closed -// timestamp approximately 2^i*scale in the past. -func NewMemStorage(scale time.Duration, buckets int) SingleStorage { - m := &memStorage{} - m.mu.buckets = make([]ctpb.Entry, buckets) - m.mu.scale = scale - return m -} - -func (m *memStorage) String() string { - m.mu.RLock() - defer m.mu.RUnlock() - - var buf bytes.Buffer - tw := tablewriter.NewWriter(&buf) - - header := make([]string, 1+len(m.mu.buckets)) - header[0] = "" - align := make([]int, 1+len(m.mu.buckets)) - align[0] = tablewriter.ALIGN_LEFT - - for i := range m.mu.buckets { - header[1+i] = m.mu.buckets[i].ClosedTimestamp.String() + "\nage=" + time.Duration( - m.mu.buckets[0].ClosedTimestamp.WallTime-m.mu.buckets[i].ClosedTimestamp.WallTime, - ).String() + " (target ≤" + m.bucketMaxAge(i).String() + ")\nepoch=" + fmt.Sprintf("%d", m.mu.buckets[i].Epoch) - align[1+i] = tablewriter.ALIGN_RIGHT - } - tw.SetAutoFormatHeaders(false) - tw.SetColumnAlignment(align) - tw.SetHeader(header) - tw.SetHeaderLine(true) - tw.SetRowLine(false) - tw.SetColumnSeparator(" ") - tw.SetBorder(true) - tw.SetTrimWhiteSpaceAtEOL(true) - rangeIDs := make([]roachpb.RangeID, 0, len(m.mu.buckets[0].MLAI)) - for rangeID := range m.mu.buckets[0].MLAI { - rangeIDs = append(rangeIDs, rangeID) - } - sort.Slice(rangeIDs, func(i, j int) bool { - return rangeIDs[i] < rangeIDs[j] - }) - - row := make([]string, 1+len(m.mu.buckets)) - for _, rangeID := range rangeIDs { - row[0] = "r" + strconv.FormatInt(int64(rangeID), 10) - for i, entry := range m.mu.buckets { - lai, ok := entry.MLAI[rangeID] - if ok { - row[1+i] = strconv.FormatInt(int64(lai), 10) - } else { - row[1+i] = "" - } - } - tw.Append(row) - } - - tw.Render() - return buf.String() -} - -func (m *memStorage) bucketMaxAge(index int) time.Duration { - if index == 0 { - return 0 - } - return (1 << uint(index-1)) * m.mu.scale -} - -func (m *memStorage) Add(e ctpb.Entry) { - m.mu.Lock() - defer m.mu.Unlock() - - now := e.ClosedTimestamp.WallTime - - for i := 0; i < len(m.mu.buckets); i++ { - if time.Duration(now-m.mu.buckets[i].ClosedTimestamp.WallTime) <= m.bucketMaxAge(i) { - break - } - mergedEntry := merge(m.mu.buckets[i], e) - e = m.mu.buckets[i] - m.mu.buckets[i] = mergedEntry - } -} - -func (m *memStorage) VisitAscending(f func(ctpb.Entry) (done bool)) { - m.mu.RLock() - defer m.mu.RUnlock() - - for i := len(m.mu.buckets) - 1; i >= 0; i-- { - entry := m.mu.buckets[i] - if entry.Epoch == 0 { - // Skip empty buckets. - continue - } - if f(entry) { - return - } - } -} - -func (m *memStorage) VisitDescending(f func(ctpb.Entry) (done bool)) { - m.mu.RLock() - defer m.mu.RUnlock() - - for l, i := len(m.mu.buckets), 0; i < l; i++ { - entry := m.mu.buckets[i] - // Stop once we hit an empty bucket (which implies that all further buckets - // are also empty), or once the visitor is satisfied. - if entry.Epoch == 0 || f(entry) { - return - } - } -} - -func (m *memStorage) Clear() { - m.mu.Lock() - defer m.mu.Unlock() - for i := 0; i < len(m.mu.buckets); i++ { - m.mu.buckets[i] = ctpb.Entry{} - } -} - -func merge(e, ee ctpb.Entry) ctpb.Entry { - // TODO(tschottdorf): if either of these hit, check that what we're - // returning has Full set. If we make it past, check that either of - // them has it set. The first Entry the Storage sees for an epoch must have it - // set, so the assertions should never fire. - if e.Epoch < ee.Epoch { - return ee - } else if e.Epoch > ee.Epoch { - return e - } - - // Epochs match, so we can actually update. - - // Initialize re as a deep copy of e. - re := e - re.MLAI = map[roachpb.RangeID]ctpb.LAI{} - for rangeID, mlai := range e.MLAI { - re.MLAI[rangeID] = mlai - } - // The result is full if either operand is. - re.Full = e.Full || ee.Full - // Use the larger of both timestamps with the union of the MLAIs, preferring larger - // ones on conflict. - re.ClosedTimestamp.Forward(ee.ClosedTimestamp) - for rangeID, mlai := range ee.MLAI { - if cur, found := re.MLAI[rangeID]; !found || cur < mlai { - re.MLAI[rangeID] = mlai - } - } - return re -} diff --git a/pkg/kv/kvserver/closedts/storage/storage_test.go b/pkg/kv/kvserver/closedts/storage/storage_test.go deleted file mode 100644 index a09856b9a7cd..000000000000 --- a/pkg/kv/kvserver/closedts/storage/storage_test.go +++ /dev/null @@ -1,467 +0,0 @@ -// Copyright 2018 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 storage - -import ( - "fmt" - "math/rand" - "testing" - "time" - - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/cockroachdb/cockroach/pkg/util/randutil" - "github.com/cockroachdb/cockroach/pkg/util/timeutil" - "github.com/cockroachdb/errors" - "golang.org/x/sync/errgroup" -) - -func ExampleSingleStorage() { - s := NewMemStorage(10*time.Second, 4) - fmt.Println("The empty storage renders as below:") - fmt.Println(s) - - fmt.Println("After adding the following entry:") - e1 := ctpb.Entry{ - Full: true, - ClosedTimestamp: hlc.Timestamp{WallTime: 123e9}, - MLAI: map[roachpb.RangeID]ctpb.LAI{ - 1: 1000, - 9: 2000, - }, - } - fmt.Println(e1) - s.Add(e1) - fmt.Println("the result is:") - fmt.Println(s) - fmt.Println("Note how the most recent bucket picked up the update.") - - fmt.Println("A new update comes in only two seconds later:") - e2 := ctpb.Entry{ - ClosedTimestamp: hlc.Timestamp{WallTime: 125e9}, - MLAI: map[roachpb.RangeID]ctpb.LAI{ - 1: 1001, - 7: 12, - }, - } - fmt.Println(e2) - s.Add(e2) - fmt.Println("The first bucket now contains the union of both updates.") - fmt.Println("The second bucket holds on to the previous value of the first.") - fmt.Println("The remaining buckets are unchanged. The best we could do is") - fmt.Println("give them identical copies of the second, but that's nonsense.") - fmt.Println(s) - - fmt.Println("Another update, another eight seconds later:") - e3 := ctpb.Entry{ - ClosedTimestamp: hlc.Timestamp{WallTime: 133e9}, - MLAI: map[roachpb.RangeID]ctpb.LAI{ - 9: 2020, - 1: 999, - }, - } - fmt.Println(e3) - s.Add(e3) - fmt.Println("Note how the second bucket didn't rotate, for it is not yet") - fmt.Println("older than 10s. Note also how the first bucket ignores the") - fmt.Println("downgrade for r1; these can occur in practice.") - fmt.Println(s) - - fmt.Println("Half a second later, with the next update, it will rotate:") - e4 := ctpb.Entry{ - ClosedTimestamp: hlc.Timestamp{WallTime: 133e9 + 1e9/2}, - MLAI: map[roachpb.RangeID]ctpb.LAI{ - 7: 17, - 8: 711, - }, - } - fmt.Println(e4) - s.Add(e4) - fmt.Println("Consequently we now see the third bucket fill up.") - fmt.Println(s) - - fmt.Println("Next update arrives a whopping 46.5s later (why not).") - e5 := ctpb.Entry{ - ClosedTimestamp: hlc.Timestamp{WallTime: 180e9}, - MLAI: map[roachpb.RangeID]ctpb.LAI{ - 1: 1004, - 7: 19, - 2: 929922, - }, - } - fmt.Println(e5) - s.Add(e5) - fmt.Println("The second bucket rotated, but due to the sparseness of updates,") - fmt.Println("it's still above its target age and will rotate again next time.") - fmt.Println("The same is true for the remaining buckets.") - fmt.Println(s) - - fmt.Println("Another five seconds later, another update:") - e6 := ctpb.Entry{ - ClosedTimestamp: hlc.Timestamp{WallTime: 185e9}, - MLAI: map[roachpb.RangeID]ctpb.LAI{ - 3: 1771, - }, - } - fmt.Println(e6) - s.Add(e6) - fmt.Println("All buckets rotate, but the third and fourth remain over target age.") - fmt.Println("This would resolve itself if reasonably spaced updates kept coming in.") - fmt.Println(s) - - fmt.Println("Finally, when the storage is cleared, all buckets are reset.") - s.Clear() - fmt.Println(s) - - // Output: - // The empty storage renders as below: - // +--+---------------------+----------------------+----------------------+----------------------+ - // 0,0 age=0s (target 0,0 age=0s (target 0,0 age=0s (target 0,0 age=0s (target - // ≤0s) epoch=0 ≤10s) epoch=0 ≤20s) epoch=0 ≤40s) epoch=0 - // +--+---------------------+----------------------+----------------------+----------------------+ - // +--+---------------------+----------------------+----------------------+----------------------+ - // - // After adding the following entry: - // CT: 123.000000000,0 @ Epoch 0 - // Full: true - // MLAI: r1: 1000, r9: 2000 - // - // the result is: - // +----+---------------------+------------------------+------------------------+------------------------+ - // 123.000000000,0 0,0 age=2m3s (target 0,0 age=2m3s (target 0,0 age=2m3s (target - // age=0s (target ≤0s) ≤10s) epoch=0 ≤20s) epoch=0 ≤40s) epoch=0 - // epoch=0 - // +----+---------------------+------------------------+------------------------+------------------------+ - // r1 1000 - // r9 2000 - // +----+---------------------+------------------------+------------------------+------------------------+ - // - // Note how the most recent bucket picked up the update. - // A new update comes in only two seconds later: - // CT: 125.000000000,0 @ Epoch 0 - // Full: false - // MLAI: r1: 1001, r7: 12 - // - // The first bucket now contains the union of both updates. - // The second bucket holds on to the previous value of the first. - // The remaining buckets are unchanged. The best we could do is - // give them identical copies of the second, but that's nonsense. - // +----+---------------------+----------------------+------------------------+------------------------+ - // 125.000000000,0 123.000000000,0 0,0 age=2m5s (target 0,0 age=2m5s (target - // age=0s (target ≤0s) age=2s (target ≤10s) ≤20s) epoch=0 ≤40s) epoch=0 - // epoch=0 epoch=0 - // +----+---------------------+----------------------+------------------------+------------------------+ - // r1 1001 1000 - // r7 12 - // r9 2000 2000 - // +----+---------------------+----------------------+------------------------+------------------------+ - // - // Another update, another eight seconds later: - // CT: 133.000000000,0 @ Epoch 0 - // Full: false - // MLAI: r1: 999, r9: 2020 - // - // Note how the second bucket didn't rotate, for it is not yet - // older than 10s. Note also how the first bucket ignores the - // downgrade for r1; these can occur in practice. - // +----+---------------------+-----------------------+-------------------------+-------------------------+ - // 133.000000000,0 123.000000000,0 0,0 age=2m13s (target 0,0 age=2m13s (target - // age=0s (target ≤0s) age=10s (target ≤10s) ≤20s) epoch=0 ≤40s) epoch=0 - // epoch=0 epoch=0 - // +----+---------------------+-----------------------+-------------------------+-------------------------+ - // r1 1001 1000 - // r7 12 - // r9 2020 2000 - // +----+---------------------+-----------------------+-------------------------+-------------------------+ - // - // Half a second later, with the next update, it will rotate: - // CT: 133.500000000,0 @ Epoch 0 - // Full: false - // MLAI: r7: 17, r8: 711 - // - // Consequently we now see the third bucket fill up. - // +----+---------------------+-------------------------+-------------------------+---------------------------+ - // 133.500000000,0 133.000000000,0 123.000000000,0 0,0 age=2m13.5s (target - // age=0s (target ≤0s) age=500ms (target ≤10s) age=10.5s (target ≤20s) ≤40s) epoch=0 - // epoch=0 epoch=0 epoch=0 - // +----+---------------------+-------------------------+-------------------------+---------------------------+ - // r1 1001 1001 1000 - // r7 17 12 - // r8 711 - // r9 2020 2020 2000 - // +----+---------------------+-------------------------+-------------------------+---------------------------+ - // - // Next update arrives a whopping 46.5s later (why not). - // CT: 180.000000000,0 @ Epoch 0 - // Full: false - // MLAI: r1: 1004, r2: 929922, r7: 19 - // - // The second bucket rotated, but due to the sparseness of updates, - // it's still above its target age and will rotate again next time. - // The same is true for the remaining buckets. - // +----+---------------------+-------------------------+-----------------------+-----------------------+ - // 180.000000000,0 133.500000000,0 133.000000000,0 123.000000000,0 - // age=0s (target ≤0s) age=46.5s (target ≤10s) age=47s (target ≤20s) age=57s (target ≤40s) - // epoch=0 epoch=0 epoch=0 epoch=0 - // +----+---------------------+-------------------------+-----------------------+-----------------------+ - // r1 1004 1001 1001 1000 - // r2 929922 - // r7 19 17 12 - // r8 711 711 - // r9 2020 2020 2020 2000 - // +----+---------------------+-------------------------+-----------------------+-----------------------+ - // - // Another five seconds later, another update: - // CT: 185.000000000,0 @ Epoch 0 - // Full: false - // MLAI: r3: 1771 - // - // All buckets rotate, but the third and fourth remain over target age. - // This would resolve itself if reasonably spaced updates kept coming in. - // +----+---------------------+----------------------+-------------------------+-----------------------+ - // 185.000000000,0 180.000000000,0 133.500000000,0 133.000000000,0 - // age=0s (target ≤0s) age=5s (target ≤10s) age=51.5s (target ≤20s) age=52s (target ≤40s) - // epoch=0 epoch=0 epoch=0 epoch=0 - // +----+---------------------+----------------------+-------------------------+-----------------------+ - // r1 1004 1004 1001 1001 - // r2 929922 929922 - // r3 1771 - // r7 19 19 17 12 - // r8 711 711 711 - // r9 2020 2020 2020 2020 - // +----+---------------------+----------------------+-------------------------+-----------------------+ - // - // Finally, when the storage is cleared, all buckets are reset. - // +--+---------------------+----------------------+----------------------+----------------------+ - // 0,0 age=0s (target 0,0 age=0s (target 0,0 age=0s (target 0,0 age=0s (target - // ≤0s) epoch=0 ≤10s) epoch=0 ≤20s) epoch=0 ≤40s) epoch=0 - // +--+---------------------+----------------------+----------------------+----------------------+ - // +--+---------------------+----------------------+----------------------+----------------------+ -} - -func ExampleMultiStorage_epoch() { - ms := NewMultiStorage(func() SingleStorage { - return NewMemStorage(time.Millisecond, 2) - }) - - e1 := ctpb.Entry{ - Epoch: 10, - ClosedTimestamp: hlc.Timestamp{WallTime: 1e9}, - MLAI: map[roachpb.RangeID]ctpb.LAI{ - 9: 17, - }, - } - fmt.Println("First, the following entry is added:") - fmt.Println(e1) - ms.Add(1, e1) - fmt.Println(ms) - - fmt.Println("The epoch changes. It can only increase, for we receive Entries in a fixed order.") - e2 := ctpb.Entry{ - Epoch: 11, - ClosedTimestamp: hlc.Timestamp{WallTime: 2e9}, - MLAI: map[roachpb.RangeID]ctpb.LAI{ - 9: 18, - 10: 99, - }, - } - ms.Add(1, e2) - fmt.Println(e2) - fmt.Println(ms) - - fmt.Println("If it *did* decrease, a higher level component should trigger an assertion.") - fmt.Println("The storage itself will simply ignore such updates:") - e3 := ctpb.Entry{ - Epoch: 8, - ClosedTimestamp: hlc.Timestamp{WallTime: 3e9}, - MLAI: map[roachpb.RangeID]ctpb.LAI{ - 9: 19, - 10: 199, - }, - } - fmt.Println(e3) - ms.Add(1, e3) - fmt.Println(ms) - - // Output: - // First, the following entry is added: - // CT: 1.000000000,0 @ Epoch 10 - // Full: false - // MLAI: r9: 17 - // - // ***** n1 ***** - // +----+---------------------+----------------------+ - // 1.000000000,0 0,0 age=1s (target - // age=0s (target ≤0s) ≤1ms) epoch=0 - // epoch=10 - // +----+---------------------+----------------------+ - // r9 17 - // +----+---------------------+----------------------+ - // - // The epoch changes. It can only increase, for we receive Entries in a fixed order. - // CT: 2.000000000,0 @ Epoch 11 - // Full: false - // MLAI: r9: 18, r10: 99 - // - // ***** n1 ***** - // +-----+---------------------+----------------------+ - // 2.000000000,0 1.000000000,0 - // age=0s (target ≤0s) age=1s (target ≤1ms) - // epoch=11 epoch=10 - // +-----+---------------------+----------------------+ - // r9 18 17 - // r10 99 - // +-----+---------------------+----------------------+ - // - // If it *did* decrease, a higher level component should trigger an assertion. - // The storage itself will simply ignore such updates: - // CT: 3.000000000,0 @ Epoch 8 - // Full: false - // MLAI: r9: 19, r10: 199 - // - // ***** n1 ***** - // +-----+---------------------+----------------------+ - // 2.000000000,0 2.000000000,0 - // age=0s (target ≤0s) age=0s (target ≤1ms) - // epoch=11 epoch=11 - // +-----+---------------------+----------------------+ - // r9 18 18 - // r10 99 99 - // +-----+---------------------+----------------------+ -} - -func TestZeroValueGetsStored(t *testing.T) { - defer leaktest.AfterTest(t)() - // This test ensures that a zero values MLAI is stored for an epoch especially - // after we've already stored a non-zero MLAI for a different range in the - // same epoch. See #32904. - ms := NewMultiStorage(func() SingleStorage { - return NewMemStorage(time.Millisecond, 10) - }) - e := ctpb.Entry{ - Epoch: 1, - ClosedTimestamp: hlc.Timestamp{WallTime: timeutil.Now().UnixNano()}, - MLAI: map[roachpb.RangeID]ctpb.LAI{1: 1}, - } - ms.Add(1, e) - e.ClosedTimestamp.WallTime++ - r := roachpb.RangeID(2) - e.MLAI = map[roachpb.RangeID]ctpb.LAI{r: 0} - ms.Add(1, e) - var seen bool - ms.VisitDescending(1, func(e ctpb.Entry) (done bool) { - for rr, mlai := range e.MLAI { - if rr == r && mlai == 0 { - seen = true - return true - } - } - return false - }) - if !seen { - t.Fatalf("Failed to see added zero value MLAI for range %v", r) - } -} - -// TestConcurrent runs a very basic sanity check against a Storage, verifiying -// that the bucketed Entries don't regress in obvious ways. -func TestConcurrent(t *testing.T) { - defer leaktest.AfterTest(t)() - - ms := NewMultiStorage(func() SingleStorage { - return NewMemStorage(time.Millisecond, 10) - }) - - var g errgroup.Group - - const ( - iters = 10 - numNodes = roachpb.NodeID(2) - numRanges = roachpb.RangeID(3) - numReadersPerNode = 3 - numWritersPerNode = 3 - ) - - // concurrently add and read from storage - // after add: needs to be visible to future read - // read ts never regresses - globalRand, seed := randutil.NewPseudoRand() - t.Log("seed is", seed) - - for i := 0; i < numWritersPerNode; i++ { - for nodeID := roachpb.NodeID(1); nodeID <= numNodes; nodeID++ { - nodeID := nodeID // goroutine-local copy - for i := 0; i < iters; i++ { - r := rand.New(rand.NewSource(globalRand.Int63())) - m := make(map[roachpb.RangeID]ctpb.LAI) - for rangeID := roachpb.RangeID(1); rangeID < numRanges; rangeID++ { - if r.Intn(int(numRanges)) == 0 { - continue - } - m[rangeID] = ctpb.LAI(rand.Intn(100)) - } - ct := hlc.Timestamp{WallTime: r.Int63n(100), Logical: r.Int31n(10)} - epo := ctpb.Epoch(r.Int63n(100)) - g.Go(func() error { - <-time.After(time.Duration(rand.Intn(1e7))) - ms.Add(nodeID, ctpb.Entry{ - Epoch: epo, - ClosedTimestamp: ct, - MLAI: m, - }) - return nil - }) - } - } - } - - for i := 0; i < numReadersPerNode; i++ { - for nodeID := roachpb.NodeID(1); nodeID <= numNodes; nodeID++ { - nodeID := nodeID - g.Go(func() error { - epo := ctpb.Epoch(-1) - var ct hlc.Timestamp - var mlai map[roachpb.RangeID]ctpb.LAI - var err error - var n int - ms.VisitDescending(nodeID, func(e ctpb.Entry) bool { - n++ - if n > 1 && e.Epoch > epo { - err = errors.Errorf("epoch regressed from %d to %d", epo, e.Epoch) - return true // done - } - if n > 1 && ct.Less(e.ClosedTimestamp) { - err = errors.Errorf("closed timestamp regressed from %s to %s", ct, e.ClosedTimestamp) - return true // done - } - for rangeID := roachpb.RangeID(1); rangeID <= numRanges; rangeID++ { - if l := mlai[rangeID]; l < e.MLAI[rangeID] && n > 1 { - err = errors.Errorf("MLAI for r%d regressed: %+v to %+v", rangeID, mlai, e.MLAI) - return true // done - } - } - - epo = e.Epoch - ct = e.ClosedTimestamp - mlai = e.MLAI - return false // not done - }) - return err - }) - } - } - - if err := g.Wait(); err != nil { - t.Fatal(err) - } -} diff --git a/pkg/kv/kvserver/closedts/transport/BUILD.bazel b/pkg/kv/kvserver/closedts/transport/BUILD.bazel index ba99fb585ad0..e69de29bb2d1 100644 --- a/pkg/kv/kvserver/closedts/transport/BUILD.bazel +++ b/pkg/kv/kvserver/closedts/transport/BUILD.bazel @@ -1,49 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") - -go_library( - name = "transport", - srcs = [ - "clients.go", - "server.go", - ], - importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/transport", - visibility = ["//visibility:public"], - deps = [ - "//pkg/clusterversion", - "//pkg/kv/kvserver/closedts", - "//pkg/kv/kvserver/closedts/ctpb", - "//pkg/roachpb:with-mocks", - "//pkg/settings/cluster", - "//pkg/util/log", - "//pkg/util/stop", - "//pkg/util/syncutil", - "//pkg/util/timeutil", - "@com_github_cockroachdb_errors//:errors", - "@com_github_cockroachdb_logtags//:logtags", - ], -) - -go_test( - name = "transport_test", - size = "small", - srcs = [ - "transport_test.go", - "transport_util_test.go", - ], - deps = [ - ":transport", - "//pkg/kv/kvserver/closedts", - "//pkg/kv/kvserver/closedts/ctpb", - "//pkg/kv/kvserver/closedts/transport/testutils", - "//pkg/roachpb:with-mocks", - "//pkg/settings/cluster", - "//pkg/testutils", - "//pkg/testutils/skip", - "//pkg/util/hlc", - "//pkg/util/leaktest", - "//pkg/util/stop", - "//pkg/util/syncutil", - "@com_github_cockroachdb_errors//:errors", - "@com_github_kr_pretty//:pretty", - ], -) diff --git a/pkg/kv/kvserver/closedts/transport/clients.go b/pkg/kv/kvserver/closedts/transport/clients.go deleted file mode 100644 index 3d8704b04459..000000000000 --- a/pkg/kv/kvserver/closedts/transport/clients.go +++ /dev/null @@ -1,179 +0,0 @@ -// Copyright 2018 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 transport - -import ( - "context" - "unsafe" - - "github.com/cockroachdb/cockroach/pkg/clusterversion" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/cockroach/pkg/util/stop" - "github.com/cockroachdb/cockroach/pkg/util/syncutil" - "github.com/cockroachdb/logtags" -) - -// Config holds the information necessary to create a client registry. -type Config struct { - Settings *cluster.Settings - Stopper *stop.Stopper - NodeID roachpb.NodeID - Dialer closedts.Dialer - Sink closedts.Notifyee -} - -// Clients manages clients receiving closed timestamp updates from -// peer nodes, along with facilities to request information about certain -// ranges. Received updates are relayed to a provided Notifyee. -type Clients struct { - cfg Config - - // TODO(tschottdorf): remove unused clients. Perhaps expiring them after, - // say, 24h is enough? There is no interruption when doing so; the only - // price is that a full update is sent, but that is pretty cheap too. - clients syncutil.IntMap -} - -var _ closedts.ClientRegistry = (*Clients)(nil) - -// NewClients sets up a client registry. -func NewClients(cfg Config) *Clients { - return &Clients{cfg: cfg} -} - -type client struct { - mu struct { - syncutil.Mutex - requested map[roachpb.RangeID]struct{} // never nil - } -} - -// Request is called when serving a follower read has failed due to missing or -// insufficient information. By calling this method, the caller gives the -// instruction to connect to the given node (if it hasn't already) and ask it to -// send (or re-send) up-to-date information about the specified range. Having -// done so, the information should soon thereafter be available to the Sink and -// from there, further follower read attempts. Does not block. -func (pr *Clients) Request(nodeID roachpb.NodeID, rangeID roachpb.RangeID) { - // If the new closed timestamps is enabled, this old one is disabled. - if pr.cfg.Settings.Version.IsActive(context.TODO(), clusterversion.ClosedTimestampsRaftTransport) { - return - } - - if nodeID == pr.cfg.NodeID { - return - } - if cl := pr.getOrCreateClient(nodeID); cl != nil { - cl.mu.Lock() - cl.mu.requested[rangeID] = struct{}{} - cl.mu.Unlock() - } -} - -// EnsureClient makes sure that updates from the given nodes are pulled in, if -// they aren't already. This call does not block (and is cheap). -func (pr *Clients) EnsureClient(nodeID roachpb.NodeID) { - if nodeID == pr.cfg.NodeID { - return - } - pr.getOrCreateClient(nodeID) -} - -func (pr *Clients) getOrCreateClient(nodeID roachpb.NodeID) *client { - // Fast path to check for existing client without an allocation. - p, found := pr.clients.Load(int64(nodeID)) - cl := (*client)(p) - if found { - return cl - } - if !pr.cfg.Dialer.Ready(nodeID) { - return nil - } - - if nodeID == pr.cfg.NodeID { - panic("must not create client to local node") - } - - // Slow path: create the client. Another inserter might race us to it. - - // This allocates, so only do it when necessary. - ctx := logtags.AddTag(context.Background(), "ct-client", "") - - cl = &client{} - cl.mu.requested = map[roachpb.RangeID]struct{}{} - - if firstClient, loaded := pr.clients.LoadOrStore(int64(nodeID), unsafe.Pointer(cl)); loaded { - return (*client)(firstClient) - } - - // If our client made it into the map, start it. The point in inserting - // before starting is to be able to collect RangeIDs immediately while never - // blocking callers. - if err := pr.cfg.Stopper.RunAsyncTask(ctx, "ct-client", func(ctx context.Context) { - defer pr.clients.Delete(int64(nodeID)) - - c, err := pr.cfg.Dialer.Dial(ctx, nodeID) - if err != nil { - if log.V(1) { - log.Warningf(ctx, "error opening closed timestamp stream to n%d: %+v", nodeID, err) - } - return - } - defer func() { - _ = c.CloseSend() - }() - - ctx = c.Context() - - ch := pr.cfg.Sink.Notify(nodeID) - defer close(ch) - - reaction := &ctpb.Reaction{} - for { - if err := c.Send(reaction); err != nil { - return - } - entry, err := c.Recv() - if err != nil { - return - } - - select { - case ch <- *entry: - case <-ctx.Done(): - return - case <-pr.cfg.Stopper.ShouldQuiesce(): - return - } - - var requested map[roachpb.RangeID]struct{} - cl.mu.Lock() - requested, cl.mu.requested = cl.mu.requested, map[roachpb.RangeID]struct{}{} - cl.mu.Unlock() - - slice := make([]roachpb.RangeID, 0, len(requested)) - for rangeID := range requested { - slice = append(slice, rangeID) - } - reaction = &ctpb.Reaction{ - Requested: slice, - } - } - }); err != nil { - pr.clients.Delete(int64(nodeID)) - } - - return cl -} diff --git a/pkg/kv/kvserver/closedts/transport/server.go b/pkg/kv/kvserver/closedts/transport/server.go deleted file mode 100644 index dc492e6ff65d..000000000000 --- a/pkg/kv/kvserver/closedts/transport/server.go +++ /dev/null @@ -1,100 +0,0 @@ -// Copyright 2018 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 transport - -import ( - "context" - "time" - - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" - "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/cockroach/pkg/util/stop" - "github.com/cockroachdb/cockroach/pkg/util/timeutil" - "github.com/cockroachdb/errors" -) - -// Server handles incoming closed timestamp update stream requests. -type Server struct { - stopper *stop.Stopper - p closedts.Producer - refresh closedts.RefreshFn -} - -// NewServer sets up a Server which relays information from the given producer -// to incoming clients. -func NewServer(stopper *stop.Stopper, p closedts.Producer, refresh closedts.RefreshFn) *Server { - return &Server{ - stopper: stopper, - p: p, - refresh: refresh, - } -} - -var _ ctpb.Server = (*Server)(nil) - -// Get handles incoming client connections. -func (s *Server) Get(client ctpb.InboundClient) error { - // TODO(tschottdorf): the InboundClient API isn't great since it - // is blocking. How can we eagerly terminate these connections when - // the server shuts down? I think we need to inject a cancellation - // into the context, but grpc hands that to us. - // This problem has likely been solved somewhere in our codebase. - ctx := client.Context() - ch := make(chan ctpb.Entry, 10) - - if log.V(1) { - log.Infof(ctx, "closed timestamp server serving new inbound client connection") - } - - // TODO(tschottdorf): make this, say, 2*closedts.CloseFraction*closedts.TargetInterval. - const closedTimestampNoUpdateWarnThreshold = 10 * time.Second - t := timeutil.NewTimer() - - if err := s.stopper.RunAsyncTask(ctx, "closedts-subscription", func(ctx context.Context) { - s.p.Subscribe(ctx, ch) - }); err != nil { - return err - } - for { - reaction, err := client.Recv() - if err != nil { - return err - } - - if len(reaction.Requested) != 0 { - s.refresh(reaction.Requested...) - } - - t.Reset(closedTimestampNoUpdateWarnThreshold) - var entry ctpb.Entry - var ok bool - select { - case <-ctx.Done(): - return ctx.Err() - case <-s.stopper.ShouldQuiesce(): - return errors.New("node is draining") - case entry, ok = <-ch: - if !ok { - return errors.New("subscription dropped unexpectedly") - } - case <-t.C: - t.Read = true - // Send an empty entry to the client, which can use that to warn - // about the absence of heartbeats. We don't log here since it - // would log a message per incoming stream, which makes little - // sense. It's the producer's job to warn on this node. - } - if err := client.Send(&entry); err != nil { - return err - } - } -} diff --git a/pkg/kv/kvserver/closedts/transport/testutils/BUILD.bazel b/pkg/kv/kvserver/closedts/transport/testutils/BUILD.bazel index 3905dbecdfcd..e69de29bb2d1 100644 --- a/pkg/kv/kvserver/closedts/transport/testutils/BUILD.bazel +++ b/pkg/kv/kvserver/closedts/transport/testutils/BUILD.bazel @@ -1,14 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") - -go_library( - name = "testutils", - srcs = ["chan_dialer.go"], - importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/transport/testutils", - visibility = ["//visibility:public"], - deps = [ - "//pkg/kv/kvserver/closedts/ctpb", - "//pkg/roachpb:with-mocks", - "//pkg/util/stop", - "//pkg/util/syncutil", - ], -) diff --git a/pkg/kv/kvserver/closedts/transport/testutils/chan_dialer.go b/pkg/kv/kvserver/closedts/transport/testutils/chan_dialer.go deleted file mode 100644 index 303d9023e8fb..000000000000 --- a/pkg/kv/kvserver/closedts/transport/testutils/chan_dialer.go +++ /dev/null @@ -1,146 +0,0 @@ -// Copyright 2018 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 testutils - -import ( - "context" - "io" - - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/util/stop" - "github.com/cockroachdb/cockroach/pkg/util/syncutil" -) - -// ChanDialer is an implementation of closedts.Dialer that connects clients -// directly via a channel to a Server. -type ChanDialer struct { - stopper *stop.Stopper - server ctpb.Server - - mu struct { - syncutil.Mutex - transcripts map[roachpb.NodeID][]interface{} - } -} - -// NewChanDialer sets up a ChanDialer. -func NewChanDialer(stopper *stop.Stopper, server ctpb.Server) *ChanDialer { - d := &ChanDialer{ - stopper: stopper, - server: server, - } - d.mu.transcripts = make(map[roachpb.NodeID][]interface{}) - return d -} - -// Transcript returns a slice of messages sent over the "wire". -func (d *ChanDialer) Transcript(nodeID roachpb.NodeID) []interface{} { - d.mu.Lock() - defer d.mu.Unlock() - return append([]interface{}(nil), d.mu.transcripts[nodeID]...) -} - -// Dial implements closedts.Dialer. -func (d *ChanDialer) Dial(ctx context.Context, nodeID roachpb.NodeID) (ctpb.Client, error) { - c := &client{ - ctx: ctx, - send: make(chan *ctpb.Reaction), - recv: make(chan *ctpb.Entry), - stopper: d.stopper, - observe: func(msg interface{}) { - d.mu.Lock() - if d.mu.transcripts == nil { - d.mu.transcripts = map[roachpb.NodeID][]interface{}{} - } - d.mu.transcripts[nodeID] = append(d.mu.transcripts[nodeID], msg) - d.mu.Unlock() - }, - } - - if err := d.stopper.RunAsyncTask(ctx, "closedts-dial", func(ctx context.Context) { - _ = d.server.Get((*incomingClient)(c)) - }); err != nil { - return nil, err - } - return c, nil - -} - -// Ready implements closedts.Dialer by always returning true. -func (d *ChanDialer) Ready(nodeID roachpb.NodeID) bool { - return true -} - -type client struct { - ctx context.Context - stopper *stop.Stopper - send chan *ctpb.Reaction - recv chan *ctpb.Entry - - observe func(interface{}) -} - -func (c *client) Send(msg *ctpb.Reaction) error { - select { - case <-c.stopper.ShouldQuiesce(): - return io.EOF - case c.send <- msg: - c.observe(msg) - return nil - } -} - -func (c *client) Recv() (*ctpb.Entry, error) { - select { - case <-c.stopper.ShouldQuiesce(): - return nil, io.EOF - case msg := <-c.recv: - c.observe(msg) - return msg, nil - } -} - -func (c *client) CloseSend() error { - close(c.send) - return nil -} - -func (c *client) Context() context.Context { - return c.ctx -} - -type incomingClient client - -func (c *incomingClient) Send(msg *ctpb.Entry) error { - select { - case <-c.stopper.ShouldQuiesce(): - return io.EOF - case c.recv <- msg: - return nil - } -} - -func (c *incomingClient) Recv() (*ctpb.Reaction, error) { - select { - case <-c.stopper.ShouldQuiesce(): - return nil, io.EOF - case msg, ok := <-c.send: - if !ok { - return nil, io.EOF - } - return msg, nil - } -} - -func (c *incomingClient) Context() context.Context { - return c.ctx -} diff --git a/pkg/kv/kvserver/closedts/transport/transport_test.go b/pkg/kv/kvserver/closedts/transport/transport_test.go deleted file mode 100644 index cf84159320d5..000000000000 --- a/pkg/kv/kvserver/closedts/transport/transport_test.go +++ /dev/null @@ -1,199 +0,0 @@ -// Copyright 2018 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 transport_test - -import ( - "context" - "fmt" - "strings" - "testing" - - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/transport" - transporttestutils "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/transport/testutils" - "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/skip" - "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/cockroachdb/cockroach/pkg/util/stop" - "github.com/cockroachdb/errors" - "github.com/kr/pretty" -) - -// NewTestContainer sets up an environment suitable for black box testing the -// transport subsystem. The returned test container contains most notably a -// Clients and Server set up to communicate to each other via a Dialer (which -// keeps a transcript that can be verified). -func NewTestContainer() *TestContainer { - stopper := stop.NewStopper() - - st := cluster.MakeTestingClusterSettings() - p := &TestProducer{} - sink := newTestNotifyee(stopper) - refreshed := &RefreshTracker{} - s := transport.NewServer(stopper, p, refreshed.Add) - dialer := transporttestutils.NewChanDialer(stopper, s) - c := transport.NewClients(transport.Config{ - NodeID: roachpb.NodeID(12345), - Settings: st, - Stopper: stopper, - Dialer: dialer, - Sink: sink, - }) - return &TestContainer{ - Settings: st, - Stopper: stopper, - Producer: p, - Notifyee: sink, - Refreshed: refreshed, - Server: s, - Dialer: dialer, - Clients: c, - } -} - -func assertNumSubscribers(t *testing.T, p *TestProducer, exp int) { - testutils.SucceedsSoon(t, func() error { - n := p.numSubscriptions() - if n > exp { - t.Fatalf("expected a single subscription, got %d", n) - } - if n < exp { - return errors.New("waiting for subscription") - } - return nil - }) -} - -func TestTransportConnectOnRequest(t *testing.T) { - defer leaktest.AfterTest(t)() - skip.WithIssue(t, closedts.IssueTrackingRemovalOfOldClosedTimestampsCode) - - container := NewTestContainer() - defer container.Stopper.Stop(context.Background()) - - const ( - nodeID = 1 - rangeID = 13 - ) - - // Requesting an update for a Range implies a connection attempt. - container.Clients.Request(nodeID, rangeID) - - // Find the connection (via its subscription to receive new Entries). - assertNumSubscribers(t, container.Producer, 1) - - // Verify that the client soon asks the server for an update for this range. - testutils.SucceedsSoon(t, func() error { - act := container.Refreshed.Get() - exp := []roachpb.RangeID{rangeID} - - if diff := pretty.Diff(act, exp); len(diff) != 0 { - // We have to kick the tires a little bit. The client can only send - // the request as the reaction to an Entry. - container.Producer.sendAll(ctpb.Entry{}) - return errors.Errorf("diff(act, exp): %s", strings.Join(diff, "\n")) - } - return nil - }) -} - -func TestTransportClientReceivesEntries(t *testing.T) { - defer leaktest.AfterTest(t)() - skip.WithIssue(t, closedts.IssueTrackingRemovalOfOldClosedTimestampsCode) - - container := NewTestContainer() - defer container.Stopper.Stop(context.Background()) - - const nodeID = 7 - - // Manual reconnections don't spawn new clients. - container.Clients.EnsureClient(nodeID) - container.Clients.EnsureClient(nodeID) - container.Clients.EnsureClient(nodeID) - assertNumSubscribers(t, container.Producer, 1) - - // But connecting to other nodes does (only once). - for i := 0; i < 7; i++ { - container.Clients.EnsureClient(nodeID + 1) - container.Clients.EnsureClient(nodeID + 2) - container.Clients.Request(nodeID+3, roachpb.RangeID(7)) - } - assertNumSubscribers(t, container.Producer, 4) - - // Our initial client doesn't do anything except say "hello" via - // a Reaction. - testutils.SucceedsSoon(t, func() error { - expectedTranscript := []interface{}{ - &ctpb.Reaction{}, - } - return checkTranscript(t, container.Dialer.Transcript(nodeID), expectedTranscript) - }) - - // Now the producer (to which the server should maintain a subscription for this client, and - // notifications from which it should relay) emits an Entry. - e1 := ctpb.Entry{ClosedTimestamp: hlc.Timestamp{WallTime: 1e9}, Epoch: 12, MLAI: map[roachpb.RangeID]ctpb.LAI{12: 7}} - container.Producer.sendAll(e1) - - // The client should see this entry soon thereafter. it responds with an empty - // Reaction (since we haven't Request()ed anything). - testutils.SucceedsSoon(t, func() error { - expectedTranscript := []interface{}{ - &ctpb.Reaction{}, - &e1, - &ctpb.Reaction{}, - } - return checkTranscript(t, container.Dialer.Transcript(nodeID), expectedTranscript) - }) - - // And again, but only after Request() is called (which should be reflected in the transcript). - const rangeID = 7 - container.Clients.Request(nodeID, rangeID) - e2 := ctpb.Entry{ClosedTimestamp: hlc.Timestamp{WallTime: 2e9}, Epoch: 13, MLAI: map[roachpb.RangeID]ctpb.LAI{13: 8}} - container.Producer.sendAll(e2) - testutils.SucceedsSoon(t, func() error { - expectedTranscript := []interface{}{ - &ctpb.Reaction{}, - &e1, - &ctpb.Reaction{}, - &e2, - &ctpb.Reaction{Requested: []roachpb.RangeID{rangeID}}, - } - return checkTranscript(t, container.Dialer.Transcript(nodeID), expectedTranscript) - }) - -} - -func checkTranscript(t *testing.T, actI, expI []interface{}) error { - t.Helper() - var act, exp []string - for _, i := range actI { - act = append(act, strings.TrimSpace(fmt.Sprintf("%v", i))) - } - for _, i := range expI { - exp = append(exp, strings.TrimSpace(fmt.Sprintf("%v", i))) - } - - diffErr := errors.Errorf("actual:\n%s\nexpected:\n%s", strings.Join(act, "\n"), strings.Join(exp, "\n")) - if len(act) > len(exp) { - t.Fatal(errors.Wrap(diffErr, "actual transcript longer than expected")) - } - if len(act) < len(exp) { - return errors.Wrap(diffErr, "waiting for more") - } - if diff := pretty.Diff(actI, expI); len(diff) != 0 { - t.Fatal(errors.Wrapf(diffErr, "diff:\n%v\n", strings.Join(diff, "\n"))) - } - return nil -} diff --git a/pkg/kv/kvserver/closedts/transport/transport_util_test.go b/pkg/kv/kvserver/closedts/transport/transport_util_test.go deleted file mode 100644 index b964a168337f..000000000000 --- a/pkg/kv/kvserver/closedts/transport/transport_util_test.go +++ /dev/null @@ -1,104 +0,0 @@ -// Copyright 2018 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 transport_test - -import ( - "context" - - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/transport" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/transport/testutils" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/util/stop" - "github.com/cockroachdb/cockroach/pkg/util/syncutil" -) - -type TestContainer struct { - Settings *cluster.Settings - Stopper *stop.Stopper - Producer *TestProducer - Notifyee *TestNotifyee - Refreshed *RefreshTracker - Server *transport.Server - Dialer *testutils.ChanDialer - Clients *transport.Clients -} - -type TestProducer struct { - syncutil.Mutex - chs []chan<- ctpb.Entry -} - -func (tp *TestProducer) Subscribe(ctx context.Context, ch chan<- ctpb.Entry) { - tp.Lock() - tp.chs = append(tp.chs, ch) - tp.Unlock() -} - -func (tp *TestProducer) numSubscriptions() int { - tp.Lock() - defer tp.Unlock() - return len(tp.chs) -} - -func (tp *TestProducer) sendAll(entry ctpb.Entry) { - tp.Lock() - for _, ch := range tp.chs { - ch <- entry - } - tp.Unlock() -} - -type TestNotifyee struct { - stopper *stop.Stopper - mu struct { - syncutil.Mutex - entries map[roachpb.NodeID][]ctpb.Entry - } -} - -func newTestNotifyee(stopper *stop.Stopper) *TestNotifyee { - tn := &TestNotifyee{ - stopper: stopper, - } - tn.mu.entries = make(map[roachpb.NodeID][]ctpb.Entry) - return tn -} - -func (tn *TestNotifyee) Notify(nodeID roachpb.NodeID) chan<- ctpb.Entry { - ch := make(chan ctpb.Entry) - _ = tn.stopper.RunAsyncTask(context.Background(), "test-notify", func(ctx context.Context) { - for entry := range ch { - tn.mu.Lock() - tn.mu.entries[nodeID] = append(tn.mu.entries[nodeID], entry) - tn.mu.Unlock() - } - }) - return ch -} - -type RefreshTracker struct { - syncutil.Mutex - rangeIDs []roachpb.RangeID -} - -func (r *RefreshTracker) Get() []roachpb.RangeID { - r.Lock() - defer r.Unlock() - return append([]roachpb.RangeID(nil), r.rangeIDs...) -} - -func (r *RefreshTracker) Add(rangeIDs ...roachpb.RangeID) { - r.Lock() - r.rangeIDs = append(r.rangeIDs, rangeIDs...) - r.Unlock() -} diff --git a/pkg/kv/kvserver/helpers_test.go b/pkg/kv/kvserver/helpers_test.go index 598a507bbfd3..758034d95253 100644 --- a/pkg/kv/kvserver/helpers_test.go +++ b/pkg/kv/kvserver/helpers_test.go @@ -205,18 +205,6 @@ func (s *Store) RaftSchedulerPriorityID() roachpb.RangeID { return s.scheduler.PriorityID() } -// ClearClosedTimestampStorage clears the closed timestamp storage of all -// knowledge about closed timestamps. -func (s *Store) ClearClosedTimestampStorage() { - s.cfg.ClosedTimestamp.Storage.Clear() -} - -// RequestClosedTimestamp instructs the closed timestamp client to request the -// relevant node to publish its MLAI for the provided range. -func (s *Store) RequestClosedTimestamp(nodeID roachpb.NodeID, rangeID roachpb.RangeID) { - s.cfg.ClosedTimestamp.Clients.Request(nodeID, rangeID) -} - func NewTestStorePool(cfg StoreConfig) *StorePool { TimeUntilStoreDead.Override(context.Background(), &cfg.Settings.SV, TestTimeUntilStoreDeadOff) return NewStorePool( @@ -266,7 +254,7 @@ func (r *Replica) LastAssignedLeaseIndex() uint64 { } // MaxClosed returns the maximum closed timestamp known to the Replica. -func (r *Replica) MaxClosed(ctx context.Context) (_ hlc.Timestamp, ok bool) { +func (r *Replica) MaxClosed(ctx context.Context) hlc.Timestamp { return r.maxClosed(ctx) } diff --git a/pkg/kv/kvserver/kvserverpb/state.pb.go b/pkg/kv/kvserver/kvserverpb/state.pb.go index f6e5f8e91350..ce7660630fac 100644 --- a/pkg/kv/kvserver/kvserverpb/state.pb.go +++ b/pkg/kv/kvserver/kvserverpb/state.pb.go @@ -6,7 +6,6 @@ package kvserverpb import ( fmt "fmt" github_com_cockroachdb_cockroach_pkg_kv_kvserver_closedts_ctpb "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" - github_com_cockroachdb_cockroach_pkg_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" enginepb "github.com/cockroachdb/cockroach/pkg/storage/enginepb" hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -164,18 +163,11 @@ type RangeInfo struct { ProposalQuotaReleaseQueue []int64 `protobuf:"varint,15,rep,packed,name=proposal_quota_release_queue,json=proposalQuotaReleaseQueue,proto3" json:"proposal_quota_release_queue,omitempty"` // The max size the range can grow to before it will be split. RangeMaxBytes int64 `protobuf:"varint,8,opt,name=range_max_bytes,json=rangeMaxBytes,proto3" json:"range_max_bytes,omitempty"` - // The highest closed timestamp known to have data for this replica, taken - // across the data received from all nodes. This does not reflect whether - // the replica can use this closed timestamp (it may, for example, not have - // caught up sufficiently to do so). - NewestClosedTimestamp RangeInfo_CTEntry `protobuf:"bytes,11,opt,name=newest_closed_timestamp,json=newestClosedTimestamp,proto3" json:"newest_closed_timestamp"` // The closed timestamp active on the replica when the info was generated. // This is the actual timestamp at or below which requests can be served from - // this replica at this moment (assuming it is not the leaseholder). This takes - // into account the lease start time, the current lease applied index, and the - // closed timestamp information received from other nodes, among other things. - // In practice, this should not usually trail newest_closed_timestamp except - // for a short moment after newest_closed_timestamp gets updated. + // this replica at this moment (assuming it is not the leaseholder). This + // takes into account the current lease applied index, and the closed + // timestamp information received from other nodes, among other things. ActiveClosedTimestamp hlc.Timestamp `protobuf:"bytes,12,opt,name=active_closed_timestamp,json=activeClosedTimestamp,proto3" json:"active_closed_timestamp"` // The number of Rangefeed registrations attached to the Replica. RangefeedRegistrations int64 `protobuf:"varint,13,opt,name=rangefeed_registrations,json=rangefeedRegistrations,proto3" json:"rangefeed_registrations,omitempty"` @@ -216,42 +208,6 @@ func (m *RangeInfo) XXX_DiscardUnknown() { var xxx_messageInfo_RangeInfo proto.InternalMessageInfo -type RangeInfo_CTEntry struct { - NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` - ClosedTimestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=closed_timestamp,json=closedTimestamp,proto3" json:"closed_timestamp"` - MLAI int64 `protobuf:"varint,3,opt,name=mlai,proto3" json:"mlai,omitempty"` - Epoch int64 `protobuf:"varint,4,opt,name=epoch,proto3" json:"epoch,omitempty"` -} - -func (m *RangeInfo_CTEntry) Reset() { *m = RangeInfo_CTEntry{} } -func (m *RangeInfo_CTEntry) String() string { return proto.CompactTextString(m) } -func (*RangeInfo_CTEntry) ProtoMessage() {} -func (*RangeInfo_CTEntry) Descriptor() ([]byte, []int) { - return fileDescriptor_cc107fbd3ff296cb, []int{1, 0} -} -func (m *RangeInfo_CTEntry) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *RangeInfo_CTEntry) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - b = b[:cap(b)] - n, err := m.MarshalToSizedBuffer(b) - if err != nil { - return nil, err - } - return b[:n], nil -} -func (m *RangeInfo_CTEntry) XXX_Merge(src proto.Message) { - xxx_messageInfo_RangeInfo_CTEntry.Merge(m, src) -} -func (m *RangeInfo_CTEntry) XXX_Size() int { - return m.Size() -} -func (m *RangeInfo_CTEntry) XXX_DiscardUnknown() { - xxx_messageInfo_RangeInfo_CTEntry.DiscardUnknown(m) -} - -var xxx_messageInfo_RangeInfo_CTEntry proto.InternalMessageInfo - // RangeSideTransportInfo describes a range's closed timestamp info communicated // through the side-transport. // @@ -296,7 +252,6 @@ var xxx_messageInfo_RangeSideTransportInfo proto.InternalMessageInfo func init() { proto.RegisterType((*ReplicaState)(nil), "cockroach.kv.kvserver.storagepb.ReplicaState") proto.RegisterType((*RangeInfo)(nil), "cockroach.kv.kvserver.storagepb.RangeInfo") - proto.RegisterType((*RangeInfo_CTEntry)(nil), "cockroach.kv.kvserver.storagepb.RangeInfo.CTEntry") proto.RegisterType((*RangeSideTransportInfo)(nil), "cockroach.kv.kvserver.storagepb.RangeSideTransportInfo") } @@ -305,83 +260,76 @@ func init() { } var fileDescriptor_cc107fbd3ff296cb = []byte{ - // 1210 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x56, 0xcf, 0x6e, 0xdb, 0xc6, - 0x13, 0x36, 0x23, 0xca, 0x96, 0x57, 0xfe, 0xa3, 0x6c, 0x62, 0x9b, 0x71, 0x12, 0x51, 0xf0, 0x0f, - 0xbf, 0xc2, 0x05, 0x52, 0x12, 0x4d, 0xda, 0x06, 0xfd, 0x03, 0x14, 0x96, 0x5d, 0x14, 0x72, 0xed, - 0x20, 0xd9, 0x08, 0x29, 0x90, 0x1e, 0x88, 0x15, 0xb9, 0xa6, 0x16, 0xa6, 0xb8, 0xcc, 0xee, 0x4a, - 0xb5, 0xf3, 0x14, 0x45, 0x9f, 0xa0, 0x87, 0x3e, 0x40, 0x1f, 0x23, 0x40, 0x2f, 0x39, 0xe6, 0x24, - 0xb4, 0xca, 0xa5, 0xb7, 0xde, 0x73, 0x2a, 0x76, 0x97, 0x94, 0x25, 0xd9, 0x40, 0x54, 0xa0, 0xb7, - 0xd5, 0xcc, 0x37, 0xb3, 0x33, 0xdf, 0x7c, 0x3b, 0x22, 0xd8, 0x39, 0x1d, 0xf8, 0xa7, 0x03, 0x41, - 0xf8, 0x80, 0xf0, 0xf1, 0x21, 0xeb, 0xf8, 0x42, 0x62, 0x49, 0xbc, 0x8c, 0x33, 0xc9, 0xa0, 0x1b, - 0xb2, 0xf0, 0x94, 0x33, 0x1c, 0x76, 0xbd, 0xd3, 0x81, 0x57, 0x80, 0x3c, 0x21, 0x19, 0xc7, 0x31, - 0xc9, 0x3a, 0xdb, 0xb7, 0xf3, 0xa3, 0x4f, 0xd2, 0x98, 0xa6, 0x24, 0xeb, 0xf8, 0xbd, 0x41, 0x18, - 0x9a, 0xe8, 0xed, 0xdb, 0x3a, 0x32, 0xeb, 0xf8, 0x34, 0x95, 0x84, 0xa7, 0x38, 0x09, 0x38, 0x3e, - 0x91, 0xb9, 0x73, 0xb3, 0x70, 0xf6, 0x88, 0xc4, 0x11, 0x96, 0x38, 0xb7, 0xc3, 0xc2, 0x3e, 0x61, - 0x73, 0xfa, 0x92, 0x26, 0x7e, 0x37, 0x09, 0x7d, 0x49, 0x7b, 0x44, 0x48, 0xdc, 0xcb, 0x72, 0xcf, - 0xcd, 0x98, 0xc5, 0x4c, 0x1f, 0x7d, 0x75, 0x32, 0xd6, 0x9d, 0x9f, 0xcb, 0x60, 0x05, 0x91, 0x2c, - 0xa1, 0x21, 0x7e, 0xaa, 0xba, 0x81, 0xf7, 0x00, 0x54, 0x57, 0x07, 0x38, 0xcb, 0x12, 0x4a, 0xa2, - 0x80, 0xa6, 0x11, 0x39, 0x73, 0xac, 0x86, 0xb5, 0x6b, 0xa3, 0x9a, 0xf2, 0xec, 0x19, 0x47, 0x4b, - 0xd9, 0xa1, 0x07, 0x6e, 0x24, 0x04, 0x0b, 0x32, 0x03, 0xbf, 0xa6, 0xe1, 0xd7, 0xb5, 0x6b, 0x0a, - 0xff, 0x19, 0xb0, 0x23, 0x22, 0x42, 0xa7, 0xd4, 0xb0, 0x76, 0xab, 0xf7, 0x77, 0xbc, 0x0b, 0xd2, - 0xf2, 0x5e, 0x3c, 0x84, 0xd3, 0x98, 0x1c, 0x10, 0x11, 0x72, 0x9a, 0x49, 0xc6, 0x91, 0xc6, 0x43, - 0x0f, 0x94, 0x75, 0x32, 0xc7, 0xd6, 0x81, 0xce, 0x15, 0x81, 0x47, 0xca, 0x8f, 0x0c, 0x0c, 0x3e, - 0x02, 0xeb, 0x92, 0xf7, 0xd3, 0x10, 0x4b, 0x12, 0x05, 0x7a, 0x4c, 0x4e, 0x59, 0x47, 0xfe, 0xff, - 0xca, 0x2b, 0x4f, 0x64, 0xbb, 0x40, 0x6b, 0x16, 0xd0, 0x9a, 0x9c, 0xfa, 0x0d, 0x9f, 0x80, 0x95, - 0x38, 0x0c, 0x64, 0x97, 0x13, 0xd1, 0x65, 0x49, 0xe4, 0x2c, 0xea, 0x64, 0x77, 0x27, 0x92, 0x29, - 0xde, 0xbd, 0x6e, 0x12, 0x7a, 0xed, 0x82, 0xf7, 0xe6, 0xfa, 0x68, 0xe8, 0x56, 0xbf, 0xdd, 0x6f, - 0x17, 0x51, 0xa8, 0x1a, 0x87, 0xe3, 0x1f, 0xf0, 0x4b, 0x50, 0x56, 0x85, 0x09, 0x67, 0xe9, 0x52, - 0x61, 0xb9, 0x52, 0xbc, 0x42, 0x29, 0xde, 0xf1, 0xb3, 0xfd, 0x7d, 0x55, 0x88, 0x40, 0x26, 0x06, - 0x7e, 0x0a, 0xb6, 0xfa, 0x82, 0xa6, 0xf1, 0x98, 0x77, 0xdd, 0x63, 0x70, 0x4a, 0xce, 0x9d, 0x6a, - 0xc3, 0xda, 0xad, 0xa0, 0x9b, 0xda, 0x9d, 0x73, 0xaf, 0x7b, 0xf8, 0x8e, 0x9c, 0xc3, 0x4f, 0xc0, - 0xd2, 0x80, 0x70, 0x41, 0x59, 0xea, 0xac, 0xe8, 0x5b, 0xb7, 0xaf, 0xa0, 0xe3, 0x99, 0x41, 0xa0, - 0x02, 0x0a, 0xbf, 0x07, 0x1b, 0x5a, 0x12, 0x61, 0xc2, 0x04, 0x89, 0x82, 0xb1, 0xb0, 0x9c, 0xd5, - 0x79, 0x58, 0xb0, 0x5f, 0x0d, 0xdd, 0x05, 0x74, 0x43, 0x65, 0xd8, 0xd7, 0x09, 0xc6, 0xae, 0x2f, - 0xec, 0xbf, 0x7e, 0x71, 0xad, 0x43, 0xbb, 0x52, 0xa9, 0x2d, 0x1f, 0xda, 0x95, 0xe5, 0x1a, 0x38, - 0xb4, 0x2b, 0xa0, 0x56, 0xdd, 0xf9, 0x1d, 0x80, 0x65, 0xad, 0x83, 0x56, 0x7a, 0xc2, 0xe0, 0xb1, - 0x21, 0x8a, 0x68, 0x11, 0x56, 0xef, 0x7f, 0xe4, 0xbd, 0xe7, 0xa5, 0x79, 0x93, 0x7a, 0x6e, 0x56, - 0xd4, 0xf5, 0xaf, 0x87, 0xae, 0x65, 0xa8, 0x23, 0xf0, 0x2e, 0x00, 0x09, 0x16, 0x72, 0x4a, 0xa9, - 0xcb, 0xca, 0x62, 0x14, 0xea, 0x82, 0x6a, 0xda, 0xef, 0x05, 0x19, 0x49, 0x23, 0x9a, 0xc6, 0x5a, - 0xa8, 0x36, 0x02, 0x69, 0xbf, 0xf7, 0xd8, 0x58, 0x0a, 0x40, 0xc4, 0x59, 0x96, 0x91, 0x48, 0xcb, - 0xca, 0x00, 0x0e, 0x8c, 0x05, 0xee, 0x80, 0x55, 0x4d, 0x57, 0xc2, 0xe2, 0x40, 0xd0, 0x97, 0x44, - 0x8b, 0xa5, 0x84, 0xaa, 0xca, 0x78, 0xc4, 0xe2, 0xa7, 0xf4, 0x25, 0x81, 0x1f, 0xe7, 0x94, 0x16, - 0x98, 0x40, 0xf2, 0xbe, 0x90, 0x24, 0x72, 0x80, 0x9e, 0x1e, 0x9c, 0xc0, 0xb6, 0x8d, 0x07, 0x7e, - 0x05, 0xb6, 0x71, 0x96, 0x71, 0x76, 0x46, 0x7b, 0x6a, 0xd4, 0x19, 0x67, 0x19, 0x13, 0x38, 0x09, - 0x5e, 0xf4, 0x99, 0xc4, 0x5a, 0x44, 0x25, 0xe4, 0x4c, 0x20, 0x1e, 0xe7, 0x80, 0x27, 0xca, 0x0f, - 0x3f, 0x07, 0xb7, 0xa6, 0x23, 0x82, 0x8e, 0x7a, 0xb6, 0x86, 0x84, 0x35, 0x1d, 0xbc, 0x99, 0x4d, - 0x46, 0x34, 0xb1, 0x20, 0x86, 0x91, 0xaf, 0xc1, 0x9d, 0x99, 0x50, 0x4e, 0xcc, 0xa3, 0x7f, 0xd1, - 0x27, 0x7d, 0xe2, 0xac, 0x37, 0x4a, 0xbb, 0x25, 0x74, 0x6b, 0x2a, 0x1a, 0x19, 0xc4, 0x13, 0x05, - 0x80, 0x1f, 0x80, 0x75, 0xae, 0xa6, 0x19, 0xf4, 0xf0, 0x59, 0xd0, 0x39, 0x97, 0x44, 0x38, 0x15, - 0x7d, 0xe3, 0xaa, 0x36, 0x1f, 0xe3, 0xb3, 0xa6, 0x32, 0xc2, 0x0c, 0x6c, 0xa5, 0xe4, 0x47, 0x22, - 0xae, 0x50, 0x5a, 0x55, 0x8f, 0xfe, 0xfe, 0xfb, 0x47, 0x5f, 0xa8, 0xc6, 0xdb, 0x6f, 0x7f, 0x93, - 0x4a, 0x7e, 0x9e, 0xcb, 0x6f, 0xc3, 0x24, 0x9e, 0x11, 0x20, 0xfc, 0x01, 0x6c, 0xe1, 0x50, 0xd2, - 0x01, 0xb9, 0x7c, 0xe3, 0xca, 0xfc, 0xda, 0xde, 0x30, 0x39, 0x66, 0x93, 0x3f, 0x04, 0x5b, 0xba, - 0xbf, 0x13, 0x42, 0xa2, 0x80, 0x93, 0x98, 0x0a, 0xc9, 0xb1, 0xa4, 0x2c, 0x15, 0xfa, 0xe1, 0x94, - 0xd0, 0xe6, 0xd8, 0x8d, 0x26, 0xbd, 0xf0, 0x43, 0xb0, 0x2c, 0x49, 0x8a, 0x53, 0x19, 0xd0, 0xc8, - 0xa9, 0x29, 0x7d, 0x35, 0x57, 0x46, 0x43, 0xb7, 0xd2, 0xd6, 0xc6, 0xd6, 0x01, 0xaa, 0x18, 0x77, - 0x2b, 0x82, 0x04, 0x6c, 0xcd, 0x56, 0x1e, 0x64, 0x2c, 0xa1, 0xe1, 0xb9, 0x03, 0x1b, 0xd6, 0xee, - 0xda, 0xd4, 0x6b, 0x99, 0x5a, 0xb1, 0x33, 0xd5, 0x3e, 0xd6, 0x41, 0x68, 0x23, 0xbc, 0xca, 0x0c, - 0x7f, 0xb3, 0xc0, 0xff, 0x2e, 0xdd, 0x23, 0x68, 0x44, 0x24, 0xc7, 0xa9, 0xc8, 0x18, 0x57, 0xcf, - 0xe9, 0x84, 0x39, 0x37, 0x34, 0x69, 0x0f, 0xe7, 0x1b, 0xd3, 0x53, 0x1a, 0x91, 0x76, 0x11, 0xaf, - 0x66, 0xd6, 0xdc, 0x55, 0x74, 0x8e, 0x86, 0x6e, 0x63, 0xa6, 0xb8, 0x4b, 0x48, 0xd4, 0x08, 0x2f, - 0x23, 0xe4, 0x24, 0x62, 0xfb, 0x6f, 0x0b, 0x2c, 0xe5, 0x1a, 0x80, 0xcf, 0xc1, 0x52, 0xca, 0x22, - 0xa2, 0xe8, 0x54, 0x3b, 0xa4, 0xdc, 0xdc, 0x1b, 0x0d, 0xdd, 0xc5, 0x47, 0x2c, 0x22, 0xad, 0x83, - 0x77, 0x43, 0xf7, 0x41, 0x4c, 0x65, 0xb7, 0xdf, 0xf1, 0x42, 0xd6, 0xf3, 0xc7, 0x95, 0x47, 0x9d, - 0x8b, 0xb3, 0x9f, 0x9d, 0xc6, 0x7e, 0xc1, 0x9e, 0x09, 0x43, 0x8b, 0x2a, 0x63, 0x2b, 0x82, 0x8f, - 0x40, 0xed, 0x92, 0x76, 0xae, 0xcd, 0xaf, 0x9d, 0xf5, 0x99, 0x46, 0xe0, 0x1d, 0x60, 0xf7, 0x12, - 0x4c, 0xf5, 0xe2, 0x29, 0x35, 0x2b, 0xa3, 0xa1, 0x6b, 0x1f, 0x1f, 0xed, 0xb5, 0x90, 0xb6, 0xc2, - 0x9b, 0xa0, 0x4c, 0x32, 0x16, 0x76, 0xf5, 0xff, 0x60, 0x09, 0x99, 0x1f, 0x66, 0x8f, 0x8e, 0xb7, - 0xa9, 0x5d, 0x2b, 0x8f, 0xb7, 0xe9, 0xf5, 0x1a, 0xdc, 0xf9, 0xb5, 0x04, 0x36, 0xaf, 0x26, 0x1c, - 0x1e, 0x82, 0x35, 0x6e, 0x96, 0x65, 0xfe, 0x00, 0xf2, 0x1d, 0x3b, 0x57, 0xe9, 0xab, 0x79, 0xa8, - 0x19, 0x16, 0xec, 0x83, 0x6a, 0x91, 0x4b, 0xd5, 0x7f, 0x4d, 0xd7, 0xdf, 0x1e, 0x0d, 0x5d, 0x90, - 0xef, 0xe3, 0xa3, 0xbd, 0xd6, 0xbb, 0xa1, 0xdb, 0x9c, 0x8b, 0xec, 0xc9, 0x0f, 0x2e, 0x53, 0x99, - 0x14, 0x7e, 0x28, 0xd5, 0xdf, 0xfc, 0x5e, 0x0b, 0x81, 0xfc, 0xa2, 0x23, 0x4c, 0x55, 0x0b, 0x21, - 0x49, 0x25, 0xc7, 0x49, 0xd1, 0x42, 0xe9, 0x5f, 0xb4, 0x90, 0x87, 0x5e, 0xb4, 0x50, 0xe4, 0x52, - 0x2d, 0xd8, 0x17, 0x2d, 0xec, 0x1b, 0xf3, 0x7f, 0xd8, 0x42, 0x7e, 0xd1, 0x11, 0xa6, 0x66, 0x70, - 0xcd, 0x7b, 0xaf, 0xfe, 0xac, 0x2f, 0xbc, 0x1a, 0xd5, 0xad, 0xd7, 0xa3, 0xba, 0xf5, 0x66, 0x54, - 0xb7, 0xfe, 0x18, 0xd5, 0xad, 0x9f, 0xde, 0xd6, 0x17, 0x5e, 0xbf, 0xad, 0x2f, 0xbc, 0x79, 0x5b, - 0x5f, 0x78, 0x0e, 0x2e, 0x3e, 0x3d, 0x3b, 0x8b, 0xfa, 0xf3, 0xed, 0xc1, 0x3f, 0x01, 0x00, 0x00, - 0xff, 0xff, 0x09, 0x9d, 0x67, 0xeb, 0x9b, 0x0a, 0x00, 0x00, + // 1091 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x56, 0xdd, 0x6e, 0xdb, 0x36, + 0x14, 0x8e, 0x66, 0xa5, 0x75, 0x68, 0x27, 0x71, 0xd9, 0x26, 0x51, 0xd3, 0xd5, 0x36, 0x3c, 0x6c, + 0xf0, 0x80, 0x4e, 0xc2, 0xba, 0x9f, 0x62, 0x3f, 0xc0, 0x10, 0x27, 0xc0, 0xe0, 0x2c, 0x19, 0x12, + 0xc6, 0xe8, 0x80, 0xed, 0x42, 0xa0, 0x25, 0x46, 0x26, 0x2c, 0x8b, 0x2a, 0x49, 0x1b, 0x49, 0x9f, + 0x62, 0xd8, 0x13, 0xec, 0x62, 0x0f, 0xb0, 0xc7, 0xc8, 0x65, 0x2e, 0x7b, 0x65, 0x6c, 0xce, 0xcd, + 0x6e, 0xf6, 0x02, 0xbb, 0x1a, 0x48, 0x4a, 0x8e, 0xed, 0x04, 0x68, 0x06, 0xec, 0x4e, 0x3e, 0xe7, + 0xfb, 0x78, 0xce, 0xf9, 0xf8, 0x1d, 0x59, 0xa0, 0xd1, 0x1f, 0x79, 0xfd, 0x91, 0x20, 0x7c, 0x44, + 0xf8, 0xf4, 0x21, 0xed, 0x7a, 0x42, 0x62, 0x49, 0xdc, 0x94, 0x33, 0xc9, 0x60, 0x2d, 0x60, 0x41, + 0x9f, 0x33, 0x1c, 0xf4, 0xdc, 0xfe, 0xc8, 0xcd, 0x41, 0xae, 0x90, 0x8c, 0xe3, 0x88, 0xa4, 0xdd, + 0xed, 0x27, 0xd9, 0xa3, 0x47, 0x92, 0x88, 0x26, 0x24, 0xed, 0x7a, 0x83, 0x51, 0x10, 0x18, 0xf6, + 0xf6, 0x13, 0xcd, 0x4c, 0xbb, 0x1e, 0x4d, 0x24, 0xe1, 0x09, 0x8e, 0x7d, 0x8e, 0x4f, 0x65, 0x96, + 0xdc, 0xcc, 0x93, 0x03, 0x22, 0x71, 0x88, 0x25, 0xce, 0xe2, 0x30, 0x8f, 0xcf, 0xc4, 0x9c, 0xa1, + 0xa4, 0xb1, 0xd7, 0x8b, 0x03, 0x4f, 0xd2, 0x01, 0x11, 0x12, 0x0f, 0xd2, 0x2c, 0xf3, 0x28, 0x62, + 0x11, 0xd3, 0x8f, 0x9e, 0x7a, 0x32, 0xd1, 0xc6, 0x2f, 0xcb, 0xa0, 0x8c, 0x48, 0x1a, 0xd3, 0x00, + 0x9f, 0xa8, 0x69, 0xe0, 0x33, 0x00, 0x55, 0x69, 0x1f, 0xa7, 0x69, 0x4c, 0x49, 0xe8, 0xd3, 0x24, + 0x24, 0x67, 0x8e, 0x55, 0xb7, 0x9a, 0x36, 0xaa, 0xa8, 0xcc, 0x8e, 0x49, 0xb4, 0x55, 0x1c, 0xba, + 0xe0, 0x61, 0x4c, 0xb0, 0x20, 0x0b, 0xf0, 0x77, 0x34, 0xfc, 0x81, 0x4e, 0xcd, 0xe1, 0x3f, 0x07, + 0x76, 0x48, 0x44, 0xe0, 0x14, 0xea, 0x56, 0xb3, 0xf4, 0xbc, 0xe1, 0x5e, 0x8b, 0x96, 0xcd, 0xe2, + 0x22, 0x9c, 0x44, 0x64, 0x8f, 0x88, 0x80, 0xd3, 0x54, 0x32, 0x8e, 0x34, 0x1e, 0xba, 0x60, 0x59, + 0x1f, 0xe6, 0xd8, 0x9a, 0xe8, 0xdc, 0x42, 0x3c, 0x50, 0x79, 0x64, 0x60, 0xf0, 0x7b, 0xb0, 0x2e, + 0xf9, 0x30, 0x09, 0xb0, 0x24, 0xa1, 0xaf, 0xaf, 0xc9, 0x59, 0xd6, 0xcc, 0xf7, 0x6f, 0x2d, 0x79, + 0x2a, 0x3b, 0x39, 0x5a, 0xab, 0x80, 0xd6, 0xe4, 0xdc, 0x6f, 0x78, 0x0c, 0xca, 0x51, 0xe0, 0xcb, + 0x1e, 0x27, 0xa2, 0xc7, 0xe2, 0xd0, 0xb9, 0xa7, 0x0f, 0x7b, 0x3a, 0x73, 0x98, 0xd2, 0xdd, 0xed, + 0xc5, 0x81, 0xdb, 0xc9, 0x75, 0x6f, 0xad, 0x4f, 0xc6, 0xb5, 0xd2, 0xb7, 0xbb, 0x9d, 0x9c, 0x85, + 0x4a, 0x51, 0x30, 0xfd, 0x01, 0xbf, 0x02, 0xcb, 0xaa, 0x31, 0xe1, 0xdc, 0xbf, 0xd1, 0x58, 0xe6, + 0x14, 0x37, 0x77, 0x8a, 0x7b, 0xf8, 0x72, 0x77, 0x57, 0x35, 0x22, 0x90, 0xe1, 0xc0, 0xcf, 0xc0, + 0xd6, 0x50, 0xd0, 0x24, 0x9a, 0xea, 0xae, 0x67, 0xf4, 0xfb, 0xe4, 0xdc, 0x29, 0xd5, 0xad, 0x66, + 0x11, 0x3d, 0xd2, 0xe9, 0x4c, 0x7b, 0x3d, 0xc3, 0x77, 0xe4, 0x1c, 0x7e, 0x0a, 0xee, 0x8f, 0x08, + 0x17, 0x94, 0x25, 0x4e, 0x59, 0x57, 0xdd, 0xbe, 0x45, 0x8e, 0x97, 0x06, 0x81, 0x72, 0x28, 0xfc, + 0x01, 0x6c, 0x68, 0x4b, 0x04, 0x31, 0x13, 0x24, 0xf4, 0xa7, 0xc6, 0x72, 0x56, 0xef, 0xa2, 0x82, + 0x7d, 0x31, 0xae, 0x2d, 0xa1, 0x87, 0xea, 0x84, 0x5d, 0x7d, 0xc0, 0x34, 0xf5, 0xa5, 0xfd, 0xd7, + 0xaf, 0x35, 0x6b, 0xdf, 0x2e, 0x16, 0x2b, 0x2b, 0xfb, 0x76, 0x71, 0xa5, 0x02, 0xf6, 0xed, 0x22, + 0xa8, 0x94, 0x1a, 0x7f, 0xdf, 0x07, 0x2b, 0xda, 0x07, 0xed, 0xe4, 0x94, 0xc1, 0x43, 0x23, 0x14, + 0xd1, 0x26, 0x2c, 0x3d, 0xff, 0xc8, 0x7d, 0xcb, 0xa6, 0xb9, 0xb3, 0x7e, 0x6e, 0x15, 0x55, 0xf9, + 0xcb, 0x71, 0xcd, 0x32, 0xd2, 0x11, 0xf8, 0x14, 0x80, 0x18, 0x0b, 0x39, 0xe7, 0xd4, 0x15, 0x15, + 0x31, 0x0e, 0xad, 0x81, 0x52, 0x32, 0x1c, 0xf8, 0x29, 0x49, 0x42, 0x9a, 0x44, 0xda, 0xa8, 0x36, + 0x02, 0xc9, 0x70, 0x70, 0x64, 0x22, 0x39, 0x20, 0xe4, 0x2c, 0x4d, 0x49, 0xa8, 0x6d, 0x65, 0x00, + 0x7b, 0x26, 0x02, 0x1b, 0x60, 0x55, 0xcb, 0x15, 0xb3, 0xc8, 0x17, 0xf4, 0x35, 0xd1, 0x66, 0x29, + 0xa0, 0x92, 0x0a, 0x1e, 0xb0, 0xe8, 0x84, 0xbe, 0x26, 0xf0, 0xe3, 0x4c, 0xd2, 0x1c, 0xe3, 0x4b, + 0x3e, 0x14, 0x92, 0x84, 0x0e, 0xd0, 0xb7, 0x07, 0x67, 0xb0, 0x1d, 0x93, 0x81, 0x5f, 0x83, 0x6d, + 0x9c, 0xa6, 0x9c, 0x9d, 0xd1, 0x81, 0xba, 0xea, 0x94, 0xb3, 0x94, 0x09, 0x1c, 0xfb, 0xaf, 0x86, + 0x4c, 0x62, 0x6d, 0xa2, 0x02, 0x72, 0x66, 0x10, 0x47, 0x19, 0xe0, 0x58, 0xe5, 0xe1, 0x17, 0xe0, + 0xf1, 0x3c, 0xc3, 0xef, 0xaa, 0xb5, 0x35, 0x22, 0xac, 0x69, 0xf2, 0x66, 0x3a, 0xcb, 0x68, 0x61, + 0x41, 0x8c, 0x22, 0xdf, 0x80, 0x77, 0x17, 0xa8, 0x9c, 0x98, 0xa5, 0x7f, 0x35, 0x24, 0x43, 0xe2, + 0xac, 0xd7, 0x0b, 0xcd, 0x02, 0x7a, 0x3c, 0xc7, 0x46, 0x06, 0x71, 0xac, 0x00, 0xf0, 0x03, 0xb0, + 0xce, 0xd5, 0x6d, 0xfa, 0x03, 0x7c, 0xe6, 0x77, 0xcf, 0x25, 0x11, 0x4e, 0x51, 0x57, 0x5c, 0xd5, + 0xe1, 0x43, 0x7c, 0xd6, 0x52, 0x41, 0xf8, 0x13, 0xd8, 0xc2, 0x81, 0xa4, 0x23, 0x72, 0xd3, 0x69, + 0xe5, 0xbb, 0x3b, 0x6d, 0xc3, 0x9c, 0xb1, 0xe0, 0x35, 0xf8, 0x02, 0x6c, 0xe9, 0x6a, 0xa7, 0x84, + 0x84, 0x3e, 0x27, 0x11, 0x15, 0x92, 0x63, 0x49, 0x59, 0x22, 0xb4, 0x8d, 0x0b, 0x68, 0x73, 0x9a, + 0x46, 0xb3, 0x59, 0xf8, 0x21, 0x58, 0x91, 0x24, 0xc1, 0x89, 0xf4, 0x69, 0xe8, 0x54, 0xd4, 0x6d, + 0xb7, 0xca, 0x93, 0x71, 0xad, 0xd8, 0xd1, 0xc1, 0xf6, 0x1e, 0x2a, 0x9a, 0x74, 0x3b, 0x84, 0x04, + 0x6c, 0x2d, 0x76, 0xee, 0xa7, 0x2c, 0xa6, 0xc1, 0xb9, 0x03, 0xeb, 0x56, 0x73, 0x6d, 0xce, 0xbb, + 0x73, 0x2f, 0xbc, 0x85, 0x6e, 0x8f, 0x34, 0x09, 0x6d, 0x04, 0xb7, 0x85, 0xe1, 0xef, 0x16, 0x78, + 0xef, 0x46, 0x1d, 0x41, 0x43, 0x22, 0x39, 0x4e, 0x44, 0xca, 0xb8, 0x32, 0xf7, 0x29, 0x73, 0x1e, + 0x6a, 0xd1, 0x5e, 0xbc, 0x7d, 0x5f, 0x54, 0x07, 0x27, 0x34, 0x24, 0x9d, 0x9c, 0xaf, 0xf6, 0xae, + 0xd5, 0x54, 0x72, 0x4e, 0xc6, 0xb5, 0xfa, 0x42, 0x73, 0x37, 0x90, 0xa8, 0x1e, 0xdc, 0x44, 0xc8, + 0x59, 0xc4, 0x74, 0xd3, 0xed, 0xca, 0xf2, 0x74, 0xd3, 0x4b, 0x95, 0xf2, 0xbe, 0x5d, 0x7c, 0x50, + 0x81, 0x8d, 0xdf, 0x0a, 0x60, 0xf3, 0xf6, 0x26, 0xe0, 0x3e, 0x58, 0xe3, 0x66, 0x9d, 0x33, 0x53, + 0x64, 0x6f, 0x81, 0x3b, 0x59, 0x61, 0x35, 0xa3, 0x9a, 0x01, 0xe0, 0x10, 0x94, 0xf2, 0xb3, 0x62, + 0x4c, 0xf5, 0xea, 0x17, 0x5a, 0x9d, 0xc9, 0xb8, 0x06, 0xb2, 0x37, 0xc6, 0xc1, 0x4e, 0xfb, 0x9f, + 0x71, 0xad, 0x15, 0x51, 0xd9, 0x1b, 0x76, 0xdd, 0x80, 0x0d, 0xbc, 0x69, 0x91, 0xb0, 0x7b, 0xfd, + 0xec, 0xa5, 0xfd, 0xc8, 0x9b, 0xfd, 0x24, 0x30, 0x9d, 0x49, 0xe1, 0x05, 0x52, 0xfd, 0x11, 0xed, + 0xb4, 0x11, 0xc8, 0x0a, 0x1d, 0x60, 0xaa, 0x46, 0x08, 0x48, 0x22, 0x39, 0x8e, 0xf3, 0x11, 0x0a, + 0xff, 0x61, 0x84, 0x8c, 0x7a, 0x3d, 0x42, 0x7e, 0x96, 0x1a, 0xc1, 0xbe, 0x1e, 0x61, 0xd7, 0x84, + 0xff, 0xc7, 0x11, 0xb2, 0x42, 0x07, 0x98, 0x9a, 0xeb, 0x6b, 0x3d, 0xbb, 0xf8, 0xb3, 0xba, 0x74, + 0x31, 0xa9, 0x5a, 0x97, 0x93, 0xaa, 0xf5, 0x66, 0x52, 0xb5, 0xfe, 0x98, 0x54, 0xad, 0x9f, 0xaf, + 0xaa, 0x4b, 0x97, 0x57, 0xd5, 0xa5, 0x37, 0x57, 0xd5, 0xa5, 0x1f, 0xc1, 0xf5, 0xc7, 0x51, 0xf7, + 0x9e, 0xfe, 0xc0, 0xf8, 0xe4, 0xdf, 0x00, 0x00, 0x00, 0xff, 0xff, 0x91, 0x73, 0x5f, 0x8e, 0x3d, + 0x09, 0x00, 0x00, } func (this *ReplicaState) Equal(that interface{}) bool { @@ -489,9 +437,6 @@ func (this *RangeInfo) Equal(that interface{}) bool { if this.RangeMaxBytes != that1.RangeMaxBytes { return false } - if !this.NewestClosedTimestamp.Equal(&that1.NewestClosedTimestamp) { - return false - } if !this.ActiveClosedTimestamp.Equal(&that1.ActiveClosedTimestamp) { return false } @@ -509,39 +454,6 @@ func (this *RangeInfo) Equal(that interface{}) bool { } return true } -func (this *RangeInfo_CTEntry) Equal(that interface{}) bool { - if that == nil { - return this == nil - } - - that1, ok := that.(*RangeInfo_CTEntry) - if !ok { - that2, ok := that.(RangeInfo_CTEntry) - if ok { - that1 = &that2 - } else { - return false - } - } - if that1 == nil { - return this == nil - } else if this == nil { - return false - } - if this.NodeID != that1.NodeID { - return false - } - if !this.ClosedTimestamp.Equal(&that1.ClosedTimestamp) { - return false - } - if this.MLAI != that1.MLAI { - return false - } - if this.Epoch != that1.Epoch { - return false - } - return true -} func (this *RangeSideTransportInfo) Equal(that interface{}) bool { if that == nil { return this == nil @@ -785,16 +697,6 @@ func (m *RangeInfo) MarshalToSizedBuffer(dAtA []byte) (int, error) { } i-- dAtA[i] = 0x62 - { - size, err := m.NewestClosedTimestamp.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintState(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x5a if m.RaftLogSizeTrusted { i-- if m.RaftLogSizeTrusted { @@ -848,54 +750,6 @@ func (m *RangeInfo) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } -func (m *RangeInfo_CTEntry) Marshal() (dAtA []byte, err error) { - size := m.Size() - dAtA = make([]byte, size) - n, err := m.MarshalToSizedBuffer(dAtA[:size]) - if err != nil { - return nil, err - } - return dAtA[:n], nil -} - -func (m *RangeInfo_CTEntry) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *RangeInfo_CTEntry) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if m.Epoch != 0 { - i = encodeVarintState(dAtA, i, uint64(m.Epoch)) - i-- - dAtA[i] = 0x20 - } - if m.MLAI != 0 { - i = encodeVarintState(dAtA, i, uint64(m.MLAI)) - i-- - dAtA[i] = 0x18 - } - { - size, err := m.ClosedTimestamp.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintState(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x12 - if m.NodeID != 0 { - i = encodeVarintState(dAtA, i, uint64(m.NodeID)) - i-- - dAtA[i] = 0x8 - } - return len(dAtA) - i, nil -} - func (m *RangeSideTransportInfo) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -1033,8 +887,6 @@ func (m *RangeInfo) Size() (n int) { if m.RaftLogSizeTrusted { n += 2 } - l = m.NewestClosedTimestamp.Size() - n += 1 + l + sovState(uint64(l)) l = m.ActiveClosedTimestamp.Size() n += 1 + l + sovState(uint64(l)) if m.RangefeedRegistrations != 0 { @@ -1061,26 +913,6 @@ func (m *RangeInfo) Size() (n int) { return n } -func (m *RangeInfo_CTEntry) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if m.NodeID != 0 { - n += 1 + sovState(uint64(m.NodeID)) - } - l = m.ClosedTimestamp.Size() - n += 1 + l + sovState(uint64(l)) - if m.MLAI != 0 { - n += 1 + sovState(uint64(m.MLAI)) - } - if m.Epoch != 0 { - n += 1 + sovState(uint64(m.Epoch)) - } - return n -} - func (m *RangeSideTransportInfo) Size() (n int) { if m == nil { return 0 @@ -1659,39 +1491,6 @@ func (m *RangeInfo) Unmarshal(dAtA []byte) error { } } m.RaftLogSizeTrusted = bool(v != 0) - case 11: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field NewestClosedTimestamp", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowState - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthState - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthState - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if err := m.NewestClosedTimestamp.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex case 12: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field ActiveClosedTimestamp", wireType) @@ -1931,146 +1730,6 @@ func (m *RangeInfo) Unmarshal(dAtA []byte) error { } return nil } -func (m *RangeInfo_CTEntry) Unmarshal(dAtA []byte) error { - l := len(dAtA) - iNdEx := 0 - for iNdEx < l { - preIndex := iNdEx - var wire uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowState - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: CTEntry: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: CTEntry: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field NodeID", wireType) - } - m.NodeID = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowState - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.NodeID |= github_com_cockroachdb_cockroach_pkg_roachpb.NodeID(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ClosedTimestamp", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowState - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthState - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthState - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if err := m.ClosedTimestamp.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field MLAI", wireType) - } - m.MLAI = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowState - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.MLAI |= int64(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) - } - m.Epoch = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowState - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.Epoch |= int64(b&0x7F) << shift - if b < 0x80 { - break - } - } - default: - iNdEx = preIndex - skippy, err := skipState(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthState - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} func (m *RangeSideTransportInfo) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 diff --git a/pkg/kv/kvserver/kvserverpb/state.proto b/pkg/kv/kvserver/kvserverpb/state.proto index 7862fe5a6385..4caca7500bd1 100644 --- a/pkg/kv/kvserver/kvserverpb/state.proto +++ b/pkg/kv/kvserver/kvserverpb/state.proto @@ -135,27 +135,12 @@ message RangeInfo { repeated int64 proposal_quota_release_queue = 15; // The max size the range can grow to before it will be split. int64 range_max_bytes = 8; - reserved 9; - message CTEntry { - option (gogoproto.equal) = true; - int32 node_id = 1 [(gogoproto.customname) = "NodeID", - (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID"]; - util.hlc.Timestamp closed_timestamp = 2 [(gogoproto.nullable) = false]; - int64 mlai = 3 [(gogoproto.customname) = "MLAI"]; - int64 epoch = 4; - } - // The highest closed timestamp known to have data for this replica, taken - // across the data received from all nodes. This does not reflect whether - // the replica can use this closed timestamp (it may, for example, not have - // caught up sufficiently to do so). - CTEntry newest_closed_timestamp = 11 [(gogoproto.nullable) = false]; + reserved 9, 11; // The closed timestamp active on the replica when the info was generated. // This is the actual timestamp at or below which requests can be served from - // this replica at this moment (assuming it is not the leaseholder). This takes - // into account the lease start time, the current lease applied index, and the - // closed timestamp information received from other nodes, among other things. - // In practice, this should not usually trail newest_closed_timestamp except - // for a short moment after newest_closed_timestamp gets updated. + // this replica at this moment (assuming it is not the leaseholder). This + // takes into account the current lease applied index, and the closed + // timestamp information received from other nodes, among other things. util.hlc.Timestamp active_closed_timestamp = 12 [(gogoproto.nullable) = false]; // The number of Rangefeed registrations attached to the Replica. int64 rangefeed_registrations = 13; diff --git a/pkg/kv/kvserver/liveness/BUILD.bazel b/pkg/kv/kvserver/liveness/BUILD.bazel index 8933d2a498da..92e2a790a5b5 100644 --- a/pkg/kv/kvserver/liveness/BUILD.bazel +++ b/pkg/kv/kvserver/liveness/BUILD.bazel @@ -10,8 +10,6 @@ go_library( "//pkg/gossip", "//pkg/keys", "//pkg/kv", - "//pkg/kv/kvserver/closedts", - "//pkg/kv/kvserver/closedts/ctpb", "//pkg/kv/kvserver/liveness/livenesspb", "//pkg/roachpb:with-mocks", "//pkg/settings/cluster", diff --git a/pkg/kv/kvserver/liveness/liveness.go b/pkg/kv/kvserver/liveness/liveness.go index 4d2d2c64ad34..ce32c4014cca 100644 --- a/pkg/kv/kvserver/liveness/liveness.go +++ b/pkg/kv/kvserver/liveness/liveness.go @@ -21,8 +21,6 @@ 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/kvserver/closedts" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -64,8 +62,6 @@ var ( // someone else has already incremented the epoch to the desired // value. ErrEpochAlreadyIncremented = errors.New("epoch already incremented") - - errLiveClockNotLive = errors.New("not live") ) type errRetryLiveness struct { @@ -1458,23 +1454,6 @@ func (nl *NodeLiveness) numLiveNodes() int64 { return liveNodes } -// AsLiveClock returns a closedts.LiveClockFn that takes a current timestamp off -// the clock and returns it only if node liveness indicates that the node is live -// at that timestamp and the returned epoch. -func (nl *NodeLiveness) AsLiveClock() closedts.LiveClockFn { - return func(nodeID roachpb.NodeID) (hlc.Timestamp, ctpb.Epoch, error) { - now := nl.clock.Now() - liveness, ok := nl.GetLiveness(nodeID) - if !ok { - return hlc.Timestamp{}, 0, ErrRecordCacheMiss - } - if !liveness.IsLive(now.GoTime()) { - return hlc.Timestamp{}, 0, errLiveClockNotLive - } - return now, ctpb.Epoch(liveness.Epoch), nil - } -} - // GetNodeCount returns a count of the number of nodes in the cluster, // including dead nodes, but excluding decommissioning or decommissioned nodes. func (nl *NodeLiveness) GetNodeCount() int { diff --git a/pkg/kv/kvserver/metrics.go b/pkg/kv/kvserver/metrics.go index ea57230f45a0..d33369c57e06 100644 --- a/pkg/kv/kvserver/metrics.go +++ b/pkg/kv/kvserver/metrics.go @@ -1327,8 +1327,7 @@ type StoreMetrics struct { MaxLockWaitQueueWaitersForLock *metric.Gauge // Closed timestamp metrics. - ClosedTimestampMaxBehindNanos *metric.Gauge - ClosedTimestampFailuresToClose *metric.Gauge + ClosedTimestampMaxBehindNanos *metric.Gauge } // TenantsStorageMetrics are metrics which are aggregated over all tenants @@ -1723,8 +1722,7 @@ func newStoreMetrics(histogramWindow time.Duration) *StoreMetrics { MaxLockWaitQueueWaitersForLock: metric.NewGauge(metaConcurrencyMaxLockWaitQueueWaitersForLock), // Closed timestamp metrics. - ClosedTimestampMaxBehindNanos: metric.NewGauge(metaClosedTimestampMaxBehindNanos), - ClosedTimestampFailuresToClose: metric.NewGauge(metaClosedTimestampFailuresToClose), + ClosedTimestampMaxBehindNanos: metric.NewGauge(metaClosedTimestampMaxBehindNanos), } storeRegistry.AddMetricStruct(sm) diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 347fcd0e852d..fd0858d489fb 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -24,7 +24,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/abortspan" "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" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/gc" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" @@ -1145,7 +1144,7 @@ func (r *Replica) State(ctx context.Context) kvserverpb.RangeInfo { // NB: this acquires an RLock(). Reentrant RLocks are deadlock prone, so do // this first before RLocking below. Performance of this extra lock // acquisition is not a concern. - ri.ActiveClosedTimestamp, _ = r.maxClosed(context.Background()) + ri.ActiveClosedTimestamp = r.maxClosed(ctx) // NB: numRangefeedRegistrations doesn't require Replica.mu to be locked. // However, it does require coordination between multiple goroutines, so @@ -1172,26 +1171,6 @@ func (r *Replica) State(ctx context.Context) kvserverpb.RangeInfo { } ri.RangeMaxBytes = *r.mu.zone.RangeMaxBytes if desc := ri.ReplicaState.Desc; desc != nil { - // Learner replicas don't serve follower reads, but they still receive - // closed timestamp updates, so include them here. - allReplicas := desc.Replicas().Descriptors() - for i := range allReplicas { - replDesc := &allReplicas[i] - r.store.cfg.ClosedTimestamp.Storage.VisitDescending(replDesc.NodeID, func(e ctpb.Entry) (done bool) { - mlai, found := e.MLAI[r.RangeID] - if !found { - return false // not done - } - if ri.NewestClosedTimestamp.ClosedTimestamp.Less(e.ClosedTimestamp) { - ri.NewestClosedTimestamp.NodeID = replDesc.NodeID - ri.NewestClosedTimestamp.ClosedTimestamp = e.ClosedTimestamp - ri.NewestClosedTimestamp.MLAI = int64(mlai) - ri.NewestClosedTimestamp.Epoch = int64(e.Epoch) - } - return true // done - }) - } - if r.mu.tenantID != (roachpb.TenantID{}) { ri.TenantID = r.mu.tenantID.ToUint64() } diff --git a/pkg/kv/kvserver/replica_application_result.go b/pkg/kv/kvserver/replica_application_result.go index c86f35ab5fa7..7dbcdcb9b0f1 100644 --- a/pkg/kv/kvserver/replica_application_result.go +++ b/pkg/kv/kvserver/replica_application_result.go @@ -13,7 +13,6 @@ package kvserver import ( "context" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "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" @@ -205,16 +204,12 @@ func (r *Replica) tryReproposeWithNewLeaseIndex( return nil } - minTS, untrack := r.store.cfg.ClosedTimestamp.Tracker.Track(ctx) - defer untrack(ctx, 0, 0, 0) // covers all error paths below - // We need to track the request again in order to protect its timestamp until // it gets reproposed. // TODO(andrei): Only track if the request consults the ts cache. Some // requests (e.g. EndTxn) don't care about closed timestamps. - minTS2, tok := r.mu.proposalBuf.TrackEvaluatingRequest(ctx, p.Request.WriteTimestamp()) + minTS, tok := r.mu.proposalBuf.TrackEvaluatingRequest(ctx, p.Request.WriteTimestamp()) defer tok.DoneIfNotMoved(ctx) - minTS.Forward(minTS2) // NB: p.Request.Timestamp reflects the action of ba.SetActiveTimestamp. // The IsIntentWrite condition matches the similar logic for caring @@ -238,10 +233,6 @@ func (r *Replica) tryReproposeWithNewLeaseIndex( if pErr != nil { return pErr } - // NB: The caller already promises that the lease check succeeded, meaning - // the sequence numbers match, implying that the lease epoch hasn't changed - // from what it was under the proposal-time lease. - untrack(ctx, ctpb.Epoch(r.mu.state.Lease.Epoch), r.RangeID, ctpb.LAI(maxLeaseIndex)) log.VEventf(ctx, 2, "reproposed command %x at maxLeaseIndex=%d", cmd.idKey, maxLeaseIndex) return nil } diff --git a/pkg/kv/kvserver/replica_application_state_machine.go b/pkg/kv/kvserver/replica_application_state_machine.go index 80de1ca9c959..5606b2f62e2e 100644 --- a/pkg/kv/kvserver/replica_application_state_machine.go +++ b/pkg/kv/kvserver/replica_application_state_machine.go @@ -929,11 +929,7 @@ func (b *replicaAppBatch) ApplyToStateMachine(ctx context.Context) error { tenantID := r.mu.tenantID r.mu.Unlock() if closedTimestampUpdated { - // TODO(andrei): Pass in the new closed timestamp to - // r.handleClosedTimestampUpdateRaftMuLocked directly after the old closed - // ts tracker goes away. Until then we can't do it; we have to let the - // method consult r.maxClosed(). - r.handleClosedTimestampUpdateRaftMuLocked(ctx) + r.handleClosedTimestampUpdateRaftMuLocked(ctx, b.state.RaftClosedTimestamp) } // Record the stats delta in the StoreMetrics. diff --git a/pkg/kv/kvserver/replica_closedts.go b/pkg/kv/kvserver/replica_closedts.go index 0fe9bbea9713..f9548e94ab7f 100644 --- a/pkg/kv/kvserver/replica_closedts.go +++ b/pkg/kv/kvserver/replica_closedts.go @@ -23,48 +23,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/syncutil" ) -// EmitMLAI registers the replica's last assigned max lease index with the -// closed timestamp tracker. This is called to emit an update about this -// replica in the absence of write activity. -func (r *Replica) EmitMLAI() { - r.mu.RLock() - lai := r.mu.proposalBuf.LastAssignedLeaseIndexRLocked() - if r.mu.state.LeaseAppliedIndex > lai { - lai = r.mu.state.LeaseAppliedIndex - } - epoch := r.mu.state.Lease.Epoch - isLeaseholder := r.mu.state.Lease.Replica.ReplicaID == r.mu.replicaID - isMergeInProgress := r.mergeInProgressRLocked() - r.mu.RUnlock() - - // If we're the leaseholder of an epoch-based lease, notify the minPropTracker - // of the current LAI to trigger a re-broadcast of this range's LAI. - if isLeaseholder && epoch > 0 { - ctx := r.AnnotateCtx(context.Background()) - _, untrack := r.store.cfg.ClosedTimestamp.Tracker.Track(ctx) - if isMergeInProgress { - // A critical requirement for the correctness of range merges is that we - // don't allow follower reads on closed timestamps that are greater than - // the subsumption time of the RHS range. Thus, while a range is subsumed, - // we ensure that any intervening closed timestamp updates (until the - // merge either commits or aborts) can only be activated *after* the merge - // has completed (successfully or otherwise), by requiring that follower - // replicas must catch up to an MLAI that succeeds the range's current - // lease applied index. See comment block at the end of Subsume() in - // cmd_subsume.go for more details. - // - // Omitting the closed timestamp update here would be legal, but - // undesirable because if the range were to go on to quiesce, the follower - // replicas would not be able to implicitly tick their closed timestamps - // without `Request`ing it from the new leaseholder. Emitting it here - // avoids that little bit of latency. - untrack(ctx, ctpb.Epoch(epoch), r.RangeID, ctpb.LAI(lai+1)) - } else { - untrack(ctx, ctpb.Epoch(epoch), r.RangeID, ctpb.LAI(lai)) - } - } -} - // BumpSideTransportClosed advances the range's closed timestamp if it can. If // the closed timestamp is advanced, the function synchronizes with incoming // requests, making sure that future requests are not allowed to write below the diff --git a/pkg/kv/kvserver/replica_closedts_internal_test.go b/pkg/kv/kvserver/replica_closedts_internal_test.go index 25e21b8a842a..1a2a65fd8d43 100644 --- a/pkg/kv/kvserver/replica_closedts_internal_test.go +++ b/pkg/kv/kvserver/replica_closedts_internal_test.go @@ -508,8 +508,8 @@ func (r *mockReceiver) HTML() string { return "" } -// Test that r.GetClosedTimestampV2() mixes its sources of information correctly. -func TestReplicaClosedTimestampV2(t *testing.T) { +// Test that r.GetClosedTimestamp() mixes its sources of information correctly. +func TestReplicaClosedTimestamp(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) ctx := context.Background() @@ -566,7 +566,7 @@ func TestReplicaClosedTimestampV2(t *testing.T) { tc.repl.mu.state.RaftClosedTimestamp = test.raftClosed tc.repl.mu.state.LeaseAppliedIndex = uint64(test.applied) tc.repl.mu.Unlock() - require.Equal(t, test.expClosed, tc.repl.GetClosedTimestampV2(ctx)) + require.Equal(t, test.expClosed, tc.repl.maxClosed(ctx)) }) } } diff --git a/pkg/kv/kvserver/replica_eval_context_span.go b/pkg/kv/kvserver/replica_eval_context_span.go index db9ef0b4a02b..5e197e16ad8a 100644 --- a/pkg/kv/kvserver/replica_eval_context_span.go +++ b/pkg/kv/kvserver/replica_eval_context_span.go @@ -17,7 +17,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/abortspan" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb" @@ -101,12 +100,6 @@ func (rec *SpanSetReplicaEvalContext) GetLeaseAppliedIndex() uint64 { return rec.i.GetLeaseAppliedIndex() } -// GetTracker returns the min prop tracker that keeps tabs over ongoing command -// evaluations for the closed timestamp subsystem. -func (rec *SpanSetReplicaEvalContext) GetTracker() closedts.TrackerI { - return rec.i.GetTracker() -} - // IsFirstRange returns true iff the replica belongs to the first range. func (rec *SpanSetReplicaEvalContext) IsFirstRange() bool { return rec.i.IsFirstRange() @@ -223,9 +216,9 @@ func (rec *SpanSetReplicaEvalContext) GetCurrentReadSummary(ctx context.Context) return rec.i.GetCurrentReadSummary(ctx) } -// GetClosedTimestampV2 is part of the EvalContext interface. -func (rec *SpanSetReplicaEvalContext) GetClosedTimestampV2(ctx context.Context) hlc.Timestamp { - return rec.i.GetClosedTimestampV2(ctx) +// GetClosedTimestamp is part of the EvalContext interface. +func (rec *SpanSetReplicaEvalContext) GetClosedTimestamp(ctx context.Context) hlc.Timestamp { + return rec.i.GetClosedTimestamp(ctx) } // GetExternalStorage returns an ExternalStorage object, based on diff --git a/pkg/kv/kvserver/replica_follower_read.go b/pkg/kv/kvserver/replica_follower_read.go index 9765bb9a7aa0..f1819ecc6fe9 100644 --- a/pkg/kv/kvserver/replica_follower_read.go +++ b/pkg/kv/kvserver/replica_follower_read.go @@ -15,7 +15,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" - ctstorage "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/storage" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -67,7 +66,6 @@ func (r *Replica) canServeFollowerReadRLocked( ) bool { var lErr *roachpb.NotLeaseHolderError eligible := errors.As(err, &lErr) && - lErr.LeaseHolder != nil && lErr.Lease != nil && lErr.Lease.Type() == roachpb.LeaseEpoch && BatchCanBeEvaluatedOnFollower(*ba) && FollowerReadsEnabled.Get(&r.store.cfg.Settings.SV) @@ -89,7 +87,7 @@ func (r *Replica) canServeFollowerReadRLocked( } requiredFrontier := ba.RequiredFrontier() - maxClosed, _ := r.maxClosedRLocked(ctx, requiredFrontier /* sufficient */) + maxClosed := r.maxClosedRLocked(ctx, requiredFrontier /* sufficient */) canServeFollowerRead := requiredFrontier.LessEq(maxClosed) tsDiff := requiredFrontier.GoTime().Sub(maxClosed.GoTime()) if !canServeFollowerRead { @@ -100,19 +98,8 @@ func (r *Replica) canServeFollowerReadRLocked( // We can't actually serve the read based on the closed timestamp. // Signal the clients that we want an update so that future requests can succeed. - r.store.cfg.ClosedTimestamp.Clients.Request(lErr.LeaseHolder.NodeID, r.RangeID) log.Eventf(ctx, "can't serve follower read; closed timestamp too low by: %s; maxClosed: %s ts: %s uncertaintyLimit: %s", tsDiff, maxClosed, ba.Timestamp, uncertaintyLimitStr) - - if false { - // NB: this can't go behind V(x) because the log message created by the - // storage might be gigantic in real clusters, and we don't want to trip it - // using logspy. - log.Warningf(ctx, "can't serve follower read for %s at epo %d, storage is %s", - ba.Timestamp, lErr.Lease.Epoch, - r.store.cfg.ClosedTimestamp.Storage.(*ctstorage.MultiStorage).StringForNodes(lErr.LeaseHolder.NodeID), - ) - } return false } @@ -132,14 +119,7 @@ func (r *Replica) canServeFollowerReadRLocked( // start time of the current lease because leasePostApply bumps the timestamp // cache forward to at least the new lease start time. Using this combination // allows the closed timestamp mechanism to be robust to lease transfers. -// If the ok return value is false, the Replica is a member of a range which -// uses an expiration-based lease. Expiration-based leases do not support the -// closed timestamp subsystem. A zero-value timestamp will be returned if ok -// is false. -// -// TODO(andrei): Remove the bool retval once we remove the old closed timestamp -// mechanism. -func (r *Replica) maxClosed(ctx context.Context) (_ hlc.Timestamp, ok bool) { +func (r *Replica) maxClosed(ctx context.Context) hlc.Timestamp { r.mu.RLock() defer r.mu.RUnlock() return r.maxClosedRLocked(ctx, hlc.Timestamp{} /* sufficient */) @@ -151,50 +131,21 @@ func (r *Replica) maxClosed(ctx context.Context) (_ hlc.Timestamp, ok bool) { // maximum closed timestamp that we know about, as long as the returned // timestamp is still >= sufficient. This is a performance optimization because // we can avoid consulting the ClosedTimestampReceiver. -func (r *Replica) maxClosedRLocked( - ctx context.Context, sufficient hlc.Timestamp, -) (_ hlc.Timestamp, ok bool) { +func (r *Replica) maxClosedRLocked(ctx context.Context, sufficient hlc.Timestamp) hlc.Timestamp { appliedLAI := ctpb.LAI(r.mu.state.LeaseAppliedIndex) - lease := r.mu.state.Lease + leaseholder := r.mu.state.Lease.Replica.NodeID initialMaxClosed := r.mu.initialMaxClosed raftClosed := r.mu.state.RaftClosedTimestamp - sideTransportClosed := r.sideTransportClosedTimestamp.get(ctx, lease.Replica.NodeID, appliedLAI, sufficient) + sideTransportClosed := r.sideTransportClosedTimestamp.get(ctx, leaseholder, appliedLAI, sufficient) - // TODO(andrei): In 21.1 we added support for closed timestamps on ranges with - // expiration-based leases. Once the old closed timestamp transport is gone in - // 21.2, this can go away. - if lease.Expiration != nil { - return hlc.Timestamp{}, false - } - // Look at the legacy closed timestamp propagation mechanism. - maxClosed := r.store.cfg.ClosedTimestamp.Provider.MaxClosed( - lease.Replica.NodeID, r.RangeID, ctpb.Epoch(lease.Epoch), appliedLAI) + var maxClosed hlc.Timestamp maxClosed.Forward(initialMaxClosed) - - // Look at the "new" closed timestamp propagation mechanism. maxClosed.Forward(raftClosed) maxClosed.Forward(sideTransportClosed) - - return maxClosed, true + return maxClosed } -// GetClosedTimestampV2 returns the closed timestamp. Unlike MaxClosedTimestamp, -// it only looks at the "new" closed timestamp mechanism, ignoring the old one. -// It returns an empty result if the new mechanism is not enabled yet. The new -// mechanism has better properties than the old one - namely the closing of -// timestamps is synchronized with lease transfers and subsumption requests. -// Callers who need that property should be prepared to get an empty result -// back, meaning that the closed timestamp cannot be known. -// -// TODO(andrei): Remove this in favor of maxClosed() once the old closed -// timestamp mechanism is deleted. At that point, the two should be equivalent. -func (r *Replica) GetClosedTimestampV2(ctx context.Context) hlc.Timestamp { - r.mu.RLock() - appliedLAI := ctpb.LAI(r.mu.state.LeaseAppliedIndex) - leaseholder := r.mu.state.Lease.Replica.NodeID - raftClosed := r.mu.state.RaftClosedTimestamp - r.mu.RUnlock() - sideTransportClosed := r.sideTransportClosedTimestamp.get(ctx, leaseholder, appliedLAI, hlc.Timestamp{} /* sufficient */) - raftClosed.Forward(sideTransportClosed) - return raftClosed +// GetClosedTimestamp is part of the EvalContext interface. +func (r *Replica) GetClosedTimestamp(ctx context.Context) hlc.Timestamp { + return r.maxClosed(ctx) } diff --git a/pkg/kv/kvserver/replica_proposal.go b/pkg/kv/kvserver/replica_proposal.go index d21be9bbe723..1b0777b430e0 100644 --- a/pkg/kv/kvserver/replica_proposal.go +++ b/pkg/kv/kvserver/replica_proposal.go @@ -517,11 +517,6 @@ func (r *Replica) leasePostApplyLocked( if err := r.MaybeGossipNodeLivenessRaftMuLocked(ctx, keys.NodeLivenessSpan); err != nil { log.Errorf(ctx, "%v", err) } - - // Emit an MLAI on the leaseholder replica, as follower will be looking - // for one and if we went on to quiesce, they wouldn't necessarily get - // one otherwise (unless they ask for it, which adds latency). - r.EmitMLAI() }) if leaseChangingHands && log.V(1) { // This logging is useful to troubleshoot incomplete drains. diff --git a/pkg/kv/kvserver/replica_proposal_buf.go b/pkg/kv/kvserver/replica_proposal_buf.go index af1882cf0fb6..160346e3ccdc 100644 --- a/pkg/kv/kvserver/replica_proposal_buf.go +++ b/pkg/kv/kvserver/replica_proposal_buf.go @@ -15,7 +15,6 @@ import ( "sync" "sync/atomic" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/tracker" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -215,11 +214,6 @@ type proposer interface { leaseAppliedIndex() uint64 enqueueUpdateCheck() closedTimestampTarget() hlc.Timestamp - // raftTransportClosedTimestampEnabled returns whether the range has switched - // to the Raft-based closed timestamp transport. - // TODO(andrei): This shouldn't be needed any more in 21.2, once the Raft - // transport is unconditionally enabled. - raftTransportClosedTimestampEnabled() bool // The following require the proposer to hold an exclusive lock. withGroupLocked(func(proposerRaft) error) error registerProposalLocked(*ProposalData) @@ -708,13 +702,6 @@ func (b *propBuf) assignClosedTimestampToProposalLocked( if b.testing.dontCloseTimestamps { return nil } - // If the Raft transport is not enabled yet, bail. If the range has already - // started publishing closed timestamps using Raft, then it doesn't matter - // whether this node found out about the version bump yet. - if !b.p.raftTransportClosedTimestampEnabled() && - !b.settings.Version.IsActive(ctx, clusterversion.ClosedTimestampsRaftTransport) { - return nil - } // Lease requests don't carry closed timestamps. The reason for this differ // between lease extensions and brand new leases: @@ -1078,10 +1065,6 @@ func (rp *replicaProposer) closedTimestampTarget() hlc.Timestamp { return (*Replica)(rp).closedTimestampTargetRLocked() } -func (rp *replicaProposer) raftTransportClosedTimestampEnabled() bool { - return !(*Replica)(rp).mu.state.RaftClosedTimestamp.IsEmpty() -} - func (rp *replicaProposer) withGroupLocked(fn func(raftGroup proposerRaft) error) error { // Pass true for mayCampaignOnWake because we're about to propose a command. return (*Replica)(rp).withRaftGroupLocked(true, func(raftGroup *raft.RawNode) (bool, error) { diff --git a/pkg/kv/kvserver/replica_proposal_buf_test.go b/pkg/kv/kvserver/replica_proposal_buf_test.go index d0cb37662485..71ba1c8af070 100644 --- a/pkg/kv/kvserver/replica_proposal_buf_test.go +++ b/pkg/kv/kvserver/replica_proposal_buf_test.go @@ -137,10 +137,6 @@ func (t *testProposer) closedTimestampTarget() hlc.Timestamp { ) } -func (t *testProposer) raftTransportClosedTimestampEnabled() bool { - return true -} - func (t *testProposer) withGroupLocked(fn func(proposerRaft) error) error { // Note that t.raftGroup can be nil, which FlushLockedWithRaftGroup supports. return fn(t.raftGroup) diff --git a/pkg/kv/kvserver/replica_raftstorage.go b/pkg/kv/kvserver/replica_raftstorage.go index cf03ab608877..913ed9b4cbf9 100644 --- a/pkg/kv/kvserver/replica_raftstorage.go +++ b/pkg/kv/kvserver/replica_raftstorage.go @@ -18,7 +18,6 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftentry" @@ -383,12 +382,6 @@ func (r *Replica) GetLeaseAppliedIndex() uint64 { return r.mu.state.LeaseAppliedIndex } -// GetTracker returns the min prop tracker that keeps tabs over ongoing command -// evaluations for the closed timestamp subsystem. -func (r *Replica) GetTracker() closedts.TrackerI { - return r.store.cfg.ClosedTimestamp.Tracker -} - // Snapshot implements the raft.Storage interface. Snapshot requires that // r.mu is held. Note that the returned snapshot is a placeholder and // does not contain any of the replica data. The snapshot is actually generated diff --git a/pkg/kv/kvserver/replica_rangefeed.go b/pkg/kv/kvserver/replica_rangefeed.go index 85b9f6ac4beb..6aa7434471d4 100644 --- a/pkg/kv/kvserver/replica_rangefeed.go +++ b/pkg/kv/kvserver/replica_rangefeed.go @@ -410,7 +410,7 @@ func (r *Replica) registerWithRangefeedRaftMuLocked( // Check for an initial closed timestamp update immediately to help // initialize the rangefeed's resolved timestamp as soon as possible. - r.handleClosedTimestampUpdateRaftMuLocked(ctx) + r.handleClosedTimestampUpdateRaftMuLocked(ctx, r.maxClosed(ctx)) return p } @@ -602,27 +602,30 @@ func (r *Replica) handleLogicalOpLogRaftMuLocked( } } -// handleClosedTimestampUpdate determines the current maximum closed timestamp -// for the replica and informs the rangefeed, if one is running. No-op if a +// handleClosedTimestampUpdate takes the a closed timestamp for the replica +// and informs the rangefeed, if one is running. No-op if a // rangefeed is not active. -func (r *Replica) handleClosedTimestampUpdate(ctx context.Context) { +// +// closeTS is generally expected to be the highest closed timestamp known, but +// it doesn't need to be - handleClosedTimestampUpdate can be called with +// updates out of order. +func (r *Replica) handleClosedTimestampUpdate(ctx context.Context, closedTS hlc.Timestamp) { ctx = r.AnnotateCtx(ctx) r.raftMu.Lock() defer r.raftMu.Unlock() - r.handleClosedTimestampUpdateRaftMuLocked(ctx) + r.handleClosedTimestampUpdateRaftMuLocked(ctx, closedTS) } // handleClosedTimestampUpdateRaftMuLocked is like handleClosedTimestampUpdate, // but it requires raftMu to be locked. -func (r *Replica) handleClosedTimestampUpdateRaftMuLocked(ctx context.Context) { +func (r *Replica) handleClosedTimestampUpdateRaftMuLocked( + ctx context.Context, closedTS hlc.Timestamp, +) { p := r.getRangefeedProcessor() if p == nil { return } - // Determine what the maximum closed timestamp is for this replica. - closedTS, _ := r.maxClosed(ctx) - // If the closed timestamp is sufficiently stale, signal that we want an // update to the leaseholder so that it will eventually begin to progress // again. @@ -713,25 +716,5 @@ func (r *Replica) ensureClosedTimestampStarted(ctx context.Context) *roachpb.Err return roachpb.NewError(err) } } - - if r.store.cfg.Settings.Version.IsActive(ctx, clusterversion.ClosedTimestampsRaftTransport) { - // In the "new closed timestamps subsystem", there's nothing more to do. - // Once there's a leaseholder, that node will connect to us and inform us of - // updates. - return nil - } - - lease = r.CurrentLeaseStatus(ctx) - if lease.OwnedBy(r.StoreID()) { - // We have the lease. Request is essentially a wrapper for calling EmitMLAI - // on a remote node, so cut out the middleman. - r.EmitMLAI() - return nil - } - leaseholderNodeID := lease.Lease.Replica.NodeID - // Request fixes any issues where we've missed a closed timestamp update or - // where we're not connected to receive them from this node in the first - // place. - r.store.cfg.ClosedTimestamp.Clients.Request(leaseholderNodeID, r.RangeID) return nil } diff --git a/pkg/kv/kvserver/replica_rangefeed_test.go b/pkg/kv/kvserver/replica_rangefeed_test.go index aed2b116f2eb..95c16669ce1a 100644 --- a/pkg/kv/kvserver/replica_rangefeed_test.go +++ b/pkg/kv/kvserver/replica_rangefeed_test.go @@ -895,114 +895,6 @@ func TestReplicaRangefeedPushesTransactions(t *testing.T) { rangeFeedCancel() } -// TestReplicaRangefeedNudgeSlowClosedTimestamp tests that rangefeed detects -// that its closed timestamp updates have stalled and requests new information -// from its Range's leaseholder. This is a regression test for #35142. -func TestReplicaRangefeedNudgeSlowClosedTimestamp(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - ctx := context.Background() - tc, db, desc := setupClusterForClosedTSTesting(ctx, t, testingTargetDuration, - testingCloseFraction, aggressiveResolvedTimestampClusterArgs, "cttest", "kv") - defer tc.Stopper().Stop(ctx) - repls := replsForRange(ctx, t, tc, desc, numNodes) - - sqlDB := sqlutils.MakeSQLRunner(db) - sqlDB.Exec(t, `SET CLUSTER SETTING kv.rangefeed.enabled = true`) - // While we're here, drop the target duration. This was set to - // testingTargetDuration above, but this is higher then it needs to be now - // that cluster and schema setup is complete. - sqlDB.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '10ms'`) - - // Make sure all the nodes have gotten the rangefeed enabled setting from - // gossip, so that they will immediately be able to accept RangeFeeds. The - // target_duration one is just to speed up the test, we don't care if it has - // propagated everywhere yet. - testutils.SucceedsSoon(t, func() error { - for i := 0; i < tc.NumServers(); i++ { - var enabled bool - if err := tc.ServerConn(i).QueryRow( - `SHOW CLUSTER SETTING kv.rangefeed.enabled`, - ).Scan(&enabled); err != nil { - return err - } - if !enabled { - return errors.Errorf(`waiting for rangefeed to be enabled on node %d`, i) - } - } - return nil - }) - - ts1 := tc.Server(0).Clock().Now() - rangeFeedCtx, rangeFeedCancel := context.WithCancel(ctx) - defer rangeFeedCancel() - rangeFeedChs := make([]chan *roachpb.RangeFeedEvent, len(repls)) - rangeFeedErrC := make(chan error, len(repls)) - for i := range repls { - ds := tc.Server(i).DistSenderI().(*kvcoord.DistSender) - rangeFeedCh := make(chan *roachpb.RangeFeedEvent) - rangeFeedChs[i] = rangeFeedCh - go func() { - span := roachpb.Span{ - Key: desc.StartKey.AsRawKey(), EndKey: desc.EndKey.AsRawKey(), - } - rangeFeedErrC <- ds.RangeFeed(rangeFeedCtx, span, ts1, false /* withDiff */, rangeFeedCh) - }() - } - - // Wait for a RangeFeed checkpoint on each RangeFeed after the RangeFeed - // initial scan time (which is the timestamp passed in the request) to make - // sure everything is set up. We intentionally don't care about the spans in - // the checkpoints, just verifying that something has made it past the - // initial scan and is running. - waitForCheckpoint := func(ts hlc.Timestamp) { - t.Helper() - for _, rangeFeedCh := range rangeFeedChs { - checkpointed := false - for !checkpointed { - select { - case event := <-rangeFeedCh: - if c := event.Checkpoint; c != nil && ts.Less(c.ResolvedTS) { - checkpointed = true - } - case err := <-rangeFeedErrC: - t.Fatal(err) - } - } - } - } - waitForCheckpoint(ts1) - - // Clear the closed timestamp storage on each server. This simulates the case - // where a closed timestamp message is lost or a node restarts. To recover, - // the servers will need to request an update from the leaseholder. - for i := 0; i < tc.NumServers(); i++ { - stores := tc.Server(i).GetStores().(*kvserver.Stores) - err := stores.VisitStores(func(s *kvserver.Store) error { - s.ClearClosedTimestampStorage() - return nil - }) - require.NoError(t, err) - } - - // Wait for another RangeFeed checkpoint after the store was cleared. Without - // RangeFeed nudging closed timestamps, this doesn't happen on its own. Again, - // we intentionally don't care about the spans in the checkpoints, just - // verifying that something has made it past the cleared time. - ts2 := tc.Server(0).Clock().Now() - waitForCheckpoint(ts2) - - // Make sure the RangeFeed hasn't errored yet. - select { - case err := <-rangeFeedErrC: - t.Fatal(err) - default: - } - // Now cancel it and wait for it to shut down. - rangeFeedCancel() -} - // Test that a rangefeed registration receives checkpoints even if the lease // expires at some point. In other words, test that the rangefeed forces the // range to maintain a lease (i.e. renew the lease when the old one expires). In diff --git a/pkg/kv/kvserver/replica_tscache.go b/pkg/kv/kvserver/replica_tscache.go index 103f1f73c019..098ca40f0a71 100644 --- a/pkg/kv/kvserver/replica_tscache.go +++ b/pkg/kv/kvserver/replica_tscache.go @@ -577,7 +577,7 @@ func (r *Replica) GetCurrentReadSummary(ctx context.Context) rspb.ReadSummary { // Forward the read summary by the range's closed timestamp, because any // replica could have served reads below this time. We also return the // closed timestamp separately, in case callers want it split out. - closedTS := r.GetClosedTimestampV2(ctx) + closedTS := r.maxClosed(ctx) sum.Merge(rspb.FromTimestamp(closedTS)) return sum } diff --git a/pkg/kv/kvserver/replica_write.go b/pkg/kv/kvserver/replica_write.go index e40fd58cb793..0bf64418f527 100644 --- a/pkg/kv/kvserver/replica_write.go +++ b/pkg/kv/kvserver/replica_write.go @@ -18,7 +18,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" @@ -89,9 +88,6 @@ func (r *Replica) executeWriteBatch( // timestamps, which can help avoid uncertainty restarts. localUncertaintyLimit := observedts.ComputeLocalUncertaintyLimit(ba.Txn, st) - minTS, untrack := r.store.cfg.ClosedTimestamp.Tracker.Track(ctx) - defer untrack(ctx, 0, 0, 0) // covers all error returns below - // Start tracking this request if it is an MVCC write (i.e. if it's the kind // of request that needs to obey the closed timestamp). The act of tracking // also gives us a closed timestamp, which we must ensure to evaluate above @@ -99,17 +95,15 @@ func (r *Replica) executeWriteBatch( // accordingly if necessary. We need to start tracking this request before we // know the final write timestamp at which this request will evaluate because // we need to atomically read the closed timestamp and start to be tracked. - // TODO(andrei): The timestamp cache (and also the "old closed timestamp - // mechanism" in the form of minTS) might bump us above the timestamp at which + // TODO(andrei): The timestamp cache might bump us above the timestamp at which // we're registering with the proposalBuf. In that case, this request will be // tracked at an unnecessarily low timestamp which can block the closing of // this low timestamp for no reason. We should refactor such that the request // starts being tracked after we apply the timestamp cache. var tok TrackedRequestToken + var minTS hlc.Timestamp if ba.IsIntentWrite() { - var minTS2 hlc.Timestamp - minTS2, tok = r.mu.proposalBuf.TrackEvaluatingRequest(ctx, ba.WriteTimestamp()) - minTS.Forward(minTS2) + minTS, tok = r.mu.proposalBuf.TrackEvaluatingRequest(ctx, ba.WriteTimestamp()) } defer tok.DoneIfNotMoved(ctx) @@ -165,15 +159,6 @@ func (r *Replica) executeWriteBatch( } g = nil // ownership passed to Raft, prevent misuse - // A max lease index of zero is returned when no proposal was made or a lease - // was proposed. In case no proposal was made or a lease was proposed, we - // don't need to communicate a MLAI. Furthermore, for lease proposals we - // cannot communicate under the lease's epoch. Instead the code calls EmitMLAI - // explicitly as a side effect of stepping up as leaseholder. - if maxLeaseIndex != 0 { - untrack(ctx, ctpb.Epoch(st.Lease.Epoch), r.RangeID, ctpb.LAI(maxLeaseIndex)) - } - // We are done with pre-Raft evaluation at this point, and have to release the // read-only command lock to avoid deadlocks during Raft evaluation. r.readOnlyCmdMu.RUnlock() diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index 36fbece41b1d..3a0e60e3af8d 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -35,8 +35,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangecache" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/container" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/sidetransport" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/idalloc" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/intentresolver" @@ -208,7 +206,6 @@ func TestStoreConfig(clock *hlc.Clock) StoreConfig { CoalescedHeartbeatsInterval: 50 * time.Millisecond, ScanInterval: 10 * time.Minute, HistogramWindowInterval: metric.TestSampleInterval, - ClosedTimestamp: container.NoopContainer(), ProtectedTimestampCache: protectedts.EmptyCache(clock), } @@ -648,7 +645,6 @@ type StoreConfig struct { RPCContext *rpc.Context RangeDescriptorCache *rangecache.RangeCache - ClosedTimestamp *container.Container ClosedTimestampSender *sidetransport.Sender ClosedTimestampReceiver sidetransportReceiver @@ -1594,7 +1590,6 @@ func (s *Store) Start(ctx context.Context, stopper *stop.Stopper) error { } // Connect rangefeeds to closed timestamp updates. - s.startClosedTimestampRangefeedSubscriber(ctx) s.startRangefeedUpdater(ctx) if s.replicateQueue != nil { @@ -1791,67 +1786,6 @@ func (s *Store) startLeaseRenewer(ctx context.Context) { }) } -// startClosedTimestampRangefeedSubscriber establishes a new ClosedTimestamp -// subscription and runs an infinite loop to listen for closed timestamp updates -// and inform Replicas with active Rangefeeds about them. -func (s *Store) startClosedTimestampRangefeedSubscriber(ctx context.Context) { - // NB: We can't use Stopper.RunWorker because doing so would race with - // calling Stopper.Stop. We give the subscription channel a small capacity - // to avoid blocking the closed timestamp goroutine. - ch := make(chan ctpb.Entry, 8) - const name = "closedts-rangefeed-subscriber" - if err := s.stopper.RunAsyncTask(ctx, name, func(ctx context.Context) { - s.cfg.ClosedTimestamp.Provider.Subscribe(ctx, ch) - }); err != nil { - return - } - - _ = s.stopper.RunAsyncTask(ctx, "ct-subscriber", func(ctx context.Context) { - var replIDs []roachpb.RangeID - for { - if s.cfg.Settings.Version.IsActive(ctx, clusterversion.ClosedTimestampsRaftTransport) { - // The startRangefeedUpdater goroutine takes over. - return - } - select { - case <-ch: - // Drain all notifications from the channel. - loop: - for { - select { - case _, ok := <-ch: - if !ok { - break loop - } - default: - break loop - } - } - - // Gather replicas to notify under lock. - s.rangefeedReplicas.Lock() - for replID := range s.rangefeedReplicas.m { - replIDs = append(replIDs, replID) - } - s.rangefeedReplicas.Unlock() - - // Notify each replica with an active rangefeed to - // check for an updated closed timestamp. - for _, replID := range replIDs { - repl, err := s.GetReplica(replID) - if err != nil { - continue - } - repl.handleClosedTimestampUpdate(ctx) - } - replIDs = replIDs[:0] - case <-s.stopper.ShouldQuiesce(): - return - } - } - }) -} - // startRangefeedUpdater periodically informs all the replicas with rangefeeds // about closed timestamp updates. func (s *Store) startRangefeedUpdater(ctx context.Context) { @@ -1892,9 +1826,6 @@ func (s *Store) startRangefeedUpdater(ctx context.Context) { select { case <-timer.C: timer.Read = true - if !s.cfg.Settings.Version.IsActive(ctx, clusterversion.ClosedTimestampsRaftTransport) { - continue - } s.rangefeedReplicas.Lock() replIDs = replIDs[:0] for replID := range s.rangefeedReplicas.m { @@ -1908,7 +1839,7 @@ func (s *Store) startRangefeedUpdater(ctx context.Context) { if r == nil { continue } - r.handleClosedTimestampUpdate(ctx) + r.handleClosedTimestampUpdate(ctx, r.maxClosed(ctx)) } case <-confCh: // Loop around to use the updated timer. @@ -2619,8 +2550,8 @@ func (s *Store) updateReplicationGauges(ctx context.Context) error { if w := metrics.LockTableMetrics.TopKLocksByWaiters[0].Waiters; w > maxLockWaitQueueWaitersForLock { maxLockWaitQueueWaitersForLock = w } - mc, ok := rep.maxClosed(ctx) - if ok && (minMaxClosedTS.IsEmpty() || mc.Less(minMaxClosedTS)) { + mc := rep.maxClosed(ctx) + if minMaxClosedTS.IsEmpty() || mc.Less(minMaxClosedTS) { minMaxClosedTS = mc } return true // more @@ -2651,9 +2582,6 @@ func (s *Store) updateReplicationGauges(ctx context.Context) error { nanos := timeutil.Since(minMaxClosedTS.GoTime()).Nanoseconds() s.metrics.ClosedTimestampMaxBehindNanos.Update(nanos) } - s.metrics.ClosedTimestampFailuresToClose.Update( - s.cfg.ClosedTimestamp.Tracker.FailedCloseAttempts(), - ) s.metrics.RaftEnqueuedPending.Update(s.cfg.Transport.queuedMessageCount()) diff --git a/pkg/kv/kvserver/store_raft.go b/pkg/kv/kvserver/store_raft.go index c2a27d18aacc..65b2eb89029f 100644 --- a/pkg/kv/kvserver/store_raft.go +++ b/pkg/kv/kvserver/store_raft.go @@ -655,8 +655,6 @@ func (s *Store) updateLivenessMap() { nextMap := s.cfg.NodeLiveness.GetIsLiveMap() for nodeID, entry := range nextMap { if entry.IsLive { - // Make sure we ask all live nodes for closed timestamp updates. - s.cfg.ClosedTimestamp.Clients.EnsureClient(nodeID) continue } // Liveness claims that this node is down, but ConnHealth gets the last say diff --git a/pkg/kv/kvserver/store_split.go b/pkg/kv/kvserver/store_split.go index ed03315fc134..39ad17c356eb 100644 --- a/pkg/kv/kvserver/store_split.go +++ b/pkg/kv/kvserver/store_split.go @@ -153,7 +153,7 @@ func splitPreApply( // the hazard and ensures that no replica on the RHS is created with an // initialMaxClosed that could be violated by a proposal on the RHS's // initial leaseholder. See #44878. - initialMaxClosed, _ := r.maxClosed(ctx) + initialMaxClosed := r.maxClosed(ctx) rightRepl.mu.Lock() rightRepl.mu.initialMaxClosed = initialMaxClosed rightRepl.mu.Unlock() diff --git a/pkg/rpc/nodedialer/BUILD.bazel b/pkg/rpc/nodedialer/BUILD.bazel index bcc9a6fd0348..2a1fd63165c6 100644 --- a/pkg/rpc/nodedialer/BUILD.bazel +++ b/pkg/rpc/nodedialer/BUILD.bazel @@ -7,8 +7,6 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/kv/kvbase", - "//pkg/kv/kvserver/closedts", - "//pkg/kv/kvserver/closedts/ctpb", "//pkg/roachpb:with-mocks", "//pkg/rpc", "//pkg/util/grpcutil", diff --git a/pkg/rpc/nodedialer/nodedialer.go b/pkg/rpc/nodedialer/nodedialer.go index 1d42d308d557..835164178362 100644 --- a/pkg/rpc/nodedialer/nodedialer.go +++ b/pkg/rpc/nodedialer/nodedialer.go @@ -19,8 +19,6 @@ import ( circuit "github.com/cockroachdb/circuitbreaker" "github.com/cockroachdb/cockroach/pkg/kv/kvbase" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/util/grpcutil" @@ -258,24 +256,3 @@ func (n *Dialer) Latency(nodeID roachpb.NodeID) (time.Duration, error) { } return latency, nil } - -type dialerAdapter Dialer - -func (da *dialerAdapter) Ready(nodeID roachpb.NodeID) bool { - return (*Dialer)(da).GetCircuitBreaker(nodeID, rpc.DefaultClass).Ready() -} - -func (da *dialerAdapter) Dial(ctx context.Context, nodeID roachpb.NodeID) (ctpb.Client, error) { - c, err := (*Dialer)(da).Dial(ctx, nodeID, rpc.DefaultClass) - if err != nil { - return nil, err - } - return ctpb.NewClosedTimestampClient(c).Get(ctx) -} - -var _ closedts.Dialer = (*Dialer)(nil).CTDialer() - -// CTDialer wraps the NodeDialer into a closedts.Dialer. -func (n *Dialer) CTDialer() closedts.Dialer { - return (*dialerAdapter)(n) -} diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index c43376d0a443..3d759af8ef61 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -75,7 +75,6 @@ go_library( "//pkg/kv/kvclient/rangefeed:with-mocks", "//pkg/kv/kvprober", "//pkg/kv/kvserver", - "//pkg/kv/kvserver/closedts/container", "//pkg/kv/kvserver/closedts/ctpb", "//pkg/kv/kvserver/closedts/sidetransport", "//pkg/kv/kvserver/kvserverbase", diff --git a/pkg/server/authentication_test.go b/pkg/server/authentication_test.go index fcf0cf192213..62db06c1ded0 100644 --- a/pkg/server/authentication_test.go +++ b/pkg/server/authentication_test.go @@ -753,11 +753,11 @@ func TestGRPCAuthentication(t *testing.T) { return err }}, {"closedTimestamp", func(ctx context.Context, conn *grpc.ClientConn) error { - stream, err := ctpb.NewClosedTimestampClient(conn).Get(ctx) + stream, err := ctpb.NewSideTransportClient(conn).PushUpdates(ctx) if err != nil { return err } - _ = stream.Send(&ctpb.Reaction{}) + _ = stream.Send(&ctpb.Update{}) _, err = stream.Recv() return err }}, diff --git a/pkg/server/node.go b/pkg/server/node.go index 56a89dfc3645..de589dad8be2 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -389,9 +389,6 @@ func (n *Node) start( return errors.Errorf("couldn't gossip descriptor for node %d: %s", n.Descriptor.NodeID, err) } - // Start the closed timestamp subsystem. - n.storeCfg.ClosedTimestamp.Start(n.Descriptor.NodeID) - // Create stores from the engines that were already initialized. for _, e := range state.initializedEngines { s := kvserver.NewStore(ctx, n.storeCfg, e, &n.Descriptor) diff --git a/pkg/server/server.go b/pkg/server/server.go index 63e19a8a94be..85fa1c8653b3 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -45,7 +45,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" "github.com/cockroachdb/cockroach/pkg/kv/kvprober" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/container" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/sidetransport" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" @@ -557,10 +556,6 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { return nil, err } - // Break a circular dependency: we need a Node to make a StoreConfig (for - // ClosedTimestamp), but the Node needs a StoreConfig to be made. - var lateBoundNode *Node - // Break a circular dependency: we need the rootSQLMemoryMonitor to construct // the KV memory monitor for the StoreConfig. sqlMonitorAndMetrics := newRootSQLMemoryMonitor(monitorAndMetricsOptions{ @@ -598,28 +593,6 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { TimeSeriesDataStore: tsDB, ClosedTimestampSender: ctSender, ClosedTimestampReceiver: ctReceiver, - - // Initialize the closed timestamp subsystem. Note that it won't - // be ready until it is .Start()ed, but the grpc server can be - // registered early. - ClosedTimestamp: container.NewContainer(container.Config{ - Settings: st, - Stopper: stopper, - Clock: nodeLiveness.AsLiveClock(), - // NB: s.node is not defined at this point, but it will be - // before this is ever called. - Refresh: func(rangeIDs ...roachpb.RangeID) { - for _, rangeID := range rangeIDs { - repl, _, err := lateBoundNode.stores.GetReplicaForRangeID(ctx, rangeID) - if err != nil || repl == nil { - continue - } - repl.EmitMLAI() - } - }, - Dialer: nodeDialer.CTDialer(), - }), - ExternalStorage: externalStorage, ExternalStorageFromURI: externalStorageFromURI, ProtectedTimestampCache: protectedtsProvider, @@ -658,11 +631,9 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { gcoords.Regular.GetWorkQueue(admission.KVWork), gcoords.Stores, tenantUsage, ) - lateBoundNode = node roachpb.RegisterInternalServer(grpcServer.Server, node) kvserver.RegisterPerReplicaServer(grpcServer.Server, node.perReplicaServer) kvserver.RegisterPerStoreServer(grpcServer.Server, node.perReplicaServer) - node.storeCfg.ClosedTimestamp.RegisterClosedTimestampServer(grpcServer.Server) ctpb.RegisterSideTransportServer(grpcServer.Server, ctReceiver) replicationReporter := reports.NewReporter( db, node.stores, storePool, st, nodeLiveness, internalExecutor)