Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
107570: sql,jobs: short-term fix for `UndefinedColumn` job_type error r=HonoreDB a=adityamaru

In #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 #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

107641: sql: disallow aggregate functions in ORDER BY in DELETE and UPDATE r=mgartner a=mgartner

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.


Co-authored-by: adityamaru <[email protected]>
Co-authored-by: Marcus Gartner <[email protected]>
  • Loading branch information
3 people committed Jul 26, 2023
3 parents d93799f + ac55715 + b3a90bd commit 1d3c11e
Show file tree
Hide file tree
Showing 12 changed files with 87 additions and 10 deletions.
1 change: 1 addition & 0 deletions pkg/jobs/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
32 changes: 32 additions & 0 deletions pkg/jobs/utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
)
Expand Down Expand Up @@ -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
}
4 changes: 2 additions & 2 deletions pkg/sql/crdb_internal.go
Original file line number Diff line number Diff line change
Expand Up @@ -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) {
Expand All @@ -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()
Expand Down
11 changes: 11 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/delete
Original file line number Diff line number Diff line change
Expand Up @@ -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
11 changes: 11 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/update
Original file line number Diff line number Diff line change
Expand Up @@ -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
6 changes: 4 additions & 2 deletions pkg/sql/opt/optbuilder/mutation_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)

Expand Down Expand Up @@ -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)

Expand Down
9 changes: 6 additions & 3 deletions pkg/sql/opt/optbuilder/orderby.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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)
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/opt/optbuilder/scope.go
Original file line number Diff line number Diff line change
Expand Up @@ -132,6 +132,8 @@ const (
exprKindOffset
exprKindOn
exprKindOrderBy
exprKindOrderByDelete
exprKindOrderByUpdate
exprKindReturning
exprKindSelect
exprKindStoreID
Expand All @@ -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",
Expand Down
6 changes: 4 additions & 2 deletions pkg/sql/opt/optbuilder/select.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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
Expand Down
6 changes: 6 additions & 0 deletions pkg/sql/opt/optbuilder/testdata/delete
Original file line number Diff line number Diff line change
Expand Up @@ -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.
# ------------------------------------------------------------------------------
Expand Down
6 changes: 6 additions & 0 deletions pkg/sql/opt/optbuilder/testdata/update
Original file line number Diff line number Diff line change
Expand Up @@ -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.
# ------------------------------------------------------------------------------
Expand Down
1 change: 0 additions & 1 deletion pkg/upgrade/upgrades/json_forward_indexes_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)()
Expand Down

0 comments on commit 1d3c11e

Please sign in to comment.