Skip to content

Commit

Permalink
Merge #34784 #34785
Browse files Browse the repository at this point in the history
34784: kv: remove VersionTxnSpanRefresh gate r=nvanbenschoten a=nvanbenschoten

The second commit deletes all unused `VersionKeys`, in accordance with 10a7dd3.

34785: storage: remove DisallowUnsequencedTransactionalWrites r=nvanbenschoten a=nvanbenschoten

The assertion is no longer necessary and wasn't even being used anymore.

Release note: None

Co-authored-by: Nathan VanBenschoten <[email protected]>
  • Loading branch information
craig[bot] and nvanbenschoten committed Feb 11, 2019
3 parents 8ad594e + 0433e5e + 333d48a commit ac51198
Show file tree
Hide file tree
Showing 4 changed files with 65 additions and 80 deletions.
2 changes: 1 addition & 1 deletion pkg/kv/txn_interceptor_span_refresher.go
Original file line number Diff line number Diff line change
Expand Up @@ -188,7 +188,7 @@ func (sr *txnSpanRefresher) maybeRetrySend(

// Check for an error which can be retried after updating spans.
canRetryTxn, retryTxn := roachpb.CanTransactionRetryAtRefreshedTimestamp(ctx, pErr)
if !canRetryTxn || !sr.canAutoRetry || !sr.st.Version.IsActive(cluster.VersionTxnSpanRefresh) {
if !canRetryTxn || !sr.canAutoRetry {
return nil, pErr, hlc.Timestamp{}
}

Expand Down
130 changes: 64 additions & 66 deletions pkg/settings/cluster/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,26 +29,19 @@ type VersionKey int
//
// To delete a version.
// - Remove its associated runtime checks.
// - If the version is not the latest one, delete the constant and comment out
// its stanza and say "deleted" in the versionsSingleton.
// - If the version is not the latest one, delete the constant, comment out
// its stanza, and say "Removed." above the versionsSingleton.
const (
VersionBase VersionKey = iota
VersionRaftLogTruncationBelowRaft // unused
VersionBase VersionKey = iota
VersionSplitHardStateBelowRaft
VersionStatsBasedRebalancing
Version1_1
VersionMVCCNetworkStats
VersionMeta2Splits // unused
VersionRPCNetworkStats
VersionRPCVersionCheck
VersionClearRange
VersionPartitioning
VersionLeaseSequence // unused
VersionUnreplicatedTombstoneKey // unused
VersionRecomputeStats
VersionNoRaftProposalKeys // unused
VersionTxnSpanRefresh
VersionReadUncommittedRangeLookups // unused
VersionPerReplicaZoneConstraints
VersionLeasePreferences
Version2_0
Expand All @@ -57,10 +50,7 @@ const (
VersionRangeAppliedStateKey
VersionImportFormats
VersionSecondaryLookupJoins
VersionClientSideWritingFlag // unused
VersionColumnarTimeSeries
VersionTxnCoordMetaInvalidField // unused
VersionAsyncConsensus // unused
VersionBatchResponse
VersionCreateChangefeed
VersionBitArrayColumns
Expand Down Expand Up @@ -101,11 +91,12 @@ var versionsSingleton = keyedVersions([]keyedVersion{
Key: VersionBase,
Version: roachpb.Version{Major: 1},
},
{
// VersionRaftLogTruncationBelowRaft is https://github.com/cockroachdb/cockroach/pull/16993.
Key: VersionRaftLogTruncationBelowRaft,
Version: roachpb.Version{Major: 1, Minor: 0, Unstable: 1},
},
// Removed.
// {
// // VersionRaftLogTruncationBelowRaft is https://github.com/cockroachdb/cockroach/pull/16993.
// Key: VersionRaftLogTruncationBelowRaft,
// Version: roachpb.Version{Major: 1, Minor: 0, Unstable: 1},
// },
{
// VersionSplitHardStateBelowRaft is https://github.com/cockroachdb/cockroach/pull/17051.
Key: VersionSplitHardStateBelowRaft,
Expand All @@ -132,11 +123,12 @@ var versionsSingleton = keyedVersions([]keyedVersion{
Key: VersionMVCCNetworkStats,
Version: roachpb.Version{Major: 1, Minor: 1, Unstable: 2},
},
{
// VersionMeta2Splits is https://github.com/cockroachdb/cockroach/pull/18970.
Key: VersionMeta2Splits,
Version: roachpb.Version{Major: 1, Minor: 1, Unstable: 3},
},
// Removed.
// {
// // VersionMeta2Splits is https://github.com/cockroachdb/cockroach/pull/18970.
// Key: VersionMeta2Splits,
// Version: roachpb.Version{Major: 1, Minor: 1, Unstable: 3},
// },
{
// VersionRPCNetworkStats is https://github.com/cockroachdb/cockroach/pull/19897.
Key: VersionRPCNetworkStats,
Expand Down Expand Up @@ -167,36 +159,41 @@ var versionsSingleton = keyedVersions([]keyedVersion{
Key: VersionPartitioning,
Version: roachpb.Version{Major: 1, Minor: 1, Unstable: 7},
},
{
// VersionLeaseSequence is https://github.com/cockroachdb/cockroach/pull/20953.
Key: VersionLeaseSequence,
Version: roachpb.Version{Major: 1, Minor: 1, Unstable: 8},
},
{
// VersionUnreplicatedTombstoneKey is https://github.com/cockroachdb/cockroach/pull/21120.
Key: VersionUnreplicatedTombstoneKey,
Version: roachpb.Version{Major: 1, Minor: 1, Unstable: 9},
},
// Removed.
// {
// // VersionLeaseSequence is https://github.com/cockroachdb/cockroach/pull/20953.
// Key: VersionLeaseSequence,
// Version: roachpb.Version{Major: 1, Minor: 1, Unstable: 8},
// },
// Removed.
// {
// // VersionUnreplicatedTombstoneKey is https://github.com/cockroachdb/cockroach/pull/21120.
// Key: VersionUnreplicatedTombstoneKey,
// Version: roachpb.Version{Major: 1, Minor: 1, Unstable: 9},
// },
{
// VersionRecomputeStats is https://github.com/cockroachdb/cockroach/pull/21345.
Key: VersionRecomputeStats,
Version: roachpb.Version{Major: 1, Minor: 1, Unstable: 10},
},
{
// VersionNoRaftProposalKeys is https://github.com/cockroachdb/cockroach/pull/20647.
Key: VersionNoRaftProposalKeys,
Version: roachpb.Version{Major: 1, Minor: 1, Unstable: 11},
},
{
// VersionTxnSpanRefresh is https://github.com/cockroachdb/cockroach/pull/21140.
Key: VersionTxnSpanRefresh,
Version: roachpb.Version{Major: 1, Minor: 1, Unstable: 12},
},
{
// VersionReadUncommittedRangeLookups is https://github.com/cockroachdb/cockroach/pull/21276.
Key: VersionReadUncommittedRangeLookups,
Version: roachpb.Version{Major: 1, Minor: 1, Unstable: 13},
},
// Removed.
// {
// // VersionNoRaftProposalKeys is https://github.com/cockroachdb/cockroach/pull/20647.
// Key: VersionNoRaftProposalKeys,
// Version: roachpb.Version{Major: 1, Minor: 1, Unstable: 11},
// },
// Removed.
// {
// // VersionTxnSpanRefresh is https://github.com/cockroachdb/cockroach/pull/21140.
// Key: VersionTxnSpanRefresh,
// Version: roachpb.Version{Major: 1, Minor: 1, Unstable: 12},
// },
// Removed.
// {
// // VersionReadUncommittedRangeLookups is https://github.com/cockroachdb/cockroach/pull/21276.
// Key: VersionReadUncommittedRangeLookups,
// Version: roachpb.Version{Major: 1, Minor: 1, Unstable: 13},
// },
{
// VersionPerReplicaZoneConstraints is https://github.com/cockroachdb/cockroach/pull/22819.
Key: VersionPerReplicaZoneConstraints,
Expand Down Expand Up @@ -237,28 +234,29 @@ var versionsSingleton = keyedVersions([]keyedVersion{
Key: VersionSecondaryLookupJoins,
Version: roachpb.Version{Major: 2, Minor: 0, Unstable: 5},
},
{
// VersionClientsideWritingFlag is https://github.com/cockroachdb/cockroach/pull/25541.
// Before this version, the Transaction.Writing flag used to be set by the
// server. After, it is set only by the client.
Key: VersionClientSideWritingFlag,
Version: roachpb.Version{Major: 2, Minor: 0, Unstable: 6},
},
// Removed.
// {
// // VersionClientsideWritingFlag is https://github.com/cockroachdb/cockroach/pull/25541.
// Key: VersionClientSideWritingFlag,
// Version: roachpb.Version{Major: 2, Minor: 0, Unstable: 6},
// },
{
// VersionColumnarTimeSeries is https://github.com/cockroachdb/cockroach/pull/26614.
Key: VersionColumnarTimeSeries,
Version: roachpb.Version{Major: 2, Minor: 0, Unstable: 7},
},
{
// VersionTxnCoordMetaInvalidField is https://github.com/cockroachdb/cockroach/pull/27420.
Key: VersionTxnCoordMetaInvalidField,
Version: roachpb.Version{Major: 2, Minor: 0, Unstable: 8},
},
{
// VersionAsyncConsensus is https://github.com/cockroachdb/cockroach/pull/26599.
Key: VersionAsyncConsensus,
Version: roachpb.Version{Major: 2, Minor: 0, Unstable: 9},
},
// Removed.
// {
// // VersionTxnCoordMetaInvalidField is https://github.com/cockroachdb/cockroach/pull/27420.
// Key: VersionTxnCoordMetaInvalidField,
// Version: roachpb.Version{Major: 2, Minor: 0, Unstable: 8},
// },
// Removed.
// {
// // VersionAsyncConsensus is https://github.com/cockroachdb/cockroach/pull/26599.
// Key: VersionAsyncConsensus,
// Version: roachpb.Version{Major: 2, Minor: 0, Unstable: 9},
// },
{
// VersionBatchResponse is https://github.com/cockroachdb/cockroach/pull/26553.
Key: VersionBatchResponse,
Expand All @@ -269,7 +267,7 @@ var versionsSingleton = keyedVersions([]keyedVersion{
Key: VersionCreateChangefeed,
Version: roachpb.Version{Major: 2, Minor: 0, Unstable: 11},
},
// deleted
// Removed.
// {
// // VersionRangeMerges is https://github.com/cockroachdb/cockroach/pull/28865.
// Key: VersionRangeMerges,
Expand Down
9 changes: 0 additions & 9 deletions pkg/storage/storagebase/knobs.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,15 +34,6 @@ type BatchEvalTestingKnobs struct {
// NumKeysEvaluatedForRangeIntentResolution is set by the stores to the
// number of keys evaluated for range intent resolution.
NumKeysEvaluatedForRangeIntentResolution *int64
// DisallowUnsequencedTransactionalWrites enables an assertion that all
// transactional writes include Request-scoped sequence numbers. This
// assertion is not safe to enable by default, because it would trigger
// in mixed-version clusters. However, it is useful in testing to ensure
// that tests properly assign these sequence numbers.
//
// TODO(nvanbenschoten): Remove this testing knob in 2.2. The corresponding
// assertion can be performed unconditionally.
DisallowUnsequencedTransactionalWrites bool
}

// IntentResolverTestingKnobs contains testing helpers that are used during
Expand Down
4 changes: 0 additions & 4 deletions pkg/storage/store.go
Original file line number Diff line number Diff line change
Expand Up @@ -154,10 +154,6 @@ func TestStoreConfig(clock *hlc.Clock) StoreConfig {
ClosedTimestamp: container.NoopContainer(),
}

// Tests should never send unsequenced transactional writes,
// so we set this testing knob in all tests.
sc.TestingKnobs.EvalKnobs.DisallowUnsequencedTransactionalWrites = true

// Use shorter Raft tick settings in order to minimize start up and failover
// time in tests.
sc.RaftElectionTimeoutTicks = 3
Expand Down

0 comments on commit ac51198

Please sign in to comment.