Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
39439: sql: optimizer always on r=RaduBerinde a=RaduBerinde

#### sql, workload: don't disable the optimizer in tests

In preparation for removing the possibility of turning off the
optimizer, adjust tests and benchmarks that are disabling the
optimizer.

Release note: None

#### sql: optimizer always on

This change deprecates the `optimizer` session setting and associated
cluster setting. It is no longer possible to disable the optimizer.
The settings still exist but the only allowed value is "on" (mostly so
that `SET OPTIMIZER = ON` still works).

The logic test configurations are reworked: they now all use the
optimizer, and the `-opt` versions are gone (e.g. what used to be
`local-opt` is now `local`). The HP-specific planner tests are
removed.

Release note: None


39463: roachtest: Bump min version for direct ingest import roachtest r=adityamaru27 a=adityamaru27

Bumping the roachtest version to 19.2.0 to ensure it runs on
master branch.

Closes: #39395

Release note: None

39467: roachtest: add backward compat to replicate/wide r=solongordon a=solongordon

In 6c24930, I updated the way replication zones are named, which
affected the replicate/wide roachtest. However I neglected to add
backward compatibility for testing against previous releases. This
commit restores the old logic for that scenario.

Release note: None

Co-authored-by: Radu Berinde <[email protected]>
Co-authored-by: Aditya Maru <[email protected]>
Co-authored-by: Solon Gordon <[email protected]>
  • Loading branch information
4 people committed Aug 8, 2019
4 parents 311f7e4 + 57f7ad3 + 442a934 + 4c30451 commit f1ef7e6
Show file tree
Hide file tree
Showing 208 changed files with 307 additions and 14,750 deletions.
16 changes: 6 additions & 10 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -51,10 +51,10 @@ TESTS := .
## Benchmarks to run for use with `make bench`.
BENCHES :=

## Space delimited list of logic test files to run, for make testlogic/testccllogic/testoptlogic/testplannerlogic.
## Space delimited list of logic test files to run, for make testlogic/testccllogic/testoptlogic.
FILES :=

## Name of a logic test configuration to run, for make testlogic/testccllogic/testoptlogic/testplannerlogic.
## Name of a logic test configuration to run, for make testlogic/testccllogic/testoptlogic.
## (default: all configs. It's not possible yet to specify multiple configs in this way.)
TESTCONFIG :=

Expand Down Expand Up @@ -947,7 +947,7 @@ bench benchshort: TESTTIMEOUT := $(BENCHTIMEOUT)
# that longer running benchmarks can skip themselves.
benchshort: override TESTFLAGS += -benchtime=1ns -short

.PHONY: check test testshort testrace testlogic testbaselogic testplannerlogic testccllogic testoptlogic bench benchshort
.PHONY: check test testshort testrace testlogic testbaselogic testccllogic testoptlogic bench benchshort
test: ## Run tests.
check test testshort testrace bench benchshort:
$(xgo) test $(GOFLAGS) -tags '$(TAGS)' -ldflags '$(LINKFLAGS)' -run "$(TESTS)" $(if $(BENCHES),-bench "$(BENCHES)") -timeout $(TESTTIMEOUT) $(PKG) $(TESTFLAGS)
Expand All @@ -970,14 +970,11 @@ roachprod-stress roachprod-stressrace: bin/roachprod-stress
-test.run "$(TESTS)" $(filter-out -v,$(TESTFLAGS)) -test.v -test.timeout $(TESTTIMEOUT); \
fi

testlogic: testbaselogic testplannerlogic testoptlogic testccllogic
testlogic: testbaselogic testoptlogic testccllogic

testbaselogic: ## Run SQL Logic Tests.
testbaselogic: bin/logictest

testplannerlogic: ## Run SQL Logic Tests for the heuristic planner.
testplannerlogic: bin/logictest

testccllogic: ## Run SQL CCL Logic Tests.
testccllogic: bin/logictestccl

Expand All @@ -987,7 +984,6 @@ testoptlogic: bin/logictestopt
logic-test-selector := $(if $(TESTCONFIG),^$(TESTCONFIG)$$)/$(if $(FILES),^$(subst $(space),$$|^,$(FILES))$$)/$(SUBTESTS)
testbaselogic: TESTS := TestLogic/$(logic-test-selector)
testccllogic: TESTS := TestCCLLogic/$(logic-test-selector)
testplannerlogic: TESTS := TestPlannerLogic/$(logic-test-selector)
testoptlogic: TESTS := TestExecBuild/$(logic-test-selector)

# Note: we specify -config here in addition to the filter on TESTS
Expand All @@ -996,8 +992,8 @@ testoptlogic: TESTS := TestExecBuild/$(logic-test-selector)
# does not prevent loading and initializing every default config in
# turn (including setting up the test clusters, etc.). By specifying
# -config, the extra initialization overhead is averted.
testbaselogic testccllogic testplannerlogic testoptlogic: TESTFLAGS := -test.v $(if $(FILES),-show-sql) $(if $(TESTCONFIG),-config $(TESTCONFIG))
testbaselogic testccllogic testplannerlogic testoptlogic:
testbaselogic testccllogic testoptlogic: TESTFLAGS := -test.v $(if $(FILES),-show-sql) $(if $(TESTCONFIG),-config $(TESTCONFIG))
testbaselogic testccllogic testoptlogic:
cd $($(<F)-package) && $(<F) -test.run "$(TESTS)" -test.timeout $(TESTTIMEOUT) $(TESTFLAGS)

testraceslow: override GOFLAGS += -race
Expand Down
2 changes: 1 addition & 1 deletion build/teamcity-sqllogictest.sh
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,6 @@ build/builder.sh \
# numeric typing differences.
build/builder.sh \
stdbuf -oL -eL \
make test TESTFLAGS="-v -bigtest -config local-opt,fakedist-opt -flex-types" TESTTIMEOUT='24h' PKG='./pkg/sql/logictest' TESTS='^TestSqlLiteCorrelatedLogic$$' 2>&1 \
make test TESTFLAGS="-v -bigtest -config local,fakedist -flex-types" TESTTIMEOUT='24h' PKG='./pkg/sql/logictest' TESTS='^TestSqlLiteCorrelatedLogic$$' 2>&1 \
| tee "artifacts/sqllite-correlated.log" \
| go-test-teamcity
4 changes: 2 additions & 2 deletions build/teamcity-testlogicrace.sh
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ build/builder.sh \
stdbuf -oL -eL \
make testrace \
PKG=./pkg/sql/logictest \
TESTS='^TestLogic/local-opt$$' \
TESTS='^TestLogic/local$$' \
TESTFLAGS='-optimizer-cost-perturbation=0.9 -v' \
ENABLE_ROCKSDB_ASSERTIONS=1 \
2>&1 \
Expand All @@ -48,7 +48,7 @@ for file in $LOGICTESTS; do
stdbuf -oL -eL \
make testrace \
PKG=./pkg/sql/logictest \
TESTS='^TestLogic/local-opt/'${file}'$$' \
TESTS='^TestLogic/local/'${file}'$$' \
TESTFLAGS='-disable-opt-rule-probability=0.5 -v' \
ENABLE_ROCKSDB_ASSERTIONS=1 \
2>&1 \
Expand Down
1 change: 0 additions & 1 deletion docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -84,7 +84,6 @@
<tr><td><code>server.web_session_timeout</code></td><td>duration</td><td><code>168h0m0s</code></td><td>the duration that a newly created web session will be valid</td></tr>
<tr><td><code>sql.defaults.default_int_size</code></td><td>integer</td><td><code>8</code></td><td>the size, in bytes, of an INT type</td></tr>
<tr><td><code>sql.defaults.distsql</code></td><td>enumeration</td><td><code>auto</code></td><td>default distributed SQL execution mode [off = 0, auto = 1, on = 2]</td></tr>
<tr><td><code>sql.defaults.optimizer</code></td><td>enumeration</td><td><code>on</code></td><td>default cost-based optimizer mode [off = 0, on = 1, local = 2]</td></tr>
<tr><td><code>sql.defaults.reorder_joins_limit</code></td><td>integer</td><td><code>4</code></td><td>default number of joins to reorder</td></tr>
<tr><td><code>sql.defaults.results_buffer.size</code></td><td>byte size</td><td><code>16 KiB</code></td><td>default size of the buffer that accumulates results for a statement or a batch of statements before they are sent to the client. This can be overridden on an individual connection with the 'results_buffer_size' parameter. Note that auto-retries generally only happen while no results have been delivered to the client, so reducing this size can increase the number of retriable errors a client receives. On the other hand, increasing the buffer size can increase the delay until the client receives the first result row. Updating the setting only affects new connections. Setting to 0 disables any buffering.</td></tr>
<tr><td><code>sql.defaults.serial_normalization</code></td><td>enumeration</td><td><code>rowid</code></td><td>default handling of SERIAL in table definitions [rowid = 0, virtual_sequence = 1, sql_sequence = 2]</td></tr>
Expand Down
21 changes: 0 additions & 21 deletions pkg/bench/foreachdb.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,26 +37,6 @@ func benchmarkCockroach(b *testing.B, f func(b *testing.B, db *gosql.DB)) {
b.Fatal(err)
}

if _, err := db.Exec(`SET OPTIMIZER=OFF`); err != nil {
b.Fatal(err)
}

f(b, db)
}

func benchmarkCockroachOpt(b *testing.B, f func(b *testing.B, db *gosql.DB)) {
s, db, _ := serverutils.StartServer(
b, base.TestServerArgs{UseDatabase: "bench"})
defer s.Stopper().Stop(context.TODO())

if _, err := db.Exec(`CREATE DATABASE bench`); err != nil {
b.Fatal(err)
}

if _, err := db.Exec(`SET OPTIMIZER=ON`); err != nil {
b.Fatal(err)
}

f(b, db)
}

Expand Down Expand Up @@ -150,7 +130,6 @@ func benchmarkMySQL(b *testing.B, f func(b *testing.B, db *gosql.DB)) {
func ForEachDB(b *testing.B, fn func(*testing.B, *gosql.DB)) {
for _, dbFn := range []func(*testing.B, func(*testing.B, *gosql.DB)){
benchmarkCockroach,
benchmarkCockroachOpt,
benchmarkMultinodeCockroach,
benchmarkPostgres,
benchmarkMySQL,
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/logictestccl/testdata/logic_test/as_of
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
# LogicTest: local-opt
# LogicTest: local

statement ok
CREATE TABLE t (i INT)
Expand Down
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
# LogicTest: local-opt
# LogicTest: local

statement ok
CREATE TABLE p (a INT PRIMARY KEY) PARTITION BY LIST (a) (
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/logictestccl/testdata/logic_test/crdb_internal
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
# LogicTest: local-opt
# LogicTest: local

query IITTITTT colnames
SELECT * FROM crdb_internal.partitions
Expand Down
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
# LogicTest: 5node-dist-opt
# LogicTest: 5node-dist

# Tests for the show partitions command.

Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/logictestccl/testdata/logic_test/drop_index
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
# LogicTest: 5node-dist-opt
# LogicTest: 5node-dist

statement ok
CREATE TABLE t (
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/logictestccl/testdata/logic_test/partitioning
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
# LogicTest: local-opt
# LogicTest: local

statement error syntax
CREATE TABLE t (a INT, b INT, c INT, PRIMARY KEY (a, b)) PARTITION BY LIST ()
Expand Down
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
# LogicTest: local-opt fakedist-opt
# LogicTest: local fakedist

# Simple partitioning example.
statement ok
Expand Down
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
# LogicTest: local-opt
# LogicTest: local

statement ok
CREATE TABLE ok1 (
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/logictestccl/testdata/logic_test/restore
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
# LogicTest: local-opt
# LogicTest: local

# Check that we get through parsing and license check.
statement error pq: failed to read backup descriptor: unsupported storage scheme: ""
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/logictestccl/testdata/logic_test/role
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
# LogicTest: local-opt
# LogicTest: local

query T colnames
SHOW ROLES
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/logictestccl/testdata/logic_test/zone
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
# LogicTest: 5node-dist-opt
# LogicTest: 5node-dist

# Ensure that cost-based-optimizer uses an index with zone constraints that most
# closely matches the gateway's locality. Use "retry" option, since it can take
Expand Down
21 changes: 21 additions & 0 deletions pkg/cmd/roachtest/allocator.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ import (
gosql "database/sql"
"fmt"
"math"
"strings"
"time"

"github.com/cockroachdb/cockroach/pkg/util/timeutil"
Expand Down Expand Up @@ -259,7 +260,14 @@ func runWideReplication(ctx context.Context, t *test, c *cluster) {
defer db.Close()

zones := func() []string {
oldVersion := false
rows, err := db.Query(`SELECT target FROM crdb_internal.zones`)
// TODO(solon): Remove this block once we are no longer running roachtest
// against version 19.1 and earlier.
if err != nil && strings.Contains(err.Error(), `column "target" does not exist`) {
oldVersion = true
rows, err = db.Query(`SELECT zone_name FROM crdb_internal.zones`)
}
if err != nil {
t.Fatal(err)
}
Expand All @@ -270,6 +278,19 @@ func runWideReplication(ctx context.Context, t *test, c *cluster) {
if err := rows.Scan(&name); err != nil {
t.Fatal(err)
}
// TODO(solon): Remove this block once we are no longer running roachtest
// against version 19.1 and earlier.
if oldVersion {
which := "RANGE"
if name[0] == '.' {
name = name[1:]
} else if strings.Count(name, ".") == 0 {
which = "DATABASE"
} else {
which = "TABLE"
}
name = fmt.Sprintf("%s %s", which, name)
}
results = append(results, name)
}
return results
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/import.go
Original file line number Diff line number Diff line change
Expand Up @@ -97,7 +97,7 @@ func registerImportTPCC(r *testRegistry) {
})
r.Add(testSpec{
Name: `import/experimental-direct-ingestion`,
MinVersion: `v19.1.0`,
MinVersion: `v19.2.0`,
Cluster: makeClusterSpec(3, cpu(16)),
Timeout: 2 * time.Hour,
Run: func(ctx context.Context, t *test, c *cluster) {
Expand Down
1 change: 1 addition & 0 deletions pkg/settings/registry.go
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@ var retiredSettings = map[string]struct{}{
"server.heap_profile.system_memory_threshold_fraction": {},
"timeseries.storage.10s_resolution_ttl": {},
"changefeed.push.enabled": {},
"sql.defaults.optimizer": {},
}

// Register adds a setting to the registry.
Expand Down
38 changes: 12 additions & 26 deletions pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
Expand Down Expand Up @@ -659,13 +658,9 @@ func (ex *connExecutor) dispatchToExecutionEngine(

ex.sessionTracing.TracePlanCheckStart(ctx)
distributePlan := false
// If we use the optimizer and we are in "local" mode, don't try to
// distribute.
if ex.sessionData.OptimizerMode != sessiondata.OptimizerLocal {
planner.prepareForDistSQLSupportCheck()
distributePlan = shouldDistributePlan(
ctx, ex.sessionData.DistSQLMode, ex.server.cfg.DistSQLPlanner, planner.curPlan.plan)
}
planner.prepareForDistSQLSupportCheck()
distributePlan = shouldDistributePlan(
ctx, ex.sessionData.DistSQLMode, ex.server.cfg.DistSQLPlanner, planner.curPlan.plan)
ex.sessionTracing.TracePlanCheckEnd(ctx, nil, distributePlan)

if ex.server.cfg.TestingKnobs.BeforeExecute != nil {
Expand Down Expand Up @@ -727,25 +722,16 @@ func (ex *connExecutor) makeExecPlan(ctx context.Context, planner *planner) erro
// in error cases.
planner.curPlan = planTop{AST: stmt.AST}

if optMode := ex.sessionData.OptimizerMode; optMode != sessiondata.OptimizerOff {
log.VEvent(ctx, 2, "generating optimizer plan")
var result *planTop
var err error
result, err = planner.makeOptimizerPlan(ctx)
if err != nil {
log.VEventf(ctx, 1, "optimizer plan failed: %v", err)
return err
}
planner.curPlan = *result
return nil
log.VEvent(ctx, 2, "generating optimizer plan")
var result *planTop
var err error
result, err = planner.makeOptimizerPlan(ctx)
if err != nil {
log.VEventf(ctx, 1, "optimizer plan failed: %v", err)
return err
}

log.VEvent(ctx, 2, "optimizer disabled")
// Use the heuristic planner.
optFlags := planner.curPlan.flags
err := planner.makePlan(ctx)
planner.curPlan.flags |= optFlags
return err
planner.curPlan = *result
return nil
}

// saveLogicalPlanDescription returns whether we should save this as a sample logical plan
Expand Down
51 changes: 5 additions & 46 deletions pkg/sql/conn_executor_prepare.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/util/fsm"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -182,8 +181,6 @@ func (ex *connExecutor) populatePrepared(
if err := p.semaCtx.Placeholders.Init(stmt.NumPlaceholders, placeholderHints); err != nil {
return 0, err
}
prepared := stmt.Prepared

p.extendedEvalCtx.PrepareOnly = true

protoTS, err := p.isAsOf(stmt.AST)
Expand All @@ -203,52 +200,14 @@ func (ex *connExecutor) populatePrepared(
// As of right now, the optimizer only works on SELECT statements and will
// fallback for all others, so this should be safe for the foreseeable
// future.
var flags planFlags
if optMode := ex.sessionData.OptimizerMode; optMode != sessiondata.OptimizerOff {
log.VEvent(ctx, 2, "preparing using optimizer")
var err error
flags, err = p.prepareUsingOptimizer(ctx)
if err == nil {
log.VEvent(ctx, 2, "optimizer prepare succeeded")
// stmt.Prepared fields have been populated.
return flags, nil
}
flags, err := p.prepareUsingOptimizer(ctx)
if err != nil {
log.VEventf(ctx, 1, "optimizer prepare failed: %v", err)
return 0, err
}
log.VEvent(ctx, 2, "optimizer disabled (prepare)")

// Fallback on the heuristic planner if the optimizer was not enabled: create
// a plan for the statement to figure out the typing, then close the plan.
prepared.AnonymizedStr = anonymizeStmt(stmt.AST)
if err := p.prepare(ctx, stmt.AST); err != nil {
return 0, err
}

if p.curPlan.plan == nil {
// Statement with no result columns and no support for placeholders.
//
// Note: we're combining `flags` which comes from
// `prepareUsingOptimizer`, with `p.curPlan.flags` which ensures
// the new flags combine with the existing flags (this is used
// e.g. to maintain the count of times the optimizer was used).
return flags | p.curPlan.flags, nil
}
defer p.curPlan.close(ctx)

prepared.Columns = p.curPlan.columns()
for _, c := range prepared.Columns {
if err := checkResultType(c.Typ); err != nil {
return 0, err
}
}
// Verify that all placeholder types have been set.
if err := p.semaCtx.Placeholders.Types.AssertAllSet(); err != nil {
return 0, err
}
prepared.Types = p.semaCtx.Placeholders.Types
// The flags are combined, see the comment above for why.
return flags | p.curPlan.flags, nil
log.VEvent(ctx, 2, "optimizer prepare succeeded")
// stmt.Prepared fields have been populated.
return flags, nil
}

func (ex *connExecutor) execBind(
Expand Down
Loading

0 comments on commit f1ef7e6

Please sign in to comment.