Skip to content

Commit

Permalink
opt, sql: fix type inference of TypeCheck for subqueries
Browse files Browse the repository at this point in the history
Prior to this commit, the optimizer was not correctly inferring the types of
columns in subqueries for expressions of the form `scalar IN (subquery)`.
This was due to two problems which have now been fixed:

1. The subquery was built as a relational expression before the desired types
   were known. Now the subquery build is delayed until TypeCheck is called for
   the first time.

2. For subqueries on the right side of an IN expression, the desired type
   passed into TypeCheck was AnyTuple. This resulted in an error later on in
   typeCheckSubqueryWithIn, which checks to make sure the type of the subquery
   is tuple{T} where T is the type of the left expression. Now the desired
   type passed into TypeCheck is tuple{T}.

Note that this commit only fixes type inference for the optimizer. It is still
broken in the heuristic planner.

Fixes #37263
Fixes #14554

Release note (bug fix): Fixed type inference of columns in subqueries for
some expressions of the form `scalar IN (subquery)`.
  • Loading branch information
rytaft committed May 20, 2019
1 parent f9a8fee commit 6d7188c
Show file tree
Hide file tree
Showing 10 changed files with 262 additions and 118 deletions.
26 changes: 26 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/subquery-opt
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
# LogicTest: local-opt fakedist-opt

# Regression test for #37263. This test is broken in the heuristic planner
# because it does not correctly type check subqueries.
query B
SELECT 3::decimal IN (SELECT 1)
----
false

query error unsupported comparison operator
SELECT 3::decimal IN (SELECT 1::int)

query B
SELECT 1 IN (SELECT '1');
----
true

# Regression test for #14554.
query ITIIIII
SELECT t.oid, t.typname, t.typsend, t.typreceive, t.typoutput, t.typinput, t.typelem
FROM pg_type AS t
WHERE t.oid NOT IN (
SELECT (ARRAY[704,11676,10005,3912,11765,59410,11397])[i]
FROM generate_series(1, 376) AS i
)
----
4 changes: 3 additions & 1 deletion pkg/sql/logictest/testdata/logic_test/typing
Original file line number Diff line number Diff line change
Expand Up @@ -167,7 +167,9 @@ SELECT 1 in (SELECT 1)
----
true

statement error unsupported comparison operator: <int> IN <tuple{string}>
# The heuristic planner and the optimizer give different errors for this query.
# Accept them both.
statement error (unsupported comparison operator: <int> IN <tuple{string}>|could not parse "a" as type int)
SELECT 1 IN (SELECT 'a')

statement error unsupported comparison operator: <int> IN <tuple{tuple{int, int}}>
Expand Down
17 changes: 10 additions & 7 deletions pkg/sql/opt/optbuilder/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/transform"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
)

// Builder holds the context needed for building a memo structure from a SQL
Expand Down Expand Up @@ -154,7 +155,7 @@ func (b *Builder) Build() (err error) {

// Build the memo, and call SetRoot on the memo to indicate the root group
// and physical properties.
outScope := b.buildStmt(b.stmt, b.allocScope())
outScope := b.buildStmt(b.stmt, nil /* desiredTypes */, b.allocScope())
physical := outScope.makePhysicalProps()
b.factory.Memo().SetRoot(outScope.expr, physical)
return nil
Expand Down Expand Up @@ -192,9 +193,11 @@ func unimplementedWithIssueDetailf(
//
// outScope This return value contains the newly bound variables that will be
// visible to enclosing statements, as well as a pointer to any
// "parent" scope that is still visible. The top-level memo group ID
// for the built statement/expression is returned in outScope.group.
func (b *Builder) buildStmt(stmt tree.Statement, inScope *scope) (outScope *scope) {
// "parent" scope that is still visible. The top-level memo expression
// for the built statement/expression is returned in outScope.expr.
func (b *Builder) buildStmt(
stmt tree.Statement, desiredTypes []*types.T, inScope *scope,
) (outScope *scope) {
// NB: The case statements are sorted lexicographically.
switch stmt := stmt.(type) {
case *tree.CreateTable:
Expand All @@ -210,10 +213,10 @@ func (b *Builder) buildStmt(stmt tree.Statement, inScope *scope) (outScope *scop
return b.buildInsert(stmt, inScope)

case *tree.ParenSelect:
return b.buildSelect(stmt.Select, nil /* desiredTypes */, inScope)
return b.buildSelect(stmt.Select, desiredTypes, inScope)

case *tree.Select:
return b.buildSelect(stmt, nil /* desiredTypes */, inScope)
return b.buildSelect(stmt, desiredTypes, inScope)

case *tree.ShowTraceForSession:
return b.buildShowTrace(stmt, inScope)
Expand All @@ -231,7 +234,7 @@ func (b *Builder) buildStmt(stmt tree.Statement, inScope *scope) (outScope *scop
// register all those dependencies with the metadata (for cache
// invalidation). We don't care about caching plans for these statements.
b.DisableMemoReuse = true
return b.buildStmt(newStmt, inScope)
return b.buildStmt(newStmt, desiredTypes, inScope)
}
panic(unimplementedWithIssueDetailf(34848, stmt.StatementTag(), "unsupported statement: %T", stmt))
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/optbuilder/explain.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ func (b *Builder) buildExplain(explain *tree.Explain, inScope *scope) (outScope

// We don't allow the statement under Explain to reference outer columns, so we
// pass a "blank" scope rather than inScope.
stmtScope := b.buildStmt(explain.Statement, &scope{builder: b})
stmtScope := b.buildStmt(explain.Statement, nil /* desiredTypes */, &scope{builder: b})
outScope = inScope.push()

var cols sqlbase.ResultColumns
Expand Down
84 changes: 24 additions & 60 deletions pkg/sql/opt/optbuilder/scope.go
Original file line number Diff line number Diff line change
Expand Up @@ -861,7 +861,9 @@ func (s *scope) VisitPre(expr tree.Expr) (recurse bool, newExpr tree.Expr) {
if sub, ok := t.Subquery.(*tree.Subquery); ok {
// Copy the ArrayFlatten expression so that the tree isn't mutated.
copy := *t
copy.Subquery = s.replaceSubquery(sub, false /* wrapInTuple */, 1 /* desiredColumns */, extraColsAllowed)
copy.Subquery = s.replaceSubquery(
sub, false /* wrapInTuple */, 1 /* desiredNumColumns */, extraColsAllowed,
)
expr = &copy
}

Expand All @@ -876,7 +878,9 @@ func (s *scope) VisitPre(expr tree.Expr) (recurse bool, newExpr tree.Expr) {
if sub, ok := t.Right.(*tree.Subquery); ok {
// Copy the Comparison expression so that the tree isn't mutated.
copy := *t
copy.Right = s.replaceSubquery(sub, true /* wrapInTuple */, -1 /* desiredColumns */, noExtraColsAllowed)
copy.Right = s.replaceSubquery(
sub, true /* wrapInTuple */, -1 /* desiredNumColumns */, noExtraColsAllowed,
)
expr = &copy
}
}
Expand All @@ -888,9 +892,13 @@ func (s *scope) VisitPre(expr tree.Expr) (recurse bool, newExpr tree.Expr) {
}

if t.Exists {
expr = s.replaceSubquery(t, true /* wrapInTuple */, -1 /* desiredColumns */, noExtraColsAllowed)
expr = s.replaceSubquery(
t, true /* wrapInTuple */, -1 /* desiredNumColumns */, noExtraColsAllowed,
)
} else {
expr = s.replaceSubquery(t, false /* wrapInTuple */, s.columns /* desiredColumns */, noExtraColsAllowed)
expr = s.replaceSubquery(
t, false /* wrapInTuple */, s.columns /* desiredNumColumns */, noExtraColsAllowed,
)
}
}

Expand Down Expand Up @@ -1146,72 +1154,28 @@ const (
noExtraColsAllowed = false
)

// Replace a raw subquery node with a typed subquery. wrapInTuple specifies
// whether the return type of the subquery should be wrapped in a tuple.
// wrapInTuple is true for subqueries that may return multiple rows in
// Replace a raw tree.Subquery node with a lazily typed subquery. wrapInTuple
// specifies whether the return type of the subquery should be wrapped in a
// tuple. wrapInTuple is true for subqueries that may return multiple rows in
// comparison expressions (e.g., IN, ANY, ALL) and EXISTS expressions.
// desiredColumns specifies the desired number of columns for the
// subquery. Specifying -1 for desiredColumns allows the subquery to return any
// desiredNumColumns specifies the desired number of columns for the subquery.
// Specifying -1 for desiredNumColumns allows the subquery to return any
// number of columns and is used when the normal type checking machinery will
// verify that the correct number of columns is returned.
// If extraColsAllowed is true, extra columns built from the subquery (such as
// columns for which orderings have been requested) will not be stripped away.
// It is the duty of the caller to ensure that those columns are eventually
// dealt with.
func (s *scope) replaceSubquery(
sub *tree.Subquery, wrapInTuple bool, desiredColumns int, extraColsAllowed bool,
sub *tree.Subquery, wrapInTuple bool, desiredNumColumns int, extraColsAllowed bool,
) *subquery {
if s.replaceSRFs {
// We need to save and restore the previous value of the replaceSRFs field in
// case we are recursively called within a subquery context.
defer func() { s.replaceSRFs = true }()
s.replaceSRFs = false
return &subquery{
Subquery: sub,
wrapInTuple: wrapInTuple,
desiredNumColumns: desiredNumColumns,
extraColsAllowed: extraColsAllowed,
scope: s,
}

subq := subquery{
Subquery: sub,
wrapInTuple: wrapInTuple,
}

// Save and restore the previous value of s.builder.subquery in case we are
// recursively called within a subquery context.
outer := s.builder.subquery
defer func() { s.builder.subquery = outer }()
s.builder.subquery = &subq

outScope := s.builder.buildStmt(sub.Select, s)
ord := outScope.ordering

// Treat the subquery result as an anonymous data source (i.e. column names
// are not qualified). Remove hidden columns, as they are not accessible
// outside the subquery.
outScope.setTableAlias("")
outScope.removeHiddenCols()

if desiredColumns > 0 && len(outScope.cols) != desiredColumns {
n := len(outScope.cols)
switch desiredColumns {
case 1:
panic(pgerror.Newf(pgerror.CodeSyntaxError,
"subquery must return only one column, found %d", n))
default:
panic(pgerror.Newf(pgerror.CodeSyntaxError,
"subquery must return %d columns, found %d", desiredColumns, n))
}
}

if len(outScope.extraCols) > 0 && !extraColsAllowed {
// We need to add a projection to remove the extra columns.
projScope := outScope.push()
projScope.appendColumnsFromScope(outScope)
projScope.expr = s.builder.constructProject(outScope.expr.(memo.RelExpr), projScope.cols)
outScope = projScope
}

subq.cols = outScope.cols
subq.node = outScope.expr.(memo.RelExpr)
subq.ordering = ord
return &subq
}

// VisitPost is part of the Visitor interface.
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/opt/optbuilder/select.go
Original file line number Diff line number Diff line change
Expand Up @@ -108,7 +108,7 @@ func (b *Builder) buildDataSource(
return b.buildZip(source.Items, inScope)

case *tree.Subquery:
outScope = b.buildStmt(source.Select, inScope)
outScope = b.buildStmt(source.Select, nil /* desiredTypes */, inScope)

// Treat the subquery result as an anonymous data source (i.e. column names
// are not qualified). Remove hidden columns, as they are not accessible
Expand All @@ -119,7 +119,7 @@ func (b *Builder) buildDataSource(
return outScope

case *tree.StatementSource:
outScope = b.buildStmt(source.Statement, inScope)
outScope = b.buildStmt(source.Statement, nil /* desiredTypes */, inScope)
if len(outScope.cols) == 0 {
panic(pgerror.Newf(pgerror.CodeUndefinedColumnError,
"statement source \"%v\" does not return any columns", source.Statement))
Expand Down Expand Up @@ -447,7 +447,7 @@ func (b *Builder) buildCTE(ctes []*tree.CTE, inScope *scope) (outScope *scope) {

outScope.ctes = make(map[string]*cteSource)
for i := range ctes {
cteScope := b.buildStmt(ctes[i].Stmt, outScope)
cteScope := b.buildStmt(ctes[i].Stmt, nil /* desiredTypes */, outScope)
cols := cteScope.cols
name := ctes[i].Name.Alias

Expand Down
85 changes: 85 additions & 0 deletions pkg/sql/opt/optbuilder/subquery.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,21 @@ type subquery struct {
// columns which are referenced within the subquery but are bound in an
// outer scope.
outerCols opt.ColSet

// desiredNumColumns specifies the desired number of columns for the subquery.
// Specifying -1 for desiredNumColumns allows the subquery to return any
// number of columns and is used when the normal type checking machinery will
// verify that the correct number of columns is returned.
desiredNumColumns int

// extraColsAllowed indicates that extra columns built from the subquery
// (such as columns for which orderings have been requested) will not be
// stripped away.
extraColsAllowed bool

// scope is the input scope of the subquery. It is needed to lazily build
// the subquery in TypeCheck.
scope *scope
}

// isMultiRow returns whether the subquery can return multiple rows.
Expand All @@ -71,6 +86,20 @@ func (s *subquery) TypeCheck(_ *tree.SemaContext, desired *types.T) (tree.TypedE
return s, nil
}

// Convert desired to an array of desired types for building the subquery.
var desiredTypes []*types.T
tupleContents := desired.TupleContents()
if tupleContents != nil {
desiredTypes = make([]*types.T, len(tupleContents))
for i := range desiredTypes {
desiredTypes[i] = &tupleContents[i]
}
}

// Build the subquery. We cannot build the subquery earlier because we do
// not know the desired types until TypeCheck is called.
s.buildSubquery(desiredTypes)

// The typing for subqueries is complex, but regular.
//
// * If the subquery is part of an EXISTS statement:
Expand Down Expand Up @@ -171,6 +200,57 @@ func (s *subquery) Eval(_ *tree.EvalContext) (tree.Datum, error) {
panic(pgerror.AssertionFailedf("subquery must be replaced before evaluation"))
}

// buildSubquery builds a relational expression that represents this subquery.
// It stores the resulting relational expression in s.node, and also updates
// s.cols and s.ordering with the output columns and ordering of the subquery.
func (s *subquery) buildSubquery(desiredTypes []*types.T) {
if s.scope.replaceSRFs {
// We need to save and restore the previous value of the replaceSRFs field in
// case we are recursively called within a subquery context.
defer func() { s.scope.replaceSRFs = true }()
s.scope.replaceSRFs = false
}

// Save and restore the previous value of s.builder.subquery in case we are
// recursively called within a subquery context.
outer := s.scope.builder.subquery
defer func() { s.scope.builder.subquery = outer }()
s.scope.builder.subquery = s

outScope := s.scope.builder.buildStmt(s.Subquery.Select, desiredTypes, s.scope)
ord := outScope.ordering

// Treat the subquery result as an anonymous data source (i.e. column names
// are not qualified). Remove hidden columns, as they are not accessible
// outside the subquery.
outScope.setTableAlias("")
outScope.removeHiddenCols()

if s.desiredNumColumns > 0 && len(outScope.cols) != s.desiredNumColumns {
n := len(outScope.cols)
switch s.desiredNumColumns {
case 1:
panic(pgerror.Newf(pgerror.CodeSyntaxError,
"subquery must return only one column, found %d", n))
default:
panic(pgerror.Newf(pgerror.CodeSyntaxError,
"subquery must return %d columns, found %d", s.desiredNumColumns, n))
}
}

if len(outScope.extraCols) > 0 && !s.extraColsAllowed {
// We need to add a projection to remove the extra columns.
projScope := outScope.push()
projScope.appendColumnsFromScope(outScope)
projScope.expr = s.scope.builder.constructProject(outScope.expr.(memo.RelExpr), projScope.cols)
outScope = projScope
}

s.cols = outScope.cols
s.node = outScope.expr.(memo.RelExpr)
s.ordering = ord
}

// buildSubqueryProjection ensures that a subquery returns exactly one column.
// If the original subquery has more than one column, buildSubqueryProjection
// wraps it in a projection which has a single tuple column containing all the
Expand Down Expand Up @@ -309,3 +389,8 @@ func (b *Builder) buildMultiRowSubquery(

var _ tree.Expr = &subquery{}
var _ tree.TypedExpr = &subquery{}

// SubqueryExpr implements the SubqueryExpr interface.
func (*subquery) SubqueryExpr() {}

var _ tree.SubqueryExpr = &subquery{}
Loading

0 comments on commit 6d7188c

Please sign in to comment.