Skip to content

Commit 6492c94

Browse files
committed
colexecerror: add benchmarks for CatchVectorizedRuntimeError
Informs: cockroachdb#123235 Release note: None
1 parent 086f930 commit 6492c94

File tree

6 files changed

+305
-9
lines changed

6 files changed

+305
-9
lines changed

pkg/sql/colexecerror/BUILD.bazel

Lines changed: 24 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -15,11 +15,34 @@ go_library(
1515

1616
go_test(
1717
name = "colexecerror_test",
18-
srcs = ["error_test.go"],
18+
srcs = [
19+
"error_test.go",
20+
"main_test.go",
21+
],
1922
deps = [
2023
":colexecerror",
24+
"//pkg/base",
25+
"//pkg/col/coldata",
26+
"//pkg/col/coldataext",
27+
"//pkg/security/securityassets",
28+
"//pkg/security/securitytest",
29+
"//pkg/server",
30+
"//pkg/settings/cluster",
31+
"//pkg/sql/colexec/colexectestutils",
32+
"//pkg/sql/colmem",
33+
"//pkg/sql/execinfra",
34+
"//pkg/sql/pgwire/pgcode",
35+
"//pkg/sql/pgwire/pgerror",
36+
"//pkg/sql/sem/eval",
37+
"//pkg/testutils/serverutils",
38+
"//pkg/testutils/skip",
39+
"//pkg/testutils/sqlutils",
40+
"//pkg/testutils/testcluster",
2141
"//pkg/util/leaktest",
2242
"//pkg/util/log",
43+
"//pkg/util/mon",
44+
"//pkg/util/randutil",
45+
"@com_github_cockroachdb_errors//:errors",
2346
"@com_github_stretchr_testify//require",
2447
],
2548
)

pkg/sql/colexecerror/error.go

Lines changed: 9 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -100,12 +100,13 @@ func CatchVectorizedRuntimeError(operation func()) (retErr error) {
100100
// Multiple actual packages can have the same prefix as a single constant string
101101
// defined below, but all of such packages are allowed to be caught from.
102102
const (
103-
colPackagesPrefix = "github.com/cockroachdb/cockroach/pkg/col"
104-
encodingPackagePrefix = "github.com/cockroachdb/cockroach/pkg/util/encoding"
105-
execinfraPackagePrefix = "github.com/cockroachdb/cockroach/pkg/sql/execinfra"
106-
sqlColPackagesPrefix = "github.com/cockroachdb/cockroach/pkg/sql/col"
107-
sqlRowPackagesPrefix = "github.com/cockroachdb/cockroach/pkg/sql/row"
108-
sqlSemPackagesPrefix = "github.com/cockroachdb/cockroach/pkg/sql/sem"
103+
colPackagesPrefix = "github.com/cockroachdb/cockroach/pkg/col"
104+
encodingPackagePrefix = "github.com/cockroachdb/cockroach/pkg/util/encoding"
105+
execinfraPackagePrefix = "github.com/cockroachdb/cockroach/pkg/sql/execinfra"
106+
sqlColPackagesPrefix = "github.com/cockroachdb/cockroach/pkg/sql/col"
107+
sqlRowPackagesPrefix = "github.com/cockroachdb/cockroach/pkg/sql/row"
108+
sqlSemPackagesPrefix = "github.com/cockroachdb/cockroach/pkg/sql/sem"
109+
testSqlColPackagesPrefix = "pkg/sql/col"
109110
)
110111

111112
// shouldCatchPanic checks whether the panic that was emitted from
@@ -135,7 +136,8 @@ func shouldCatchPanic(panicEmittedFrom string) bool {
135136
strings.HasPrefix(panicEmittedFrom, execinfraPackagePrefix) ||
136137
strings.HasPrefix(panicEmittedFrom, sqlColPackagesPrefix) ||
137138
strings.HasPrefix(panicEmittedFrom, sqlRowPackagesPrefix) ||
138-
strings.HasPrefix(panicEmittedFrom, sqlSemPackagesPrefix)
139+
strings.HasPrefix(panicEmittedFrom, sqlSemPackagesPrefix) ||
140+
strings.HasPrefix(panicEmittedFrom, testSqlColPackagesPrefix)
139141
}
140142

141143
// StorageError is an error that was created by a component below the sql

pkg/sql/colexecerror/error_test.go

Lines changed: 156 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -11,13 +11,22 @@
1111
package colexecerror_test
1212

1313
import (
14-
"errors"
14+
"context"
15+
gosql "database/sql"
16+
"fmt"
17+
"runtime"
1518
"strings"
1619
"testing"
1720

21+
"github.com/cockroachdb/cockroach/pkg/base"
1822
"github.com/cockroachdb/cockroach/pkg/sql/colexecerror"
23+
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
24+
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
25+
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
26+
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
1927
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
2028
"github.com/cockroachdb/cockroach/pkg/util/log"
29+
"github.com/cockroachdb/errors"
2130
"github.com/stretchr/testify/require"
2231
)
2332

@@ -74,3 +83,149 @@ func TestNonCatchablePanicIsNotCaught(t *testing.T) {
7483
}))
7584
})
7685
}
86+
87+
// BenchmarkCatchVectorizedRuntimeError measures the time for
88+
// CatchVectorizedRuntimeError to catch and process an error.
89+
func BenchmarkCatchVectorizedRuntimeError(b *testing.B) {
90+
err := errors.New("oops")
91+
storageErr := colexecerror.NewStorageError(err)
92+
pgErr := pgerror.WithCandidateCode(err, pgcode.Warning)
93+
94+
cases := []struct {
95+
name string
96+
thrower func()
97+
}{
98+
{
99+
"noError",
100+
func() {},
101+
},
102+
{
103+
"expected",
104+
func() {
105+
colexecerror.ExpectedError(err)
106+
},
107+
},
108+
{
109+
"storage",
110+
func() {
111+
colexecerror.InternalError(storageErr)
112+
},
113+
},
114+
{
115+
"contextCanceled",
116+
func() {
117+
colexecerror.InternalError(context.Canceled)
118+
},
119+
},
120+
{
121+
"internalWithCode",
122+
func() {
123+
colexecerror.InternalError(pgErr)
124+
},
125+
},
126+
{
127+
"internal",
128+
func() {
129+
colexecerror.InternalError(err)
130+
},
131+
},
132+
{
133+
"runtime",
134+
func() {
135+
arr := []int{0, 1, 2}
136+
_ = arr[3]
137+
},
138+
},
139+
}
140+
141+
for _, tc := range cases {
142+
b.Run(tc.name, func(b *testing.B) {
143+
b.RunParallel(func(pb *testing.PB) {
144+
for pb.Next() {
145+
_ = colexecerror.CatchVectorizedRuntimeError(tc.thrower)
146+
}
147+
})
148+
})
149+
}
150+
}
151+
152+
func BenchmarkSQLCatchVectorizedRuntimeError(b *testing.B) {
153+
defer leaktest.AfterTest(b)()
154+
defer log.Scope(b).Close(b)
155+
156+
cases := []struct {
157+
name string
158+
builtin string
159+
}{
160+
{
161+
"noError",
162+
"crdb_internal.void_func()",
163+
},
164+
{
165+
"expectedWithCode",
166+
"crdb_internal.force_error('01000', 'oops')",
167+
},
168+
{
169+
"expectedAssertion",
170+
"crdb_internal.force_assertion_error('oops')",
171+
},
172+
{
173+
"internal",
174+
"crdb_internal.force_vectorized_assertion_error('oops')",
175+
},
176+
}
177+
178+
sqlFmt := `SELECT %s,
179+
0,
180+
'',
181+
0.0,
182+
NULL,
183+
'2000-01-01 00:00:00'::timestamptz,
184+
b'00000000',
185+
i + 0,
186+
i * 1.5,
187+
i / 100
188+
FROM generate_series(0, 0) AS s(i)
189+
`
190+
191+
ctx := context.Background()
192+
s := serverutils.StartServerOnly(b, base.TestServerArgs{SQLMemoryPoolSize: 10 << 30})
193+
defer s.Stopper().Stop(ctx)
194+
195+
for _, parallelism := range []int{1, 20, 50} {
196+
numConns := runtime.GOMAXPROCS(0) * parallelism
197+
b.Run(fmt.Sprintf("conns=%d", numConns), func(b *testing.B) {
198+
for _, tc := range cases {
199+
stmt := fmt.Sprintf(sqlFmt, tc.builtin)
200+
b.Run(tc.name, func(b *testing.B) {
201+
// Create as many warm connections as we will need for the benchmark.
202+
conns := make(chan *gosql.DB, numConns)
203+
for range numConns {
204+
conn := s.ApplicationLayer().SQLConn(b, serverutils.DBName(""))
205+
// Make sure we're using local, vectorized execution.
206+
sqlDB := sqlutils.MakeSQLRunner(conn)
207+
sqlDB.Exec(b, "SET distsql = off")
208+
sqlDB.Exec(b, "SET vectorize = on")
209+
// Warm up the connection by executing the statement once. We should
210+
// always go through the query plan cache after this.
211+
_, _ = conn.Exec(stmt)
212+
conns <- conn
213+
}
214+
b.SetParallelism(parallelism)
215+
b.ResetTimer()
216+
b.RunParallel(func(pb *testing.PB) {
217+
var conn *gosql.DB
218+
select {
219+
case conn = <-conns:
220+
default:
221+
b.Fatal("not enough warm connections")
222+
}
223+
for pb.Next() {
224+
_, _ = conn.Exec(stmt)
225+
}
226+
})
227+
})
228+
}
229+
})
230+
}
231+
}

pkg/sql/colexecerror/main_test.go

Lines changed: 90 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,90 @@
1+
// Copyright 2024 The Cockroach Authors.
2+
//
3+
// Use of this software is governed by the Business Source License
4+
// included in the file licenses/BSL.txt.
5+
//
6+
// As of the Change Date specified in that file, in accordance with
7+
// the Business Source License, use of this software will be governed
8+
// by the Apache License, Version 2.0, included in the file
9+
// licenses/APL.txt.
10+
11+
package colexecerror_test
12+
13+
import (
14+
"context"
15+
"flag"
16+
"fmt"
17+
"os"
18+
"testing"
19+
20+
"github.com/cockroachdb/cockroach/pkg/col/coldata"
21+
"github.com/cockroachdb/cockroach/pkg/col/coldataext"
22+
"github.com/cockroachdb/cockroach/pkg/security/securityassets"
23+
"github.com/cockroachdb/cockroach/pkg/security/securitytest"
24+
"github.com/cockroachdb/cockroach/pkg/server"
25+
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
26+
"github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils"
27+
"github.com/cockroachdb/cockroach/pkg/sql/colexecerror"
28+
"github.com/cockroachdb/cockroach/pkg/sql/colmem"
29+
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
30+
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
31+
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
32+
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
33+
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
34+
"github.com/cockroachdb/cockroach/pkg/util/mon"
35+
"github.com/cockroachdb/cockroach/pkg/util/randutil"
36+
)
37+
38+
//go:generate ../../util/leaktest/add-leaktest.sh *_test.go
39+
40+
var (
41+
// testAllocator is an Allocator with an unlimited budget for use in tests.
42+
testAllocator *colmem.Allocator
43+
testColumnFactory coldata.ColumnFactory
44+
45+
// testMemMonitor and testMemAcc are a test monitor with an unlimited budget
46+
// and a memory account bound to it for use in tests.
47+
testMemMonitor *mon.BytesMonitor
48+
testMemAcc *mon.BoundAccount
49+
50+
// testDiskMonitor and testDiskmAcc are a test monitor with an unlimited budget
51+
// and a disk account bound to it for use in tests.
52+
testDiskMonitor *mon.BytesMonitor
53+
testDiskAcc *mon.BoundAccount
54+
)
55+
56+
func TestMain(m *testing.M) {
57+
securityassets.SetLoader(securitytest.EmbeddedAssets)
58+
randutil.SeedForTests()
59+
serverutils.InitTestServerFactory(server.TestServerFactory)
60+
serverutils.InitTestClusterFactory(testcluster.TestClusterFactory)
61+
os.Exit(func() int {
62+
ctx := context.Background()
63+
st := cluster.MakeTestingClusterSettings()
64+
testMemMonitor = execinfra.NewTestMemMonitor(ctx, st)
65+
defer testMemMonitor.Stop(ctx)
66+
memAcc := testMemMonitor.MakeBoundAccount()
67+
testMemAcc = &memAcc
68+
evalCtx := eval.MakeTestingEvalContext(st)
69+
testColumnFactory = coldataext.NewExtendedColumnFactory(&evalCtx)
70+
testAllocator = colmem.NewAllocator(ctx, testMemAcc, testColumnFactory)
71+
defer testMemAcc.Close(ctx)
72+
73+
testDiskMonitor = execinfra.NewTestDiskMonitor(ctx, cluster.MakeTestingClusterSettings())
74+
defer testDiskMonitor.Stop(ctx)
75+
diskAcc := testDiskMonitor.MakeBoundAccount()
76+
testDiskAcc = &diskAcc
77+
defer testDiskAcc.Close(ctx)
78+
79+
flag.Parse()
80+
if !skip.UnderBench() {
81+
// (If we're running benchmarks, don't set a random batch size.)
82+
randomBatchSize := colexectestutils.GenerateBatchSize()
83+
fmt.Printf("coldata.BatchSize() is set to %d\n", randomBatchSize)
84+
if err := coldata.SetBatchSizeForTests(randomBatchSize); err != nil {
85+
colexecerror.InternalError(err)
86+
}
87+
}
88+
return m.Run()
89+
}())
90+
}

pkg/sql/sem/builtins/builtins.go

Lines changed: 25 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -5642,6 +5642,31 @@ SELECT
56425642
},
56435643
),
56445644

5645+
"crdb_internal.force_vectorized_assertion_error": makeBuiltin(
5646+
tree.FunctionProperties{
5647+
Category: builtinconstants.CategorySystemInfo,
5648+
},
5649+
tree.Overload{
5650+
Types: tree.ParamTypes{{Name: "msg", Typ: types.String}},
5651+
ReturnType: tree.FixedReturnType(types.Int),
5652+
Fn: func(_ context.Context, _ *eval.Context, args tree.Datums) (tree.Datum, error) {
5653+
s, ok := tree.AsDString(args[0])
5654+
if !ok {
5655+
return nil, errors.Newf("expected string value, got %T", args[0])
5656+
}
5657+
msg := string(s)
5658+
err := errors.AssertionFailedf("%s", msg)
5659+
// Panic instead of returning the error. The vectorized panic-catcher
5660+
// will catch the panic and convert it into an internal error.
5661+
colexecerror.InternalError(err)
5662+
// This code is unreachable.
5663+
panic(err)
5664+
},
5665+
Info: "This function is used only by CockroachDB's developers for testing purposes.",
5666+
Volatility: volatility.Volatile,
5667+
},
5668+
),
5669+
56455670
"crdb_internal.void_func": makeBuiltin(
56465671
tree.FunctionProperties{
56475672
Category: builtinconstants.CategorySystemInfo,

pkg/sql/sem/builtins/fixed_oids.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2573,6 +2573,7 @@ var builtinOidsArray = []string{
25732573
2605: `merge_aggregated_stmt_metadata(arg1: jsonb) -> jsonb`,
25742574
2606: `crdb_internal.protect_mvcc_history(timestamp: decimal, expiration_window: interval, description: string) -> int`,
25752575
2607: `crdb_internal.extend_mvcc_history_protection(job_id: int) -> void`,
2576+
2608: `crdb_internal.force_vectorized_assertion_error(msg: string) -> int`,
25762577
}
25772578

25782579
var builtinOidsBySignature map[string]oid.Oid

0 commit comments

Comments
 (0)