From 1c1cc244d127d58aeec8b60100a23c16e33f0c8d Mon Sep 17 00:00:00 2001 From: Radu Berinde Date: Wed, 6 Dec 2017 15:21:29 -0500 Subject: [PATCH] opt: introduce optimizer expression trees and build them from TypedExprs This change brings in a subset of https://github.com/petermattis/opttoy/tree/master/v3 This change introduces: - the expr tree: cascades-style optimizers operate on expression trees which can represent both scalar and relational expressions; this is a departure from the way we represent expressions and statements (sem/tree) so we need a new tree structure. - scalar operators: initially, we focus only on scalar expressions. - building an expr tree from a sem/tree.TypedExpr. - opt version of logic tests See the RFC in #19135 for more context on the optimizer. This is the first step of an initial project related to the optimizer: generating index constraints from scalar expressions. This will be a rewrite of the current index constraint generation code (which has many problems, see #6346). Roughly, the existing `makeIndexConstraints` will call into the optimizer with a `TypedExpr` and the optimizer will return index constraints. Release note: None --- pkg/sql/opt/build.go | 165 +++++++++++++++++ pkg/sql/opt/expr.go | 61 ++++++ pkg/sql/opt/misc.go | 21 +++ pkg/sql/opt/operator.go | 111 +++++++++++ pkg/sql/opt/opt_test.go | 299 ++++++++++++++++++++++++++++++ pkg/sql/opt/scalar.go | 123 ++++++++++++ pkg/sql/opt/testdata/build-scalar | 53 ++++++ pkg/sql/opt/tree_print.go | 99 ++++++++++ pkg/sql/opt/tree_print_test.go | 55 ++++++ 9 files changed, 987 insertions(+) create mode 100644 pkg/sql/opt/build.go create mode 100644 pkg/sql/opt/expr.go create mode 100644 pkg/sql/opt/misc.go create mode 100644 pkg/sql/opt/operator.go create mode 100644 pkg/sql/opt/opt_test.go create mode 100644 pkg/sql/opt/scalar.go create mode 100644 pkg/sql/opt/testdata/build-scalar create mode 100644 pkg/sql/opt/tree_print.go create mode 100644 pkg/sql/opt/tree_print_test.go diff --git a/pkg/sql/opt/build.go b/pkg/sql/opt/build.go new file mode 100644 index 000000000000..960d3da0da03 --- /dev/null +++ b/pkg/sql/opt/build.go @@ -0,0 +1,165 @@ +// Copyright 2017 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. + +package opt + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" +) + +// Map from tree.ComparisonOperator to operator. +var comparisonOpMap = [...]operator{ + tree.EQ: eqOp, + tree.LT: ltOp, + tree.GT: gtOp, + tree.LE: leOp, + tree.GE: geOp, + tree.NE: neOp, + tree.In: inOp, + tree.NotIn: notInOp, + tree.Like: likeOp, + tree.NotLike: notLikeOp, + tree.ILike: iLikeOp, + tree.NotILike: notILikeOp, + tree.SimilarTo: similarToOp, + tree.NotSimilarTo: notSimilarToOp, + tree.RegMatch: regMatchOp, + tree.NotRegMatch: notRegMatchOp, + tree.RegIMatch: regIMatchOp, + tree.NotRegIMatch: notRegIMatchOp, + tree.IsDistinctFrom: isDistinctFromOp, + tree.IsNotDistinctFrom: isNotDistinctFromOp, + tree.Is: isOp, + tree.IsNot: isNotOp, + tree.Any: anyOp, + tree.Some: someOp, + tree.All: allOp, +} + +// Map from tree.BinaryOperator to operator. +var binaryOpMap = [...]operator{ + tree.Bitand: bitandOp, + tree.Bitor: bitorOp, + tree.Bitxor: bitxorOp, + tree.Plus: plusOp, + tree.Minus: minusOp, + tree.Mult: multOp, + tree.Div: divOp, + tree.FloorDiv: floorDivOp, + tree.Mod: modOp, + tree.Pow: powOp, + tree.Concat: concatOp, + tree.LShift: lShiftOp, + tree.RShift: rShiftOp, +} + +// Map from tree.UnaryOperator to operator. +var unaryOpMap = [...]operator{ + tree.UnaryPlus: unaryPlusOp, + tree.UnaryMinus: unaryMinusOp, + tree.UnaryComplement: unaryComplementOp, +} + +type buildContext struct { + // We allocate *scalarProps and *expr in chunks. + preallocScalarProps []scalarProps + preallocExprs []expr +} + +const exprAllocChunk = 16 +const scalarPropsAllocChunk = 16 + +func (bc *buildContext) newScalarProps() *scalarProps { + if len(bc.preallocScalarProps) == 0 { + bc.preallocScalarProps = make([]scalarProps, scalarPropsAllocChunk) + } + p := &bc.preallocScalarProps[0] + bc.preallocScalarProps = bc.preallocScalarProps[1:] + return p +} + +// newExpr returns a new *expr with a new, blank scalarProps. +func (bc *buildContext) newExpr() *expr { + if len(bc.preallocExprs) == 0 { + bc.preallocExprs = make([]expr, exprAllocChunk) + } + e := &bc.preallocExprs[0] + bc.preallocExprs = bc.preallocExprs[1:] + e.scalarProps = bc.newScalarProps() + return e +} + +// buildScalar converts a tree.TypedExpr to an expr tree. +func buildScalar(buildCtx *buildContext, pexpr tree.TypedExpr) *expr { + switch t := pexpr.(type) { + case *tree.ParenExpr: + return buildScalar(buildCtx, t.TypedInnerExpr()) + } + + e := buildCtx.newExpr() + e.scalarProps.typ = pexpr.ResolvedType() + + switch t := pexpr.(type) { + case *tree.AndExpr: + initBinaryExpr( + e, andOp, + buildScalar(buildCtx, t.TypedLeft()), + buildScalar(buildCtx, t.TypedRight()), + ) + case *tree.OrExpr: + initBinaryExpr( + e, orOp, + buildScalar(buildCtx, t.TypedLeft()), + buildScalar(buildCtx, t.TypedRight()), + ) + case *tree.NotExpr: + initUnaryExpr(e, notOp, buildScalar(buildCtx, t.TypedInnerExpr())) + + case *tree.BinaryExpr: + initBinaryExpr( + e, binaryOpMap[t.Operator], + buildScalar(buildCtx, t.TypedLeft()), + buildScalar(buildCtx, t.TypedRight()), + ) + case *tree.ComparisonExpr: + initBinaryExpr( + e, comparisonOpMap[t.Operator], + buildScalar(buildCtx, t.TypedLeft()), + buildScalar(buildCtx, t.TypedRight()), + ) + case *tree.UnaryExpr: + initUnaryExpr(e, unaryOpMap[t.Operator], buildScalar(buildCtx, t.TypedInnerExpr())) + + case *tree.FuncExpr: + def, err := t.Func.Resolve(tree.SearchPath{}) + if err != nil { + panic(err.Error()) + } + children := make([]*expr, len(t.Exprs)) + for i, pexpr := range t.Exprs { + children[i] = buildScalar(buildCtx, pexpr.(tree.TypedExpr)) + } + initFunctionExpr(e, def, children) + + case *tree.IndexedVar: + initVariableExpr(e, t.Idx) + + case tree.Datum: + initConstExpr(e, t) + + default: + panicf("node %T not supported", t) + } + return e +} diff --git a/pkg/sql/opt/expr.go b/pkg/sql/opt/expr.go new file mode 100644 index 000000000000..1f14a8e84082 --- /dev/null +++ b/pkg/sql/opt/expr.go @@ -0,0 +1,61 @@ +// Copyright 2017 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. + +package opt + +// expr is a unified interface for both relational and scalar expressions in a +// query. TODO(radu): currently, it only supports scalar expressions. +type expr struct { + op operator + // Child expressions. The interpretation of the children is operator + // dependent. + children []*expr + // Scalar properties. + scalarProps *scalarProps + // Operator-dependent data used by this expression. For example, constOp + // stores a pointer to the constant value. + private interface{} +} + +func (e *expr) opClass() operatorClass { + return operatorTab[e.op].class +} + +func (e *expr) inputs() []*expr { + return e.children +} + +func formatExprs(tp *treePrinter, title string, exprs []*expr) { + if len(exprs) > 0 { + tp.Add(title) + tp.Enter() + for _, e := range exprs { + if e != nil { + e.format(tp) + } + } + tp.Exit() + } +} + +// format is part of the operatorInfo interface. +func (e *expr) format(tp *treePrinter) { + e.opClass().format(e, tp) +} + +func (e *expr) String() string { + tp := makeTreePrinter() + e.format(&tp) + return tp.String() +} diff --git a/pkg/sql/opt/misc.go b/pkg/sql/opt/misc.go new file mode 100644 index 000000000000..a9abf266c3b6 --- /dev/null +++ b/pkg/sql/opt/misc.go @@ -0,0 +1,21 @@ +// Copyright 2017 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. + +package opt + +import "fmt" + +func panicf(format string, args ...interface{}) { + panic(fmt.Sprintf(format, args...)) +} diff --git a/pkg/sql/opt/operator.go b/pkg/sql/opt/operator.go new file mode 100644 index 000000000000..1b96612e05f2 --- /dev/null +++ b/pkg/sql/opt/operator.go @@ -0,0 +1,111 @@ +// Copyright 2017 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. + +package opt + +import "fmt" + +type operator uint8 + +const ( + unknownOp operator = iota + + // TODO(radu): no relational operators yet. + + // Scalar operators + variableOp + constOp + listOp + orderedListOp + + existsOp + + andOp + orOp + notOp + + eqOp + ltOp + gtOp + leOp + geOp + neOp + inOp + notInOp + likeOp + notLikeOp + iLikeOp + notILikeOp + similarToOp + notSimilarToOp + regMatchOp + notRegMatchOp + regIMatchOp + notRegIMatchOp + isDistinctFromOp + isNotDistinctFromOp + isOp + isNotOp + anyOp + someOp + allOp + + bitandOp + bitorOp + bitxorOp + plusOp + minusOp + multOp + divOp + floorDivOp + modOp + powOp + concatOp + lShiftOp + rShiftOp + + unaryPlusOp + unaryMinusOp + unaryComplementOp + + functionOp + + numOperators +) + +type operatorInfo struct { + name string + class operatorClass +} + +var operatorTab = [numOperators]operatorInfo{ + unknownOp: {name: "unknown"}, +} + +func (op operator) String() string { + if op >= numOperators { + return fmt.Sprintf("operator(%d)", op) + } + return operatorTab[op].name +} + +func registerOperator(op operator, name string, class operatorClass) { + operatorTab[op].name = name + operatorTab[op].class = class +} + +type operatorClass interface { + // format outputs information about the expr tree to a treePrinter. + format(e *expr, tp *treePrinter) +} diff --git a/pkg/sql/opt/opt_test.go b/pkg/sql/opt/opt_test.go new file mode 100644 index 000000000000..6970a4e24f9e --- /dev/null +++ b/pkg/sql/opt/opt_test.go @@ -0,0 +1,299 @@ +// Copyright 2017 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. + +package opt + +// This file is home to TestOpt, which is similar to the logic tests, except it +// is used for optimizer-specific testcases. +// +// Each testfile contains testcases of the form +// +// +// ---- +// +// +// The supported commands are: +// +// - build-scalar [ ...] +// +// Builds an expression tree from a scalar SQL expression and outputs a +// representation of the tree. The expression can refer to external variables +// using @1, @2, etc. in which case the types of the variables must be passed +// on the command line. + +import ( + "bufio" + "bytes" + "flag" + "fmt" + "io" + "io/ioutil" + "os" + "path/filepath" + "strings" + "testing" + + "github.com/cockroachdb/cockroach/pkg/sql/coltypes" + "github.com/cockroachdb/cockroach/pkg/sql/parser" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sem/types" +) + +var ( + logicTestData = flag.String("d", "testdata/[^.]*", "test data glob") + rewriteTestFiles = flag.Bool("rewrite", false, "") +) + +type lineScanner struct { + *bufio.Scanner + line int +} + +func newLineScanner(r io.Reader) *lineScanner { + return &lineScanner{ + Scanner: bufio.NewScanner(r), + line: 0, + } +} + +func (l *lineScanner) Scan() bool { + ok := l.Scanner.Scan() + if ok { + l.line++ + } + return ok +} + +type testdata struct { + pos string // file and line number + cmd string // exec, query, ... + cmdArgs []string + sql string + expected string +} + +func (td testdata) fatalf(t *testing.T, format string, args ...interface{}) { + t.Helper() + t.Fatalf("%s: %s", td.pos, fmt.Sprintf(format, args...)) +} + +type testdataReader struct { + path string + file *os.File + scanner *lineScanner + data testdata + rewrite *bytes.Buffer +} + +func newTestdataReader(t *testing.T, path string) *testdataReader { + t.Helper() + + file, err := os.Open(path) + if err != nil { + t.Fatal(err) + } + var rewrite *bytes.Buffer + if *rewriteTestFiles { + rewrite = &bytes.Buffer{} + } + return &testdataReader{ + path: path, + file: file, + scanner: newLineScanner(file), + rewrite: rewrite, + } +} + +func (r *testdataReader) Close() error { + return r.file.Close() +} + +func (r *testdataReader) Next(t *testing.T) bool { + t.Helper() + + r.data = testdata{} + for r.scanner.Scan() { + line := r.scanner.Text() + r.emit(line) + + fields := strings.Fields(line) + if len(fields) == 0 { + continue + } + cmd := fields[0] + if strings.HasPrefix(cmd, "#") { + // Skip comment lines. + continue + } + r.data.pos = fmt.Sprintf("%s:%d", r.path, r.scanner.line) + r.data.cmd = cmd + r.data.cmdArgs = fields[1:] + + var buf bytes.Buffer + var separator bool + for r.scanner.Scan() { + line := r.scanner.Text() + if strings.TrimSpace(line) == "" { + break + } + + r.emit(line) + if line == "----" { + separator = true + break + } + fmt.Fprintln(&buf, line) + } + + r.data.sql = strings.TrimSpace(buf.String()) + + if separator { + buf.Reset() + for r.scanner.Scan() { + line := r.scanner.Text() + if strings.TrimSpace(line) == "" { + break + } + fmt.Fprintln(&buf, line) + } + r.data.expected = buf.String() + } + return true + } + return false +} + +func (r *testdataReader) emit(s string) { + if r.rewrite != nil { + r.rewrite.WriteString(s) + r.rewrite.WriteString("\n") + } +} + +// runTest reads through a file; for every testcase, it breaks it up into +// testdata.cmd, testdata.sql, testdata.expected, calls f, then compares the +// results. +func runTest(t *testing.T, path string, f func(d *testdata) string) { + r := newTestdataReader(t, path) + for r.Next(t) { + d := &r.data + str := f(d) + if r.rewrite != nil { + r.emit(str) + } else if d.expected != str { + t.Fatalf("%s: %s\nexpected:\n%s\nfound:\n%s", d.pos, d.sql, d.expected, str) + } else if testing.Verbose() { + fmt.Printf("%s:\n%s\n----\n%s", d.pos, d.sql, str) + } + } + + if r.rewrite != nil { + data := r.rewrite.Bytes() + if l := len(data); l > 2 && data[l-1] == '\n' && data[l-2] == '\n' { + data = data[:l-1] + } + err := ioutil.WriteFile(path, data, 0644) + if err != nil { + t.Fatal(err) + } + } +} + +func TestOpt(t *testing.T) { + paths, err := filepath.Glob(*logicTestData) + if err != nil { + t.Fatal(err) + } + if len(paths) == 0 { + t.Fatalf("no testfiles found matching: %s", *logicTestData) + } + + for _, path := range paths { + t.Run(filepath.Base(path), func(t *testing.T) { + runTest(t, path, func(d *testdata) string { + switch d.cmd { + case "build-scalar": + typedExpr, err := parseScalarExpr(d.sql, d.cmdArgs) + if err != nil { + d.fatalf(t, "%v", err) + } + + e := func() *expr { + defer func() { + if r := recover(); r != nil { + d.fatalf(t, "buildScalar: %v", r) + } + }() + return buildScalar(&buildContext{}, typedExpr) + }() + return e.String() + default: + d.fatalf(t, "unsupported command: %s", d.cmd) + return "" + } + }) + }) + } +} + +func parseType(typeStr string) (types.T, error) { + colType, err := parser.ParseType(typeStr) + if err != nil { + return nil, err + } + return coltypes.CastTargetToDatumType(colType), nil +} + +type indexedVars struct { + types []types.T +} + +var _ tree.IndexedVarContainer = &indexedVars{} + +func (*indexedVars) IndexedVarEval(idx int, ctx *tree.EvalContext) (tree.Datum, error) { + panic("unimplemented") +} + +func (iv *indexedVars) IndexedVarResolvedType(idx int) types.T { + return iv.types[idx] +} + +func (*indexedVars) IndexedVarNodeFormatter(idx int) tree.NodeFormatter { + panic("unimplemented") +} + +func parseScalarExpr(sql string, indexVarTypes []string) (tree.TypedExpr, error) { + expr, err := parser.ParseExpr(sql) + if err != nil { + return nil, err + } + + // Set up an indexed var helper so we can type-check the expression. + iv := &indexedVars{ + types: make([]types.T, len(indexVarTypes)), + } + for i, typeStr := range indexVarTypes { + var err error + iv.types[i], err = parseType(typeStr) + if err != nil { + return nil, err + } + } + + sema := tree.MakeSemaContext(false /* privileged */) + iVarHelper := tree.MakeIndexedVarHelper(iv, len(iv.types)) + sema.IVarHelper = &iVarHelper + + return expr.TypeCheck(&sema, types.Any) +} diff --git a/pkg/sql/opt/scalar.go b/pkg/sql/opt/scalar.go new file mode 100644 index 000000000000..b63c88f8f981 --- /dev/null +++ b/pkg/sql/opt/scalar.go @@ -0,0 +1,123 @@ +// Copyright 2017 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. + +package opt + +import ( + "bytes" + "fmt" + + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sem/types" +) + +func init() { + registerOperator(variableOp, "variable", scalarClass{}) + registerOperator(constOp, "const", scalarClass{}) + registerOperator(listOp, "list", scalarClass{}) + registerOperator(orderedListOp, "ordered-list", scalarClass{}) + registerOperator(existsOp, "exists", scalarClass{}) + registerOperator(andOp, "AND", scalarClass{}) + registerOperator(orOp, "OR", scalarClass{}) + registerOperator(notOp, "NOT", scalarClass{}) + registerOperator(eqOp, "eq", scalarClass{}) + registerOperator(ltOp, "lt", scalarClass{}) + registerOperator(gtOp, "gt", scalarClass{}) + registerOperator(leOp, "le", scalarClass{}) + registerOperator(geOp, "ge", scalarClass{}) + registerOperator(neOp, "ne", scalarClass{}) + registerOperator(inOp, "IN", scalarClass{}) + registerOperator(notInOp, "NOT-IN", scalarClass{}) + registerOperator(likeOp, "LIKE", scalarClass{}) + registerOperator(notLikeOp, "NOT-LIKE", scalarClass{}) + registerOperator(iLikeOp, "ILIKE", scalarClass{}) + registerOperator(notILikeOp, "NOT-ILIKE", scalarClass{}) + registerOperator(similarToOp, "SIMILAR-TO", scalarClass{}) + registerOperator(notSimilarToOp, "NOT-SIMILAR-TO", scalarClass{}) + registerOperator(regMatchOp, "regmatch", scalarClass{}) + registerOperator(notRegMatchOp, "not-regmatch", scalarClass{}) + registerOperator(regIMatchOp, "regimatch", scalarClass{}) + registerOperator(notRegIMatchOp, "not-regimatch", scalarClass{}) + registerOperator(isDistinctFromOp, "IS-DISTINCT-FROM", scalarClass{}) + registerOperator(isNotDistinctFromOp, "IS-NOT-DISTINCT-FROM", scalarClass{}) + registerOperator(isOp, "IS", scalarClass{}) + registerOperator(isNotOp, "IS-NOT", scalarClass{}) + registerOperator(anyOp, "ANY", scalarClass{}) + registerOperator(someOp, "SOME", scalarClass{}) + registerOperator(allOp, "ALL", scalarClass{}) + registerOperator(bitandOp, "bitand", scalarClass{}) + registerOperator(bitorOp, "bitor", scalarClass{}) + registerOperator(bitxorOp, "bitxor", scalarClass{}) + registerOperator(plusOp, "plus", scalarClass{}) + registerOperator(minusOp, "minus", scalarClass{}) + registerOperator(multOp, "mult", scalarClass{}) + registerOperator(divOp, "div", scalarClass{}) + registerOperator(floorDivOp, "floor-div", scalarClass{}) + registerOperator(modOp, "mod", scalarClass{}) + registerOperator(powOp, "pow", scalarClass{}) + registerOperator(concatOp, "concat", scalarClass{}) + registerOperator(lShiftOp, "lshift", scalarClass{}) + registerOperator(rShiftOp, "rshift", scalarClass{}) + registerOperator(unaryPlusOp, "unary-plus", scalarClass{}) + registerOperator(unaryMinusOp, "unary-minus", scalarClass{}) + registerOperator(unaryComplementOp, "complement", scalarClass{}) + registerOperator(functionOp, "func", scalarClass{}) +} + +type scalarProps struct { + typ types.T +} + +type scalarClass struct{} + +var _ operatorClass = scalarClass{} + +func (scalarClass) format(e *expr, tp *treePrinter) { + var buf bytes.Buffer + fmt.Fprintf(&buf, "%v", e.op) + if e.private != nil { + fmt.Fprintf(&buf, " (%v)", e.private) + } + fmt.Fprintf(&buf, " (type: %s)", e.scalarProps.typ) + tp.Add(buf.String()) + tp.Enter() + formatExprs(tp, "inputs", e.inputs()) + tp.Exit() +} + +func initConstExpr(e *expr, datum tree.Datum) { + e.op = constOp + e.private = datum +} + +func initFunctionExpr(e *expr, def *tree.FunctionDefinition, children []*expr) { + e.op = functionOp + e.children = children + e.private = def +} + +func initUnaryExpr(e *expr, op operator, input *expr) { + e.op = op + e.children = []*expr{input} +} + +func initBinaryExpr(e *expr, op operator, input1 *expr, input2 *expr) { + e.op = op + e.children = []*expr{input1, input2} +} + +func initVariableExpr(e *expr, index int) { + e.op = variableOp + e.private = index +} diff --git a/pkg/sql/opt/testdata/build-scalar b/pkg/sql/opt/testdata/build-scalar new file mode 100644 index 000000000000..89274b06d4f4 --- /dev/null +++ b/pkg/sql/opt/testdata/build-scalar @@ -0,0 +1,53 @@ +build-scalar +1 +---- +const (1) (type: int) + +build-scalar +1 +---- +const (1) (type: int) + + +build-scalar +1 + 2 +---- +plus (type: int) + |- inputs + |- const (1) (type: int) + |- const (2) (type: int) + + +build-scalar string +@1 +---- +variable (0) (type: string) + + +build-scalar int +@1 + 2 +---- +plus (type: int) + |- inputs + |- variable (0) (type: int) + |- const (2) (type: int) + +build-scalar int int +@1 >= 5 AND @1 <= 10 AND @2 < 4 +---- +AND (type: bool) + |- inputs + |- AND (type: bool) + | |- inputs + | |- ge (type: bool) + | | |- inputs + | | |- variable (0) (type: int) + | | |- const (5) (type: int) + | |- le (type: bool) + | |- inputs + | |- variable (0) (type: int) + | |- const (10) (type: int) + |- lt (type: bool) + |- inputs + |- variable (1) (type: int) + |- const (4) (type: int) diff --git a/pkg/sql/opt/tree_print.go b/pkg/sql/opt/tree_print.go new file mode 100644 index 000000000000..1555bb9ebc55 --- /dev/null +++ b/pkg/sql/opt/tree_print.go @@ -0,0 +1,99 @@ +// Copyright 2017 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. + +package opt + +import ( + "bytes" + "fmt" +) + +// treePrinter pretty-prints a tree: +// +// root +// |- child1 +// | |- grandchild1 +// | |- grandchild2 +// |- child2 +// +type treePrinter struct { + level int + + // We maintain the rows accumulated so far. + // When a new child is adedd (e.g. child2 above), we may have to go back up + // and replace spaces with "|". + rows [][]byte + + // The index of the last row for a given level. + lastEntry []int +} + +func makeTreePrinter() treePrinter { + return treePrinter{ + lastEntry: make([]int, 1, 4), + } +} + +// Enter indicates that entries that follow are children of the last entry. +// Each Enter() call must be paired with a subsequent Exit() call. +func (tp *treePrinter) Enter() { + tp.level++ + tp.lastEntry = append(tp.lastEntry, -1) +} + +// Exit is the reverse of Enter. +func (tp *treePrinter) Exit() { + if tp.level == 0 { + panic("Exit without Enter") + } + tp.level-- + tp.lastEntry = tp.lastEntry[:len(tp.lastEntry)-1] +} + +func (tp *treePrinter) Addf(format string, args ...interface{}) { + tp.Add(fmt.Sprintf(format, args...)) +} + +func (tp *treePrinter) Add(entry string) { + // Each level indents by four spaces (" |- "). + indent := 4 * tp.level + row := make([]byte, indent+len(entry)) + for i := 0; i < indent-4; i++ { + row[i] = ' ' + } + if indent >= 4 { + copy(row[indent-4:], " |- ") + } + copy(row[indent:], entry) + // Connect to the previous sibling. + if tp.level > 0 && tp.lastEntry[tp.level] != -1 { + for i := tp.lastEntry[tp.level] + 1; i < len(tp.rows); i++ { + tp.rows[i][indent-3] = '|' + } + } + tp.lastEntry[tp.level] = len(tp.rows) + tp.rows = append(tp.rows, row) +} + +func (tp *treePrinter) String() string { + if tp.level != 0 { + panic("Enter without Exit") + } + var buf bytes.Buffer + for _, r := range tp.rows { + buf.Write(r) + buf.WriteByte('\n') + } + return buf.String() +} diff --git a/pkg/sql/opt/tree_print_test.go b/pkg/sql/opt/tree_print_test.go new file mode 100644 index 000000000000..6867edbdfe80 --- /dev/null +++ b/pkg/sql/opt/tree_print_test.go @@ -0,0 +1,55 @@ +// Copyright 2017 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. + +package opt + +import ( + "strings" + "testing" +) + +func TestTreePrinter(t *testing.T) { + tp := makeTreePrinter() + + tp.Add("root") + tp.Enter() + tp.Addf("%d.%d", 1, 1) + tp.Enter() + tp.Add("1.1.1") + tp.Add("1.1.2") + tp.Enter() + tp.Add("1.1.2.1") + tp.Add("1.1.2.2") + tp.Exit() + tp.Add("1.1.3") + tp.Exit() + tp.Add("1.2") + tp.Exit() + + res := tp.String() + exp := ` +root + |- 1.1 + | |- 1.1.1 + | |- 1.1.2 + | | |- 1.1.2.1 + | | |- 1.1.2.2 + | |- 1.1.3 + |- 1.2 +` + exp = strings.TrimLeft(exp, "\n") + if res != exp { + t.Errorf("incorrect result:\n%s", res) + } +}