Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
33312: sql/opt: avoid swallowing TransactionAbortedErrors r=benesch a=benesch

An optimizer code path could, in rare cases, fail to propagate a
transaction aborted error. This proved disastrous as, due to a footgun
in our transaction API (#22615), swallowing a transaction aborted error
results in proceeding with a brand new transaction that has no knowledge
of the earlier operations performed on the original transaction.

This presented as a rare and confusing bug in splits, as the split
transaction uses an internal executor. The internal executor would
occasionally silently return a new transaction that only had half of the
necessary operations performed on it, and committing that partial
transaction would result in a "range does not match splits" error.

Fixes #32784.

Release note: None

/cc @tbg

33417: opt: Inline constant values r=andy-kimball a=andy-kimball

Inline constants in expressions like:

  SELECT x, x+1 FROM (VALUES (1)) AS t(x) ;

with the new inlining rules, this becomes:

  VALUES (1, 2)

The new inlining rules are useful for mutation expressions (e.g. UPDATE),
which can nest multiple Project and Values expressions that often use
constant values. For example:

  CREATE TABLE ab (a INT PRIMARY KEY, b INT AS (a + 1) STORED);
  UPDATE ab SET a=1

This now gets mapped by the optimizer to this internal equivalent:

  UPDATE ab SET a=1, b=2

Release note: None

33421: opt: Tighten up InlineProjectInProject rule r=andy-kimball a=andy-kimball

Allow inlining nested Project in case where there are duplicate refs
to same inner passthrough column. Previously, this case prevented
inlining. However, only duplicate references to inner *synthesized*
columns need to be detected.

Release note: None

Co-authored-by: Nikhil Benesch <[email protected]>
Co-authored-by: Andrew Kimball <[email protected]>
  • Loading branch information
3 people committed Jan 3, 2019
4 parents 4cc6356 + 0285c3c + c920fe0 + d712a0a commit f3eed4a
Show file tree
Hide file tree
Showing 15 changed files with 672 additions and 150 deletions.
3 changes: 2 additions & 1 deletion pkg/sql/logictest/testdata/logic_test/prepare
Original file line number Diff line number Diff line change
Expand Up @@ -739,7 +739,8 @@ EXECUTE change_view
statement ok
ALTER VIEW otherview RENAME TO otherview2

query error pq: relation "otherview" does not exist
# HP and CBO return slightly different errors, so accept both.
query error pq: relation "(otherdb.public.)?otherview" does not exist
EXECUTE change_view

statement ok
Expand Down
8 changes: 3 additions & 5 deletions pkg/sql/opt/exec/execbuilder/testdata/explain
Original file line number Diff line number Diff line change
Expand Up @@ -605,11 +605,9 @@ values · · (a int) ·
query TTTTT
EXPLAIN (TYPES) SELECT x[1] FROM (SELECT ARRAY[1,2,3] AS x)
----
render · · (x int) ·
│ render 0 ((x)[int[]][(1)[int]])[int] · ·
└── values · · (x int[]) ·
· size 1 column, 1 row · ·
· row 0, expr 0 (ARRAY[(1)[int],(2)[int],(3)[int]])[int[]] · ·
values · · (x int) ·
· size 1 column, 1 row · ·
· row 0, expr 0 (1)[int] · ·

query T
EXPLAIN (OPT) SELECT 1 AS r
Expand Down
29 changes: 18 additions & 11 deletions pkg/sql/opt/exec/execbuilder/testdata/orderby
Original file line number Diff line number Diff line change
Expand Up @@ -733,17 +733,24 @@ render · · (k) ·
#

query TTTTT
EXPLAIN (VERBOSE) SELECT k FROM kv JOIN (VALUES (1,2)) AS z(a,b) ON kv.k = z.a ORDER BY INDEX kv@foo
----
render · · (k) ·
│ render 0 k · ·
└── lookup-join · · (column1, k) ·
│ type inner · ·
├── values · · (column1) ·
│ size 1 column, 1 row · ·
│ row 0, expr 0 1 · ·
└── scan · · (k) ·
· table kv@primary · ·
EXPLAIN (VERBOSE)
SELECT k FROM kv JOIN (VALUES (1,2), (3,4)) AS z(a,b) ON kv.k = z.a ORDER BY INDEX kv@foo
----
render · · (k) ·
│ render 0 k · ·
└── sort · · (k, v) -v,+k
│ order -v,+k · ·
└── render · · (k, v) ·
│ render 0 k · ·
│ render 1 v · ·
└── lookup-join · · (column1, k, v) ·
│ type inner · ·
├── values · · (column1) ·
│ size 1 column, 2 rows · ·
│ row 0, expr 0 1 · ·
│ row 1, expr 0 3 · ·
└── scan · · (k, v) ·
· table kv@primary · ·

query TTTTT
EXPLAIN (VERBOSE) SELECT k FROM kv a NATURAL JOIN kv ORDER BY INDEX kv@foo
Expand Down
27 changes: 10 additions & 17 deletions pkg/sql/opt/exec/execbuilder/testdata/select
Original file line number Diff line number Diff line change
Expand Up @@ -984,14 +984,11 @@ CREATE TABLE abcd (a INT, b INT, c INT, d INT, PRIMARY KEY (a, b))
query TTTTT
EXPLAIN (VERBOSE) SELECT a + x FROM (SELECT a, b + c AS x FROM abcd) ORDER BY a
----
render · · ("?column?") ·
│ render 0 a + x · ·
└── render · · (x, a) +a
│ render 0 b + c · ·
│ render 1 a · ·
└── scan · · (a, b, c) +a
· table abcd@primary · ·
· spans ALL · ·
render · · ("?column?") ·
│ render 0 a + (b + c) · ·
└── scan · · (a, b, c) +a
· table abcd@primary · ·
· spans ALL · ·

query TTTTT
EXPLAIN (VERBOSE) SELECT a + x FROM (SELECT a, b, a + b + c AS x FROM abcd) ORDER BY b
Expand All @@ -1011,15 +1008,11 @@ render · · ("?column?") ·
query TTTTT
EXPLAIN (VERBOSE) SELECT a + x FROM (SELECT a, b, a + b + c AS x FROM abcd) ORDER BY a DESC, b DESC
----
render · · ("?column?") ·
│ render 0 a + x · ·
└── render · · (x, a, b) -a,-b
│ render 0 c + (a + b) · ·
│ render 1 a · ·
│ render 2 b · ·
└── revscan · · (a, b, c) -a,-b
· table abcd@primary · ·
· spans ALL · ·
render · · ("?column?") ·
│ render 0 a + (c + (a + b)) · ·
└── revscan · · (a, b, c) -a,-b
· table abcd@primary · ·
· spans ALL · ·

# Ensure that filter nodes (and filtered scan nodes) get populated with the correct ordering.
query TTTTT
Expand Down
21 changes: 14 additions & 7 deletions pkg/sql/opt/memo/memo.go
Original file line number Diff line number Diff line change
Expand Up @@ -247,29 +247,36 @@ func (m *Memo) HasPlaceholders() bool {
// 5. Data source privileges: current user may no longer have access to one or
// more data sources.
//
func (m *Memo) IsStale(ctx context.Context, evalCtx *tree.EvalContext, catalog cat.Catalog) bool {
// This function cannot swallow errors and return only a boolean, as it may
// perform KV operations on behalf of the transaction associated with the
// provided catalog, and those errors are required to be propagated.
func (m *Memo) IsStale(
ctx context.Context, evalCtx *tree.EvalContext, catalog cat.Catalog,
) (bool, error) {
// Memo is stale if the current database has changed.
if m.dbName != evalCtx.SessionData.Database {
return true
return true, nil
}

// Memo is stale if the search path has changed.
if !m.searchPath.Equals(&evalCtx.SessionData.SearchPath) {
return true
return true, nil
}

// Memo is stale if the location has changed.
if m.locName != evalCtx.GetLocation().String() {
return true
return true, nil
}

// Memo is stale if the fingerprint of any data source in the memo's metadata
// has changed, or if the current user no longer has sufficient privilege to
// access the data source.
if !m.Metadata().CheckDependencies(ctx, catalog) {
return true
if depsUpToDate, err := m.Metadata().CheckDependencies(ctx, catalog); err != nil {
return false, err
} else if !depsUpToDate {
return true, nil
}
return false
return false, nil
}

// InternPhysicalProps adds the given physical props to the memo if they haven't
Expand Down
38 changes: 27 additions & 11 deletions pkg/sql/opt/memo/memo_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,12 +22,13 @@ import (
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/opt/memo"
"github.com/cockroachdb/cockroach/pkg/sql/opt/optbuilder"
"github.com/cockroachdb/cockroach/pkg/sql/opt/testutils"
opttestutils "github.com/cockroachdb/cockroach/pkg/sql/opt/testutils"
"github.com/cockroachdb/cockroach/pkg/sql/opt/testutils/testcat"
"github.com/cockroachdb/cockroach/pkg/sql/opt/xform"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
testutils "github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/datadriven"
)

Expand Down Expand Up @@ -121,31 +122,41 @@ func TestMemoIsStale(t *testing.T) {
if err != nil {
t.Fatal(err)
}
if o.Memo().IsStale(ctx, &evalCtx, catalog) {
if isStale, err := o.Memo().IsStale(ctx, &evalCtx, catalog); err != nil {
t.Fatal(err)
} else if isStale {
t.Errorf("memo should not be stale")
}

// Stale current database.
evalCtx.SessionData.Database = "newdb"
if !o.Memo().IsStale(ctx, &evalCtx, catalog) {
if isStale, err := o.Memo().IsStale(ctx, &evalCtx, catalog); err != nil {
t.Fatal(err)
} else if !isStale {
t.Errorf("expected stale current database")
}
evalCtx.SessionData.Database = "t"

// Stale search path.
evalCtx.SessionData.SearchPath = sessiondata.SearchPath{}
if !o.Memo().IsStale(ctx, &evalCtx, catalog) {
if isStale, err := o.Memo().IsStale(ctx, &evalCtx, catalog); err != nil {
t.Fatal(err)
} else if !isStale {
t.Errorf("expected stale search path")
}
evalCtx.SessionData.SearchPath = sessiondata.MakeSearchPath([]string{"path1", "path2"})
if o.Memo().IsStale(ctx, &evalCtx, catalog) {
if isStale, err := o.Memo().IsStale(ctx, &evalCtx, catalog); err != nil {
t.Fatal(err)
} else if isStale {
t.Errorf("memo should not be stale")
}
evalCtx.SessionData.SearchPath = sessiondata.MakeSearchPath(searchPath)

// Stale location.
evalCtx.SessionData.DataConversion.Location = time.FixedZone("PST", -8*60*60)
if !o.Memo().IsStale(ctx, &evalCtx, catalog) {
if isStale, err := o.Memo().IsStale(ctx, &evalCtx, catalog); err != nil {
t.Fatal(err)
} else if !isStale {
t.Errorf("expected stale location")
}
evalCtx.SessionData.DataConversion.Location = time.UTC
Expand All @@ -159,7 +170,9 @@ func TestMemoIsStale(t *testing.T) {
if err != nil {
t.Fatal(err)
}
if !o.Memo().IsStale(ctx, &evalCtx, catalog) {
if isStale, err := o.Memo().IsStale(ctx, &evalCtx, catalog); err != nil {
t.Fatal(err)
} else if !isStale {
t.Errorf("expected stale schema")
}
catalog = testcat.New()
Expand All @@ -168,13 +181,16 @@ func TestMemoIsStale(t *testing.T) {

// User no longer has access to view.
catalog.View(tree.NewTableName("t", "abcview")).Revoked = true
if !o.Memo().IsStale(ctx, &evalCtx, catalog) {
t.Errorf("expected user not to have SELECT privilege on view")
_, err = o.Memo().IsStale(ctx, &evalCtx, catalog)
if exp := "user does not have privilege"; !testutils.IsError(err, exp) {
t.Fatalf("expected %q error, but got %+v", exp, err)
}
catalog.View(tree.NewTableName("t", "abcview")).Revoked = false

// Ensure that memo is not stale after restoring to original state.
if o.Memo().IsStale(ctx, &evalCtx, catalog) {
if isStale, err := o.Memo().IsStale(ctx, &evalCtx, catalog); err != nil {
t.Fatal(err)
} else if isStale {
t.Errorf("memo should not be stale")
}
}
Expand All @@ -190,7 +206,7 @@ func runDataDrivenTest(t *testing.T, path string, fmtFlags memo.ExprFmtFlags) {
datadriven.Walk(t, path, func(t *testing.T, path string) {
catalog := testcat.New()
datadriven.RunTest(t, path, func(d *datadriven.TestData) string {
tester := testutils.NewOptTester(catalog, d.Input)
tester := opttestutils.NewOptTester(catalog, d.Input)
tester.Flags.ExprFormat = fmtFlags
return tester.RunCommand(t, d)
})
Expand Down
14 changes: 6 additions & 8 deletions pkg/sql/opt/memo/testdata/logprops/join
Original file line number Diff line number Diff line change
Expand Up @@ -838,21 +838,19 @@ semi-join-apply

# Calculate anti-join cardinality when left side has non-zero cardinality.
opt
SELECT * FROM (SELECT * FROM (VALUES (1))) WHERE NOT EXISTS(SELECT * FROM uv WHERE u=column1)
SELECT * FROM (SELECT * FROM (VALUES (1), (2))) WHERE NOT EXISTS(SELECT * FROM uv WHERE u=column1)
----
anti-join
├── columns: column1:1(int)
├── cardinality: [0 - 1]
├── key: ()
├── fd: ()-->(1)
├── cardinality: [0 - 2]
├── values
│ ├── columns: column1:1(int)
│ ├── cardinality: [1 - 1]
│ ├── key: ()
│ ├── fd: ()-->(1)
│ ├── cardinality: [2 - 2]
│ ├── prune: (1)
│ ├── tuple [type=tuple{int}]
│ │ └── const: 1 [type=int]
│ └── tuple [type=tuple{int}]
│ └── const: 1 [type=int]
│ └── const: 2 [type=int]
├── scan uv
│ ├── columns: u:2(int) v:3(int!null)
│ └── prune: (2,3)
Expand Down
16 changes: 10 additions & 6 deletions pkg/sql/opt/metadata.go
Original file line number Diff line number Diff line change
Expand Up @@ -119,17 +119,21 @@ func (md *Metadata) AddDependency(ds cat.DataSource, priv privilege.Kind) {
// in order to check that the fully qualified data source names still resolve to
// the same version of the same data source, and that the user still has
// sufficient privileges to access the data source.
func (md *Metadata) CheckDependencies(ctx context.Context, catalog cat.Catalog) bool {
//
// This function cannot swallow errors and return only a boolean, as it may
// perform KV operations on behalf of the transaction associated with the
// provided catalog, and those errors are required to be propagated.
func (md *Metadata) CheckDependencies(ctx context.Context, catalog cat.Catalog) (bool, error) {
for dep, privs := range md.deps {
ds, err := catalog.ResolveDataSource(ctx, dep.Name())
if err != nil {
return false
return false, err
}
if dep.ID() != ds.ID() {
return false
return false, nil
}
if dep.Version() != ds.Version() {
return false
return false, nil
}

for privs != 0 {
Expand All @@ -140,15 +144,15 @@ func (md *Metadata) CheckDependencies(ctx context.Context, catalog cat.Catalog)
priv := privilege.Kind(bits.TrailingZeros32(uint32(privs)))
if priv != 0 {
if err = catalog.CheckPrivilege(ctx, ds, priv); err != nil {
return false
return false, err
}
}

// Set the just-handled privilege bit to zero and look for next.
privs &= ^(1 << priv)
}
}
return true
return true, nil
}

// AddTable indexes a new reference to a table within the query. Separate
Expand Down
11 changes: 11 additions & 0 deletions pkg/sql/opt/norm/custom_funcs.go
Original file line number Diff line number Diff line change
Expand Up @@ -277,6 +277,11 @@ func (c *CustomFuncs) CanHaveZeroRows(input memo.RelExpr) bool {
return input.Relational().Cardinality.CanBeZero()
}

// ColsAreEmpty returns true if the column set is empty.
func (c *CustomFuncs) ColsAreEmpty(cols opt.ColSet) bool {
return cols.Empty()
}

// ColsAreSubset returns true if the left columns are a subset of the right
// columns.
func (c *CustomFuncs) ColsAreSubset(left, right opt.ColSet) bool {
Expand All @@ -288,6 +293,12 @@ func (c *CustomFuncs) ColsAreEqual(left, right opt.ColSet) bool {
return left.Equals(right)
}

// ColsIntersect returns true if at least one column appears in both the left
// and right sets.
func (c *CustomFuncs) ColsIntersect(left, right opt.ColSet) bool {
return left.Intersects(right)
}

// UnionCols returns the union of the left and right column sets.
func (c *CustomFuncs) UnionCols(left, right opt.ColSet) opt.ColSet {
return left.Union(right)
Expand Down
Loading

0 comments on commit f3eed4a

Please sign in to comment.