From 4f1698fab68927fb65d9d54bf05fd2782905f400 Mon Sep 17 00:00:00 2001 From: Oliver Tan Date: Tue, 6 Apr 2021 18:56:37 +1000 Subject: [PATCH 1/3] sql: fix ADD COLUMN ... UNIQUE for PARTITION ALL BY tables Release note (bug fix): Fixed a bug where doing ALTER TABLE ... ADD COLUMN ... UNIQUE would error if the table had a PARTITION ALL BY / REGIONAL BY ROW definition. --- .../testdata/logic_test/partitioning_implicit | 15 +++++++++-- .../testdata/logic_test/regional_by_row | 27 +++++++++++++++++++ pkg/sql/add_column.go | 13 +++++++++ 3 files changed, 53 insertions(+), 2 deletions(-) diff --git a/pkg/ccl/logictestccl/testdata/logic_test/partitioning_implicit b/pkg/ccl/logictestccl/testdata/logic_test/partitioning_implicit index 4c66365363cf..f48289dadb78 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/partitioning_implicit +++ b/pkg/ccl/logictestccl/testdata/logic_test/partitioning_implicit @@ -454,6 +454,9 @@ query T noticetrace CREATE INDEX created_idx ON t(c) ---- +statement ok +ALTER TABLE t ADD COLUMN e INT8 NOT NULL UNIQUE + statement ok ALTER TABLE t ADD CONSTRAINT unique_c_d UNIQUE(c, d) @@ -470,6 +473,7 @@ CREATE TABLE public.t ( d INT8 NOT NULL, j JSONB NULL, u STRING NULL, + e INT8 NOT NULL, CONSTRAINT "primary" PRIMARY KEY (pk ASC), UNIQUE INDEX t_u_key (u ASC), INDEX t_a_idx (a ASC), @@ -477,8 +481,9 @@ CREATE TABLE public.t ( INDEX t_partition_by_c_idx (partition_by ASC, c ASC), INVERTED INDEX t_j_idx (j), INDEX created_idx (c ASC), + UNIQUE INDEX t_e_key (e ASC), UNIQUE INDEX unique_c_d (c ASC, d ASC), - FAMILY fam_0_pk_pk2_partition_by_a_b_c_d_j_u (pk, pk2, partition_by, a, b, c, d, j, u) + FAMILY fam_0_pk_pk2_partition_by_a_b_c_d_j_u (pk, pk2, partition_by, a, b, c, d, j, u, e) ) PARTITION ALL BY LIST (partition_by) ( PARTITION one VALUES IN ((1)), PARTITION two VALUES IN ((2)) @@ -499,6 +504,8 @@ t_a_idx a false t_a_idx partition_by true t_b_key b false t_b_key partition_by true +t_e_key e false +t_e_key partition_by true t_j_idx j false t_j_idx partition_by true t_partition_by_c_idx c false @@ -525,6 +532,7 @@ CREATE TABLE public.t ( d INT8 NOT NULL, j JSONB NULL, u STRING NULL, + e INT8 NOT NULL, CONSTRAINT "primary" PRIMARY KEY (pk2 ASC), UNIQUE INDEX t_pk_key (pk ASC), UNIQUE INDEX t_u_key (u ASC), @@ -533,8 +541,9 @@ CREATE TABLE public.t ( INDEX t_partition_by_c_idx (partition_by ASC, c ASC), INVERTED INDEX t_j_idx (j), INDEX created_idx (c ASC), + UNIQUE INDEX t_e_key (e ASC), UNIQUE INDEX unique_c_d (c ASC, d ASC), - FAMILY fam_0_pk_pk2_partition_by_a_b_c_d_j_u (pk, pk2, partition_by, a, b, c, d, j, u) + FAMILY fam_0_pk_pk2_partition_by_a_b_c_d_j_u (pk, pk2, partition_by, a, b, c, d, j, u, e) ) PARTITION ALL BY LIST (partition_by) ( PARTITION one VALUES IN ((1)), PARTITION two VALUES IN ((2)) @@ -555,6 +564,8 @@ t_a_idx a false t_a_idx partition_by true t_b_key b false t_b_key partition_by true +t_e_key e false +t_e_key partition_by true t_j_idx j false t_j_idx partition_by true t_partition_by_c_idx c false diff --git a/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row b/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row index e81a2eb32fad..75fad290e408 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row +++ b/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row @@ -513,6 +513,33 @@ ALTER TABLE regional_by_row_table ALTER PRIMARY KEY USING COLUMNS(pk2) USING HAS statement error interleaved tables are not compatible with REGIONAL BY ROW tables CREATE INDEX bad_idx ON regional_by_row_table(pk) INTERLEAVE IN PARENT parent_table(pk) +# Try add a new unique column. +statement ok +ALTER TABLE regional_by_row_table ADD COLUMN unique_col INT8 NOT NULL UNIQUE + +query T +SELECT create_statement FROM [SHOW CREATE TABLE regional_by_row_table] +---- +CREATE TABLE public.regional_by_row_table ( + pk INT8 NOT NULL, + pk2 INT8 NOT NULL, + a INT8 NOT NULL, + b INT8 NOT NULL, + j JSONB NULL, + crdb_region public.crdb_internal_region NOT VISIBLE NOT NULL DEFAULT default_to_database_primary_region(gateway_region())::public.crdb_internal_region, + unique_col INT8 NOT NULL, + CONSTRAINT "primary" PRIMARY KEY (pk ASC), + INDEX regional_by_row_table_a_idx (a ASC), + UNIQUE INDEX regional_by_row_table_b_key (b ASC), + INVERTED INDEX regional_by_row_table_j_idx (j), + UNIQUE INDEX regional_by_row_table_unique_col_key (unique_col ASC), + FAMILY fam_0_pk_pk2_a_b_j_crdb_region (pk, pk2, a, b, j, crdb_region, unique_col) +) LOCALITY REGIONAL BY ROW; +ALTER PARTITION "us-east-1" OF INDEX multi_region_test_db.public.regional_by_row_table@regional_by_row_table_a_idx CONFIGURE ZONE USING "gc.ttlseconds" = 10 + +statement ok +ALTER TABLE regional_by_row_table DROP COLUMN unique_col + # Insert some rows into the regional_by_row_table. query TI INSERT INTO regional_by_row_table (pk, pk2, a, b, j) VALUES diff --git a/pkg/sql/add_column.go b/pkg/sql/add_column.go index 8499aa838740..aef19bed41db 100644 --- a/pkg/sql/add_column.go +++ b/pkg/sql/add_column.go @@ -81,6 +81,19 @@ func (p *planner) addColumnImpl( } incTelemetryForNewColumn(d, col) + // Ensure all new indexes are partitioned appropriately. + if idx != nil { + *idx, err = p.configureIndexDescForNewIndexPartitioning( + params.ctx, + desc, + *idx, + nil, /* PartitionByIndex */ + ) + if err != nil { + return err + } + } + // If the new column has a DEFAULT expression that uses a sequence, add references between // its descriptor and this column descriptor. if d.HasDefaultExpr() { From d8f85e2d58c1d50d99956b2a4ad2c1cb477b00f1 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Thu, 1 Apr 2021 22:49:31 -0400 Subject: [PATCH 2/3] sql: clean up ieResultChannel concepts and fix race condition The async and sync implementations were too close to justify two structs. Also, the async behavior of not stopping the writer in case the reader called close wasn't desireable. This commit unifies the implementation. It also ensures that we propagate context errors in all cases triggered by the closure of the done channel. It also makes closing the channel idempotent. Additionally, this commit transitions the execution flow into draining state without setting our customer error on the resultWriter. Release note: None --- pkg/sql/distsql_running.go | 8 +- pkg/sql/internal.go | 2 +- pkg/sql/internal_result_channel.go | 208 +++++++++++++---------------- pkg/sql/user_test.go | 2 - 4 files changed, 98 insertions(+), 122 deletions(-) diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index 539cfc891da7..73e9e3d77109 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -738,10 +738,10 @@ func (r *DistSQLReceiver) Push( } r.tracing.TraceExecRowsResult(r.ctx, r.row) if commErr := r.resultWriter.AddRow(r.ctx, r.row); commErr != nil { - if errors.Is(commErr, ErrLimitedResultClosed) { - // ErrLimitedResultClosed is not a real error, it is a signal to - // stop distsql and return success to the client (that's why we - // don't set the error on the resultWriter). + if errors.Is(commErr, ErrLimitedResultClosed) || errors.Is(commErr, errIEResultChannelClosed) { + // ErrLimitedResultClosed and errIEResultChannelClosed are not real + // errors, it is a signal to stop distsql and return success to the + // client (that's why we don't set the error on the resultWriter). r.status = execinfra.DrainRequested } else { // Set the error on the resultWriter to notify the consumer about diff --git a/pkg/sql/internal.go b/pkg/sql/internal.go index 173fbf8e5ef5..e6411c1ea510 100644 --- a/pkg/sql/internal.go +++ b/pkg/sql/internal.go @@ -604,7 +604,7 @@ var rowsAffectedResultColumns = colinfo.ResultColumns{ func (ie *InternalExecutor) execInternal( ctx context.Context, opName string, - rw ieResultChannel, + rw *ieResultChannel, txn *kv.Txn, sessionDataOverride sessiondata.InternalExecutorOverride, stmt string, diff --git a/pkg/sql/internal_result_channel.go b/pkg/sql/internal_result_channel.go index 33a41540d3d7..d8e4a282dddc 100644 --- a/pkg/sql/internal_result_channel.go +++ b/pkg/sql/internal_result_channel.go @@ -18,13 +18,6 @@ import ( "github.com/cockroachdb/errors" ) -// ieResultChannel is used to coordinate passing results from an -// internalExecutor to its corresponding iterator. -type ieResultChannel interface { - ieResultReader - ieResultWriter -} - // ieResultReader is used to read internalExecutor results. // It is managed by the rowsIterator. type ieResultReader interface { @@ -34,15 +27,13 @@ type ieResultReader interface { // currently blocked and waits for the initial result to be written. firstResult(ctx context.Context) (_ ieIteratorResult, done bool, err error) - // nextResult returns the nextResult. Done will always be true if err + // nextResult returns the next result. Done will always be true if err // is non-nil. Err will be non-nil if either close has been called or // the passed context is finished. nextResult(ctx context.Context) (_ ieIteratorResult, done bool, err error) - // close ensures that the either writer has finished writing. In the case - // of an asynchronous channel, close will drain the writer's channel. In the - // case of the synchronous channel, it will ensure that the writer receives - // an error when it wakes. + // close ensures that either writer has finished writing. The writer will + // receive a signal to drain, and close will drain the writer's channel. close() error } @@ -67,174 +58,161 @@ var asyncIEResultChannelBufferSize = util.ConstantWithMetamorphicTestRange( // newAsyncIEResultChannel returns an ieResultChannel which does not attempt to // synchronize the writer with the reader. -func newAsyncIEResultChannel() ieResultChannel { - return &asyncIEResultChannel{ +func newAsyncIEResultChannel() *ieResultChannel { + return &ieResultChannel{ dataCh: make(chan ieIteratorResult, asyncIEResultChannelBufferSize), + doneCh: make(chan struct{}), } } -type asyncIEResultChannel struct { - dataCh chan ieIteratorResult -} - -var _ ieResultChannel = &asyncIEResultChannel{} - -func (c *asyncIEResultChannel) firstResult( - ctx context.Context, -) (_ ieIteratorResult, done bool, err error) { - select { - case <-ctx.Done(): - return ieIteratorResult{}, true, ctx.Err() - case res, ok := <-c.dataCh: - if !ok { - return ieIteratorResult{}, true, nil - } - return res, false, nil - } -} - -func (c *asyncIEResultChannel) nextResult( - ctx context.Context, -) (_ ieIteratorResult, done bool, err error) { - return c.firstResult(ctx) -} - -func (c *asyncIEResultChannel) close() error { - var firstErr error - for { - res, done, err := c.nextResult(context.TODO()) - if firstErr == nil { - if res.err != nil { - firstErr = res.err - } else if err != nil { - firstErr = err - } - } - if done { - return firstErr - } - } -} - -func (c *asyncIEResultChannel) addResult(ctx context.Context, result ieIteratorResult) error { - select { - case <-ctx.Done(): - return ctx.Err() - case c.dataCh <- result: - return nil - } -} - -func (c *asyncIEResultChannel) finish() { - close(c.dataCh) -} - -// syncIEResultChannel is used to ensure that in execution scenarios which -// do not permit concurrency that there is none. It works by blocking the -// writing goroutine immediately upon sending on the data channel and only -// unblocking it after the reader signals. -type syncIEResultChannel struct { +// ieResultChannel is used to coordinate passing results from an +// internalExecutor to its corresponding iterator. It can be constructed to +// ensure that there is no concurrency between the reader and writer. +type ieResultChannel struct { // dataCh is the channel on which the connExecutor goroutine sends the rows - // (in addResult) and will block on waitCh after each send. The iterator - // goroutine blocks on dataCh until there is something to receive (rows or - // other metadata) and will return the data to the caller. On the next call - // to Next(), the iterator goroutine unblocks the producer and will block - // itself again. dataCh will be closed (in finish()) when the connExecutor - // goroutine exits its run() loop whereas waitCh is closed when closing the - // iterator. + // (in addResult) and, in the synchronous case, will block on waitCh after + // each send. The iterator goroutine blocks on dataCh until there is + // something to receive (rows or other metadata) and will return the data to + // the caller. On the next call to Next(), the iterator goroutine unblocks + // the producer and will block itself again. dataCh will be closed (in + // finish()) when the connExecutor goroutine exits its run() loop whereas + // waitCh is closed when closing the iterator. dataCh chan ieIteratorResult - // waitCh is never closed. In all places where the caller may interact with it - // the doneCh is also used. This policy is in place to make it safe to unblock - // both the reader and the writer without any hazards of a blocked reader - // attempting to send on a closed channel. + // waitCh is nil for async ieResultChannels. It is never closed. In all places + // where the caller may interact with it the doneCh is also used. This policy + // is in place to make it safe to unblock both the reader and the writer + // without any hazards of a blocked reader attempting to send on a closed + // channel. waitCh chan struct{} - // doneCh is used to indicate that the ReadWriter has been closed. - // doneCh is closed under the doneOnce. The doneCh is only used for the - // syncIEResultChannel. This is crucial to ensure that a synchronous writer - // does not attempt to continue to operate after the reader has called close. + // doneCh is used to indicate that the ieResultReader has been closed and is + // closed under the doneOnce, the writer will transition to draining. This + // is crucial to ensure that a synchronous writer does not attempt to + // continue to operate after the reader has called close. doneCh chan struct{} + doneErr error doneOnce sync.Once } -var _ ieResultChannel = &syncIEResultChannel{} - -// newSyncIEResultChannel returns an ieResultChannel which synchronizes the -// writer with the reader. -func newSyncIEResultChannel() ieResultChannel { - return &syncIEResultChannel{ +// newSyncIEResultChannel is used to ensure that in execution scenarios which +// do not permit concurrency that there is none. It works by blocking the +// writing goroutine immediately upon sending on the data channel and only +// unblocking it after the reader signals. +func newSyncIEResultChannel() *ieResultChannel { + return &ieResultChannel{ dataCh: make(chan ieIteratorResult), waitCh: make(chan struct{}), doneCh: make(chan struct{}), } } -func (i *syncIEResultChannel) firstResult( +func (i *ieResultChannel) firstResult( ctx context.Context, ) (_ ieIteratorResult, done bool, err error) { select { case <-ctx.Done(): return ieIteratorResult{}, true, ctx.Err() case <-i.doneCh: - return ieIteratorResult{}, true, nil + return ieIteratorResult{}, true, ctx.Err() case res, ok := <-i.dataCh: if !ok { - return ieIteratorResult{}, true, nil + return ieIteratorResult{}, true, ctx.Err() } return res, false, nil } } -func (i *syncIEResultChannel) unblockWriter(ctx context.Context) (done bool, err error) { +func (i *ieResultChannel) maybeUnblockWriter(ctx context.Context) (done bool, err error) { + if i.async() { + return false, nil + } select { case <-ctx.Done(): return true, ctx.Err() case <-i.doneCh: - return true, nil + return true, ctx.Err() case i.waitCh <- struct{}{}: return false, nil } } -func (i *syncIEResultChannel) finish() { - close(i.dataCh) +func (i *ieResultChannel) async() bool { + return i.waitCh == nil } -func (i *syncIEResultChannel) nextResult( +func (i *ieResultChannel) nextResult( ctx context.Context, ) (_ ieIteratorResult, done bool, err error) { - if done, err = i.unblockWriter(ctx); done { + if done, err = i.maybeUnblockWriter(ctx); done { return ieIteratorResult{}, done, err } return i.firstResult(ctx) } -func (i *syncIEResultChannel) close() error { - i.doneOnce.Do(func() { close(i.doneCh) }) - return nil +func (i *ieResultChannel) close() error { + i.doneOnce.Do(func() { + close(i.doneCh) + for { + // In the async case, res might contain some actual rows, but we're + // not interested in them; in the sync case, only errors are + // expected to be retrieved from now one because the writer + // transitions to draining. + res, done, err := i.nextResult(context.TODO()) + if i.doneErr == nil { + if res.err != nil { + i.doneErr = res.err + } else if err != nil { + i.doneErr = err + } + } + if done { + return + } + } + }) + return i.doneErr } -// errSyncIEResultReaderCanceled is returned by the writer when the reader has -// closed syncIEResultChannel. The error indicates to the writer to shut down -// the query execution, but the reader won't propagate it further. -var errSyncIEResultReaderCanceled = errors.New("synchronous ieResultReader closed") +// errIEResultChannelClosed is returned by the writer when the reader has closed +// ieResultChannel. The error indicates to the writer to drain the query +// execution, but the reader won't propagate it further. +var errIEResultChannelClosed = errors.New("ieResultReader closed") -func (i *syncIEResultChannel) addResult(ctx context.Context, result ieIteratorResult) error { +func (i *ieResultChannel) addResult(ctx context.Context, result ieIteratorResult) error { select { case <-ctx.Done(): return ctx.Err() case <-i.doneCh: - return errSyncIEResultReaderCanceled + // Prefer the context error if there is one. + if ctxErr := ctx.Err(); ctxErr != nil { + return ctxErr + } + return errIEResultChannelClosed case i.dataCh <- result: } + return i.maybeBlock(ctx) +} + +func (i *ieResultChannel) maybeBlock(ctx context.Context) error { + if i.async() { + return nil + } select { case <-ctx.Done(): return ctx.Err() case <-i.doneCh: - return errSyncIEResultReaderCanceled + // Prefer the context error if there is one. + if ctxErr := ctx.Err(); ctxErr != nil { + return ctxErr + } + return errIEResultChannelClosed case <-i.waitCh: return nil } } + +func (i *ieResultChannel) finish() { + close(i.dataCh) +} diff --git a/pkg/sql/user_test.go b/pkg/sql/user_test.go index aa884f7ffef2..7e277a7c9faf 100644 --- a/pkg/sql/user_test.go +++ b/pkg/sql/user_test.go @@ -45,8 +45,6 @@ func TestGetUserHashedPasswordTimeout(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - skip.WithIssue(t, 62948 /* githubIssueID */) - // We want to use a low timeout below to prevent // this test from taking forever, however // race builds are so slow as to trigger this timeout spuriously. From ae6233f5e39dc60c87e84d5d11736da563518401 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Tue, 6 Apr 2021 14:58:12 -0700 Subject: [PATCH 3/3] sem/builtins: vectorized engine no longer catches crdb_internal.force_panic Previously, when executing `crdb_internal.force_panic` builtin, if it was executed via the vectorized engine, we would catch the panic and convert it into an internal error instead. This is undesirable and is now fixed. Release note: None --- pkg/sql/colexecerror/error.go | 17 +++++++++-------- pkg/sql/colexecerror/error_test.go | 10 +++++----- .../vectorized_panic_propagation_test.go | 2 +- pkg/sql/sem/builtins/BUILD.bazel | 1 + pkg/sql/sem/builtins/builtins.go | 7 +++++++ 5 files changed, 23 insertions(+), 14 deletions(-) diff --git a/pkg/sql/colexecerror/error.go b/pkg/sql/colexecerror/error.go index 1f5b060c14f2..5658b16112ad 100644 --- a/pkg/sql/colexecerror/error.go +++ b/pkg/sql/colexecerror/error.go @@ -122,10 +122,10 @@ func shouldCatchPanic(panicEmittedFrom string) bool { // unchanged by the higher-level catchers. return false } - const nonVectorizedTestPrefix = "github.com/cockroachdb/cockroach/pkg/sql/colexecerror.NonVectorizedTestPanic" - if strings.HasPrefix(panicEmittedFrom, nonVectorizedTestPrefix) { - // This panic came from NonVectorizedTestPanic() method and should not - // be caught for testing purposes. + const nonCatchablePanicPrefix = "github.com/cockroachdb/cockroach/pkg/sql/colexecerror.NonCatchablePanic" + if strings.HasPrefix(panicEmittedFrom, nonCatchablePanicPrefix) { + // This panic came from NonCatchablePanic() method and should not be + // caught. return false } return strings.HasPrefix(panicEmittedFrom, colPackagesPrefix) || @@ -200,9 +200,10 @@ func ExpectedError(err error) { panic(newNotInternalError(err)) } -// NonVectorizedTestPanic is the equivalent of Golang's 'panic' word that should -// be used by the testing code within the vectorized engine to simulate a panic -// that occurs outside of the engine (and, thus, should not be caught). -func NonVectorizedTestPanic(object interface{}) { +// NonCatchablePanic is the equivalent of Golang's 'panic' word that can be used +// in order to crash the goroutine. It could be used by the testing code within +// the vectorized engine to simulate a panic that occurs outside of the engine +// (and, thus, should not be caught). +func NonCatchablePanic(object interface{}) { panic(object) } diff --git a/pkg/sql/colexecerror/error_test.go b/pkg/sql/colexecerror/error_test.go index 8f7606e958e4..79c07c587606 100644 --- a/pkg/sql/colexecerror/error_test.go +++ b/pkg/sql/colexecerror/error_test.go @@ -34,7 +34,7 @@ func TestCatchVectorizedRuntimeError(t *testing.T) { require.Panics(t, func() { require.NoError(t, colexecerror.CatchVectorizedRuntimeError(func() { require.NoError(t, colexecerror.CatchVectorizedRuntimeError(func() { - colexecerror.NonVectorizedTestPanic(errors.New("should not be caught")) + colexecerror.NonCatchablePanic(errors.New("should not be caught")) })) })) }) @@ -62,15 +62,15 @@ func TestCatchVectorizedRuntimeError(t *testing.T) { require.False(t, strings.Contains(notAnnotatedErr.Error(), annotationText)) } -// TestNonVectorizedTestPanicIsNotCaught verifies that panics emitted via -// NonVectorizedTestPanic() method are not caught by the catcher. -func TestNonVectorizedTestPanicIsNotCaught(t *testing.T) { +// TestNonCatchablePanicIsNotCaught verifies that panics emitted via +// NonCatchablePanic() method are not caught by the catcher. +func TestNonCatchablePanicIsNotCaught(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) require.Panics(t, func() { require.NoError(t, colexecerror.CatchVectorizedRuntimeError(func() { - colexecerror.NonVectorizedTestPanic("should panic") + colexecerror.NonCatchablePanic("should panic") })) }) } diff --git a/pkg/sql/colflow/vectorized_panic_propagation_test.go b/pkg/sql/colflow/vectorized_panic_propagation_test.go index 010bcc37916e..d7d159d291b4 100644 --- a/pkg/sql/colflow/vectorized_panic_propagation_test.go +++ b/pkg/sql/colflow/vectorized_panic_propagation_test.go @@ -179,7 +179,7 @@ func (e *testNonVectorizedPanicEmitter) Init() { func (e *testNonVectorizedPanicEmitter) Next(ctx context.Context) coldata.Batch { if !e.emitBatch { e.emitBatch = true - colexecerror.NonVectorizedTestPanic("") + colexecerror.NonCatchablePanic("") } e.emitBatch = false diff --git a/pkg/sql/sem/builtins/BUILD.bazel b/pkg/sql/sem/builtins/BUILD.bazel index afb900d4a30d..78387d827cbc 100644 --- a/pkg/sql/sem/builtins/BUILD.bazel +++ b/pkg/sql/sem/builtins/BUILD.bazel @@ -42,6 +42,7 @@ go_library( "//pkg/sql/catalog/catalogkv", "//pkg/sql/catalog/catconstants", "//pkg/sql/catalog/descpb", + "//pkg/sql/colexecerror", "//pkg/sql/lex", "//pkg/sql/lexbase", "//pkg/sql/paramparse", diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index d977d5e7f053..9a0771abac0b 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -44,6 +44,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/lexbase" "github.com/cockroachdb/cockroach/pkg/sql/parser" @@ -4076,6 +4077,12 @@ may increase either contention or retry errors, or both.`, return nil, err } msg := string(*args[0].(*tree.DString)) + // Use a special method to panic in order to go around the + // vectorized panic-catcher (which would catch the panic from + // Golang's 'panic' and would convert it into an internal + // error). + colexecerror.NonCatchablePanic(msg) + // This code is unreachable. panic(msg) }, Info: "This function is used only by CockroachDB's developers for testing purposes.",