diff --git a/pkg/cli/debug_check_store.go b/pkg/cli/debug_check_store.go index 9799a570052a..10a1529faed1 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.LocalRaftTruncatedStateSuffix): + case bytes.Equal(suffix, keys.LocalRaftTruncatedStateLegacySuffix): var trunc roachpb.RaftTruncatedState if err := kv.Value.GetProto(&trunc); err != nil { return err @@ -264,6 +264,12 @@ 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 921b46d38345..40a5c8bcd606 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -155,8 +155,37 @@ const ( _ Key = iota - 1 // want first named one to start at zero // v21.1 versions. + // + // 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 // V21_1 is CockroachDB v21.1. It's used for all v21.1.x patch releases. - // TODO(irfansharif): This can be removed as part of once #69828 lands (bumping the min cluster version. + // + // TODO(irfansharif): This can be removed as part of #69828 (bumping the min + // cluster version). V21_1 // v21.1PLUS release. This is a special v21.1.x release with extra changes, @@ -278,6 +307,22 @@ const ( // to be added (i.e., when cutting the final release candidate). var versionsSingleton = keyedVersions{ // v21.1 versions. + { + 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}, + }, { // V21_1 is CockroachDB v21.1. It's used for all v21.1.x patch releases. Key: V21_1, diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index c7996d346bd4..73b8619e518d 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -8,45 +8,50 @@ func _() { // An "invalid array index" compiler error signifies that the constant values have changed. // Re-run the stringer command to generate them again. var x [1]struct{} - _ = x[V21_1-0] - _ = x[Start21_1PLUS-1] - _ = x[Start21_2-2] - _ = x[JoinTokensTable-3] - _ = x[AcquisitionTypeInLeaseHistory-4] - _ = x[SerializeViewUDTs-5] - _ = x[ExpressionIndexes-6] - _ = x[DeleteDeprecatedNamespaceTableDescriptorMigration-7] - _ = x[FixDescriptors-8] - _ = x[SQLStatsTable-9] - _ = x[DatabaseRoleSettings-10] - _ = x[TenantUsageTable-11] - _ = x[SQLInstancesTable-12] - _ = x[NewRetryableRangefeedErrors-13] - _ = x[AlterSystemWebSessionsCreateIndexes-14] - _ = x[SeparatedIntentsMigration-15] - _ = x[PostSeparatedIntentsMigration-16] - _ = x[RetryJobsWithExponentialBackoff-17] - _ = x[RecordsBasedRegistry-18] - _ = x[AutoSpanConfigReconciliationJob-19] - _ = x[PreventNewInterleavedTables-20] - _ = x[EnsureNoInterleavedTables-21] - _ = x[DefaultPrivileges-22] - _ = x[ZonesTableForSecondaryTenants-23] - _ = x[UseKeyEncodeForHashShardedIndexes-24] - _ = x[DatabasePlacementPolicy-25] - _ = x[GeneratedAsIdentity-26] - _ = x[OnUpdateExpressions-27] - _ = x[SpanConfigurationsTable-28] - _ = x[BoundedStaleness-29] - _ = x[SQLStatsCompactionScheduledJob-30] - _ = x[DateAndIntervalStyle-31] - _ = x[PebbleFormatVersioned-32] - _ = x[MarkerDataKeysRegistry-33] + _ = x[Start21_1-0] + _ = x[replacedTruncatedAndRangeAppliedStateMigration-1] + _ = x[replacedPostTruncatedAndRangeAppliedStateMigration-2] + _ = x[TruncatedAndRangeAppliedStateMigration-3] + _ = x[PostTruncatedAndRangeAppliedStateMigration-4] + _ = x[V21_1-5] + _ = x[Start21_1PLUS-6] + _ = x[Start21_2-7] + _ = x[JoinTokensTable-8] + _ = x[AcquisitionTypeInLeaseHistory-9] + _ = x[SerializeViewUDTs-10] + _ = x[ExpressionIndexes-11] + _ = x[DeleteDeprecatedNamespaceTableDescriptorMigration-12] + _ = x[FixDescriptors-13] + _ = x[SQLStatsTable-14] + _ = x[DatabaseRoleSettings-15] + _ = x[TenantUsageTable-16] + _ = x[SQLInstancesTable-17] + _ = x[NewRetryableRangefeedErrors-18] + _ = x[AlterSystemWebSessionsCreateIndexes-19] + _ = x[SeparatedIntentsMigration-20] + _ = x[PostSeparatedIntentsMigration-21] + _ = x[RetryJobsWithExponentialBackoff-22] + _ = x[RecordsBasedRegistry-23] + _ = x[AutoSpanConfigReconciliationJob-24] + _ = x[PreventNewInterleavedTables-25] + _ = x[EnsureNoInterleavedTables-26] + _ = x[DefaultPrivileges-27] + _ = x[ZonesTableForSecondaryTenants-28] + _ = x[UseKeyEncodeForHashShardedIndexes-29] + _ = x[DatabasePlacementPolicy-30] + _ = x[GeneratedAsIdentity-31] + _ = x[OnUpdateExpressions-32] + _ = x[SpanConfigurationsTable-33] + _ = x[BoundedStaleness-34] + _ = x[SQLStatsCompactionScheduledJob-35] + _ = x[DateAndIntervalStyle-36] + _ = x[PebbleFormatVersioned-37] + _ = x[MarkerDataKeysRegistry-38] } -const _Key_name = "V21_1Start21_1PLUSStart21_2JoinTokensTableAcquisitionTypeInLeaseHistorySerializeViewUDTsExpressionIndexesDeleteDeprecatedNamespaceTableDescriptorMigrationFixDescriptorsSQLStatsTableDatabaseRoleSettingsTenantUsageTableSQLInstancesTableNewRetryableRangefeedErrorsAlterSystemWebSessionsCreateIndexesSeparatedIntentsMigrationPostSeparatedIntentsMigrationRetryJobsWithExponentialBackoffRecordsBasedRegistryAutoSpanConfigReconciliationJobPreventNewInterleavedTablesEnsureNoInterleavedTablesDefaultPrivilegesZonesTableForSecondaryTenantsUseKeyEncodeForHashShardedIndexesDatabasePlacementPolicyGeneratedAsIdentityOnUpdateExpressionsSpanConfigurationsTableBoundedStalenessSQLStatsCompactionScheduledJobDateAndIntervalStylePebbleFormatVersionedMarkerDataKeysRegistry" +const _Key_name = "Start21_1replacedTruncatedAndRangeAppliedStateMigrationreplacedPostTruncatedAndRangeAppliedStateMigrationTruncatedAndRangeAppliedStateMigrationPostTruncatedAndRangeAppliedStateMigrationV21_1Start21_1PLUSStart21_2JoinTokensTableAcquisitionTypeInLeaseHistorySerializeViewUDTsExpressionIndexesDeleteDeprecatedNamespaceTableDescriptorMigrationFixDescriptorsSQLStatsTableDatabaseRoleSettingsTenantUsageTableSQLInstancesTableNewRetryableRangefeedErrorsAlterSystemWebSessionsCreateIndexesSeparatedIntentsMigrationPostSeparatedIntentsMigrationRetryJobsWithExponentialBackoffRecordsBasedRegistryAutoSpanConfigReconciliationJobPreventNewInterleavedTablesEnsureNoInterleavedTablesDefaultPrivilegesZonesTableForSecondaryTenantsUseKeyEncodeForHashShardedIndexesDatabasePlacementPolicyGeneratedAsIdentityOnUpdateExpressionsSpanConfigurationsTableBoundedStalenessSQLStatsCompactionScheduledJobDateAndIntervalStylePebbleFormatVersionedMarkerDataKeysRegistry" -var _Key_index = [...]uint16{0, 5, 18, 27, 42, 71, 88, 105, 154, 168, 181, 201, 217, 234, 261, 296, 321, 350, 381, 401, 432, 459, 484, 501, 530, 563, 586, 605, 624, 647, 663, 693, 713, 734, 756} +var _Key_index = [...]uint16{0, 9, 55, 105, 143, 185, 190, 203, 212, 227, 256, 273, 290, 339, 353, 366, 386, 402, 419, 446, 481, 506, 535, 566, 586, 617, 644, 669, 686, 715, 748, 771, 790, 809, 832, 848, 878, 898, 919, 941} 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 127df894965f..c40f3657734e 100644 --- a/pkg/keys/constants.go +++ b/pkg/keys/constants.go @@ -80,12 +80,17 @@ var ( // LocalRangeAppliedStateSuffix is the suffix for the range applied state // key. LocalRangeAppliedStateSuffix = []byte("rask") - // LocalRaftTruncatedStateSuffix is the suffix for the - // RaftTruncatedState. + // LocalRaftAppliedIndexLegacySuffix is the suffix for the raft applied index. + LocalRaftAppliedIndexLegacySuffix = []byte("rfta") + // LocalRaftTruncatedStateLegacySuffix is the suffix for the legacy + // RaftTruncatedState. See VersionUnreplicatedRaftTruncatedState. // Note: This suffix is also used for unreplicated Range-ID keys. - LocalRaftTruncatedStateSuffix = []byte("rftt") + LocalRaftTruncatedStateLegacySuffix = []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 edc93fee371c..a4f00c5dabb8 100644 --- a/pkg/keys/doc.go +++ b/pkg/keys/doc.go @@ -185,12 +185,16 @@ 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" - RangeLeaseKey, // "rll-" - RangePriorReadSummaryKey, // "rprs" - RangeVersionKey, // "rver" + AbortSpanKey, // "abc-" + RangeGCThresholdKey, // "lgc-" + RangeAppliedStateKey, // "rask" + RaftAppliedIndexLegacyKey, // "rfta" + RaftTruncatedStateLegacyKey, // "rftt" + RangeLeaseKey, // "rll-" + LeaseAppliedIndexLegacyKey, // "rlla" + RangePriorReadSummaryKey, // "rprs" + RangeVersionKey, // "rver" + RangeStatsLegacyKey, // "stat" // 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 dea29eaa0fdc..d9ff59035b22 100644 --- a/pkg/keys/keys.go +++ b/pkg/keys/keys.go @@ -230,10 +230,34 @@ 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() @@ -245,6 +269,13 @@ 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 { @@ -912,6 +943,23 @@ 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...) @@ -923,6 +971,13 @@ 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...) @@ -940,7 +995,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(), LocalRaftTruncatedStateSuffix...) + return append(b.unreplicatedPrefix(), LocalRaftTruncatedStateLegacySuffix...) } // 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 c54fc0c55ab8..f92d803dcc84 100644 --- a/pkg/keys/keys_test.go +++ b/pkg/keys/keys_test.go @@ -156,7 +156,10 @@ 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 5f33e1d1491c..e3a67528b355 100644 --- a/pkg/keys/printer.go +++ b/pkg/keys/printer.go @@ -160,11 +160,13 @@ 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: LocalRaftTruncatedStateSuffix}, + {name: "RaftTruncatedState", suffix: LocalRaftTruncatedStateLegacySuffix}, {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 50dbaf7a0287..579ec2480005 100644 --- a/pkg/keys/printer_test.go +++ b/pkg/keys/printer_test.go @@ -69,9 +69,13 @@ 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 f78d8432b36c..3fb743879db2 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go @@ -982,6 +982,34 @@ 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. // @@ -1011,13 +1039,10 @@ 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, replicaVersion, + *gcThreshold, truncStateType, 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 9cfe0294475f..288d8330c911 100644 --- a/pkg/kv/kvserver/batcheval/cmd_migrate.go +++ b/pkg/kv/kvserver/batcheval/cmd_migrate.go @@ -20,6 +20,7 @@ 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" ) @@ -41,6 +42,8 @@ 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 @@ -51,6 +54,7 @@ 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) } @@ -92,6 +96,41 @@ 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 f34dc1f9fc6f..19bd9c3138f2 100644 --- a/pkg/kv/kvserver/batcheval/cmd_truncate_log.go +++ b/pkg/kv/kvserver/batcheval/cmd_truncate_log.go @@ -19,6 +19,7 @@ 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" ) @@ -30,6 +31,7 @@ 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()}) } @@ -52,6 +54,15 @@ 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") @@ -119,5 +130,18 @@ 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 b914a92457e6..3475f7bebfd5 100644 --- a/pkg/kv/kvserver/batcheval/cmd_truncate_log_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_truncate_log_test.go @@ -12,6 +12,7 @@ package batcheval import ( "context" + "fmt" "testing" "github.com/cockroachdb/cockroach/pkg/keys" @@ -21,13 +22,19 @@ 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( - t *testing.T, eng storage.Engine, rangeID roachpb.RangeID, truncState roachpb.RaftTruncatedState, + t *testing.T, + 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, @@ -38,24 +45,75 @@ func putTruncatedState( func readTruncStates( t *testing.T, eng storage.Engine, rangeID roachpb.RangeID, -) (truncatedState roachpb.RaftTruncatedState) { +) (legacy roachpb.RaftTruncatedState, unreplicated roachpb.RaftTruncatedState) { t.Helper() - found, err := storage.MVCCGetProto( + 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( context.Background(), eng, keys.RaftTruncatedStateKey(rangeID), - hlc.Timestamp{}, &truncatedState, storage.MVCCGetOptions{}, + hlc.Timestamp{}, &unreplicated, storage.MVCCGetOptions{}, ) if err != nil { t.Fatal(err) } - require.True(t, found) + if unreplicatedFound != (unreplicated != roachpb.RaftTruncatedState{}) { + t.Fatalf("unreplicated key found=%t but state is %+v", unreplicatedFound, unreplicated) + } return } -func TestTruncateLog(t *testing.T) { +const ( + expectationNeither = iota + expectationLegacy + expectationUnreplicated +) + +type unreplicatedTruncStateTest struct { + startsWithLegacy bool + exp int // see consts above +} + +func TestTruncateLogUnreplicatedTruncatedState(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 @@ -76,7 +134,8 @@ func TestTruncateLog(t *testing.T) { Term: term, } - putTruncatedState(t, eng, rangeID, truncState) + // Put down the TruncatedState specified by the test case. + putTruncatedState(t, eng, rangeID, truncState, tc.startsWithLegacy) // Send a truncation request. req := roachpb.TruncateLogRequest{ @@ -98,13 +157,25 @@ func TestTruncateLog(t *testing.T) { Term: term, } - // 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) + 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) + } 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 4d5fac25d5a6..9d15017ea4be 100644 --- a/pkg/kv/kvserver/debug_print.go +++ b/pkg/kv/kvserver/debug_print.go @@ -13,6 +13,7 @@ package kvserver import ( "bytes" "fmt" + "strconv" "strings" "github.com/cockroachdb/cockroach/pkg/keys" @@ -292,6 +293,15 @@ 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{} @@ -304,7 +314,7 @@ func tryRangeIDKey(kv storage.MVCCKeyValue) (string, error) { case bytes.Equal(suffix, keys.LocalRangeTombstoneSuffix): msg = &roachpb.RangeTombstone{} - case bytes.Equal(suffix, keys.LocalRaftTruncatedStateSuffix): + case bytes.Equal(suffix, keys.LocalRaftTruncatedStateLegacySuffix): 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 77b5e7a4a728..ce7660630fac 100644 --- a/pkg/kv/kvserver/kvserverpb/state.pb.go +++ b/pkg/kv/kvserver/kvserverpb/state.pb.go @@ -64,6 +64,15 @@ 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 @@ -251,74 +260,76 @@ func init() { } var fileDescriptor_cc107fbd3ff296cb = []byte{ - // 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, + // 1091 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x56, 0xdd, 0x6e, 0xdb, 0x36, + 0x14, 0x8e, 0x66, 0xa5, 0x75, 0x68, 0x27, 0x71, 0xd9, 0x26, 0x51, 0xd3, 0xd5, 0x36, 0x3c, 0x6c, + 0xf0, 0x80, 0x4e, 0xc2, 0xba, 0x9f, 0x62, 0x3f, 0xc0, 0x10, 0x27, 0xc0, 0xe0, 0x2c, 0x19, 0x12, + 0xc6, 0xe8, 0x80, 0xed, 0x42, 0xa0, 0x25, 0x46, 0x26, 0x2c, 0x8b, 0x2a, 0x49, 0x1b, 0x49, 0x9f, + 0x62, 0xd8, 0x13, 0xec, 0x62, 0x0f, 0xb0, 0xc7, 0xc8, 0x65, 0x2e, 0x7b, 0x65, 0x6c, 0xce, 0xcd, + 0x6e, 0xf6, 0x02, 0xbb, 0x1a, 0x48, 0x4a, 0x8e, 0xed, 0x04, 0x68, 0x06, 0xec, 0x4e, 0x3e, 0xe7, + 0xfb, 0x78, 0xce, 0xf9, 0xf8, 0x1d, 0x59, 0xa0, 0xd1, 0x1f, 0x79, 0xfd, 0x91, 0x20, 0x7c, 0x44, + 0xf8, 0xf4, 0x21, 0xed, 0x7a, 0x42, 0x62, 0x49, 0xdc, 0x94, 0x33, 0xc9, 0x60, 0x2d, 0x60, 0x41, + 0x9f, 0x33, 0x1c, 0xf4, 0xdc, 0xfe, 0xc8, 0xcd, 0x41, 0xae, 0x90, 0x8c, 0xe3, 0x88, 0xa4, 0xdd, + 0xed, 0x27, 0xd9, 0xa3, 0x47, 0x92, 0x88, 0x26, 0x24, 0xed, 0x7a, 0x83, 0x51, 0x10, 0x18, 0xf6, + 0xf6, 0x13, 0xcd, 0x4c, 0xbb, 0x1e, 0x4d, 0x24, 0xe1, 0x09, 0x8e, 0x7d, 0x8e, 0x4f, 0x65, 0x96, + 0xdc, 0xcc, 0x93, 0x03, 0x22, 0x71, 0x88, 0x25, 0xce, 0xe2, 0x30, 0x8f, 0xcf, 0xc4, 0x9c, 0xa1, + 0xa4, 0xb1, 0xd7, 0x8b, 0x03, 0x4f, 0xd2, 0x01, 0x11, 0x12, 0x0f, 0xd2, 0x2c, 0xf3, 0x28, 0x62, + 0x11, 0xd3, 0x8f, 0x9e, 0x7a, 0x32, 0xd1, 0xc6, 0x2f, 0xcb, 0xa0, 0x8c, 0x48, 0x1a, 0xd3, 0x00, + 0x9f, 0xa8, 0x69, 0xe0, 0x33, 0x00, 0x55, 0x69, 0x1f, 0xa7, 0x69, 0x4c, 0x49, 0xe8, 0xd3, 0x24, + 0x24, 0x67, 0x8e, 0x55, 0xb7, 0x9a, 0x36, 0xaa, 0xa8, 0xcc, 0x8e, 0x49, 0xb4, 0x55, 0x1c, 0xba, + 0xe0, 0x61, 0x4c, 0xb0, 0x20, 0x0b, 0xf0, 0x77, 0x34, 0xfc, 0x81, 0x4e, 0xcd, 0xe1, 0x3f, 0x07, + 0x76, 0x48, 0x44, 0xe0, 0x14, 0xea, 0x56, 0xb3, 0xf4, 0xbc, 0xe1, 0x5e, 0x8b, 0x96, 0xcd, 0xe2, + 0x22, 0x9c, 0x44, 0x64, 0x8f, 0x88, 0x80, 0xd3, 0x54, 0x32, 0x8e, 0x34, 0x1e, 0xba, 0x60, 0x59, + 0x1f, 0xe6, 0xd8, 0x9a, 0xe8, 0xdc, 0x42, 0x3c, 0x50, 0x79, 0x64, 0x60, 0xf0, 0x7b, 0xb0, 0x2e, + 0xf9, 0x30, 0x09, 0xb0, 0x24, 0xa1, 0xaf, 0xaf, 0xc9, 0x59, 0xd6, 0xcc, 0xf7, 0x6f, 0x2d, 0x79, + 0x2a, 0x3b, 0x39, 0x5a, 0xab, 0x80, 0xd6, 0xe4, 0xdc, 0x6f, 0x78, 0x0c, 0xca, 0x51, 0xe0, 0xcb, + 0x1e, 0x27, 0xa2, 0xc7, 0xe2, 0xd0, 0xb9, 0xa7, 0x0f, 0x7b, 0x3a, 0x73, 0x98, 0xd2, 0xdd, 0xed, + 0xc5, 0x81, 0xdb, 0xc9, 0x75, 0x6f, 0xad, 0x4f, 0xc6, 0xb5, 0xd2, 0xb7, 0xbb, 0x9d, 0x9c, 0x85, + 0x4a, 0x51, 0x30, 0xfd, 0x01, 0xbf, 0x02, 0xcb, 0xaa, 0x31, 0xe1, 0xdc, 0xbf, 0xd1, 0x58, 0xe6, + 0x14, 0x37, 0x77, 0x8a, 0x7b, 0xf8, 0x72, 0x77, 0x57, 0x35, 0x22, 0x90, 0xe1, 0xc0, 0xcf, 0xc0, + 0xd6, 0x50, 0xd0, 0x24, 0x9a, 0xea, 0xae, 0x67, 0xf4, 0xfb, 0xe4, 0xdc, 0x29, 0xd5, 0xad, 0x66, + 0x11, 0x3d, 0xd2, 0xe9, 0x4c, 0x7b, 0x3d, 0xc3, 0x77, 0xe4, 0x1c, 0x7e, 0x0a, 0xee, 0x8f, 0x08, + 0x17, 0x94, 0x25, 0x4e, 0x59, 0x57, 0xdd, 0xbe, 0x45, 0x8e, 0x97, 0x06, 0x81, 0x72, 0x28, 0xfc, + 0x01, 0x6c, 0x68, 0x4b, 0x04, 0x31, 0x13, 0x24, 0xf4, 0xa7, 0xc6, 0x72, 0x56, 0xef, 0xa2, 0x82, + 0x7d, 0x31, 0xae, 0x2d, 0xa1, 0x87, 0xea, 0x84, 0x5d, 0x7d, 0xc0, 0x34, 0xf5, 0xa5, 0xfd, 0xd7, + 0xaf, 0x35, 0x6b, 0xdf, 0x2e, 0x16, 0x2b, 0x2b, 0xfb, 0x76, 0x71, 0xa5, 0x02, 0xf6, 0xed, 0x22, + 0xa8, 0x94, 0x1a, 0x7f, 0xdf, 0x07, 0x2b, 0xda, 0x07, 0xed, 0xe4, 0x94, 0xc1, 0x43, 0x23, 0x14, + 0xd1, 0x26, 0x2c, 0x3d, 0xff, 0xc8, 0x7d, 0xcb, 0xa6, 0xb9, 0xb3, 0x7e, 0x6e, 0x15, 0x55, 0xf9, + 0xcb, 0x71, 0xcd, 0x32, 0xd2, 0x11, 0xf8, 0x14, 0x80, 0x18, 0x0b, 0x39, 0xe7, 0xd4, 0x15, 0x15, + 0x31, 0x0e, 0xad, 0x81, 0x52, 0x32, 0x1c, 0xf8, 0x29, 0x49, 0x42, 0x9a, 0x44, 0xda, 0xa8, 0x36, + 0x02, 0xc9, 0x70, 0x70, 0x64, 0x22, 0x39, 0x20, 0xe4, 0x2c, 0x4d, 0x49, 0xa8, 0x6d, 0x65, 0x00, + 0x7b, 0x26, 0x02, 0x1b, 0x60, 0x55, 0xcb, 0x15, 0xb3, 0xc8, 0x17, 0xf4, 0x35, 0xd1, 0x66, 0x29, + 0xa0, 0x92, 0x0a, 0x1e, 0xb0, 0xe8, 0x84, 0xbe, 0x26, 0xf0, 0xe3, 0x4c, 0xd2, 0x1c, 0xe3, 0x4b, + 0x3e, 0x14, 0x92, 0x84, 0x0e, 0xd0, 0xb7, 0x07, 0x67, 0xb0, 0x1d, 0x93, 0x81, 0x5f, 0x83, 0x6d, + 0x9c, 0xa6, 0x9c, 0x9d, 0xd1, 0x81, 0xba, 0xea, 0x94, 0xb3, 0x94, 0x09, 0x1c, 0xfb, 0xaf, 0x86, + 0x4c, 0x62, 0x6d, 0xa2, 0x02, 0x72, 0x66, 0x10, 0x47, 0x19, 0xe0, 0x58, 0xe5, 0xe1, 0x17, 0xe0, + 0xf1, 0x3c, 0xc3, 0xef, 0xaa, 0xb5, 0x35, 0x22, 0xac, 0x69, 0xf2, 0x66, 0x3a, 0xcb, 0x68, 0x61, + 0x41, 0x8c, 0x22, 0xdf, 0x80, 0x77, 0x17, 0xa8, 0x9c, 0x98, 0xa5, 0x7f, 0x35, 0x24, 0x43, 0xe2, + 0xac, 0xd7, 0x0b, 0xcd, 0x02, 0x7a, 0x3c, 0xc7, 0x46, 0x06, 0x71, 0xac, 0x00, 0xf0, 0x03, 0xb0, + 0xce, 0xd5, 0x6d, 0xfa, 0x03, 0x7c, 0xe6, 0x77, 0xcf, 0x25, 0x11, 0x4e, 0x51, 0x57, 0x5c, 0xd5, + 0xe1, 0x43, 0x7c, 0xd6, 0x52, 0x41, 0xf8, 0x13, 0xd8, 0xc2, 0x81, 0xa4, 0x23, 0x72, 0xd3, 0x69, + 0xe5, 0xbb, 0x3b, 0x6d, 0xc3, 0x9c, 0xb1, 0xe0, 0x35, 0xf8, 0x02, 0x6c, 0xe9, 0x6a, 0xa7, 0x84, + 0x84, 0x3e, 0x27, 0x11, 0x15, 0x92, 0x63, 0x49, 0x59, 0x22, 0xb4, 0x8d, 0x0b, 0x68, 0x73, 0x9a, + 0x46, 0xb3, 0x59, 0xf8, 0x21, 0x58, 0x91, 0x24, 0xc1, 0x89, 0xf4, 0x69, 0xe8, 0x54, 0xd4, 0x6d, + 0xb7, 0xca, 0x93, 0x71, 0xad, 0xd8, 0xd1, 0xc1, 0xf6, 0x1e, 0x2a, 0x9a, 0x74, 0x3b, 0x84, 0x04, + 0x6c, 0x2d, 0x76, 0xee, 0xa7, 0x2c, 0xa6, 0xc1, 0xb9, 0x03, 0xeb, 0x56, 0x73, 0x6d, 0xce, 0xbb, + 0x73, 0x2f, 0xbc, 0x85, 0x6e, 0x8f, 0x34, 0x09, 0x6d, 0x04, 0xb7, 0x85, 0xe1, 0xef, 0x16, 0x78, + 0xef, 0x46, 0x1d, 0x41, 0x43, 0x22, 0x39, 0x4e, 0x44, 0xca, 0xb8, 0x32, 0xf7, 0x29, 0x73, 0x1e, + 0x6a, 0xd1, 0x5e, 0xbc, 0x7d, 0x5f, 0x54, 0x07, 0x27, 0x34, 0x24, 0x9d, 0x9c, 0xaf, 0xf6, 0xae, + 0xd5, 0x54, 0x72, 0x4e, 0xc6, 0xb5, 0xfa, 0x42, 0x73, 0x37, 0x90, 0xa8, 0x1e, 0xdc, 0x44, 0xc8, + 0x59, 0xc4, 0x74, 0xd3, 0xed, 0xca, 0xf2, 0x74, 0xd3, 0x4b, 0x95, 0xf2, 0xbe, 0x5d, 0x7c, 0x50, + 0x81, 0x8d, 0xdf, 0x0a, 0x60, 0xf3, 0xf6, 0x26, 0xe0, 0x3e, 0x58, 0xe3, 0x66, 0x9d, 0x33, 0x53, + 0x64, 0x6f, 0x81, 0x3b, 0x59, 0x61, 0x35, 0xa3, 0x9a, 0x01, 0xe0, 0x10, 0x94, 0xf2, 0xb3, 0x62, + 0x4c, 0xf5, 0xea, 0x17, 0x5a, 0x9d, 0xc9, 0xb8, 0x06, 0xb2, 0x37, 0xc6, 0xc1, 0x4e, 0xfb, 0x9f, + 0x71, 0xad, 0x15, 0x51, 0xd9, 0x1b, 0x76, 0xdd, 0x80, 0x0d, 0xbc, 0x69, 0x91, 0xb0, 0x7b, 0xfd, + 0xec, 0xa5, 0xfd, 0xc8, 0x9b, 0xfd, 0x24, 0x30, 0x9d, 0x49, 0xe1, 0x05, 0x52, 0xfd, 0x11, 0xed, + 0xb4, 0x11, 0xc8, 0x0a, 0x1d, 0x60, 0xaa, 0x46, 0x08, 0x48, 0x22, 0x39, 0x8e, 0xf3, 0x11, 0x0a, + 0xff, 0x61, 0x84, 0x8c, 0x7a, 0x3d, 0x42, 0x7e, 0x96, 0x1a, 0xc1, 0xbe, 0x1e, 0x61, 0xd7, 0x84, + 0xff, 0xc7, 0x11, 0xb2, 0x42, 0x07, 0x98, 0x9a, 0xeb, 0x6b, 0x3d, 0xbb, 0xf8, 0xb3, 0xba, 0x74, + 0x31, 0xa9, 0x5a, 0x97, 0x93, 0xaa, 0xf5, 0x66, 0x52, 0xb5, 0xfe, 0x98, 0x54, 0xad, 0x9f, 0xaf, + 0xaa, 0x4b, 0x97, 0x57, 0xd5, 0xa5, 0x37, 0x57, 0xd5, 0xa5, 0x1f, 0xc1, 0xf5, 0xc7, 0x51, 0xf7, + 0x9e, 0xfe, 0xc0, 0xf8, 0xe4, 0xdf, 0x00, 0x00, 0x00, 0xff, 0xff, 0x91, 0x73, 0x5f, 0x8e, 0x3d, + 0x09, 0x00, 0x00, } func (this *ReplicaState) Equal(that interface{}) bool { @@ -361,6 +372,9 @@ 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 } @@ -515,6 +529,16 @@ 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]) @@ -822,6 +846,9 @@ 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)) @@ -1158,6 +1185,26 @@ 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 857873a119ff..4caca7500bd1 100644 --- a/pkg/kv/kvserver/kvserverpb/state.proto +++ b/pkg/kv/kvserver/kvserverpb/state.proto @@ -59,6 +59,15 @@ 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 @@ -99,7 +108,7 @@ message ReplicaState { // "follower reads" at or below this timestamp. util.hlc.Timestamp raft_closed_timestamp = 13 [(gogoproto.nullable) = false]; - reserved 8, 9, 10, 11; + reserved 8, 9, 10; } // 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 f57fc144b047..052648ba5f52 100644 --- a/pkg/kv/kvserver/raft.pb.go +++ b/pkg/kv/kvserver/raft.pb.go @@ -446,16 +446,15 @@ 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 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. + // 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). // - // TODO(irfansharif): Remove this in v22.1. - DeprecatedUnreplicatedTruncatedState bool `protobuf:"varint,8,opt,name=deprecated_unreplicated_truncated_state,json=deprecatedUnreplicatedTruncatedState,proto3" json:"deprecated_unreplicated_truncated_state,omitempty"` + // See VersionUnreplicatedRaftTruncatedState. + UnreplicatedTruncatedState bool `protobuf:"varint,8,opt,name=unreplicated_truncated_state,json=unreplicatedTruncatedState,proto3" json:"unreplicated_truncated_state,omitempty"` } func (m *SnapshotRequest_Header) Reset() { *m = SnapshotRequest_Header{} } @@ -577,89 +576,88 @@ func init() { func init() { proto.RegisterFile("kv/kvserver/raft.proto", fileDescriptor_acdcf79fd972c844) } var fileDescriptor_acdcf79fd972c844 = []byte{ - // 1302 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x57, 0x4f, 0x6f, 0xdb, 0xc6, - 0x12, 0x17, 0x6d, 0x5a, 0xa2, 0x56, 0x56, 0xcc, 0xb7, 0xcf, 0xcf, 0x8f, 0xcf, 0x78, 0x95, 0x5c, - 0x21, 0x6d, 0xdd, 0x7f, 0x14, 0xe2, 0xa4, 0x3d, 0xf4, 0x52, 0xe8, 0x0f, 0x03, 0xcb, 0x72, 0x6c, - 0x67, 0x65, 0xbb, 0x68, 0x8b, 0x96, 0x58, 0x51, 0x2b, 0x99, 0x90, 0xc4, 0x65, 0x96, 0x2b, 0xb5, - 0xca, 0xa7, 0xe8, 0x47, 0xe8, 0xad, 0xb7, 0x7e, 0x8b, 0x02, 0x3e, 0xe6, 0x98, 0x43, 0x61, 0xb4, - 0xca, 0xb1, 0xe8, 0x17, 0xc8, 0xa9, 0xd8, 0xe5, 0x52, 0x56, 0x9c, 0x38, 0x8d, 0x80, 0x06, 0x68, - 0x4f, 0x9e, 0x1d, 0xcd, 0xfc, 0x66, 0x76, 0xe6, 0x37, 0xb3, 0x34, 0xd8, 0xe8, 0x8f, 0xcb, 0xfd, - 0x71, 0x44, 0xd8, 0x98, 0xb0, 0x32, 0xc3, 0x5d, 0x6e, 0x87, 0x8c, 0x72, 0x0a, 0xff, 0xe3, 0x51, - 0xaf, 0xcf, 0x28, 0xf6, 0xce, 0xec, 0xfe, 0xd8, 0x4e, 0x2c, 0x36, 0xd7, 0xa5, 0x2a, 0x6c, 0x97, - 0x09, 0x63, 0x94, 0x45, 0xb1, 0xf1, 0xe6, 0x46, 0xa2, 0x1d, 0x12, 0x8e, 0x3b, 0x98, 0x63, 0xa5, - 0xb7, 0xe7, 0xc1, 0x07, 0xfe, 0x98, 0x04, 0x24, 0x8a, 0x66, 0x42, 0xd8, 0x9e, 0x89, 0xca, 0xbe, - 0x34, 0x6f, 0x9f, 0x08, 0x61, 0xbb, 0x1c, 0x71, 0xcc, 0x89, 0xb2, 0x29, 0x10, 0xee, 0x75, 0x64, - 0xa6, 0xe5, 0xf1, 0x6d, 0xf9, 0x37, 0x6c, 0xcf, 0x25, 0xbe, 0xb9, 0xde, 0xa3, 0x3d, 0x2a, 0xc5, - 0xb2, 0x90, 0x62, 0x6d, 0xe9, 0x77, 0x1d, 0xe4, 0x11, 0xee, 0xf2, 0x5d, 0x82, 0x19, 0x6f, 0x13, - 0xcc, 0xe1, 0xd7, 0xc0, 0x60, 0x38, 0xe8, 0x11, 0xd7, 0xef, 0x58, 0xda, 0x96, 0xb6, 0xad, 0x57, - 0x6b, 0xd3, 0x8b, 0x62, 0x06, 0x09, 0x5d, 0xa3, 0xfe, 0xf4, 0xa2, 0x78, 0xa7, 0xe7, 0xf3, 0xb3, - 0x51, 0xdb, 0xf6, 0xe8, 0xb0, 0x3c, 0x2b, 0x46, 0xa7, 0x7d, 0x29, 0x97, 0xc3, 0x7e, 0xaf, 0xac, - 0x6e, 0x6e, 0x2b, 0x3f, 0x94, 0x91, 0xa0, 0x8d, 0x0e, 0x8c, 0xc0, 0x5a, 0x97, 0xd1, 0xa1, 0xcb, - 0x48, 0x38, 0xf0, 0x3d, 0x2c, 0xc2, 0x2c, 0x6d, 0x69, 0xdb, 0xf9, 0x6a, 0x73, 0x7a, 0x51, 0xcc, - 0xdf, 0x65, 0x74, 0x88, 0xe2, 0x5f, 0x64, 0xb0, 0x8f, 0x17, 0x0b, 0x96, 0x78, 0xa2, 0x7c, 0x77, - 0x0e, 0xa8, 0x03, 0x87, 0x20, 0xcf, 0xe9, 0x7c, 0xc8, 0x65, 0x19, 0xb2, 0x31, 0xbd, 0x28, 0xe6, - 0x8e, 0xe9, 0x5f, 0x11, 0x30, 0xc7, 0xe9, 0x65, 0x38, 0x08, 0x74, 0x4e, 0xd8, 0xd0, 0xd2, 0x45, - 0xfd, 0x90, 0x94, 0xe1, 0x06, 0x48, 0x7b, 0x74, 0x38, 0xf4, 0xb9, 0xb5, 0x22, 0xb5, 0xea, 0x04, - 0x2d, 0x90, 0x79, 0x30, 0xf2, 0x49, 0xe4, 0x11, 0x2b, 0xbd, 0xa5, 0x6d, 0x1b, 0x28, 0x39, 0xc2, - 0x87, 0xe0, 0xff, 0x03, 0xdc, 0xeb, 0xf9, 0x41, 0xcf, 0xed, 0xd2, 0xc1, 0x80, 0x7e, 0x43, 0x58, - 0xe4, 0xd2, 0xc0, 0x4d, 0xcc, 0x8d, 0xad, 0xe5, 0xed, 0xdc, 0xce, 0x6d, 0xfb, 0x85, 0x8c, 0xb4, - 0x67, 0x14, 0xba, 0xa4, 0x95, 0xbd, 0xaf, 0xc4, 0xaa, 0x7e, 0x7e, 0x51, 0x4c, 0xa1, 0xff, 0x29, - 0xf8, 0xbb, 0x09, 0xfa, 0x61, 0x70, 0x5f, 0xc5, 0x3e, 0x02, 0x6f, 0xbd, 0x2c, 0xb6, 0x8b, 0x3d, - 0x6f, 0xc4, 0x30, 0x27, 0x16, 0x90, 0x39, 0xbf, 0x79, 0x2d, 0x52, 0x45, 0x19, 0xee, 0xe9, 0x46, - 0xc6, 0x34, 0x4a, 0x3f, 0xa4, 0x01, 0x14, 0x7c, 0xbb, 0x47, 0xa2, 0x08, 0xf7, 0x08, 0x22, 0x0f, - 0x46, 0x24, 0x7a, 0xfd, 0xa4, 0xfb, 0x0a, 0xac, 0xc5, 0xf8, 0x11, 0xc7, 0x8c, 0xbb, 0x7d, 0x32, - 0xb1, 0x8c, 0x2d, 0x6d, 0x7b, 0xb5, 0xfa, 0xd1, 0xd3, 0x8b, 0xe2, 0xad, 0xc5, 0xb0, 0x9b, 0x64, - 0x82, 0xf2, 0x12, 0xad, 0x25, 0xc0, 0x9a, 0x64, 0x02, 0xef, 0x81, 0xd5, 0x79, 0x4e, 0x4b, 0x42, - 0xe7, 0x76, 0x6e, 0xce, 0x75, 0xe6, 0x0a, 0x61, 0xea, 0x24, 0xf2, 0x98, 0x1f, 0x72, 0xca, 0x54, - 0x2b, 0x72, 0x73, 0x7c, 0x85, 0x0d, 0x00, 0x2e, 0xd9, 0x2a, 0xa9, 0xba, 0x18, 0x58, 0x76, 0xc6, - 0x45, 0x58, 0x06, 0x99, 0x61, 0x5c, 0x6a, 0x49, 0xc6, 0xdc, 0xce, 0x9a, 0x1d, 0xaf, 0x06, 0x5b, - 0x75, 0x40, 0xb9, 0x24, 0x56, 0xf3, 0x74, 0x5c, 0x59, 0x8c, 0x8e, 0xd9, 0x7f, 0x12, 0x1d, 0xe1, - 0x1e, 0x00, 0x67, 0xc9, 0xce, 0x8b, 0xac, 0xb4, 0xcc, 0xfd, 0xe6, 0x35, 0xb9, 0x3f, 0xb3, 0x20, - 0x55, 0xb2, 0x73, 0xde, 0xb0, 0x05, 0xd6, 0x66, 0x27, 0x97, 0x91, 0x28, 0x8c, 0xac, 0xcc, 0xc2, - 0x80, 0x37, 0x66, 0x10, 0x48, 0x20, 0x94, 0xba, 0xe0, 0xbf, 0xcf, 0x0f, 0x4a, 0x15, 0x73, 0xef, - 0x0c, 0x36, 0x81, 0xc1, 0xe2, 0x73, 0x64, 0x69, 0x32, 0xd0, 0xbb, 0x2f, 0x09, 0x74, 0x05, 0x21, - 0x8e, 0x36, 0x03, 0x28, 0x1d, 0x01, 0xeb, 0x19, 0xab, 0x28, 0xa4, 0x41, 0x44, 0x4e, 0x02, 0x9f, - 0x06, 0xd0, 0x06, 0x2b, 0xf2, 0x3d, 0x93, 0x33, 0x99, 0xdb, 0xb1, 0xae, 0x46, 0x09, 0xdb, 0xb6, - 0x23, 0x7e, 0x47, 0xb1, 0xd9, 0x27, 0xfa, 0xf9, 0xf7, 0x45, 0xad, 0xf4, 0xf3, 0x12, 0xf8, 0xf7, - 0x0b, 0x20, 0x5f, 0xfb, 0x90, 0xff, 0x7d, 0xa7, 0xb0, 0x09, 0x56, 0x46, 0xa2, 0xa0, 0x6a, 0x06, - 0xcb, 0xaf, 0xd2, 0xad, 0xb9, 0x3e, 0x28, 0xc0, 0x18, 0xa3, 0xf4, 0x5b, 0x1a, 0xac, 0xb5, 0x02, - 0x1c, 0x46, 0x67, 0x94, 0x27, 0xfb, 0xd3, 0x01, 0xe9, 0x33, 0x82, 0x3b, 0x24, 0xe9, 0xd4, 0x87, - 0xd7, 0x44, 0xb8, 0xe2, 0x67, 0xef, 0x4a, 0x27, 0xa4, 0x9c, 0xe1, 0xdb, 0xc0, 0xe8, 0x8f, 0xdd, - 0xb6, 0x20, 0x99, 0xac, 0xde, 0x6a, 0x35, 0x27, 0x3a, 0xd4, 0x3c, 0x95, 0xbc, 0x43, 0x99, 0xfe, - 0x38, 0x26, 0x60, 0x11, 0xe4, 0x06, 0xb4, 0xe7, 0x92, 0x80, 0x33, 0x9f, 0x44, 0xd6, 0xf2, 0xd6, - 0xf2, 0xf6, 0x2a, 0x02, 0x03, 0xda, 0x73, 0x62, 0x0d, 0x5c, 0x07, 0x2b, 0x5d, 0x3f, 0xc0, 0x03, - 0x79, 0x61, 0x03, 0xc5, 0x87, 0xcd, 0x1f, 0x75, 0x90, 0x8e, 0x23, 0xc2, 0x06, 0x58, 0x91, 0x1f, - 0x2f, 0x72, 0xc9, 0x5c, 0x9f, 0x6f, 0xc4, 0x29, 0xc3, 0x3d, 0x72, 0x59, 0xe5, 0x96, 0x70, 0x4a, - 0xea, 0x21, 0x11, 0x20, 0x06, 0xeb, 0x62, 0xa5, 0xb9, 0x6a, 0x83, 0xb9, 0x8a, 0xd9, 0xaa, 0xfd, - 0x0b, 0x4f, 0x06, 0x64, 0xcf, 0x3f, 0x4f, 0x6f, 0x00, 0xa0, 0x9e, 0x0f, 0xff, 0x21, 0x91, 0x54, - 0x58, 0x46, 0xd9, 0xf8, 0x09, 0xf0, 0x1f, 0x12, 0x51, 0x0e, 0x0f, 0x07, 0x6e, 0x87, 0x78, 0x03, - 0x3f, 0x20, 0xea, 0xce, 0xc0, 0xc3, 0x41, 0x3d, 0xd6, 0x88, 0x81, 0x0d, 0x99, 0x4f, 0x99, 0xcf, - 0x27, 0xf2, 0x91, 0xbf, 0x71, 0x2d, 0x05, 0xae, 0x36, 0xe8, 0x48, 0xb9, 0xa1, 0x19, 0x80, 0x00, - 0x8b, 0xb8, 0xd8, 0x61, 0xbd, 0x89, 0x95, 0x59, 0x08, 0xac, 0xa5, 0xdc, 0xd0, 0x0c, 0x00, 0x7e, - 0x0a, 0x74, 0x3e, 0x09, 0xc5, 0xf2, 0x16, 0x40, 0xef, 0xbf, 0x22, 0xd0, 0xf1, 0x24, 0x24, 0x48, - 0x3a, 0xc2, 0x13, 0xf0, 0x4e, 0x87, 0x84, 0x8c, 0x78, 0x98, 0x93, 0x8e, 0x3b, 0x0a, 0xd4, 0xbc, - 0x88, 0x03, 0x67, 0xa3, 0x20, 0x96, 0xe2, 0x56, 0x1b, 0xb2, 0x2e, 0x37, 0x2f, 0xcd, 0x4f, 0xe6, - 0xac, 0x8f, 0x13, 0x63, 0xd9, 0xe1, 0x3d, 0xdd, 0xd0, 0xcc, 0xa5, 0xd2, 0x1d, 0x60, 0x24, 0x05, - 0x80, 0x39, 0x90, 0x39, 0x39, 0x68, 0x1e, 0x1c, 0x7e, 0x76, 0x60, 0xa6, 0xe0, 0x2a, 0x30, 0x90, - 0x53, 0x3b, 0x3c, 0x75, 0xd0, 0xe7, 0xa6, 0x06, 0xf3, 0x20, 0x8b, 0x9c, 0x6a, 0x65, 0xbf, 0x72, - 0x50, 0x73, 0xcc, 0xa5, 0x92, 0x05, 0x8c, 0xe4, 0xa6, 0xc2, 0xb0, 0x79, 0xea, 0x56, 0x2b, 0xc7, - 0xb5, 0x5d, 0x33, 0x55, 0xba, 0x05, 0x74, 0x91, 0x3a, 0xdc, 0x00, 0xf0, 0xb4, 0x51, 0x71, 0x5b, - 0x07, 0x95, 0xa3, 0xd6, 0xee, 0xe1, 0xb1, 0x7b, 0xff, 0xc4, 0x39, 0x71, 0xcc, 0x94, 0x88, 0xd1, - 0x38, 0x68, 0x1c, 0x37, 0x2a, 0xfb, 0xa6, 0x56, 0xd2, 0x8d, 0x25, 0x73, 0xa9, 0xf4, 0x93, 0x06, - 0xcc, 0xcb, 0xeb, 0xab, 0x4d, 0x76, 0x17, 0xa4, 0xc5, 0x95, 0x46, 0x91, 0x1c, 0xb7, 0x1b, 0x3b, - 0xf6, 0x9f, 0xd6, 0x2d, 0x76, 0xb4, 0x5b, 0xd2, 0x0b, 0x29, 0x6f, 0xf1, 0xd8, 0x26, 0xaf, 0xb3, - 0x60, 0x6b, 0x76, 0xf6, 0x0c, 0x97, 0x1a, 0x20, 0x1d, 0xdb, 0x3e, 0x77, 0xef, 0x4a, 0xad, 0xe6, - 0x1c, 0x1d, 0x3b, 0x75, 0x53, 0x13, 0x3f, 0x55, 0x8e, 0x8e, 0xf6, 0x1b, 0x4e, 0xdd, 0x5c, 0x82, - 0x59, 0xb0, 0xe2, 0x20, 0x74, 0x88, 0xcc, 0x65, 0x61, 0x55, 0x77, 0x6a, 0xfb, 0x8d, 0x03, 0xa7, - 0x6e, 0xea, 0x7b, 0xba, 0xb1, 0x6c, 0xea, 0xa5, 0x2f, 0xc1, 0xbf, 0x6a, 0x34, 0xe8, 0xd6, 0xce, - 0x04, 0x6b, 0x6b, 0x34, 0xe0, 0xe4, 0x5b, 0x0e, 0x3f, 0x00, 0x40, 0x7c, 0x85, 0xe2, 0xa0, 0x93, - 0xec, 0xe4, 0x6c, 0x35, 0x3f, 0xbd, 0x28, 0x66, 0x6b, 0xb1, 0xb6, 0x51, 0x47, 0x59, 0x65, 0xd0, - 0xe8, 0x88, 0x6c, 0x43, 0x3c, 0x19, 0x50, 0x1c, 0x7f, 0xb1, 0xaf, 0xa2, 0xe4, 0x58, 0x7d, 0xef, - 0xfc, 0xd7, 0x42, 0xea, 0x7c, 0x5a, 0xd0, 0x1e, 0x4d, 0x0b, 0xda, 0xe3, 0x69, 0x41, 0xfb, 0x65, - 0x5a, 0xd0, 0xbe, 0x7b, 0x52, 0x48, 0x3d, 0x7a, 0x52, 0x48, 0x3d, 0x7e, 0x52, 0x48, 0x7d, 0x61, - 0x24, 0x35, 0x69, 0xa7, 0xe5, 0x3f, 0x1e, 0xb7, 0xff, 0x08, 0x00, 0x00, 0xff, 0xff, 0x1a, 0x9b, - 0xc1, 0x73, 0x61, 0x0d, 0x00, 0x00, + // 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, } func (m *RaftHeartbeat) Marshal() (dAtA []byte, err error) { @@ -1087,9 +1085,9 @@ func (m *SnapshotRequest_Header) MarshalToSizedBuffer(dAtA []byte) (int, error) i-- dAtA[i] = 0x48 } - if m.DeprecatedUnreplicatedTruncatedState { + if m.UnreplicatedTruncatedState { i-- - if m.DeprecatedUnreplicatedTruncatedState { + if m.UnreplicatedTruncatedState { dAtA[i] = 1 } else { dAtA[i] = 0 @@ -1404,7 +1402,7 @@ func (m *SnapshotRequest_Header) Size() (n int) { if m.Strategy != 0 { n += 1 + sovRaft(uint64(m.Strategy)) } - if m.DeprecatedUnreplicatedTruncatedState { + if m.UnreplicatedTruncatedState { n += 2 } if m.Type != 0 { @@ -2715,7 +2713,7 @@ func (m *SnapshotRequest_Header) Unmarshal(dAtA []byte) error { } case 8: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field DeprecatedUnreplicatedTruncatedState", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field UnreplicatedTruncatedState", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -2732,7 +2730,7 @@ func (m *SnapshotRequest_Header) Unmarshal(dAtA []byte) error { break } } - m.DeprecatedUnreplicatedTruncatedState = bool(v != 0) + m.UnreplicatedTruncatedState = 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 de23d895581b..d525efe43b0d 100644 --- a/pkg/kv/kvserver/raft.proto +++ b/pkg/kv/kvserver/raft.proto @@ -183,16 +183,15 @@ message SnapshotRequest { Type type = 9; // Whether the snapshot uses the unreplicated RaftTruncatedState or not. - // 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. + // 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). // - // TODO(irfansharif): Remove this in v22.1. - bool deprecated_unreplicated_truncated_state = 8; + // See VersionUnreplicatedRaftTruncatedState. + bool 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 8f071be6f7e8..0f4aecaabbfc 100644 --- a/pkg/kv/kvserver/rditer/replica_data_iter_test.go +++ b/pkg/kv/kvserver/rditer/replica_data_iter_test.go @@ -82,7 +82,11 @@ 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 33e1a3e33789..7c933ef1bd35 100644 --- a/pkg/kv/kvserver/replica_application_result.go +++ b/pkg/kv/kvserver/replica_application_result.go @@ -320,6 +320,12 @@ 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 30dda3f5eae4..c5112c60e764 100644 --- a/pkg/kv/kvserver/replica_application_state_machine.go +++ b/pkg/kv/kvserver/replica_application_state_machine.go @@ -15,6 +15,7 @@ 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" @@ -395,6 +396,10 @@ 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 @@ -724,8 +729,19 @@ func (b *replicaAppBatch) runPreApplyTriggersAfterStagingWriteBatch( } if res.State != nil && res.State.TruncatedState != nil { - if apply, err := handleTruncatedStateBelowRaftPreApply( + 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 { @@ -835,6 +851,10 @@ 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 @@ -940,13 +960,48 @@ 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 - return loader.SetRangeAppliedState( - ctx, b.batch, b.state.RaftAppliedIndex, b.state.LeaseAppliedIndex, - b.state.Stats, &b.state.RaftClosedTimestamp, - ) + 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 } func (b *replicaAppBatch) recordStatsOnCommit() { @@ -1246,6 +1301,11 @@ 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 cf7a9db9a649..2b7572ff4667 100644 --- a/pkg/kv/kvserver/replica_command.go +++ b/pkg/kv/kvserver/replica_command.go @@ -2440,20 +2440,42 @@ func (r *Replica) sendSnapshot( return &benignError{errors.Wrap(errMarkSnapshotError, "raft status not initialized")} } - // We avoid shipping over the past Raft log in the snapshot by changing - // the truncated state (we're allowed to -- it's an unreplicated key and not - // subject to mapping across replicas). The actual sending happens here: - _ = (*kvBatchSnapshotStrategy)(nil).Send - // and results in no log entries being sent at all. Note that - // Metadata.Index is really the applied index of the replica. - snap.State.TruncatedState = &roachpb.RaftTruncatedState{ - Index: snap.RaftSnap.Metadata.Index, - Term: snap.RaftSnap.Metadata.Term, + 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 + + if canAvoidSendingLog { + // If we're not using a legacy (replicated) truncated state, we avoid + // sending the (past) Raft log in the snapshot in the first place and + // send only those entries that are actually useful to the follower. + // This is done by changing the truncated state, which we're allowed + // to do since it is not a replicated key (and thus not subject to + // matching across replicas). The actual sending happens here: + _ = (*kvBatchSnapshotStrategy)(nil).Send + // and results in no log entries being sent at all. Note that + // Metadata.Index is really the applied index of the replica. + snap.State.TruncatedState = &roachpb.RaftTruncatedState{ + Index: snap.RaftSnap.Metadata.Index, + Term: snap.RaftSnap.Metadata.Term, + } } req := SnapshotRequest_Header{ - State: snap.State, - DeprecatedUnreplicatedTruncatedState: true, + 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, RaftMessageRequest: RaftMessageRequest{ RangeID: r.RangeID, FromReplica: sender, diff --git a/pkg/kv/kvserver/replica_consistency.go b/pkg/kv/kvserver/replica_consistency.go index cbde7ff82255..16f9c036a06d 100644 --- a/pkg/kv/kvserver/replica_consistency.go +++ b/pkg/kv/kvserver/replica_consistency.go @@ -651,10 +651,17 @@ func (r *Replica) sha512( if err != nil { return nil, err } + if rangeAppliedState == nil { + // This error is transient: the range applied state is used in v2.1 already + // but is migrated into on a per-range basis for clusters bootstrapped before + // v2.1. Clusters bootstrapped at v2.1 or higher will never hit this path since + // there's always an applied state. + return nil, errors.New("no range applied state found") + } result.PersistedMS = rangeAppliedState.RangeStats.ToStats() if statsOnly { - b, err := protoutil.Marshal(&rangeAppliedState) + b, err := protoutil.Marshal(rangeAppliedState) if err != nil { return nil, err } @@ -665,7 +672,7 @@ func (r *Replica) sha512( } kv.Key = keys.RangeAppliedStateKey(desc.RangeID) var v roachpb.Value - if err := v.SetProto(&rangeAppliedState); err != nil { + if err := v.SetProto(rangeAppliedState); err != nil { return nil, err } kv.Value = v.RawBytes diff --git a/pkg/kv/kvserver/replica_proposal.go b/pkg/kv/kvserver/replica_proposal.go index 7c0a6f8a4da1..5ffb39d9966c 100644 --- a/pkg/kv/kvserver/replica_proposal.go +++ b/pkg/kv/kvserver/replica_proposal.go @@ -18,6 +18,7 @@ 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" @@ -218,12 +219,12 @@ func (r *Replica) computeChecksumPostApply(ctx context.Context, cc kvserverpb.Co snap := r.store.engine.NewSnapshot() if cc.Checkpoint { sl := stateloader.Make(r.RangeID) - as, err := sl.LoadRangeAppliedState(ctx, snap) + rai, _, err := sl.LoadAppliedIndex(ctx, snap) if err != nil { log.Warningf(ctx, "unable to load applied index, continuing anyway") } // NB: the names here will match on all nodes, which is nice for debugging. - tag := fmt.Sprintf("r%d_at_%d", r.RangeID, as.RaftAppliedIndex) + tag := fmt.Sprintf("r%d_at_%d", r.RangeID, rai) if dir, err := r.store.checkpoint(ctx, tag); err != nil { log.Warningf(ctx, "unable to create checkpoint %s: %+v", dir, err) } else { @@ -862,6 +863,41 @@ 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 80b9503b8f08..3d1445d302d0 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -1789,44 +1789,46 @@ func (r *Replica) acquireMergeLock( return rightRepl.raftMu.Unlock, nil } -// handleTruncatedStateBelowRaftPreApply is called before applying a Raft -// command that updates the truncated state. -// -// The truncated state of a replica determines where its Raft log starts (by -// giving the last index that was already deleted). It's unreplicated -- it can -// differ between replicas at the same applied index. This divergence occurs -// primarily occurs through snapshots that contain no log entries; the truncated -// index in the snapshot is set to equal the applied index it was generated -// from. The truncation itself then is a purely replicated side effect. -// -// 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 truncated state -// provided by the leader then is merely a suggested one -- we could ignore it -// and still be correct. -// -// We also rely on log truncations happening in the apply loop -- this makes -// sure that a truncation does not remove entries to be applied that we haven't -// yet. Since a truncation only ever removes committed log indexes, and after -// choosing the index gets proposed, the truncation command itself will be -// assigned an index higher than the one it could possibly remove. By the time -// the truncation itself is handled, the state machine will have applied all -// entries the truncation could possibly affect. +// 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. // // The returned boolean tells the caller whether to apply the truncated state's // side effects, which means replacing the in-memory TruncatedState and applying // the associated RaftLogDelta. It is usually expected to be true, but may not // be for the first truncation after on a replica that recently received a // snapshot. -func handleTruncatedStateBelowRaftPreApply( +func handleTruncatedStateBelowRaft( ctx context.Context, - currentTruncatedState, suggestedTruncatedState *roachpb.RaftTruncatedState, + 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) + 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 @@ -1840,31 +1842,42 @@ func handleTruncatedStateBelowRaftPreApply( // perform well here because the tombstones could be "collapsed", // but it is hardly worth the risk at this point. prefixBuf := &loader.RangeIDPrefixBuf - for idx := currentTruncatedState.Index + 1; idx <= suggestedTruncatedState.Index; idx++ { + for idx := oldTruncatedState.Index + 1; idx <= newTruncatedState.Index; idx++ { // NB: RangeIDPrefixBufs have sufficient capacity (32 bytes) to // avoid allocating when constructing Raft log keys (16 bytes). unsafeKey := prefixBuf.RaftLogKey(idx) if err := readWriter.ClearUnversioned(unsafeKey); err != nil { - return false, errors.Wrapf(err, "unable to clear truncated Raft entries for %+v at index %d", - suggestedTruncatedState, idx) + return false, errors.Wrapf(err, "unable to clear truncated Raft entries for %+v", newTruncatedState) } } - if suggestedTruncatedState.Index <= currentTruncatedState.Index { - // The suggested truncated state moves us backwards; instruct the - // caller to not update the in-memory state. + 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 + } + // Have migrated, but this truncated state moves the existing one + // backwards, so instruct caller to not update in-memory state. return false, nil } - - // The suggested truncated state moves us forward; apply it and tell - // the caller as much. - if err := storage.MVCCPutProto( - ctx, readWriter, nil /* ms */, prefixBuf.RaftTruncatedStateKey(), - hlc.Timestamp{}, nil /* txn */, suggestedTruncatedState, - ); err != nil { - return false, errors.Wrap(err, "unable to write RaftTruncatedState") - } - + // Haven't migrated yet, don't ever discard the update. return true, nil } diff --git a/pkg/kv/kvserver/replica_raft_truncation_test.go b/pkg/kv/kvserver/replica_raft_truncation_test.go index ba25718fe9d0..4867384b1c10 100644 --- a/pkg/kv/kvserver/replica_raft_truncation_test.go +++ b/pkg/kv/kvserver/replica_raft_truncation_test.go @@ -25,7 +25,6 @@ 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) { @@ -33,16 +32,23 @@ 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. + // truncation code, in particular regarding the + // VersionUnreplicatedRaftTruncatedState migration. ctx := context.Background() - datadriven.Walk(t, "testdata/truncated_state", func(t *testing.T, path string) { + + // 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) { const rangeID = 12 loader := stateloader.Make(rangeID) eng := storage.NewDefaultInMemForTesting() defer eng.Close() - var prevTruncatedState roachpb.RaftTruncatedState datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { case "prev": @@ -52,15 +58,21 @@ 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, } - assert.NoError(t, loader.SetRaftTruncatedState(ctx, eng, truncState)) + if legacy { + assert.NoError(t, loader.SetLegacyRaftTruncatedState(ctx, eng, nil, truncState)) + } else { + assert.NoError(t, loader.SetRaftTruncatedState(ctx, eng, truncState)) + } return "" case "handle": var buf bytes.Buffer @@ -70,28 +82,31 @@ func TestHandleTruncatedStateBelowRaft(t *testing.T) { d.ScanArgs(t, "index", &index) d.ScanArgs(t, "term", &term) - suggestedTruncatedState := &roachpb.RaftTruncatedState{ + newTruncatedState := &roachpb.RaftTruncatedState{ Index: index, Term: term, } - currentTruncatedState, err := loader.LoadRaftTruncatedState(ctx, eng) - assert.NoError(t, err) - apply, err := handleTruncatedStateBelowRaftPreApply(ctx, ¤tTruncatedState, suggestedTruncatedState, loader, eng) + apply, err := handleTruncatedStateBelowRaft(ctx, &prevTruncatedState, newTruncatedState, loader, eng, false) if err != nil { return err.Error() } - fmt.Fprintf(&buf, "apply: %t\n", apply) - 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) + 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) } - 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 8587db094378..226c54c261f1 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,9 +519,18 @@ 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 - snapType SnapshotRequest_Type - placeholder *ReplicaPlaceholder + 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 } func (s *IncomingSnapshot) String() string { @@ -877,10 +886,13 @@ func (r *Replica) applySnapshot( } r.store.raftEntryCache.Drop(r.RangeID) - if err := r.raftMu.stateLoader.SetRaftTruncatedState( - ctx, &unreplicatedSST, s.TruncatedState, - ); err != nil { - return errors.Wrapf(err, "unable to write TruncatedState to unreplicated SST writer") + // 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 := unreplicatedSST.Finish(); err != nil { diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index c68e5b5fb468..3a87f6e44843 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -613,6 +613,9 @@ 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 f1688147a310..65177fea2c69 100644 --- a/pkg/kv/kvserver/stateloader/initial.go +++ b/pkg/kv/kvserver/stateloader/initial.go @@ -45,6 +45,7 @@ func WriteInitialReplicaState( desc roachpb.RangeDescriptor, lease roachpb.Lease, gcThreshold hlc.Timestamp, + truncStateType TruncatedStateType, replicaVersion roachpb.Version, ) (enginepb.MVCCStats, error) { rsl := Make(desc.RangeID) @@ -63,6 +64,9 @@ 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") @@ -82,7 +86,7 @@ func WriteInitialReplicaState( log.Fatalf(ctx, "expected trivial version, but found %+v", existingVersion) } - newMS, err := rsl.Save(ctx, readWriter, s) + newMS, err := rsl.Save(ctx, readWriter, s, truncStateType) if err != nil { return enginepb.MVCCStats{}, err } @@ -97,14 +101,32 @@ 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, - replicaVersion, + initialTruncStateType, replicaVersion, ); err != nil { return err } diff --git a/pkg/kv/kvserver/stateloader/stateloader.go b/pkg/kv/kvserver/stateloader/stateloader.go index 2ab3273f5320..d296219d0697 100644 --- a/pkg/kv/kvserver/stateloader/stateloader.go +++ b/pkg/kv/kvserver/stateloader/stateloader.go @@ -73,21 +73,34 @@ func (rsl StateLoader) Load( return kvserverpb.ReplicaState{}, err } - as, err := rsl.LoadRangeAppliedState(ctx, reader) - if err != nil { + if as, err := rsl.LoadRangeAppliedState(ctx, reader); err != nil { return kvserverpb.ReplicaState{}, err - } - 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 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 } // 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 } @@ -104,6 +117,20 @@ 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. @@ -116,7 +143,10 @@ func (rsl StateLoader) Load( // missing whenever save is called. Optional values should be reserved // strictly for use in Result. Do before merge. func (rsl StateLoader) Save( - ctx context.Context, readWriter storage.ReadWriter, state kvserverpb.ReplicaState, + 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 { @@ -125,17 +155,34 @@ func (rsl StateLoader) Save( if err := rsl.SetGCThreshold(ctx, readWriter, ms, state.GCThreshold); err != nil { return enginepb.MVCCStats{}, err } - if err := rsl.SetRaftTruncatedState(ctx, readWriter, state.TruncatedState); 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 state.Version != nil { if err := rsl.SetVersion(ctx, readWriter, ms, state.Version); 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 + 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 + } } return *ms, nil } @@ -158,14 +205,70 @@ func (rsl StateLoader) SetLease( hlc.Timestamp{}, nil, &lease) } -// LoadRangeAppliedState loads the Range applied state. +// LoadRangeAppliedState loads the Range applied state. The returned pointer +// will be nil if the applied state key is not found. func (rsl StateLoader) LoadRangeAppliedState( ctx context.Context, reader storage.Reader, -) (enginepb.RangeAppliedState, error) { +) (*enginepb.RangeAppliedState, error) { var as enginepb.RangeAppliedState - _, err := storage.MVCCGetProto(ctx, reader, rsl.RangeAppliedStateKey(), hlc.Timestamp{}, &as, + found, err := storage.MVCCGetProto(ctx, reader, rsl.RangeAppliedStateKey(), hlc.Timestamp{}, &as, storage.MVCCGetOptions{}) - return as, err + if !found { + return nil, err + } + 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{}) + 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 } // LoadMVCCStats loads the MVCC stats. @@ -173,11 +276,19 @@ func (rsl StateLoader) LoadMVCCStats( ctx context.Context, reader storage.Reader, ) (enginepb.MVCCStats, error) { // Check the applied state key. - as, err := rsl.LoadRangeAppliedState(ctx, reader) - if err != nil { + if as, err := rsl.LoadRangeAppliedState(ctx, reader); err != nil { return enginepb.MVCCStats{}, err + } else if as != nil { + return as.RangeStats.ToStats(), 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 } // SetRangeAppliedState overwrites the range applied state. This state is a @@ -212,18 +323,177 @@ 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) + meta := enginepb.MVCCMetadata{RawBytes: value.RawBytes} + 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 { - as, err := rsl.LoadRangeAppliedState(ctx, readWriter) - if err != nil { + if as, err := rsl.LoadRangeAppliedState(ctx, readWriter); err != nil { return err + } else if as != nil { + return rsl.SetRangeAppliedState( + ctx, readWriter, as.RaftAppliedIndex, as.LeaseAppliedIndex, newMS, as.RaftClosedTimestamp) } - return rsl.SetRangeAppliedState( - ctx, readWriter, as.RaftAppliedIndex, as.LeaseAppliedIndex, newMS, as.RaftClosedTimestamp) + + return rsl.writeLegacyMVCCStatsInternal(ctx, readWriter, newMS) } // SetClosedTimestamp overwrites the closed timestamp. @@ -239,6 +509,20 @@ 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, @@ -307,7 +591,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 } @@ -316,17 +600,35 @@ func (rsl StateLoader) LoadLastIndex(ctx context.Context, reader storage.Reader) return lastIndex, nil } -// LoadRaftTruncatedState loads the truncated state. +// 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. func (rsl StateLoader) LoadRaftTruncatedState( ctx context.Context, reader storage.Reader, -) (roachpb.RaftTruncatedState, error) { +) (_ roachpb.RaftTruncatedState, isLegacy bool, _ error) { var truncState roachpb.RaftTruncatedState - if _, err := storage.MVCCGetProto( + if found, err := storage.MVCCGetProto( ctx, reader, rsl.RaftTruncatedStateKey(), hlc.Timestamp{}, &truncState, storage.MVCCGetOptions{}, ); err != nil { - return roachpb.RaftTruncatedState{}, err + return roachpb.RaftTruncatedState{}, false, err + } else if found { + return truncState, false, nil } - return truncState, 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 truncState, legacyFound, nil } // SetRaftTruncatedState overwrites the truncated state. @@ -389,15 +691,15 @@ 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 } - as, err := rsl.LoadRangeAppliedState(ctx, readWriter) + raftAppliedIndex, _, err := rsl.LoadAppliedIndex(ctx, readWriter) if err != nil { return err } - return rsl.SynthesizeHardState(ctx, readWriter, hs, truncState, as.RaftAppliedIndex) + return rsl.SynthesizeHardState(ctx, readWriter, hs, truncState, raftAppliedIndex) } // SynthesizeHardState synthesizes an on-disk HardState from the given input, diff --git a/pkg/kv/kvserver/store_init.go b/pkg/kv/kvserver/store_init.go index 33aac9a89072..f1617daa0cd5 100644 --- a/pkg/kv/kvserver/store_init.go +++ b/pkg/kv/kvserver/store_init.go @@ -244,8 +244,16 @@ func WriteInitialClusterData( } } - if err := stateloader.WriteInitialRangeState(ctx, batch, *desc, initialReplicaVersion); err != nil { - return err + 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 + } } 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 9ed32235f07c..649f2171e8d2 100644 --- a/pkg/kv/kvserver/store_snapshot.go +++ b/pkg/kv/kvserver/store_snapshot.go @@ -296,11 +296,12 @@ func (kvSS *kvBatchSnapshotStrategy) Receive( } inSnap := IncomingSnapshot{ - SnapUUID: snapUUID, - SSTStorageScratch: kvSS.scratch, - LogEntries: logEntries, - State: &header.State, - snapType: header.Type, + UsesUnreplicatedTruncatedState: header.UnreplicatedTruncatedState, + SnapUUID: snapUUID, + SSTStorageScratch: kvSS.scratch, + LogEntries: logEntries, + State: &header.State, + snapType: header.Type, } expLen := inSnap.State.RaftAppliedIndex - inSnap.State.TruncatedState.Index @@ -929,6 +930,7 @@ func SendEmptySnapshot( desc, roachpb.Lease{}, hlc.Timestamp{}, // gcThreshold + stateloader.TruncatedStateUnreplicated, st.Version.ActiveVersionOrEmpty(ctx).Version, ) if err != nil { @@ -992,14 +994,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, - DeprecatedUnreplicatedTruncatedState: true, + State: state, + RaftMessageRequest: req, + RangeSize: ms.Total(), + CanDecline: false, + Priority: SnapshotRequest_RECOVERY, + Strategy: SnapshotRequest_KV_BATCH, + Type: SnapshotRequest_VIA_SNAPSHOT_QUEUE, + UnreplicatedTruncatedState: true, } stream, err := NewMultiRaftClient(cc).RaftSnapshot(ctx) diff --git a/pkg/kv/kvserver/store_test.go b/pkg/kv/kvserver/store_test.go index c6e99c7e1a0e..0844eed52bcc 100644 --- a/pkg/kv/kvserver/store_test.go +++ b/pkg/kv/kvserver/store_test.go @@ -313,7 +313,8 @@ func TestIterateIDPrefixKeys(t *testing.T) { rng := rand.New(rand.NewSource(seed)) ops := []func(rangeID roachpb.RangeID) roachpb.Key{ - keys.RaftHardStateKey, // unreplicated; sorts after tombstone + keys.RaftAppliedIndexLegacyKey, // replicated; sorts before 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/truncated_state b/pkg/kv/kvserver/testdata/truncated_state_migration/migration similarity index 100% rename from pkg/kv/kvserver/testdata/truncated_state/truncated_state rename to pkg/kv/kvserver/testdata/truncated_state_migration/migration diff --git a/pkg/kv/kvserver/testdata/truncated_state_migration/pre_migration b/pkg/kv/kvserver/testdata/truncated_state_migration/pre_migration new file mode 100644 index 000000000000..e84177bd0b51 --- /dev/null +++ b/pkg/kv/kvserver/testdata/truncated_state_migration/pre_migration @@ -0,0 +1,26 @@ +# 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 786b01a86c98..7fa2b71e6de3 100644 --- a/pkg/kv/kvserver/testing_knobs.go +++ b/pkg/kv/kvserver/testing_knobs.go @@ -15,6 +15,7 @@ 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,6 +328,9 @@ 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/BUILD.bazel b/pkg/migration/migrations/BUILD.bazel index 109418dcc205..3e6daf34c343 100644 --- a/pkg/migration/migrations/BUILD.bazel +++ b/pkg/migration/migrations/BUILD.bazel @@ -18,6 +18,7 @@ go_library( "sql_instances.go", "sql_stats.go", "tenant_usage.go", + "truncated_state.go", "zones.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/migration/migrations", @@ -76,6 +77,7 @@ go_test( "retry_jobs_with_exponential_backoff_external_test.go", "separated_intents_external_test.go", "separated_intents_test.go", + "truncated_state_external_test.go", ], data = glob(["testdata/**"]), embed = [":migrations"], @@ -89,6 +91,7 @@ go_test( "//pkg/kv/kvclient/kvcoord:with-mocks", "//pkg/kv/kvserver", "//pkg/kv/kvserver/intentresolver", + "//pkg/kv/kvserver/stateloader", "//pkg/migration", "//pkg/roachpb:with-mocks", "//pkg/security", diff --git a/pkg/migration/migrations/migrations.go b/pkg/migration/migrations/migrations.go index f8f93811763f..105b4cf25e8e 100644 --- a/pkg/migration/migrations/migrations.go +++ b/pkg/migration/migrations/migrations.go @@ -39,6 +39,16 @@ 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 26b43f8ffcb1..5e99c4796015 100644 --- a/pkg/migration/migrations/separated_intents.go +++ b/pkg/migration/migrations/separated_intents.go @@ -54,19 +54,6 @@ 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 new file mode 100644 index 000000000000..6085f049e08d --- /dev/null +++ b/pkg/migration/migrations/truncated_state.go @@ -0,0 +1,95 @@ +// 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 new file mode 100644 index 000000000000..fee03c429125 --- /dev/null +++ b/pkg/migration/migrations/truncated_state_external_test.go @@ -0,0 +1,153 @@ +// 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/roachpb" + "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 := roachpb.Version{Major: 20, Minor: 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 da306fc2ea25..3508802d554f 100644 --- a/pkg/storage/enginepb/mvcc.pb.go +++ b/pkg/storage/enginepb/mvcc.pb.go @@ -324,66 +324,123 @@ 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{ - // 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, + // 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, } func (this *MVCCMetadata_SequencedIntent) Equal(that interface{}) bool { @@ -509,6 +566,69 @@ 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) @@ -774,6 +894,84 @@ 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 @@ -934,6 +1132,69 @@ 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 @@ -1097,6 +1358,29 @@ 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 } @@ -1847,6 +2131,205 @@ 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 ffaf911e898e..863b9f146250 100644 --- a/pkg/storage/enginepb/mvcc.proto +++ b/pkg/storage/enginepb/mvcc.proto @@ -218,3 +218,30 @@ 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]; +}