diff --git a/pkg/cli/debug_check_store.go b/pkg/cli/debug_check_store.go index 10a1529faed1..9799a570052a 100644 --- a/pkg/cli/debug_check_store.go +++ b/pkg/cli/debug_check_store.go @@ -252,7 +252,7 @@ func checkStoreRaftState( return err } getReplicaInfo(rangeID).committedIndex = hs.Commit - case bytes.Equal(suffix, keys.LocalRaftTruncatedStateLegacySuffix): + case bytes.Equal(suffix, keys.LocalRaftTruncatedStateSuffix): var trunc roachpb.RaftTruncatedState if err := kv.Value.GetProto(&trunc); err != nil { return err @@ -264,12 +264,6 @@ func checkStoreRaftState( return err } getReplicaInfo(rangeID).appliedIndex = state.RaftAppliedIndex - case bytes.Equal(suffix, keys.LocalRaftAppliedIndexLegacySuffix): - idx, err := kv.Value.GetInt() - if err != nil { - return err - } - getReplicaInfo(rangeID).appliedIndex = uint64(idx) case bytes.Equal(suffix, keys.LocalRaftLogSuffix): _, index, err := encoding.DecodeUint64Ascending(detail) if err != nil { diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 04fcb942998d..f58ad2418d8c 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -180,32 +180,6 @@ const ( // // Start21_1 demarcates work towards CockroachDB v21.1. Start21_1 - // replacedTruncatedAndRangeAppliedStateMigration stands in for - // TruncatedAndRangeAppliedStateMigration which was re-introduced after the - // migration job was introduced. This is necessary because the jobs - // infrastructure used to run this migration in v21.1 and its later alphas - // was introduced after this version was first introduced. Later code in the - // release relies on the job to run the migration but the job relies on - // its startup migrations having been run. Versions associated with long - // running migrations must follow deletedLongRunningMigrations. - replacedTruncatedAndRangeAppliedStateMigration - // replacedPostTruncatedAndRangeAppliedStateMigration is like the above - // version. See its comment. - replacedPostTruncatedAndRangeAppliedStateMigration - // TruncatedAndRangeAppliedStateMigration is part of the migration to stop - // using the legacy truncated state within KV. After the migration, we'll be - // using the unreplicated truncated state and the RangeAppliedState on all - // ranges. Callers that wish to assert on there no longer being any legacy - // will be able to do so after PostTruncatedAndRangeAppliedStateMigration is - // active. This lets remove any holdover code handling the possibility of - // replicated truncated state in 21.2. - // - // TODO(irfansharif): Do the above in 21.2. - TruncatedAndRangeAppliedStateMigration - // PostTruncatedAndRangeAppliedStateMigration is used to purge all replicas - // using the replicated legacy TruncatedState. It's also used in asserting - // that no replicated truncated state representation is found. - PostTruncatedAndRangeAppliedStateMigration // TracingVerbosityIndependentSemantics marks a change in which trace spans // are propagated across RPC boundaries independently of their verbosity setting. // This requires a version gate this violates implicit assumptions in v20.2. @@ -349,22 +323,6 @@ var versionsSingleton = keyedVersions{ Key: Start21_1, Version: roachpb.Version{Major: 20, Minor: 2, Internal: 2}, }, - { - Key: replacedTruncatedAndRangeAppliedStateMigration, - Version: roachpb.Version{Major: 20, Minor: 2, Internal: 14}, - }, - { - Key: replacedPostTruncatedAndRangeAppliedStateMigration, - Version: roachpb.Version{Major: 20, Minor: 2, Internal: 16}, - }, - { - Key: TruncatedAndRangeAppliedStateMigration, - Version: roachpb.Version{Major: 20, Minor: 2, Internal: 22}, - }, - { - Key: PostTruncatedAndRangeAppliedStateMigration, - Version: roachpb.Version{Major: 20, Minor: 2, Internal: 24}, - }, { Key: TracingVerbosityIndependentSemantics, Version: roachpb.Version{Major: 20, Minor: 2, Internal: 28}, diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index 866500386868..2966163f6b38 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -14,48 +14,44 @@ func _() { _ = x[HBAForNonTLS-3] _ = x[V20_2-4] _ = x[Start21_1-5] - _ = x[replacedTruncatedAndRangeAppliedStateMigration-6] - _ = x[replacedPostTruncatedAndRangeAppliedStateMigration-7] - _ = x[TruncatedAndRangeAppliedStateMigration-8] - _ = x[PostTruncatedAndRangeAppliedStateMigration-9] - _ = x[TracingVerbosityIndependentSemantics-10] - _ = x[V21_1-11] - _ = x[Start21_1PLUS-12] - _ = x[Start21_2-13] - _ = x[JoinTokensTable-14] - _ = x[AcquisitionTypeInLeaseHistory-15] - _ = x[SerializeViewUDTs-16] - _ = x[ExpressionIndexes-17] - _ = x[DeleteDeprecatedNamespaceTableDescriptorMigration-18] - _ = x[FixDescriptors-19] - _ = x[SQLStatsTable-20] - _ = x[DatabaseRoleSettings-21] - _ = x[TenantUsageTable-22] - _ = x[SQLInstancesTable-23] - _ = x[NewRetryableRangefeedErrors-24] - _ = x[AlterSystemWebSessionsCreateIndexes-25] - _ = x[SeparatedIntentsMigration-26] - _ = x[PostSeparatedIntentsMigration-27] - _ = x[RetryJobsWithExponentialBackoff-28] - _ = x[RecordsBasedRegistry-29] - _ = x[AutoSpanConfigReconciliationJob-30] - _ = x[PreventNewInterleavedTables-31] - _ = x[EnsureNoInterleavedTables-32] - _ = x[DefaultPrivileges-33] - _ = x[ZonesTableForSecondaryTenants-34] - _ = x[UseKeyEncodeForHashShardedIndexes-35] - _ = x[DatabasePlacementPolicy-36] - _ = x[GeneratedAsIdentity-37] - _ = x[OnUpdateExpressions-38] - _ = x[SpanConfigurationsTable-39] - _ = x[BoundedStaleness-40] - _ = x[SQLStatsCompactionScheduledJob-41] - _ = x[DateAndIntervalStyle-42] + _ = x[TracingVerbosityIndependentSemantics-6] + _ = x[V21_1-7] + _ = x[Start21_1PLUS-8] + _ = x[Start21_2-9] + _ = x[JoinTokensTable-10] + _ = x[AcquisitionTypeInLeaseHistory-11] + _ = x[SerializeViewUDTs-12] + _ = x[ExpressionIndexes-13] + _ = x[DeleteDeprecatedNamespaceTableDescriptorMigration-14] + _ = x[FixDescriptors-15] + _ = x[SQLStatsTable-16] + _ = x[DatabaseRoleSettings-17] + _ = x[TenantUsageTable-18] + _ = x[SQLInstancesTable-19] + _ = x[NewRetryableRangefeedErrors-20] + _ = x[AlterSystemWebSessionsCreateIndexes-21] + _ = x[SeparatedIntentsMigration-22] + _ = x[PostSeparatedIntentsMigration-23] + _ = x[RetryJobsWithExponentialBackoff-24] + _ = x[RecordsBasedRegistry-25] + _ = x[AutoSpanConfigReconciliationJob-26] + _ = x[PreventNewInterleavedTables-27] + _ = x[EnsureNoInterleavedTables-28] + _ = x[DefaultPrivileges-29] + _ = x[ZonesTableForSecondaryTenants-30] + _ = x[UseKeyEncodeForHashShardedIndexes-31] + _ = x[DatabasePlacementPolicy-32] + _ = x[GeneratedAsIdentity-33] + _ = x[OnUpdateExpressions-34] + _ = x[SpanConfigurationsTable-35] + _ = x[BoundedStaleness-36] + _ = x[SQLStatsCompactionScheduledJob-37] + _ = x[DateAndIntervalStyle-38] } -const _Key_name = "Start20_2MinPasswordLengthCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1replacedTruncatedAndRangeAppliedStateMigrationreplacedPostTruncatedAndRangeAppliedStateMigrationTruncatedAndRangeAppliedStateMigrationPostTruncatedAndRangeAppliedStateMigrationTracingVerbosityIndependentSemanticsV21_1Start21_1PLUSStart21_2JoinTokensTableAcquisitionTypeInLeaseHistorySerializeViewUDTsExpressionIndexesDeleteDeprecatedNamespaceTableDescriptorMigrationFixDescriptorsSQLStatsTableDatabaseRoleSettingsTenantUsageTableSQLInstancesTableNewRetryableRangefeedErrorsAlterSystemWebSessionsCreateIndexesSeparatedIntentsMigrationPostSeparatedIntentsMigrationRetryJobsWithExponentialBackoffRecordsBasedRegistryAutoSpanConfigReconciliationJobPreventNewInterleavedTablesEnsureNoInterleavedTablesDefaultPrivilegesZonesTableForSecondaryTenantsUseKeyEncodeForHashShardedIndexesDatabasePlacementPolicyGeneratedAsIdentityOnUpdateExpressionsSpanConfigurationsTableBoundedStalenessSQLStatsCompactionScheduledJobDateAndIntervalStyle" +const _Key_name = "Start20_2MinPasswordLengthCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1TracingVerbosityIndependentSemanticsV21_1Start21_1PLUSStart21_2JoinTokensTableAcquisitionTypeInLeaseHistorySerializeViewUDTsExpressionIndexesDeleteDeprecatedNamespaceTableDescriptorMigrationFixDescriptorsSQLStatsTableDatabaseRoleSettingsTenantUsageTableSQLInstancesTableNewRetryableRangefeedErrorsAlterSystemWebSessionsCreateIndexesSeparatedIntentsMigrationPostSeparatedIntentsMigrationRetryJobsWithExponentialBackoffRecordsBasedRegistryAutoSpanConfigReconciliationJobPreventNewInterleavedTablesEnsureNoInterleavedTablesDefaultPrivilegesZonesTableForSecondaryTenantsUseKeyEncodeForHashShardedIndexesDatabasePlacementPolicyGeneratedAsIdentityOnUpdateExpressionsSpanConfigurationsTableBoundedStalenessSQLStatsCompactionScheduledJobDateAndIntervalStyle" -var _Key_index = [...]uint16{0, 9, 26, 46, 58, 63, 72, 118, 168, 206, 248, 284, 289, 302, 311, 326, 355, 372, 389, 438, 452, 465, 485, 501, 518, 545, 580, 605, 634, 665, 685, 716, 743, 768, 785, 814, 847, 870, 889, 908, 931, 947, 977, 997} +var _Key_index = [...]uint16{0, 9, 26, 46, 58, 63, 72, 108, 113, 126, 135, 150, 179, 196, 213, 262, 276, 289, 309, 325, 342, 369, 404, 429, 458, 489, 509, 540, 567, 592, 609, 638, 671, 694, 713, 732, 755, 771, 801, 821} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/keys/constants.go b/pkg/keys/constants.go index e65d1be779b8..fd84110f3750 100644 --- a/pkg/keys/constants.go +++ b/pkg/keys/constants.go @@ -80,17 +80,12 @@ var ( // LocalRangeAppliedStateSuffix is the suffix for the range applied state // key. LocalRangeAppliedStateSuffix = []byte("rask") - // LocalRaftAppliedIndexLegacySuffix is the suffix for the raft applied index. - LocalRaftAppliedIndexLegacySuffix = []byte("rfta") - // LocalRaftTruncatedStateLegacySuffix is the suffix for the legacy - // RaftTruncatedState. See VersionUnreplicatedRaftTruncatedState. + // LocalRaftTruncatedStateSuffix is the suffix for the + // RaftTruncatedState. // Note: This suffix is also used for unreplicated Range-ID keys. - LocalRaftTruncatedStateLegacySuffix = []byte("rftt") + LocalRaftTruncatedStateSuffix = []byte("rftt") // LocalRangeLeaseSuffix is the suffix for a range lease. LocalRangeLeaseSuffix = []byte("rll-") - // LocalLeaseAppliedIndexLegacySuffix is the suffix for the applied lease - // index. - LocalLeaseAppliedIndexLegacySuffix = []byte("rlla") // LocalRangePriorReadSummarySuffix is the suffix for a range's prior read // summary. LocalRangePriorReadSummarySuffix = []byte("rprs") diff --git a/pkg/keys/doc.go b/pkg/keys/doc.go index a4f00c5dabb8..edc93fee371c 100644 --- a/pkg/keys/doc.go +++ b/pkg/keys/doc.go @@ -185,16 +185,12 @@ var _ = [...]interface{}{ // range as a whole. Though they are replicated, they are unaddressable. // Typical examples are MVCC stats and the abort span. They all share // `LocalRangeIDPrefix` and `LocalRangeIDReplicatedInfix`. - AbortSpanKey, // "abc-" - RangeGCThresholdKey, // "lgc-" - RangeAppliedStateKey, // "rask" - RaftAppliedIndexLegacyKey, // "rfta" - RaftTruncatedStateLegacyKey, // "rftt" - RangeLeaseKey, // "rll-" - LeaseAppliedIndexLegacyKey, // "rlla" - RangePriorReadSummaryKey, // "rprs" - RangeVersionKey, // "rver" - RangeStatsLegacyKey, // "stat" + AbortSpanKey, // "abc-" + RangeGCThresholdKey, // "lgc-" + RangeAppliedStateKey, // "rask" + RangeLeaseKey, // "rll-" + RangePriorReadSummaryKey, // "rprs" + RangeVersionKey, // "rver" // 2. Unreplicated range-ID local keys: These contain metadata that // pertain to just one replica of a range. They are unreplicated and diff --git a/pkg/keys/keys.go b/pkg/keys/keys.go index 4f0cb8c15851..7485adafa3d6 100644 --- a/pkg/keys/keys.go +++ b/pkg/keys/keys.go @@ -239,33 +239,11 @@ func DecodeAbortSpanKey(key roachpb.Key, dest []byte) (uuid.UUID, error) { // RangeAppliedStateKey returns a system-local key for the range applied state key. // This key has subsumed the responsibility of the following three keys: -// - RaftAppliedIndexLegacyKey -// - LeaseAppliedIndexLegacyKey // - RangeStatsLegacyKey func RangeAppliedStateKey(rangeID roachpb.RangeID) roachpb.Key { return MakeRangeIDPrefixBuf(rangeID).RangeAppliedStateKey() } -// RaftAppliedIndexLegacyKey returns a system-local key for a raft applied index. -// The key is no longer written to. Its responsibility has been subsumed by the -// RangeAppliedStateKey. -func RaftAppliedIndexLegacyKey(rangeID roachpb.RangeID) roachpb.Key { - return MakeRangeIDPrefixBuf(rangeID).RaftAppliedIndexLegacyKey() -} - -// LeaseAppliedIndexLegacyKey returns a system-local key for a lease applied index. -// The key is no longer written to. Its responsibility has been subsumed by the -// RangeAppliedStateKey. -func LeaseAppliedIndexLegacyKey(rangeID roachpb.RangeID) roachpb.Key { - return MakeRangeIDPrefixBuf(rangeID).LeaseAppliedIndexLegacyKey() -} - -// RaftTruncatedStateLegacyKey returns a system-local key for a RaftTruncatedState. -// See VersionUnreplicatedRaftTruncatedState. -func RaftTruncatedStateLegacyKey(rangeID roachpb.RangeID) roachpb.Key { - return MakeRangeIDPrefixBuf(rangeID).RaftTruncatedStateLegacyKey() -} - // RangeLeaseKey returns a system-local key for a range lease. func RangeLeaseKey(rangeID roachpb.RangeID) roachpb.Key { return MakeRangeIDPrefixBuf(rangeID).RangeLeaseKey() @@ -277,13 +255,6 @@ func RangePriorReadSummaryKey(rangeID roachpb.RangeID) roachpb.Key { return MakeRangeIDPrefixBuf(rangeID).RangePriorReadSummaryKey() } -// RangeStatsLegacyKey returns the key for accessing the MVCCStats struct for -// the specified Range ID. The key is no longer written to. Its responsibility -// has been subsumed by the RangeAppliedStateKey. -func RangeStatsLegacyKey(rangeID roachpb.RangeID) roachpb.Key { - return MakeRangeIDPrefixBuf(rangeID).RangeStatsLegacyKey() -} - // RangeGCThresholdKey returns a system-local key for last used GC threshold on the // user keyspace. Reads and writes <= this timestamp will not be served. func RangeGCThresholdKey(rangeID roachpb.RangeID) roachpb.Key { @@ -951,23 +922,6 @@ func (b RangeIDPrefixBuf) RangeAppliedStateKey() roachpb.Key { return append(b.replicatedPrefix(), LocalRangeAppliedStateSuffix...) } -// RaftAppliedIndexLegacyKey returns a system-local key for a raft applied index. -// See comment on RaftAppliedIndexLegacyKey function. -func (b RangeIDPrefixBuf) RaftAppliedIndexLegacyKey() roachpb.Key { - return append(b.replicatedPrefix(), LocalRaftAppliedIndexLegacySuffix...) -} - -// LeaseAppliedIndexLegacyKey returns a system-local key for a lease applied index. -// See comment on LeaseAppliedIndexLegacyKey function. -func (b RangeIDPrefixBuf) LeaseAppliedIndexLegacyKey() roachpb.Key { - return append(b.replicatedPrefix(), LocalLeaseAppliedIndexLegacySuffix...) -} - -// RaftTruncatedStateLegacyKey returns a system-local key for a RaftTruncatedState. -func (b RangeIDPrefixBuf) RaftTruncatedStateLegacyKey() roachpb.Key { - return append(b.replicatedPrefix(), LocalRaftTruncatedStateLegacySuffix...) -} - // RangeLeaseKey returns a system-local key for a range lease. func (b RangeIDPrefixBuf) RangeLeaseKey() roachpb.Key { return append(b.replicatedPrefix(), LocalRangeLeaseSuffix...) @@ -979,13 +933,6 @@ func (b RangeIDPrefixBuf) RangePriorReadSummaryKey() roachpb.Key { return append(b.replicatedPrefix(), LocalRangePriorReadSummarySuffix...) } -// RangeStatsLegacyKey returns the key for accessing the MVCCStats struct -// for the specified Range ID. -// See comment on RangeStatsLegacyKey function. -func (b RangeIDPrefixBuf) RangeStatsLegacyKey() roachpb.Key { - return append(b.replicatedPrefix(), LocalRangeStatsLegacySuffix...) -} - // RangeGCThresholdKey returns a system-local key for the GC threshold. func (b RangeIDPrefixBuf) RangeGCThresholdKey() roachpb.Key { return append(b.replicatedPrefix(), LocalRangeGCThresholdSuffix...) @@ -1003,7 +950,7 @@ func (b RangeIDPrefixBuf) RangeTombstoneKey() roachpb.Key { // RaftTruncatedStateKey returns a system-local key for a RaftTruncatedState. func (b RangeIDPrefixBuf) RaftTruncatedStateKey() roachpb.Key { - return append(b.unreplicatedPrefix(), LocalRaftTruncatedStateLegacySuffix...) + return append(b.unreplicatedPrefix(), LocalRaftTruncatedStateSuffix...) } // RaftHardStateKey returns a system-local key for a Raft HardState. diff --git a/pkg/keys/keys_test.go b/pkg/keys/keys_test.go index f92d803dcc84..c54fc0c55ab8 100644 --- a/pkg/keys/keys_test.go +++ b/pkg/keys/keys_test.go @@ -156,10 +156,7 @@ func TestKeyAddressError(t *testing.T) { "local range ID key .* is not addressable": { AbortSpanKey(0, uuid.MakeV4()), RangeTombstoneKey(0), - RaftAppliedIndexLegacyKey(0), - RaftTruncatedStateLegacyKey(0), RangeLeaseKey(0), - RangeStatsLegacyKey(0), RaftHardStateKey(0), RaftLogPrefix(0), RaftLogKey(0, 0), diff --git a/pkg/keys/printer.go b/pkg/keys/printer.go index e60a0831104b..fc9515768f1b 100644 --- a/pkg/keys/printer.go +++ b/pkg/keys/printer.go @@ -160,13 +160,11 @@ var ( {name: "RangeTombstone", suffix: LocalRangeTombstoneSuffix}, {name: "RaftHardState", suffix: LocalRaftHardStateSuffix}, {name: "RangeAppliedState", suffix: LocalRangeAppliedStateSuffix}, - {name: "RaftAppliedIndex", suffix: LocalRaftAppliedIndexLegacySuffix}, - {name: "LeaseAppliedIndex", suffix: LocalLeaseAppliedIndexLegacySuffix}, {name: "RaftLog", suffix: LocalRaftLogSuffix, ppFunc: raftLogKeyPrint, psFunc: raftLogKeyParse, }, - {name: "RaftTruncatedState", suffix: LocalRaftTruncatedStateLegacySuffix}, + {name: "RaftTruncatedState", suffix: LocalRaftTruncatedStateSuffix}, {name: "RangeLastReplicaGCTimestamp", suffix: LocalRangeLastReplicaGCTimestampSuffix}, {name: "RangeLease", suffix: LocalRangeLeaseSuffix}, {name: "RangePriorReadSummary", suffix: LocalRangePriorReadSummarySuffix}, diff --git a/pkg/keys/printer_test.go b/pkg/keys/printer_test.go index 579ec2480005..50dbaf7a0287 100644 --- a/pkg/keys/printer_test.go +++ b/pkg/keys/printer_test.go @@ -69,13 +69,9 @@ func TestPrettyPrint(t *testing.T) { {keys.AbortSpanKey(roachpb.RangeID(1000001), txnID), fmt.Sprintf(`/Local/RangeID/1000001/r/AbortSpan/%q`, txnID), revertSupportUnknown}, {keys.RangeAppliedStateKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/RangeAppliedState", revertSupportUnknown}, - {keys.RaftAppliedIndexLegacyKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/RaftAppliedIndex", revertSupportUnknown}, - {keys.LeaseAppliedIndexLegacyKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/LeaseAppliedIndex", revertSupportUnknown}, - {keys.RaftTruncatedStateLegacyKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/RaftTruncatedState", revertSupportUnknown}, {keys.RaftTruncatedStateKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/u/RaftTruncatedState", revertSupportUnknown}, {keys.RangeLeaseKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/RangeLease", revertSupportUnknown}, {keys.RangePriorReadSummaryKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/RangePriorReadSummary", revertSupportUnknown}, - {keys.RangeStatsLegacyKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/RangeStats", revertSupportUnknown}, {keys.RangeGCThresholdKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/RangeGCThreshold", revertSupportUnknown}, {keys.RangeVersionKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/RangeVersion", revertSupportUnknown}, diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go index 90a2f5920bc1..06619f5029b7 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go @@ -983,34 +983,6 @@ func splitTriggerHelper( log.VEventf(ctx, 1, "LHS's GCThreshold of split is not set") } - // We're about to write the initial state for the replica. We migrated - // the formerly replicated truncated state into unreplicated keyspace - // in 19.1, but this range may still be using the replicated version - // and we need to make a decision about what to use for the RHS that - // is consistent across the followers: do for the RHS what the LHS - // does: if the LHS has the legacy key, initialize the RHS with a - // legacy key as well. - // - // See VersionUnreplicatedRaftTruncatedState. - truncStateType := stateloader.TruncatedStateUnreplicated - if found, err := storage.MVCCGetProto( - ctx, - batch, - keys.RaftTruncatedStateLegacyKey(rec.GetRangeID()), - hlc.Timestamp{}, - nil, - storage.MVCCGetOptions{}, - ); err != nil { - return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to load legacy truncated state") - } else if found { - truncStateType = stateloader.TruncatedStateLegacyReplicated - } - - replicaVersion, err := sl.LoadVersion(ctx, batch) - if err != nil { - return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to load GCThreshold") - } - // Writing the initial state is subtle since this also seeds the Raft // group. It becomes more subtle due to proposer-evaluated Raft. // @@ -1040,10 +1012,13 @@ func splitTriggerHelper( // HardState via a call to synthesizeRaftState. Here, we only call // writeInitialReplicaState which essentially writes a ReplicaState // only. - + replicaVersion, err := sl.LoadVersion(ctx, batch) + if err != nil { + return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to load replica version") + } *h.AbsPostSplitRight(), err = stateloader.WriteInitialReplicaState( ctx, batch, *h.AbsPostSplitRight(), split.RightDesc, rightLease, - *gcThreshold, truncStateType, replicaVersion, + *gcThreshold, replicaVersion, ) if err != nil { return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to write initial Replica state") diff --git a/pkg/kv/kvserver/batcheval/cmd_migrate.go b/pkg/kv/kvserver/batcheval/cmd_migrate.go index 288d8330c911..9cfe0294475f 100644 --- a/pkg/kv/kvserver/batcheval/cmd_migrate.go +++ b/pkg/kv/kvserver/batcheval/cmd_migrate.go @@ -20,7 +20,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/errors" ) @@ -42,8 +41,6 @@ func declareKeysMigrate( latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.RangeVersionKey(rs.GetRangeID())}) latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(rs.GetStartKey())}) - latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.RaftTruncatedStateLegacyKey(rs.GetRangeID())}) - lockSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.RaftTruncatedStateLegacyKey(rs.GetRangeID())}) } // migrationRegistry is a global registry of all KV-level migrations. See @@ -54,7 +51,6 @@ var migrationRegistry = make(map[roachpb.Version]migration) type migration func(context.Context, storage.ReadWriter, CommandArgs) (result.Result, error) func init() { - registerMigration(clusterversion.TruncatedAndRangeAppliedStateMigration, truncatedAndAppliedStateMigration) registerMigration(clusterversion.PostSeparatedIntentsMigration, postSeparatedIntentsMigration) } @@ -96,41 +92,6 @@ func Migrate( return pd, nil } -// truncatedAndRangeAppliedStateMigration lets us stop using the legacy -// replicated truncated state and start using the new RangeAppliedState for this -// specific range. -func truncatedAndAppliedStateMigration( - ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, -) (result.Result, error) { - var legacyTruncatedState roachpb.RaftTruncatedState - legacyKeyFound, err := storage.MVCCGetProto( - ctx, readWriter, keys.RaftTruncatedStateLegacyKey(cArgs.EvalCtx.GetRangeID()), - hlc.Timestamp{}, &legacyTruncatedState, storage.MVCCGetOptions{}, - ) - if err != nil { - return result.Result{}, err - } - - var pd result.Result - if legacyKeyFound { - // Time to migrate by deleting the legacy key. The downstream-of-Raft - // code will atomically rewrite the truncated state (supplied via the - // side effect) into the new unreplicated key. - if err := storage.MVCCDelete( - ctx, readWriter, cArgs.Stats, keys.RaftTruncatedStateLegacyKey(cArgs.EvalCtx.GetRangeID()), - hlc.Timestamp{}, nil, /* txn */ - ); err != nil { - return result.Result{}, err - } - pd.Replicated.State = &kvserverpb.ReplicaState{ - // We need to pass in a truncated state to enable the migration. - // Passing the same one is the easiest thing to do. - TruncatedState: &legacyTruncatedState, - } - } - return pd, nil -} - // postSeparatedIntentsMigration is the below-raft part of the migration for // interleaved to separated intents. It is a no-op as the only purpose of // running the Migrate command here is to clear out any orphaned replicas with diff --git a/pkg/kv/kvserver/batcheval/cmd_truncate_log.go b/pkg/kv/kvserver/batcheval/cmd_truncate_log.go index 19bd9c3138f2..f34dc1f9fc6f 100644 --- a/pkg/kv/kvserver/batcheval/cmd_truncate_log.go +++ b/pkg/kv/kvserver/batcheval/cmd_truncate_log.go @@ -19,7 +19,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -31,7 +30,6 @@ func init() { func declareKeysTruncateLog( rs ImmutableRangeState, _ roachpb.Header, req roachpb.Request, latchSpans, _ *spanset.SpanSet, ) { - latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.RaftTruncatedStateLegacyKey(rs.GetRangeID())}) prefix := keys.RaftLogPrefix(rs.GetRangeID()) latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: prefix, EndKey: prefix.PrefixEnd()}) } @@ -54,15 +52,6 @@ func TruncateLog( return result.Result{}, nil } - var legacyTruncatedState roachpb.RaftTruncatedState - legacyKeyFound, err := storage.MVCCGetProto( - ctx, readWriter, keys.RaftTruncatedStateLegacyKey(cArgs.EvalCtx.GetRangeID()), - hlc.Timestamp{}, &legacyTruncatedState, storage.MVCCGetOptions{}, - ) - if err != nil { - return result.Result{}, err - } - firstIndex, err := cArgs.EvalCtx.GetFirstIndex() if err != nil { return result.Result{}, errors.Wrap(err, "getting first index") @@ -130,18 +119,5 @@ func TruncateLog( } pd.Replicated.RaftLogDelta = ms.SysBytes - - if legacyKeyFound { - // Time to migrate by deleting the legacy key. The downstream-of-Raft - // code will atomically rewrite the truncated state (supplied via the - // side effect) into the new unreplicated key. - if err := storage.MVCCDelete( - ctx, readWriter, cArgs.Stats, keys.RaftTruncatedStateLegacyKey(cArgs.EvalCtx.GetRangeID()), - hlc.Timestamp{}, nil, /* txn */ - ); err != nil { - return result.Result{}, err - } - } - return pd, nil } diff --git a/pkg/kv/kvserver/batcheval/cmd_truncate_log_test.go b/pkg/kv/kvserver/batcheval/cmd_truncate_log_test.go index 3475f7bebfd5..76458372e8cb 100644 --- a/pkg/kv/kvserver/batcheval/cmd_truncate_log_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_truncate_log_test.go @@ -12,7 +12,6 @@ package batcheval import ( "context" - "fmt" "testing" "github.com/cockroachdb/cockroach/pkg/keys" @@ -22,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) func putTruncatedState( @@ -29,12 +29,8 @@ func putTruncatedState( eng storage.Engine, rangeID roachpb.RangeID, truncState roachpb.RaftTruncatedState, - legacy bool, ) { key := keys.RaftTruncatedStateKey(rangeID) - if legacy { - key = keys.RaftTruncatedStateLegacyKey(rangeID) - } if err := storage.MVCCPutProto( context.Background(), eng, nil, key, hlc.Timestamp{}, nil /* txn */, &truncState, @@ -45,75 +41,24 @@ func putTruncatedState( func readTruncStates( t *testing.T, eng storage.Engine, rangeID roachpb.RangeID, -) (legacy roachpb.RaftTruncatedState, unreplicated roachpb.RaftTruncatedState) { +) (truncatedState roachpb.RaftTruncatedState) { t.Helper() - legacyFound, err := storage.MVCCGetProto( - context.Background(), eng, keys.RaftTruncatedStateLegacyKey(rangeID), - hlc.Timestamp{}, &legacy, storage.MVCCGetOptions{}, - ) - if err != nil { - t.Fatal(err) - } - if legacyFound != (legacy != roachpb.RaftTruncatedState{}) { - t.Fatalf("legacy key found=%t but state is %+v", legacyFound, legacy) - } - - unreplicatedFound, err := storage.MVCCGetProto( + found, err := storage.MVCCGetProto( context.Background(), eng, keys.RaftTruncatedStateKey(rangeID), - hlc.Timestamp{}, &unreplicated, storage.MVCCGetOptions{}, + hlc.Timestamp{}, &truncatedState, storage.MVCCGetOptions{}, ) if err != nil { t.Fatal(err) } - if unreplicatedFound != (unreplicated != roachpb.RaftTruncatedState{}) { - t.Fatalf("unreplicated key found=%t but state is %+v", unreplicatedFound, unreplicated) - } + require.True(t, found) return } -const ( - expectationNeither = iota - expectationLegacy - expectationUnreplicated -) - -type unreplicatedTruncStateTest struct { - startsWithLegacy bool - exp int // see consts above -} - -func TestTruncateLogUnreplicatedTruncatedState(t *testing.T) { +func TestTruncateLog(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - // Check out the old clusterversion.VersionUnreplicatedRaftTruncatedState - // for information on what's being tested. The cluster version is gone, but - // the migration is done range by range and so it still exists. - - const ( - startsLegacy = true - startsUnreplicated = false - ) - - testCases := []unreplicatedTruncStateTest{ - // This is the case where we've already migrated. - {startsUnreplicated, expectationUnreplicated}, - // This is the case in which the migration is triggered. As a result, - // we see neither of the keys written. The new key will be written - // atomically as a side effect (outside of the scope of this test). - {startsLegacy, expectationNeither}, - } - - for _, tc := range testCases { - t.Run(fmt.Sprintf("%v", tc), func(t *testing.T) { - runUnreplicatedTruncatedState(t, tc) - }) - } -} - -func runUnreplicatedTruncatedState(t *testing.T, tc unreplicatedTruncStateTest) { ctx := context.Background() - const ( rangeID = 12 term = 10 @@ -134,8 +79,7 @@ func runUnreplicatedTruncatedState(t *testing.T, tc unreplicatedTruncStateTest) Term: term, } - // Put down the TruncatedState specified by the test case. - putTruncatedState(t, eng, rangeID, truncState, tc.startsWithLegacy) + putTruncatedState(t, eng, rangeID, truncState) // Send a truncation request. req := roachpb.TruncateLogRequest{ @@ -157,25 +101,13 @@ func runUnreplicatedTruncatedState(t *testing.T, tc unreplicatedTruncStateTest) Term: term, } - legacy, unreplicated := readTruncStates(t, eng, rangeID) - - switch tc.exp { - case expectationLegacy: - assert.Equal(t, expTruncState, legacy) - assert.Zero(t, unreplicated) - case expectationUnreplicated: - // The unreplicated key that we see should be the initial truncated - // state (it's only updated below Raft). - assert.Equal(t, truncState, unreplicated) - assert.Zero(t, legacy) - case expectationNeither: - assert.Zero(t, unreplicated) - assert.Zero(t, legacy) - default: - t.Fatalf("unknown expectation %d", tc.exp) - } + // The unreplicated key that we see should be the initial truncated + // state (it's only updated below Raft). + gotTruncatedState := readTruncStates(t, eng, rangeID) + assert.Equal(t, truncState, gotTruncatedState) assert.NotNil(t, res.Replicated.State) assert.NotNil(t, res.Replicated.State.TruncatedState) assert.Equal(t, expTruncState, *res.Replicated.State.TruncatedState) + } diff --git a/pkg/kv/kvserver/debug_print.go b/pkg/kv/kvserver/debug_print.go index 37cea1310d90..079aecc76198 100644 --- a/pkg/kv/kvserver/debug_print.go +++ b/pkg/kv/kvserver/debug_print.go @@ -13,7 +13,6 @@ package kvserver import ( "bytes" "fmt" - "strconv" "strings" "github.com/cockroachdb/cockroach/pkg/keys" @@ -262,15 +261,6 @@ func tryRangeIDKey(kv storage.MVCCKeyValue) (string, error) { // switch. Other types are handled inside the switch and return. var msg protoutil.Message switch { - case bytes.Equal(suffix, keys.LocalLeaseAppliedIndexLegacySuffix): - fallthrough - case bytes.Equal(suffix, keys.LocalRaftAppliedIndexLegacySuffix): - i, err := value.GetInt() - if err != nil { - return "", err - } - return strconv.FormatInt(i, 10), nil - case bytes.Equal(suffix, keys.LocalAbortSpanSuffix): msg = &roachpb.AbortSpanEntry{} @@ -283,7 +273,7 @@ func tryRangeIDKey(kv storage.MVCCKeyValue) (string, error) { case bytes.Equal(suffix, keys.LocalRangeTombstoneSuffix): msg = &roachpb.RangeTombstone{} - case bytes.Equal(suffix, keys.LocalRaftTruncatedStateLegacySuffix): + case bytes.Equal(suffix, keys.LocalRaftTruncatedStateSuffix): msg = &roachpb.RaftTruncatedState{} case bytes.Equal(suffix, keys.LocalRangeLeaseSuffix): diff --git a/pkg/kv/kvserver/kvserverpb/state.pb.go b/pkg/kv/kvserver/kvserverpb/state.pb.go index ce7660630fac..77b5e7a4a728 100644 --- a/pkg/kv/kvserver/kvserverpb/state.pb.go +++ b/pkg/kv/kvserver/kvserverpb/state.pb.go @@ -64,15 +64,6 @@ type ReplicaState struct { // not be served. GCThreshold *hlc.Timestamp `protobuf:"bytes,6,opt,name=gc_threshold,json=gcThreshold,proto3" json:"gc_threshold,omitempty"` Stats *enginepb.MVCCStats `protobuf:"bytes,7,opt,name=stats,proto3" json:"stats,omitempty"` - // using_applied_state_key specifies whether the Range has been upgraded - // to begin using the RangeAppliedState key. This key holds a combination - // of the Raft applied index, the lease applied index, and the MVCC stats. - // - // When set to true in a ReplicatedEvalResult, the flag indicates that the - // range should begin using the RangeAppliedState key. Handling of this flag - // is idempotent by Replica state machines, meaning that it is ok for multiple - // Raft commands to set it to true. - UsingAppliedStateKey bool `protobuf:"varint,11,opt,name=using_applied_state_key,json=usingAppliedStateKey,proto3" json:"using_applied_state_key,omitempty"` // Version tells us which migrations can be assumed to have run against this // particular replica. When we introduce backwards incompatible changes to the // replica state (for example using the unreplicated truncated state instead @@ -260,76 +251,74 @@ func init() { } var fileDescriptor_cc107fbd3ff296cb = []byte{ - // 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, + // 1065 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x56, 0xcf, 0x6e, 0xdb, 0x36, + 0x18, 0x8f, 0x66, 0xa5, 0x71, 0x68, 0x27, 0x71, 0x99, 0x25, 0x51, 0xd3, 0xd5, 0x36, 0x3c, 0x6c, + 0xf0, 0x80, 0x4e, 0xc2, 0xba, 0x61, 0xc5, 0xfe, 0x00, 0x43, 0x9c, 0x00, 0x83, 0x8d, 0x64, 0x48, + 0x18, 0xa3, 0x03, 0xb6, 0x83, 0x40, 0x4b, 0x8c, 0x4c, 0x44, 0x16, 0x55, 0x92, 0x36, 0xd2, 0x3e, + 0xc5, 0x1e, 0x61, 0x87, 0x3d, 0xc0, 0x1e, 0x23, 0xc7, 0x1c, 0x7b, 0x32, 0x56, 0xe7, 0xb2, 0xcb, + 0x5e, 0x60, 0xa7, 0x81, 0xa4, 0xe4, 0xd8, 0x4e, 0x80, 0x66, 0x40, 0x6f, 0xd4, 0xf7, 0xfd, 0x7e, + 0xdf, 0x3f, 0xfe, 0x3e, 0x49, 0xa0, 0x71, 0x3e, 0xf2, 0xce, 0x47, 0x82, 0xf0, 0x11, 0xe1, 0xd3, + 0x43, 0xda, 0xf3, 0x84, 0xc4, 0x92, 0xb8, 0x29, 0x67, 0x92, 0xc1, 0x5a, 0xc0, 0x82, 0x73, 0xce, + 0x70, 0xd0, 0x77, 0xcf, 0x47, 0x6e, 0x0e, 0x72, 0x85, 0x64, 0x1c, 0x47, 0x24, 0xed, 0xed, 0x3e, + 0xce, 0x8e, 0x1e, 0x49, 0x22, 0x9a, 0x90, 0xb4, 0xe7, 0x0d, 0x46, 0x41, 0x60, 0xd8, 0xbb, 0x8f, + 0x35, 0x33, 0xed, 0x79, 0x34, 0x91, 0x84, 0x27, 0x38, 0xf6, 0x39, 0x3e, 0x93, 0x99, 0x73, 0x3b, + 0x77, 0x0e, 0x88, 0xc4, 0x21, 0x96, 0x38, 0xb3, 0xc3, 0xdc, 0x3e, 0x63, 0x73, 0x86, 0x92, 0xc6, + 0x5e, 0x3f, 0x0e, 0x3c, 0x49, 0x07, 0x44, 0x48, 0x3c, 0x48, 0x33, 0xcf, 0x87, 0x11, 0x8b, 0x98, + 0x3e, 0x7a, 0xea, 0x64, 0xac, 0x8d, 0xb7, 0x36, 0x28, 0x23, 0x92, 0xc6, 0x34, 0xc0, 0xa7, 0xaa, + 0x1b, 0xf8, 0x14, 0x40, 0x95, 0xda, 0xc7, 0x69, 0x1a, 0x53, 0x12, 0xfa, 0x34, 0x09, 0xc9, 0x85, + 0x63, 0xd5, 0xad, 0xa6, 0x8d, 0x2a, 0xca, 0xb3, 0x67, 0x1c, 0x6d, 0x65, 0x87, 0x2e, 0xd8, 0x8c, + 0x09, 0x16, 0x64, 0x01, 0xfe, 0x81, 0x86, 0x3f, 0xd4, 0xae, 0x39, 0xfc, 0xd7, 0xc0, 0x0e, 0x89, + 0x08, 0x9c, 0x42, 0xdd, 0x6a, 0x96, 0x9e, 0x35, 0xdc, 0x9b, 0xa1, 0x65, 0xbd, 0xb8, 0x08, 0x27, + 0x11, 0x39, 0x20, 0x22, 0xe0, 0x34, 0x95, 0x8c, 0x23, 0x8d, 0x87, 0x2e, 0x58, 0xd6, 0xc1, 0x1c, + 0x5b, 0x13, 0x9d, 0x3b, 0x88, 0x87, 0xca, 0x8f, 0x0c, 0x0c, 0xfe, 0x04, 0x36, 0x24, 0x1f, 0x26, + 0x01, 0x96, 0x24, 0xf4, 0xf5, 0x35, 0x39, 0xcb, 0x9a, 0xf9, 0xc9, 0x9d, 0x29, 0xcf, 0x64, 0x37, + 0x47, 0xeb, 0x29, 0xa0, 0x75, 0x39, 0xf7, 0x0c, 0x4f, 0x40, 0x39, 0x0a, 0x7c, 0xd9, 0xe7, 0x44, + 0xf4, 0x59, 0x1c, 0x3a, 0x0f, 0x74, 0xb0, 0x27, 0x33, 0xc1, 0xd4, 0xdc, 0xdd, 0x7e, 0x1c, 0xb8, + 0xdd, 0x7c, 0xee, 0xad, 0x8d, 0xc9, 0xb8, 0x56, 0xfa, 0x71, 0xbf, 0x9b, 0xb3, 0x50, 0x29, 0x0a, + 0xa6, 0x0f, 0xf0, 0x3b, 0xb0, 0xac, 0x0a, 0x13, 0xce, 0xca, 0xad, 0xc2, 0x32, 0xa5, 0xb8, 0xb9, + 0x52, 0xdc, 0xa3, 0x17, 0xfb, 0xfb, 0xaa, 0x10, 0x81, 0x0c, 0x07, 0x7e, 0x05, 0x56, 0x46, 0x84, + 0x0b, 0xca, 0x12, 0xa7, 0xac, 0xe9, 0xbb, 0x77, 0xf4, 0xf5, 0xc2, 0x20, 0x50, 0x0e, 0x85, 0x3f, + 0x83, 0x2d, 0x7d, 0xb7, 0x41, 0xcc, 0x04, 0x09, 0xfd, 0xa9, 0x42, 0x9c, 0xb5, 0xfb, 0xb4, 0x63, + 0x5f, 0x8e, 0x6b, 0x4b, 0x68, 0x53, 0x45, 0xd8, 0xd7, 0x01, 0xa6, 0xae, 0x6f, 0xed, 0xbf, 0x7f, + 0xaf, 0x59, 0x1d, 0xbb, 0x58, 0xac, 0xac, 0x76, 0xec, 0xe2, 0x6a, 0x05, 0x74, 0xec, 0x22, 0xa8, + 0x94, 0x3a, 0x76, 0xb1, 0x54, 0x29, 0x37, 0xfe, 0x59, 0x01, 0xab, 0xfa, 0x5a, 0xdb, 0xc9, 0x19, + 0x83, 0x47, 0xa6, 0x6f, 0xa2, 0x35, 0x55, 0x7a, 0xf6, 0xb9, 0xfb, 0x8e, 0xc5, 0x71, 0x67, 0xe5, + 0xd9, 0x2a, 0xaa, 0x22, 0xae, 0xc6, 0x35, 0xcb, 0x4c, 0x82, 0xc0, 0x27, 0x00, 0xc4, 0x58, 0xc8, + 0x39, 0xe1, 0xad, 0x2a, 0x8b, 0x11, 0x5c, 0x0d, 0x94, 0x92, 0xe1, 0xc0, 0x4f, 0x49, 0x12, 0xd2, + 0x24, 0xd2, 0xba, 0xb3, 0x11, 0x48, 0x86, 0x83, 0x63, 0x63, 0xc9, 0x01, 0x21, 0x67, 0x69, 0x4a, + 0x42, 0xad, 0x12, 0x03, 0x38, 0x30, 0x16, 0xd8, 0x00, 0x6b, 0x7a, 0x68, 0x31, 0x8b, 0x7c, 0x41, + 0x5f, 0x13, 0x7d, 0xf7, 0x05, 0x54, 0x52, 0xc6, 0x43, 0x16, 0x9d, 0xd2, 0xd7, 0x04, 0x7e, 0x91, + 0x0d, 0x36, 0xc7, 0xf8, 0x92, 0x0f, 0x85, 0x24, 0xa1, 0x03, 0xea, 0x56, 0xb3, 0x88, 0xe0, 0x0c, + 0xb6, 0x6b, 0x3c, 0xf0, 0x7b, 0xb0, 0x8b, 0xd3, 0x94, 0xb3, 0x0b, 0x3a, 0xc0, 0x92, 0xf8, 0x29, + 0x67, 0x29, 0x13, 0x38, 0xf6, 0x5f, 0x0e, 0x99, 0xc4, 0x5a, 0x13, 0x05, 0xe4, 0xcc, 0x20, 0x8e, + 0x33, 0xc0, 0x89, 0xf2, 0xc3, 0x6f, 0xc0, 0xa3, 0x79, 0x86, 0xdf, 0x53, 0x5b, 0x68, 0x86, 0xb0, + 0xae, 0xc9, 0xdb, 0xe9, 0x2c, 0xa3, 0x85, 0x05, 0x31, 0x13, 0xf9, 0x01, 0x7c, 0xb4, 0x40, 0xe5, + 0xc4, 0xec, 0xf0, 0xcb, 0x21, 0x19, 0x12, 0x67, 0xa3, 0x5e, 0x68, 0x16, 0xd0, 0xa3, 0x39, 0x36, + 0x32, 0x88, 0x13, 0x05, 0x80, 0x9f, 0x82, 0x0d, 0xae, 0x6e, 0xd3, 0x1f, 0xe0, 0x0b, 0xbf, 0xf7, + 0x4a, 0x12, 0xe1, 0x14, 0x75, 0xc6, 0x35, 0x6d, 0x3e, 0xc2, 0x17, 0x2d, 0x65, 0x84, 0xbf, 0x82, + 0x1d, 0x1c, 0x48, 0x3a, 0x22, 0xb7, 0xf5, 0x56, 0xbe, 0xbf, 0xde, 0xb6, 0x4c, 0x8c, 0x05, 0xc5, + 0xc1, 0xe7, 0x60, 0x47, 0x67, 0x3b, 0x23, 0x24, 0xf4, 0x39, 0x89, 0xa8, 0x90, 0x1c, 0x4b, 0xca, + 0x12, 0xa1, 0xc5, 0x5c, 0x40, 0xdb, 0x53, 0x37, 0x9a, 0xf5, 0xc2, 0xcf, 0xc0, 0xaa, 0x24, 0x09, + 0x4e, 0xa4, 0x4f, 0x43, 0xa7, 0xa2, 0x6e, 0xbb, 0x55, 0x9e, 0x8c, 0x6b, 0xc5, 0xae, 0x36, 0xb6, + 0x0f, 0x50, 0xd1, 0xb8, 0xdb, 0x21, 0x24, 0x60, 0x67, 0xb1, 0x72, 0x3f, 0x65, 0x31, 0x0d, 0x5e, + 0x39, 0xb0, 0x6e, 0x35, 0xd7, 0xe7, 0xb4, 0x3b, 0xf7, 0xfe, 0x5a, 0xa8, 0xf6, 0x58, 0x93, 0xd0, + 0x56, 0x70, 0x97, 0x19, 0xfe, 0x69, 0x81, 0x8f, 0x6f, 0xe5, 0x11, 0x34, 0x24, 0x92, 0xe3, 0x44, + 0xa4, 0x8c, 0x2b, 0x71, 0x9f, 0x31, 0x67, 0x53, 0x0f, 0xed, 0xf9, 0xbb, 0xf7, 0x45, 0x55, 0x70, + 0x4a, 0x43, 0xd2, 0xcd, 0xf9, 0x6a, 0xef, 0x5a, 0x4d, 0x35, 0xce, 0xc9, 0xb8, 0x56, 0x5f, 0x28, + 0xee, 0x16, 0x12, 0xd5, 0x83, 0xdb, 0x08, 0x39, 0x8b, 0x98, 0xee, 0xbb, 0x5d, 0x59, 0x9e, 0xee, + 0x7b, 0xa9, 0x52, 0xee, 0xd8, 0xc5, 0x87, 0x15, 0xd8, 0xf8, 0xa3, 0x00, 0xb6, 0xef, 0x2e, 0x02, + 0x76, 0xc0, 0x3a, 0x37, 0xeb, 0x9c, 0x89, 0x22, 0x7b, 0x0b, 0xdc, 0x4b, 0x0a, 0x6b, 0x19, 0xd5, + 0x34, 0x00, 0x87, 0xa0, 0x94, 0xc7, 0x8a, 0x31, 0xd5, 0xab, 0x5f, 0x68, 0x75, 0x27, 0xe3, 0x1a, + 0xc8, 0xde, 0x18, 0x87, 0x7b, 0xed, 0x7f, 0xc7, 0xb5, 0x56, 0x44, 0x65, 0x7f, 0xd8, 0x73, 0x03, + 0x36, 0xf0, 0xa6, 0x49, 0xc2, 0xde, 0xcd, 0xd9, 0x4b, 0xcf, 0x23, 0x6f, 0xf6, 0x0b, 0x6f, 0x2a, + 0x93, 0xc2, 0x0b, 0xa4, 0xfa, 0xae, 0xec, 0xb5, 0x11, 0xc8, 0x12, 0x1d, 0x62, 0xaa, 0x5a, 0x08, + 0x48, 0x22, 0x39, 0x8e, 0xf3, 0x16, 0x0a, 0xff, 0xa3, 0x85, 0x8c, 0x7a, 0xd3, 0x42, 0x1e, 0x4b, + 0xb5, 0x60, 0xdf, 0xb4, 0xb0, 0x6f, 0xcc, 0xef, 0xb1, 0x85, 0x2c, 0xd1, 0x21, 0xa6, 0xe6, 0xfa, + 0x5a, 0x4f, 0x2f, 0xdf, 0x56, 0x97, 0x2e, 0x27, 0x55, 0xeb, 0x6a, 0x52, 0xb5, 0xde, 0x4c, 0xaa, + 0xd6, 0x5f, 0x93, 0xaa, 0xf5, 0xdb, 0x75, 0x75, 0xe9, 0xea, 0xba, 0xba, 0xf4, 0xe6, 0xba, 0xba, + 0xf4, 0x0b, 0xb8, 0xf9, 0xd7, 0xe9, 0x3d, 0xd0, 0xff, 0x0b, 0x5f, 0xfe, 0x17, 0x00, 0x00, 0xff, + 0xff, 0x26, 0xb9, 0x1b, 0x85, 0x0c, 0x09, 0x00, 0x00, } func (this *ReplicaState) Equal(that interface{}) bool { @@ -372,9 +361,6 @@ func (this *ReplicaState) Equal(that interface{}) bool { if !this.Stats.Equal(that1.Stats) { return false } - if this.UsingAppliedStateKey != that1.UsingAppliedStateKey { - return false - } if !this.Version.Equal(that1.Version) { return false } @@ -529,16 +515,6 @@ func (m *ReplicaState) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x62 } - if m.UsingAppliedStateKey { - i-- - if m.UsingAppliedStateKey { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i-- - dAtA[i] = 0x58 - } if m.Stats != nil { { size, err := m.Stats.MarshalToSizedBuffer(dAtA[:i]) @@ -846,9 +822,6 @@ func (m *ReplicaState) Size() (n int) { l = m.Stats.Size() n += 1 + l + sovState(uint64(l)) } - if m.UsingAppliedStateKey { - n += 2 - } if m.Version != nil { l = m.Version.Size() n += 1 + l + sovState(uint64(l)) @@ -1185,26 +1158,6 @@ func (m *ReplicaState) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex - case 11: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field UsingAppliedStateKey", wireType) - } - var v int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowState - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.UsingAppliedStateKey = bool(v != 0) case 12: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) diff --git a/pkg/kv/kvserver/kvserverpb/state.proto b/pkg/kv/kvserver/kvserverpb/state.proto index 4caca7500bd1..857873a119ff 100644 --- a/pkg/kv/kvserver/kvserverpb/state.proto +++ b/pkg/kv/kvserver/kvserverpb/state.proto @@ -59,15 +59,6 @@ message ReplicaState { // not be served. util.hlc.Timestamp gc_threshold = 6 [(gogoproto.customname) = "GCThreshold"]; storage.enginepb.MVCCStats stats = 7; - // using_applied_state_key specifies whether the Range has been upgraded - // to begin using the RangeAppliedState key. This key holds a combination - // of the Raft applied index, the lease applied index, and the MVCC stats. - // - // When set to true in a ReplicatedEvalResult, the flag indicates that the - // range should begin using the RangeAppliedState key. Handling of this flag - // is idempotent by Replica state machines, meaning that it is ok for multiple - // Raft commands to set it to true. - bool using_applied_state_key = 11; // Version tells us which migrations can be assumed to have run against this // particular replica. When we introduce backwards incompatible changes to the // replica state (for example using the unreplicated truncated state instead @@ -108,7 +99,7 @@ message ReplicaState { // "follower reads" at or below this timestamp. util.hlc.Timestamp raft_closed_timestamp = 13 [(gogoproto.nullable) = false]; - reserved 8, 9, 10; + reserved 8, 9, 10, 11; } // RangeInfo is used for reporting status information about a range out through diff --git a/pkg/kv/kvserver/raft.pb.go b/pkg/kv/kvserver/raft.pb.go index 052648ba5f52..bc680be139f5 100644 --- a/pkg/kv/kvserver/raft.pb.go +++ b/pkg/kv/kvserver/raft.pb.go @@ -446,15 +446,16 @@ type SnapshotRequest_Header struct { // The type of the snapshot. Type SnapshotRequest_Type `protobuf:"varint,9,opt,name=type,proto3,enum=cockroach.kv.kvserver.SnapshotRequest_Type" json:"type,omitempty"` // Whether the snapshot uses the unreplicated RaftTruncatedState or not. - // This is generally always true at 2.2 and above outside of the migration - // phase, though theoretically it could take a long time for all ranges - // to update to the new mechanism. This bool is true iff the Raft log at - // the snapshot's applied index is using the new key. In particular, it - // is true if the index itself carries out the migration (in which case - // the data in the snapshot contains neither key). + // This is always true for snapshots generated in v21.1+ clusters. In v20.2 + // it was possible for ranges to be using the replicated variant. v21.1 + // therefore had code expecting that possibility (unlike v21.2 code, where + // this field is assumed to always be true and thus never read). For + // compatibility with v21.1 nodes however, v21.2 has to explicitly set this + // field to true. In v22.1 we can remove it entirely seeing as how v21.2 + // code never reads the field. // - // See VersionUnreplicatedRaftTruncatedState. - UnreplicatedTruncatedState bool `protobuf:"varint,8,opt,name=unreplicated_truncated_state,json=unreplicatedTruncatedState,proto3" json:"unreplicated_truncated_state,omitempty"` + // TODO(irfansharif): Remove this in v22.1. + LegacyUnreplicatedTruncatedState bool `protobuf:"varint,8,opt,name=legacy_unreplicated_truncated_state,json=legacyUnreplicatedTruncatedState,proto3" json:"legacy_unreplicated_truncated_state,omitempty"` } func (m *SnapshotRequest_Header) Reset() { *m = SnapshotRequest_Header{} } @@ -576,88 +577,88 @@ func init() { func init() { proto.RegisterFile("kv/kvserver/raft.proto", fileDescriptor_acdcf79fd972c844) } var fileDescriptor_acdcf79fd972c844 = []byte{ - // 1289 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x57, 0x4f, 0x73, 0xdb, 0x44, - 0x14, 0xb7, 0x12, 0xc5, 0x96, 0xd7, 0x71, 0x23, 0x96, 0x10, 0x44, 0xa6, 0xd8, 0x41, 0x53, 0x98, - 0xf0, 0x4f, 0x9e, 0xa6, 0x85, 0x03, 0x17, 0xf0, 0x1f, 0x75, 0xa2, 0x38, 0x4d, 0xd2, 0xb5, 0x13, - 0x06, 0x18, 0xd0, 0xc8, 0xf2, 0xda, 0xd6, 0xd8, 0xd6, 0xaa, 0xd2, 0xda, 0xe0, 0x7e, 0x0a, 0xae, - 0xdc, 0xb8, 0xf1, 0x49, 0x98, 0xc9, 0xb1, 0x27, 0xa6, 0x07, 0xc6, 0x03, 0xee, 0x99, 0x2f, 0xd0, - 0x13, 0xb3, 0xab, 0x95, 0xe3, 0xa6, 0x4d, 0xa9, 0x67, 0xe8, 0x0c, 0x9c, 0xf2, 0xf6, 0xf9, 0xbd, - 0xdf, 0x7b, 0xfb, 0xde, 0xef, 0xbd, 0x55, 0xc0, 0x56, 0x7f, 0x5c, 0xea, 0x8f, 0x23, 0x1c, 0x8e, - 0x71, 0x58, 0x0a, 0x9d, 0x0e, 0x35, 0x82, 0x90, 0x50, 0x02, 0xdf, 0x70, 0x89, 0xdb, 0x0f, 0x89, - 0xe3, 0xf6, 0x8c, 0xfe, 0xd8, 0x48, 0x2c, 0xb6, 0x37, 0xb9, 0x2a, 0x68, 0x95, 0x70, 0x18, 0x92, - 0x30, 0x8a, 0x8d, 0xb7, 0xb7, 0x12, 0xed, 0x10, 0x53, 0xa7, 0xed, 0x50, 0x47, 0xe8, 0x8d, 0x45, - 0xf0, 0x81, 0x37, 0xc6, 0x3e, 0x8e, 0xa2, 0xb9, 0x10, 0xb4, 0xe6, 0xa2, 0xb0, 0xd7, 0x17, 0xed, - 0x13, 0x21, 0x68, 0x95, 0x22, 0xea, 0x50, 0x2c, 0x6c, 0x0a, 0x98, 0xba, 0x6d, 0x9e, 0x69, 0x69, - 0x7c, 0x8b, 0xff, 0x0d, 0x5a, 0x0b, 0x89, 0x6f, 0x6f, 0x76, 0x49, 0x97, 0x70, 0xb1, 0xc4, 0xa4, - 0x58, 0xab, 0xff, 0x25, 0x83, 0x3c, 0x72, 0x3a, 0x74, 0x1f, 0x3b, 0x21, 0x6d, 0x61, 0x87, 0xc2, - 0xef, 0x80, 0x12, 0x3a, 0x7e, 0x17, 0xdb, 0x5e, 0x5b, 0x93, 0x76, 0xa4, 0x5d, 0xb9, 0x52, 0x9d, - 0x4d, 0x8b, 0x19, 0xc4, 0x74, 0x56, 0xed, 0xc9, 0xb4, 0x78, 0xbb, 0xeb, 0xd1, 0xde, 0xa8, 0x65, - 0xb8, 0x64, 0x58, 0x9a, 0x17, 0xa3, 0xdd, 0xba, 0x90, 0x4b, 0x41, 0xbf, 0x5b, 0x12, 0x37, 0x37, - 0x84, 0x1f, 0xca, 0x70, 0x50, 0xab, 0x0d, 0x23, 0xb0, 0xd1, 0x09, 0xc9, 0xd0, 0x0e, 0x71, 0x30, - 0xf0, 0x5c, 0x87, 0x85, 0x59, 0xd9, 0x91, 0x76, 0xf3, 0x95, 0xfa, 0x6c, 0x5a, 0xcc, 0xdf, 0x09, - 0xc9, 0x10, 0xc5, 0xbf, 0xf0, 0x60, 0x9f, 0x2e, 0x17, 0x2c, 0xf1, 0x44, 0xf9, 0xce, 0x02, 0x50, - 0x1b, 0x0e, 0x41, 0x9e, 0x92, 0xc5, 0x90, 0xab, 0x3c, 0xa4, 0x35, 0x9b, 0x16, 0x73, 0x4d, 0xf2, - 0x6f, 0x04, 0xcc, 0x51, 0x72, 0x11, 0x0e, 0x02, 0x99, 0xe2, 0x70, 0xa8, 0xc9, 0xac, 0x7e, 0x88, - 0xcb, 0x70, 0x0b, 0xa4, 0x5d, 0x32, 0x1c, 0x7a, 0x54, 0x5b, 0xe3, 0x5a, 0x71, 0x82, 0x1a, 0xc8, - 0xdc, 0x1f, 0x79, 0x38, 0x72, 0xb1, 0x96, 0xde, 0x91, 0x76, 0x15, 0x94, 0x1c, 0xe1, 0x03, 0x70, - 0x7d, 0xe0, 0x74, 0xbb, 0x9e, 0xdf, 0xb5, 0x3b, 0x64, 0x30, 0x20, 0xdf, 0xe3, 0x30, 0xb2, 0x89, - 0x6f, 0x27, 0xe6, 0xca, 0xce, 0xea, 0x6e, 0x6e, 0xef, 0x96, 0xf1, 0x5c, 0x46, 0x1a, 0x73, 0x0a, - 0x5d, 0xd0, 0xca, 0x38, 0x14, 0x62, 0x45, 0x3e, 0x9f, 0x16, 0x53, 0xe8, 0x2d, 0x01, 0x7f, 0x27, - 0x41, 0x3f, 0xf6, 0xef, 0x89, 0xd8, 0x27, 0xe0, 0xdd, 0x17, 0xc5, 0xb6, 0x1d, 0xd7, 0x1d, 0x85, - 0x0e, 0xc5, 0x1a, 0xe0, 0x39, 0xbf, 0x73, 0x25, 0x52, 0x59, 0x18, 0x1e, 0xc8, 0x4a, 0x46, 0x55, - 0xf4, 0x5f, 0xd2, 0x00, 0x32, 0xbe, 0xdd, 0xc5, 0x51, 0xe4, 0x74, 0x31, 0xc2, 0xf7, 0x47, 0x38, - 0x7a, 0xf5, 0xa4, 0xfb, 0x16, 0x6c, 0xc4, 0xf8, 0x11, 0x75, 0x42, 0x6a, 0xf7, 0xf1, 0x44, 0x53, - 0x76, 0xa4, 0xdd, 0xf5, 0xca, 0x27, 0x4f, 0xa6, 0xc5, 0x9b, 0xcb, 0x61, 0xd7, 0xf1, 0x04, 0xe5, - 0x39, 0x5a, 0x83, 0x81, 0xd5, 0xf1, 0x04, 0xde, 0x05, 0xeb, 0x8b, 0x9c, 0xe6, 0x84, 0xce, 0xed, - 0xdd, 0x58, 0xe8, 0xcc, 0x25, 0xc2, 0xd4, 0x70, 0xe4, 0x86, 0x5e, 0x40, 0x49, 0x28, 0x5a, 0x91, - 0x5b, 0xe0, 0x2b, 0xb4, 0x00, 0xb8, 0x60, 0x2b, 0xa7, 0xea, 0x72, 0x60, 0xd9, 0x39, 0x17, 0x61, - 0x09, 0x64, 0x86, 0x71, 0xa9, 0x39, 0x19, 0x73, 0x7b, 0x1b, 0x46, 0xbc, 0x1a, 0x0c, 0xd1, 0x01, - 0xe1, 0x92, 0x58, 0x2d, 0xd2, 0x71, 0x6d, 0x39, 0x3a, 0x66, 0xff, 0x4f, 0x74, 0x84, 0x07, 0x00, - 0xf4, 0x92, 0x9d, 0x17, 0x69, 0x69, 0x9e, 0xfb, 0x8d, 0x2b, 0x72, 0x7f, 0x6a, 0x41, 0x8a, 0x64, - 0x17, 0xbc, 0x61, 0x03, 0x6c, 0xcc, 0x4f, 0x76, 0x88, 0xa3, 0x20, 0xd2, 0x32, 0x4b, 0x03, 0x5e, - 0x9b, 0x43, 0x20, 0x86, 0xa0, 0x77, 0xc0, 0x9b, 0xcf, 0x0e, 0x4a, 0xc5, 0xa1, 0x6e, 0x0f, 0xd6, - 0x81, 0x12, 0xc6, 0xe7, 0x48, 0x93, 0x78, 0xa0, 0xf7, 0x5f, 0x10, 0xe8, 0x12, 0x42, 0x1c, 0x6d, - 0x0e, 0xa0, 0x9f, 0x00, 0xed, 0x29, 0xab, 0x28, 0x20, 0x7e, 0x84, 0x4f, 0x7d, 0x8f, 0xf8, 0xd0, - 0x00, 0x6b, 0xfc, 0x3d, 0xe3, 0x33, 0x99, 0xdb, 0xd3, 0x2e, 0x47, 0x09, 0x5a, 0x86, 0xc9, 0x7e, - 0x47, 0xb1, 0xd9, 0x67, 0xf2, 0xf9, 0xcf, 0x45, 0x49, 0xff, 0x7d, 0x05, 0xbc, 0xfe, 0x1c, 0xc8, - 0x57, 0x3e, 0xe4, 0xff, 0xdd, 0x29, 0xac, 0x83, 0xb5, 0x11, 0x2b, 0xa8, 0x98, 0xc1, 0xd2, 0xcb, - 0x74, 0x6b, 0xa1, 0x0f, 0x02, 0x30, 0xc6, 0xd0, 0x7f, 0x4b, 0x83, 0x8d, 0x86, 0xef, 0x04, 0x51, - 0x8f, 0xd0, 0x64, 0x7f, 0x9a, 0x20, 0xdd, 0xc3, 0x4e, 0x1b, 0x27, 0x9d, 0xfa, 0xf8, 0x8a, 0x08, - 0x97, 0xfc, 0x8c, 0x7d, 0xee, 0x84, 0x84, 0x33, 0x7c, 0x0f, 0x28, 0xfd, 0xb1, 0xdd, 0x62, 0x24, - 0xe3, 0xd5, 0x5b, 0xaf, 0xe4, 0x58, 0x87, 0xea, 0x67, 0x9c, 0x77, 0x28, 0xd3, 0x1f, 0xc7, 0x04, - 0x2c, 0x82, 0xdc, 0x80, 0x74, 0x6d, 0xec, 0xd3, 0xd0, 0xc3, 0x91, 0xb6, 0xba, 0xb3, 0xba, 0xbb, - 0x8e, 0xc0, 0x80, 0x74, 0xcd, 0x58, 0x03, 0x37, 0xc1, 0x5a, 0xc7, 0xf3, 0x9d, 0x01, 0xbf, 0xb0, - 0x82, 0xe2, 0xc3, 0xf6, 0x4f, 0x32, 0x48, 0xc7, 0x11, 0xa1, 0x05, 0xd6, 0xf8, 0xc7, 0x0b, 0x5f, - 0x32, 0x57, 0xe7, 0x1b, 0x51, 0x12, 0x3a, 0x5d, 0x7c, 0x51, 0xe5, 0x06, 0x73, 0x4a, 0xea, 0xc1, - 0x11, 0xa0, 0x03, 0x36, 0xd9, 0x4a, 0xb3, 0xc5, 0x06, 0xb3, 0x05, 0xb3, 0x45, 0xfb, 0x97, 0x9e, - 0x0c, 0x18, 0x3e, 0xfb, 0x3c, 0xbd, 0x0d, 0x80, 0x78, 0x3e, 0xbc, 0x07, 0x98, 0x53, 0x61, 0x15, - 0x65, 0xe3, 0x27, 0xc0, 0x7b, 0x80, 0x59, 0x39, 0x5c, 0xc7, 0xb7, 0xdb, 0xd8, 0x1d, 0x78, 0x3e, - 0x16, 0x77, 0x06, 0xae, 0xe3, 0xd7, 0x62, 0x0d, 0x1b, 0xd8, 0x20, 0xf4, 0x48, 0xe8, 0xd1, 0x09, - 0x7f, 0xe4, 0xaf, 0x5d, 0x49, 0x81, 0xcb, 0x0d, 0x3a, 0x11, 0x6e, 0x68, 0x0e, 0xc0, 0xc0, 0x22, - 0xca, 0x76, 0x58, 0x77, 0xa2, 0x65, 0x96, 0x02, 0x6b, 0x08, 0x37, 0x34, 0x07, 0x80, 0x9f, 0x03, - 0x99, 0x4e, 0x02, 0xb6, 0xbc, 0x19, 0xd0, 0x87, 0x2f, 0x09, 0xd4, 0x9c, 0x04, 0x18, 0x71, 0x47, - 0xf8, 0x05, 0xb8, 0x3e, 0xf2, 0xc5, 0x90, 0x50, 0xdc, 0xb6, 0x69, 0x38, 0xf2, 0x63, 0x29, 0xee, - 0xaf, 0xc2, 0x8b, 0xb1, 0xbd, 0x68, 0xd3, 0x4c, 0x4c, 0x78, 0x33, 0x0f, 0x64, 0x45, 0x52, 0x57, - 0xf4, 0xdb, 0x40, 0x49, 0xee, 0x0a, 0x73, 0x20, 0x73, 0x7a, 0x54, 0x3f, 0x3a, 0xfe, 0xf2, 0x48, - 0x4d, 0xc1, 0x75, 0xa0, 0x20, 0xb3, 0x7a, 0x7c, 0x66, 0xa2, 0xaf, 0x54, 0x09, 0xe6, 0x41, 0x16, - 0x99, 0x95, 0xf2, 0x61, 0xf9, 0xa8, 0x6a, 0xaa, 0x2b, 0xba, 0x06, 0x94, 0xe4, 0x52, 0xcc, 0xb0, - 0x7e, 0x66, 0x57, 0xca, 0xcd, 0xea, 0xbe, 0x9a, 0xd2, 0x6f, 0x02, 0x99, 0x65, 0x09, 0xb7, 0x00, - 0x3c, 0xb3, 0xca, 0x76, 0xe3, 0xa8, 0x7c, 0xd2, 0xd8, 0x3f, 0x6e, 0xda, 0xf7, 0x4e, 0xcd, 0x53, - 0x53, 0x4d, 0xb1, 0x18, 0xd6, 0x91, 0xd5, 0xb4, 0xca, 0x87, 0xaa, 0xa4, 0xcb, 0xca, 0x8a, 0xba, - 0xa2, 0xff, 0x2a, 0x01, 0xf5, 0xe2, 0xa6, 0x62, 0x69, 0xdd, 0x01, 0x69, 0x76, 0x91, 0x51, 0xc4, - 0x27, 0xeb, 0xda, 0x9e, 0xf1, 0x8f, 0x25, 0x8a, 0x1d, 0x8d, 0x06, 0xf7, 0x42, 0xc2, 0x9b, 0xbd, - 0xab, 0xc9, 0x43, 0xcc, 0x88, 0x99, 0x9d, 0xbf, 0xb8, 0xba, 0x05, 0xd2, 0xb1, 0xed, 0x33, 0xf7, - 0x2e, 0x57, 0xab, 0xe6, 0x49, 0xd3, 0xac, 0xa9, 0x12, 0xfb, 0xa9, 0x7c, 0x72, 0x72, 0x68, 0x99, - 0x35, 0x75, 0x05, 0x66, 0xc1, 0x9a, 0x89, 0xd0, 0x31, 0x52, 0x57, 0x99, 0x55, 0xcd, 0xac, 0x1e, - 0x5a, 0x47, 0x66, 0x4d, 0x95, 0x0f, 0x64, 0x65, 0x55, 0x95, 0xf5, 0x6f, 0xc0, 0x6b, 0x55, 0xe2, - 0x77, 0xaa, 0x3d, 0x46, 0xd0, 0x2a, 0xf1, 0x29, 0xfe, 0x81, 0xc2, 0x8f, 0x00, 0x60, 0x1f, 0x9c, - 0x8e, 0xdf, 0x4e, 0xd6, 0x6f, 0xb6, 0x92, 0x9f, 0x4d, 0x8b, 0xd9, 0x6a, 0xac, 0xb5, 0x6a, 0x28, - 0x2b, 0x0c, 0xac, 0x36, 0xcb, 0x36, 0x70, 0x26, 0x03, 0xe2, 0xc4, 0x1f, 0xe7, 0xeb, 0x28, 0x39, - 0x56, 0x3e, 0x38, 0xff, 0xb3, 0x90, 0x3a, 0x9f, 0x15, 0xa4, 0x87, 0xb3, 0x82, 0xf4, 0x68, 0x56, - 0x90, 0xfe, 0x98, 0x15, 0xa4, 0x1f, 0x1f, 0x17, 0x52, 0x0f, 0x1f, 0x17, 0x52, 0x8f, 0x1e, 0x17, - 0x52, 0x5f, 0x2b, 0x49, 0x4d, 0x5a, 0x69, 0xfe, 0x3f, 0xc6, 0xad, 0xbf, 0x03, 0x00, 0x00, 0xff, - 0xff, 0x50, 0x56, 0x82, 0x44, 0x4c, 0x0d, 0x00, 0x00, + // 1296 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x57, 0x5f, 0x73, 0xdb, 0x44, + 0x10, 0xb7, 0x12, 0xc5, 0x96, 0xcf, 0x71, 0x23, 0x8e, 0x10, 0x44, 0x06, 0xec, 0x60, 0x0a, 0x13, + 0xfe, 0xc9, 0xd3, 0xb4, 0xf0, 0xc0, 0x0b, 0xe3, 0x3f, 0xea, 0x44, 0x71, 0x9a, 0xa4, 0xe7, 0x24, + 0x0c, 0x30, 0xa0, 0x39, 0xcb, 0x67, 0x45, 0x63, 0x5b, 0xa7, 0x4a, 0x67, 0x83, 0xfb, 0x29, 0xf8, + 0x08, 0x3c, 0xc1, 0x27, 0x61, 0x26, 0x8f, 0x7d, 0xec, 0x03, 0x93, 0x01, 0xe7, 0x99, 0x2f, 0xd0, + 0x27, 0xe6, 0x4e, 0x27, 0xc7, 0x4d, 0x9b, 0xd2, 0xcc, 0xd0, 0x19, 0x78, 0xca, 0xde, 0x7a, 0xf7, + 0xb7, 0x7b, 0xbb, 0xbf, 0xdd, 0x53, 0xc0, 0x5a, 0x7f, 0x5c, 0xed, 0x8f, 0x63, 0x12, 0x8d, 0x49, + 0x54, 0x8d, 0x70, 0x8f, 0x99, 0x61, 0x44, 0x19, 0x85, 0x6f, 0xb8, 0xd4, 0xed, 0x47, 0x14, 0xbb, + 0x27, 0x66, 0x7f, 0x6c, 0xa6, 0x16, 0xeb, 0xab, 0x42, 0x15, 0x76, 0xaa, 0x24, 0x8a, 0x68, 0x14, + 0x27, 0xc6, 0xeb, 0x6b, 0xa9, 0x76, 0x48, 0x18, 0xee, 0x62, 0x86, 0xa5, 0xde, 0x9c, 0x07, 0x1f, + 0xf8, 0x63, 0x12, 0x90, 0x38, 0x9e, 0x09, 0x61, 0x67, 0x26, 0x4a, 0xfb, 0xca, 0xbc, 0x7d, 0x2a, + 0x84, 0x9d, 0x6a, 0xcc, 0x30, 0x23, 0xd2, 0xa6, 0x44, 0x98, 0xdb, 0x15, 0x99, 0x56, 0xc7, 0xb7, + 0xc5, 0xdf, 0xb0, 0x33, 0x97, 0xf8, 0xfa, 0xaa, 0x47, 0x3d, 0x2a, 0xc4, 0x2a, 0x97, 0x12, 0x6d, + 0xe5, 0x2f, 0x15, 0x14, 0x11, 0xee, 0xb1, 0x6d, 0x82, 0x23, 0xd6, 0x21, 0x98, 0xc1, 0xef, 0x81, + 0x16, 0xe1, 0xc0, 0x23, 0x8e, 0xdf, 0x35, 0x94, 0x0d, 0x65, 0x53, 0xad, 0x37, 0xa6, 0x67, 0xe5, + 0x1c, 0xe2, 0x3a, 0xbb, 0xf9, 0xe4, 0xac, 0x7c, 0xc7, 0xf3, 0xd9, 0xc9, 0xa8, 0x63, 0xba, 0x74, + 0x58, 0x9d, 0x15, 0xa3, 0xdb, 0xb9, 0x90, 0xab, 0x61, 0xdf, 0xab, 0xca, 0x9b, 0x9b, 0xd2, 0x0f, + 0xe5, 0x04, 0xa8, 0xdd, 0x85, 0x31, 0x58, 0xe9, 0x45, 0x74, 0xe8, 0x44, 0x24, 0x1c, 0xf8, 0x2e, + 0xe6, 0x61, 0x16, 0x36, 0x94, 0xcd, 0x62, 0xbd, 0x35, 0x3d, 0x2b, 0x17, 0xef, 0x46, 0x74, 0x88, + 0x92, 0x5f, 0x44, 0xb0, 0xcf, 0xaf, 0x17, 0x2c, 0xf5, 0x44, 0xc5, 0xde, 0x1c, 0x50, 0x17, 0x0e, + 0x41, 0x91, 0xd1, 0xf9, 0x90, 0x8b, 0x22, 0xa4, 0x3d, 0x3d, 0x2b, 0x17, 0x0e, 0xe9, 0xbf, 0x11, + 0xb0, 0xc0, 0xe8, 0x45, 0x38, 0x08, 0x54, 0x46, 0xa2, 0xa1, 0xa1, 0xf2, 0xfa, 0x21, 0x21, 0xc3, + 0x35, 0x90, 0x75, 0xe9, 0x70, 0xe8, 0x33, 0x63, 0x49, 0x68, 0xe5, 0x09, 0x1a, 0x20, 0xf7, 0x60, + 0xe4, 0x93, 0xd8, 0x25, 0x46, 0x76, 0x43, 0xd9, 0xd4, 0x50, 0x7a, 0x84, 0x0f, 0xc1, 0xdb, 0x03, + 0xec, 0x79, 0x7e, 0xe0, 0x39, 0x3d, 0x3a, 0x18, 0xd0, 0x1f, 0x48, 0x14, 0x3b, 0x34, 0x70, 0x52, + 0x73, 0x6d, 0x63, 0x71, 0xb3, 0xb0, 0x75, 0xdb, 0x7c, 0x2e, 0x23, 0xcd, 0x19, 0x85, 0x2e, 0x68, + 0x65, 0xee, 0x4a, 0xb1, 0xae, 0x9e, 0x9e, 0x95, 0x33, 0xe8, 0x2d, 0x09, 0x7f, 0x37, 0x45, 0xdf, + 0x0f, 0xee, 0xcb, 0xd8, 0x07, 0xe0, 0xfd, 0x17, 0xc5, 0x76, 0xb0, 0xeb, 0x8e, 0x22, 0xcc, 0x88, + 0x01, 0x44, 0xce, 0xef, 0x5e, 0x89, 0x54, 0x93, 0x86, 0x3b, 0xaa, 0x96, 0xd3, 0xb5, 0xca, 0xaf, + 0x59, 0x00, 0x39, 0xdf, 0xee, 0x91, 0x38, 0xc6, 0x1e, 0x41, 0xe4, 0xc1, 0x88, 0xc4, 0xaf, 0x9e, + 0x74, 0xdf, 0x81, 0x95, 0x04, 0x3f, 0x66, 0x38, 0x62, 0x4e, 0x9f, 0x4c, 0x0c, 0x6d, 0x43, 0xd9, + 0x5c, 0xae, 0x7f, 0xf6, 0xe4, 0xac, 0x7c, 0xeb, 0x7a, 0xd8, 0x2d, 0x32, 0x41, 0x45, 0x81, 0xd6, + 0xe6, 0x60, 0x2d, 0x32, 0x81, 0xf7, 0xc0, 0xf2, 0x3c, 0xa7, 0x05, 0xa1, 0x0b, 0x5b, 0x37, 0xe7, + 0x3a, 0x73, 0x89, 0x30, 0x4d, 0x12, 0xbb, 0x91, 0x1f, 0x32, 0x1a, 0xc9, 0x56, 0x14, 0xe6, 0xf8, + 0x0a, 0x6d, 0x00, 0x2e, 0xd8, 0x2a, 0xa8, 0x7a, 0x3d, 0xb0, 0xfc, 0x8c, 0x8b, 0xb0, 0x0a, 0x72, + 0xc3, 0xa4, 0xd4, 0x82, 0x8c, 0x85, 0xad, 0x15, 0x33, 0x59, 0x0d, 0xa6, 0xec, 0x80, 0x74, 0x49, + 0xad, 0xe6, 0xe9, 0xb8, 0x74, 0x3d, 0x3a, 0xe6, 0xff, 0x4f, 0x74, 0x84, 0x3b, 0x00, 0x9c, 0xa4, + 0x3b, 0x2f, 0x36, 0xb2, 0x22, 0xf7, 0x9b, 0x57, 0xe4, 0xfe, 0xd4, 0x82, 0x94, 0xc9, 0xce, 0x79, + 0xc3, 0x36, 0x58, 0x99, 0x9d, 0x9c, 0x88, 0xc4, 0x61, 0x6c, 0xe4, 0xae, 0x0d, 0x78, 0x63, 0x06, + 0x81, 0x38, 0x42, 0xa5, 0x07, 0xde, 0x7c, 0x76, 0x50, 0xea, 0x98, 0xb9, 0x27, 0xb0, 0x05, 0xb4, + 0x28, 0x39, 0xc7, 0x86, 0x22, 0x02, 0x7d, 0xf8, 0x82, 0x40, 0x97, 0x10, 0x92, 0x68, 0x33, 0x80, + 0xca, 0x01, 0x30, 0x9e, 0xb2, 0x8a, 0x43, 0x1a, 0xc4, 0xe4, 0x28, 0xf0, 0x69, 0x00, 0x4d, 0xb0, + 0x24, 0xde, 0x33, 0x31, 0x93, 0x85, 0x2d, 0xe3, 0x72, 0x94, 0xb0, 0x63, 0x5a, 0xfc, 0x77, 0x94, + 0x98, 0x7d, 0xa1, 0x9e, 0xfe, 0x5c, 0x56, 0x2a, 0xbf, 0x2f, 0x80, 0xd7, 0x9f, 0x03, 0xf9, 0xca, + 0x87, 0xfc, 0xbf, 0x3b, 0x85, 0x2d, 0xb0, 0x34, 0xe2, 0x05, 0x95, 0x33, 0x58, 0x7d, 0x99, 0x6e, + 0xcd, 0xf5, 0x41, 0x02, 0x26, 0x18, 0x95, 0xf3, 0x2c, 0x58, 0x69, 0x07, 0x38, 0x8c, 0x4f, 0x28, + 0x4b, 0xf7, 0xa7, 0x05, 0xb2, 0x27, 0x04, 0x77, 0x49, 0xda, 0xa9, 0x4f, 0xaf, 0x88, 0x70, 0xc9, + 0xcf, 0xdc, 0x16, 0x4e, 0x48, 0x3a, 0xc3, 0x0f, 0x80, 0xd6, 0x1f, 0x3b, 0x1d, 0x4e, 0x32, 0x51, + 0xbd, 0xe5, 0x7a, 0x81, 0x77, 0xa8, 0x75, 0x2c, 0x78, 0x87, 0x72, 0xfd, 0x71, 0x42, 0xc0, 0x32, + 0x28, 0x0c, 0xa8, 0xe7, 0x90, 0x80, 0x45, 0x3e, 0x89, 0x8d, 0xc5, 0x8d, 0xc5, 0xcd, 0x65, 0x04, + 0x06, 0xd4, 0xb3, 0x12, 0x0d, 0x5c, 0x05, 0x4b, 0x3d, 0x3f, 0xc0, 0x03, 0x71, 0x61, 0x0d, 0x25, + 0x87, 0xf5, 0x5f, 0x54, 0x90, 0x4d, 0x22, 0x42, 0x1b, 0x2c, 0x89, 0x8f, 0x17, 0xb1, 0x64, 0xae, + 0xce, 0x37, 0x66, 0x34, 0xc2, 0x1e, 0xb9, 0xa8, 0x72, 0x9b, 0x3b, 0xa5, 0xf5, 0x10, 0x08, 0x10, + 0x83, 0x55, 0xbe, 0xd2, 0x1c, 0xb9, 0xc1, 0x1c, 0xc9, 0x6c, 0xd9, 0xfe, 0x6b, 0x4f, 0x06, 0x8c, + 0x9e, 0x7d, 0x9e, 0xde, 0x01, 0x40, 0x3e, 0x1f, 0xfe, 0x43, 0x22, 0xa8, 0xb0, 0x88, 0xf2, 0xc9, + 0x13, 0xe0, 0x3f, 0x24, 0xbc, 0x1c, 0x2e, 0x0e, 0x9c, 0x2e, 0x71, 0x07, 0x7e, 0x40, 0xe4, 0x9d, + 0x81, 0x8b, 0x83, 0x66, 0xa2, 0xe1, 0x03, 0x1b, 0x46, 0x3e, 0x8d, 0x7c, 0x36, 0x11, 0x8f, 0xfc, + 0x8d, 0x2b, 0x29, 0x70, 0xb9, 0x41, 0x07, 0xd2, 0x0d, 0xcd, 0x00, 0x38, 0x58, 0xcc, 0xf8, 0x0e, + 0xf3, 0x26, 0x46, 0xee, 0x5a, 0x60, 0x6d, 0xe9, 0x86, 0x66, 0x00, 0xf0, 0x4b, 0xa0, 0xb2, 0x49, + 0xc8, 0x97, 0x37, 0x07, 0xfa, 0xf8, 0x25, 0x81, 0x0e, 0x27, 0x21, 0x41, 0xc2, 0x11, 0xde, 0x03, + 0xef, 0x0d, 0x88, 0x87, 0xdd, 0x89, 0x33, 0x0a, 0xe4, 0xac, 0x30, 0xd2, 0x75, 0x58, 0x34, 0x0a, + 0x12, 0x29, 0x69, 0xb3, 0x26, 0x6a, 0xb2, 0x91, 0x98, 0x1e, 0xcd, 0x59, 0x1e, 0xa6, 0x86, 0xa2, + 0xb3, 0x3b, 0xaa, 0xa6, 0xe8, 0x0b, 0x95, 0x3b, 0x40, 0x4b, 0x2f, 0x0e, 0x0b, 0x20, 0x77, 0xb4, + 0xd7, 0xda, 0xdb, 0xff, 0x6a, 0x4f, 0xcf, 0xc0, 0x65, 0xa0, 0x21, 0xab, 0xb1, 0x7f, 0x6c, 0xa1, + 0xaf, 0x75, 0x05, 0x16, 0x41, 0x1e, 0x59, 0xf5, 0xda, 0x6e, 0x6d, 0xaf, 0x61, 0xe9, 0x0b, 0x15, + 0x03, 0x68, 0xe9, 0x0d, 0xb9, 0x61, 0xeb, 0xd8, 0xa9, 0xd7, 0x0e, 0x1b, 0xdb, 0x7a, 0xa6, 0x72, + 0x0b, 0xa8, 0x3c, 0x65, 0xb8, 0x06, 0xe0, 0xb1, 0x5d, 0x73, 0xda, 0x7b, 0xb5, 0x83, 0xf6, 0xf6, + 0xfe, 0xa1, 0x73, 0xff, 0xc8, 0x3a, 0xb2, 0xf4, 0x0c, 0x8f, 0x61, 0xef, 0xd9, 0x87, 0x76, 0x6d, + 0x57, 0x57, 0x2a, 0xaa, 0xb6, 0xa0, 0x2f, 0x54, 0x7e, 0x53, 0x80, 0x7e, 0x71, 0x6d, 0xb9, 0xc1, + 0xee, 0x82, 0x2c, 0xbf, 0xce, 0x28, 0x16, 0x63, 0x76, 0x63, 0xcb, 0xfc, 0xc7, 0x7a, 0x25, 0x8e, + 0x66, 0x5b, 0x78, 0x21, 0xe9, 0xcd, 0x1f, 0xd9, 0xf4, 0x55, 0xe6, 0x2c, 0xcd, 0xcf, 0x9e, 0xdf, + 0x8a, 0x0d, 0xb2, 0x89, 0xed, 0x33, 0xf7, 0xae, 0x35, 0x1a, 0xd6, 0xc1, 0xa1, 0xd5, 0xd4, 0x15, + 0xfe, 0x53, 0xed, 0xe0, 0x60, 0xd7, 0xb6, 0x9a, 0xfa, 0x02, 0xcc, 0x83, 0x25, 0x0b, 0xa1, 0x7d, + 0xa4, 0x2f, 0x72, 0xab, 0xa6, 0xd5, 0xd8, 0xb5, 0xf7, 0xac, 0xa6, 0xae, 0xee, 0xa8, 0xda, 0xa2, + 0xae, 0x56, 0xbe, 0x05, 0xaf, 0x35, 0x68, 0xd0, 0x6b, 0x9c, 0x70, 0xb6, 0x36, 0x68, 0xc0, 0xc8, + 0x8f, 0x0c, 0x7e, 0x02, 0x00, 0xff, 0xfa, 0xc4, 0x41, 0x37, 0xdd, 0xc5, 0xf9, 0x7a, 0x71, 0x7a, + 0x56, 0xce, 0x37, 0x12, 0xad, 0xdd, 0x44, 0x79, 0x69, 0x60, 0x77, 0x79, 0xb6, 0x21, 0x9e, 0x0c, + 0x28, 0x4e, 0xbe, 0xd4, 0x97, 0x51, 0x7a, 0xac, 0x7f, 0x74, 0xfa, 0x67, 0x29, 0x73, 0x3a, 0x2d, + 0x29, 0x8f, 0xa6, 0x25, 0xe5, 0xf1, 0xb4, 0xa4, 0xfc, 0x31, 0x2d, 0x29, 0x3f, 0x9d, 0x97, 0x32, + 0x8f, 0xce, 0x4b, 0x99, 0xc7, 0xe7, 0xa5, 0xcc, 0x37, 0x5a, 0x5a, 0x93, 0x4e, 0x56, 0xfc, 0xc3, + 0x71, 0xfb, 0xef, 0x00, 0x00, 0x00, 0xff, 0xff, 0x2c, 0x3a, 0x80, 0x7f, 0x59, 0x0d, 0x00, 0x00, } func (m *RaftHeartbeat) Marshal() (dAtA []byte, err error) { @@ -1085,9 +1086,9 @@ func (m *SnapshotRequest_Header) MarshalToSizedBuffer(dAtA []byte) (int, error) i-- dAtA[i] = 0x48 } - if m.UnreplicatedTruncatedState { + if m.LegacyUnreplicatedTruncatedState { i-- - if m.UnreplicatedTruncatedState { + if m.LegacyUnreplicatedTruncatedState { dAtA[i] = 1 } else { dAtA[i] = 0 @@ -1402,7 +1403,7 @@ func (m *SnapshotRequest_Header) Size() (n int) { if m.Strategy != 0 { n += 1 + sovRaft(uint64(m.Strategy)) } - if m.UnreplicatedTruncatedState { + if m.LegacyUnreplicatedTruncatedState { n += 2 } if m.Type != 0 { @@ -2713,7 +2714,7 @@ func (m *SnapshotRequest_Header) Unmarshal(dAtA []byte) error { } case 8: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field UnreplicatedTruncatedState", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field LegacyUnreplicatedTruncatedState", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -2730,7 +2731,7 @@ func (m *SnapshotRequest_Header) Unmarshal(dAtA []byte) error { break } } - m.UnreplicatedTruncatedState = bool(v != 0) + m.LegacyUnreplicatedTruncatedState = bool(v != 0) case 9: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) diff --git a/pkg/kv/kvserver/raft.proto b/pkg/kv/kvserver/raft.proto index d525efe43b0d..eb531fdaa340 100644 --- a/pkg/kv/kvserver/raft.proto +++ b/pkg/kv/kvserver/raft.proto @@ -183,15 +183,16 @@ message SnapshotRequest { Type type = 9; // Whether the snapshot uses the unreplicated RaftTruncatedState or not. - // This is generally always true at 2.2 and above outside of the migration - // phase, though theoretically it could take a long time for all ranges - // to update to the new mechanism. This bool is true iff the Raft log at - // the snapshot's applied index is using the new key. In particular, it - // is true if the index itself carries out the migration (in which case - // the data in the snapshot contains neither key). + // This is always true for snapshots generated in v21.1+ clusters. In v20.2 + // it was possible for ranges to be using the replicated variant. v21.1 + // therefore had code expecting that possibility (unlike v21.2 code, where + // this field is assumed to always be true and thus never read). For + // compatibility with v21.1 nodes however, v21.2 has to explicitly set this + // field to true. In v22.1 we can remove it entirely seeing as how v21.2 + // code never reads the field. // - // See VersionUnreplicatedRaftTruncatedState. - bool unreplicated_truncated_state = 8; + // TODO(irfansharif): Remove this in v22.1. + bool legacy_unreplicated_truncated_state = 8; } Header header = 1; diff --git a/pkg/kv/kvserver/rditer/replica_data_iter_test.go b/pkg/kv/kvserver/rditer/replica_data_iter_test.go index 0f4aecaabbfc..8f071be6f7e8 100644 --- a/pkg/kv/kvserver/rditer/replica_data_iter_test.go +++ b/pkg/kv/kvserver/rditer/replica_data_iter_test.go @@ -82,11 +82,7 @@ func createRangeData( {keys.AbortSpanKey(desc.RangeID, testTxnID2), ts0}, {keys.RangeGCThresholdKey(desc.RangeID), ts0}, {keys.RangeAppliedStateKey(desc.RangeID), ts0}, - {keys.RaftAppliedIndexLegacyKey(desc.RangeID), ts0}, - {keys.RaftTruncatedStateLegacyKey(desc.RangeID), ts0}, {keys.RangeLeaseKey(desc.RangeID), ts0}, - {keys.LeaseAppliedIndexLegacyKey(desc.RangeID), ts0}, - {keys.RangeStatsLegacyKey(desc.RangeID), ts0}, {keys.RangeTombstoneKey(desc.RangeID), ts0}, {keys.RaftHardStateKey(desc.RangeID), ts0}, {keys.RaftLogKey(desc.RangeID, 1), ts0}, diff --git a/pkg/kv/kvserver/replica_application_result.go b/pkg/kv/kvserver/replica_application_result.go index 7c933ef1bd35..33e1a3e33789 100644 --- a/pkg/kv/kvserver/replica_application_result.go +++ b/pkg/kv/kvserver/replica_application_result.go @@ -320,12 +320,6 @@ func (r *Replica) handleVersionResult(ctx context.Context, version *roachpb.Vers r.mu.Unlock() } -func (r *Replica) handleUsingAppliedStateKeyResult(ctx context.Context) { - r.mu.Lock() - r.mu.state.UsingAppliedStateKey = true - r.mu.Unlock() -} - func (r *Replica) handleComputeChecksumResult(ctx context.Context, cc *kvserverpb.ComputeChecksum) { r.computeChecksumPostApply(ctx, *cc) } diff --git a/pkg/kv/kvserver/replica_application_state_machine.go b/pkg/kv/kvserver/replica_application_state_machine.go index c5112c60e764..165bd523f828 100644 --- a/pkg/kv/kvserver/replica_application_state_machine.go +++ b/pkg/kv/kvserver/replica_application_state_machine.go @@ -15,7 +15,6 @@ import ( "fmt" "time" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/apply" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" @@ -396,10 +395,6 @@ type replicaAppBatch struct { // closed timestamp carried by this command. Synthetic timestamps are not // registered here. maxTS hlc.ClockTimestamp - // migrateToAppliedStateKey tracks whether any command in the batch - // triggered a migration to the replica applied state key. If so, this - // migration will be performed when the application batch is committed. - migrateToAppliedStateKey bool // changeRemovesReplica tracks whether the command in the batch (there must // be only one) removes this replica from the range. changeRemovesReplica bool @@ -729,19 +724,8 @@ func (b *replicaAppBatch) runPreApplyTriggersAfterStagingWriteBatch( } if res.State != nil && res.State.TruncatedState != nil { - activeVersion := b.r.ClusterSettings().Version.ActiveVersion(ctx).Version - migrationVersion := clusterversion.ByKey(clusterversion.TruncatedAndRangeAppliedStateMigration) - // NB: We're being deliberate here in using the less-than operator (as - // opposed to LessEq). TruncatedAndRangeAppliedStateMigration indicates - // that the migration to move to the unreplicated truncated - // state is currently underway. It's only when the active cluster - // version has moved past it that we can assume that the migration has - // completed. - assertNoLegacy := migrationVersion.Less(activeVersion) - if apply, err := handleTruncatedStateBelowRaft( ctx, b.state.TruncatedState, res.State.TruncatedState, b.r.raftMu.stateLoader, b.batch, - assertNoLegacy, ); err != nil { return wrapWithNonDeterministicFailure(err, "unable to handle truncated state") } else if !apply { @@ -851,10 +835,6 @@ func (b *replicaAppBatch) stageTrivialReplicatedEvalResult( // serialize on the stats key. deltaStats := res.Delta.ToStats() b.state.Stats.Add(deltaStats) - - if res.State != nil && res.State.UsingAppliedStateKey && !b.state.UsingAppliedStateKey { - b.migrateToAppliedStateKey = true - } } // ApplyToStateMachine implements the apply.Batch interface. The method handles @@ -960,48 +940,13 @@ func (b *replicaAppBatch) ApplyToStateMachine(ctx context.Context) error { // batch's RocksDB batch. This records the highest raft and lease index that // have been applied as of this batch. It also records the Range's mvcc stats. func (b *replicaAppBatch) addAppliedStateKeyToBatch(ctx context.Context) error { + // Set the range applied state, which includes the last applied raft and + // lease index along with the mvcc stats, all in one key. loader := &b.r.raftMu.stateLoader - if b.migrateToAppliedStateKey { - // A Raft command wants us to begin using the RangeAppliedState key - // and we haven't performed the migration yet. Delete the old keys - // that this new key is replacing. - // - // NB: entering this branch indicates that the batch contains only a - // single non-trivial command. - err := loader.MigrateToRangeAppliedStateKey(ctx, b.batch, b.state.Stats) - if err != nil { - return wrapWithNonDeterministicFailure(err, "unable to migrate to range applied state") - } - b.state.UsingAppliedStateKey = true - } - if b.state.UsingAppliedStateKey { - // Set the range applied state, which includes the last applied raft and - // lease index along with the mvcc stats, all in one key. - if err := loader.SetRangeAppliedState( - ctx, b.batch, b.state.RaftAppliedIndex, b.state.LeaseAppliedIndex, - b.state.Stats, &b.state.RaftClosedTimestamp, - ); err != nil { - return wrapWithNonDeterministicFailure(err, "unable to set range applied state") - } - } else { - // Advance the last applied index. We use a blind write in order to avoid - // reading the previous applied index keys on every write operation. This - // requires a little additional work in order maintain the MVCC stats. - var appliedIndexNewMS enginepb.MVCCStats - if err := loader.SetLegacyAppliedIndexBlind( - ctx, b.batch, &appliedIndexNewMS, b.state.RaftAppliedIndex, b.state.LeaseAppliedIndex, - ); err != nil { - return wrapWithNonDeterministicFailure(err, "unable to set applied index") - } - b.state.Stats.SysBytes += appliedIndexNewMS.SysBytes - - loader.CalcAppliedIndexSysBytes(b.state.RaftAppliedIndex, b.state.LeaseAppliedIndex) - - // Set the legacy MVCC stats key. - if err := loader.SetMVCCStats(ctx, b.batch, b.state.Stats); err != nil { - return wrapWithNonDeterministicFailure(err, "unable to update MVCCStats") - } - } - return nil + return loader.SetRangeAppliedState( + ctx, b.batch, b.state.RaftAppliedIndex, b.state.LeaseAppliedIndex, + b.state.Stats, &b.state.RaftClosedTimestamp, + ) } func (b *replicaAppBatch) recordStatsOnCommit() { @@ -1301,11 +1246,6 @@ func (sm *replicaStateMachine) handleNonTrivialReplicatedEvalResult( rResult.State.Desc = nil } - if rResult.State.UsingAppliedStateKey { - sm.r.handleUsingAppliedStateKeyResult(ctx) - rResult.State.UsingAppliedStateKey = false - } - if (*rResult.State == kvserverpb.ReplicaState{}) { rResult.State = nil } diff --git a/pkg/kv/kvserver/replica_command.go b/pkg/kv/kvserver/replica_command.go index 142bde8cf25f..6476c71facf9 100644 --- a/pkg/kv/kvserver/replica_command.go +++ b/pkg/kv/kvserver/replica_command.go @@ -2440,15 +2440,7 @@ func (r *Replica) sendSnapshot( return &benignError{errors.Wrap(errMarkSnapshotError, "raft status not initialized")} } - usesReplicatedTruncatedState, err := storage.MVCCGetProto( - ctx, snap.EngineSnap, keys.RaftTruncatedStateLegacyKey(r.RangeID), hlc.Timestamp{}, nil, storage.MVCCGetOptions{}, - ) - if err != nil { - return errors.Wrap(err, "loading legacy truncated state") - } - - canAvoidSendingLog := !usesReplicatedTruncatedState && - snap.State.TruncatedState.Index < snap.State.RaftAppliedIndex + canAvoidSendingLog := snap.State.TruncatedState.Index < snap.State.RaftAppliedIndex if canAvoidSendingLog { // If we're not using a legacy (replicated) truncated state, we avoid @@ -2467,15 +2459,8 @@ func (r *Replica) sendSnapshot( } req := SnapshotRequest_Header{ - State: snap.State, - // Tell the recipient whether it needs to synthesize the new - // unreplicated TruncatedState. It could tell by itself by peeking into - // the data, but it uses a write only batch for performance which - // doesn't support that; this is easier. Notably, this is true if the - // snap index itself is the one at which the migration happens. - // - // See VersionUnreplicatedRaftTruncatedState. - UnreplicatedTruncatedState: !usesReplicatedTruncatedState, + State: snap.State, + LegacyUnreplicatedTruncatedState: true, RaftMessageRequest: RaftMessageRequest{ RangeID: r.RangeID, FromReplica: sender, diff --git a/pkg/kv/kvserver/replica_proposal.go b/pkg/kv/kvserver/replica_proposal.go index 5ffb39d9966c..53271d6d72ea 100644 --- a/pkg/kv/kvserver/replica_proposal.go +++ b/pkg/kv/kvserver/replica_proposal.go @@ -18,7 +18,6 @@ import ( "unsafe" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" @@ -863,41 +862,6 @@ func (r *Replica) evaluateProposal( if res.Replicated.Delta.ContainsEstimates > 0 { res.Replicated.Delta.ContainsEstimates *= 2 } - - // If the RangeAppliedState key is not being used and the cluster version is - // high enough to guarantee that all current and future binaries will - // understand the key, we send the migration flag through Raft. Because - // there is a delay between command proposal and application, we may end up - // setting this migration flag multiple times. This is ok, because the - // migration is idempotent. - // TODO(nvanbenschoten): This will be baked in to 2.1, so it can be removed - // in the 2.2 release. - r.mu.RLock() - usingAppliedStateKey := r.mu.state.UsingAppliedStateKey - r.mu.RUnlock() - if !usingAppliedStateKey { - // The range applied state was originally introduced in v2.1, and in - // v21.1 we guarantee that it's used for all ranges, which we assert - // on below. If we're not running 21.1 yet, migrate over as we've - // done since the introduction of the applied state key. - activeVersion := r.ClusterSettings().Version.ActiveVersion(ctx).Version - migrationVersion := clusterversion.ByKey(clusterversion.TruncatedAndRangeAppliedStateMigration) - if migrationVersion.Less(activeVersion) { - log.Fatal(ctx, "not using applied state key in v21.1") - } - // The range applied state was introduced in v2.1. It's possible to - // still find ranges that haven't activated it. If so, activate it. - // We can remove this code if we introduce a boot-time check that - // fails the startup process when any legacy replicas are found. The - // operator can then run the old binary for a while to upgrade the - // stragglers. - // - // TODO(irfansharif): Is this still applicable? - if res.Replicated.State == nil { - res.Replicated.State = &kvserverpb.ReplicaState{} - } - res.Replicated.State.UsingAppliedStateKey = true - } } return &res, needConsensus, nil diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go index ffd3ac5347f6..771aa806ccb4 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -1788,16 +1788,16 @@ func (r *Replica) acquireMergeLock( return rightRepl.raftMu.Unlock, nil } -// handleTruncatedStateBelowRaft is called when a Raft command updates the truncated -// state. This isn't 100% trivial for two reasons: -// - in 19.1 we're making the TruncatedState key unreplicated, so there's a migration -// - we're making use of the above by not sending the Raft log in snapshots (the truncated -// state effectively determines the first index of the log, which requires it to be unreplicated). -// Updates to the HardState are sent out by a leaseholder truncating the log based on its local -// knowledge. For example, the leader might have a log 10..100 and truncates to 50, and will send -// out a TruncatedState with Index 50 to that effect. However, some replicas may not even have log -// entries that old, and must make sure to ignore this update to the truncated state, as it would -// otherwise clobber their "newer" truncated state. +// handleTruncatedStateBelowRaft is called when a Raft command updates the +// truncated state. This isn't 100% trivial. The truncated state is +// unreplicated, and effectively determines the first index of the log. We make +// use of this fact by not sending the raft log in snapshots. Updates to the +// HardState are sent out by a leaseholder truncating the log based on its local +// knowledge. For example, the leader might have a log 10..100 and truncates to +// 50, and will send out a TruncatedState with Index 50 to that effect. However, +// some replicas may not even have log entries that old, and must make sure to +// ignore this update to the truncated state, as it would otherwise clobber +// their "newer" truncated state. // // The returned boolean tells the caller whether to apply the truncated state's // side effects, which means replacing the in-memory TruncatedState and applying @@ -1809,25 +1809,12 @@ func handleTruncatedStateBelowRaft( oldTruncatedState, newTruncatedState *roachpb.RaftTruncatedState, loader stateloader.StateLoader, readWriter storage.ReadWriter, - assertNoLegacy bool, ) (_apply bool, _ error) { - // If this is a log truncation, load the resulting unreplicated or legacy - // replicated truncated state (in that order). If the migration is happening - // in this command, the result will be an empty message. In steady state - // after the migration, it's the unreplicated truncated state not taking - // into account the current truncation (since the key is unreplicated). - // Either way, we'll update it below. - // - // See VersionUnreplicatedRaftTruncatedState for details. - truncStatePostApply, truncStateIsLegacy, err := loader.LoadRaftTruncatedState(ctx, readWriter) + truncStatePostApply, err := loader.LoadRaftTruncatedState(ctx, readWriter) if err != nil { return false, errors.Wrap(err, "loading truncated state") } - if assertNoLegacy && truncStateIsLegacy { - log.Fatalf(ctx, "found legacy truncated state which should no longer exist") - } - // Truncate the Raft log from the entry after the previous // truncation index to the new truncation index. This is performed // atomically with the raft command application so that the @@ -1850,34 +1837,20 @@ func handleTruncatedStateBelowRaft( } } - if !truncStateIsLegacy { - if truncStatePostApply.Index < newTruncatedState.Index { - // There are two cases here (though handled just the same). In the - // first case, the Raft command has just deleted the legacy - // replicated truncated state key as part of the migration (so - // truncStateIsLegacy is now false for the first time and - // truncStatePostApply is zero) and we need to atomically write the - // new, unreplicated, key. Or we've already migrated earlier, in - // which case truncStatePostApply equals the current value of the - // new key (which wasn't touched by the batch), and we need to - // overwrite it if this truncation "moves it forward". - - if err := storage.MVCCPutProto( - ctx, readWriter, nil /* ms */, prefixBuf.RaftTruncatedStateKey(), - hlc.Timestamp{}, nil /* txn */, newTruncatedState, - ); err != nil { - return false, errors.Wrap(err, "unable to migrate RaftTruncatedState") - } - // Have migrated and this new truncated state is moving us forward. - // Tell caller that we applied it and that so should they. - return true, nil + if truncStatePostApply.Index < newTruncatedState.Index { + if err := storage.MVCCPutProto( + ctx, readWriter, nil /* ms */, prefixBuf.RaftTruncatedStateKey(), + hlc.Timestamp{}, nil /* txn */, newTruncatedState, + ); err != nil { + return false, errors.Wrap(err, "unable to migrate RaftTruncatedState") } - // Have migrated, but this truncated state moves the existing one - // backwards, so instruct caller to not update in-memory state. - return false, nil + // The new truncated state is moving us forward. Tell caller that we + // applied it and that so should they. + return true, nil } - // Haven't migrated yet, don't ever discard the update. - return true, nil + // The truncated state moves the existing one backwards; instruct caller + // to not update in-memory state. + return false, nil } // ComputeRaftLogSize computes the size (in bytes) of the Raft log from the diff --git a/pkg/kv/kvserver/replica_raft_truncation_test.go b/pkg/kv/kvserver/replica_raft_truncation_test.go index 4867384b1c10..2cf87ff7b5c4 100644 --- a/pkg/kv/kvserver/replica_raft_truncation_test.go +++ b/pkg/kv/kvserver/replica_raft_truncation_test.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/datadriven" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) func TestHandleTruncatedStateBelowRaft(t *testing.T) { @@ -32,18 +33,11 @@ func TestHandleTruncatedStateBelowRaft(t *testing.T) { defer log.Scope(t).Close(t) // This test verifies the expected behavior of the downstream-of-Raft log - // truncation code, in particular regarding the - // VersionUnreplicatedRaftTruncatedState migration. + // truncation code. ctx := context.Background() - - // neither exists (migration) - // old one exists (no migration) - // new one exists (migrated already) - // truncstate regresses - var prevTruncatedState roachpb.RaftTruncatedState - datadriven.Walk(t, "testdata/truncated_state_migration", func(t *testing.T, path string) { + datadriven.Walk(t, "testdata/truncated_state", func(t *testing.T, path string) { const rangeID = 12 loader := stateloader.Make(rangeID) eng := storage.NewDefaultInMemForTesting() @@ -58,21 +52,15 @@ func TestHandleTruncatedStateBelowRaft(t *testing.T) { case "put": var index uint64 var term uint64 - var legacy bool d.ScanArgs(t, "index", &index) d.ScanArgs(t, "term", &term) - d.ScanArgs(t, "legacy", &legacy) truncState := &roachpb.RaftTruncatedState{ Index: index, Term: term, } - if legacy { - assert.NoError(t, loader.SetLegacyRaftTruncatedState(ctx, eng, nil, truncState)) - } else { - assert.NoError(t, loader.SetRaftTruncatedState(ctx, eng, truncState)) - } + assert.NoError(t, loader.SetRaftTruncatedState(ctx, eng, truncState)) return "" case "handle": var buf bytes.Buffer @@ -87,26 +75,20 @@ func TestHandleTruncatedStateBelowRaft(t *testing.T) { Term: term, } - apply, err := handleTruncatedStateBelowRaft(ctx, &prevTruncatedState, newTruncatedState, loader, eng, false) + apply, err := handleTruncatedStateBelowRaft(ctx, &prevTruncatedState, newTruncatedState, loader, eng) if err != nil { return err.Error() } fmt.Fprintf(&buf, "apply: %t\n", apply) - for _, key := range []roachpb.Key{ - keys.RaftTruncatedStateLegacyKey(rangeID), - keys.RaftTruncatedStateKey(rangeID), - } { - var truncatedState roachpb.RaftTruncatedState - ok, err := storage.MVCCGetProto(ctx, eng, key, hlc.Timestamp{}, &truncatedState, storage.MVCCGetOptions{}) - if err != nil { - t.Fatal(err) - } - if !ok { - continue - } - fmt.Fprintf(&buf, "%s -> index=%d term=%d\n", key, truncatedState.Index, truncatedState.Term) + key := keys.RaftTruncatedStateKey(rangeID) + var truncatedState roachpb.RaftTruncatedState + ok, err := storage.MVCCGetProto(ctx, eng, key, hlc.Timestamp{}, &truncatedState, storage.MVCCGetOptions{}) + if err != nil { + t.Fatal(err) } + require.True(t, ok) + fmt.Fprintf(&buf, "%s -> index=%d term=%d\n", key, truncatedState.Index, truncatedState.Term) return buf.String() default: } diff --git a/pkg/kv/kvserver/replica_raftstorage.go b/pkg/kv/kvserver/replica_raftstorage.go index 226c54c261f1..8587db094378 100644 --- a/pkg/kv/kvserver/replica_raftstorage.go +++ b/pkg/kv/kvserver/replica_raftstorage.go @@ -209,7 +209,7 @@ func entries( } // No results, was it due to unavailability or truncation? - ts, _, err := rsl.LoadRaftTruncatedState(ctx, reader) + ts, err := rsl.LoadRaftTruncatedState(ctx, reader) if err != nil { return nil, err } @@ -305,7 +305,7 @@ func term( // sideloaded entries. We only need the term, so this is what we do. ents, err := entries(ctx, rsl, reader, rangeID, eCache, nil /* sideloaded */, i, i+1, math.MaxUint64 /* maxBytes */) if errors.Is(err, raft.ErrCompacted) { - ts, _, err := rsl.LoadRaftTruncatedState(ctx, reader) + ts, err := rsl.LoadRaftTruncatedState(ctx, reader) if err != nil { return 0, err } @@ -342,7 +342,7 @@ func (r *Replica) raftTruncatedStateLocked( if r.mu.state.TruncatedState != nil { return *r.mu.state.TruncatedState, nil } - ts, _, err := r.mu.stateLoader.LoadRaftTruncatedState(ctx, r.store.Engine()) + ts, err := r.mu.stateLoader.LoadRaftTruncatedState(ctx, r.store.Engine()) if err != nil { return ts, err } @@ -519,18 +519,9 @@ type IncomingSnapshot struct { // The Raft log entries for this snapshot. LogEntries [][]byte // The replica state at the time the snapshot was generated (never nil). - State *kvserverpb.ReplicaState - // - // When true, this snapshot contains an unreplicated TruncatedState. When - // false, the TruncatedState is replicated (see the reference below) and the - // recipient must avoid also writing the unreplicated TruncatedState. The - // migration to an unreplicated TruncatedState will be carried out during - // the next log truncation (assuming cluster version is bumped at that - // point). - // See the comment on VersionUnreplicatedRaftTruncatedState for details. - UsesUnreplicatedTruncatedState bool - snapType SnapshotRequest_Type - placeholder *ReplicaPlaceholder + State *kvserverpb.ReplicaState + snapType SnapshotRequest_Type + placeholder *ReplicaPlaceholder } func (s *IncomingSnapshot) String() string { @@ -886,13 +877,10 @@ func (r *Replica) applySnapshot( } r.store.raftEntryCache.Drop(r.RangeID) - // Update TruncatedState if it is unreplicated. - if inSnap.UsesUnreplicatedTruncatedState { - if err := r.raftMu.stateLoader.SetRaftTruncatedState( - ctx, &unreplicatedSST, s.TruncatedState, - ); err != nil { - return errors.Wrapf(err, "unable to write UnreplicatedTruncatedState to unreplicated SST writer") - } + if err := r.raftMu.stateLoader.SetRaftTruncatedState( + ctx, &unreplicatedSST, s.TruncatedState, + ); err != nil { + return errors.Wrapf(err, "unable to write TruncatedState to unreplicated SST writer") } if err := unreplicatedSST.Finish(); err != nil { diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index f7d31de02a4e..381f14669c82 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -612,9 +612,6 @@ func TestReplicaContains(t *testing.T) { r.mu.state.Desc = desc r.rangeStr.store(0, desc) - if statsKey := keys.RangeStatsLegacyKey(desc.RangeID); !r.ContainsKey(statsKey) { - t.Errorf("expected range to contain range stats key %q", statsKey) - } if !r.ContainsKey(roachpb.Key("aa")) { t.Errorf("expected range to contain key \"aa\"") } diff --git a/pkg/kv/kvserver/stateloader/initial.go b/pkg/kv/kvserver/stateloader/initial.go index 65177fea2c69..f1688147a310 100644 --- a/pkg/kv/kvserver/stateloader/initial.go +++ b/pkg/kv/kvserver/stateloader/initial.go @@ -45,7 +45,6 @@ func WriteInitialReplicaState( desc roachpb.RangeDescriptor, lease roachpb.Lease, gcThreshold hlc.Timestamp, - truncStateType TruncatedStateType, replicaVersion roachpb.Version, ) (enginepb.MVCCStats, error) { rsl := Make(desc.RangeID) @@ -64,9 +63,6 @@ func WriteInitialReplicaState( if (replicaVersion != roachpb.Version{}) { s.Version = &replicaVersion } - if truncStateType != TruncatedStateLegacyReplicatedAndNoAppliedKey { - s.UsingAppliedStateKey = true - } if existingLease, err := rsl.LoadLease(ctx, readWriter); err != nil { return enginepb.MVCCStats{}, errors.Wrap(err, "error reading lease") @@ -86,7 +82,7 @@ func WriteInitialReplicaState( log.Fatalf(ctx, "expected trivial version, but found %+v", existingVersion) } - newMS, err := rsl.Save(ctx, readWriter, s, truncStateType) + newMS, err := rsl.Save(ctx, readWriter, s) if err != nil { return enginepb.MVCCStats{}, err } @@ -101,32 +97,14 @@ func WriteInitialRangeState( readWriter storage.ReadWriter, desc roachpb.RangeDescriptor, replicaVersion roachpb.Version, -) error { - const initialTruncStateType = TruncatedStateUnreplicated - return WriteInitialRangeStateWithTruncatedState(ctx, readWriter, desc, replicaVersion, initialTruncStateType) -} - -// WriteInitialRangeStateWithTruncatedState is the same as -// WriteInitialRangeState, but allows the caller to override the truncated state -// type. -// -// TODO(irfansharif): This can be removed in the v21.2 cycle after we no longer -// need to test the truncated state migration. -func WriteInitialRangeStateWithTruncatedState( - ctx context.Context, - readWriter storage.ReadWriter, - desc roachpb.RangeDescriptor, - replicaVersion roachpb.Version, - truncState TruncatedStateType, ) error { initialLease := roachpb.Lease{} initialGCThreshold := hlc.Timestamp{} initialMS := enginepb.MVCCStats{} - initialTruncStateType := truncState if _, err := WriteInitialReplicaState( ctx, readWriter, initialMS, desc, initialLease, initialGCThreshold, - initialTruncStateType, replicaVersion, + replicaVersion, ); err != nil { return err } diff --git a/pkg/kv/kvserver/stateloader/stateloader.go b/pkg/kv/kvserver/stateloader/stateloader.go index d296219d0697..b05d5bca2b1a 100644 --- a/pkg/kv/kvserver/stateloader/stateloader.go +++ b/pkg/kv/kvserver/stateloader/stateloader.go @@ -73,34 +73,21 @@ func (rsl StateLoader) Load( return kvserverpb.ReplicaState{}, err } - if as, err := rsl.LoadRangeAppliedState(ctx, reader); err != nil { + as, err := rsl.LoadRangeAppliedState(ctx, reader) + if err != nil { return kvserverpb.ReplicaState{}, err - } else if as != nil { - s.UsingAppliedStateKey = true - - s.RaftAppliedIndex = as.RaftAppliedIndex - s.LeaseAppliedIndex = as.LeaseAppliedIndex - - ms := as.RangeStats.ToStats() - s.Stats = &ms - if as.RaftClosedTimestamp != nil { - s.RaftClosedTimestamp = *as.RaftClosedTimestamp - } - } else { - if s.RaftAppliedIndex, s.LeaseAppliedIndex, err = rsl.LoadAppliedIndex(ctx, reader); err != nil { - return kvserverpb.ReplicaState{}, err - } - - ms, err := rsl.LoadMVCCStats(ctx, reader) - if err != nil { - return kvserverpb.ReplicaState{}, err - } - s.Stats = &ms + } + s.RaftAppliedIndex = as.RaftAppliedIndex + s.LeaseAppliedIndex = as.LeaseAppliedIndex + ms := as.RangeStats.ToStats() + s.Stats = &ms + if as.RaftClosedTimestamp != nil { + s.RaftClosedTimestamp = *as.RaftClosedTimestamp } // The truncated state should not be optional (i.e. the pointer is // pointless), but it is and the migration is not worth it. - truncState, _, err := rsl.LoadRaftTruncatedState(ctx, reader) + truncState, err := rsl.LoadRaftTruncatedState(ctx, reader) if err != nil { return kvserverpb.ReplicaState{}, err } @@ -117,20 +104,6 @@ func (rsl StateLoader) Load( return s, nil } -// TruncatedStateType determines whether to use a replicated (legacy) or an -// unreplicated TruncatedState. See VersionUnreplicatedRaftTruncatedStateKey. -type TruncatedStateType int - -const ( - // TruncatedStateLegacyReplicated means use the legacy (replicated) key. - TruncatedStateLegacyReplicated TruncatedStateType = iota - // TruncatedStateLegacyReplicatedAndNoAppliedKey means use the legacy key - // and also don't use the RangeAppliedKey. This is for testing use only. - TruncatedStateLegacyReplicatedAndNoAppliedKey - // TruncatedStateUnreplicated means use the new (unreplicated) key. - TruncatedStateUnreplicated -) - // Save persists the given ReplicaState to disk. It assumes that the contained // Stats are up-to-date and returns the stats which result from writing the // updated State. @@ -146,7 +119,6 @@ func (rsl StateLoader) Save( ctx context.Context, readWriter storage.ReadWriter, state kvserverpb.ReplicaState, - truncStateType TruncatedStateType, ) (enginepb.MVCCStats, error) { ms := state.Stats if err := rsl.SetLease(ctx, readWriter, ms, *state.Lease); err != nil { @@ -155,34 +127,17 @@ func (rsl StateLoader) Save( if err := rsl.SetGCThreshold(ctx, readWriter, ms, state.GCThreshold); err != nil { return enginepb.MVCCStats{}, err } - if truncStateType != TruncatedStateUnreplicated { - if err := rsl.SetLegacyRaftTruncatedState(ctx, readWriter, ms, state.TruncatedState); err != nil { - return enginepb.MVCCStats{}, err - } - } else { - if err := rsl.SetRaftTruncatedState(ctx, readWriter, state.TruncatedState); err != nil { - return enginepb.MVCCStats{}, err - } + if err := rsl.SetRaftTruncatedState(ctx, readWriter, state.TruncatedState); err != nil { + return enginepb.MVCCStats{}, err } if state.Version != nil { if err := rsl.SetVersion(ctx, readWriter, ms, state.Version); err != nil { return enginepb.MVCCStats{}, err } } - if state.UsingAppliedStateKey { - rai, lai, ct := state.RaftAppliedIndex, state.LeaseAppliedIndex, &state.RaftClosedTimestamp - if err := rsl.SetRangeAppliedState(ctx, readWriter, rai, lai, ms, ct); err != nil { - return enginepb.MVCCStats{}, err - } - } else { - if err := rsl.SetLegacyAppliedIndex( - ctx, readWriter, ms, state.RaftAppliedIndex, state.LeaseAppliedIndex, - ); err != nil { - return enginepb.MVCCStats{}, err - } - if err := rsl.SetLegacyMVCCStats(ctx, readWriter, ms); err != nil { - return enginepb.MVCCStats{}, err - } + rai, lai, ct := state.RaftAppliedIndex, state.LeaseAppliedIndex, &state.RaftClosedTimestamp + if err := rsl.SetRangeAppliedState(ctx, readWriter, rai, lai, ms, ct); err != nil { + return enginepb.MVCCStats{}, err } return *ms, nil } @@ -219,56 +174,16 @@ func (rsl StateLoader) LoadRangeAppliedState( return &as, err } -// AssertNoRangeAppliedState asserts that no Range applied state key is present. -func (rsl StateLoader) AssertNoRangeAppliedState(ctx context.Context, reader storage.Reader) error { - if as, err := rsl.LoadRangeAppliedState(ctx, reader); err != nil { - return err - } else if as != nil { - log.Fatalf(ctx, "unexpected RangeAppliedState present: %v", as) - } - return nil -} - // LoadAppliedIndex returns the Raft applied index and the lease applied index. func (rsl StateLoader) LoadAppliedIndex( ctx context.Context, reader storage.Reader, ) (raftAppliedIndex uint64, leaseAppliedIndex uint64, err error) { // Check the applied state key. - if as, err := rsl.LoadRangeAppliedState(ctx, reader); err != nil { - return 0, 0, err - } else if as != nil { - return as.RaftAppliedIndex, as.LeaseAppliedIndex, nil - } - - // If the range applied state is not found, check the legacy Raft applied - // index and the lease applied index keys. This is where these indices were - // stored before the range applied state was introduced. - v, _, err := storage.MVCCGet(ctx, reader, rsl.RaftAppliedIndexLegacyKey(), - hlc.Timestamp{}, storage.MVCCGetOptions{}) - if err != nil { - return 0, 0, err - } - if v != nil { - int64AppliedIndex, err := v.GetInt() - if err != nil { - return 0, 0, err - } - raftAppliedIndex = uint64(int64AppliedIndex) - } - // TODO(tschottdorf): code duplication. - v, _, err = storage.MVCCGet(ctx, reader, rsl.LeaseAppliedIndexLegacyKey(), - hlc.Timestamp{}, storage.MVCCGetOptions{}) + as, err := rsl.LoadRangeAppliedState(ctx, reader) if err != nil { return 0, 0, err } - if v != nil { - int64LeaseAppliedIndex, err := v.GetInt() - if err != nil { - return 0, 0, err - } - leaseAppliedIndex = uint64(int64LeaseAppliedIndex) - } - return raftAppliedIndex, leaseAppliedIndex, nil + return as.RaftAppliedIndex, as.LeaseAppliedIndex, nil } // LoadMVCCStats loads the MVCC stats. @@ -276,19 +191,11 @@ func (rsl StateLoader) LoadMVCCStats( ctx context.Context, reader storage.Reader, ) (enginepb.MVCCStats, error) { // Check the applied state key. - if as, err := rsl.LoadRangeAppliedState(ctx, reader); err != nil { + as, err := rsl.LoadRangeAppliedState(ctx, reader) + if err != nil { return enginepb.MVCCStats{}, err - } else if as != nil { - return as.RangeStats.ToStats(), nil } - - // If the range applied state is not found, check the legacy stats - // key. This is where stats were stored before the range applied - // state was introduced. - var ms enginepb.MVCCStats - _, err := storage.MVCCGetProto(ctx, reader, rsl.RangeStatsLegacyKey(), hlc.Timestamp{}, &ms, - storage.MVCCGetOptions{}) - return ms, err + return as.RangeStats.ToStats(), nil } // SetRangeAppliedState overwrites the range applied state. This state is a @@ -323,88 +230,6 @@ func (rsl StateLoader) SetRangeAppliedState( return storage.MVCCPutProto(ctx, readWriter, ms, rsl.RangeAppliedStateKey(), hlc.Timestamp{}, nil, &as) } -// MigrateToRangeAppliedStateKey deletes the keys that were replaced by the -// RangeAppliedState key. -func (rsl StateLoader) MigrateToRangeAppliedStateKey( - ctx context.Context, readWriter storage.ReadWriter, ms *enginepb.MVCCStats, -) error { - noTS := hlc.Timestamp{} - if err := storage.MVCCDelete(ctx, readWriter, ms, rsl.RaftAppliedIndexLegacyKey(), noTS, nil); err != nil { - return err - } - if err := storage.MVCCDelete(ctx, readWriter, ms, rsl.LeaseAppliedIndexLegacyKey(), noTS, nil); err != nil { - return err - } - return storage.MVCCDelete(ctx, readWriter, ms, rsl.RangeStatsLegacyKey(), noTS, nil) -} - -// SetLegacyAppliedIndex sets the legacy {raft,lease} applied index values, -// properly accounting for existing keys in the returned stats. -// -// The range applied state key cannot already exist or an assetion will be -// triggered. See comment on SetRangeAppliedState for why this is "legacy". -func (rsl StateLoader) SetLegacyAppliedIndex( - ctx context.Context, - readWriter storage.ReadWriter, - ms *enginepb.MVCCStats, - appliedIndex, leaseAppliedIndex uint64, -) error { - if err := rsl.AssertNoRangeAppliedState(ctx, readWriter); err != nil { - return err - } - - var value roachpb.Value - value.SetInt(int64(appliedIndex)) - if err := storage.MVCCPut(ctx, readWriter, ms, - rsl.RaftAppliedIndexLegacyKey(), - hlc.Timestamp{}, - value, - nil /* txn */); err != nil { - return err - } - value.SetInt(int64(leaseAppliedIndex)) - return storage.MVCCPut(ctx, readWriter, ms, - rsl.LeaseAppliedIndexLegacyKey(), - hlc.Timestamp{}, - value, - nil /* txn */) -} - -// SetLegacyAppliedIndexBlind sets the legacy {raft,lease} applied index values -// using a "blind" put which ignores any existing keys. This is identical to -// SetLegacyAppliedIndex but is used to optimize the writing of the applied -// index values during write operations where we definitively know the size of -// the previous values. -// -// The range applied state key cannot already exist or an assetion will be -// triggered. See comment on SetRangeAppliedState for why this is "legacy". -func (rsl StateLoader) SetLegacyAppliedIndexBlind( - ctx context.Context, - readWriter storage.ReadWriter, - ms *enginepb.MVCCStats, - appliedIndex, leaseAppliedIndex uint64, -) error { - if err := rsl.AssertNoRangeAppliedState(ctx, readWriter); err != nil { - return err - } - - var value roachpb.Value - value.SetInt(int64(appliedIndex)) - if err := storage.MVCCBlindPut(ctx, readWriter, ms, - rsl.RaftAppliedIndexLegacyKey(), - hlc.Timestamp{}, - value, - nil /* txn */); err != nil { - return err - } - value.SetInt(int64(leaseAppliedIndex)) - return storage.MVCCBlindPut(ctx, readWriter, ms, - rsl.LeaseAppliedIndexLegacyKey(), - hlc.Timestamp{}, - value, - nil /* txn */) -} - func inlineValueIntEncodedSize(v int64) int { var value roachpb.Value value.SetInt(v) @@ -412,88 +237,18 @@ func inlineValueIntEncodedSize(v int64) int { return meta.Size() } -// CalcAppliedIndexSysBytes calculates the size (MVCCStats.SysBytes) of the {raft,lease} applied -// index keys/values. -func (rsl StateLoader) CalcAppliedIndexSysBytes(appliedIndex, leaseAppliedIndex uint64) int64 { - return int64(storage.MakeMVCCMetadataKey(rsl.RaftAppliedIndexLegacyKey()).EncodedSize() + - storage.MakeMVCCMetadataKey(rsl.LeaseAppliedIndexLegacyKey()).EncodedSize() + - inlineValueIntEncodedSize(int64(appliedIndex)) + - inlineValueIntEncodedSize(int64(leaseAppliedIndex))) -} - -func (rsl StateLoader) writeLegacyMVCCStatsInternal( - ctx context.Context, readWriter storage.ReadWriter, newMS *enginepb.MVCCStats, -) error { - // We added a new field to the MVCCStats struct to track abort span bytes, which - // enlarges the size of the struct itself. This is mostly fine - we persist - // MVCCStats under the RangeAppliedState key and don't account for the size of - // the MVCCStats struct itself when doing so (we ignore the RangeAppliedState key - // in ComputeStatsForRange). This would not therefore not cause replica state divergence - // in mixed version clusters (the enlarged struct does not contribute to a - // persisted stats difference on disk because we're not accounting for the size of - // the struct itself). - - // That's all fine and good except for the fact that historically we persisted - // MVCCStats under a dedicated RangeStatsLegacyKey (as we're doing so here), and - // in this key we also accounted for the size of the MVCCStats object itself - // (which made it super cumbersome to update the schema of the MVCCStats object, - // and we basically never did). - - // Now, in order to add this extra field to the MVCCStats object, we need to be - // careful with what we write to the RangeStatsLegacyKey. We can't write this new - // version of MVCCStats, as it is going to account for it's now (enlarged) size - // and persist a value for sysbytes different from other replicas that are unaware - // of this new representation (as would be the case in mixed-version settings). To - // this end we've constructed a copy of the legacy MVCCStats representation and - // are careful to persist only that (and sidestepping any inconsistencies due to - // the differing MVCCStats schema). - legacyMS := enginepb.MVCCStatsLegacyRepresentation{ - ContainsEstimates: newMS.ContainsEstimates, - LastUpdateNanos: newMS.LastUpdateNanos, - IntentAge: newMS.IntentAge, - GCBytesAge: newMS.GCBytesAge, - LiveBytes: newMS.LiveBytes, - LiveCount: newMS.LiveCount, - KeyBytes: newMS.KeyBytes, - KeyCount: newMS.KeyCount, - ValBytes: newMS.ValBytes, - ValCount: newMS.ValCount, - IntentBytes: newMS.IntentBytes, - IntentCount: newMS.IntentCount, - SysBytes: newMS.SysBytes, - SysCount: newMS.SysCount, - } - return storage.MVCCPutProto(ctx, readWriter, nil, rsl.RangeStatsLegacyKey(), hlc.Timestamp{}, nil, &legacyMS) -} - -// SetLegacyMVCCStats overwrites the legacy MVCC stats key. -// -// The range applied state key cannot already exist or an assertion will be -// triggered. See comment on SetRangeAppliedState for why this is "legacy". -func (rsl StateLoader) SetLegacyMVCCStats( - ctx context.Context, readWriter storage.ReadWriter, newMS *enginepb.MVCCStats, -) error { - if err := rsl.AssertNoRangeAppliedState(ctx, readWriter); err != nil { - return err - } - - return rsl.writeLegacyMVCCStatsInternal(ctx, readWriter, newMS) -} - // SetMVCCStats overwrites the MVCC stats. This needs to perform a read on the // RangeAppliedState key before overwriting the stats. Use SetRangeAppliedState // when performance is important. func (rsl StateLoader) SetMVCCStats( ctx context.Context, readWriter storage.ReadWriter, newMS *enginepb.MVCCStats, ) error { - if as, err := rsl.LoadRangeAppliedState(ctx, readWriter); err != nil { + as, err := rsl.LoadRangeAppliedState(ctx, readWriter) + if err != nil { return err - } else if as != nil { - return rsl.SetRangeAppliedState( - ctx, readWriter, as.RaftAppliedIndex, as.LeaseAppliedIndex, newMS, as.RaftClosedTimestamp) } - - return rsl.writeLegacyMVCCStatsInternal(ctx, readWriter, newMS) + return rsl.SetRangeAppliedState( + ctx, readWriter, as.RaftAppliedIndex, as.LeaseAppliedIndex, newMS, as.RaftClosedTimestamp) } // SetClosedTimestamp overwrites the closed timestamp. @@ -509,20 +264,6 @@ func (rsl StateLoader) SetClosedTimestamp( as.RangeStats.ToStatsPtr(), closedTS) } -// SetLegacyRaftTruncatedState overwrites the truncated state. -func (rsl StateLoader) SetLegacyRaftTruncatedState( - ctx context.Context, - readWriter storage.ReadWriter, - ms *enginepb.MVCCStats, - truncState *roachpb.RaftTruncatedState, -) error { - if (*truncState == roachpb.RaftTruncatedState{}) { - return errors.New("cannot persist empty RaftTruncatedState") - } - return storage.MVCCPutProto(ctx, readWriter, ms, - rsl.RaftTruncatedStateLegacyKey(), hlc.Timestamp{}, nil, truncState) -} - // LoadGCThreshold loads the GC threshold. func (rsl StateLoader) LoadGCThreshold( ctx context.Context, reader storage.Reader, @@ -591,7 +332,7 @@ func (rsl StateLoader) LoadLastIndex(ctx context.Context, reader storage.Reader) if lastIndex == 0 { // The log is empty, which means we are either starting from scratch // or the entire log has been truncated away. - lastEnt, _, err := rsl.LoadRaftTruncatedState(ctx, reader) + lastEnt, err := rsl.LoadRaftTruncatedState(ctx, reader) if err != nil { return 0, err } @@ -600,35 +341,17 @@ func (rsl StateLoader) LoadLastIndex(ctx context.Context, reader storage.Reader) return lastIndex, nil } -// LoadRaftTruncatedState loads the truncated state. The returned boolean returns -// whether the result was read from the TruncatedStateLegacyKey. If both keys -// are missing, it is false which is used to migrate into the unreplicated key. -// -// See VersionUnreplicatedRaftTruncatedState. +// LoadRaftTruncatedState loads the truncated state. func (rsl StateLoader) LoadRaftTruncatedState( ctx context.Context, reader storage.Reader, -) (_ roachpb.RaftTruncatedState, isLegacy bool, _ error) { +) (roachpb.RaftTruncatedState, error) { var truncState roachpb.RaftTruncatedState - if found, err := storage.MVCCGetProto( + if _, err := storage.MVCCGetProto( ctx, reader, rsl.RaftTruncatedStateKey(), hlc.Timestamp{}, &truncState, storage.MVCCGetOptions{}, ); err != nil { - return roachpb.RaftTruncatedState{}, false, err - } else if found { - return truncState, false, nil - } - - // If the "new" truncated state isn't there (yet), fall back to the legacy - // truncated state. The next log truncation will atomically rewrite them - // assuming the cluster version has advanced sufficiently. - // - // See VersionUnreplicatedRaftTruncatedState. - legacyFound, err := storage.MVCCGetProto( - ctx, reader, rsl.RaftTruncatedStateLegacyKey(), hlc.Timestamp{}, &truncState, storage.MVCCGetOptions{}, - ) - if err != nil { - return roachpb.RaftTruncatedState{}, false, err + return roachpb.RaftTruncatedState{}, err } - return truncState, legacyFound, nil + return truncState, nil } // SetRaftTruncatedState overwrites the truncated state. @@ -691,7 +414,7 @@ func (rsl StateLoader) SynthesizeRaftState( if err != nil { return err } - truncState, _, err := rsl.LoadRaftTruncatedState(ctx, readWriter) + truncState, err := rsl.LoadRaftTruncatedState(ctx, readWriter) if err != nil { return err } diff --git a/pkg/kv/kvserver/store_init.go b/pkg/kv/kvserver/store_init.go index f1617daa0cd5..33aac9a89072 100644 --- a/pkg/kv/kvserver/store_init.go +++ b/pkg/kv/kvserver/store_init.go @@ -244,16 +244,8 @@ func WriteInitialClusterData( } } - if tt := knobs.TruncatedStateTypeOverride; tt != nil { - if err := stateloader.WriteInitialRangeStateWithTruncatedState( - ctx, batch, *desc, initialReplicaVersion, *tt, - ); err != nil { - return err - } - } else { - if err := stateloader.WriteInitialRangeState(ctx, batch, *desc, initialReplicaVersion); err != nil { - return err - } + if err := stateloader.WriteInitialRangeState(ctx, batch, *desc, initialReplicaVersion); err != nil { + return err } computedStats, err := rditer.ComputeStatsForRange(desc, batch, now.WallTime) if err != nil { diff --git a/pkg/kv/kvserver/store_snapshot.go b/pkg/kv/kvserver/store_snapshot.go index 649f2171e8d2..fc070fadf9ea 100644 --- a/pkg/kv/kvserver/store_snapshot.go +++ b/pkg/kv/kvserver/store_snapshot.go @@ -296,12 +296,11 @@ func (kvSS *kvBatchSnapshotStrategy) Receive( } inSnap := IncomingSnapshot{ - UsesUnreplicatedTruncatedState: header.UnreplicatedTruncatedState, - SnapUUID: snapUUID, - SSTStorageScratch: kvSS.scratch, - LogEntries: logEntries, - State: &header.State, - snapType: header.Type, + SnapUUID: snapUUID, + SSTStorageScratch: kvSS.scratch, + LogEntries: logEntries, + State: &header.State, + snapType: header.Type, } expLen := inSnap.State.RaftAppliedIndex - inSnap.State.TruncatedState.Index @@ -930,7 +929,6 @@ func SendEmptySnapshot( desc, roachpb.Lease{}, hlc.Timestamp{}, // gcThreshold - stateloader.TruncatedStateUnreplicated, st.Version.ActiveVersionOrEmpty(ctx).Version, ) if err != nil { @@ -994,14 +992,14 @@ func SendEmptySnapshot( } header := SnapshotRequest_Header{ - State: state, - RaftMessageRequest: req, - RangeSize: ms.Total(), - CanDecline: false, - Priority: SnapshotRequest_RECOVERY, - Strategy: SnapshotRequest_KV_BATCH, - Type: SnapshotRequest_VIA_SNAPSHOT_QUEUE, - UnreplicatedTruncatedState: true, + State: state, + RaftMessageRequest: req, + RangeSize: ms.Total(), + CanDecline: false, + Priority: SnapshotRequest_RECOVERY, + Strategy: SnapshotRequest_KV_BATCH, + Type: SnapshotRequest_VIA_SNAPSHOT_QUEUE, + LegacyUnreplicatedTruncatedState: true, } stream, err := NewMultiRaftClient(cc).RaftSnapshot(ctx) diff --git a/pkg/kv/kvserver/store_test.go b/pkg/kv/kvserver/store_test.go index 12ca1da9e5ed..b4e27153dc7f 100644 --- a/pkg/kv/kvserver/store_test.go +++ b/pkg/kv/kvserver/store_test.go @@ -313,8 +313,7 @@ func TestIterateIDPrefixKeys(t *testing.T) { rng := rand.New(rand.NewSource(seed)) ops := []func(rangeID roachpb.RangeID) roachpb.Key{ - keys.RaftAppliedIndexLegacyKey, // replicated; sorts before tombstone - keys.RaftHardStateKey, // unreplicated; sorts after tombstone + keys.RaftHardStateKey, // unreplicated; sorts after tombstone // Replicated key-anchored local key (i.e. not one we should care about). // Will be written at zero timestamp, but that's ok. func(rangeID roachpb.RangeID) roachpb.Key { diff --git a/pkg/kv/kvserver/testdata/truncated_state_migration/migration b/pkg/kv/kvserver/testdata/truncated_state/truncated_state similarity index 100% rename from pkg/kv/kvserver/testdata/truncated_state_migration/migration rename to pkg/kv/kvserver/testdata/truncated_state/truncated_state diff --git a/pkg/kv/kvserver/testdata/truncated_state_migration/pre_migration b/pkg/kv/kvserver/testdata/truncated_state_migration/pre_migration deleted file mode 100644 index e84177bd0b51..000000000000 --- a/pkg/kv/kvserver/testdata/truncated_state_migration/pre_migration +++ /dev/null @@ -1,26 +0,0 @@ -# Mode of operation below VersionUnreplicatedRaftTruncatedState. -# We don't mess with the on-disk state nor do we ever drop updates. - -prev index=100 term=9 ----- - -put legacy=true index=100 term=9 ----- - -handle index=100 term=9 ----- -apply: true -/Local/RangeID/12/r/RaftTruncatedState -> index=100 term=9 - -# Note that the below aren't actually possible in practice -# as a divergence won't happen before the migration. - -handle index=150 term=9 ----- -apply: true -/Local/RangeID/12/r/RaftTruncatedState -> index=100 term=9 - -handle index=60 term=9 ----- -apply: true -/Local/RangeID/12/r/RaftTruncatedState -> index=100 term=9 diff --git a/pkg/kv/kvserver/testing_knobs.go b/pkg/kv/kvserver/testing_knobs.go index 73443190424e..9729aed0adde 100644 --- a/pkg/kv/kvserver/testing_knobs.go +++ b/pkg/kv/kvserver/testing_knobs.go @@ -15,7 +15,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/tenantrate" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/txnwait" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -327,9 +326,6 @@ type StoreTestingKnobs struct { // PurgeOutdatedReplicasInterceptor intercepts attempts to purge outdated // replicas in the store. PurgeOutdatedReplicasInterceptor func() - // If set, use the given truncated state type when bootstrapping ranges. - // This is used for testing the truncated state migration. - TruncatedStateTypeOverride *stateloader.TruncatedStateType // If set, use the given version as the initial replica version when // bootstrapping ranges. This is used for testing the migration // infrastructure. diff --git a/pkg/migration/migrations/migrations.go b/pkg/migration/migrations/migrations.go index 105b4cf25e8e..f8f93811763f 100644 --- a/pkg/migration/migrations/migrations.go +++ b/pkg/migration/migrations/migrations.go @@ -39,16 +39,6 @@ func NoPrecondition(context.Context, clusterversion.ClusterVersion, migration.Te var registry = make(map[clusterversion.ClusterVersion]migration.Migration) var migrations = []migration.Migration{ - migration.NewSystemMigration( - "use unreplicated TruncatedState and RangeAppliedState for all ranges", - toCV(clusterversion.TruncatedAndRangeAppliedStateMigration), - truncatedStateMigration, - ), - migration.NewSystemMigration( - "purge all replicas using the replicated TruncatedState", - toCV(clusterversion.PostTruncatedAndRangeAppliedStateMigration), - postTruncatedStateMigration, - ), migration.NewSystemMigration( "stop using monolithic encryption-at-rest registry for all stores", toCV(clusterversion.RecordsBasedRegistry), diff --git a/pkg/migration/migrations/separated_intents.go b/pkg/migration/migrations/separated_intents.go index 017cc78a487e..224ae5785f77 100644 --- a/pkg/migration/migrations/separated_intents.go +++ b/pkg/migration/migrations/separated_intents.go @@ -52,6 +52,19 @@ const concurrentMigrateLockTableRequests = 4 // the migration. const migrateLockTableRetries = 3 +// defaultPageSize controls how many ranges are paged in by default when +// iterating through all ranges in a cluster during any given migration. We +// pulled this number out of thin air(-ish). Let's consider a cluster with 50k +// ranges, with each range taking ~200ms. We're being somewhat conservative with +// the duration, but in a wide-area cluster with large hops between the manager +// and the replicas, it could be true. Here's how long it'll take for various +// block sizes: +// +// page size of 1 ~ 2h 46m +// page size of 50 ~ 3m 20s +// page size of 200 ~ 50s +const defaultPageSize = 200 + // migrateLockTableRequest represents migration of one slice of the keyspace. As // part of this request, multiple non-transactional requests would need to be // run: a Barrier, a ScanInterleavedIntents, then multiple txn pushes and intent diff --git a/pkg/migration/migrations/truncated_state.go b/pkg/migration/migrations/truncated_state.go deleted file mode 100644 index 6085f049e08d..000000000000 --- a/pkg/migration/migrations/truncated_state.go +++ /dev/null @@ -1,95 +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 migrations - -import ( - "bytes" - "context" - "fmt" - - "github.com/cockroachdb/cockroach/pkg/clusterversion" - "github.com/cockroachdb/cockroach/pkg/jobs" - "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/migration" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/server/serverpb" - "github.com/cockroachdb/cockroach/pkg/util/log" -) - -// defaultPageSize controls how many ranges are paged in by default when -// iterating through all ranges in a cluster during any given migration. We -// pulled this number out of thin air(-ish). Let's consider a cluster with 50k -// ranges, with each range taking ~200ms. We're being somewhat conservative with -// the duration, but in a wide-area cluster with large hops between the manager -// and the replicas, it could be true. Here's how long it'll take for various -// block sizes: -// -// page size of 1 ~ 2h 46m -// page size of 50 ~ 3m 20s -// page size of 200 ~ 50s -const defaultPageSize = 200 - -func truncatedStateMigration( - ctx context.Context, cv clusterversion.ClusterVersion, deps migration.SystemDeps, _ *jobs.Job, -) error { - var batchIdx, numMigratedRanges int - init := func() { batchIdx, numMigratedRanges = 1, 0 } - if err := deps.Cluster.IterateRangeDescriptors(ctx, defaultPageSize, init, func(descriptors ...roachpb.RangeDescriptor) error { - for _, desc := range descriptors { - // NB: This is a bit of a wart. We want to reach the first range, - // but we can't address the (local) StartKey. However, keys.LocalMax - // is on r1, so we'll just use that instead to target r1. - start, end := desc.StartKey, desc.EndKey - if bytes.Compare(desc.StartKey, keys.LocalMax) < 0 { - start, _ = keys.Addr(keys.LocalMax) - } - if err := deps.DB.Migrate(ctx, start, end, cv.Version); err != nil { - return err - } - } - - // TODO(irfansharif): Instead of logging this to the debug log, we - // should insert these into a `system.migrations` table for external - // observability. - numMigratedRanges += len(descriptors) - log.Infof(ctx, "[batch %d/??] migrated %d ranges", batchIdx, numMigratedRanges) - batchIdx++ - - return nil - }); err != nil { - return err - } - - log.Infof(ctx, "[batch %d/%d] migrated %d ranges", batchIdx, batchIdx, numMigratedRanges) - - // Make sure that all stores have synced. Given we're a below-raft - // migrations, this ensures that the applied state is flushed to disk. - req := &serverpb.SyncAllEnginesRequest{} - op := "flush-stores" - return deps.Cluster.ForEveryNode(ctx, op, func(ctx context.Context, client serverpb.MigrationClient) error { - _, err := client.SyncAllEngines(ctx, req) - return err - }) -} - -func postTruncatedStateMigration( - ctx context.Context, cv clusterversion.ClusterVersion, deps migration.SystemDeps, _ *jobs.Job, -) error { - // Purge all replicas that haven't been migrated to use the unreplicated - // truncated state and the range applied state. - truncStateVersion := clusterversion.ByKey(clusterversion.TruncatedAndRangeAppliedStateMigration) - req := &serverpb.PurgeOutdatedReplicasRequest{Version: &truncStateVersion} - op := fmt.Sprintf("purge-outdated-replicas=%s", req.Version) - return deps.Cluster.ForEveryNode(ctx, op, func(ctx context.Context, client serverpb.MigrationClient) error { - _, err := client.PurgeOutdatedReplicas(ctx, req) - return err - }) -} diff --git a/pkg/migration/migrations/truncated_state_external_test.go b/pkg/migration/migrations/truncated_state_external_test.go deleted file mode 100644 index 7cdc26eabf0b..000000000000 --- a/pkg/migration/migrations/truncated_state_external_test.go +++ /dev/null @@ -1,152 +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 migrations_test - -import ( - "context" - "fmt" - "testing" - - "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/clusterversion" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader" - "github.com/cockroachdb/cockroach/pkg/server" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/cockroachdb/errors" - "github.com/stretchr/testify/require" -) - -func TestTruncatedStateMigration(t *testing.T) { - defer leaktest.AfterTest(t)() - ctx := context.Background() - - minVersion := clusterversion.ByKey(clusterversion.V20_2) - binaryVersion := clusterversion.ByKey(clusterversion.TruncatedAndRangeAppliedStateMigration + 1) - truncStateVersion := clusterversion.ByKey(clusterversion.TruncatedAndRangeAppliedStateMigration) - bootstrapVersion := clusterversion.ByKey(clusterversion.TruncatedAndRangeAppliedStateMigration - 1) - for _, testCase := range []struct { - name string - typ stateloader.TruncatedStateType - }{ - {"ts=new,as=new", stateloader.TruncatedStateUnreplicated}, - {"ts=legacy,as=new", stateloader.TruncatedStateLegacyReplicated}, - {"ts=legacy,as=legacy", stateloader.TruncatedStateLegacyReplicatedAndNoAppliedKey}, - } { - t.Run(testCase.name, func(t *testing.T) { - makeArgs := func() (args base.TestServerArgs) { - args.Settings = cluster.MakeTestingClusterSettingsWithVersions( - binaryVersion, minVersion, false, - ) - args.Knobs.Store = &kvserver.StoreTestingKnobs{TruncatedStateTypeOverride: &testCase.typ} - args.Knobs.Server = &server.TestingKnobs{ - // TruncatedAndRangeAppliedStateMigration is part of the - // migration that lets us stop using the legacy truncated state. - // When the active cluster version is greater than it, we assert - // against the presence of legacy truncated state and ensure - // we're using the range applied state key. In this test we'll - // start of at the version immediately preceding the migration, - // and migrate past it. - BinaryVersionOverride: bootstrapVersion, - // We want to exercise manual control over the upgrade process. - DisableAutomaticVersionUpgrade: 1, - } - return args - } - - tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{ - ServerArgsPerNode: map[int]base.TestServerArgs{ - 0: makeArgs(), - 1: makeArgs(), - 2: makeArgs(), - }, - }) - defer tc.Stopper().Stop(ctx) - - forAllReplicas := func(f func(*kvserver.Replica) error) error { - for i := 0; i < tc.NumServers(); i++ { - err := tc.Server(i).GetStores().(*kvserver.Stores).VisitStores(func(s *kvserver.Store) error { - var err error - s.VisitReplicas(func(repl *kvserver.Replica) bool { - err = f(repl) - return err == nil - }) - return err - }) - if err != nil { - return err - } - } - return nil - } - - getLegacyRanges := func() []string { - t.Helper() - var out []string - require.NoError(t, forAllReplicas(func(repl *kvserver.Replica) error { - sl := stateloader.Make(repl.RangeID) - - _, legacy, err := sl.LoadRaftTruncatedState(ctx, repl.Engine()) - if err != nil { - return err - } - if legacy { - // We're using the legacy truncated state, record ourselves. - out = append(out, fmt.Sprintf("ts(r%d)", repl.RangeID)) - } - - as, err := sl.LoadRangeAppliedState(ctx, repl.Engine()) - if err != nil { - return err - } - if as == nil { - // We're not using the new applied state key, record ourselves. - out = append(out, fmt.Sprintf("as(r%d)", repl.RangeID)) - } - return nil - })) - return out - } - - legacyRanges := getLegacyRanges() - switch testCase.typ { - case stateloader.TruncatedStateUnreplicated: - if len(legacyRanges) != 0 { - t.Fatalf("expected no ranges with legacy keys if bootstrapped with unreplicated truncated state, got: %v", legacyRanges) - } - case stateloader.TruncatedStateLegacyReplicated, stateloader.TruncatedStateLegacyReplicatedAndNoAppliedKey: - if len(legacyRanges) == 0 { - t.Fatalf("expected ranges with legacy keys if bootstrapped with replicated truncated state, got none") - } - } - - // NB: we'll never spot a legacy applied state here. This is - // because that migration is so aggressive that it has already - // happened as part of the initial up-replication. - t.Logf("ranges with legacy keys before migration: %v", legacyRanges) - - _, err := tc.Conns[0].ExecContext( - ctx, `SET CLUSTER SETTING version = $1`, binaryVersion.String(), - ) - require.NoError(t, err) - require.Zero(t, getLegacyRanges()) - - require.NoError(t, forAllReplicas(func(repl *kvserver.Replica) error { - if repl.Version().Less(truncStateVersion) { - return errors.Newf("unexpected version %s", repl.Version()) - } - return nil - })) - }) - } -} diff --git a/pkg/storage/enginepb/mvcc.pb.go b/pkg/storage/enginepb/mvcc.pb.go index 3508802d554f..da306fc2ea25 100644 --- a/pkg/storage/enginepb/mvcc.pb.go +++ b/pkg/storage/enginepb/mvcc.pb.go @@ -324,123 +324,66 @@ func (m *MVCCStats) XXX_DiscardUnknown() { var xxx_messageInfo_MVCCStats proto.InternalMessageInfo -// MVCCStatsLegacyRepresentation is almost identical to MVCCStats, except -// it does not have a field tracking abort span bytes. -// The abort span bytes field in MVCCStats didn't exist in earlier versions, -// and its addition of it to MVCCStats causes the struct to be larger than it was -// previously. The discrepancy between the size of MVCCStats between versions -// could cause incorrect sysbyte counts during version migration. -// MVCCStatsLegacyRepresentation is the older version of MVCCStats, and allows -// us to prevent this discrepancy during version migration. -type MVCCStatsLegacyRepresentation struct { - ContainsEstimates int64 `protobuf:"varint,14,opt,name=contains_estimates,json=containsEstimates" json:"contains_estimates"` - LastUpdateNanos int64 `protobuf:"fixed64,1,opt,name=last_update_nanos,json=lastUpdateNanos" json:"last_update_nanos"` - IntentAge int64 `protobuf:"fixed64,2,opt,name=intent_age,json=intentAge" json:"intent_age"` - GCBytesAge int64 `protobuf:"fixed64,3,opt,name=gc_bytes_age,json=gcBytesAge" json:"gc_bytes_age"` - LiveBytes int64 `protobuf:"fixed64,4,opt,name=live_bytes,json=liveBytes" json:"live_bytes"` - LiveCount int64 `protobuf:"fixed64,5,opt,name=live_count,json=liveCount" json:"live_count"` - KeyBytes int64 `protobuf:"fixed64,6,opt,name=key_bytes,json=keyBytes" json:"key_bytes"` - KeyCount int64 `protobuf:"fixed64,7,opt,name=key_count,json=keyCount" json:"key_count"` - ValBytes int64 `protobuf:"fixed64,8,opt,name=val_bytes,json=valBytes" json:"val_bytes"` - ValCount int64 `protobuf:"fixed64,9,opt,name=val_count,json=valCount" json:"val_count"` - IntentBytes int64 `protobuf:"fixed64,10,opt,name=intent_bytes,json=intentBytes" json:"intent_bytes"` - IntentCount int64 `protobuf:"fixed64,11,opt,name=intent_count,json=intentCount" json:"intent_count"` - SysBytes int64 `protobuf:"fixed64,12,opt,name=sys_bytes,json=sysBytes" json:"sys_bytes"` - SysCount int64 `protobuf:"fixed64,13,opt,name=sys_count,json=sysCount" json:"sys_count"` -} - -func (m *MVCCStatsLegacyRepresentation) Reset() { *m = MVCCStatsLegacyRepresentation{} } -func (m *MVCCStatsLegacyRepresentation) String() string { return proto.CompactTextString(m) } -func (*MVCCStatsLegacyRepresentation) ProtoMessage() {} -func (*MVCCStatsLegacyRepresentation) Descriptor() ([]byte, []int) { - return fileDescriptor_324ef2186f146e22, []int{3} -} -func (m *MVCCStatsLegacyRepresentation) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *MVCCStatsLegacyRepresentation) 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 *MVCCStatsLegacyRepresentation) XXX_Merge(src proto.Message) { - xxx_messageInfo_MVCCStatsLegacyRepresentation.Merge(m, src) -} -func (m *MVCCStatsLegacyRepresentation) XXX_Size() int { - return m.Size() -} -func (m *MVCCStatsLegacyRepresentation) XXX_DiscardUnknown() { - xxx_messageInfo_MVCCStatsLegacyRepresentation.DiscardUnknown(m) -} - -var xxx_messageInfo_MVCCStatsLegacyRepresentation proto.InternalMessageInfo - func init() { proto.RegisterType((*MVCCMetadata)(nil), "cockroach.storage.enginepb.MVCCMetadata") proto.RegisterType((*MVCCMetadata_SequencedIntent)(nil), "cockroach.storage.enginepb.MVCCMetadata.SequencedIntent") proto.RegisterType((*MVCCMetadataSubsetForMergeSerialization)(nil), "cockroach.storage.enginepb.MVCCMetadataSubsetForMergeSerialization") proto.RegisterType((*MVCCStats)(nil), "cockroach.storage.enginepb.MVCCStats") - proto.RegisterType((*MVCCStatsLegacyRepresentation)(nil), "cockroach.storage.enginepb.MVCCStatsLegacyRepresentation") } func init() { proto.RegisterFile("storage/enginepb/mvcc.proto", fileDescriptor_324ef2186f146e22) } var fileDescriptor_324ef2186f146e22 = []byte{ - // 803 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x95, 0xb1, 0x6f, 0xfb, 0x44, - 0x14, 0xc7, 0x63, 0x92, 0xb4, 0xf6, 0x25, 0xbf, 0xa4, 0x35, 0x15, 0x44, 0x29, 0x38, 0x69, 0x3b, - 0x34, 0xea, 0xe0, 0xa0, 0xb6, 0x48, 0x28, 0x5b, 0x13, 0xa0, 0x54, 0x6a, 0x3b, 0x38, 0x85, 0x81, - 0xc5, 0xba, 0xd8, 0x4f, 0x8e, 0x55, 0xe7, 0xec, 0xfa, 0x2e, 0x69, 0xc2, 0x5f, 0xc1, 0xc8, 0x58, - 0xf1, 0x1f, 0xb0, 0xf1, 0x27, 0x74, 0xec, 0x58, 0x96, 0x0a, 0xd2, 0x05, 0xfe, 0x05, 0x26, 0x74, - 0x3e, 0xdb, 0x71, 0x52, 0xa8, 0x18, 0xba, 0xc1, 0xe6, 0x7b, 0xef, 0xf3, 0xbe, 0x79, 0xf7, 0xbd, - 0xf7, 0x14, 0xb4, 0x4d, 0x99, 0x1f, 0x62, 0x07, 0xda, 0x40, 0x1c, 0x97, 0x40, 0x30, 0x68, 0x8f, - 0x26, 0x96, 0xa5, 0x07, 0xa1, 0xcf, 0x7c, 0xb5, 0x6e, 0xf9, 0xd6, 0x75, 0xe8, 0x63, 0x6b, 0xa8, - 0xc7, 0x98, 0x9e, 0x60, 0xf5, 0x8f, 0xfe, 0xb6, 0xf0, 0x48, 0x54, 0xd6, 0x1b, 0x63, 0xe6, 0x7a, - 0xed, 0xa1, 0x67, 0xb5, 0x3d, 0x70, 0xb0, 0x35, 0x33, 0x99, 0x3b, 0x02, 0xca, 0xf0, 0x28, 0x88, - 0x81, 0x2d, 0xc7, 0x77, 0xfc, 0xe8, 0xb3, 0xcd, 0xbf, 0x44, 0x74, 0xf7, 0x8f, 0x02, 0x2a, 0x5f, - 0x7c, 0xd3, 0xeb, 0x5d, 0x00, 0xc3, 0x36, 0x66, 0x58, 0xfd, 0x14, 0xe5, 0xd9, 0x94, 0xd4, 0xa4, - 0xa6, 0xd4, 0x2a, 0x1d, 0xee, 0xe9, 0xff, 0xdc, 0x8f, 0x7e, 0x35, 0x25, 0xbc, 0xca, 0xe0, 0xbc, - 0x7a, 0x8a, 0x94, 0xf4, 0x07, 0x6b, 0xef, 0xbd, 0x28, 0xe6, 0xcd, 0xe9, 0x43, 0xcf, 0xd2, 0xcf, - 0xa3, 0xe6, 0xae, 0x12, 0xb4, 0x5b, 0xb8, 0x7f, 0x6a, 0xe4, 0x8c, 0x45, 0xad, 0xaa, 0xa1, 0x75, - 0x1b, 0x3c, 0x60, 0x60, 0xd7, 0xf2, 0x4d, 0xa9, 0x25, 0xc7, 0x44, 0x12, 0x54, 0x77, 0x90, 0x72, - 0x0d, 0x33, 0x73, 0x30, 0x63, 0x40, 0x6b, 0x85, 0xa6, 0xd4, 0xca, 0xc7, 0x84, 0x7c, 0x0d, 0xb3, - 0x2e, 0x8f, 0x72, 0x64, 0x82, 0xbd, 0x18, 0x29, 0x66, 0x91, 0x09, 0xf6, 0x04, 0xb2, 0x8d, 0x94, - 0x10, 0xdf, 0xc6, 0xc8, 0x5a, 0x53, 0x6a, 0x95, 0x0d, 0x39, 0xc4, 0xb7, 0x22, 0x09, 0xa8, 0xe2, - 0x12, 0x06, 0x84, 0x99, 0x43, 0x97, 0xdf, 0x7a, 0x56, 0x93, 0x9b, 0xf9, 0x56, 0xe9, 0xf0, 0xb3, - 0xd7, 0xdc, 0xc8, 0x9a, 0xa8, 0xf7, 0xe1, 0x66, 0x0c, 0xc4, 0x02, 0xfb, 0x2c, 0xd2, 0x89, 0x7f, - 0xfe, 0x9d, 0x50, 0xfd, 0x4a, 0x88, 0xaa, 0xe7, 0xa8, 0x3a, 0x82, 0xd0, 0x81, 0xc5, 0x4b, 0xd5, - 0xd6, 0xff, 0xb5, 0x71, 0x46, 0x25, 0xaa, 0x4d, 0xcf, 0xea, 0x31, 0xfa, 0x90, 0x4d, 0x89, 0x69, - 0xbb, 0xb6, 0x49, 0x7c, 0x66, 0x8e, 0x03, 0x1b, 0x33, 0x30, 0x47, 0xc0, 0x70, 0x4d, 0xe1, 0x3e, - 0x1a, 0xef, 0xb3, 0x29, 0xf9, 0xdc, 0xb5, 0x2f, 0x7d, 0xf6, 0x75, 0x94, 0xe3, 0xcd, 0xd6, 0x31, - 0xaa, 0xae, 0xf4, 0xaa, 0x1e, 0x20, 0x99, 0xc6, 0xa1, 0x68, 0x0a, 0x8a, 0xdd, 0x0a, 0xef, 0xfe, - 0xcf, 0xa7, 0xc6, 0xda, 0xd5, 0x94, 0xf4, 0xe1, 0xc6, 0x48, 0xf3, 0xea, 0x16, 0x2a, 0x4e, 0xb0, - 0x37, 0x86, 0xe8, 0xc5, 0xcb, 0x86, 0x38, 0x74, 0xca, 0x3f, 0xdc, 0x35, 0x72, 0x3f, 0xdf, 0x35, - 0xa4, 0xdf, 0xef, 0x1a, 0x52, 0x47, 0x4e, 0x4e, 0xbb, 0x3f, 0x4a, 0x68, 0x3f, 0x6b, 0x53, 0x7f, - 0x3c, 0xa0, 0xc0, 0xbe, 0xf4, 0xc3, 0x0b, 0x7e, 0x95, 0x3e, 0x84, 0x2e, 0xf6, 0xdc, 0xef, 0x30, - 0x73, 0x7d, 0xf2, 0xfa, 0x03, 0xbd, 0xa9, 0x73, 0xcb, 0xed, 0xee, 0xfe, 0x54, 0x44, 0x0a, 0x6f, - 0xb2, 0xcf, 0x30, 0xa3, 0xea, 0x11, 0x52, 0x2d, 0x9f, 0x30, 0xec, 0x12, 0x6a, 0x02, 0x65, 0xee, - 0x08, 0xf3, 0x7e, 0x2a, 0x99, 0x99, 0xda, 0x4c, 0xf2, 0x5f, 0x24, 0x69, 0xf5, 0x13, 0xb4, 0xe9, - 0x61, 0x9a, 0xbe, 0x01, 0xc1, 0xc4, 0xa7, 0x91, 0x95, 0x1b, 0x71, 0x4d, 0x95, 0xa7, 0xc5, 0x2b, - 0x5c, 0xf2, 0xa4, 0xba, 0x87, 0x50, 0x3c, 0x71, 0xd8, 0x11, 0x66, 0x26, 0xa8, 0x22, 0xe2, 0x27, - 0x0e, 0xa8, 0xc7, 0xa8, 0xec, 0x58, 0xc2, 0x91, 0x08, 0xcb, 0x47, 0x98, 0xca, 0xb1, 0xf9, 0x53, - 0x03, 0x9d, 0xf6, 0x22, 0x73, 0x4e, 0x1c, 0x30, 0x90, 0x63, 0x25, 0xdf, 0x5c, 0xda, 0x73, 0x27, - 0x90, 0x59, 0x98, 0x54, 0x9a, 0xc7, 0x85, 0xa1, 0x09, 0x64, 0xf9, 0x63, 0xc2, 0xa2, 0x95, 0x59, - 0x82, 0x7a, 0x3c, 0xbc, 0xbc, 0x79, 0x6b, 0x19, 0x66, 0x69, 0xf3, 0x38, 0x22, 0x64, 0xd6, 0x57, - 0x90, 0x54, 0x65, 0xb1, 0x9c, 0x72, 0x16, 0x49, 0x97, 0x33, 0x46, 0x84, 0x8a, 0xb2, 0x82, 0x08, - 0x95, 0x7d, 0x54, 0x8e, 0x0d, 0x13, 0x42, 0x28, 0x43, 0x95, 0x44, 0x46, 0x68, 0x2d, 0x40, 0x21, - 0x57, 0x7a, 0x09, 0x0a, 0xc5, 0x0e, 0xfa, 0x80, 0x42, 0x80, 0x43, 0xcc, 0xc0, 0x36, 0x97, 0x4a, - 0x36, 0x32, 0x25, 0x5b, 0x29, 0x73, 0x96, 0xa9, 0xdd, 0x41, 0x0a, 0x9d, 0xd1, 0xb8, 0x95, 0x72, - 0xb6, 0x61, 0x3a, 0xa3, 0xe9, 0x9d, 0x38, 0x22, 0x14, 0xdf, 0xad, 0x20, 0x42, 0x45, 0x47, 0x1b, - 0x78, 0xe0, 0x87, 0xcc, 0xa4, 0x01, 0x26, 0xb1, 0x58, 0x35, 0x43, 0x56, 0xa2, 0x6c, 0x3f, 0xc0, - 0x24, 0x92, 0xec, 0xc8, 0xe9, 0xcc, 0xfe, 0x52, 0x40, 0x1f, 0xa7, 0x33, 0x2b, 0xc6, 0xdd, 0x80, - 0x20, 0x04, 0x0a, 0x84, 0x89, 0x75, 0xfa, 0x7f, 0x8e, 0xff, 0x2b, 0x73, 0xfc, 0x26, 0xb3, 0xb8, - 0x98, 0xad, 0xee, 0xc1, 0xfd, 0x6f, 0x5a, 0xee, 0x7e, 0xae, 0x49, 0x0f, 0x73, 0x4d, 0x7a, 0x9c, - 0x6b, 0xd2, 0xaf, 0x73, 0x4d, 0xfa, 0xfe, 0x59, 0xcb, 0x3d, 0x3c, 0x6b, 0xb9, 0xc7, 0x67, 0x2d, - 0xf7, 0xad, 0x9c, 0xfc, 0x07, 0xfe, 0x15, 0x00, 0x00, 0xff, 0xff, 0xaf, 0xd1, 0x7e, 0xfe, 0xdb, - 0x08, 0x00, 0x00, + // 775 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x94, 0x41, 0x4f, 0xdb, 0x48, + 0x14, 0xc7, 0xe3, 0x0d, 0x01, 0x7b, 0x12, 0x12, 0xf0, 0xa2, 0xdd, 0x28, 0xac, 0x9c, 0x00, 0x07, + 0x22, 0x0e, 0xce, 0x0a, 0x58, 0x69, 0x95, 0x1b, 0xc9, 0xee, 0xb2, 0x48, 0xc0, 0xc1, 0x61, 0xf7, + 0xd0, 0x8b, 0x35, 0xb1, 0x9f, 0x1c, 0x0b, 0x67, 0x1c, 0x3c, 0x93, 0x90, 0xf4, 0x53, 0xf4, 0xd8, + 0x23, 0xea, 0x37, 0xe8, 0xad, 0x1f, 0x81, 0x23, 0x47, 0x4e, 0xa8, 0x0d, 0x97, 0xf6, 0x2b, 0xf4, + 0x54, 0x8d, 0xc7, 0x76, 0x9c, 0xd0, 0xa2, 0x1e, 0x7a, 0x9b, 0x79, 0xef, 0xf7, 0xfe, 0x7e, 0xf3, + 0x9f, 0x79, 0x46, 0x9b, 0x94, 0xf9, 0x01, 0x76, 0xa0, 0x01, 0xc4, 0x71, 0x09, 0x0c, 0xba, 0x8d, + 0xfe, 0xc8, 0xb2, 0xf4, 0x41, 0xe0, 0x33, 0x5f, 0xad, 0x58, 0xbe, 0x75, 0x19, 0xf8, 0xd8, 0xea, + 0xe9, 0x11, 0xa6, 0xc7, 0x58, 0xe5, 0xb7, 0xaf, 0x16, 0x1e, 0x88, 0xca, 0x4a, 0x75, 0xc8, 0x5c, + 0xaf, 0xd1, 0xf3, 0xac, 0x86, 0x07, 0x0e, 0xb6, 0x26, 0x26, 0x73, 0xfb, 0x40, 0x19, 0xee, 0x0f, + 0x22, 0x60, 0xc3, 0xf1, 0x1d, 0x3f, 0x5c, 0x36, 0xf8, 0x4a, 0x44, 0xb7, 0x3f, 0x2d, 0xa1, 0xc2, + 0xd9, 0xff, 0xed, 0xf6, 0x19, 0x30, 0x6c, 0x63, 0x86, 0xd5, 0x3f, 0x50, 0x96, 0x8d, 0x49, 0x59, + 0xaa, 0x49, 0xf5, 0xfc, 0xfe, 0x8e, 0xfe, 0xed, 0x7e, 0xf4, 0x8b, 0x31, 0xe1, 0x55, 0x06, 0xe7, + 0xd5, 0x63, 0xa4, 0x24, 0x1f, 0x2c, 0xff, 0xf4, 0xa4, 0x98, 0x37, 0xa7, 0xf7, 0x3c, 0x4b, 0x3f, + 0x0d, 0x9b, 0xbb, 0x88, 0xd1, 0xd6, 0xd2, 0xed, 0x43, 0x35, 0x63, 0xcc, 0x6a, 0x55, 0x0d, 0xad, + 0xd8, 0xe0, 0x01, 0x03, 0xbb, 0x9c, 0xad, 0x49, 0x75, 0x39, 0x22, 0xe2, 0xa0, 0xba, 0x85, 0x94, + 0x4b, 0x98, 0x98, 0xdd, 0x09, 0x03, 0x5a, 0x5e, 0xaa, 0x49, 0xf5, 0x6c, 0x44, 0xc8, 0x97, 0x30, + 0x69, 0xf1, 0x28, 0x47, 0x46, 0xd8, 0x8b, 0x90, 0x5c, 0x1a, 0x19, 0x61, 0x4f, 0x20, 0x9b, 0x48, + 0x09, 0xf0, 0x75, 0x84, 0x2c, 0xd7, 0xa4, 0x7a, 0xc1, 0x90, 0x03, 0x7c, 0x2d, 0x92, 0x80, 0x8a, + 0x2e, 0x61, 0x40, 0x98, 0xd9, 0x73, 0xf9, 0xa9, 0x27, 0x65, 0xb9, 0x96, 0xad, 0xe7, 0xf7, 0xff, + 0x7c, 0xce, 0x8d, 0xb4, 0x89, 0x7a, 0x07, 0xae, 0x86, 0x40, 0x2c, 0xb0, 0x4f, 0x42, 0x9d, 0xe8, + 0xf3, 0xab, 0x42, 0xf5, 0x5f, 0x21, 0xaa, 0x9e, 0xa2, 0x52, 0x1f, 0x02, 0x07, 0x66, 0x37, 0x55, + 0x5e, 0xf9, 0x6e, 0xe3, 0x8c, 0x62, 0x58, 0x9b, 0xec, 0xd5, 0x43, 0xf4, 0x2b, 0x1b, 0x13, 0xd3, + 0x76, 0x6d, 0x93, 0xf8, 0xcc, 0x1c, 0x0e, 0x6c, 0xcc, 0xc0, 0xec, 0x03, 0xc3, 0x65, 0x85, 0xfb, + 0x68, 0xfc, 0xcc, 0xc6, 0xe4, 0x2f, 0xd7, 0x3e, 0xf7, 0xd9, 0x7f, 0x61, 0x8e, 0x37, 0x5b, 0xc1, + 0xa8, 0xb4, 0xd0, 0xab, 0xba, 0x87, 0x64, 0x1a, 0x85, 0xc2, 0x57, 0x90, 0x6b, 0x15, 0x79, 0xf7, + 0x9f, 0x1f, 0xaa, 0xcb, 0x17, 0x63, 0xd2, 0x81, 0x2b, 0x23, 0xc9, 0xab, 0x1b, 0x28, 0x37, 0xc2, + 0xde, 0x10, 0xc2, 0x1b, 0x2f, 0x18, 0x62, 0xd3, 0x2c, 0xbc, 0xbe, 0xa9, 0x66, 0xde, 0xdd, 0x54, + 0xa5, 0x8f, 0x37, 0x55, 0xa9, 0x29, 0xc7, 0xbb, 0xed, 0x37, 0x12, 0xda, 0x4d, 0xdb, 0xd4, 0x19, + 0x76, 0x29, 0xb0, 0x7f, 0xfc, 0xe0, 0x8c, 0x1f, 0xa5, 0x03, 0x81, 0x8b, 0x3d, 0xf7, 0x25, 0x66, + 0xae, 0x4f, 0x9e, 0xbf, 0xa0, 0x1f, 0xea, 0xdc, 0x7c, 0xbb, 0xdb, 0x6f, 0x73, 0x48, 0xe1, 0x4d, + 0x76, 0x18, 0x66, 0x54, 0x3d, 0x40, 0xaa, 0xe5, 0x13, 0x86, 0x5d, 0x42, 0x4d, 0xa0, 0xcc, 0xed, + 0x63, 0xde, 0x4f, 0x31, 0xf5, 0xa6, 0xd6, 0xe3, 0xfc, 0xdf, 0x71, 0x5a, 0xfd, 0x1d, 0xad, 0x7b, + 0x98, 0x26, 0x77, 0x40, 0x30, 0xf1, 0x69, 0x68, 0xe5, 0x5a, 0x54, 0x53, 0xe2, 0x69, 0x71, 0x0b, + 0xe7, 0x3c, 0xa9, 0xee, 0x20, 0x14, 0xbd, 0x38, 0xec, 0x08, 0x33, 0x63, 0x54, 0x11, 0xf1, 0x23, + 0x07, 0xd4, 0x43, 0x54, 0x70, 0x2c, 0xe1, 0x48, 0x88, 0x65, 0x43, 0x4c, 0xe5, 0xd8, 0xf4, 0xa1, + 0x8a, 0x8e, 0xdb, 0xa1, 0x39, 0x47, 0x0e, 0x18, 0xc8, 0xb1, 0xe2, 0x35, 0x97, 0xf6, 0xdc, 0x11, + 0xa4, 0x06, 0x26, 0x91, 0xe6, 0x71, 0x61, 0x68, 0x0c, 0x59, 0xfe, 0x90, 0xb0, 0x70, 0x64, 0xe6, + 0xa0, 0x36, 0x0f, 0xcf, 0x4f, 0xde, 0x72, 0x8a, 0x99, 0x9b, 0x3c, 0x8e, 0x08, 0x99, 0x95, 0x05, + 0x24, 0x51, 0x99, 0x0d, 0xa7, 0x9c, 0x46, 0x92, 0xe1, 0x8c, 0x10, 0xa1, 0xa2, 0x2c, 0x20, 0x42, + 0x65, 0x17, 0x15, 0x22, 0xc3, 0x84, 0x10, 0x4a, 0x51, 0x79, 0x91, 0x11, 0x5a, 0x33, 0x50, 0xc8, + 0xe5, 0x9f, 0x82, 0x42, 0xb1, 0x89, 0x7e, 0xa1, 0x30, 0xc0, 0x01, 0x66, 0x60, 0x9b, 0x73, 0x25, + 0x6b, 0xa9, 0x92, 0x8d, 0x84, 0x39, 0x49, 0xd5, 0x6e, 0x21, 0x85, 0x4e, 0x68, 0xd4, 0x4a, 0x21, + 0xdd, 0x30, 0x9d, 0xd0, 0xe4, 0x4c, 0x1c, 0x11, 0x8a, 0xab, 0x0b, 0x88, 0x50, 0xd1, 0xd1, 0x1a, + 0xee, 0xfa, 0x01, 0x33, 0xe9, 0x00, 0x93, 0x48, 0xac, 0x94, 0x22, 0x8b, 0x61, 0xb6, 0x33, 0xc0, + 0x24, 0x94, 0x6c, 0xca, 0xf1, 0x9b, 0x6d, 0xed, 0xdd, 0x7e, 0xd0, 0x32, 0xb7, 0x53, 0x4d, 0xba, + 0x9b, 0x6a, 0xd2, 0xfd, 0x54, 0x93, 0xde, 0x4f, 0x35, 0xe9, 0xd5, 0xa3, 0x96, 0xb9, 0x7b, 0xd4, + 0x32, 0xf7, 0x8f, 0x5a, 0xe6, 0x85, 0x1c, 0xff, 0xa7, 0xbe, 0x04, 0x00, 0x00, 0xff, 0xff, 0x51, + 0x21, 0x67, 0x2c, 0x7f, 0x06, 0x00, 0x00, } func (this *MVCCMetadata_SequencedIntent) Equal(that interface{}) bool { @@ -566,69 +509,6 @@ func (this *MVCCStats) Equal(that interface{}) bool { } return true } -func (this *MVCCStatsLegacyRepresentation) Equal(that interface{}) bool { - if that == nil { - return this == nil - } - - that1, ok := that.(*MVCCStatsLegacyRepresentation) - if !ok { - that2, ok := that.(MVCCStatsLegacyRepresentation) - if ok { - that1 = &that2 - } else { - return false - } - } - if that1 == nil { - return this == nil - } else if this == nil { - return false - } - if this.ContainsEstimates != that1.ContainsEstimates { - return false - } - if this.LastUpdateNanos != that1.LastUpdateNanos { - return false - } - if this.IntentAge != that1.IntentAge { - return false - } - if this.GCBytesAge != that1.GCBytesAge { - return false - } - if this.LiveBytes != that1.LiveBytes { - return false - } - if this.LiveCount != that1.LiveCount { - return false - } - if this.KeyBytes != that1.KeyBytes { - return false - } - if this.KeyCount != that1.KeyCount { - return false - } - if this.ValBytes != that1.ValBytes { - return false - } - if this.ValCount != that1.ValCount { - return false - } - if this.IntentBytes != that1.IntentBytes { - return false - } - if this.IntentCount != that1.IntentCount { - return false - } - if this.SysBytes != that1.SysBytes { - return false - } - if this.SysCount != that1.SysCount { - return false - } - return true -} func (m *MVCCMetadata) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -894,84 +774,6 @@ func (m *MVCCStats) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } -func (m *MVCCStatsLegacyRepresentation) 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 *MVCCStatsLegacyRepresentation) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *MVCCStatsLegacyRepresentation) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - i = encodeVarintMvcc(dAtA, i, uint64(m.ContainsEstimates)) - i-- - dAtA[i] = 0x70 - i -= 8 - encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.SysCount)) - i-- - dAtA[i] = 0x69 - i -= 8 - encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.SysBytes)) - i-- - dAtA[i] = 0x61 - i -= 8 - encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.IntentCount)) - i-- - dAtA[i] = 0x59 - i -= 8 - encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.IntentBytes)) - i-- - dAtA[i] = 0x51 - i -= 8 - encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.ValCount)) - i-- - dAtA[i] = 0x49 - i -= 8 - encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.ValBytes)) - i-- - dAtA[i] = 0x41 - i -= 8 - encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.KeyCount)) - i-- - dAtA[i] = 0x39 - i -= 8 - encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.KeyBytes)) - i-- - dAtA[i] = 0x31 - i -= 8 - encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.LiveCount)) - i-- - dAtA[i] = 0x29 - i -= 8 - encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.LiveBytes)) - i-- - dAtA[i] = 0x21 - i -= 8 - encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.GCBytesAge)) - i-- - dAtA[i] = 0x19 - i -= 8 - encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.IntentAge)) - i-- - dAtA[i] = 0x11 - i -= 8 - encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.LastUpdateNanos)) - i-- - dAtA[i] = 0x9 - return len(dAtA) - i, nil -} - func encodeVarintMvcc(dAtA []byte, offset int, v uint64) int { offset -= sovMvcc(v) base := offset @@ -1132,69 +934,6 @@ func NewPopulatedMVCCStats(r randyMvcc, easy bool) *MVCCStats { return this } -func NewPopulatedMVCCStatsLegacyRepresentation(r randyMvcc, easy bool) *MVCCStatsLegacyRepresentation { - this := &MVCCStatsLegacyRepresentation{} - this.LastUpdateNanos = int64(r.Int63()) - if r.Intn(2) == 0 { - this.LastUpdateNanos *= -1 - } - this.IntentAge = int64(r.Int63()) - if r.Intn(2) == 0 { - this.IntentAge *= -1 - } - this.GCBytesAge = int64(r.Int63()) - if r.Intn(2) == 0 { - this.GCBytesAge *= -1 - } - this.LiveBytes = int64(r.Int63()) - if r.Intn(2) == 0 { - this.LiveBytes *= -1 - } - this.LiveCount = int64(r.Int63()) - if r.Intn(2) == 0 { - this.LiveCount *= -1 - } - this.KeyBytes = int64(r.Int63()) - if r.Intn(2) == 0 { - this.KeyBytes *= -1 - } - this.KeyCount = int64(r.Int63()) - if r.Intn(2) == 0 { - this.KeyCount *= -1 - } - this.ValBytes = int64(r.Int63()) - if r.Intn(2) == 0 { - this.ValBytes *= -1 - } - this.ValCount = int64(r.Int63()) - if r.Intn(2) == 0 { - this.ValCount *= -1 - } - this.IntentBytes = int64(r.Int63()) - if r.Intn(2) == 0 { - this.IntentBytes *= -1 - } - this.IntentCount = int64(r.Int63()) - if r.Intn(2) == 0 { - this.IntentCount *= -1 - } - this.SysBytes = int64(r.Int63()) - if r.Intn(2) == 0 { - this.SysBytes *= -1 - } - this.SysCount = int64(r.Int63()) - if r.Intn(2) == 0 { - this.SysCount *= -1 - } - this.ContainsEstimates = int64(r.Int63()) - if r.Intn(2) == 0 { - this.ContainsEstimates *= -1 - } - if !easy && r.Intn(10) != 0 { - } - return this -} - type randyMvcc interface { Float32() float32 Float64() float64 @@ -1358,29 +1097,6 @@ func (m *MVCCStats) Size() (n int) { return n } -func (m *MVCCStatsLegacyRepresentation) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - n += 9 - n += 9 - n += 9 - n += 9 - n += 9 - n += 9 - n += 9 - n += 9 - n += 9 - n += 9 - n += 9 - n += 9 - n += 9 - n += 1 + sovMvcc(uint64(m.ContainsEstimates)) - return n -} - func sovMvcc(x uint64) (n int) { return (math_bits.Len64(x|1) + 6) / 7 } @@ -2131,205 +1847,6 @@ func (m *MVCCStats) Unmarshal(dAtA []byte) error { } return nil } -func (m *MVCCStatsLegacyRepresentation) 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 ErrIntOverflowMvcc - } - 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: MVCCStatsLegacyRepresentation: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: MVCCStatsLegacyRepresentation: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 1 { - return fmt.Errorf("proto: wrong wireType = %d for field LastUpdateNanos", wireType) - } - m.LastUpdateNanos = 0 - if (iNdEx + 8) > l { - return io.ErrUnexpectedEOF - } - m.LastUpdateNanos = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) - iNdEx += 8 - case 2: - if wireType != 1 { - return fmt.Errorf("proto: wrong wireType = %d for field IntentAge", wireType) - } - m.IntentAge = 0 - if (iNdEx + 8) > l { - return io.ErrUnexpectedEOF - } - m.IntentAge = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) - iNdEx += 8 - case 3: - if wireType != 1 { - return fmt.Errorf("proto: wrong wireType = %d for field GCBytesAge", wireType) - } - m.GCBytesAge = 0 - if (iNdEx + 8) > l { - return io.ErrUnexpectedEOF - } - m.GCBytesAge = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) - iNdEx += 8 - case 4: - if wireType != 1 { - return fmt.Errorf("proto: wrong wireType = %d for field LiveBytes", wireType) - } - m.LiveBytes = 0 - if (iNdEx + 8) > l { - return io.ErrUnexpectedEOF - } - m.LiveBytes = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) - iNdEx += 8 - case 5: - if wireType != 1 { - return fmt.Errorf("proto: wrong wireType = %d for field LiveCount", wireType) - } - m.LiveCount = 0 - if (iNdEx + 8) > l { - return io.ErrUnexpectedEOF - } - m.LiveCount = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) - iNdEx += 8 - case 6: - if wireType != 1 { - return fmt.Errorf("proto: wrong wireType = %d for field KeyBytes", wireType) - } - m.KeyBytes = 0 - if (iNdEx + 8) > l { - return io.ErrUnexpectedEOF - } - m.KeyBytes = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) - iNdEx += 8 - case 7: - if wireType != 1 { - return fmt.Errorf("proto: wrong wireType = %d for field KeyCount", wireType) - } - m.KeyCount = 0 - if (iNdEx + 8) > l { - return io.ErrUnexpectedEOF - } - m.KeyCount = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) - iNdEx += 8 - case 8: - if wireType != 1 { - return fmt.Errorf("proto: wrong wireType = %d for field ValBytes", wireType) - } - m.ValBytes = 0 - if (iNdEx + 8) > l { - return io.ErrUnexpectedEOF - } - m.ValBytes = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) - iNdEx += 8 - case 9: - if wireType != 1 { - return fmt.Errorf("proto: wrong wireType = %d for field ValCount", wireType) - } - m.ValCount = 0 - if (iNdEx + 8) > l { - return io.ErrUnexpectedEOF - } - m.ValCount = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) - iNdEx += 8 - case 10: - if wireType != 1 { - return fmt.Errorf("proto: wrong wireType = %d for field IntentBytes", wireType) - } - m.IntentBytes = 0 - if (iNdEx + 8) > l { - return io.ErrUnexpectedEOF - } - m.IntentBytes = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) - iNdEx += 8 - case 11: - if wireType != 1 { - return fmt.Errorf("proto: wrong wireType = %d for field IntentCount", wireType) - } - m.IntentCount = 0 - if (iNdEx + 8) > l { - return io.ErrUnexpectedEOF - } - m.IntentCount = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) - iNdEx += 8 - case 12: - if wireType != 1 { - return fmt.Errorf("proto: wrong wireType = %d for field SysBytes", wireType) - } - m.SysBytes = 0 - if (iNdEx + 8) > l { - return io.ErrUnexpectedEOF - } - m.SysBytes = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) - iNdEx += 8 - case 13: - if wireType != 1 { - return fmt.Errorf("proto: wrong wireType = %d for field SysCount", wireType) - } - m.SysCount = 0 - if (iNdEx + 8) > l { - return io.ErrUnexpectedEOF - } - m.SysCount = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) - iNdEx += 8 - case 14: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field ContainsEstimates", wireType) - } - m.ContainsEstimates = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowMvcc - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.ContainsEstimates |= int64(b&0x7F) << shift - if b < 0x80 { - break - } - } - default: - iNdEx = preIndex - skippy, err := skipMvcc(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthMvcc - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} func skipMvcc(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 diff --git a/pkg/storage/enginepb/mvcc.proto b/pkg/storage/enginepb/mvcc.proto index 863b9f146250..8b021dfc103b 100644 --- a/pkg/storage/enginepb/mvcc.proto +++ b/pkg/storage/enginepb/mvcc.proto @@ -217,31 +217,4 @@ message MVCCStats { // WARNING: Do not add any PII-holding fields here, as this // whole message is marked as safe for log redaction. -} - -// MVCCStatsLegacyRepresentation is almost identical to MVCCStats, except -// it does not have a field tracking abort span bytes. -// The abort span bytes field in MVCCStats didn't exist in earlier versions, -// and its addition of it to MVCCStats causes the struct to be larger than it was -// previously. The discrepancy between the size of MVCCStats between versions -// could cause incorrect sysbyte counts during version migration. -// MVCCStatsLegacyRepresentation is the older version of MVCCStats, and allows -// us to prevent this discrepancy during version migration. -message MVCCStatsLegacyRepresentation { - option (gogoproto.equal) = true; - option (gogoproto.populate) = true; - optional int64 contains_estimates = 14 [(gogoproto.nullable) = false]; - optional sfixed64 last_update_nanos = 1 [(gogoproto.nullable) = false]; - optional sfixed64 intent_age = 2 [(gogoproto.nullable) = false]; - optional sfixed64 gc_bytes_age = 3 [(gogoproto.nullable) = false, (gogoproto.customname) = "GCBytesAge"]; - optional sfixed64 live_bytes = 4 [(gogoproto.nullable) = false]; - optional sfixed64 live_count = 5 [(gogoproto.nullable) = false]; - optional sfixed64 key_bytes = 6 [(gogoproto.nullable) = false]; - optional sfixed64 key_count = 7 [(gogoproto.nullable) = false]; - optional sfixed64 val_bytes = 8 [(gogoproto.nullable) = false]; - optional sfixed64 val_count = 9 [(gogoproto.nullable) = false]; - optional sfixed64 intent_bytes = 10 [(gogoproto.nullable) = false]; - optional sfixed64 intent_count = 11 [(gogoproto.nullable) = false]; - optional sfixed64 sys_bytes = 12 [(gogoproto.nullable) = false]; - optional sfixed64 sys_count = 13 [(gogoproto.nullable) = false]; -} +} \ No newline at end of file