Skip to content

Commit

Permalink
Merge #55713
Browse files Browse the repository at this point in the history
55713: sql: decrease vectorize_row_count_threshold to 0 r=yuzefovich a=yuzefovich

**colexec: add context to errors from builtin functions**

This commit wraps the errors that occur during builtin functions
evaluations to provide more context.

Release note: None

**sql: decrease vectorize_row_count_threshold to 0**

This commit decreases the default value for
`vectorize_row_count_threshold` setting to 0 which means that we will be
using the vectorized engine for all supported queries. We intend to
remove that setting entirely in 21.1 release, but for now we choose the
option of effectively disabling it, just in case.

The benchmarks have shown the following:
- -1.5% on KV95
- similar performance on TPCC
- -3% on movr
- -10% on miscellaneous operations (joins, aggregations) on small tables.

We think that such gap is small enough to merge this change, and we
intend to optimize the vectorized engine more before making the final
call for the default value for the 21.1 release.

Additionally, this commit collects the trace metadata on the outboxes.

Informs: #53893.

Release note (sql change): The default value for
`vectorize_row_count_threshold` setting has been decreased from 1000 to
0 meaning that from now on we will always use the vectorized engine for
all supported queries regardless of the row estimate (unless
`vectorize=off` is set).

Co-authored-by: Yahor Yuzefovich <yahor@cockroachlabs.com>
  • Loading branch information
craig[bot] and yuzefovich committed Dec 3, 2020
2 parents 58e9eb7 + 6203345 commit 05c825b
Show file tree
Hide file tree
Showing 11 changed files with 54 additions and 42 deletions.
13 changes: 8 additions & 5 deletions pkg/cli/cli_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -530,8 +530,12 @@ func Example_sql() {
c.RunWithArgs([]string{`sql`, `--set=errexit=0`, `-e`, `select nonexistent`, `-e`, `select 123 as "123"`})
c.RunWithArgs([]string{`sql`, `--set`, `echo=true`, `-e`, `select 123 as "123"`})
c.RunWithArgs([]string{`sql`, `--set`, `unknownoption`, `-e`, `select 123 as "123"`})
// Check that partial results + error get reported together.
c.RunWithArgs([]string{`sql`, `-e`, `select 1/(@1-3) from generate_series(1,4)`})
// Check that partial results + error get reported together. The query will
// run via the vectorized execution engine which operates on the batches of
// growing capacity starting at 1 (the batch sizes will be 1, 2, 4, ...),
// and with the query below the division by zero error will occur after the
// first batch consisting of 1 row has been returned to the client.
c.RunWithArgs([]string{`sql`, `-e`, `select 1/(@1-2) from generate_series(1,3)`})

// Output:
// sql -e show application_name
Expand Down Expand Up @@ -589,9 +593,8 @@ func Example_sql() {
// sql --set unknownoption -e select 123 as "123"
// invalid syntax: \set unknownoption. Try \? for help.
// ERROR: invalid syntax
// sql -e select 1/(@1-3) from generate_series(1,4)
// sql -e select 1/(@1-2) from generate_series(1,3)
// ?column?
// -0.5
// -1
// (error encountered after some results were delivered)
// ERROR: division by zero
Expand Down Expand Up @@ -1371,7 +1374,7 @@ func Example_misc_table() {
// info
// --------------------------
// distribution: full
// vectorized: false
// vectorized: true
//
// • render
// │
Expand Down
7 changes: 0 additions & 7 deletions pkg/cmd/smithcmp/tpch.toml
Original file line number Diff line number Diff line change
Expand Up @@ -513,13 +513,6 @@ initsql = """
set vectorize=off;
"""

[databases.vec-201auto]
addr = "postgresql://root@localhost:26257/tpch?sslmode=disable"
allowmutations = true
initsql = """
set vectorize=201auto;
"""

[databases.vec-on]
addr = "postgresql://root@localhost:26257/tpch?sslmode=disable"
allowmutations = true
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/colexec/builtin_funcs.go
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@ func (b *defaultBuiltinFuncOperator) Next(ctx context.Context) coldata.Batch {
} else {
res, err = b.funcExpr.ResolvedOverload().Fn(b.evalCtx, b.row)
if err != nil {
colexecerror.ExpectedError(err)
colexecerror.ExpectedError(b.funcExpr.MaybeWrapError(err))
}
}

Expand Down
9 changes: 3 additions & 6 deletions pkg/sql/colexec/constants.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,12 +12,9 @@ package colexec

// DefaultVectorizeRowCountThreshold denotes the default row count threshold.
// When it is met, the vectorized execution engine will be used if possible.
// The current number 1000 was chosen upon comparing `SELECT count(*) FROM t`
// query running through the row and the vectorized execution engines on a
// single node with tables having different number of columns.
// Note: if you are updating this field, please make sure to update
// vectorize_threshold logic test accordingly.
const DefaultVectorizeRowCountThreshold = 1000
// 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
Expand Down
9 changes: 9 additions & 0 deletions pkg/sql/colflow/colrpc/outbox.go
Original file line number Diff line number Diff line change
Expand Up @@ -275,6 +275,15 @@ func (o *Outbox) sendMetadata(ctx context.Context, stream flowStreamClient, errT
msg.Data.Metadata, execinfrapb.LocalMetaToRemoteProducerMeta(ctx, execinfrapb.ProducerMetadata{Err: errToSend}),
)
}
if trace := execinfra.GetTraceData(ctx); trace != nil {
msg.Data.Metadata = append(msg.Data.Metadata, execinfrapb.RemoteProducerMetadata{
Value: &execinfrapb.RemoteProducerMetadata_TraceData_{
TraceData: &execinfrapb.RemoteProducerMetadata_TraceData{
CollectedSpans: trace,
},
},
})
}
for _, src := range o.metadataSources {
for _, meta := range src.DrainMeta(ctx) {
msg.Data.Metadata = append(msg.Data.Metadata, execinfrapb.LocalMetaToRemoteProducerMeta(ctx, meta))
Expand Down
6 changes: 6 additions & 0 deletions pkg/sql/conn_executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -544,6 +544,12 @@ func TestQueryProgress(t *testing.T) {

db := sqlutils.MakeSQLRunner(rawDB)

// TODO(yuzefovich): the vectorized cfetcher doesn't emit metadata about
// the progress nor do we have an infrastructure to emit such metadata at
// the runtime (we can only propagate the metadata during the draining of
// the flow which defeats the purpose of the progress meta), so we use the
// old row-by-row engine in this test. We should fix that (#55758).
db.Exec(t, `SET vectorize=off`)
db.Exec(t, `SET CLUSTER SETTING sql.stats.automatic_collection.enabled = false`)
db.Exec(t, `CREATE DATABASE t; CREATE TABLE t.test (x INT PRIMARY KEY);`)
db.Exec(t, `INSERT INTO t.test SELECT generate_series(1, $1)::INT`, rows)
Expand Down
5 changes: 1 addition & 4 deletions pkg/sql/logictest/testdata/logic_test/vectorize_local
Original file line number Diff line number Diff line change
Expand Up @@ -35,9 +35,6 @@ SET vectorize = on
statement ok
SET distsql = on

statement ok
SET vectorize_row_count_threshold = 0

query T
SELECT url FROM [EXPLAIN ANALYZE (DISTSQL) SELECT a FROM a]
----
Expand All @@ -54,7 +51,7 @@ SELECT url FROM [EXPLAIN ANALYZE (DISTSQL) SELECT c.a FROM c INNER MERGE JOIN d
https://cockroachdb.github.io/distsqlplan/decode.html#eJzMUl1v0zAUfedXXN2nTZgtySQeLE1qgYAy2mSkFQKmPLj2pYtI7GA7Uquq_x0lKR8pY2jwsrfc82HnHN8duq8VclzEs_jlElpbwes8m8NN_OF6Nk1SmKbT2cdPMZy8ShbLxbvZKRyk8kwMUglJmsY5zOP8TQxXWZKCgiztBZegzlYFMtRGUSpqcshvMMSCYWONJOeM7aBdL0jUBnnAsNRN6zu4YCiNJeQ79KWvCDkuxaqinIQiex4gQ0VelFV_rJw0tqyF3SLDRSO04_AMGWat5zAJkeHb9-DLmjgEbpik0Z60L40-InzbVOTAklAcogFbbf0PKHwOL5DhSnh5Sw5M65vulu6HDtbvUITFnuEwHSI5L9aEPNyzf4sdjmOr-2JHjy529MfYP9O22lhFltQoadE5_ya5o7s52TVdmVKTPY_G3VX02Z9Mwqenl7Zc3w6fo5WhDcn2uKlabKCm2tgtiKoyUnhSHIK-m45z0nYNgSrdl98V_9XexUOWJifXGO3ouMU7Tw666kitaXgKZ1or6doa2V8zjFnv6wFFzg9sNAyJ7ql-q381hw8wR8fm6F7zxcgc7Iv9k28BAAD__-x5iuI=

statement ok
RESET vectorize; RESET distsql; RESET vectorize_row_count_threshold
RESET vectorize; RESET distsql

statement ok
SET tracing=off
Expand Down
5 changes: 2 additions & 3 deletions pkg/sql/logictest/testdata/logic_test/vectorize_threshold
Original file line number Diff line number Diff line change
Expand Up @@ -4,9 +4,8 @@
statement ok
SET CLUSTER SETTING sql.stats.automatic_collection.enabled = false

# Check that vectorize row count threshold is respected. The test relies on the
# fact that DistSQL and vectorized execution engines output execution stats in
# a different format.
# Check that vectorize row count threshold is respected.

statement ok
CREATE TABLE small (a INT PRIMARY KEY)

Expand Down
34 changes: 19 additions & 15 deletions pkg/sql/sem/tree/eval.go
Original file line number Diff line number Diff line change
Expand Up @@ -3913,6 +3913,24 @@ func (expr *FuncExpr) evalArgs(ctx *EvalContext) (bool, Datums, error) {
return false, args, nil
}

// MaybeWrapError updates non-nil error depending on the FuncExpr to provide
// more context.
func (expr *FuncExpr) MaybeWrapError(err error) error {
// If we are facing an explicit error, propagate it unchanged.
fName := expr.Func.String()
if fName == `crdb_internal.force_error` {
return err
}
// Otherwise, wrap it with context.
newErr := errors.Wrapf(err, "%s()", errors.Safe(fName))
// Count function errors as it flows out of the system. We need to handle
// them this way because if we are facing a retry error, in particular those
// generated by crdb_internal.force_retry(), Wrap() will propagate it as a
// non-pgerror error (so that the executor can see it with the right type).
newErr = errors.WithTelemetry(newErr, fName+"()")
return newErr
}

// Eval implements the TypedExpr interface.
func (expr *FuncExpr) Eval(ctx *EvalContext) (Datum, error) {
nullResult, args, err := expr.evalArgs(ctx)
Expand All @@ -3925,21 +3943,7 @@ func (expr *FuncExpr) Eval(ctx *EvalContext) (Datum, error) {

res, err := expr.fn.Fn(ctx, args)
if err != nil {
// If we are facing an explicit error, propagate it unchanged.
fName := expr.Func.String()
if fName == `crdb_internal.force_error` {
return nil, err
}
// Otherwise, wrap it with context.
newErr := errors.Wrapf(err, "%s()", errors.Safe(fName))
// Count function errors as it flows out of the system. We need
// to have this inside a if because if we are facing a retry
// error, in particular those generated by
// crdb_internal.force_retry(), Wrap() will propagate it as a
// non-pgerror error (so that the executor can see it with the
// right type).
newErr = errors.WithTelemetry(newErr, fName+"()")
return nil, newErr
return nil, expr.MaybeWrapError(err)
}
if ctx.TestingKnobs.AssertFuncExprReturnTypes {
if err := ensureExpectedType(expr.fn.FixedReturnType(), res); err != nil {
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/trace_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -603,6 +603,10 @@ func TestTraceDistSQL(t *testing.T) {
defer cluster.Stopper().Stop(ctx)

r := sqlutils.MakeSQLRunner(cluster.ServerConn(0))
// TODO(yuzefovich): tracing in the vectorized engine is very limited since
// only wrapped processors and the materializers use it outside of the
// stats information propagation. We should fix that (#55821).
r.Exec(t, "SET vectorize=off")
r.Exec(t, "CREATE DATABASE test")
r.Exec(t, "CREATE TABLE test.a (a INT PRIMARY KEY)")
// Put the table on the 2nd node so that the flow is planned on the 2nd node
Expand Down
2 changes: 1 addition & 1 deletion pkg/util/log/logcrash/crash_reporting_packet_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -293,7 +293,7 @@ func TestInternalErrorReporting(t *testing.T) {
assert.Regexp(t, `.*/builtins.go`, fr[len(fr)-1].Filename)
assert.Regexp(t, `.*/eval.go`, fr[len(fr)-2].Filename)

assert.Regexp(t, `^\(3\) eval.go:\d+ \(Eval\)$`, p.Exception[0].Type)
assert.Regexp(t, `^\(3\) eval.go:\d+ \(MaybeWrapError\)$`, p.Exception[0].Type)
assert.Regexp(t, `^\*withstack\.withStack$`, p.Exception[0].Value)
fr = p.Exception[0].Stacktrace.Frames
assert.Regexp(t, `.*/eval.go`, fr[len(fr)-1].Filename)
Expand Down

0 comments on commit 05c825b

Please sign in to comment.