diff --git a/pkg/ccl/changefeedccl/helpers_test.go b/pkg/ccl/changefeedccl/helpers_test.go index 2545c6c2dae0..4c246d78a886 100644 --- a/pkg/ccl/changefeedccl/helpers_test.go +++ b/pkg/ccl/changefeedccl/helpers_test.go @@ -271,7 +271,6 @@ func sinklessTest(testFn func(*testing.T, *gosql.DB, cdctest.TestFeedFactory)) f // order to ensure that changefeeds work as expected with them (note // that we'll still use the row-by-row engine, see #55605). sqlDB.Exec(t, `SET CLUSTER SETTING sql.defaults.vectorize=on`) - sqlDB.Exec(t, `SET CLUSTER SETTING sql.defaults.vectorize_row_count_threshold=0`) sqlDB.Exec(t, `CREATE DATABASE d`) sink, cleanup := sqlutils.PGUrl(t, s.ServingSQLAddr(), t.Name(), url.User(security.RootUser)) diff --git a/pkg/ccl/importccl/exportcsv_test.go b/pkg/ccl/importccl/exportcsv_test.go index 07f33cd0ae14..58ab4fb8b7eb 100644 --- a/pkg/ccl/importccl/exportcsv_test.go +++ b/pkg/ccl/importccl/exportcsv_test.go @@ -408,7 +408,6 @@ func TestExportVectorized(t *testing.T) { sqlDB := sqlutils.MakeSQLRunner(db) sqlDB.Exec(t, `CREATE TABLE t(a INT PRIMARY KEY)`) - sqlDB.Exec(t, `SET vectorize_row_count_threshold=0`) sqlDB.Exec(t, `EXPORT INTO CSV 'http://0.1:37957/exp_1' FROM TABLE t`) } diff --git a/pkg/cmd/roachtest/tpc_utils.go b/pkg/cmd/roachtest/tpc_utils.go index a4108a55bac5..43bc52f699ff 100644 --- a/pkg/cmd/roachtest/tpc_utils.go +++ b/pkg/cmd/roachtest/tpc_utils.go @@ -110,15 +110,3 @@ func createStatsFromTables(t *test, conn *gosql.DB, tableNames []string) { } } } - -// disableVectorizeRowCountThresholdHeuristic sets -// 'vectorize_row_count_threshold' cluster setting to zero so that the test -// would use the vectorized engine with 'vectorize=on' regardless of the -// fact whether the stats are present or not (if we don't set it, then when -// the stats are not present, we fallback to row-by-row engine even with -// `vectorize=on` set). -func disableVectorizeRowCountThresholdHeuristic(t *test, conn *gosql.DB) { - if _, err := conn.Exec("SET CLUSTER SETTING sql.defaults.vectorize_row_count_threshold=0"); err != nil { - t.Fatal(err) - } -} diff --git a/pkg/cmd/roachtest/tpcdsvec.go b/pkg/cmd/roachtest/tpcdsvec.go index 873721320228..c45b0e29b47f 100644 --- a/pkg/cmd/roachtest/tpcdsvec.go +++ b/pkg/cmd/roachtest/tpcdsvec.go @@ -57,7 +57,6 @@ func registerTPCDSVec(r *testRegistry) { clusterConn := c.Conn(ctx, 1) disableAutoStats(t, clusterConn) - disableVectorizeRowCountThresholdHeuristic(t, clusterConn) t.Status("restoring TPCDS dataset for Scale Factor 1") if _, err := clusterConn.Exec( `RESTORE DATABASE tpcds FROM 'gs://cockroach-fixtures/workload/tpcds/scalefactor=1/backup';`, diff --git a/pkg/cmd/roachtest/tpchvec.go b/pkg/cmd/roachtest/tpchvec.go index 525ea84bda39..666a80586890 100644 --- a/pkg/cmd/roachtest/tpchvec.go +++ b/pkg/cmd/roachtest/tpchvec.go @@ -617,7 +617,6 @@ func runTPCHVec( conn := c.Conn(ctx, 1) disableAutoStats(t, conn) - disableVectorizeRowCountThresholdHeuristic(t, conn) t.Status("restoring TPCH dataset for Scale Factor 1") if err := loadTPCHDataset(ctx, t, c, 1 /* sf */, newMonitor(ctx, c), c.All()); err != nil { t.Fatal(err) diff --git a/pkg/settings/registry.go b/pkg/settings/registry.go index a8864ec110c4..145d93a39330 100644 --- a/pkg/settings/registry.go +++ b/pkg/settings/registry.go @@ -77,6 +77,8 @@ var retiredSettings = map[string]struct{}{ "sql.testing.mutations.max_batch_size": {}, "sql.testing.mock_contention.enabled": {}, "kv.atomic_replication_changes.enabled": {}, + // removed as of 21.2. + "sql.defaults.vectorize_row_count_threshold": {}, } // register adds a setting to the registry. diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index fe860099cf2a..b3ac252f92a3 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -283,7 +283,6 @@ go_library( "//pkg/sql/catalog/schemaexpr", "//pkg/sql/catalog/tabledesc", "//pkg/sql/catalog/typedesc", - "//pkg/sql/colexec", "//pkg/sql/colflow", "//pkg/sql/contention", "//pkg/sql/covering", diff --git a/pkg/sql/colexec/constants.go b/pkg/sql/colexec/constants.go index eceb0906b106..d1d501eb3241 100644 --- a/pkg/sql/colexec/constants.go +++ b/pkg/sql/colexec/constants.go @@ -10,12 +10,6 @@ package colexec -// DefaultVectorizeRowCountThreshold denotes the default row count threshold. -// When it is met, the vectorized execution engine will be used if possible. -// TODO(yuzefovich): remove this together with vectorize_row_count_threshold -// setting. -const DefaultVectorizeRowCountThreshold = 0 - // VecMaxOpenFDsLimit specifies the maximum number of open file descriptors // that the vectorized engine can have (globally) for use of the temporary // storage. diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index 1f6b0683fbb7..c2661a23bbc7 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -1323,9 +1323,6 @@ func (dsp *DistSQLPlanner) planTableReaders( tr.Parallelize = info.parallelize p.TotalEstimatedScannedRows += info.estimatedRowCount - if info.estimatedRowCount > p.MaxEstimatedRowCount { - p.MaxEstimatedRowCount = info.estimatedRowCount - } corePlacement[i].NodeID = sp.Node corePlacement[i].Core.TableReader = tr diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index febe3365a987..161a1af227ce 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -126,7 +126,6 @@ func (dsp *DistSQLPlanner) setupFlows( flows map[roachpb.NodeID]*execinfrapb.FlowSpec, recv *DistSQLReceiver, localState distsql.LocalState, - vectorizeThresholdMet bool, collectStats bool, ) (context.Context, flowinfra.Flow, error) { thisNodeID := dsp.gatewayNodeID @@ -155,26 +154,18 @@ func (dsp *DistSQLPlanner) setupFlows( } if vectorizeMode := evalCtx.SessionData.VectorizeMode; vectorizeMode != sessiondatapb.VectorizeOff { - if !vectorizeThresholdMet && vectorizeMode == sessiondatapb.VectorizeOn { - // Vectorization is not justified for this flow because the expected - // amount of data is too small and the overhead of pre-allocating data - // structures needed for the vectorized engine is expected to dominate - // the execution time. - setupReq.EvalContext.SessionData.VectorizeMode = sessiondatapb.VectorizeOff - } else { - // Now we determine whether the vectorized engine supports the flow - // specs. - for _, spec := range flows { - if err := colflow.IsSupported(vectorizeMode, spec); err != nil { - log.VEventf(ctx, 1, "failed to vectorize: %s", err) - if vectorizeMode == sessiondatapb.VectorizeExperimentalAlways { - return nil, nil, err - } - // Vectorization is not supported for this flow, so we override the - // setting. - setupReq.EvalContext.SessionData.VectorizeMode = sessiondatapb.VectorizeOff - break + // Now we determine whether the vectorized engine supports the flow + // specs. + for _, spec := range flows { + if err := colflow.IsSupported(vectorizeMode, spec); err != nil { + log.VEventf(ctx, 1, "failed to vectorize: %s", err) + if vectorizeMode == sessiondatapb.VectorizeExperimentalAlways { + return nil, nil, err } + // Vectorization is not supported for this flow, so we override the + // setting. + setupReq.EvalContext.SessionData.VectorizeMode = sessiondatapb.VectorizeOff + break } } } @@ -325,8 +316,6 @@ func (dsp *DistSQLPlanner) Run( recv.outputTypes = plan.GetResultTypes() recv.contendedQueryMetric = dsp.distSQLSrv.Metrics.ContendedQueriesCount - vectorizedThresholdMet := plan.MaxEstimatedRowCount >= evalCtx.SessionData.VectorizeRowCountThreshold - if len(flows) == 1 { // We ended up planning everything locally, regardless of whether we // intended to distribute or not. @@ -334,7 +323,7 @@ func (dsp *DistSQLPlanner) Run( } ctx, flow, err := dsp.setupFlows( - ctx, evalCtx, leafInputState, flows, recv, localState, vectorizedThresholdMet, planCtx.collectExecStats, + ctx, evalCtx, leafInputState, flows, recv, localState, planCtx.collectExecStats, ) if err != nil { recv.SetError(err) diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index d653373a211a..1017c33130df 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -50,7 +50,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/catalog/hydratedtables" "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" - "github.com/cockroachdb/cockroach/pkg/sql/colexec" "github.com/cockroachdb/cockroach/pkg/sql/contention" "github.com/cockroachdb/cockroach/pkg/sql/distsql" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" @@ -422,22 +421,6 @@ var VectorizeClusterMode = settings.RegisterEnumSetting( }, ) -// VectorizeRowCountThresholdClusterValue controls the cluster default for the -// vectorize row count threshold. When it is met, the vectorized execution -// engine will be used if possible. -var VectorizeRowCountThresholdClusterValue = settings.RegisterIntSetting( - "sql.defaults.vectorize_row_count_threshold", - "default vectorize row count threshold", - colexec.DefaultVectorizeRowCountThreshold, - func(v int64) error { - if v < 0 { - return pgerror.Newf(pgcode.InvalidParameterValue, - "cannot set sql.defaults.vectorize_row_count_threshold to a negative value: %d", v) - } - return nil - }, -) - // DistSQLClusterExecMode controls the cluster default for when DistSQL is used. var DistSQLClusterExecMode = settings.RegisterEnumSetting( "sql.defaults.distsql", @@ -2248,10 +2231,6 @@ func (m *sessionDataMutator) SetVectorize(val sessiondatapb.VectorizeExecMode) { m.data.VectorizeMode = val } -func (m *sessionDataMutator) SetVectorizeRowCountThreshold(val uint64) { - m.data.VectorizeRowCountThreshold = val -} - func (m *sessionDataMutator) SetTestingVectorizeInjectPanics(val bool) { m.data.TestingVectorizeInjectPanics = val } diff --git a/pkg/sql/explain_plan.go b/pkg/sql/explain_plan.go index 640fab634418..59b064a1289c 100644 --- a/pkg/sql/explain_plan.go +++ b/pkg/sql/explain_plan.go @@ -85,12 +85,9 @@ func (e *explainPlanNode) startExec(params runParams) error { flowCtx := newFlowCtxForExplainPurposes(planCtx, params.p, &distSQLPlanner.rpcCtx.ClusterID) ctxSessionData := flowCtx.EvalCtx.SessionData - vectorizedThresholdMet := physicalPlan.MaxEstimatedRowCount >= ctxSessionData.VectorizeRowCountThreshold var willVectorize bool if ctxSessionData.VectorizeMode == sessiondatapb.VectorizeOff { willVectorize = false - } else if !vectorizedThresholdMet && ctxSessionData.VectorizeMode == sessiondatapb.VectorizeOn { - willVectorize = false } else { willVectorize = true for _, flow := range flows { diff --git a/pkg/sql/flowinfra/flow_test.go b/pkg/sql/flowinfra/flow_test.go index 4d3eddd4688d..1aa8a675d859 100644 --- a/pkg/sql/flowinfra/flow_test.go +++ b/pkg/sql/flowinfra/flow_test.go @@ -35,12 +35,8 @@ func BenchmarkFlowSetup(b *testing.B) { defer logScope.Close(b) ctx := context.Background() - st := cluster.MakeTestingClusterSettings() - // Set the threshold to 0 so that we can control which engine is used for - // the query execution via the vectorize mode. - sql.VectorizeRowCountThresholdClusterValue.Override(&st.SV, 0 /* v */) s, conn, _ := serverutils.StartServer(b, base.TestServerArgs{ - Settings: st, + Settings: cluster.MakeTestingClusterSettings(), }) defer s.Stopper().Stop(ctx) diff --git a/pkg/sql/logictest/logic.go b/pkg/sql/logictest/logic.go index e0e09222e3a1..95d4739332df 100644 --- a/pkg/sql/logictest/logic.go +++ b/pkg/sql/logictest/logic.go @@ -1520,14 +1520,6 @@ func (t *logicTest) newCluster(serverArgs TestServerArgs) { } } - // Always override the vectorize row count threshold. This runs all supported - // queries (relative to the mode) through the vectorized execution engine. - if _, err := conn.Exec( - "SET CLUSTER SETTING sql.defaults.vectorize_row_count_threshold = 0", - ); err != nil { - t.Fatal(err) - } - if cfg.overrideAutoStats != "" { if _, err := conn.Exec( "SET CLUSTER SETTING sql.stats.automatic_collection.enabled = $1::bool", cfg.overrideAutoStats, diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index 0d6346112258..72e6e8308a70 100755 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -3668,7 +3668,6 @@ transaction_isolation serializable transaction_priority normal transaction_read_only off transaction_status NoTxn -vectorize_row_count_threshold 0 # information_schema can be used with the anonymous database. # It should show information across all databases. diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index a16e06bfd3a0..0d904f14c85f 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -2158,7 +2158,6 @@ transaction_priority normal NULL transaction_read_only off NULL NULL NULL string transaction_status NoTxn NULL NULL NULL string vectorize on NULL NULL NULL string -vectorize_row_count_threshold 0 NULL NULL NULL string query TTTTTTT colnames SELECT @@ -2238,7 +2237,6 @@ transaction_priority normal NULL transaction_read_only off NULL user NULL off off transaction_status NoTxn NULL user NULL NoTxn NoTxn vectorize on NULL user NULL on on -vectorize_row_count_threshold 0 NULL user NULL 0 0 query TTTTTT colnames SELECT name, source, min_val, max_val, sourcefile, sourceline FROM pg_catalog.pg_settings @@ -2316,7 +2314,6 @@ transaction_priority NULL NULL NULL transaction_read_only NULL NULL NULL NULL NULL transaction_status NULL NULL NULL NULL NULL vectorize NULL NULL NULL NULL NULL -vectorize_row_count_threshold NULL NULL NULL NULL NULL # pg_catalog.pg_sequence diff --git a/pkg/sql/logictest/testdata/logic_test/show_source b/pkg/sql/logictest/testdata/logic_test/show_source index 29284fe41144..96422142a148 100644 --- a/pkg/sql/logictest/testdata/logic_test/show_source +++ b/pkg/sql/logictest/testdata/logic_test/show_source @@ -95,7 +95,6 @@ transaction_priority normal transaction_read_only off transaction_status NoTxn vectorize on -vectorize_row_count_threshold 0 query T colnames SELECT * FROM [SHOW CLUSTER SETTING sql.defaults.distsql] diff --git a/pkg/sql/logictest/testdata/logic_test/system b/pkg/sql/logictest/testdata/logic_test/system index 85321ca55eb8..fbfce8af9d83 100644 --- a/pkg/sql/logictest/testdata/logic_test/system +++ b/pkg/sql/logictest/testdata/logic_test/system @@ -602,7 +602,6 @@ SELECT name, value FROM system.settings WHERE name NOT IN ('version', 'sql.defaults.distsql', 'cluster.secret', 'sql.stats.automatic_collection.enabled', 'sql.defaults.vectorize', - 'sql.defaults.vectorize_row_count_threshold', 'sql.defaults.experimental_distsql_planning') ORDER BY name ---- diff --git a/pkg/sql/logictest/testdata/logic_test/vectorize_threshold b/pkg/sql/logictest/testdata/logic_test/vectorize_threshold deleted file mode 100644 index 79c774ad618c..000000000000 --- a/pkg/sql/logictest/testdata/logic_test/vectorize_threshold +++ /dev/null @@ -1,170 +0,0 @@ -# LogicTest: 5node - -# Disable automatic stats. -statement ok -SET CLUSTER SETTING sql.stats.automatic_collection.enabled = false - -# Check that vectorize row count threshold is respected. - -statement ok -CREATE TABLE small (a INT PRIMARY KEY) - -statement ok -SET vectorize_row_count_threshold = 1000 - -# There are no stats available, so this should run through the row execution -# engine. -query T -EXPLAIN SELECT count(*) from small ----- -distribution: full -vectorized: false -· -• group (scalar) -│ -└── • scan - missing stats - table: small@primary - spans: FULL SCAN - -statement ok -SET vectorize_row_count_threshold = 0 - -# This should run through the vectorized execution engine because we disabled -# the threshold. -query T -EXPLAIN SELECT count(*) from small ----- -distribution: full -vectorized: true -· -• group (scalar) -│ -└── • scan - missing stats - table: small@primary - spans: FULL SCAN - -statement ok -SET vectorize_row_count_threshold = 1000 - -statement ok -ALTER TABLE small INJECT STATISTICS '[ - { - "columns": ["a"], - "created_at": "2018-01-01 1:00:00.00000+00:00", - "row_count": 100, - "distinct_count": 100 - } -]' - -# This should run through the row execution engine. -query T -EXPLAIN SELECT count(*) from small ----- -distribution: full -vectorized: false -· -• group (scalar) -│ estimated row count: 1 -│ -└── • scan - estimated row count: 100 (100% of the table; stats collected ago) - table: small@primary - spans: FULL SCAN - -statement ok -SET vectorize_row_count_threshold = 1 - -# This should run through the vectorized execution engine because we lowered -# the threshold. -query T -EXPLAIN SELECT count(*) from small ----- -distribution: full -vectorized: true -· -• group (scalar) -│ estimated row count: 1 -│ -└── • scan - estimated row count: 100 (100% of the table; stats collected ago) - table: small@primary - spans: FULL SCAN - -statement ok -SET vectorize_row_count_threshold = 1000 - -statement ok -CREATE TABLE large (a INT PRIMARY KEY) - -statement ok -ALTER TABLE large INJECT STATISTICS '[ - { - "columns": ["a"], - "created_at": "2018-01-01 1:00:00.00000+00:00", - "row_count": 100000, - "distinct_count": 100000 - } -]' - -# This should run through the vectorized execution engine. -query T -EXPLAIN SELECT count(*) from large ----- -distribution: full -vectorized: true -· -• group (scalar) -│ estimated row count: 1 -│ -└── • scan - estimated row count: 100,000 (100% of the table; stats collected ago) - table: large@primary - spans: FULL SCAN - -statement ok -SET vectorize_row_count_threshold = 1000000 - -# This should run through the row execution engine because we increased the -# threshold. -query T -EXPLAIN SELECT count(*) from large ----- -distribution: full -vectorized: false -· -• group (scalar) -│ estimated row count: 1 -│ -└── • scan - estimated row count: 100,000 (100% of the table; stats collected ago) - table: large@primary - spans: FULL SCAN - -statement ok -SET vectorize_row_count_threshold = 1000 - -# Check that we estimate the row count correctly when multiple tables are -# scanned. -query T -EXPLAIN SELECT * FROM small INNER MERGE JOIN large ON small.a = large.a ----- -distribution: full -vectorized: true -· -• merge join -│ estimated row count: 100 -│ equality: (a) = (a) -│ left cols are key -│ right cols are key -│ -├── • scan -│ estimated row count: 100 (100% of the table; stats collected ago) -│ table: small@primary -│ spans: FULL SCAN -│ -└── • scan - estimated row count: 100,000 (100% of the table; stats collected ago) - table: large@primary - spans: FULL SCAN diff --git a/pkg/sql/physicalplan/physical_plan.go b/pkg/sql/physicalplan/physical_plan.go index fdef2842c374..7cadc13e9ad9 100644 --- a/pkg/sql/physicalplan/physical_plan.go +++ b/pkg/sql/physicalplan/physical_plan.go @@ -138,11 +138,6 @@ type PhysicalPlan struct { // want to pay this cost if we don't have multiple streams to merge. MergeOrdering execinfrapb.Ordering - // MaxEstimatedRowCount tracks the maximum estimated row count that a table - // reader in this plan will output. This information is used to decide - // whether to use the vectorized execution engine. - // TODO(radu): move this field to PlanInfrastructure. - MaxEstimatedRowCount uint64 // TotalEstimatedScannedRows is the sum of the row count estimate of all the // table readers in the plan. // TODO(radu): move this field to PlanInfrastructure. @@ -943,10 +938,6 @@ func (p *PhysicalPlan) GenerateFlowSpecs() map[roachpb.NodeID]*execinfrapb.FlowS // plans. func (p *PhysicalPlan) SetRowEstimates(left, right *PhysicalPlan) { p.TotalEstimatedScannedRows = left.TotalEstimatedScannedRows + right.TotalEstimatedScannedRows - p.MaxEstimatedRowCount = left.MaxEstimatedRowCount - if right.MaxEstimatedRowCount > p.MaxEstimatedRowCount { - p.MaxEstimatedRowCount = right.MaxEstimatedRowCount - } } // MergePlans is used when merging two plans into a new plan. All plans must diff --git a/pkg/sql/rowexec/processors_test.go b/pkg/sql/rowexec/processors_test.go index c00687960255..4e0e4ca5aec2 100644 --- a/pkg/sql/rowexec/processors_test.go +++ b/pkg/sql/rowexec/processors_test.go @@ -422,7 +422,7 @@ func getPGXConnAndCleanupFunc(t *testing.T, servingSQLAddr string) (*pgx.Conn, f require.NoError(t, err) defaultConn, err := pgx.Connect(pgxConfig) require.NoError(t, err) - _, err = defaultConn.Exec("set distsql='always'; set vectorize_row_count_threshold=0") + _, err = defaultConn.Exec("set distsql='always'") require.NoError(t, err) return defaultConn, cleanup } @@ -561,7 +561,7 @@ func TestDrainingProcessorSwallowsUncertaintyError(t *testing.T) { vectorizeMode = "on" } - if _, err := defaultConn.Exec(fmt.Sprintf("set vectorize='%s'; set vectorize_row_count_threshold=0", vectorizeMode)); err != nil { + if _, err := defaultConn.Exec(fmt.Sprintf("set vectorize='%s'", vectorizeMode)); err != nil { t.Fatal(err) } diff --git a/pkg/sql/sessiondata/session_data.go b/pkg/sql/sessiondata/session_data.go index 8ca5d550abf9..eb0a48282679 100644 --- a/pkg/sql/sessiondata/session_data.go +++ b/pkg/sql/sessiondata/session_data.go @@ -116,9 +116,6 @@ type LocalOnlySessionData struct { SaveTablesPrefix string // RemoteAddr is used to generate logging events. RemoteAddr net.Addr - // VectorizeRowCountThreshold indicates the row count above which the - // vectorized execution engine will be used if possible. - VectorizeRowCountThreshold uint64 // ExperimentalDistSQLPlanningMode indicates whether the experimental // DistSQL planning driven by the optimizer is enabled. ExperimentalDistSQLPlanningMode ExperimentalDistSQLPlanningMode diff --git a/pkg/sql/trace_test.go b/pkg/sql/trace_test.go index 564db4e9ef26..ab61007bf848 100644 --- a/pkg/sql/trace_test.go +++ b/pkg/sql/trace_test.go @@ -226,7 +226,7 @@ func TestTrace(t *testing.T) { if _, err := sqlDB.Exec("SET distsql = off"); err != nil { t.Fatal(err) } - if _, err := sqlDB.Exec("SET vectorize = on; SET vectorize_row_count_threshold=0"); err != nil { + if _, err := sqlDB.Exec("SET vectorize = on"); err != nil { t.Fatal(err) } if _, err := sqlDB.Exec("SET tracing = on; SELECT * FROM test.foo; SET tracing = off"); err != nil { diff --git a/pkg/sql/vars.go b/pkg/sql/vars.go index 79c08766d35b..413002f756fd 100644 --- a/pkg/sql/vars.go +++ b/pkg/sql/vars.go @@ -532,29 +532,6 @@ var varGen = map[string]sessionVar{ }, }, - // CockroachDB extension. - `vectorize_row_count_threshold`: { - GetStringVal: makeIntGetStringValFn(`vectorize_row_count_threshold`), - Set: func(_ context.Context, m *sessionDataMutator, s string) error { - b, err := strconv.ParseInt(s, 10, 64) - if err != nil { - return err - } - if b < 0 { - return pgerror.Newf(pgcode.InvalidParameterValue, - "cannot set vectorize_row_count_threshold to a negative value: %d", b) - } - m.SetVectorizeRowCountThreshold(uint64(b)) - return nil - }, - Get: func(evalCtx *extendedEvalContext) string { - return strconv.FormatInt(int64(evalCtx.SessionData.VectorizeRowCountThreshold), 10) - }, - GlobalDefault: func(sv *settings.Values) string { - return strconv.FormatInt(VectorizeRowCountThresholdClusterValue.Get(sv), 10) - }, - }, - // CockroachDB extension. `testing_vectorize_inject_panics`: { GetStringVal: makePostgresBoolGetStringValFn(`testing_vectorize_inject_panics`),