Skip to content

Commit

Permalink
Merge #62164
Browse files Browse the repository at this point in the history
62164: sql: remove vectorize_row_count_threshold setting r=yuzefovich a=yuzefovich

This commit removes `vectorize_row_count_threshold` session variable and
the corresponding cluster setting. In 21.1 release these are set to 0 by
default, and in 21.2 release that default behavior cannot be changed.
This marks another step towards the start of deprecating the row-by-row
execution engine.

Fixes: #53893.

Release note (sql change): `sql.defaults.vectorize_row_count_threshold`
cluster setting as well as corresponding `vectorize_row_count_threshold`
session variable have been removed. From now on, CockroachDB will behave
exactly as if these were set to 0 (last default value).

Co-authored-by: Yahor Yuzefovich <yahor@cockroachlabs.com>
  • Loading branch information
craig[bot] and yuzefovich committed Mar 17, 2021
2 parents 054169b + 124e401 commit 56842a8
Show file tree
Hide file tree
Showing 24 changed files with 18 additions and 300 deletions.
1 change: 0 additions & 1 deletion pkg/ccl/changefeedccl/helpers_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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))
Expand Down
1 change: 0 additions & 1 deletion pkg/ccl/importccl/exportcsv_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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`)
}

Expand Down
12 changes: 0 additions & 12 deletions pkg/cmd/roachtest/tpc_utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
}
1 change: 0 additions & 1 deletion pkg/cmd/roachtest/tpcdsvec.go
Original file line number Diff line number Diff line change
Expand Up @@ -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';`,
Expand Down
1 change: 0 additions & 1 deletion pkg/cmd/roachtest/tpchvec.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
2 changes: 2 additions & 0 deletions pkg/settings/registry.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down
1 change: 0 additions & 1 deletion pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
6 changes: 0 additions & 6 deletions pkg/sql/colexec/constants.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down
3 changes: 0 additions & 3 deletions pkg/sql/distsql_physical_planner.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
35 changes: 12 additions & 23 deletions pkg/sql/distsql_running.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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
}
}
}
Expand Down Expand Up @@ -325,16 +316,14 @@ 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.
localState.IsLocal = true
}

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)
Expand Down
21 changes: 0 additions & 21 deletions pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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",
Expand Down Expand Up @@ -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
}
Expand Down
3 changes: 0 additions & 3 deletions pkg/sql/explain_plan.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
6 changes: 1 addition & 5 deletions pkg/sql/flowinfra/flow_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)

Expand Down
8 changes: 0 additions & 8 deletions pkg/sql/logictest/logic.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down
1 change: 0 additions & 1 deletion pkg/sql/logictest/testdata/logic_test/information_schema
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down
3 changes: 0 additions & 3 deletions pkg/sql/logictest/testdata/logic_test/pg_catalog
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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

Expand Down
1 change: 0 additions & 1 deletion pkg/sql/logictest/testdata/logic_test/show_source
Original file line number Diff line number Diff line change
Expand Up @@ -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]
Expand Down
1 change: 0 additions & 1 deletion pkg/sql/logictest/testdata/logic_test/system
Original file line number Diff line number Diff line change
Expand Up @@ -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
----
Expand Down
Loading

0 comments on commit 56842a8

Please sign in to comment.