Skip to content

Commit

Permalink
Merge pull request #10799 from knz/fix-ord
Browse files Browse the repository at this point in the history
sql: fix the handling of sorting, grouping and column ordinals.
  • Loading branch information
knz committed Nov 22, 2016
2 parents a14f174 + 68502db commit a83c960
Show file tree
Hide file tree
Showing 17 changed files with 1,042 additions and 976 deletions.
2 changes: 1 addition & 1 deletion pkg/sql/analyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -1624,7 +1624,7 @@ func (p *planner) analyzeExpr(
if sources == nil {
resolved = replaced
} else {
resolved, err = p.resolveNames(replaced, sources, iVarHelper)
resolved, _, err = p.resolveNames(replaced, sources, iVarHelper)
if err != nil {
return nil, err
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,7 @@ func parseAndNormalizeExpr(t *testing.T, sql string, sel *selectNode) parser.Typ

// Perform name resolution because {analyze,simplify}Expr want
// expressions containing IndexedVars.
if expr, err = sel.resolveNames(expr); err != nil {
if expr, _, err = sel.resolveNames(expr); err != nil {
t.Fatalf("%s: %v", sql, err)
}
typedExpr, err := parser.TypeCheck(expr, nil, parser.NoTypePreference)
Expand Down
62 changes: 0 additions & 62 deletions pkg/sql/data_source.go
Original file line number Diff line number Diff line change
Expand Up @@ -484,68 +484,6 @@ func (src *dataSourceInfo) expandStar(
return columns, exprs, nil
}

// findUnaliasedColumn looks up the column specified by a VarName, not
// taking column renames into account (but table renames will be taken
// into account). That is, given a table "blah" with single column "y",
// findUnaliasedColumn("y") returns a valid index even in the context
// of:
// SELECT * FROM blah as foo(x)
// If the VarName specifies a table name, only columns that have that
// name as their source alias are considered. If the VarName does not
// specify a table name, all columns in the data source are
// considered. If no column is found, invalidColIdx is returned with
// no error.
func (p *planDataSource) findUnaliasedColumn(c *parser.ColumnItem) (colIdx int, err error) {
colName := c.ColumnName.Normalize()
tableName := c.TableName.NormalizedTableName()

if tableName.Table() != "" {
tn, err := p.info.checkDatabaseName(tableName)
if err != nil {
return invalidColIdx, nil
}
tableName = tn
}

colIdx = invalidColIdx
planColumns := p.plan.Columns()

selCol := func(colIdx int, idx int) (int, error) {
col := planColumns[idx]
if parser.ReNormalizeName(col.Name) == colName {
if colIdx != invalidColIdx {
return invalidColIdx, fmt.Errorf("column reference %q is ambiguous", c)
}
colIdx = idx
}
return colIdx, nil
}

if tableName.Table() == "" {
for idx := 0; idx < len(p.info.sourceColumns); idx++ {
colIdx, err = selCol(colIdx, idx)
if err != nil {
return colIdx, err
}
}
} else {
colRange, ok := p.info.sourceAliases[tableName]
if !ok {
// A table name is specified, but there is no column with this
// table name.
return invalidColIdx, nil
}
for _, idx := range colRange {
colIdx, err = selCol(colIdx, idx)
if err != nil {
return colIdx, err
}
}
}

return colIdx, nil
}

type multiSourceInfo []*dataSourceInfo

// checkDatabaseName checks whether the given TableName is unambiguous
Expand Down
24 changes: 13 additions & 11 deletions pkg/sql/group.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,24 +51,26 @@ func (p *planner) groupBy(n *parser.SelectClause, s *selectNode) (*groupNode, er
// been modified by upper layers.
rawExpr := groupBy[i]

// We do not need to fully analyze the GROUP BY expression here
// (as per analyzeExpr) because this is taken care of by addRender
// below.
resolved, err := p.resolveNames(groupBy[i], s.sourceInfo, s.ivarHelper)
expr := parser.StripParens(rawExpr)

// Check whether the GROUP BY clause refers to a rendered column
// specified in the original query.
col, err := p.colIndex(s.numOriginalCols, expr, "GROUP BY")
if err != nil {
return nil, err
}

// If a col index is specified, replace it with that expression first.
// NB: This is not a deep copy, and thus when extractAggregatesVisitor runs
// on s.render, the GroupBy expressions can contain wrapped IndexedVars.
// aggregateFuncHolder's Eval() method handles being called during grouping.
if col, err := colIndex(s.numOriginalCols, resolved); err != nil {
return nil, err
} else if col >= 0 {
if col != -1 {
groupBy[i] = s.render[col]
rawExpr = n.Exprs[col].Expr
} else {
// We do not need to fully analyze the GROUP BY expression here
// (as per analyzeExpr) because this is taken care of by addRender
// below.
resolved, _, err := p.resolveNames(expr, s.sourceInfo, s.ivarHelper)
if err != nil {
return nil, err
}
groupBy[i] = resolved
}

Expand Down
59 changes: 11 additions & 48 deletions pkg/sql/parser/constant.go
Original file line number Diff line number Diff line change
Expand Up @@ -132,31 +132,31 @@ func (expr *NumVal) Format(buf *bytes.Buffer, f FmtFlags) {
// 1.1 = no
// 123...overflow...456 = no
func (expr *NumVal) canBeInt64() bool {
_, err := expr.asInt64()
_, err := expr.AsInt64()
return err == nil
}

// shouldBeInt64 checks if the value naturally is an int64:
// ShouldBeInt64 checks if the value naturally is an int64:
// 1 = yes
// 1.0 = no
// 1.1 = no
// 123...overflow...456 = no
//
// Currently unused so commented out, but useful even just for
// its documentation value.
// func (expr *NumVal) shouldBeInt64() bool {
// return expr.Kind() == constant.Int && expr.canBeInt64()
// }
func (expr *NumVal) ShouldBeInt64() bool {
return expr.Kind() == constant.Int && expr.canBeInt64()
}

// These errors are statically allocated, because they are returned in the
// common path of asInt64.
// common path of AsInt64.
var errConstNotInt = errors.New("cannot represent numeric constant as an int")
var errConstOutOfRange = errors.New("numeric constant out of int64 range")

// asInt64 returns the value as a 64-bit integer if possible, or returns an
// AsInt64 returns the value as a 64-bit integer if possible, or returns an
// error if not possible. The method will set expr.resInt to the value of
// this int64 if it is successful, avoiding the need to call the method again.
func (expr *NumVal) asInt64() (int64, error) {
func (expr *NumVal) AsInt64() (int64, error) {
intVal, ok := expr.asConstantInt()
if !ok {
return 0, errConstNotInt
Expand Down Expand Up @@ -202,9 +202,9 @@ func (expr *NumVal) AvailableTypes() []Type {
func (expr *NumVal) ResolveAsType(ctx *SemaContext, typ Type) (Datum, error) {
switch typ {
case TypeInt:
// We may have already set expr.resInt in asInt64.
// We may have already set expr.resInt in AsInt64.
if expr.resInt == 0 {
if _, err := expr.asInt64(); err != nil {
if _, err := expr.AsInt64(); err != nil {
return nil, err
}
}
Expand Down Expand Up @@ -430,7 +430,7 @@ func (constantFolderVisitor) VisitPost(expr Expr) (retExpr Expr) {
}
if token, ok := binaryShiftOpToToken[t.Operator]; ok {
if lInt, ok := l.asConstantInt(); ok {
if rInt64, err := r.asInt64(); err == nil && rInt64 >= 0 {
if rInt64, err := r.AsInt64(); err == nil && rInt64 >= 0 {
return &NumVal{Value: constant.Shift(lInt, token, uint(rInt64))}
}
}
Expand Down Expand Up @@ -491,40 +491,3 @@ func foldConstantLiterals(expr Expr) (Expr, error) {
expr, _ = WalkExpr(v, expr)
return expr, nil
}

type constantTypeVisitor struct {
cfv constantFolderVisitor
}

var _ Visitor = constantTypeVisitor{}

func (constantTypeVisitor) VisitPre(expr Expr) (recurse bool, newExpr Expr) {
switch t := expr.(type) {
case Constant:
typedConst, err := t.TypeCheck(nil, nil)
if err != nil {
panic(err)
}
return false, typedConst
}
return true, expr
}

func (constantTypeVisitor) VisitPost(expr Expr) (retExpr Expr) { return expr }

// TypeConstants type checks all Constant literal expressions, resolving
// them as the Datum representations of their values. Before doing so,
// it first folds all numeric constants.
//
// This means that Constants will become TypedExprs so that they can be
// used in contexts which expect a TypedExpr tree (such as Normalization).
// As such, the function is primarily intended for use while testing
// expressions where full type checking is not desired.
//
// TODO(nvanbenschoten) Can this visitor be preallocated (like normalizeVisitor)?
func TypeConstants(expr Expr) (TypedExpr, error) {
v := constantTypeVisitor{}
expr, _ = WalkExpr(v.cfv, expr)
expr, _ = WalkExpr(v, expr)
return expr.(TypedExpr), nil
}
6 changes: 6 additions & 0 deletions pkg/sql/parser/format.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
type fmtFlags struct {
showTypes bool
showTableAliases bool
symbolicVars bool
// tableNameNormalizer will be called on all NormalizableTableNames if it is
// non-nil. Its results will be used if they are non-nil, or ignored if they
// are nil.
Expand All @@ -46,6 +47,11 @@ var FmtQualify FmtFlags = &fmtFlags{showTableAliases: true}
// annotate expressions with their resolved types.
var FmtShowTypes FmtFlags = &fmtFlags{showTypes: true}

// FmtSymbolicVars instructs the pretty-printer to
// print indexedVars using symbolic notation, to
// disambiguate columns.
var FmtSymbolicVars FmtFlags = &fmtFlags{symbolicVars: true}

// FmtNormalizeTableNames returns FmtFlags that instructs the pretty-printer
// to normalize all table names using the provided function.
func FmtNormalizeTableNames(fn func(*NormalizableTableName) *TableName) FmtFlags {
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/parser/indexed_vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,10 @@ func (v *IndexedVar) ResolvedType() Type {

// Format implements the NodeFormatter interface.
func (v *IndexedVar) Format(buf *bytes.Buffer, f FmtFlags) {
if f.symbolicVars {
fmt.Fprintf(buf, "@%d", v.Idx+1)
return
}
v.container.IndexedVarFormat(buf, f, v.Idx)
}

Expand Down
1 change: 1 addition & 0 deletions pkg/sql/parser/normalize.go
Original file line number Diff line number Diff line change
Expand Up @@ -670,6 +670,7 @@ func (v *containsVarsVisitor) VisitPre(expr Expr) (recurse bool, newExpr Expr) {
func (*containsVarsVisitor) VisitPost(expr Expr) Expr { return expr }

// ContainsVars returns true if the expression contains any variables.
// (variables = sub-expressions, placeholders, indexed vars, etc.)
func ContainsVars(expr Expr) bool {
v := containsVarsVisitor{containsVars: false}
WalkExprConst(&v, expr)
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/parser/scan_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -278,7 +278,7 @@ func TestScanPlaceholder(t *testing.T) {
if id != PLACEHOLDER {
t.Errorf("%s: expected %d, but found %d", d.sql, PLACEHOLDER, id)
}
if i, err := lval.union.numVal().asInt64(); err != nil {
if i, err := lval.union.numVal().AsInt64(); err != nil {
t.Errorf("%s: expected success, but found %v", d.sql, err)
} else if d.expected != i {
t.Errorf("%s: expected %d, but found %d", d.sql, d.expected, i)
Expand Down
Loading

0 comments on commit a83c960

Please sign in to comment.