From ac5571536bc904da9662aa34f97518f494287cd1 Mon Sep 17 00:00:00 2001 From: adityamaru Date: Tue, 25 Jul 2023 15:58:22 -0400 Subject: [PATCH 1/2] sql,jobs: short-term fix for `UndefinedColumn` job_type error In https://github.com/cockroachdb/cockroach/issues/106762 we noticed that if a query is executed with an AS OF SYSTEM TIME clause that picks a transaction timestamp before the job_type migration, then parts of the jobs infrastructure will attempt to query the job_type column even though it doesn't exist at the transaction's timestamp. As a short term fix, when we encounter an `UndefinedColumn` error for the `job_type` column in `crdb_internal.jobs` we generate a synthetic retryable error so that the txn is pushed to a higher timestamp at which the upgrade will have completed and the `job_type` column will be visible. The longer term fix is being tracked in https://github.com/cockroachdb/cockroach/issues/106764. We are intentionally approaching this issue with a whack-a-mole approach to stabilize the tests the are running into this issue. We think time is better spent designing and investing in the longer term solution that will be tracked in #106764. Fixes: #107169 Informs: #106762 Release note: None --- pkg/jobs/BUILD.bazel | 1 + pkg/jobs/utils.go | 32 +++++++++++++++++++ pkg/sql/crdb_internal.go | 4 +-- .../upgrades/json_forward_indexes_test.go | 1 - 4 files changed, 35 insertions(+), 3 deletions(-) diff --git a/pkg/jobs/BUILD.bazel b/pkg/jobs/BUILD.bazel index 9b288826a91e..08c053692117 100644 --- a/pkg/jobs/BUILD.bazel +++ b/pkg/jobs/BUILD.bazel @@ -49,6 +49,7 @@ go_library( "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/descs", "//pkg/sql/isql", + "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", "//pkg/sql/protoreflect", "//pkg/sql/sem/builtins", diff --git a/pkg/jobs/utils.go b/pkg/jobs/utils.go index 9ba2145fb6ea..17c39fee9724 100644 --- a/pkg/jobs/utils.go +++ b/pkg/jobs/utils.go @@ -19,6 +19,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/sql/isql" + "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/errors" ) @@ -153,3 +155,33 @@ func JobExists( } return row != nil, nil } + +// IsJobTypeColumnDoesNotExistError returns true if the error is of the form +// `column "job_type" does not exist`. +func isJobTypeColumnDoesNotExistError(err error) bool { + return pgerror.GetPGCode(err) == pgcode.UndefinedColumn && + strings.Contains(err.Error(), "column \"job_type\" does not exist") +} + +// MaybeGenerateForcedRetryableError returns a +// TransactionRetryWithProtoRefreshError that will cause the txn to be retried +// if the error is because of an undefined job_type column. +// +// In https://github.com/cockroachdb/cockroach/issues/106762 we noticed that if +// a query is executed with an AS OF SYSTEM TIME clause that picks a transaction +// timestamp before the job_type migration, then parts of the jobs +// infrastructure will attempt to query the job_type column even though it +// doesn't exist at the transaction's timestamp. +// +// As a short term fix, when we encounter an `UndefinedColumn` error we +// generate a synthetic retryable error so that the txn is pushed to a +// higher timestamp at which the upgrade will have completed and the +// `job_type` column will be visible. The longer term fix is being tracked +// in https://github.com/cockroachdb/cockroach/issues/106764. +func MaybeGenerateForcedRetryableError(ctx context.Context, txn *kv.Txn, err error) error { + if err != nil && isJobTypeColumnDoesNotExistError(err) { + return txn.GenerateForcedRetryableError(ctx, "synthetic error "+ + "to push timestamp to after the `job_type` upgrade has run") + } + return err +} diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index cc4cb654b2d3..51b7cd1c5679 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -1069,7 +1069,7 @@ func populateSystemJobsTableRows( params..., ) if err != nil { - return matched, err + return matched, jobs.MaybeGenerateForcedRetryableError(ctx, p.Txn(), err) } cleanup := func(ctx context.Context) { @@ -1082,7 +1082,7 @@ func populateSystemJobsTableRows( for { hasNext, err := it.Next(ctx) if !hasNext || err != nil { - return matched, err + return matched, jobs.MaybeGenerateForcedRetryableError(ctx, p.Txn(), err) } currentRow := it.Cur() diff --git a/pkg/upgrade/upgrades/json_forward_indexes_test.go b/pkg/upgrade/upgrades/json_forward_indexes_test.go index e3e51d9eb5ee..a3ceddc965fe 100644 --- a/pkg/upgrade/upgrades/json_forward_indexes_test.go +++ b/pkg/upgrade/upgrades/json_forward_indexes_test.go @@ -25,7 +25,6 @@ import ( ) func TestJSONForwardingIndexes(t *testing.T) { - skip.WithIssue(t, 107169, "flaky test") var err error skip.UnderStressRace(t) defer leaktest.AfterTest(t)() From b3a90bd3329cc2ff1b7fbed3f8f76d6fc5e9f59d Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Wed, 26 Jul 2023 10:05:15 -0700 Subject: [PATCH 2/2] sql: disallow aggregate functions in ORDER BY in DELETE and UPDATE This commit disallows aggregate functions in the context of an `ORDER BY` clause in a `DELETE` or `UPDATE` statement. An aggregate function in an `ORDER BY` would require a `GROUP BY` clause to group non-aggregate columns. A `GROUP BY` is not allowed in `DELETE` or `UPDATE` statements as it's not obvious how grouping in these statements would behave. So we simply disallow aggregates in `ORDER BY` instead. Fixes #107634 Release note (bug fix): A bug has been fixed that caused internal errors when using an aggregate function in an `ORDER BY` clause of a `DELETE` or `UPDATE` statement. Aggregate functions are no longer allowed in these contexts. The bug has been present since at least version 20.2. --- pkg/sql/logictest/testdata/logic_test/delete | 11 +++++++++++ pkg/sql/logictest/testdata/logic_test/update | 11 +++++++++++ pkg/sql/opt/optbuilder/mutation_builder.go | 6 ++++-- pkg/sql/opt/optbuilder/orderby.go | 9 ++++++--- pkg/sql/opt/optbuilder/scope.go | 4 ++++ pkg/sql/opt/optbuilder/select.go | 6 ++++-- pkg/sql/opt/optbuilder/testdata/delete | 6 ++++++ pkg/sql/opt/optbuilder/testdata/update | 6 ++++++ 8 files changed, 52 insertions(+), 7 deletions(-) diff --git a/pkg/sql/logictest/testdata/logic_test/delete b/pkg/sql/logictest/testdata/logic_test/delete index 428ffa7dcc61..66f63b3a6e69 100644 --- a/pkg/sql/logictest/testdata/logic_test/delete +++ b/pkg/sql/logictest/testdata/logic_test/delete @@ -565,3 +565,14 @@ false query I SELECT a FROM t99630a@idx WHERE a > 0 ---- + +# Regression test for #107634. Do not allow aggregate functions in ORDER BY. +subtest regression_107634 + +statement ok +CREATE TABLE t107634 (a INT) + +statement error pgcode 42803 sum\(\): aggregate functions are not allowed in ORDER BY in DELETE +DELETE FROM t107634 ORDER BY sum(a) LIMIT 1; + +subtest end diff --git a/pkg/sql/logictest/testdata/logic_test/update b/pkg/sql/logictest/testdata/logic_test/update index af79c10f50d6..6feaa006b02c 100644 --- a/pkg/sql/logictest/testdata/logic_test/update +++ b/pkg/sql/logictest/testdata/logic_test/update @@ -661,3 +661,14 @@ SELECT * FROM generated_as_id_t ORDER BY a; 7 4 4 8 5 5 9 6 6 + +# Regression test for #107634. Do not allow aggregate functions in ORDER BY. +subtest regression_107634 + +statement ok +CREATE TABLE t107634 (a INT) + +statement error pgcode 42803 sum\(\): aggregate functions are not allowed in ORDER BY in UPDATE +UPDATE t107634 SET a = 1 ORDER BY sum(a) LIMIT 1; + +subtest end diff --git a/pkg/sql/opt/optbuilder/mutation_builder.go b/pkg/sql/opt/optbuilder/mutation_builder.go index 6d9ca0076050..64598a35a3e4 100644 --- a/pkg/sql/opt/optbuilder/mutation_builder.go +++ b/pkg/sql/opt/optbuilder/mutation_builder.go @@ -336,7 +336,8 @@ func (mb *mutationBuilder) buildInputForUpdate( // SELECT + ORDER BY (which may add projected expressions) projectionsScope := mb.outScope.replace() projectionsScope.appendColumnsFromScope(mb.outScope) - orderByScope := mb.b.analyzeOrderBy(orderBy, mb.outScope, projectionsScope, tree.RejectGenerators) + orderByScope := mb.b.analyzeOrderBy(orderBy, mb.outScope, projectionsScope, + exprKindOrderByUpdate, tree.RejectGenerators|tree.RejectAggregates) mb.b.buildOrderBy(mb.outScope, projectionsScope, orderByScope) mb.b.constructProjectForScope(mb.outScope, projectionsScope) @@ -448,7 +449,8 @@ func (mb *mutationBuilder) buildInputForDelete( // SELECT + ORDER BY (which may add projected expressions) projectionsScope := mb.outScope.replace() projectionsScope.appendColumnsFromScope(mb.outScope) - orderByScope := mb.b.analyzeOrderBy(orderBy, mb.outScope, projectionsScope, tree.RejectGenerators) + orderByScope := mb.b.analyzeOrderBy(orderBy, mb.outScope, projectionsScope, + exprKindOrderByDelete, tree.RejectGenerators|tree.RejectAggregates) mb.b.buildOrderBy(mb.outScope, projectionsScope, orderByScope) mb.b.constructProjectForScope(mb.outScope, projectionsScope) diff --git a/pkg/sql/opt/optbuilder/orderby.go b/pkg/sql/opt/optbuilder/orderby.go index bfb049f386ce..7240a232b829 100644 --- a/pkg/sql/opt/optbuilder/orderby.go +++ b/pkg/sql/opt/optbuilder/orderby.go @@ -28,7 +28,10 @@ import ( // analyzeOrderBy analyzes an Ordering physical property from the ORDER BY // clause and adds the resulting typed expressions to orderByScope. func (b *Builder) analyzeOrderBy( - orderBy tree.OrderBy, inScope, projectionsScope *scope, rejectFlags tree.SemaRejectFlags, + orderBy tree.OrderBy, + inScope, projectionsScope *scope, + kind exprKind, + rejectFlags tree.SemaRejectFlags, ) (orderByScope *scope) { if orderBy == nil { return nil @@ -41,8 +44,8 @@ func (b *Builder) analyzeOrderBy( // semaCtx in case we are recursively called within a subquery // context. defer b.semaCtx.Properties.Restore(b.semaCtx.Properties) - b.semaCtx.Properties.Require(exprKindOrderBy.String(), rejectFlags) - inScope.context = exprKindOrderBy + b.semaCtx.Properties.Require(kind.String(), rejectFlags) + inScope.context = kind for i := range orderBy { b.analyzeOrderByArg(orderBy[i], inScope, projectionsScope, orderByScope) diff --git a/pkg/sql/opt/optbuilder/scope.go b/pkg/sql/opt/optbuilder/scope.go index f0f18abf3776..ad44fab80ab3 100644 --- a/pkg/sql/opt/optbuilder/scope.go +++ b/pkg/sql/opt/optbuilder/scope.go @@ -132,6 +132,8 @@ const ( exprKindOffset exprKindOn exprKindOrderBy + exprKindOrderByDelete + exprKindOrderByUpdate exprKindReturning exprKindSelect exprKindStoreID @@ -153,6 +155,8 @@ var exprKindName = [...]string{ exprKindOffset: "OFFSET", exprKindOn: "ON", exprKindOrderBy: "ORDER BY", + exprKindOrderByDelete: "ORDER BY in DELETE", + exprKindOrderByUpdate: "ORDER BY in UPDATE", exprKindReturning: "RETURNING", exprKindSelect: "SELECT", exprKindStoreID: "RELOCATE STORE ID", diff --git a/pkg/sql/opt/optbuilder/select.go b/pkg/sql/opt/optbuilder/select.go index 2c6122b35213..e64df3e57b47 100644 --- a/pkg/sql/opt/optbuilder/select.go +++ b/pkg/sql/opt/optbuilder/select.go @@ -1106,7 +1106,8 @@ func (b *Builder) buildSelectStmtWithoutParens( col := projectionsScope.addColumn(scopeColName(""), expr) b.buildScalar(expr, outScope, projectionsScope, col, nil) } - orderByScope := b.analyzeOrderBy(orderBy, outScope, projectionsScope, tree.RejectGenerators|tree.RejectAggregates|tree.RejectWindowApplications) + orderByScope := b.analyzeOrderBy(orderBy, outScope, projectionsScope, exprKindOrderBy, + tree.RejectGenerators|tree.RejectAggregates|tree.RejectWindowApplications) b.buildOrderBy(outScope, projectionsScope, orderByScope) b.constructProjectForScope(outScope, projectionsScope) outScope = projectionsScope @@ -1151,7 +1152,8 @@ func (b *Builder) buildSelectClause( // Any aggregates in the HAVING, ORDER BY and DISTINCT ON clauses (if they // exist) will be added here. havingExpr := b.analyzeHaving(sel.Having, fromScope) - orderByScope := b.analyzeOrderBy(orderBy, fromScope, projectionsScope, tree.RejectGenerators) + orderByScope := b.analyzeOrderBy(orderBy, fromScope, projectionsScope, + exprKindOrderBy, tree.RejectGenerators) distinctOnScope := b.analyzeDistinctOnArgs(sel.DistinctOn, fromScope, projectionsScope) var having opt.ScalarExpr diff --git a/pkg/sql/opt/optbuilder/testdata/delete b/pkg/sql/opt/optbuilder/testdata/delete index 8c354e8c2b46..fd6c9ce333c5 100644 --- a/pkg/sql/opt/optbuilder/testdata/delete +++ b/pkg/sql/opt/optbuilder/testdata/delete @@ -299,6 +299,12 @@ DELETE FROM abcde WHERE b=1 ORDER BY c ---- error (42601): DELETE statement requires LIMIT when ORDER BY is used +# Aggregate functions are not allowed in ORDER BY. +build +DELETE FROM abcde WHERE b=1 ORDER BY sum(c) LIMIT 1 +---- +error (42803): sum(): aggregate functions are not allowed in ORDER BY in DELETE + # ------------------------------------------------------------------------------ # Test RETURNING. # ------------------------------------------------------------------------------ diff --git a/pkg/sql/opt/optbuilder/testdata/update b/pkg/sql/opt/optbuilder/testdata/update index 471d2ab42663..d45ee13ba9a4 100644 --- a/pkg/sql/opt/optbuilder/testdata/update +++ b/pkg/sql/opt/optbuilder/testdata/update @@ -617,6 +617,12 @@ UPDATE abcde SET b=1 ORDER BY c ---- error (42601): UPDATE statement requires LIMIT when ORDER BY is used +# Aggregate functions are not allowed in ORDER BY. +build +UPDATE abcde SET b=1 ORDER BY sum(c) LIMIT 1 +---- +error (42803): sum(): aggregate functions are not allowed in ORDER BY in UPDATE + # ------------------------------------------------------------------------------ # Test RETURNING. # ------------------------------------------------------------------------------