Skip to content

Commit baf14b9

Browse files
craig[bot]RaduBerinde
andcommitted
Merge #150592
150592: kv: clean up 25.1 gates r=RaduBerinde a=RaduBerinde Epic: none Release note: None Co-authored-by: Radu Berinde <[email protected]>
2 parents 7001345 + 821d4d2 commit baf14b9

14 files changed

+5
-183
lines changed

pkg/clusterversion/cockroach_versions.go

Lines changed: 0 additions & 29 deletions
Original file line numberDiff line numberDiff line change
@@ -185,26 +185,9 @@ const (
185185
// V24_3 is CockroachDB v24.3. It's used for all v24.3.x patch releases.
186186
V24_3
187187

188-
TODO_Delete_V25_1_Start
189-
190-
// TODO_Delete_V25_1_AddRangeForceFlushKey adds the RangeForceFlushKey, a replicated
191-
// range-ID local key, which is written below raft.
192-
TODO_Delete_V25_1_AddRangeForceFlushKey
193-
194-
// TODO_Delete_V25_1_BatchStreamRPC adds the BatchStream RPC, which allows for more
195-
// efficient Batch unary RPCs.
196-
TODO_Delete_V25_1_BatchStreamRPC
197-
198-
// TODO_Delete_V25_1_PreparedTransactionsTable adds the system.prepared_transactions
199-
// table. The table is used to store information about prepared transaction
200-
// that are part of the XA two-phase commit protocol.
201-
TODO_Delete_V25_1_PreparedTransactionsTable
202-
203188
// V25_1 is CockroachDB v25.1. It's used for all v25.1.x patch releases.
204189
V25_1
205190

206-
TODO_Delete_V25_2_Start
207-
208191
// V25_2 is CockroachDB v25.2. It's used for all v25.2.x patch releases.
209192
V25_2
210193

@@ -262,19 +245,7 @@ var versionTable = [numKeys]roachpb.Version{
262245
V24_1: {Major: 24, Minor: 1, Internal: 0},
263246
V24_2: {Major: 24, Minor: 2, Internal: 0},
264247
V24_3: {Major: 24, Minor: 3, Internal: 0},
265-
266-
// v25.1 versions. Internal versions must be even.
267-
TODO_Delete_V25_1_Start: {Major: 24, Minor: 3, Internal: 2},
268-
269-
TODO_Delete_V25_1_AddRangeForceFlushKey: {Major: 24, Minor: 3, Internal: 8},
270-
TODO_Delete_V25_1_BatchStreamRPC: {Major: 24, Minor: 3, Internal: 10},
271-
TODO_Delete_V25_1_PreparedTransactionsTable: {Major: 24, Minor: 3, Internal: 12},
272-
273248
V25_1: {Major: 25, Minor: 1, Internal: 0},
274-
275-
// v25.2 versions. Internal versions must be even.
276-
TODO_Delete_V25_2_Start: {Major: 25, Minor: 1, Internal: 2},
277-
278249
V25_2: {Major: 25, Minor: 2, Internal: 0},
279250

280251
// v25.3 versions. Internal versions must be even.

pkg/kv/kvserver/batcheval/cmd_end_transaction.go

Lines changed: 2 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -1505,9 +1505,7 @@ func splitTriggerHelper(
15051505
// replicas that already have the unsplit range, *and* these snapshots are
15061506
// rejected (which is very wasteful). See the long comment in
15071507
// split_delay_helper.go for more details.
1508-
if rec.ClusterSettings().Version.IsActive(ctx, clusterversion.TODO_Delete_V25_1_AddRangeForceFlushKey) {
1509-
pd.Replicated.DoTimelyApplicationToAllReplicas = true
1510-
}
1508+
pd.Replicated.DoTimelyApplicationToAllReplicas = true
15111509

15121510
pd.Local.Metrics = &result.Metrics{
15131511
SplitsWithEstimatedStats: h.splitsWithEstimates,
@@ -1610,9 +1608,7 @@ func mergeTrigger(
16101608
// the merge distributed txn, when sending a kvpb.SubsumeRequest. But since
16111609
// we have force-flushed once during the merge txn anyway, we choose to
16121610
// complete the merge story and finish the merge on all replicas.
1613-
if rec.ClusterSettings().Version.IsActive(ctx, clusterversion.TODO_Delete_V25_1_AddRangeForceFlushKey) {
1614-
pd.Replicated.DoTimelyApplicationToAllReplicas = true
1615-
}
1611+
pd.Replicated.DoTimelyApplicationToAllReplicas = true
16161612

16171613
{
16181614
// If we have GC hints populated that means we are trying to perform

pkg/kv/kvserver/batcheval/cmd_migrate.go

Lines changed: 1 addition & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -9,7 +9,6 @@ import (
99
"context"
1010
"time"
1111

12-
"github.com/cockroachdb/cockroach/pkg/clusterversion"
1312
"github.com/cockroachdb/cockroach/pkg/keys"
1413
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
1514
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
@@ -85,10 +84,7 @@ func Migrate(
8584
// Set DoTimelyApplicationToAllReplicas so that migrates are applied on all
8685
// replicas. This is done since MigrateRequests trigger a call to
8786
// waitForApplication (see Replica.executeWriteBatch).
88-
if cArgs.EvalCtx.ClusterSettings().Version.IsActive(ctx, clusterversion.TODO_Delete_V25_1_AddRangeForceFlushKey) ||
89-
cArgs.EvalCtx.EvalKnobs().OverrideDoTimelyApplicationToAllReplicas {
90-
pd.Replicated.DoTimelyApplicationToAllReplicas = true
91-
}
87+
pd.Replicated.DoTimelyApplicationToAllReplicas = true
9288
return pd, nil
9389
}
9490

pkg/kv/kvserver/batcheval/cmd_subsume.go

Lines changed: 1 addition & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -10,7 +10,6 @@ import (
1010
"context"
1111
"time"
1212

13-
"github.com/cockroachdb/cockroach/pkg/clusterversion"
1413
"github.com/cockroachdb/cockroach/pkg/keys"
1514
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
1615
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
@@ -245,9 +244,7 @@ func Subsume(
245244
// Set DoTimelyApplicationToAllReplicas so that merges are applied on all
246245
// replicas. This is needed since Replica.AdminMerge calls
247246
// waitForApplication when sending a kvpb.SubsumeRequest.
248-
if cArgs.EvalCtx.ClusterSettings().Version.IsActive(ctx, clusterversion.TODO_Delete_V25_1_AddRangeForceFlushKey) {
249-
pd.Replicated.DoTimelyApplicationToAllReplicas = true
250-
}
247+
pd.Replicated.DoTimelyApplicationToAllReplicas = true
251248
pd.Local.RepopulateSubsumeResponseLAI = args.PreserveUnreplicatedLocks
252249
return pd, nil
253250
}

pkg/kv/kvserver/flow_control_integration_test.go

Lines changed: 0 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,6 @@ import (
2626
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol"
2727
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb"
2828
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/rac2"
29-
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
3029
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader"
3130
"github.com/cockroachdb/cockroach/pkg/roachpb"
3231
"github.com/cockroachdb/cockroach/pkg/server"
@@ -2849,15 +2848,6 @@ func TestFlowControlSendQueueRangeMigrate(t *testing.T) {
28492848
RaftReportUnreachableBypass: func(_ roachpb.ReplicaID) bool {
28502849
return true
28512850
},
2852-
EvalKnobs: kvserverbase.BatchEvalTestingKnobs{
2853-
// Because we are migrating from a version (currently) prior to the
2854-
// range force flush key version gate, we won't trigger the force
2855-
// flush via migrate until we're on the endV, which defeats the
2856-
// purpose of this test. We override the behavior here to allow the
2857-
// force flush to be triggered on the startV from a Migrate
2858-
// request.
2859-
OverrideDoTimelyApplicationToAllReplicas: true,
2860-
},
28612851
FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{
28622852
UseOnlyForScratchRanges: true,
28632853
OverrideTokenDeduction: func(tokens kvflowcontrol.Tokens) kvflowcontrol.Tokens {

pkg/kv/kvserver/kvserverbase/knobs.go

Lines changed: 0 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -52,16 +52,6 @@ type BatchEvalTestingKnobs struct {
5252
// useful in certain tests.
5353
DisableTxnAutoGC bool
5454

55-
// OverrideDoTimelyApplicationToAllReplicas overrides the cluster version
56-
// check for the timely replication directive which force flushes rac2 send
57-
// queues to all replicas, if present for *Migrate* requests only. When set
58-
// to true, the directive is always set, when set to false, the default
59-
// behavior is used.
60-
//
61-
// NOTE: This currently only applies to Migrate requests and only ignores the
62-
// cluster version.
63-
OverrideDoTimelyApplicationToAllReplicas bool
64-
6555
// CommitTriggerError is called at commit triggers to simulate errors.
6656
CommitTriggerError func() error
6757

pkg/rpc/nodedialer/BUILD.bazel

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -10,7 +10,6 @@ go_library(
1010
visibility = ["//visibility:public"],
1111
deps = [
1212
"//pkg/base",
13-
"//pkg/clusterversion",
1413
"//pkg/kv/kvbase",
1514
"//pkg/kv/kvpb",
1615
"//pkg/roachpb",

pkg/rpc/nodedialer/nodedialer.go

Lines changed: 1 addition & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -12,7 +12,6 @@ import (
1212
"time"
1313

1414
"github.com/cockroachdb/cockroach/pkg/base"
15-
"github.com/cockroachdb/cockroach/pkg/clusterversion"
1615
"github.com/cockroachdb/cockroach/pkg/kv/kvbase"
1716
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
1817
"github.com/cockroachdb/cockroach/pkg/roachpb"
@@ -369,16 +368,7 @@ var batchStreamPoolingEnabled = settings.RegisterBoolSetting(
369368
)
370369

371370
func shouldUseBatchStreamPoolClient(ctx context.Context, st *cluster.Settings) bool {
372-
// NOTE: we use ActiveVersionOrEmpty(ctx).IsActive(...) instead of the more
373-
// common IsActive(ctx, ...) to avoid a fatal error if an RPC is made before
374-
// the cluster version is initialized.
375-
if !st.Version.ActiveVersionOrEmpty(ctx).IsActive(clusterversion.TODO_Delete_V25_1_BatchStreamRPC) {
376-
return false
377-
}
378-
if !batchStreamPoolingEnabled.Get(&st.SV) {
379-
return false
380-
}
381-
return true
371+
return batchStreamPoolingEnabled.Get(&st.SV)
382372
}
383373

384374
// batchStreamPoolClient is a client that sends Batch RPCs using a pooled

pkg/sql/pg_catalog.go

Lines changed: 0 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -16,7 +16,6 @@ import (
1616
"time"
1717
"unicode"
1818

19-
"github.com/cockroachdb/cockroach/pkg/clusterversion"
2019
"github.com/cockroachdb/cockroach/pkg/keys"
2120
"github.com/cockroachdb/cockroach/pkg/security/username"
2221
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
@@ -2459,11 +2458,6 @@ var pgCatalogPreparedXactsTable = virtualSchemaTable{
24592458
https://www.postgresql.org/docs/9.6/view-pg-prepared-xacts.html`,
24602459
schema: vtable.PGCatalogPreparedXacts,
24612460
populate: func(ctx context.Context, p *planner, dbContext catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error {
2462-
if !p.IsActive(ctx, clusterversion.TODO_Delete_V25_1_PreparedTransactionsTable) {
2463-
// TODO(nvanbenschoten): Remove this logic when mixed-version support
2464-
// with v24.3 is no longer necessary.
2465-
return nil
2466-
}
24672461
rows, err := p.InternalSQLTxn().QueryBufferedEx(
24682462
ctx,
24692463
"select-prepared-transactions",

pkg/sql/two_phase_commit.go

Lines changed: 0 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -8,7 +8,6 @@ package sql
88
import (
99
"context"
1010

11-
"github.com/cockroachdb/cockroach/pkg/clusterversion"
1211
"github.com/cockroachdb/cockroach/pkg/kv"
1312
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
1413
"github.com/cockroachdb/cockroach/pkg/roachpb"
@@ -62,12 +61,6 @@ func (ex *connExecutor) execPrepareTransactionInOpenState(
6261
func (ex *connExecutor) execPrepareTransactionInOpenStateInternal(
6362
ctx context.Context, s *tree.PrepareTransaction,
6463
) error {
65-
// TODO(nvanbenschoten): Remove this logic when mixed-version support with
66-
// v24.3 is no longer necessary.
67-
if !ex.planner.EvalContext().Settings.Version.IsActive(ctx, clusterversion.TODO_Delete_V25_1_PreparedTransactionsTable) {
68-
return pgerror.Newf(pgcode.FeatureNotSupported, "PREPARE TRANSACTION unsupported in mixed-version cluster")
69-
}
70-
7164
// TODO(nvanbenschoten): why are these needed here (and in the equivalent
7265
// functions for commit and rollback)? Shouldn't they be handled by
7366
// connExecutor.resetExtraTxnState?
@@ -213,12 +206,6 @@ func (p *planner) endPreparedTxnNode(globalID *tree.StrVal, commit bool) *endPre
213206
}
214207

215208
func (f *endPreparedTxnNode) startExec(params runParams) error {
216-
// TODO(nvanbenschoten): Remove this logic when mixed-version support with
217-
// v24.3 is no longer necessary.
218-
if !params.EvalContext().Settings.Version.IsActive(params.ctx, clusterversion.TODO_Delete_V25_1_PreparedTransactionsTable) {
219-
return pgerror.Newf(pgcode.FeatureNotSupported, "%s unsupported in mixed-version cluster", f.stmtName())
220-
}
221-
222209
if err := f.checkNoActiveTxn(params); err != nil {
223210
return err
224211
}

0 commit comments

Comments
 (0)