From a924e247b58a10415e39a2b90cd6e8e358dfb60e Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Mon, 21 Dec 2020 16:37:37 +0100 Subject: [PATCH 01/40] Semantic analysis Signed-off-by: Andres Taylor --- go/vt/vtgate/semantics/analyzer.go | 211 +++++++++++++++++++++++ go/vt/vtgate/semantics/analyzer_test.go | 188 ++++++++++++++++++++ go/vt/vtgate/semantics/semantic_state.go | 126 ++++++++++++++ go/vt/vtgate/semantics/tabletset_test.go | 43 +++++ 4 files changed, 568 insertions(+) create mode 100644 go/vt/vtgate/semantics/analyzer.go create mode 100644 go/vt/vtgate/semantics/analyzer_test.go create mode 100644 go/vt/vtgate/semantics/semantic_state.go create mode 100644 go/vt/vtgate/semantics/tabletset_test.go diff --git a/go/vt/vtgate/semantics/analyzer.go b/go/vt/vtgate/semantics/analyzer.go new file mode 100644 index 00000000000..bb24521139d --- /dev/null +++ b/go/vt/vtgate/semantics/analyzer.go @@ -0,0 +1,211 @@ +/* +Copyright 2020 The Vitess 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 semantics + +import ( + "vitess.io/vitess/go/mysql" + vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" + "vitess.io/vitess/go/vt/sqlparser" + "vitess.io/vitess/go/vt/vterrors" +) + +var debug = false + +type ( + // analyzer is a struct to work with analyzing the query. + analyzer struct { + Tables []table + + scopes []*scope + exprDeps map[sqlparser.Expr]TableSet + si schemaInformation + err error + } +) + +// newAnalyzer create the semantic analyzer +func newAnalyzer(si schemaInformation) *analyzer { + return &analyzer{ + exprDeps: map[sqlparser.Expr]TableSet{}, + si: si, + } +} + +// analyzeDown pushes new scopes when we encounter sub queries, +// and resolves the table a column is using +func (a *analyzer) analyzeDown(cursor *sqlparser.Cursor) bool { + current := a.currentScope() + n := cursor.Node() + log(n, "%p analyzeDown %T", current, n) + switch node := n.(type) { + case *sqlparser.Select: + a.push(newScope(current)) + if err := a.analyzeTableExprs(node.From); err != nil { + a.err = err + return false + } + case *sqlparser.TableExprs: + // this has already been visited when we encountered the SELECT struct + return false + + // we don't need to push new scope for sub queries since we do that for SELECT and UNION + + case *sqlparser.Union: + a.push(newScope(current)) + case *sqlparser.ColName: + t, err := a.resolveColumn(node, current) + if err != nil { + a.err = err + } + a.exprDeps[node] = t + } + return a.shouldContinue() +} + +func (a *analyzer) resolveColumn(colName *sqlparser.ColName, current *scope) (TableSet, error) { + var t table + var err error + if colName.Qualifier.IsEmpty() { + t, err = a.resolveUnQualifiedColumn(current, colName) + } else { + t, err = a.resolveQualifiedColumn(current, colName) + } + if err != nil { + return 0, err + } + return a.tableSetFor(t), nil +} + +func (a *analyzer) analyzeTableExprs(tablExprs sqlparser.TableExprs) error { + for _, tableExpr := range tablExprs { + if err := a.analyzeTableExpr(tableExpr); err != nil { + return err + } + } + return nil +} + +func (a *analyzer) analyzeTableExpr(tableExpr sqlparser.TableExpr) error { + log(tableExpr, "analyzeTableExpr %T", tableExpr) + switch table := tableExpr.(type) { + case *sqlparser.AliasedTableExpr: + return a.bindTable(table, table.Expr) + case *sqlparser.JoinTableExpr: + if err := a.analyzeTableExpr(table.LeftExpr); err != nil { + return err + } + if err := a.analyzeTableExpr(table.RightExpr); err != nil { + return err + } + case *sqlparser.ParenTableExpr: + return a.analyzeTableExprs(table.Exprs) + } + return nil +} + +// resolveQualifiedColumn handles `tabl.col` expressions +func (a *analyzer) resolveQualifiedColumn(current *scope, expr *sqlparser.ColName) (table, error) { + qualifier := expr.Qualifier.Name.String() + + for current != nil { + tableExpr, found := current.tables[qualifier] + if found { + return tableExpr, nil + } + current = current.parent + } + + return nil, mysql.NewSQLError(mysql.ERBadFieldError, mysql.SSBadFieldError, "Unknown table referenced by '%s'", sqlparser.String(expr)) +} + +// resolveUnQualifiedColumn +func (a *analyzer) resolveUnQualifiedColumn(current *scope, expr *sqlparser.ColName) (table, error) { + if len(current.tables) == 1 { + for _, tableExpr := range current.tables { + return tableExpr, nil + } + } + return nil, vterrors.Errorf(vtrpcpb.Code_UNIMPLEMENTED, "todo - figure out which table this column belongs to") +} + +func (a *analyzer) tableSetFor(t table) TableSet { + for i, t2 := range a.Tables { + if t == t2 { + return TableSet(1 << i) + } + } + panic("unknown table") +} + +func (a *analyzer) bindTable(alias *sqlparser.AliasedTableExpr, expr sqlparser.SimpleTableExpr) error { + switch t := expr.(type) { + case *sqlparser.DerivedTable: + a.push(newScope(nil)) + if err := a.analyze(t.Select); err != nil { + return err + } + a.popScope() + scope := a.currentScope() + return scope.addTable(alias.As.String(), alias) + case sqlparser.TableName: + scope := a.currentScope() + a.Tables = append(a.Tables, alias) + if alias.As.IsEmpty() { + return scope.addTable(t.Name.String(), alias) + } + return scope.addTable(alias.As.String(), alias) + } + return nil +} + +func (a *analyzer) analyze(statement sqlparser.Statement) error { + log(statement, "analyse %T", statement) + _ = sqlparser.Rewrite(statement, a.analyzeDown, a.analyzeUp) + + return a.err +} + +func (a *analyzer) analyzeUp(cursor *sqlparser.Cursor) bool { + switch cursor.Node().(type) { + case *sqlparser.Union, *sqlparser.Select: + a.popScope() + } + return true +} + +func (a *analyzer) shouldContinue() bool { + return a.err == nil +} + +func (a *analyzer) push(s *scope) { + log(nil, "enter new scope") + a.scopes = append(a.scopes, s) +} + +func (a *analyzer) popScope() { + log(nil, "exit scope") + l := len(a.scopes) - 1 + a.scopes = a.scopes[:l] +} + +func (a *analyzer) currentScope() *scope { + size := len(a.scopes) + if size == 0 { + return nil + } + return a.scopes[size-1] +} diff --git a/go/vt/vtgate/semantics/analyzer_test.go b/go/vt/vtgate/semantics/analyzer_test.go new file mode 100644 index 00000000000..15be7796674 --- /dev/null +++ b/go/vt/vtgate/semantics/analyzer_test.go @@ -0,0 +1,188 @@ +/* +Copyright 2020 The Vitess 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 semantics + +import ( + "testing" + + "github.com/stretchr/testify/assert" + + "github.com/stretchr/testify/require" + "vitess.io/vitess/go/vt/sqlparser" +) + +const ( + // Just here to make outputs more readable + T0 TableSet = 1 << iota + T1 + T2 + _ // T3 is not used in the tests + T4 +) + +func extract(in *sqlparser.Select, idx int) sqlparser.Expr { + return in.SelectExprs[idx].(*sqlparser.AliasedExpr).Expr +} + +func TestScope(t *testing.T) { + query := ` +select t.col1, ( + select t.col2 from z as t) +from x as t` + stmt, semTable := parseAndAnalyze(t, query) + + sel, _ := stmt.(*sqlparser.Select) + + // extract the `t.col2` expression from the subquery + sel2 := sel.SelectExprs[1].(*sqlparser.AliasedExpr).Expr.(*sqlparser.Subquery).Select.(*sqlparser.Select) + s1 := semTable.Dependencies(extract(sel2, 0)) + + // if scoping works as expected, we should be able to see the inner table being used by the inner expression + assert.Equal(t, T1, s1) +} + +func TestBindingSingleTable(t *testing.T) { + queries := []string{ + "select col from tabl", + "select tabl.col from tabl", + "select d.tabl.col from tabl", + "select col from d.tabl", + "select tabl.col from d.tabl", + "select d.tabl.col from d.tabl", + } + for _, query := range queries { + t.Run(query, func(t *testing.T) { + stmt, semTable := parseAndAnalyze(t, query) + sel, _ := stmt.(*sqlparser.Select) + t1 := sel.From[0].(*sqlparser.AliasedTableExpr) + ts := semTable.TableSetFor(t1) + assert.EqualValues(t, 1, ts) + + d := semTable.Dependencies(extract(sel, 0)) + require.Equal(t, T0, d, query) + }) + } +} + +func TestUnion(t *testing.T) { + query := "select col1 from tabl1 union select col2 from tabl2" + + stmt, semTable := parseAndAnalyze(t, query) + union, _ := stmt.(*sqlparser.Union) + sel1 := union.FirstStatement.(*sqlparser.Select) + sel2 := union.UnionSelects[0].Statement.(*sqlparser.Select) + + t1 := sel1.From[0].(*sqlparser.AliasedTableExpr) + t2 := sel2.From[0].(*sqlparser.AliasedTableExpr) + ts1 := semTable.TableSetFor(t1) + ts2 := semTable.TableSetFor(t2) + assert.EqualValues(t, 1, ts1) + assert.EqualValues(t, 2, ts2) + + d1 := semTable.Dependencies(extract(sel1, 0)) + d2 := semTable.Dependencies(extract(sel2, 0)) + assert.Equal(t, T0, d1) + assert.Equal(t, T1, d2) +} + +func TestBindingMultiTable(t *testing.T) { + type testCase struct { + query string + deps TableSet + } + queries := []testCase{{ + query: "select t.col from t, s", + deps: T0, + }, { + query: "select t.col from t join s", + deps: T0, + }, { + query: "select max(t.col+s.col) from t, s", + deps: T0 | T1, + }, { + query: "select max(t.col+s.col) from t join s", + deps: T0 | T1, + }, { + query: "select case t.col when s.col then r.col else u.col end from t, s, r, w, u", + deps: T0 | T1 | T2 | T4, + }, { + // make sure that we don't let sub-query Dependencies leak out by mistake + query: "select t.col + (select 42 from s) from t", + deps: T0, + }, { + query: "select (select 42 from s where r.id = s.id) from r", + deps: T0 | T1, + }} + for _, query := range queries { + t.Run(query.query, func(t *testing.T) { + stmt, semTable := parseAndAnalyze(t, query.query) + sel, _ := stmt.(*sqlparser.Select) + assert.Equal(t, query.deps, semTable.Dependencies(extract(sel, 0)), query.query) + }) + } +} + +func TestBindingSingleDepPerTable(t *testing.T) { + query := "select t.col + t.col2 from t" + stmt, semTable := parseAndAnalyze(t, query) + sel, _ := stmt.(*sqlparser.Select) + + d := semTable.Dependencies(extract(sel, 0)) + assert.Equal(t, 1, d.NumberOfTables(), "size wrong") + assert.Equal(t, T0, d) +} + +func TestNotUniqueTableName(t *testing.T) { + queries := []string{ + "select * from t, t", + "select * from t, (select 1 from x) as t", + "select * from t join t", + "select * from t join (select 1 from x) as t", + } + + for _, query := range queries { + t.Run(query, func(t *testing.T) { + parse, _ := sqlparser.Parse(query) + _, err := Analyse(parse, nil) + require.Error(t, err) + require.Contains(t, err.Error(), "Not unique table/alias") + }) + } +} + +func TestMissingTable(t *testing.T) { + queries := []string{ + "select t.col from a", + } + + for _, query := range queries { + t.Run(query, func(t *testing.T) { + parse, _ := sqlparser.Parse(query) + _, err := Analyse(parse, nil) + require.Error(t, err) + require.Contains(t, err.Error(), "Unknown table") + }) + } +} + +func parseAndAnalyze(t *testing.T, query string) (sqlparser.Statement, *SemTable) { + parse, err := sqlparser.Parse(query) + require.NoError(t, err) + semTable, err := Analyse(parse, nil) + require.NoError(t, err) + return parse, semTable +} diff --git a/go/vt/vtgate/semantics/semantic_state.go b/go/vt/vtgate/semantics/semantic_state.go new file mode 100644 index 00000000000..72ee632adc1 --- /dev/null +++ b/go/vt/vtgate/semantics/semantic_state.go @@ -0,0 +1,126 @@ +/* +Copyright 2020 The Vitess 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 semantics + +import ( + "fmt" + + "vitess.io/vitess/go/mysql" + + "vitess.io/vitess/go/vt/vtgate/vindexes" + + "vitess.io/vitess/go/vt/sqlparser" +) + +type ( + table = *sqlparser.AliasedTableExpr + + // TableSet is how a set of tables is expressed. + // Tables get unique bits assigned in the order that they are encountered during semantic analysis + TableSet uint64 // we can only join 64 tables with this underlying data type + + // SemTable contains semantic analysis information about the query. + SemTable struct { + Tables []table + exprDependencies map[sqlparser.Expr]TableSet + } + schemaInformation interface { + FindTable(tablename sqlparser.TableName) (*vindexes.Table, error) + } + scope struct { + parent *scope + tables map[string]*sqlparser.AliasedTableExpr + } +) + +// NumberOfTables returns the number of bits set +func (ts TableSet) NumberOfTables() int { + // Brian Kernighan’s Algorithm + count := 0 + for ts > 0 { + ts &= ts - 1 + count++ + } + return count +} + +// TableSetFor returns the bitmask for this particular tableshoe +func (st *SemTable) TableSetFor(t table) TableSet { + for idx, t2 := range st.Tables { + if t == t2 { + return 1 << idx + } + } + return 0 +} + +// Dependencies return the table dependencies of the expression. +func (st *SemTable) Dependencies(expr sqlparser.Expr) TableSet { + var deps TableSet + + _ = sqlparser.Walk(func(node sqlparser.SQLNode) (kontinue bool, err error) { + colName, ok := node.(*sqlparser.ColName) + if ok { + set := st.exprDependencies[colName] + deps |= set + } + return true, nil + }, expr) + + return deps +} + +func newScope(parent *scope) *scope { + return &scope{tables: map[string]*sqlparser.AliasedTableExpr{}, parent: parent} +} + +func (s *scope) addTable(name string, table *sqlparser.AliasedTableExpr) error { + _, found := s.tables[name] + if found { + return mysql.NewSQLError(mysql.ERNonUniqTable, mysql.SSSyntaxErrorOrAccessViolation, "Not unique table/alias: '%s'", name) + } + s.tables[name] = table + return nil +} + +// Analyse analyzes the parsed query. +func Analyse(statement sqlparser.Statement, si schemaInformation) (*SemTable, error) { + analyzer := newAnalyzer(si) + // Initial scope + err := analyzer.analyze(statement) + if err != nil { + return nil, err + } + return &SemTable{exprDependencies: analyzer.exprDeps, Tables: analyzer.Tables}, nil +} + +func log(node sqlparser.SQLNode, format string, args ...interface{}) { + if debug { + fmt.Printf(format, args...) + if node == nil { + fmt.Println() + } else { + fmt.Println(" - " + sqlparser.String(node)) + } + } +} + +// IsOverlapping returns true if at least one table exists in both sets +func IsOverlapping(a, b TableSet) bool { return a&b != 0 } + +// IsContainedBy returns true if all of `b` is contained in `a` +func IsContainedBy(a, b TableSet) bool { return a&b == a } diff --git a/go/vt/vtgate/semantics/tabletset_test.go b/go/vt/vtgate/semantics/tabletset_test.go new file mode 100644 index 00000000000..15bc041f1c7 --- /dev/null +++ b/go/vt/vtgate/semantics/tabletset_test.go @@ -0,0 +1,43 @@ +/* +Copyright 2020 The Vitess 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 semantics + +import ( + "testing" + + "github.com/stretchr/testify/assert" +) + +const ( + _ TableSet = 1 << iota + F1 + F2 + F3 +) + +func TestTableSet(t *testing.T) { + assert.True(t, IsOverlapping(F1|F2, F1|F2)) + assert.True(t, IsOverlapping(F1|F2, F1)) + assert.True(t, IsOverlapping(F1, F1|F2)) + assert.False(t, IsOverlapping(F1|F2, F3)) + assert.False(t, IsOverlapping(F3, F1|F2)) + + assert.False(t, IsContainedBy(F1|F2, F1)) + assert.True(t, IsContainedBy(F1, F1|F2)) + assert.False(t, IsContainedBy(F1|F2, F3)) + assert.False(t, IsContainedBy(F3, F1|F2)) +} From 7f4c826316182ffcf457a26c9807805661803387 Mon Sep 17 00:00:00 2001 From: GuptaManan100 Date: Tue, 22 Dec 2020 11:30:12 +0530 Subject: [PATCH 02/40] Added tests for query graph Signed-off-by: GuptaManan100 --- go/vt/vtgate/planbuilder/builder.go | 2 + go/vt/vtgate/planbuilder/plan_test.go | 6 + go/vt/vtgate/planbuilder/querygraph.go | 210 ++++++++++++++++++++ go/vt/vtgate/planbuilder/querygraph_test.go | 126 ++++++++++++ go/vt/vtgate/semantics/semantic_state.go | 1 + go/vt/vtgate/vcursor_impl.go | 7 + 6 files changed, 352 insertions(+) create mode 100644 go/vt/vtgate/planbuilder/querygraph.go create mode 100644 go/vt/vtgate/planbuilder/querygraph_test.go diff --git a/go/vt/vtgate/planbuilder/builder.go b/go/vt/vtgate/planbuilder/builder.go index 08e15b876d0..e9a9dddd900 100644 --- a/go/vt/vtgate/planbuilder/builder.go +++ b/go/vt/vtgate/planbuilder/builder.go @@ -21,6 +21,7 @@ import ( "vitess.io/vitess/go/sqltypes" querypb "vitess.io/vitess/go/vt/proto/query" + "vitess.io/vitess/go/vt/vtgate/semantics" "vitess.io/vitess/go/vt/vterrors" @@ -48,6 +49,7 @@ type ContextVSchema interface { SysVarSetEnabled() bool KeyspaceExists(keyspace string) bool AllKeyspace() ([]*vindexes.Keyspace, error) + GetSemTable() *semantics.SemTable } type truncater interface { diff --git a/go/vt/vtgate/planbuilder/plan_test.go b/go/vt/vtgate/planbuilder/plan_test.go index 836e9f244ee..7aeb0577038 100644 --- a/go/vt/vtgate/planbuilder/plan_test.go +++ b/go/vt/vtgate/planbuilder/plan_test.go @@ -27,6 +27,8 @@ import ( "strings" "testing" + "vitess.io/vitess/go/vt/vtgate/semantics" + "github.com/google/go-cmp/cmp" vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" @@ -291,6 +293,10 @@ func (vw *vschemaWrapper) AllKeyspace() ([]*vindexes.Keyspace, error) { return []*vindexes.Keyspace{vw.keyspace}, nil } +func (vw *vschemaWrapper) GetSemTable() *semantics.SemTable { + return nil +} + func (vw *vschemaWrapper) KeyspaceExists(keyspace string) bool { if vw.keyspace != nil { return vw.keyspace.Name == keyspace diff --git a/go/vt/vtgate/planbuilder/querygraph.go b/go/vt/vtgate/planbuilder/querygraph.go new file mode 100644 index 00000000000..f460b4addb6 --- /dev/null +++ b/go/vt/vtgate/planbuilder/querygraph.go @@ -0,0 +1,210 @@ +/* +Copyright 2020 The Vitess 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 planbuilder + +import ( + vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" + "vitess.io/vitess/go/vt/sqlparser" + "vitess.io/vitess/go/vt/vterrors" + "vitess.io/vitess/go/vt/vtgate/engine" + "vitess.io/vitess/go/vt/vtgate/semantics" +) + +type ( + /* + queryGraph represents the FROM and WHERE parts of a query. + It is an intermediate representation of the query that makes it easier for the planner + to find all possible join combinations. Instead of storing the query information in a form that is close + to the syntax (AST), we extract the interesting parts into a graph form with the nodes being tables in the FROM + clause and the edges between them being predicates. We keep predicates in a hash map keyed by the dependencies of + the predicate. This makes it very fast to look up connections between tables in the query. + */ + queryGraph struct { + // the tables, including predicates that only depend on this particular table + tables []*queryTable + + // crossTable contains the predicates that need multiple tables + crossTable map[semantics.TableSet][]sqlparser.Expr + + // noDeps contains the predicates that can be evaluated anywhere. + noDeps sqlparser.Expr + } + + // queryTable is a single FROM table, including all predicates particular to this table + queryTable struct { + tableIdentifier semantics.TableSet + alias *sqlparser.AliasedTableExpr + table sqlparser.TableName + predicates []sqlparser.Expr + } +) + +func createQGFromSelect(sel *sqlparser.Select, semTable *semantics.SemTable) (*queryGraph, error) { + qg := newQueryGraph() + if err := qg.collectTables(sel.From, semTable); err != nil { + return nil, err + } + + if sel.Where != nil { + err := qg.collectPredicates(sel, semTable) + if err != nil { + return nil, err + } + } + return qg, nil +} + +func newQueryGraph() *queryGraph { + return &queryGraph{ + crossTable: map[semantics.TableSet][]sqlparser.Expr{}, + } +} + +func (qg *queryGraph) collectTable(t sqlparser.TableExpr, semTable *semantics.SemTable) error { + switch table := t.(type) { + case *sqlparser.AliasedTableExpr: + tableName := table.Expr.(sqlparser.TableName) + qt := &queryTable{alias: table, table: tableName, tableIdentifier: semTable.TableSetFor(table)} + qg.tables = append(qg.tables, qt) + case *sqlparser.JoinTableExpr: + if err := qg.collectTable(table.LeftExpr, semTable); err != nil { + return err + } + if err := qg.collectTable(table.RightExpr, semTable); err != nil { + return err + } + if err := qg.collectPredicate(table.Condition.On, semTable); err != nil { + return err + } + case *sqlparser.ParenTableExpr: + if err := qg.collectTables(table.Exprs, semTable); err != nil { + return err + } + } + return nil +} + +func (qg *queryGraph) collectTables(t sqlparser.TableExprs, semTable *semantics.SemTable) error { + for _, expr := range t { + if err := qg.collectTable(expr, semTable); err != nil { + return err + } + } + return nil +} + +func (qg *queryGraph) collectPredicates(sel *sqlparser.Select, semTable *semantics.SemTable) error { + predicates := splitAndExpression(nil, sel.Where.Expr) + + for _, predicate := range predicates { + err := qg.collectPredicate(predicate, semTable) + if err != nil { + return err + } + } + return nil +} + +func (qg *queryGraph) collectPredicate(predicate sqlparser.Expr, semTable *semantics.SemTable) error { + deps := semTable.Dependencies(predicate) + switch deps.NumberOfTables() { + case 0: + qg.addNoDepsPredicate(predicate) + case 1: + found := qg.addToSingleTable(deps, predicate) + if !found { + return vterrors.Errorf(vtrpcpb.Code_INTERNAL, "table %v for predicate %v not found", deps, sqlparser.String(predicate)) + } + default: + allPredicates, found := qg.crossTable[deps] + if found { + allPredicates = append(allPredicates, predicate) + } else { + allPredicates = []sqlparser.Expr{predicate} + } + qg.crossTable[deps] = allPredicates + } + return nil +} + +func (qg *queryGraph) addToSingleTable(table semantics.TableSet, predicate sqlparser.Expr) bool { + for _, t := range qg.tables { + if table == t.tableIdentifier { + t.predicates = append(t.predicates, predicate) + return true + } + } + return false +} + +func (qg *queryGraph) addNoDepsPredicate(predicate sqlparser.Expr) { + if qg.noDeps == nil { + qg.noDeps = predicate + } else { + qg.noDeps = &sqlparser.AndExpr{ + Left: qg.noDeps, + Right: predicate, + } + } +} + +func (dpt dpTableT) bitSetsOfSize(wanted int) []joinTree { + var result []joinTree + for bs, jt := range dpt { + size := bs.NumberOfTables() + if size == wanted { + result = append(result, jt) + } + } + return result +} + +func (qg *queryGraph) tryMerge(a, b joinTree, joinPredicates []sqlparser.Expr) joinTree { + aRoute, ok := a.(*routePlan) + if !ok { + return nil + } + bRoute, ok := b.(*routePlan) + if !ok { + return nil + } + if aRoute.keyspace != bRoute.keyspace { + return nil + } + + switch aRoute.routeOpCode { + case engine.SelectUnsharded, engine.SelectDBA: + if aRoute.routeOpCode != bRoute.routeOpCode { + return nil + } + case engine.SelectEqualUnique: + + return nil + } + newTabletSet := aRoute.solved | bRoute.solved + r := &routePlan{ + routeOpCode: aRoute.routeOpCode, + solved: newTabletSet, + tables: append(aRoute.tables, bRoute.tables...), + extraPredicates: append(aRoute.extraPredicates, bRoute.extraPredicates...), + keyspace: aRoute.keyspace, + vindexPlusPredicates: append(aRoute.vindexPlusPredicates, bRoute.vindexPlusPredicates...), + } + + r.extraPredicates = append(r.extraPredicates, joinPredicates...) + return nil +} diff --git a/go/vt/vtgate/planbuilder/querygraph_test.go b/go/vt/vtgate/planbuilder/querygraph_test.go new file mode 100644 index 00000000000..9d46180c09b --- /dev/null +++ b/go/vt/vtgate/planbuilder/querygraph_test.go @@ -0,0 +1,126 @@ +/* +Copyright 2020 The Vitess 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 planbuilder + +import ( + "testing" + + "vitess.io/vitess/go/test/utils" + + "github.com/stretchr/testify/require" + "vitess.io/vitess/go/vt/sqlparser" + "vitess.io/vitess/go/vt/vtgate/semantics" + "vitess.io/vitess/go/vt/vtgate/vindexes" +) + +type tcase struct { + input string + output *queryGraph +} + +var tcases = []tcase{ + { + input: "select * from t", + output: &queryGraph{ + tables: []*queryTable{ + { + tableIdentifier: 1, + alias: &sqlparser.AliasedTableExpr{ + Expr: sqlparser.TableName{ + Name: sqlparser.NewTableIdent("t"), + }, + }, + table: sqlparser.TableName{Name: sqlparser.NewTableIdent("t")}, + predicates: nil, + }, + }, + crossTable: map[semantics.TableSet][]sqlparser.Expr{}, + }, + }, { + input: "select t.c from t,y,z where t.c = y.c and (t.a = z.a or t.a = y.a) and 1 < 2", + output: &queryGraph{ + tables: []*queryTable{ + {tableIdentifier: 1, alias: &sqlparser.AliasedTableExpr{Expr: sqlparser.TableName{Name: sqlparser.NewTableIdent("t")}}, table: sqlparser.TableName{Name: sqlparser.NewTableIdent("t")}}, + { + tableIdentifier: 2, + alias: &sqlparser.AliasedTableExpr{Expr: sqlparser.TableName{Name: sqlparser.NewTableIdent("y")}}, + table: sqlparser.TableName{Name: sqlparser.NewTableIdent("y")}, + }, + { + tableIdentifier: 4, + alias: &sqlparser.AliasedTableExpr{Expr: sqlparser.TableName{Name: sqlparser.NewTableIdent("z")}}, + table: sqlparser.TableName{Name: sqlparser.NewTableIdent("z")}, + }, + }, + crossTable: map[semantics.TableSet][]sqlparser.Expr{ + 3: { + &sqlparser.ComparisonExpr{ + Left: &sqlparser.ColName{Name: sqlparser.NewColIdent("c"), Qualifier: sqlparser.TableName{Name: sqlparser.NewTableIdent("t")}}, + Right: &sqlparser.ColName{Name: sqlparser.NewColIdent("c"), Qualifier: sqlparser.TableName{Name: sqlparser.NewTableIdent("y")}}, + }, + }, + 7: { + &sqlparser.OrExpr{ + Left: &sqlparser.ComparisonExpr{ + Left: &sqlparser.ColName{Name: sqlparser.NewColIdent("a"), Qualifier: sqlparser.TableName{Name: sqlparser.NewTableIdent("t")}}, + Right: &sqlparser.ColName{Name: sqlparser.NewColIdent("a"), Qualifier: sqlparser.TableName{Name: sqlparser.NewTableIdent("z")}}, + }, + Right: &sqlparser.ComparisonExpr{ + Left: &sqlparser.ColName{Name: sqlparser.NewColIdent("a"), Qualifier: sqlparser.TableName{Name: sqlparser.NewTableIdent("t")}}, + Right: &sqlparser.ColName{Name: sqlparser.NewColIdent("a"), Qualifier: sqlparser.TableName{Name: sqlparser.NewTableIdent("y")}}, + }, + }, + }, + }, + noDeps: &sqlparser.ComparisonExpr{ + Operator: 1, + Left: &sqlparser.Literal{Type: 1, Val: []uint8{0x31}}, + Right: &sqlparser.Literal{Type: 1, Val: []uint8{0x32}}, + }, + }, + }, +} + +type schemaInf struct{} + +func (node *schemaInf) FindTable(tablename sqlparser.TableName) (*vindexes.Table, error) { + return nil, nil +} + +func TestQueryGraph(t *testing.T) { + for _, tc := range tcases { + sql := tc.input + t.Run(sql, func(t *testing.T) { + tree, err := sqlparser.Parse(sql) + require.NoError(t, err) + semTable, err := semantics.Analyse(tree, &schemaInf{}) + require.NoError(t, err) + qgraph, err := createQGFromSelect(tree.(*sqlparser.Select), semTable) + require.NoError(t, err) + mustMatch(t, tc.output, qgraph, "incorrect query graph") + }) + } +} + +var mustMatch = utils.MustMatchFn( + []interface{}{ // types with unexported fields + queryGraph{}, + queryTable{}, + sqlparser.TableIdent{}, + }, + []string{}, // ignored fields +) diff --git a/go/vt/vtgate/semantics/semantic_state.go b/go/vt/vtgate/semantics/semantic_state.go index 72ee632adc1..45168a3bf06 100644 --- a/go/vt/vtgate/semantics/semantic_state.go +++ b/go/vt/vtgate/semantics/semantic_state.go @@ -32,6 +32,7 @@ type ( // TableSet is how a set of tables is expressed. // Tables get unique bits assigned in the order that they are encountered during semantic analysis TableSet uint64 // we can only join 64 tables with this underlying data type + // TODO : change uint64 to struct to support arbitrary number of tables. // SemTable contains semantic analysis information about the query. SemTable struct { diff --git a/go/vt/vtgate/vcursor_impl.go b/go/vt/vtgate/vcursor_impl.go index ce954f58e34..37de75442eb 100644 --- a/go/vt/vtgate/vcursor_impl.go +++ b/go/vt/vtgate/vcursor_impl.go @@ -23,6 +23,8 @@ import ( "sync/atomic" "time" + "vitess.io/vitess/go/vt/vtgate/semantics" + "golang.org/x/sync/errgroup" "vitess.io/vitess/go/mysql" @@ -97,6 +99,7 @@ type vcursorImpl struct { ignoreMaxMemoryRows bool vschema *vindexes.VSchema vm VSchemaOperator + semTable *semantics.SemTable } func (vc *vcursorImpl) GetKeyspace() string { @@ -343,6 +346,10 @@ func (vc *vcursorImpl) AllKeyspace() ([]*vindexes.Keyspace, error) { return kss, nil } +func (vc *vcursorImpl) GetSemTable() *semantics.SemTable { + return vc.semTable +} + // TargetString returns the current TargetString of the session. func (vc *vcursorImpl) TargetString() string { return vc.safeSession.TargetString From ed8cfe780d24c8e623ce1f30f9c5e9d10c63274b Mon Sep 17 00:00:00 2001 From: GuptaManan100 Date: Tue, 22 Dec 2020 12:08:55 +0530 Subject: [PATCH 03/40] Added 2nd planner test capability to plan_test Signed-off-by: GuptaManan100 --- go/vt/vtgate/planbuilder/plan_test.go | 50 +++++++++++++++++++++------ 1 file changed, 40 insertions(+), 10 deletions(-) diff --git a/go/vt/vtgate/planbuilder/plan_test.go b/go/vt/vtgate/planbuilder/plan_test.go index 7aeb0577038..bb15506603c 100644 --- a/go/vt/vtgate/planbuilder/plan_test.go +++ b/go/vt/vtgate/planbuilder/plan_test.go @@ -376,6 +376,7 @@ func (vw *vschemaWrapper) TargetString() string { } func testFile(t *testing.T, filename, tempDir string, vschema *vschemaWrapper) { + //var checkAllTests = false t.Run(filename, func(t *testing.T) { expected := &strings.Builder{} fail := false @@ -414,11 +415,12 @@ func getPlanOrErrorOutput(err error, plan *engine.Plan) string { } type testCase struct { - file string - lineno int - input string - output string - comments string + file string + lineno int + input string + output string + output2ndPlanner string + comments string } func iterateExecFile(name string) (testCaseIterator chan testCase) { @@ -473,12 +475,40 @@ func iterateExecFile(name string) (testCaseIterator chan testCase) { break } } + + binput, err = r.ReadBytes('\n') + lineno++ + var output2Planner []byte + if err != nil && err != io.EOF { + panic(fmt.Sprintf("error reading file %s line# %d: %s", name, lineno, err.Error())) + } + if len(binput) > 0 && (binput[0] == '"' || binput[0] == '{') { + output2Planner = append(output2Planner, binput...) + for { + l, err := r.ReadBytes('\n') + lineno++ + if err != nil { + panic(fmt.Sprintf("error reading file %s line# %d: %s", name, lineno, err.Error())) + } + output2Planner = append(output2Planner, l...) + if l[0] == '}' { + output2Planner = output2Planner[:len(output2Planner)-1] + break + } + if l[0] == '"' { + output2Planner = output2Planner[1 : len(output2Planner)-2] + break + } + } + } + testCaseIterator <- testCase{ - file: name, - lineno: lineno, - input: input, - output: string(output), - comments: comments, + file: name, + lineno: lineno, + input: input, + output: string(output), + output2ndPlanner: string(output2Planner), + comments: comments, } comments = "" } From 13095ef8173d4e5d4c6b3ef0a2da29c1be12b280 Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Tue, 22 Dec 2020 08:34:09 +0100 Subject: [PATCH 04/40] Test New Planner Test the new planner in plan_test side by side with the old planner Signed-off-by: Andres Taylor --- go/vt/vtgate/planbuilder/builder.go | 1 + go/vt/vtgate/planbuilder/plan_test.go | 35 ++- go/vt/vtgate/planbuilder/route_planning.go | 306 +++++++++++++++++++++ go/vt/vtgate/planbuilder/select.go | 40 ++- go/vt/vtgate/vcursor_impl.go | 4 + 5 files changed, 374 insertions(+), 12 deletions(-) create mode 100644 go/vt/vtgate/planbuilder/route_planning.go diff --git a/go/vt/vtgate/planbuilder/builder.go b/go/vt/vtgate/planbuilder/builder.go index e9a9dddd900..1f98569ef67 100644 --- a/go/vt/vtgate/planbuilder/builder.go +++ b/go/vt/vtgate/planbuilder/builder.go @@ -50,6 +50,7 @@ type ContextVSchema interface { KeyspaceExists(keyspace string) bool AllKeyspace() ([]*vindexes.Keyspace, error) GetSemTable() *semantics.SemTable + NewPlanner() bool } type truncater interface { diff --git a/go/vt/vtgate/planbuilder/plan_test.go b/go/vt/vtgate/planbuilder/plan_test.go index bb15506603c..a9d81b83f87 100644 --- a/go/vt/vtgate/planbuilder/plan_test.go +++ b/go/vt/vtgate/planbuilder/plan_test.go @@ -284,6 +284,7 @@ type vschemaWrapper struct { tabletType topodatapb.TabletType dest key.Destination sysVarEnabled bool + newPlanner bool } func (vw *vschemaWrapper) AllKeyspace() ([]*vindexes.Keyspace, error) { @@ -293,6 +294,9 @@ func (vw *vschemaWrapper) AllKeyspace() ([]*vindexes.Keyspace, error) { return []*vindexes.Keyspace{vw.keyspace}, nil } +func (vw *vschemaWrapper) NewPlanner() bool { + return vw.newPlanner +} func (vw *vschemaWrapper) GetSemTable() *semantics.SemTable { return nil } @@ -376,31 +380,52 @@ func (vw *vschemaWrapper) TargetString() string { } func testFile(t *testing.T, filename, tempDir string, vschema *vschemaWrapper) { - //var checkAllTests = false + var checkAllTests = false t.Run(filename, func(t *testing.T) { expected := &strings.Builder{} fail := false for tcase := range iterateExecFile(filename) { t.Run(tcase.comments, func(t *testing.T) { + vschema.newPlanner = false plan, err := TestBuilder(tcase.input, vschema) - out := getPlanOrErrorOutput(err, plan) if out != tcase.output { fail = true - t.Errorf("File: %s, Line: %d\nDiff:\n%s\n[%s] \n[%s]", filename, tcase.lineno, cmp.Diff(tcase.output, out), tcase.output, out) + t.Errorf("Legacy Planner - File: %s, Line: %d\nDiff:\n%s\n[%s] \n[%s]", filename, tcase.lineno, cmp.Diff(tcase.output, out), tcase.output, out) } - if err != nil { out = `"` + out + `"` } expected.WriteString(fmt.Sprintf("%s\"%s\"\n%s\n\n", tcase.comments, tcase.input, out)) }) + + if tcase.output2ndPlanner != "" || checkAllTests { + t.Run("New Planner: "+tcase.comments, func(t *testing.T) { + if tcase.output2ndPlanner == "" { + tcase.output2ndPlanner = tcase.output + } + vschema.newPlanner = true + plan, err := TestBuilder(tcase.input, vschema) + out := getPlanOrErrorOutput(err, plan) + if out != tcase.output2ndPlanner { + fail = true + t.Errorf("New Planner - File: %s, Line: %d\nDiff:\n%s\n[%s] \n[%s]", filename, tcase.lineno, cmp.Diff(tcase.output2ndPlanner, out), tcase.output, out) + } + if err != nil { + out = `"` + out + `"` + } + + expected.WriteString(out) + }) + } + } + if fail && tempDir != "" { gotFile := fmt.Sprintf("%s/%s", tempDir, filename) - ioutil.WriteFile(gotFile, []byte(strings.TrimSpace(expected.String())+"\n"), 0644) + _ = ioutil.WriteFile(gotFile, []byte(strings.TrimSpace(expected.String())+"\n"), 0644) fmt.Println(fmt.Sprintf("Errors found in plantests. If the output is correct, run `cp %s/* testdata/` to update test expectations", tempDir)) //nolint } }) diff --git a/go/vt/vtgate/planbuilder/route_planning.go b/go/vt/vtgate/planbuilder/route_planning.go new file mode 100644 index 00000000000..3504ddbdf30 --- /dev/null +++ b/go/vt/vtgate/planbuilder/route_planning.go @@ -0,0 +1,306 @@ +/* +Copyright 2020 The Vitess 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 planbuilder + +import ( + "strings" + + "vitess.io/vitess/go/sqltypes" + + vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" + "vitess.io/vitess/go/vt/vtgate/semantics" + "vitess.io/vitess/go/vt/vtgate/vindexes" + + "vitess.io/vitess/go/vt/vterrors" + + "vitess.io/vitess/go/vt/sqlparser" + "vitess.io/vitess/go/vt/vtgate/engine" +) + +type ( + joinTree interface { + solves() semantics.TableSet + cost() int + } + routePlan struct { + routeOpCode engine.RouteOpcode + solved semantics.TableSet + tables []*queryTable + extraPredicates []sqlparser.Expr + keyspace *vindexes.Keyspace + + // vindex and conditions is set if a vindex will be used for this route. + vindex vindexes.Vindex + conditions []sqlparser.Expr + + // this state keeps track of which vindexes are available and + // whether we have seen enough predicates to satisfy the vindex + vindexPlusPredicates []*vindexPlusPredicates + vtable *vindexes.Table + } + joinPlan struct { + predicates []sqlparser.Expr + + lhs, rhs joinTree + } + dpTableT map[semantics.TableSet]joinTree +) + +func (rp *routePlan) solves() semantics.TableSet { + return rp.solved +} +func (*routePlan) cost() int { + return 1 +} + +// vindexPlusPredicates is a struct used to store all the predicates that the vindex can be used to query +type vindexPlusPredicates struct { + vindex *vindexes.ColumnVindex + covered bool + predicates []sqlparser.Expr +} + +func (rp *routePlan) addPredicate(predicates ...sqlparser.Expr) error { + + if len(rp.tables) != 1 { + return vterrors.Errorf(vtrpcpb.Code_INTERNAL, "addPredicate should only be called when the route has a single table") + } + + if rp.vindexPlusPredicates == nil { + // Add all the column vindexes to the list of vindexPlusPredicates + for _, columnVindex := range rp.vtable.ColumnVindexes { + rp.vindexPlusPredicates = append(rp.vindexPlusPredicates, &vindexPlusPredicates{vindex: columnVindex}) + } + } + + for _, filter := range predicates { + switch node := filter.(type) { + case *sqlparser.ComparisonExpr: + switch node.Operator { + case sqlparser.EqualOp: + // TODO(Manan,Andres): Remove the predicates that are repeated eg. Id=1 AND Id=1 + for _, v := range rp.vindexPlusPredicates { + column := node.Left.(*sqlparser.ColName) + for _, col := range v.vindex.Columns { + // If the column for the predicate matches any column in the vindex add it to the list + if column.Name.Equal(col) { + v.predicates = append(v.predicates, node) + // Vindex is covered if all the columns in the vindex have a associated predicate + v.covered = len(v.predicates) == len(v.vindex.Columns) + } + } + } + } + } + } + + //TODO (Manan,Andres): Improve cost metric for vindexes + for _, v := range rp.vindexPlusPredicates { + if !v.covered { + continue + } + // Choose the minimum cost vindex from the ones which are covered + if rp.vindex == nil || v.vindex.Vindex.Cost() < rp.vindex.Cost() { + rp.vindex = v.vindex.Vindex + rp.conditions = v.predicates + } + } + + if rp.vindex != nil { + rp.routeOpCode = engine.SelectEqual + if rp.vindex.IsUnique() { + rp.routeOpCode = engine.SelectEqualUnique + } + } + return nil +} + +// Predicates takes all known predicates for this route and ANDs them together +func (rp *routePlan) Predicates() sqlparser.Expr { + var result sqlparser.Expr + add := func(e sqlparser.Expr) { + if result == nil { + result = e + return + } + result = &sqlparser.AndExpr{ + Left: result, + Right: e, + } + } + for _, t := range rp.tables { + for _, predicate := range t.predicates { + add(predicate) + } + } + for _, p := range rp.extraPredicates { + add(p) + } + return result +} + +func (jp *joinPlan) solves() semantics.TableSet { + return jp.lhs.solves() | jp.rhs.solves() +} +func (jp *joinPlan) cost() int { + return jp.lhs.cost() + jp.rhs.cost() +} + +/* + we use dynamic programming to find the cheapest route/join tree possible, + where the cost of a plan is the number of joins +*/ +func solve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVSchema) (joinTree, error) { + size := len(qg.tables) + dpTable := make(dpTableT) + + var allTables semantics.TableSet + + // we start by seeding the table with the single routes + for _, table := range qg.tables { + solves := semTable.TableSetFor(table.alias) + allTables |= solves + var err error + dpTable[solves], err = createRoutePlan(table, solves, vschema) + if err != nil { + return nil, err + } + } + + for currentSize := 2; currentSize <= size; currentSize++ { + lefts := dpTable.bitSetsOfSize(1) + rights := dpTable.bitSetsOfSize(currentSize - 1) + for _, lhs := range lefts { + for _, rhs := range rights { + if semantics.IsOverlapping(lhs.solves(), rhs.solves()) { + // at least one of the tables is solved on both sides + continue + } + solves := lhs.solves() | rhs.solves() + oldPlan := dpTable[solves] + if oldPlan != nil && oldPlan.cost() == 1 { + // we already have the perfect plan. keep it + continue + } + joinPredicates := qg.crossTable[solves] + newPlan := qg.tryMerge(lhs, rhs, joinPredicates) + if newPlan == nil { + newPlan = &joinPlan{ + lhs: lhs, + rhs: rhs, + predicates: joinPredicates, + } + } + if oldPlan == nil || newPlan.cost() < oldPlan.cost() { + dpTable[solves] = newPlan + } + } + } + } + + return dpTable[allTables], nil +} + +func createRoutePlan(table *queryTable, solves semantics.TableSet, vschema ContextVSchema) (*routePlan, error) { + vschemaTable, _, _, _, _, err := vschema.FindTableOrVindex(table.table) + if err != nil { + return nil, err + } + plan := &routePlan{ + solved: solves, + tables: []*queryTable{table}, + keyspace: vschemaTable.Keyspace, + vtable: vschemaTable, + } + + switch { + case vschemaTable.Type == vindexes.TypeSequence: + plan.routeOpCode = engine.SelectNext + case vschemaTable.Type == vindexes.TypeReference: + plan.routeOpCode = engine.SelectReference + case !vschemaTable.Keyspace.Sharded: + plan.routeOpCode = engine.SelectUnsharded + case vschemaTable.Pinned != nil: + + // Pinned tables have their keyspace ids already assigned. + // Use the Binary vindex, which is the identity function + // for keyspace id. + plan.routeOpCode = engine.SelectEqualUnique + default: + plan.routeOpCode = engine.SelectScatter + err := plan.addPredicate(table.predicates...) + if err != nil { + return nil, err + } + } + + return plan, nil +} + +func transformToLogicalPlan(tree joinTree) (logicalPlan, error) { + switch n := tree.(type) { + case *routePlan: + var tablesForSelect sqlparser.TableExprs + var tableNames []string + + for _, t := range n.tables { + tablesForSelect = append(tablesForSelect, t.alias) + tableNames = append(tableNames, sqlparser.String(t.alias)) + } + predicates := n.Predicates() + var where *sqlparser.Where + if predicates != nil { + where = &sqlparser.Where{Expr: predicates, Type: sqlparser.WhereClause} + } + var values []sqltypes.PlanValue + if len(n.conditions) == 1 { + value, err := sqlparser.NewPlanValue(n.conditions[0].(*sqlparser.ComparisonExpr).Right) + if err != nil { + return nil, err + } + values = []sqltypes.PlanValue{value} + } + var singleColumn vindexes.SingleColumn + if n.vindex != nil { + singleColumn = n.vindex.(vindexes.SingleColumn) + } + return &route{ + eroute: &engine.Route{ + Opcode: n.routeOpCode, + TableName: strings.Join(tableNames, ", "), + Keyspace: n.keyspace, + Vindex: singleColumn, + Values: values, + }, + Select: &sqlparser.Select{ + From: tablesForSelect, + Where: where, + }, + }, nil + + case *joinPlan: + _, err := transformToLogicalPlan(n.lhs) + if err != nil { + return nil, err + } + _, err = transformToLogicalPlan(n.rhs) + if err != nil { + return nil, err + } + } + panic(42) +} diff --git a/go/vt/vtgate/planbuilder/select.go b/go/vt/vtgate/planbuilder/select.go index 87d48316206..f999104a829 100644 --- a/go/vt/vtgate/planbuilder/select.go +++ b/go/vt/vtgate/planbuilder/select.go @@ -20,6 +20,8 @@ import ( "errors" "fmt" + "vitess.io/vitess/go/vt/vtgate/semantics" + "vitess.io/vitess/go/mysql" "vitess.io/vitess/go/vt/key" @@ -36,23 +38,47 @@ import ( func buildSelectPlan(query string) func(sqlparser.Statement, ContextVSchema) (engine.Primitive, error) { return func(stmt sqlparser.Statement, vschema ContextVSchema) (engine.Primitive, error) { sel := stmt.(*sqlparser.Select) + if !vschema.NewPlanner() { + + p, err := handleDualSelects(sel, vschema) + if err != nil { + return nil, err + } + if p != nil { + return p, nil + } - p, err := handleDualSelects(sel, vschema) + pb := newPrimitiveBuilder(vschema, newJointab(sqlparser.GetBindvars(sel))) + if err := pb.processSelect(sel, nil, query); err != nil { + return nil, err + } + if err := pb.plan.Wireup(pb.plan, pb.jt); err != nil { + return nil, err + } + return pb.plan.Primitive(), nil + } + + semTable, err := semantics.Analyse(sel, nil) // TODO no nil no if err != nil { return nil, err } - if p != nil { - return p, nil + + qgraph, err := createQGFromSelect(sel, semTable) + if err != nil { + return nil, err } - pb := newPrimitiveBuilder(vschema, newJointab(sqlparser.GetBindvars(sel))) - if err := pb.processSelect(sel, nil, query); err != nil { + tree, err := solve(qgraph, semTable, vschema) + if err != nil { return nil, err } - if err := pb.plan.Wireup(pb.plan, pb.jt); err != nil { + + plan, err := transformToLogicalPlan(tree) + if err != nil { return nil, err } - return pb.plan.Primitive(), nil + + return plan.Primitive(), nil } } diff --git a/go/vt/vtgate/vcursor_impl.go b/go/vt/vtgate/vcursor_impl.go index 37de75442eb..2e96fe80310 100644 --- a/go/vt/vtgate/vcursor_impl.go +++ b/go/vt/vtgate/vcursor_impl.go @@ -100,6 +100,7 @@ type vcursorImpl struct { vschema *vindexes.VSchema vm VSchemaOperator semTable *semantics.SemTable + newPlanner bool } func (vc *vcursorImpl) GetKeyspace() string { @@ -346,6 +347,9 @@ func (vc *vcursorImpl) AllKeyspace() ([]*vindexes.Keyspace, error) { return kss, nil } +func (vc *vcursorImpl) NewPlanner() bool { + return vc.newPlanner +} func (vc *vcursorImpl) GetSemTable() *semantics.SemTable { return vc.semTable } From 91c8c952978c857ac8cabd38d2cb4155dd5c7aae Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Tue, 22 Dec 2020 09:00:14 +0100 Subject: [PATCH 05/40] added the first two succesful new planner tests Signed-off-by: Andres Taylor --- go/vt/vtgate/planbuilder/plan_test.go | 4 ++++ go/vt/vtgate/planbuilder/route_planning.go | 1 + go/vt/vtgate/planbuilder/select.go | 10 ++++++++++ go/vt/vtgate/planbuilder/testdata/from_cases.txt | 4 ++++ 4 files changed, 19 insertions(+) diff --git a/go/vt/vtgate/planbuilder/plan_test.go b/go/vt/vtgate/planbuilder/plan_test.go index a9d81b83f87..69b5d9d68a2 100644 --- a/go/vt/vtgate/planbuilder/plan_test.go +++ b/go/vt/vtgate/planbuilder/plan_test.go @@ -525,6 +525,10 @@ func iterateExecFile(name string) (testCaseIterator chan testCase) { break } } + if string(output2Planner) == `{ +}` { + output2Planner = output + } } testCaseIterator <- testCase{ diff --git a/go/vt/vtgate/planbuilder/route_planning.go b/go/vt/vtgate/planbuilder/route_planning.go index 3504ddbdf30..c84d2405f1c 100644 --- a/go/vt/vtgate/planbuilder/route_planning.go +++ b/go/vt/vtgate/planbuilder/route_planning.go @@ -278,6 +278,7 @@ func transformToLogicalPlan(tree joinTree) (logicalPlan, error) { if n.vindex != nil { singleColumn = n.vindex.(vindexes.SingleColumn) } + return &route{ eroute: &engine.Route{ Opcode: n.routeOpCode, diff --git a/go/vt/vtgate/planbuilder/select.go b/go/vt/vtgate/planbuilder/select.go index f999104a829..46496b0f1ac 100644 --- a/go/vt/vtgate/planbuilder/select.go +++ b/go/vt/vtgate/planbuilder/select.go @@ -78,6 +78,16 @@ func buildSelectPlan(query string) func(sqlparser.Statement, ContextVSchema) (en return nil, err } + // minimal horizon planning + rb, ok := plan.(*route) + if ok { + rb.Select = sel + rb.eroute.Query = sqlparser.String(sel) + buffer := sqlparser.NewTrackedBuffer(nil) + sqlparser.FormatImpossibleQuery(buffer, sel) + rb.eroute.FieldQuery = buffer.ParsedQuery().Query + } + return plan.Primitive(), nil } } diff --git a/go/vt/vtgate/planbuilder/testdata/from_cases.txt b/go/vt/vtgate/planbuilder/testdata/from_cases.txt index 441085535ad..38dac909f58 100644 --- a/go/vt/vtgate/planbuilder/testdata/from_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/from_cases.txt @@ -15,6 +15,8 @@ "Table": "user" } } +{ +} # Single table unsharded "select col from unsharded" @@ -33,6 +35,8 @@ "Table": "unsharded" } } +{ +} # Select from sequence "select next 2 values from seq" From bdf65b250ccf257dd048e782b93bc3ed369ad767 Mon Sep 17 00:00:00 2001 From: GuptaManan100 Date: Tue, 22 Dec 2020 14:22:30 +0530 Subject: [PATCH 06/40] checked all the tests that work with the 2nd planner Signed-off-by: GuptaManan100 --- go/vt/vtgate/planbuilder/plan_test.go | 36 ++- go/vt/vtgate/planbuilder/route.go | 2 +- .../planbuilder/testdata/aggr_cases.txt | 28 +++ .../vtgate/planbuilder/testdata/dml_cases.txt | 236 ++++++++++++++++++ .../planbuilder/testdata/filter_cases.txt | 48 ++++ .../planbuilder/testdata/from_cases.txt | 17 +- .../planbuilder/testdata/lock_cases.txt | 6 + .../testdata/postprocess_cases.txt | 28 +++ .../planbuilder/testdata/select_cases.txt | 48 ++++ .../vtgate/planbuilder/testdata/set_cases.txt | 40 +++ .../testdata/transaction_cases.txt | 14 ++ .../planbuilder/testdata/union_cases.txt | 40 +++ .../testdata/unsupported_cases.txt | 98 ++++++++ .../vtgate/planbuilder/testdata/use_cases.txt | 10 + .../testdata/vindex_func_cases.txt | 2 + .../planbuilder/testdata/wireup_cases.txt | 4 + 16 files changed, 651 insertions(+), 6 deletions(-) diff --git a/go/vt/vtgate/planbuilder/plan_test.go b/go/vt/vtgate/planbuilder/plan_test.go index 69b5d9d68a2..4cbad7834d4 100644 --- a/go/vt/vtgate/planbuilder/plan_test.go +++ b/go/vt/vtgate/planbuilder/plan_test.go @@ -379,6 +379,10 @@ func (vw *vschemaWrapper) TargetString() string { return "targetString" } +func escapeNewLines(in string) string { + return strings.ReplaceAll(in, "\n", "\\n") +} + func testFile(t *testing.T, filename, tempDir string, vschema *vschemaWrapper) { var checkAllTests = false t.Run(filename, func(t *testing.T) { @@ -398,29 +402,42 @@ func testFile(t *testing.T, filename, tempDir string, vschema *vschemaWrapper) { out = `"` + out + `"` } - expected.WriteString(fmt.Sprintf("%s\"%s\"\n%s\n\n", tcase.comments, tcase.input, out)) + expected.WriteString(fmt.Sprintf("%s\"%s\"\n%s\n", tcase.comments, escapeNewLines(tcase.input), out)) }) if tcase.output2ndPlanner != "" || checkAllTests { t.Run("New Planner: "+tcase.comments, func(t *testing.T) { + expectedVal := "{\n}\n" + empty := false if tcase.output2ndPlanner == "" { + empty = true tcase.output2ndPlanner = tcase.output } vschema.newPlanner = true - plan, err := TestBuilder(tcase.input, vschema) - out := getPlanOrErrorOutput(err, plan) + out, err := getPlanOutput(tcase, vschema) if out != tcase.output2ndPlanner { fail = true + expectedVal = "" t.Errorf("New Planner - File: %s, Line: %d\nDiff:\n%s\n[%s] \n[%s]", filename, tcase.lineno, cmp.Diff(tcase.output2ndPlanner, out), tcase.output, out) } if err != nil { out = `"` + out + `"` } - expected.WriteString(out) + if tcase.output == tcase.output2ndPlanner { + if empty { + expected.WriteString(expectedVal) + } else { + // produce empty brackets when the planners agree + expected.WriteString("{\n}\n") + } + } else { + expected.WriteString(out) + } }) } + expected.WriteString("\n") } if fail && tempDir != "" { @@ -431,6 +448,17 @@ func testFile(t *testing.T, filename, tempDir string, vschema *vschemaWrapper) { }) } +func getPlanOutput(tcase testCase, vschema *vschemaWrapper) (out string, err error) { + defer func() { + if r := recover(); r != nil { + out = fmt.Sprintf("%v", r) + } + }() + plan, err := TestBuilder(tcase.input, vschema) + out = getPlanOrErrorOutput(err, plan) + return out, err +} + func getPlanOrErrorOutput(err error, plan *engine.Plan) string { if err != nil { return err.Error() diff --git a/go/vt/vtgate/planbuilder/route.go b/go/vt/vtgate/planbuilder/route.go index ea6baaa24c7..f1ce57a1144 100644 --- a/go/vt/vtgate/planbuilder/route.go +++ b/go/vt/vtgate/planbuilder/route.go @@ -393,7 +393,7 @@ func (rb *route) JoinCanMerge(pb *primitiveBuilder, rrb *route, ajoin *sqlparser return true } hasRuntimeRoutingPredicates := false - sqlparser.Walk(func(node sqlparser.SQLNode) (kontinue bool, err error) { + _ = sqlparser.Walk(func(node sqlparser.SQLNode) (kontinue bool, err error) { col, ok := node.(*sqlparser.ColName) if ok { hasRuntimeRoutingPredicates = hasRuntimeRoutingPredicates || isTableNameCol(col) || isDbNameCol(col) diff --git a/go/vt/vtgate/planbuilder/testdata/aggr_cases.txt b/go/vt/vtgate/planbuilder/testdata/aggr_cases.txt index 8d194f0cf9d..0691dbaa8d2 100644 --- a/go/vt/vtgate/planbuilder/testdata/aggr_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/aggr_cases.txt @@ -17,6 +17,8 @@ "Table": "unsharded" } } +{ +} # Aggregate on unique sharded "select count(*), col from user where id = 1" @@ -39,6 +41,8 @@ "Vindex": "user_index" } } +{ +} # Aggregate detection (non-aggregate function) "select fun(1), col from user" @@ -57,6 +61,8 @@ "Table": "user" } } +{ +} # select distinct with unique vindex for scatter route. "select distinct col1, id from user" @@ -75,6 +81,8 @@ "Table": "user" } } +{ +} # distinct and group by together for single route. "select distinct col1, id from user group by col1" @@ -93,6 +101,8 @@ "Table": "user" } } +{ +} # scatter group by a text column "select count(*), a, textcol1, b from user group by a, textcol1, b" @@ -342,6 +352,8 @@ "Table": "user" } } +{ +} # group by a unique vindex and other column should use a simple route "select id, col, count(*) from user group by id, col" @@ -360,6 +372,8 @@ "Table": "user" } } +{ +} # group by a non-vindex column should use an OrderdAggregate primitive "select col, count(*) from user group by col" @@ -438,6 +452,8 @@ "Table": "user" } } +{ +} # group by a unique vindex where alias from select list is used "select id as val, 1+count(*) from user group by val" @@ -456,6 +472,8 @@ "Table": "user" } } +{ +} # group by a unique vindex where expression is qualified (alias should be ignored) "select val as id, 1+count(*) from user group by user.id" @@ -474,6 +492,8 @@ "Table": "user" } } +{ +} # group by a unique vindex where it should skip non-aliased expressions. "select *, id, 1+count(*) from user group by id" @@ -492,6 +512,8 @@ "Table": "user" } } +{ +} # group by a unique vindex should revert to simple route, and having clause should find the correct symbols. "select id, count(*) c from user group by id having id=1 and c=10" @@ -646,6 +668,8 @@ "Table": "user" } } +{ +} # count with distinct unique vindex "select col, count(distinct id) from user group by col" @@ -1197,6 +1221,8 @@ "Vindex": "user_index" } } +{ +} # routing rules for aggregates "select id, count(*) from route2 group by id" @@ -1233,6 +1259,8 @@ "Table": "ref" } } +{ +} # distinct and aggregate functions missing group by "select distinct a, count(*) from user" diff --git a/go/vt/vtgate/planbuilder/testdata/dml_cases.txt b/go/vt/vtgate/planbuilder/testdata/dml_cases.txt index 435378c7e04..be4db7a0b4b 100644 --- a/go/vt/vtgate/planbuilder/testdata/dml_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/dml_cases.txt @@ -1,10 +1,14 @@ # update table not found "update nouser set val = 1" "table nouser not found" +{ +} # delete table not found "delete from nouser" "table nouser not found" +{ +} # explicit keyspace reference "update main.m1 set val = 1" @@ -23,6 +27,8 @@ "Query": "update m1 set val = 1" } } +{ +} # update unsharded "update unsharded set val = 1" @@ -41,6 +47,8 @@ "Query": "update unsharded set val = 1" } } +{ +} # subqueries in unsharded update "update unsharded set col = (select col from unsharded limit 1)" @@ -59,6 +67,8 @@ "Query": "update unsharded set col = (select col from unsharded limit 1)" } } +{ +} # unsharded union in subquery of unsharded update "update unsharded set col = (select id from unsharded union select id from unsharded)" @@ -77,6 +87,8 @@ "Query": "update unsharded set col = (select id from unsharded union select id from unsharded)" } } +{ +} # unsharded join in subquery of unsharded update "update unsharded set col = (select id from unsharded a join unsharded b on a.id = b.id)" @@ -95,6 +107,8 @@ "Query": "update unsharded set col = (select id from unsharded as a join unsharded as b on a.id = b.id)" } } +{ +} # update with join subquery "update unsharded as foo left join (select id from unsharded where col is not null order by col desc limit 10) as keepers on foo.id = keepers.id set col1 = 'asdf' where keepers.id is null and foo.col is not null and foo.col < 1000" @@ -113,6 +127,8 @@ "Query": "update unsharded as foo left join (select id from unsharded where col is not null order by col desc limit 10) as keepers on foo.id = keepers.id set col1 = 'asdf' where keepers.id is null and foo.col is not null and foo.col \u003c 1000" } } +{ +} # routing rules: updated of a routed table "update route1 set a=1 where id=1" @@ -136,6 +152,8 @@ "Vindex": "user_index" } } +{ +} # update: routing rules for subquery. "update unsharded_a set a=(select a from route2)" @@ -154,6 +172,8 @@ "Query": "update unsharded_a set a = (select a from unsharded as route2)" } } +{ +} # delete unsharded "delete from unsharded" @@ -172,6 +192,8 @@ "Query": "delete from unsharded" } } +{ +} # update by primary keyspace id "update user set val = 1 where id = 1" @@ -195,6 +217,8 @@ "Vindex": "user_index" } } +{ +} # update by primary keyspace id with alias "update user as user_alias set val = 1 where user_alias.id = 1" @@ -218,6 +242,8 @@ "Vindex": "user_index" } } +{ +} # update by primary keyspace id with parenthesized expression "update user set val = 1 where (id = 1)" @@ -241,6 +267,8 @@ "Vindex": "user_index" } } +{ +} # update by primary keyspace id with multi-part where clause with parens "update user set val = 1 where (name = 'foo' and id = 1)" @@ -264,6 +292,8 @@ "Vindex": "user_index" } } +{ +} # update by primary keyspace id, changing one vindex column "update user_metadata set email = 'juan@vitess.io' where user_id = 1" @@ -292,10 +322,14 @@ "Vindex": "user_index" } } +{ +} # update by primary keyspace id, changing same vindex twice "update user_metadata set email = 'a', email = 'b' where user_id = 1" "column has duplicate set values: 'email'" +{ +} # update by primary keyspace id, changing multiple vindex columns "update user_metadata set email = 'juan@vitess.io', address = '155 5th street' where user_id = 1" @@ -325,6 +359,8 @@ "Vindex": "user_index" } } +{ +} # update by primary keyspace id, changing one vindex column, using order by and limit "update user_metadata set email = 'juan@vitess.io' where user_id = 1 order by user_id asc limit 10" @@ -353,6 +389,8 @@ "Vindex": "user_index" } } +{ +} # update by primary keyspace id, stray where clause "update user set val = 1 where id = id2 and id = 1" @@ -376,6 +414,8 @@ "Vindex": "user_index" } } +{ +} # update by primary keyspace id, stray where clause with conversion error "update user set val = 1 where id = 18446744073709551616 and id = 1" @@ -399,6 +439,8 @@ "Vindex": "user_index" } } +{ +} # delete from by primary keyspace id "delete from user where id = 1" @@ -424,6 +466,8 @@ "Vindex": "user_index" } } +{ +} # multi-table delete with comma join "delete a from unsharded_a a, unsharded_b b where a.id = b.id and b.val = 1" @@ -442,6 +486,8 @@ "Query": "delete a from unsharded_a as a, unsharded_b as b where a.id = b.id and b.val = 1" } } +{ +} # multi-table delete with ansi join "delete a from unsharded_a a join unsharded_b b on a.id = b.id where b.val = 1" @@ -460,6 +506,8 @@ "Query": "delete a from unsharded_a as a join unsharded_b as b on a.id = b.id where b.val = 1" } } +{ +} #delete with join from subquery "delete foo from unsharded as foo left join (select id from unsharded where col is not null order by col desc limit 10) as keepers on foo.id = keepers.id where keepers.id is null and foo.col is not null and foo.col < 1000" @@ -478,6 +526,8 @@ "Query": "delete foo from unsharded as foo left join (select id from unsharded where col is not null order by col desc limit 10) as keepers on foo.id = keepers.id where keepers.id is null and foo.col is not null and foo.col \u003c 1000" } } +{ +} # routing rules: deleted from a routed table "delete from route1 where id = 1" @@ -503,6 +553,8 @@ "Vindex": "user_index" } } +{ +} # delete: routing rules for subquery "delete from unsharded_a where a=(select a from route2)" @@ -521,6 +573,8 @@ "Query": "delete from unsharded_a where a = (select a from unsharded as route2)" } } +{ +} # update by lookup "update music set val = 1 where id = 1" @@ -544,6 +598,8 @@ "Vindex": "music_user_map" } } +{ +} # update multi-table ansi join "update unsharded_a a join unsharded_b b on a.id = b.id set a.val = 'foo' where b.val = 1" @@ -562,6 +618,8 @@ "Query": "update unsharded_a as a join unsharded_b as b on a.id = b.id set a.val = 'foo' where b.val = 1" } } +{ +} # update multi-table comma join "update unsharded_a a, unsharded_b b set a.val = 'foo' where a.id = b.id and b.val = 1" @@ -580,6 +638,8 @@ "Query": "update unsharded_a as a, unsharded_b as b set a.val = 'foo' where a.id = b.id and b.val = 1" } } +{ +} # delete from by lookup "delete from music where id = 1" @@ -605,6 +665,8 @@ "Vindex": "music_user_map" } } +{ +} # delete from, no owned vindexes "delete from music_extra where user_id = 1" @@ -628,6 +690,8 @@ "Vindex": "user_index" } } +{ +} # simple insert, no values "insert into unsharded values()" @@ -647,6 +711,8 @@ "TableName": "unsharded" } } +{ +} # simple insert unsharded "insert into unsharded values(1, 2)" @@ -666,6 +732,8 @@ "TableName": "unsharded" } } +{ +} # simple upsert unsharded "insert into unsharded values(1, 2) on duplicate key update x = 3" @@ -685,6 +753,8 @@ "TableName": "unsharded" } } +{ +} # unsharded insert, no col list with auto-inc and authoritative column list "insert into unsharded_authoritative values(1,1)" @@ -704,6 +774,8 @@ "TableName": "unsharded_authoritative" } } +{ +} # sharded upsert with sharding key set to vindex column "insert into music(user_id, id) values(1, 2) on duplicate key update user_id = values(user_id)" @@ -723,6 +795,8 @@ "TableName": "music" } } +{ +} # sharded bulk upsert with sharding key set to vindex column "insert into music(user_id, id) values (1, 2), (3,4) on duplicate key update user_id = values(user_id)" @@ -742,6 +816,8 @@ "TableName": "music" } } +{ +} # insert unsharded with select "insert into unsharded select id from unsharded_auto" @@ -761,6 +837,8 @@ "TableName": "unsharded" } } +{ +} # insert unsharded with select with join "insert into unsharded select id from unsharded join unsharded_auto" @@ -780,10 +858,14 @@ "TableName": "unsharded" } } +{ +} # insert unsharded, invalid value for auto-inc "insert into unsharded_auto(id, val) values(18446744073709551616, 'aa')" "could not compute value for vindex or auto-inc column: strconv.ParseUint: parsing "18446744073709551616": value out of range" +{ +} # insert unsharded, column present "insert into unsharded_auto(id, val) values(1, 'aa')" @@ -803,6 +885,8 @@ "TableName": "unsharded_auto" } } +{ +} # insert unsharded, column absent "insert into unsharded_auto(val) values('aa')" @@ -822,6 +906,8 @@ "TableName": "unsharded_auto" } } +{ +} # insert unsharded, column absent "insert into unsharded_auto(val) values(false)" @@ -841,6 +927,8 @@ "TableName": "unsharded_auto" } } +{ +} # insert unsharded, multi-val "insert into unsharded_auto(id, val) values(1, 'aa'), (null, 'bb')" @@ -860,6 +948,8 @@ "TableName": "unsharded_auto" } } +{ +} # unsharded insert subquery in insert value "insert into unsharded values((select 1 from dual), 1)" @@ -879,6 +969,8 @@ "TableName": "unsharded" } } +{ +} # sharded insert subquery in insert value "insert into user(id, val) values((select 1), 1)" @@ -898,6 +990,8 @@ "TableName": "user" } } +{ +} # insert into a routed table "insert into route1(id) values (1)" @@ -917,10 +1011,14 @@ "TableName": "user" } } +{ +} # insert with mimatched column list "insert into user(id) values (1, 2)" "column list doesn't match values" +{ +} # insert no column list for sharded authoritative table "insert into authoritative values(1, 2, 3)" @@ -940,6 +1038,8 @@ "TableName": "authoritative" } } +{ +} # insert sharded, no values "insert into user values()" @@ -959,6 +1059,8 @@ "TableName": "user" } } +{ +} # insert with one vindex "insert into user(id) values (1)" @@ -978,6 +1080,8 @@ "TableName": "user" } } +{ +} # insert ignore sharded "insert ignore into user(id) values (1)" @@ -997,6 +1101,8 @@ "TableName": "user" } } +{ +} # insert on duplicate key "insert into user(id) values(1) on duplicate key update col = 2" @@ -1016,6 +1122,8 @@ "TableName": "user" } } +{ +} # insert with one vindex and bind var "insert into user(id) values (:aa)" @@ -1035,6 +1143,8 @@ "TableName": "user" } } +{ +} # insert with non vindex "insert into user(nonid) values (2)" @@ -1054,6 +1164,8 @@ "TableName": "user" } } +{ +} # insert with default seq "insert into user(id, nonid) values (default, 2)" @@ -1073,6 +1185,8 @@ "TableName": "user" } } +{ +} # insert with non vindex bool value "insert into user(nonid) values (true)" @@ -1092,6 +1206,8 @@ "TableName": "user" } } +{ +} # insert with all vindexes supplied "insert into user(nonid, name, id) values (2, 'foo', 1)" @@ -1111,6 +1227,8 @@ "TableName": "user" } } +{ +} # insert for non-vindex autoinc "insert into user_extra(nonid) values (2)" @@ -1130,6 +1248,8 @@ "TableName": "user_extra" } } +{ +} # insert for non-compliant names "insert into `weird``name`(`a``b*c`, `b*c`) values(1, 2)" @@ -1149,6 +1269,8 @@ "TableName": "weird`name" } } +{ +} # unsharded insert from union "insert into unsharded select 1 from dual union select 1 from dual" @@ -1168,22 +1290,32 @@ "TableName": "unsharded" } } +{ +} # insert for non-vindex autoinc, invalid value "insert into user_extra(nonid, extra_id) values (2, 18446744073709551616)" "could not compute value for vindex or auto-inc column: strconv.ParseUint: parsing "18446744073709551616": value out of range" +{ +} # insert invalid index value "insert into music_extra(music_id, user_id) values(1, 18446744073709551616)" "could not compute value for vindex or auto-inc column: strconv.ParseUint: parsing "18446744073709551616": value out of range" +{ +} # insert invalid index value "insert into music_extra(music_id, user_id) values(1, id)" "could not compute value for vindex or auto-inc column: expression is too complex 'id'" +{ +} # insert invalid table "insert into noexist(music_id, user_id) values(1, 18446744073709551616)" "table noexist not found" +{ +} # insert with multiple rows "insert into user(id) values (1), (2)" @@ -1203,6 +1335,8 @@ "TableName": "user" } } +{ +} # insert with query timeout "insert /*vt+ QUERY_TIMEOUT_MS=1 */ into user(id) values (1), (2)" @@ -1223,6 +1357,8 @@ "TableName": "user" } } +{ +} # insert with multiple rows - multi-shard autocommit "insert /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ into user(id) values (1), (2)" @@ -1242,10 +1378,14 @@ "TableName": "user" } } +{ +} # insert into a vindex not allowed "insert into user_index(id) values(1)" "unsupported: multi-shard or vindex write statement" +{ +} # simple replace unsharded "replace into unsharded values(1, 2)" @@ -1265,6 +1405,8 @@ "TableName": "unsharded" } } +{ +} # replace unsharded with select "replace into unsharded select id from unsharded_auto" @@ -1284,10 +1426,14 @@ "TableName": "unsharded" } } +{ +} # replace unsharded, invalid value for auto-inc "replace into unsharded_auto(id, val) values(18446744073709551616, 'aa')" "could not compute value for vindex or auto-inc column: strconv.ParseUint: parsing "18446744073709551616": value out of range" +{ +} # replace unsharded, column present "replace into unsharded_auto(id, val) values(1, 'aa')" @@ -1307,6 +1453,8 @@ "TableName": "unsharded_auto" } } +{ +} # replace unsharded, column absent "replace into unsharded_auto(val) values('aa')" @@ -1326,6 +1474,8 @@ "TableName": "unsharded_auto" } } +{ +} # replace unsharded, multi-val "replace into unsharded_auto(id, val) values(1, 'aa'), (null, 'bb')" @@ -1345,10 +1495,14 @@ "TableName": "unsharded_auto" } } +{ +} # replace invalid table "replace into noexist(music_id, user_id) values(1, 18446744073709551616)" "table noexist not found" +{ +} # insert a row in a multi column vindex table "insert multicolvin (column_a, column_b, column_c, kid) VALUES (1,2,3,4)" @@ -1368,6 +1522,8 @@ "TableName": "multicolvin" } } +{ +} # insert for overlapped vindex columns "insert overlap_vindex (kid, column_a, column_b) VALUES (1,2,3)" @@ -1387,6 +1543,8 @@ "TableName": "overlap_vindex" } } +{ +} # insert multiple rows in a multi column vindex table "insert multicolvin (column_a, column_b, column_c, kid) VALUES (1,2,3,4), (5,6,7,8)" @@ -1406,6 +1564,8 @@ "TableName": "multicolvin" } } +{ +} # delete row in a multi column vindex table "delete from multicolvin where kid=1" @@ -1431,6 +1591,8 @@ "Vindex": "kid_index" } } +{ +} # update columns of multi column vindex "update multicolvin set column_b = 1, column_c = 2 where kid = 1" @@ -1459,6 +1621,8 @@ "Vindex": "kid_index" } } +{ +} # update multiple vindexes, with multi column vindex "update multicolvin set column_a = 0, column_b = 1, column_c = 2 where kid = 1" @@ -1488,6 +1652,8 @@ "Vindex": "kid_index" } } +{ +} # update with no primary vindex on where clause (scatter update) "update user_extra set val = 1" @@ -1507,6 +1673,8 @@ "Table": "user_extra" } } +{ +} # update with target destination "update `user[-]`.user_extra set val = 1" @@ -1526,6 +1694,8 @@ "Table": "user_extra" } } +{ +} # update with no primary vindex on where clause (scatter update) - multi shard autocommit "update /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ user_extra set val = 1" @@ -1545,6 +1715,8 @@ "Table": "user_extra" } } +{ +} # update with no primary vindex on where clause (scatter update) - query timeout "update /*vt+ QUERY_TIMEOUT_MS=1 */ user_extra set val = 1" @@ -1565,6 +1737,8 @@ "Table": "user_extra" } } +{ +} # update with non-comparison expr "update user_extra set val = 1 where id between 1 and 2" @@ -1584,6 +1758,8 @@ "Table": "user_extra" } } +{ +} # update with primary id through IN clause "update user_extra set val = 1 where user_id in (1, 2)" @@ -1610,6 +1786,8 @@ "Vindex": "user_index" } } +{ +} # update with non-unique key "update user_extra set val = 1 where name = 'foo'" @@ -1629,6 +1807,8 @@ "Table": "user_extra" } } +{ +} # update by lookup with IN clause "update user_extra set val = 1 where id in (1, 2)" @@ -1648,6 +1828,8 @@ "Table": "user_extra" } } +{ +} # update with where clause with parens "update user_extra set val = 1 where (name = 'foo' or id = 1)" @@ -1667,6 +1849,8 @@ "Table": "user_extra" } } +{ +} # delete from with no where clause "delete from user_extra" @@ -1686,6 +1870,8 @@ "Table": "user_extra" } } +{ +} # delete with target destination "delete from `user[-]`.user_extra" @@ -1705,6 +1891,8 @@ "Table": "user_extra" } } +{ +} # delete with non-comparison expr "delete from user_extra where user_id between 1 and 2" @@ -1724,6 +1912,8 @@ "Table": "user_extra" } } +{ +} # delete from with no index match "delete from user_extra where name = 'jose'" @@ -1743,6 +1933,8 @@ "Table": "user_extra" } } +{ +} # delete from with no index match - multi shard autocommit "delete /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ from user_extra where name = 'jose'" @@ -1762,6 +1954,8 @@ "Table": "user_extra" } } +{ +} # delete from with no index match - query timeout "delete /*vt+ QUERY_TIMEOUT_MS=1 */ from user_extra where name = 'jose'" @@ -1782,6 +1976,8 @@ "Table": "user_extra" } } +{ +} # delete from with primary id in through IN clause "delete from user_extra where user_id in (1, 2)" @@ -1808,6 +2004,8 @@ "Vindex": "user_index" } } +{ +} # unsharded update where inner query references outer query "update unsharded set col = (select id from unsharded_a where id = unsharded.col) where col = (select id from unsharded_b)" @@ -1826,6 +2024,8 @@ "Query": "update unsharded set col = (select id from unsharded_a where id = unsharded.col) where col = (select id from unsharded_b)" } } +{ +} # unsharded delete where inner query references outer query "delete from unsharded where col = (select id from unsharded_a where id = unsharded.col)" @@ -1844,6 +2044,8 @@ "Query": "delete from unsharded where col = (select id from unsharded_a where id = unsharded.col)" } } +{ +} # update vindex value to null "update user set name = null where id = 1" @@ -1872,6 +2074,8 @@ "Vindex": "user_index" } } +{ +} # insert using last_insert_id "insert into unsharded values(last_insert_id(), 2)" @@ -1891,6 +2095,8 @@ "TableName": "unsharded" } } +{ +} # update vindex value to null with multiple primary keyspace id "update user set name = null where id in (1, 2, 3)" @@ -1923,6 +2129,8 @@ "Vindex": "user_index" } } +{ +} # update vindex value to null without a where clause "update user set name = null" @@ -1947,6 +2155,8 @@ "Table": "user" } } +{ +} # update vindex value to null with complex where clause "update user set name = null where id + 1 = 2" @@ -1971,6 +2181,8 @@ "Table": "user" } } +{ +} # delete from user by primary keyspace id with in clause "delete from user where id in (1, 2, 3)" @@ -2000,6 +2212,8 @@ "Vindex": "user_index" } } +{ +} # delete from user by complex expression "delete from user where id + 1 = 2" @@ -2021,6 +2235,8 @@ "Table": "user" } } +{ +} # delete from user without a where clause "delete from user" @@ -2042,6 +2258,8 @@ "Table": "user" } } +{ +} # delete with single table targets "delete music from music where id = 1" @@ -2067,6 +2285,8 @@ "Vindex": "music_user_map" } } +{ +} # scatter update table with owned vindexes without changing lookup vindex "update user set val = 1" @@ -2086,6 +2306,8 @@ "Table": "user" } } +{ +} # scatter delete with owned lookup vindex "delete from user" @@ -2107,6 +2329,8 @@ "Table": "user" } } +{ +} # update multi column vindex, without values for all the vindex columns "update multicolvin set column_c = 2 where kid = 1" @@ -2135,6 +2359,8 @@ "Vindex": "kid_index" } } +{ +} # update with binary value "update user set name = _binary 'abc' where id = 1" @@ -2163,6 +2389,8 @@ "Vindex": "user_index" } } +{ +} # delete with binary value "delete from user where name = _binary 'abc'" @@ -2184,6 +2412,8 @@ "Table": "user" } } +{ +} # delete with shard targeting "delete from `user[-]`.user" @@ -2205,6 +2435,8 @@ "Table": "user" } } +{ +} # update with shard targeting "update `user[-]`.user set name = 'myname'" @@ -2229,6 +2461,8 @@ "Table": "user" } } +{ +} # update with shard targeting without vindex "update `user[-]`.user_extra set val = 1" @@ -2248,3 +2482,5 @@ "Table": "user_extra" } } +{ +} diff --git a/go/vt/vtgate/planbuilder/testdata/filter_cases.txt b/go/vt/vtgate/planbuilder/testdata/filter_cases.txt index 48682ad7d6d..353ad1a84fb 100644 --- a/go/vt/vtgate/planbuilder/testdata/filter_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/filter_cases.txt @@ -15,6 +15,8 @@ "Table": "user" } } +{ +} # Query that always return empty "select id from user where someColumn = null" @@ -55,6 +57,8 @@ "Vindex": "user_index" } } +{ +} # Single table unique vindex route, but complex expr "select id from user where user.id = 5+5" @@ -95,6 +99,8 @@ "Vindex": "user_index" } } +{ +} # Single table multiple non-unique vindex match "select id from user where costly = 'aa' and name = 'bb'" @@ -117,6 +123,8 @@ "Vindex": "name_user_map" } } +{ +} # Single table multiple non-unique vindex match for IN clause "select id from user where costly in ('aa', 'bb') and name in ('aa', 'bb')" @@ -288,6 +296,8 @@ "Vindex": "user_index" } } +{ +} # Composite IN: multiple vindex matches "select id from user where (costly, name) in (('aa', 'bb'), ('cc', 'dd'))" @@ -381,6 +391,8 @@ "Table": "user" } } +{ +} # Composite IN: RHS not tuple "select id from user where (col1, name) in (select * from music where music.user_id=user.id)" @@ -399,6 +411,8 @@ "Table": "user" } } +{ +} # Composite IN: RHS has no simple values "select id from user where (col1, name) in (('aa', 1+1))" @@ -417,6 +431,8 @@ "Table": "user" } } +{ +} # IN clause: LHS is neither column nor composite tuple "select Id from user where 1 in ('aa', 'bb')" @@ -435,6 +451,8 @@ "Table": "user" } } +{ +} # Single table complex in clause "select id from user where name in (col, 'bb')" @@ -453,6 +471,8 @@ "Table": "user" } } +{ +} # Single table equality route with val arg "select id from user where name = :a" @@ -475,6 +495,8 @@ "Vindex": "name_user_map" } } +{ +} # Single table equality route with unsigned value "select id from user where name = 18446744073709551615" @@ -497,6 +519,8 @@ "Vindex": "name_user_map" } } +{ +} # Single table in clause list arg "select id from user where name in ::list" @@ -843,6 +867,8 @@ "Vindex": "name_user_map" } } +{ +} # Route with multiple route constraints, SelectEqual is the best constraint. "select id from user where user.col = false and user.id in (1, 2) and user.name = 'aa'" @@ -865,6 +891,8 @@ "Vindex": "name_user_map" } } +{ +} # Route with multiple route constraints, SelectEqualUnique is the best constraint. "select id from user where user.col = 5 and user.id in (1, 2) and user.name = 'aa' and user.id = 1" @@ -887,6 +915,8 @@ "Vindex": "user_index" } } +{ +} # Route with multiple route constraints, SelectEqualUnique is the best constraint, order reversed. "select id from user where user.id = 1 and user.name = 'aa' and user.id in (1, 2) and user.col = 5" @@ -909,6 +939,8 @@ "Vindex": "user_index" } } +{ +} # Route with OR and AND clause, must parenthesize correctly. "select id from user where user.id = 1 or user.name = 'aa' and user.id in (1, 2)" @@ -927,6 +959,8 @@ "Table": "user" } } +{ +} # Unsharded route "select unsharded.id from user join unsharded where unsharded.id = user.id" @@ -1150,6 +1184,8 @@ "Table": "user" } } +{ +} # outer and inner subquery route by same int val "select id from user where id = 5 and user.col in (select user_extra.col from user_extra where user_extra.user_id = 5)" @@ -1172,6 +1208,8 @@ "Vindex": "user_index" } } +{ +} # outer and inner subquery route by same str val "select id from user where id = 'aa' and user.col in (select user_extra.col from user_extra where user_extra.user_id = 'aa')" @@ -1194,6 +1232,8 @@ "Vindex": "user_index" } } +{ +} # outer and inner subquery route by same val arg "select id from user where id = :a and user.col in (select user_extra.col from user_extra where user_extra.user_id = :a)" @@ -1216,6 +1256,8 @@ "Vindex": "user_index" } } +{ +} # unresolved symbol in inner subquery. "select id from user where id = :a and user.col in (select user_extra.col from user_extra where user_extra.user_id = :a and foo.id = 1)" @@ -1540,6 +1582,8 @@ "Table": "user" } } +{ +} # outer and inner subquery route reference the same "uu.id" name # but they refer to different things. The first reference is to the outermost query, @@ -1604,6 +1648,8 @@ "Table": "music" } } +{ +} # Single table with unique vindex match and null match "select id from music where user_id = 4 and id = null" @@ -1662,6 +1708,8 @@ "Vindex": "user_index" } } +{ +} # Single table with unique vindex match and NOT IN (null, 1, 2) "select id from music where user_id = 4 and id NOT IN (null, 1, 2)" diff --git a/go/vt/vtgate/planbuilder/testdata/from_cases.txt b/go/vt/vtgate/planbuilder/testdata/from_cases.txt index 38dac909f58..ea9b5555f05 100644 --- a/go/vt/vtgate/planbuilder/testdata/from_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/from_cases.txt @@ -55,6 +55,8 @@ "Table": "seq" } } +{ +} # Select from reference "select * from ref" @@ -73,6 +75,8 @@ "Table": "ref" } } +{ +} # Single information_schema query "select col from information_schema.foo" @@ -239,10 +243,14 @@ # routing rules bad table "select * from bad_table" "keyspace noks not found in vschema" +{ +} # routing rules disabled table "select * from disabled" "table disabled has been disabled" +{ +} # ',' join "select music.col from user, music" @@ -2064,6 +2072,8 @@ "Table": "user" } } +{ +} # last_insert_id for unsharded route "select last_insert_id() from main.unsharded" @@ -2127,6 +2137,8 @@ # first expression fails for ',' join (code coverage: ensure error is returned) "select user.foo.col from user.foo, user" "table foo not found" +{ +} # table names should be case-sensitive "select unsharded.id from unsharded where Unsharded.val = 1" @@ -2135,6 +2147,8 @@ # implicit table reference for sharded keyspace "select user.foo.col from user.foo" "table foo not found" +{ +} # duplicate symbols "select user.id from user join user" @@ -2147,6 +2161,8 @@ # non-existent table "select c from t" "table t not found" +{ +} # non-existent table on left of join "select c from t join user" @@ -2450,4 +2466,3 @@ "SysTableTableSchema": "VARBINARY(\"a\")" } } - diff --git a/go/vt/vtgate/planbuilder/testdata/lock_cases.txt b/go/vt/vtgate/planbuilder/testdata/lock_cases.txt index 91f88105e8e..d77f70a801a 100644 --- a/go/vt/vtgate/planbuilder/testdata/lock_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/lock_cases.txt @@ -39,6 +39,8 @@ "OperatorType": "Rows" } } +{ +} # lock tables write "lock tables t low_priority write" @@ -49,6 +51,8 @@ "OperatorType": "Rows" } } +{ +} # unlock tables "unlock tables" @@ -59,3 +63,5 @@ "OperatorType": "Rows" } } +{ +} diff --git a/go/vt/vtgate/planbuilder/testdata/postprocess_cases.txt b/go/vt/vtgate/planbuilder/testdata/postprocess_cases.txt index ff2628fb9b3..794f72864d3 100644 --- a/go/vt/vtgate/planbuilder/testdata/postprocess_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/postprocess_cases.txt @@ -15,6 +15,8 @@ "Table": "user" } } +{ +} # ambiguous symbol reference "select user.col1, user_extra.col1 from user join user_extra having col1 = 2" @@ -154,6 +156,8 @@ "Vindex": "user_index" } } +{ +} # ORDER BY uses column numbers "select col from user where id = 1 order by 1" @@ -176,6 +180,8 @@ "Vindex": "user_index" } } +{ +} # ORDER BY on scatter "select col from user order by col" @@ -312,6 +318,8 @@ "Table": "user" } } +{ +} # ORDER BY after pull-out subquery "select col from user where col in (select col2 from user) order by col" @@ -536,6 +544,8 @@ "Table": "user" } } +{ +} # ORDER BY RAND() for join "select user.col1 as a, user.col2, music.col3 from user join music on user.id = music.id where user.id = 1 order by RAND()" @@ -638,6 +648,8 @@ "Vindex": "user_index" } } +{ +} # Order by, qualified '*' expression "select user.* from user where id = 5 order by user.col" @@ -660,6 +672,8 @@ "Vindex": "user_index" } } +{ +} # Order by, '*' expression with qualified reference "select * from user where id = 5 order by user.col" @@ -682,6 +696,8 @@ "Vindex": "user_index" } } +{ +} # Order by, '*' expression in a subquery "select u.id, e.id from user u join user_extra e where u.col = e.col and u.col in (select * from user where user.id = u.id order by col)" @@ -757,6 +773,8 @@ "Vindex": "user_index" } } +{ +} #Order by with math functions "select * from user where id = 5 order by -col1" @@ -779,6 +797,8 @@ "Vindex": "user_index" } } +{ +} #Order by with string operations "select * from user where id = 5 order by concat(col,col1) collate utf8_general_ci desc" @@ -801,6 +821,8 @@ "Vindex": "user_index" } } +{ +} #Order by with math operations "select * from user where id = 5 order by id+col collate utf8_general_ci desc" @@ -823,6 +845,8 @@ "Vindex": "user_index" } } +{ +} #Order by subquery column "select * from user u join (select user_id from user_extra where user_id = 5) eu on u.id = eu.user_id where u.id = 5 order by eu.user_id" @@ -889,6 +913,8 @@ "Vindex": "user_index" } } +{ +} # limit for joins. Can't push down the limit because result # counts get multiplied by join operations. @@ -1064,6 +1090,8 @@ "Table": "ref" } } +{ +} # invalid limit expression "select id from user limit 1+1" diff --git a/go/vt/vtgate/planbuilder/testdata/select_cases.txt b/go/vt/vtgate/planbuilder/testdata/select_cases.txt index e80cb524aef..c72fa09d290 100644 --- a/go/vt/vtgate/planbuilder/testdata/select_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/select_cases.txt @@ -15,6 +15,8 @@ "Table": "user" } } +{ +} # '*' expression for simple route "select user.* from user" @@ -33,6 +35,8 @@ "Table": "user" } } +{ +} # unqualified '*' expression for simple route "select * from user" @@ -51,6 +55,8 @@ "Table": "user" } } +{ +} # select with timeout directive sets QueryTimeout in the route "select /*vt+ QUERY_TIMEOUT_MS=1000 */ * from user" @@ -69,6 +75,8 @@ "Table": "user" } } +{ +} # select aggregation with timeout directive sets QueryTimeout in the route "select /*vt+ QUERY_TIMEOUT_MS=1000 */ count(*) from user" @@ -137,6 +145,8 @@ "Table": "user" } } +{ +} # select aggregation with partial scatter directive "select /*vt+ SCATTER_ERRORS_AS_WARNINGS=1 */ count(*) from user" @@ -231,6 +241,8 @@ "Table": "user" } } +{ +} # fully qualified '*' expression for simple route "select user.user.* from user.user" @@ -463,6 +475,8 @@ "Table": "dual" } } +{ +} # select from pinned table "select * from pin_test" @@ -803,6 +817,8 @@ # syntax error "the quick brown fox" "syntax error at position 4 near 'the'" +{ +} # Hex number is not treated as a simple value "select * from user where id = 0x04" @@ -869,6 +885,8 @@ "Vindex": "user_index" } } +{ +} # Multiple parenthesized expressions "select * from user where (id = 4) AND (name ='abc') limit 5" @@ -891,6 +909,8 @@ "Vindex": "user_index" } } +{ +} # Multiple parenthesized expressions "select * from user where (id = 4 and name ='abc') limit 5" @@ -913,6 +933,8 @@ "Vindex": "user_index" } } +{ +} # Column Aliasing with Table.Column "select user0_.col as col0_ from user user0_ where id = 1 order by user0_.col desc limit 2" @@ -979,6 +1001,8 @@ "Vindex": "user_index" } } +{ +} # Column as boolean-ish "select * from user where (id = 1) AND name limit 5" @@ -1001,6 +1025,8 @@ "Vindex": "user_index" } } +{ +} # PK as fake boolean, and column as boolean-ish "select * from user where (id = 5) AND name = true limit 5" @@ -1023,6 +1049,8 @@ "Vindex": "user_index" } } +{ +} # top level subquery in select "select a, (select col from user) from unsharded" @@ -1164,6 +1192,8 @@ "Query": "select * from information_schema.a union select * from information_schema.b" } } +{ +} # union with the same target shard "select * from music where user_id = 1 union select * from user where id = 1" @@ -1186,6 +1216,8 @@ "Vindex": "user_index" } } +{ +} # union with the same target shard last_insert_id "select *, last_insert_id() from music where user_id = 1 union select * from user where id = 1" @@ -1208,6 +1240,8 @@ "Vindex": "user_index" } } +{ +} "select * from (select col1, col2 from unsharded where id = 1 union select col1, col2 from unsharded where id = 3) a" { @@ -1242,6 +1276,8 @@ "Table": "unsharded" } } +{ +} "(select id from unsharded) union (select id from unsharded_auto) order by id limit 5" { @@ -1259,6 +1295,8 @@ "Table": "unsharded" } } +{ +} "select id from unsharded union select id from unsharded_auto union select id from unsharded_auto where id in (132)" { @@ -1276,6 +1314,8 @@ "Table": "unsharded" } } +{ +} "(select id from unsharded union select id from unsharded_auto) union (select id from unsharded_auto union select name from unsharded)" { @@ -1293,6 +1333,8 @@ "Table": "unsharded" } } +{ +} "(select id from unsharded order by id asc limit 1) union (select id from unsharded order by id desc limit 1) order by id asc limit 1" { @@ -1310,6 +1352,8 @@ "Table": "unsharded" } } +{ +} # routing rules: ensure directives are not lost "select /*vt+ QUERY_TIMEOUT_MS=1000 */ * from route2" @@ -1367,6 +1411,8 @@ "Table": "dual" } } +{ +} # testing SingleRow Projection with arithmetics "select 42+2" @@ -1615,6 +1661,8 @@ # Union after into outfile is incorrect "select id from user into outfile 'out_file_name' union all select id from music" "Incorrect usage/placement of 'INTO' (errno 1234) (sqlstate 42000)" +{ +} # Into outfile s3 in sub-query is incorrect "select id from (select id from user into outfile s3 'inner_outfile') as t2" diff --git a/go/vt/vtgate/planbuilder/testdata/set_cases.txt b/go/vt/vtgate/planbuilder/testdata/set_cases.txt index 0d2289bf207..8fbb811636d 100644 --- a/go/vt/vtgate/planbuilder/testdata/set_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/set_cases.txt @@ -19,6 +19,8 @@ ] } } +{ +} # set multi user defined variable "set @foo = 42, @bar = @foo" @@ -46,6 +48,8 @@ ] } } +{ +} # set multi user defined variable with complex expression "set @foo = 42, @bar = @foo + 1" @@ -73,6 +77,8 @@ ] } } +{ +} # set UDV to expression that can't be evaluated at vtgate "set @foo = CONCAT('Any','Expression','Is','Valid')" @@ -103,6 +109,8 @@ ] } } +{ +} # single sysvar cases "SET sql_mode = 'STRICT_ALL_TABLES,NO_AUTO_VALUE_ON_ZERO'" @@ -129,6 +137,8 @@ ] } } +{ +} # multiple sysvar cases "SET @@SESSION.sql_mode = CONCAT(CONCAT(@@sql_mode, ',STRICT_ALL_TABLES'), ',NO_AUTO_VALUE_ON_ZERO'), @@SESSION.sql_safe_updates = 0" @@ -164,6 +174,8 @@ ] } } +{ +} # autocommit case "SET autocommit = 1, autocommit = on, autocommit = 'on', autocommit = @myudv, autocommit = `on`, autocommit = `off`" @@ -211,6 +223,8 @@ ] } } +{ +} # set ignore plan "set @@default_storage_engine = 'DONOTCHANGEME'" @@ -233,6 +247,8 @@ ] } } +{ +} # set check and ignore plan "set @@sql_mode = concat(@@sql_mode, ',NO_AUTO_CREATE_USER')" @@ -259,6 +275,8 @@ ] } } +{ +} # set system settings "set @@sql_safe_updates = 1" @@ -285,6 +303,8 @@ ] } } +{ +} # set plan building with ON/OFF enum "set @@innodb_strict_mode = OFF" @@ -307,6 +327,8 @@ ] } } +{ +} # set plan building with string literal "set @@innodb_strict_mode = 'OFF'" @@ -329,6 +351,8 @@ ] } } +{ +} # set plan building with string literal "set @@innodb_tmpdir = 'OFF'" @@ -351,10 +375,14 @@ ] } } +{ +} # set system settings "set @@ndbinfo_max_bytes = 192" "ndbinfo_max_bytes: system setting is not supported" +{ +} # set autocommit "set autocommit = 1" @@ -377,6 +405,8 @@ ] } } +{ +} # set autocommit false "set autocommit = 0" @@ -399,6 +429,8 @@ ] } } +{ +} # set autocommit with backticks "set @@session.`autocommit` = 0" @@ -421,6 +453,8 @@ ] } } +{ +} # more vitess aware settings "set client_found_rows = off, skip_query_plan_cache = ON, sql_select_limit=20" @@ -453,6 +487,8 @@ ] } } +{ +} # set autocommit to default "set @@autocommit = default" @@ -475,6 +511,8 @@ ] } } +{ +} # set global autocommit to default "set global autocommit = off" @@ -502,3 +540,5 @@ ] } } +{ +} diff --git a/go/vt/vtgate/planbuilder/testdata/transaction_cases.txt b/go/vt/vtgate/planbuilder/testdata/transaction_cases.txt index b29317c7a26..c3e6278ef23 100644 --- a/go/vt/vtgate/planbuilder/testdata/transaction_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/transaction_cases.txt @@ -4,6 +4,8 @@ "QueryType": "BEGIN", "Original": "begin" } +{ +} # Start Transaction "start transaction" @@ -11,6 +13,8 @@ "QueryType": "BEGIN", "Original": "start transaction" } +{ +} # Commit "commit" @@ -18,6 +22,8 @@ "QueryType": "COMMIT", "Original": "commit" } +{ +} # Rollback "rollback" @@ -25,6 +31,8 @@ "QueryType": "ROLLBACK", "Original": "rollback" } +{ +} # Savepoint "savepoint a" @@ -32,6 +40,8 @@ "QueryType": "SAVEPOINT", "Original": "savepoint a" } +{ +} # Savepoint rollback "rollback work to savepoint a" @@ -39,6 +49,8 @@ "QueryType": "SAVEPOINT_ROLLBACK", "Original": "rollback work to savepoint a" } +{ +} # Savepoint release "release savepoint a" @@ -46,3 +58,5 @@ "QueryType": "RELEASE", "Original": "release savepoint a" } +{ +} diff --git a/go/vt/vtgate/planbuilder/testdata/union_cases.txt b/go/vt/vtgate/planbuilder/testdata/union_cases.txt index 18ae0f63053..4a48e616cea 100644 --- a/go/vt/vtgate/planbuilder/testdata/union_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/union_cases.txt @@ -15,6 +15,8 @@ "Table": "user" } } +{ +} # union distinct between two scatter selects "select id from user union select id from music" @@ -54,6 +56,8 @@ ] } } +{ +} # union all between two SelectEqualUnique "select id from user where id = 1 union all select id from user where id = 5" @@ -96,6 +100,8 @@ ] } } +{ +} #almost dereks query - two queries with order by and limit being scattered to two different sets of tablets "(SELECT id FROM user ORDER BY id DESC LIMIT 1) UNION ALL (SELECT id FROM music ORDER BY id DESC LIMIT 1)" @@ -144,6 +150,8 @@ ] } } +{ +} # Union all "select col1, col2 from user union all select col1, col2 from user_extra" @@ -162,6 +170,8 @@ "Table": "user" } } +{ +} # union operations in subqueries (FROM) "select * from (select * from user union all select * from user_extra) as t" @@ -228,6 +238,8 @@ ] } } +{ +} # union all on scatter and single route "select id from user where id = 1 union select id from user where id = 1 union all select id from user" @@ -266,6 +278,8 @@ ] } } +{ +} # union of information_schema with normal table "select * from information_schema.a union select * from unsharded" @@ -304,6 +318,8 @@ ] } } +{ +} # union of information_schema with normal table "select * from unsharded union select * from information_schema.a" @@ -342,6 +358,8 @@ ] } } +{ +} # multi-shard union "(select id from user union select id from music) union select 1 from dual" @@ -402,6 +420,8 @@ ] } } +{ +} # multi-shard union "select 1 from music union (select id from user union all select name from unsharded)" @@ -457,6 +477,8 @@ ] } } +{ +} # multi-shard union "select 1 from music union (select id from user union select name from unsharded)" @@ -517,6 +539,8 @@ ] } } +{ +} # union with the same target shard because of vindex "select * from music where id = 1 union select * from user where id = 1" @@ -564,6 +588,8 @@ ] } } +{ +} # union with different target shards "select 1 from music where id = 1 union select 1 from music where id = 2" @@ -611,6 +637,8 @@ ] } } +{ +} # multiple select statement have inner order by with union - TODO (systay) no need to send down ORDER BY if we are going to loose it with UNION DISTINCT "(select 1 from user order by 1 desc) union (select 1 from user order by 1 asc)" @@ -652,6 +680,8 @@ ] } } +{ +} # multiple unions "select 1 union select null union select 1.0 union select '1' union select 2 union select 2.0 from user" @@ -691,6 +721,8 @@ ] } } +{ +} # union distinct between a scatter query and a join (other side) "(select user.id, user.name from user join user_extra where user_extra.extra = 'asdf') union select 'b','c' from user" @@ -749,6 +781,8 @@ ] } } +{ +} # union distinct between a scatter query and a join (other side) "select 'b','c' from user union (select user.id, user.name from user join user_extra where user_extra.extra = 'asdf')" @@ -807,11 +841,17 @@ ] } } +{ +} # ambiguous LIMIT "select id from user limit 1 union all select id from music limit 1" "Incorrect usage of UNION and LIMIT - add parens to disambiguate your query (errno 1221) (sqlstate 21000)" +{ +} # different number of columns "select id, 42 from user where id = 1 union all select id from user where id = 5" "The used SELECT statements have a different number of columns (errno 1222) (sqlstate 21000) during query: select id, 42 from user where id = 1 union all select id from user where id = 5" +{ +} diff --git a/go/vt/vtgate/planbuilder/testdata/unsupported_cases.txt b/go/vt/vtgate/planbuilder/testdata/unsupported_cases.txt index 273ae7362c9..c5971f890e5 100644 --- a/go/vt/vtgate/planbuilder/testdata/unsupported_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/unsupported_cases.txt @@ -1,6 +1,8 @@ # SHOW "show create database" "plan building not supported" +{ +} # union operations in subqueries (expressions) "select * from user where id in (select * from user union select * from user_extra)" @@ -13,14 +15,20 @@ # Unsupported INSERT statement with a target destination "insert into `user[-]`.user_metadata (a, b) values (1,2)" "unsupported: INSERT with a target destination" +{ +} # Unsupported delete statement with a replica target destination "DELETE FROM `user[-]@replica`.user_metadata limit 1" "unsupported: delete statement with a replica target" +{ +} # Unsupported update statement with a replica target destination "update `user[-]@replica`.user_metadata set id=2" "unsupported: update statement with a replica target" +{ +} # scatter order by with * expression "select * from user order by id" @@ -169,146 +177,218 @@ # subqueries in update "update user set col = (select id from unsharded)" "unsupported: subqueries in sharded DML" +{ +} # sharded subqueries in unsharded update "update unsharded set col = (select id from user)" "unsupported: sharded subqueries in DML" +{ +} # sharded join unsharded subqueries in unsharded update "update unsharded set col = (select id from unsharded join user on unsharded.id = user.id)" "unsupported: sharded subqueries in DML" +{ +} # subqueries in delete "delete from user where col = (select id from unsharded)" "unsupported: subqueries in sharded DML" +{ +} # sharded subqueries in unsharded delete "delete from unsharded where col = (select id from user)" "unsupported: sharded subqueries in DML" +{ +} # sharded delete with limit clasue "delete from user_extra limit 10" "unsupported: multi shard delete with limit" +{ +} # sharded subquery in unsharded subquery in unsharded delete "delete from unsharded where col = (select id from unsharded where id = (select id from user))" "unsupported: sharded subqueries in DML" +{ +} # sharded join unsharded subqueries in unsharded delete "delete from unsharded where col = (select id from unsharded join user on unsharded.id = user.id)" "unsupported: sharded subqueries in DML" +{ +} # scatter update with limit clause "update user_extra set val = 1 where (name = 'foo' or id = 1) limit 1" "unsupported: multi shard update with limit" +{ +} # multi delete multi table "delete user from user join user_extra on user.id = user_extra.id where user.name = 'foo'" "unsupported: multi-shard or vindex write statement" +{ +} # update changes primary vindex column "update user set id = 1 where id = 1" "unsupported: You can't update primary vindex columns. Invalid update on vindex: user_index" +{ +} # update changes non owned vindex column "update music_extra set music_id = 1 where user_id = 1" "unsupported: You can only update owned vindexes. Invalid update on vindex: music_user_map" +{ +} # update changes non lookup vindex column "update user_metadata set md5 = 1 where user_id = 1" "unsupported: You can only update lookup vindexes. Invalid update on vindex: user_md5_index" +{ +} # update with complex set clause "update music set id = id + 1 where id = 1" "unsupported: Only values are supported. Invalid update on column: id" +{ +} # update by primary keyspace id, changing one vindex column, limit without order clause "update user_metadata set email = 'juan@vitess.io' where user_id = 1 limit 10" "unsupported: Need to provide order by clause when using limit. Invalid update on vindex: email_user_map" +{ +} # cross-shard update tables "update (select id from user) as u set id = 4" "unsupported: subqueries in sharded DML" +{ +} # join in update tables "update user join user_extra on user.id = user_extra.id set user.name = 'foo'" "unsupported: multi-shard or vindex write statement" +{ +} # multiple tables in update "update user as u, user_extra as ue set u.name = 'foo' where u.id = ue.id" "unsupported: multi-shard or vindex write statement" +{ +} # unsharded insert with cross-shard join" "insert into unsharded select u.col from user u join user u1" "unsupported: sharded subquery in insert values" +{ +} # unsharded insert with mismatched keyspaces" "insert into unsharded select col from user where id=1" "unsupported: sharded subquery in insert values" +{ +} # unsharded insert, unqualified names and auto-inc combined "insert into unsharded_auto select col from unsharded" "unsupported: auto-inc and select in insert" +{ +} # unsharded insert, with sharded subquery in insert value "insert into unsharded values((select 1 from user), 1)" "unsupported: sharded subquery in insert values" +{ +} # unsharded insert, no col list with auto-inc "insert into unsharded_auto values(1,1)" "column list required for tables with auto-inc columns" +{ +} # unsharded insert, col list does not match values "insert into unsharded_auto(id, val) values(1)" "column list doesn't match values" +{ +} # sharded upsert can't change vindex "insert into user(id) values(1) on duplicate key update id = 3" "unsupported: DML cannot change vindex column" +{ +} # sharded upsert can't change vindex using values function "insert into music(user_id, id) values(1, 2) on duplicate key update user_id = values(id)" "unsupported: DML cannot change vindex column" +{ +} # sharded insert from select "insert into user(id) select 1 from dual" "unsupported: insert into select" +{ +} # sharded replace no vindex "replace into user(val) values(1, 'foo')" "unsupported: REPLACE INTO with sharded schema" +{ +} # sharded replace with vindex "replace into user(id, name) values(1, 'foo')" "unsupported: REPLACE INTO with sharded schema" +{ +} # replace no column list "replace into user values(1, 2, 3)" "unsupported: REPLACE INTO with sharded schema" +{ +} # replace with mimatched column list "replace into user(id) values (1, 2)" "unsupported: REPLACE INTO with sharded schema" +{ +} # replace with one vindex "replace into user(id) values (1)" "unsupported: REPLACE INTO with sharded schema" +{ +} # replace with non vindex on vindex-enabled table "replace into user(nonid) values (2)" "unsupported: REPLACE INTO with sharded schema" +{ +} # replace with all vindexes supplied "replace into user(nonid, name, id) values (2, 'foo', 1)" "unsupported: REPLACE INTO with sharded schema" +{ +} # replace for non-vindex autoinc "replace into user_extra(nonid) values (2)" "unsupported: REPLACE INTO with sharded schema" +{ +} # replace with multiple rows "replace into user(id) values (1), (2)" "unsupported: REPLACE INTO with sharded schema" +{ +} "select keyspace_id from user_index where id = 1 and id = 2" "unsupported: where clause for vindex function must be of the form id = (multiple filters)" @@ -337,18 +417,26 @@ # delete with unknown reference "delete music from user where id = 1" "Unknown table 'music' in MULTI DELETE" +{ +} # delete with multi-table targets "delete music,user from music inner join user where music.id = user.id" "unsupported: multi-shard or vindex write statement" +{ +} # order by inside and outside parenthesis select "(select 1 from user order by 1 desc) order by 1 asc limit 2" "can't do ORDER BY on top of ORDER BY" +{ +} # ambiguous ORDER BY "select id from user order by id union all select id from music order by id desc" "Incorrect usage of UNION and ORDER BY - add parens to disambiguate your query (errno 1221) (sqlstate 21000)" +{ +} # select get_lock with non-dual table "select get_lock('xyz', 10) from user" @@ -361,18 +449,26 @@ # insert using select get_lock from table "insert into user(pattern) SELECT GET_LOCK('xyz1', 10)" "unsupported: insert into select" +{ +} # union with SQL_CALC_FOUND_ROWS "(select sql_calc_found_rows id from user where id = 1 limit 1) union select id from user where id = 1" "SQL_CALC_FOUND_ROWS not supported with union" +{ +} # set with DEFAULT - vitess aware "set workload = default" "DEFAULT not supported for @@workload" +{ +} # set with DEFAULT - reserved connection "set sql_mode = default" "DEFAULT not supported for @@sql_mode" +{ +} # Multi shard query using into outfile s3 "select * from user into outfile s3 'out_file_name'" @@ -409,3 +505,5 @@ # create view with incompatible keyspaces "create view main.view_a as select * from user.user_extra" "Select query does not belong to the same keyspace as the view statement" +{ +} diff --git a/go/vt/vtgate/planbuilder/testdata/use_cases.txt b/go/vt/vtgate/planbuilder/testdata/use_cases.txt index 0e8d1f6449d..47a1f090751 100644 --- a/go/vt/vtgate/planbuilder/testdata/use_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/use_cases.txt @@ -8,6 +8,8 @@ "target": "ks" } } +{ +} # use db tablet "use ks@replica" @@ -19,6 +21,8 @@ "target": "ks@replica" } } +{ +} # use db target with : "use `ks:-80@replica`" @@ -30,6 +34,8 @@ "target": "ks:-80@replica" } } +{ +} # use db target with / "use `ks/80-@replica`" @@ -41,6 +47,8 @@ "target": "ks/80-@replica" } } +{ +} # reset db "use" @@ -51,3 +59,5 @@ "OperatorType": "UpdateTarget" } } +{ +} diff --git a/go/vt/vtgate/planbuilder/testdata/vindex_func_cases.txt b/go/vt/vtgate/planbuilder/testdata/vindex_func_cases.txt index c492ad196a7..8865eff33b0 100644 --- a/go/vt/vtgate/planbuilder/testdata/vindex_func_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/vindex_func_cases.txt @@ -81,6 +81,8 @@ # ambiguous vindex reference "select id, keyspace_id, id from hash_dup where id = :id" "ambiguous vindex reference: hash_dup" +{ +} # disambiguated vindex reference "select id, keyspace_id, id from second_user.hash_dup where id = :id" diff --git a/go/vt/vtgate/planbuilder/testdata/wireup_cases.txt b/go/vt/vtgate/planbuilder/testdata/wireup_cases.txt index 187b59e158d..b0502d888fe 100644 --- a/go/vt/vtgate/planbuilder/testdata/wireup_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/wireup_cases.txt @@ -633,7 +633,11 @@ # Invalid value in IN clause from LHS of join "select u1.id from user u1 join user u2 where u1.id = 18446744073709551616" "strconv.ParseUint: parsing "18446744073709551616": value out of range" +{ +} # Invalid value in IN clause from RHS of join "select u1.id from user u1 join user u2 where u2.id = 18446744073709551616" "strconv.ParseUint: parsing "18446744073709551616": value out of range" +{ +} From 8a148e7c74b374f2e888cd7462a16e1f8f88d7d1 Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Tue, 22 Dec 2020 10:14:10 +0100 Subject: [PATCH 07/40] support more cases with the new planner Signed-off-by: Andres Taylor --- go/vt/vtgate/planbuilder/plan_test.go | 6 +++--- go/vt/vtgate/planbuilder/querygraph.go | 2 +- go/vt/vtgate/planbuilder/route_planning.go | 9 +++++++-- .../vtgate/planbuilder/testdata/filter_cases.txt | 2 ++ go/vt/vtgate/planbuilder/testdata/from_cases.txt | 16 ++++++++++++++++ .../vtgate/planbuilder/testdata/select_cases.txt | 4 ++++ 6 files changed, 33 insertions(+), 6 deletions(-) diff --git a/go/vt/vtgate/planbuilder/plan_test.go b/go/vt/vtgate/planbuilder/plan_test.go index 4cbad7834d4..5e84d65c76a 100644 --- a/go/vt/vtgate/planbuilder/plan_test.go +++ b/go/vt/vtgate/planbuilder/plan_test.go @@ -24,6 +24,7 @@ import ( "io" "io/ioutil" "os" + "runtime/debug" "strings" "testing" @@ -451,7 +452,7 @@ func testFile(t *testing.T, filename, tempDir string, vschema *vschemaWrapper) { func getPlanOutput(tcase testCase, vschema *vschemaWrapper) (out string, err error) { defer func() { if r := recover(); r != nil { - out = fmt.Sprintf("%v", r) + out = fmt.Sprintf("panicked: %v\n%s", r, string(debug.Stack())) } }() plan, err := TestBuilder(tcase.input, vschema) @@ -553,8 +554,7 @@ func iterateExecFile(name string) (testCaseIterator chan testCase) { break } } - if string(output2Planner) == `{ -}` { + if string(output2Planner) == "{\n}" { output2Planner = output } } diff --git a/go/vt/vtgate/planbuilder/querygraph.go b/go/vt/vtgate/planbuilder/querygraph.go index f460b4addb6..f3aae564c41 100644 --- a/go/vt/vtgate/planbuilder/querygraph.go +++ b/go/vt/vtgate/planbuilder/querygraph.go @@ -206,5 +206,5 @@ func (qg *queryGraph) tryMerge(a, b joinTree, joinPredicates []sqlparser.Expr) j } r.extraPredicates = append(r.extraPredicates, joinPredicates...) - return nil + return r } diff --git a/go/vt/vtgate/planbuilder/route_planning.go b/go/vt/vtgate/planbuilder/route_planning.go index c84d2405f1c..4dd7b7389dd 100644 --- a/go/vt/vtgate/planbuilder/route_planning.go +++ b/go/vt/vtgate/planbuilder/route_planning.go @@ -255,11 +255,11 @@ func transformToLogicalPlan(tree joinTree) (logicalPlan, error) { switch n := tree.(type) { case *routePlan: var tablesForSelect sqlparser.TableExprs - var tableNames []string + tableNameMap := map[string]interface{}{} for _, t := range n.tables { tablesForSelect = append(tablesForSelect, t.alias) - tableNames = append(tableNames, sqlparser.String(t.alias)) + tableNameMap[sqlparser.String(t.alias.Expr)] = nil } predicates := n.Predicates() var where *sqlparser.Where @@ -279,6 +279,11 @@ func transformToLogicalPlan(tree joinTree) (logicalPlan, error) { singleColumn = n.vindex.(vindexes.SingleColumn) } + var tableNames []string + for name := range tableNameMap { + tableNames = append(tableNames, name) + } + return &route{ eroute: &engine.Route{ Opcode: n.routeOpCode, diff --git a/go/vt/vtgate/planbuilder/testdata/filter_cases.txt b/go/vt/vtgate/planbuilder/testdata/filter_cases.txt index 353ad1a84fb..7c3a9bbd777 100644 --- a/go/vt/vtgate/planbuilder/testdata/filter_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/filter_cases.txt @@ -1280,6 +1280,8 @@ "Table": "user" } } +{ +} # cross-shard subquery in IN clause. # Note the improved Underlying plan as SelectIN. diff --git a/go/vt/vtgate/planbuilder/testdata/from_cases.txt b/go/vt/vtgate/planbuilder/testdata/from_cases.txt index ea9b5555f05..d99c7912a34 100644 --- a/go/vt/vtgate/planbuilder/testdata/from_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/from_cases.txt @@ -112,6 +112,8 @@ "Table": "unsharded" } } +{ +} # Multi-table, multi-chunk "select music.col from user join music" @@ -306,6 +308,8 @@ "Table": "unsharded" } } +{ +} # ',' join information_schema "select a.id,b.id from information_schema.a as a, information_schema.b as b" @@ -341,6 +345,8 @@ "Table": "unsharded" } } +{ +} # Left join, single chunk "select m1.col from unsharded as m1 left join unsharded as m2 on m1.a=m2.b" @@ -359,6 +365,8 @@ "Table": "unsharded" } } +{ +} # Left join, multi-chunk "select u.col from user u left join unsharded m on u.a = m.b" @@ -561,6 +569,8 @@ "Table": "unsharded" } } +{ +} # Three-way join "select user.col from user join unsharded as m1 join unsharded as m2" @@ -727,6 +737,8 @@ "Table": "user" } } +{ +} # mergeable sharded join on unique vindex "select user.col from user join user_extra on user.id = user_extra.user_id" @@ -1054,6 +1066,8 @@ "Table": "ref" } } +{ +} # reference table can merge with other opcodes left to right. "select ref.col from ref join user" @@ -2033,6 +2047,8 @@ "Table": "unsharded" } } +{ +} # last_insert_id for dual "select last_insert_id()" diff --git a/go/vt/vtgate/planbuilder/testdata/select_cases.txt b/go/vt/vtgate/planbuilder/testdata/select_cases.txt index c72fa09d290..92c08607a9f 100644 --- a/go/vt/vtgate/planbuilder/testdata/select_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/select_cases.txt @@ -957,6 +957,8 @@ "Vindex": "user_index" } } +{ +} # Column Aliasing with Column "select user0_.col as col0_ from user user0_ where id = 1 order by col0_ desc limit 3" @@ -979,6 +981,8 @@ "Vindex": "user_index" } } +{ +} # Booleans and parenthesis "select * from user where (id = 1) AND name = true limit 5" From 09a6dac0fd928bcb417481d7f424dacdaebdad49 Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Tue, 22 Dec 2020 15:43:14 +0100 Subject: [PATCH 08/40] new planner can solve the simplest join plans Signed-off-by: Andres Taylor --- go/vt/vtgate/planbuilder/concatenate.go | 13 +++ go/vt/vtgate/planbuilder/join.go | 16 +++ go/vt/vtgate/planbuilder/join2.go | 102 ++++++++++++++++++ go/vt/vtgate/planbuilder/logical_plan.go | 14 +++ go/vt/vtgate/planbuilder/memory_sort.go | 6 ++ go/vt/vtgate/planbuilder/merge_sort.go | 5 + go/vt/vtgate/planbuilder/ordered_aggregate.go | 6 ++ go/vt/vtgate/planbuilder/pullout_subquery.go | 14 +++ go/vt/vtgate/planbuilder/querygraph.go | 9 +- go/vt/vtgate/planbuilder/route.go | 38 +++++++ go/vt/vtgate/planbuilder/route_planning.go | 71 +++++++++++- go/vt/vtgate/planbuilder/select.go | 94 ++++++++++------ .../vtgate/planbuilder/sql_calc_found_rows.go | 15 +++ .../planbuilder/testdata/from_cases.txt | 4 + .../planbuilder/testdata/select_cases.txt | 10 ++ go/vt/vtgate/planbuilder/vindex_func.go | 14 ++- go/vt/vtgate/semantics/semantic_state.go | 5 + 17 files changed, 394 insertions(+), 42 deletions(-) create mode 100644 go/vt/vtgate/planbuilder/join2.go diff --git a/go/vt/vtgate/planbuilder/concatenate.go b/go/vt/vtgate/planbuilder/concatenate.go index 4ff12ad47d5..1e0db368fa9 100644 --- a/go/vt/vtgate/planbuilder/concatenate.go +++ b/go/vt/vtgate/planbuilder/concatenate.go @@ -21,6 +21,7 @@ import ( "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vterrors" "vitess.io/vitess/go/vt/vtgate/engine" + "vitess.io/vitess/go/vt/vtgate/semantics" ) type concatenate struct { @@ -53,6 +54,14 @@ func (c *concatenate) Wireup(plan logicalPlan, jt *jointab) error { return c.rhs.Wireup(plan, jt) } +func (c *concatenate) Wireup2(semTable *semantics.SemTable) error { + err := c.lhs.Wireup2(semTable) + if err != nil { + return err + } + return c.rhs.Wireup2(semTable) +} + func (c *concatenate) SupplyVar(from, to int, col *sqlparser.ColName, varname string) { panic("implement me") } @@ -84,6 +93,10 @@ func (c *concatenate) Rewrite(inputs ...logicalPlan) error { return nil } +func (c *concatenate) Solves() semantics.TableSet { + return c.lhs.Solves().Merge(c.rhs.Solves()) +} + // Inputs implements the logicalPlan interface func (c *concatenate) Inputs() []logicalPlan { return []logicalPlan{c.lhs, c.rhs} diff --git a/go/vt/vtgate/planbuilder/join.go b/go/vt/vtgate/planbuilder/join.go index dc4ffe26732..55fa449bca7 100644 --- a/go/vt/vtgate/planbuilder/join.go +++ b/go/vt/vtgate/planbuilder/join.go @@ -19,6 +19,8 @@ package planbuilder import ( "errors" + "vitess.io/vitess/go/vt/vtgate/semantics" + vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" "vitess.io/vitess/go/vt/vterrors" @@ -151,6 +153,15 @@ func (jb *join) Wireup(plan logicalPlan, jt *jointab) error { return jb.Left.Wireup(plan, jt) } +// Wireup2 implements the logicalPlan interface +func (jb *join) Wireup2(semTable *semantics.SemTable) error { + err := jb.Right.Wireup2(semTable) + if err != nil { + return err + } + return jb.Left.Wireup2(semTable) +} + // SupplyVar implements the logicalPlan interface func (jb *join) SupplyVar(from, to int, col *sqlparser.ColName, varname string) { if !jb.isOnLeft(from) { @@ -235,6 +246,11 @@ func (jb *join) Rewrite(inputs ...logicalPlan) error { return nil } +// Solves implements the logicalPlan interface +func (jb *join) Solves() semantics.TableSet { + return jb.Left.Solves().Merge(jb.Right.Solves()) +} + // Inputs implements the logicalPlan interface func (jb *join) Inputs() []logicalPlan { return []logicalPlan{jb.Left, jb.Right} diff --git a/go/vt/vtgate/planbuilder/join2.go b/go/vt/vtgate/planbuilder/join2.go new file mode 100644 index 00000000000..7043257b6b2 --- /dev/null +++ b/go/vt/vtgate/planbuilder/join2.go @@ -0,0 +1,102 @@ +/* +Copyright 2019 The Vitess 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 planbuilder + +import ( + "vitess.io/vitess/go/vt/sqlparser" + "vitess.io/vitess/go/vt/vtgate/engine" + "vitess.io/vitess/go/vt/vtgate/semantics" +) + +var _ logicalPlan = (*join2)(nil) + +// join is used to build a Join primitive. +// It's used to build a normal join or a left join +// operation. +type join2 struct { + // Left and Right are the nodes for the join. + Left, Right logicalPlan + Cols []int +} + +// Order implements the logicalPlan interface +func (j *join2) Order() int { + panic("implement me") +} + +// ResultColumns implements the logicalPlan interface +func (j *join2) ResultColumns() []*resultColumn { + panic("implement me") +} + +// Reorder implements the logicalPlan interface +func (j *join2) Reorder(i int) { + panic("implement me") +} + +// Wireup implements the logicalPlan interface +func (j *join2) Wireup(lp logicalPlan, jt *jointab) error { + panic("implement me") +} + +// Wireup2 implements the logicalPlan interface +func (j *join2) Wireup2(semTable *semantics.SemTable) error { + err := j.Left.Wireup2(semTable) + if err != nil { + return err + } + return j.Right.Wireup2(semTable) +} + +// SupplyVar implements the logicalPlan interface +func (j *join2) SupplyVar(from, to int, col *sqlparser.ColName, varname string) { + panic("implement me") +} + +// SupplyCol implements the logicalPlan interface +func (j *join2) SupplyCol(col *sqlparser.ColName) (rc *resultColumn, colNumber int) { + panic("implement me") +} + +// SupplyWeightString implements the logicalPlan interface +func (j *join2) SupplyWeightString(colNumber int) (weightcolNumber int, err error) { + panic("implement me") +} + +// Primitive implements the logicalPlan interface +func (j *join2) Primitive() engine.Primitive { + return &engine.Join{ + Left: j.Left.Primitive(), + Right: j.Right.Primitive(), + Cols: j.Cols, + } +} + +// Inputs implements the logicalPlan interface +func (j *join2) Inputs() []logicalPlan { + panic("implement me") +} + +// Rewrite implements the logicalPlan interface +func (j *join2) Rewrite(inputs ...logicalPlan) error { + panic("implement me") +} + +// Solves implements the logicalPlan interface +func (j *join2) Solves() semantics.TableSet { + return j.Left.Solves().Merge(j.Right.Solves()) +} diff --git a/go/vt/vtgate/planbuilder/logical_plan.go b/go/vt/vtgate/planbuilder/logical_plan.go index 40608565c8b..bb0ba8004e6 100644 --- a/go/vt/vtgate/planbuilder/logical_plan.go +++ b/go/vt/vtgate/planbuilder/logical_plan.go @@ -21,6 +21,7 @@ import ( "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vterrors" "vitess.io/vitess/go/vt/vtgate/engine" + "vitess.io/vitess/go/vt/vtgate/semantics" ) // logicalPlan defines the interface that a primitive must @@ -48,6 +49,9 @@ type logicalPlan interface { // the lhs nodes. Wireup(lp logicalPlan, jt *jointab) error + // Wireup2 does the wire up work for the new planner + Wireup2(semTable *semantics.SemTable) error + // SupplyVar finds the common root between from and to. If it's // the common root, it supplies the requested var to the rhs tree. // If the primitive already has the column in its list, it should @@ -74,6 +78,7 @@ type logicalPlan interface { Inputs() []logicalPlan Rewrite(inputs ...logicalPlan) error + Solves() semantics.TableSet } //------------------------------------------------------------------------- @@ -147,6 +152,10 @@ func (bc *logicalPlanCommon) Wireup(plan logicalPlan, jt *jointab) error { return bc.input.Wireup(plan, jt) } +func (bc *logicalPlanCommon) Wireup2(semTable *semantics.SemTable) error { + return bc.input.Wireup2(semTable) +} + func (bc *logicalPlanCommon) SupplyVar(from, to int, col *sqlparser.ColName, varname string) { bc.input.SupplyVar(from, to, col, varname) } @@ -173,6 +182,11 @@ func (bc *logicalPlanCommon) Inputs() []logicalPlan { return []logicalPlan{bc.input} } +// Solves implements the logicalPlan interface +func (bc *logicalPlanCommon) Solves() semantics.TableSet { + return bc.input.Solves() +} + //------------------------------------------------------------------------- // resultsBuilder is a superset of logicalPlanCommon. It also handles diff --git a/go/vt/vtgate/planbuilder/memory_sort.go b/go/vt/vtgate/planbuilder/memory_sort.go index ade1e68817f..ae972fe6047 100644 --- a/go/vt/vtgate/planbuilder/memory_sort.go +++ b/go/vt/vtgate/planbuilder/memory_sort.go @@ -20,6 +20,8 @@ import ( "errors" "fmt" + "vitess.io/vitess/go/vt/vtgate/semantics" + "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vtgate/engine" @@ -125,3 +127,7 @@ func (ms *memorySort) Wireup(plan logicalPlan, jt *jointab) error { } return ms.input.Wireup(plan, jt) } + +func (ms *memorySort) Wireup2(semTable *semantics.SemTable) error { + return ms.input.Wireup2(semTable) +} diff --git a/go/vt/vtgate/planbuilder/merge_sort.go b/go/vt/vtgate/planbuilder/merge_sort.go index f0675489f20..91df0296cc9 100644 --- a/go/vt/vtgate/planbuilder/merge_sort.go +++ b/go/vt/vtgate/planbuilder/merge_sort.go @@ -19,6 +19,7 @@ package planbuilder import ( "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/vtgate/engine" + "vitess.io/vitess/go/vt/vtgate/semantics" ) var _ logicalPlan = (*mergeSort)(nil) @@ -82,3 +83,7 @@ func (ms *mergeSort) Wireup(plan logicalPlan, jt *jointab) error { rb.eroute.TruncateColumnCount = ms.truncateColumnCount return ms.input.Wireup(plan, jt) } + +func (ms *mergeSort) Wireup2(semTable *semantics.SemTable) error { + return ms.input.Wireup2(semTable) +} diff --git a/go/vt/vtgate/planbuilder/ordered_aggregate.go b/go/vt/vtgate/planbuilder/ordered_aggregate.go index 60fff9ffd03..566eeb9723a 100644 --- a/go/vt/vtgate/planbuilder/ordered_aggregate.go +++ b/go/vt/vtgate/planbuilder/ordered_aggregate.go @@ -21,6 +21,8 @@ import ( "fmt" "strconv" + "vitess.io/vitess/go/vt/vtgate/semantics" + "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vtgate/engine" @@ -347,3 +349,7 @@ func (oa *orderedAggregate) Wireup(plan logicalPlan, jt *jointab) error { } return oa.input.Wireup(plan, jt) } + +func (oa *orderedAggregate) Wireup2(semTable *semantics.SemTable) error { + return oa.input.Wireup2(semTable) +} diff --git a/go/vt/vtgate/planbuilder/pullout_subquery.go b/go/vt/vtgate/planbuilder/pullout_subquery.go index 7f3372a103d..1cf6267c777 100644 --- a/go/vt/vtgate/planbuilder/pullout_subquery.go +++ b/go/vt/vtgate/planbuilder/pullout_subquery.go @@ -21,6 +21,7 @@ import ( "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vterrors" "vitess.io/vitess/go/vt/vtgate/engine" + "vitess.io/vitess/go/vt/vtgate/semantics" ) var _ logicalPlan = (*pulloutSubquery)(nil) @@ -86,6 +87,14 @@ func (ps *pulloutSubquery) Wireup(plan logicalPlan, jt *jointab) error { return ps.subquery.Wireup(plan, jt) } +// Wireup2 implements the logicalPlan interface +func (ps *pulloutSubquery) Wireup2(semTable *semantics.SemTable) error { + if err := ps.underlying.Wireup2(semTable); err != nil { + return err + } + return ps.subquery.Wireup2(semTable) +} + // SupplyVar implements the logicalPlan interface func (ps *pulloutSubquery) SupplyVar(from, to int, col *sqlparser.ColName, varname string) { if from <= ps.subquery.Order() { @@ -115,6 +124,11 @@ func (ps *pulloutSubquery) Rewrite(inputs ...logicalPlan) error { return nil } +// Solves implements the logicalPlan interface +func (ps *pulloutSubquery) Solves() semantics.TableSet { + return ps.underlying.Solves().Merge(ps.subquery.Solves()) +} + // Inputs implements the logicalPlan interface func (ps *pulloutSubquery) Inputs() []logicalPlan { return []logicalPlan{ps.underlying, ps.subquery} diff --git a/go/vt/vtgate/planbuilder/querygraph.go b/go/vt/vtgate/planbuilder/querygraph.go index f3aae564c41..4362a92e535 100644 --- a/go/vt/vtgate/planbuilder/querygraph.go +++ b/go/vt/vtgate/planbuilder/querygraph.go @@ -192,9 +192,16 @@ func (qg *queryGraph) tryMerge(a, b joinTree, joinPredicates []sqlparser.Expr) j return nil } case engine.SelectEqualUnique: - return nil + case engine.SelectScatter: + //if len(joinPredicates) == 0 { + // If we are doing two Scatters, we have to make sure that the + // joins are on the correct vindex to allow them to be merged + // no join predicates - no vindex + return nil + //} } + newTabletSet := aRoute.solved | bRoute.solved r := &routePlan{ routeOpCode: aRoute.routeOpCode, diff --git a/go/vt/vtgate/planbuilder/route.go b/go/vt/vtgate/planbuilder/route.go index f1ce57a1144..898341cf9eb 100644 --- a/go/vt/vtgate/planbuilder/route.go +++ b/go/vt/vtgate/planbuilder/route.go @@ -19,6 +19,7 @@ package planbuilder import ( vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" "vitess.io/vitess/go/vt/vterrors" + "vitess.io/vitess/go/vt/vtgate/semantics" "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/sqlparser" @@ -63,6 +64,9 @@ type route struct { // eroute is the primitive being built. eroute *engine.Route + + // solvedTables keeps track of which tables this route is covering + solvedTables semantics.TableSet } type tableSubstitution struct { @@ -128,6 +132,23 @@ func (rb *route) SetLimit(limit *sqlparser.Limit) { rb.Select.SetLimit(limit) } +// Wireup2 implements the logicalPlan interface +func (rb *route) Wireup2(semTable *semantics.SemTable) error { + rb.prepareTheAST() + + rb.eroute.Query = sqlparser.String(rb.Select) + buffer := sqlparser.NewTrackedBuffer(nil) + sqlparser.FormatImpossibleQuery(buffer, rb.Select) + rb.eroute.FieldQuery = buffer.ParsedQuery().Query + + return nil +} + +// Solves implements the logicalPlan interface +func (rb *route) Solves() semantics.TableSet { + return rb.solvedTables +} + // Wireup implements the logicalPlan interface func (rb *route) Wireup(plan logicalPlan, jt *jointab) error { // Precaution: update ERoute.Values only if it's not set already. @@ -204,6 +225,23 @@ func (rb *route) Wireup(plan logicalPlan, jt *jointab) error { return nil } +// prepareTheAST does minor fixups of the SELECT struct before producing the query string +func (rb *route) prepareTheAST() { + _ = sqlparser.Walk(func(node sqlparser.SQLNode) (bool, error) { + switch node := node.(type) { + case *sqlparser.Select: + if len(node.SelectExprs) == 0 { + node.SelectExprs = []sqlparser.SelectExpr{ + &sqlparser.AliasedExpr{ + Expr: sqlparser.NewIntLiteral([]byte{'1'}), + }, + } + } + } + return true, nil + }, rb.Select) +} + // procureValues procures and converts the input into // the expected types for rb.Values. func (rb *route) procureValues(plan logicalPlan, jt *jointab, val sqlparser.Expr) (sqltypes.PlanValue, error) { diff --git a/go/vt/vtgate/planbuilder/route_planning.go b/go/vt/vtgate/planbuilder/route_planning.go index 4dd7b7389dd..5952e7c4ade 100644 --- a/go/vt/vtgate/planbuilder/route_planning.go +++ b/go/vt/vtgate/planbuilder/route_planning.go @@ -31,6 +31,61 @@ import ( "vitess.io/vitess/go/vt/vtgate/engine" ) +func newBuildSelectPlan(sel *sqlparser.Select, vschema ContextVSchema) (engine.Primitive, error) { + semTable, err := semantics.Analyse(sel, nil) // TODO no nil no + if err != nil { + return nil, err + } + + qgraph, err := createQGFromSelect(sel, semTable) + if err != nil { + return nil, err + } + + tree, err := solve(qgraph, semTable, vschema) + if err != nil { + return nil, err + } + + plan, err := transformToLogicalPlan(tree) + if err != nil { + return nil, err + } + + if err := planProjections(sel, plan, semTable); err != nil { + return nil, err + } + + if err := plan.Wireup2(semTable); err != nil { + return nil, err + } + return plan.Primitive(), nil +} + +func planProjections(sel *sqlparser.Select, plan logicalPlan, semTable *semantics.SemTable) error { + rb, ok := plan.(*route) + if ok { + rb.Select = sel + } else { + var projections []*sqlparser.AliasedExpr + + // TODO real horizon planning to be done + for _, expr := range sel.SelectExprs { + switch e := expr.(type) { + case *sqlparser.AliasedExpr: + projections = append(projections, e) + default: + return vterrors.Errorf(vtrpcpb.Code_INTERNAL, "not yet supported %T", e) + } + } + + if _, err := pushProjection(projections, plan, semTable); err != nil { + return err + } + } + return nil +} + type ( joinTree interface { solves() semantics.TableSet @@ -182,8 +237,8 @@ func solve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVSchema) } for currentSize := 2; currentSize <= size; currentSize++ { - lefts := dpTable.bitSetsOfSize(1) - rights := dpTable.bitSetsOfSize(currentSize - 1) + lefts := dpTable.bitSetsOfSize(currentSize - 1) + rights := dpTable.bitSetsOfSize(1) for _, lhs := range lefts { for _, rhs := range rights { if semantics.IsOverlapping(lhs.solves(), rhs.solves()) { @@ -296,17 +351,23 @@ func transformToLogicalPlan(tree joinTree) (logicalPlan, error) { From: tablesForSelect, Where: where, }, + solvedTables: n.solved, }, nil case *joinPlan: - _, err := transformToLogicalPlan(n.lhs) + lhs, err := transformToLogicalPlan(n.lhs) if err != nil { return nil, err } - _, err = transformToLogicalPlan(n.rhs) + rhs, err := transformToLogicalPlan(n.rhs) if err != nil { return nil, err } + return &join2{ + Left: lhs, + Right: rhs, + }, nil } - panic(42) + + return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "BUG: unknown type encountered: %T", tree) } diff --git a/go/vt/vtgate/planbuilder/select.go b/go/vt/vtgate/planbuilder/select.go index 46496b0f1ac..5b1e020fb61 100644 --- a/go/vt/vtgate/planbuilder/select.go +++ b/go/vt/vtgate/planbuilder/select.go @@ -38,57 +38,81 @@ import ( func buildSelectPlan(query string) func(sqlparser.Statement, ContextVSchema) (engine.Primitive, error) { return func(stmt sqlparser.Statement, vschema ContextVSchema) (engine.Primitive, error) { sel := stmt.(*sqlparser.Select) - if !vschema.NewPlanner() { - p, err := handleDualSelects(sel, vschema) - if err != nil { - return nil, err - } - if p != nil { - return p, nil - } - - pb := newPrimitiveBuilder(vschema, newJointab(sqlparser.GetBindvars(sel))) - if err := pb.processSelect(sel, nil, query); err != nil { - return nil, err - } - if err := pb.plan.Wireup(pb.plan, pb.jt); err != nil { - return nil, err - } - return pb.plan.Primitive(), nil + if vschema.NewPlanner() { + return newBuildSelectPlan(sel, vschema) } - semTable, err := semantics.Analyse(sel, nil) // TODO no nil no + p, err := handleDualSelects(sel, vschema) if err != nil { return nil, err } + if p != nil { + return p, nil + } - qgraph, err := createQGFromSelect(sel, semTable) - if err != nil { + pb := newPrimitiveBuilder(vschema, newJointab(sqlparser.GetBindvars(sel))) + if err := pb.processSelect(sel, nil, query); err != nil { return nil, err } - - tree, err := solve(qgraph, semTable, vschema) - if err != nil { + if err := pb.plan.Wireup(pb.plan, pb.jt); err != nil { return nil, err } + return pb.plan.Primitive(), nil + } +} - plan, err := transformToLogicalPlan(tree) +func pushProjection(expr []*sqlparser.AliasedExpr, plan logicalPlan, semTable *semantics.SemTable) (firstOffset int, err error) { + switch node := plan.(type) { + case *route: + sel := node.Select.(*sqlparser.Select) + offset := len(sel.SelectExprs) + for _, e := range expr { + sel.SelectExprs = append(sel.SelectExprs, e) + } + return offset, nil + case *join2: + cols := make([]int, len(expr)) + var lhs, rhs []*sqlparser.AliasedExpr + lhsSolves := node.Left.Solves() + rhsSolves := node.Right.Solves() + for i, e := range expr { + deps := semTable.Dependencies(e.Expr) + switch { + case semantics.IsContainedBy(deps, lhsSolves): + lhs = append(lhs, e) + cols[i] = -1 + case semantics.IsContainedBy(deps, rhsSolves): + rhs = append(rhs, e) + cols[i] = 1 + default: + return 0, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "unknown dependencies for %s", sqlparser.String(e.Expr)) + } + } + lOffset, err := pushProjection(lhs, node.Left, semTable) if err != nil { - return nil, err + return 0, err } - - // minimal horizon planning - rb, ok := plan.(*route) - if ok { - rb.Select = sel - rb.eroute.Query = sqlparser.String(sel) - buffer := sqlparser.NewTrackedBuffer(nil) - sqlparser.FormatImpossibleQuery(buffer, sel) - rb.eroute.FieldQuery = buffer.ParsedQuery().Query + rOffset, err := pushProjection(rhs, node.Right, semTable) + if err != nil { + return 0, err + } + rOffset++ + lOffset = -(lOffset + 1) + for i, col := range cols { + if col == -1 { + cols[i] = lOffset + lOffset-- + } else { + cols[i] = rOffset + rOffset++ + } } + node.Cols = cols + return 0, nil - return plan.Primitive(), nil + default: + return 0, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "not yet supported %T", node) } } diff --git a/go/vt/vtgate/planbuilder/sql_calc_found_rows.go b/go/vt/vtgate/planbuilder/sql_calc_found_rows.go index 9ec495e7f4b..96f5f602502 100644 --- a/go/vt/vtgate/planbuilder/sql_calc_found_rows.go +++ b/go/vt/vtgate/planbuilder/sql_calc_found_rows.go @@ -21,6 +21,7 @@ import ( "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vterrors" "vitess.io/vitess/go/vt/vtgate/engine" + "vitess.io/vitess/go/vt/vtgate/semantics" ) var _ logicalPlan = (*sqlCalcFoundRows)(nil) @@ -39,6 +40,20 @@ func (s *sqlCalcFoundRows) Wireup(logicalPlan, *jointab) error { return s.CountQuery.Wireup(s.CountQuery, s.cjt) } +//Wireup2 implements the logicalPlan interface +func (s *sqlCalcFoundRows) Wireup2(semTable *semantics.SemTable) error { + err := s.LimitQuery.Wireup2(semTable) + if err != nil { + return err + } + return s.CountQuery.Wireup2(semTable) +} + +// Solves implements the logicalPlan interface +func (s *sqlCalcFoundRows) Solves() semantics.TableSet { + return s.LimitQuery.Solves() +} + //Primitive implements the logicalPlan interface func (s *sqlCalcFoundRows) Primitive() engine.Primitive { return engine.SQLCalcFoundRows{ diff --git a/go/vt/vtgate/planbuilder/testdata/from_cases.txt b/go/vt/vtgate/planbuilder/testdata/from_cases.txt index d99c7912a34..f0999b9d2b2 100644 --- a/go/vt/vtgate/planbuilder/testdata/from_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/from_cases.txt @@ -151,6 +151,8 @@ ] } } +{ +} # routing rules where table name matches, and there's no alias. "select * from second_user.user" @@ -290,6 +292,8 @@ ] } } +{ +} # ',' join unsharded "select u1.a, u2.a from unsharded u1, unsharded u2" diff --git a/go/vt/vtgate/planbuilder/testdata/select_cases.txt b/go/vt/vtgate/planbuilder/testdata/select_cases.txt index 92c08607a9f..84db2518a42 100644 --- a/go/vt/vtgate/planbuilder/testdata/select_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/select_cases.txt @@ -554,6 +554,8 @@ ] } } +{ +} # Both routes referenced "select user.col, user_extra.id from user join user_extra" @@ -591,6 +593,8 @@ ] } } +{ +} # Expression with single-route reference "select user.col, user_extra.id + user_extra.col from user join user_extra" @@ -628,6 +632,8 @@ ] } } +{ +} # Jumbled references "select user.col, user_extra.id, user.col2 from user join user_extra" @@ -665,6 +671,8 @@ ] } } +{ +} # Comments "select /* comment */ user.col from user join user_extra" @@ -813,6 +821,8 @@ ] } } +{ +} # syntax error "the quick brown fox" diff --git a/go/vt/vtgate/planbuilder/vindex_func.go b/go/vt/vtgate/planbuilder/vindex_func.go index e29eb7a2308..17c6a24939d 100644 --- a/go/vt/vtgate/planbuilder/vindex_func.go +++ b/go/vt/vtgate/planbuilder/vindex_func.go @@ -19,6 +19,8 @@ package planbuilder import ( "errors" + "vitess.io/vitess/go/vt/vtgate/semantics" + vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" "vitess.io/vitess/go/vt/vterrors" @@ -92,7 +94,12 @@ func (vf *vindexFunc) ResultColumns() []*resultColumn { } // Wireup implements the logicalPlan interface -func (vf *vindexFunc) Wireup(plan logicalPlan, jt *jointab) error { +func (vf *vindexFunc) Wireup(logicalPlan, *jointab) error { + return nil +} + +// Wireup2 implements the logicalPlan interface +func (vf *vindexFunc) Wireup2(*semantics.SemTable) error { return nil } @@ -137,6 +144,11 @@ func (vf *vindexFunc) Rewrite(inputs ...logicalPlan) error { return nil } +// Solves implements the logicalPlan interface +func (vf *vindexFunc) Solves() semantics.TableSet { + return 0 +} + // Inputs implements the logicalPlan interface func (vf *vindexFunc) Inputs() []logicalPlan { return []logicalPlan{} diff --git a/go/vt/vtgate/semantics/semantic_state.go b/go/vt/vtgate/semantics/semantic_state.go index 45168a3bf06..8ebbbf8347e 100644 --- a/go/vt/vtgate/semantics/semantic_state.go +++ b/go/vt/vtgate/semantics/semantic_state.go @@ -59,6 +59,11 @@ func (ts TableSet) NumberOfTables() int { return count } +// Merge creates a TableSet that contains both inputs +func (ts TableSet) Merge(other TableSet) TableSet { + return ts | other +} + // TableSetFor returns the bitmask for this particular tableshoe func (st *SemTable) TableSetFor(t table) TableSet { for idx, t2 := range st.Tables { From 96915413e296b16099991c29bb359747cd8d88ea Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Tue, 22 Dec 2020 18:04:38 +0100 Subject: [PATCH 09/40] remove randomness from the planbuilding process Signed-off-by: Andres Taylor --- go/vt/vtgate/planbuilder/querygraph.go | 11 ----- go/vt/vtgate/planbuilder/route_planning.go | 50 +++++++++++++++++++--- 2 files changed, 43 insertions(+), 18 deletions(-) diff --git a/go/vt/vtgate/planbuilder/querygraph.go b/go/vt/vtgate/planbuilder/querygraph.go index 4362a92e535..550cd88a900 100644 --- a/go/vt/vtgate/planbuilder/querygraph.go +++ b/go/vt/vtgate/planbuilder/querygraph.go @@ -162,17 +162,6 @@ func (qg *queryGraph) addNoDepsPredicate(predicate sqlparser.Expr) { } } -func (dpt dpTableT) bitSetsOfSize(wanted int) []joinTree { - var result []joinTree - for bs, jt := range dpt { - size := bs.NumberOfTables() - if size == wanted { - result = append(result, jt) - } - } - return result -} - func (qg *queryGraph) tryMerge(a, b joinTree, joinPredicates []sqlparser.Expr) joinTree { aRoute, ok := a.(*routePlan) if !ok { diff --git a/go/vt/vtgate/planbuilder/route_planning.go b/go/vt/vtgate/planbuilder/route_planning.go index 5952e7c4ade..9dc37fc283d 100644 --- a/go/vt/vtgate/planbuilder/route_planning.go +++ b/go/vt/vtgate/planbuilder/route_planning.go @@ -112,9 +112,45 @@ type ( lhs, rhs joinTree } - dpTableT map[semantics.TableSet]joinTree + dpTableT struct { + // hash map of the best solution for each seen table + m map[semantics.TableSet]joinTree + + highest semantics.TableSet + } ) +func makeDPTable() *dpTableT { + return &dpTableT{ + m: map[semantics.TableSet]joinTree{}, + } +} + +func (dpt *dpTableT) add(tree joinTree) { + solved := tree.solves() + if dpt.highest < solved { + dpt.highest = solved + } + dpt.m[solved] = tree +} + +func (dpt *dpTableT) planFor(id semantics.TableSet) joinTree { + return dpt.m[id] +} + +func (dpt *dpTableT) bitSetsOfSize(wanted int) []joinTree { + var result []joinTree + for x := semantics.TableSet(1); x < dpt.highest; x++ { + if x.NumberOfTables() == wanted { + t, ok := dpt.m[x] + if ok { + result = append(result, t) + } + } + } + return result +} + func (rp *routePlan) solves() semantics.TableSet { return rp.solved } @@ -221,7 +257,7 @@ func (jp *joinPlan) cost() int { */ func solve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVSchema) (joinTree, error) { size := len(qg.tables) - dpTable := make(dpTableT) + dpTable := makeDPTable() var allTables semantics.TableSet @@ -229,11 +265,11 @@ func solve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVSchema) for _, table := range qg.tables { solves := semTable.TableSetFor(table.alias) allTables |= solves - var err error - dpTable[solves], err = createRoutePlan(table, solves, vschema) + plan, err := createRoutePlan(table, solves, vschema) if err != nil { return nil, err } + dpTable.add(plan) } for currentSize := 2; currentSize <= size; currentSize++ { @@ -246,7 +282,7 @@ func solve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVSchema) continue } solves := lhs.solves() | rhs.solves() - oldPlan := dpTable[solves] + oldPlan := dpTable.planFor(solves) if oldPlan != nil && oldPlan.cost() == 1 { // we already have the perfect plan. keep it continue @@ -261,13 +297,13 @@ func solve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVSchema) } } if oldPlan == nil || newPlan.cost() < oldPlan.cost() { - dpTable[solves] = newPlan + dpTable.add(newPlan) } } } } - return dpTable[allTables], nil + return dpTable.planFor(allTables), nil } func createRoutePlan(table *queryTable, solves semantics.TableSet, vschema ContextVSchema) (*routePlan, error) { From 713ea58caf7ba1617fae2d62b2d2e880864c7849 Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Wed, 23 Dec 2020 13:39:41 +0100 Subject: [PATCH 10/40] fix issue with the dp table Signed-off-by: Andres Taylor --- go/vt/vtgate/planbuilder/dptable.go | 59 +++++++++++++++++ go/vt/vtgate/planbuilder/dptable_test.go | 63 +++++++++++++++++++ go/vt/vtgate/planbuilder/route_planning.go | 37 ----------- go/vt/vtgate/planbuilder/testdata/onecase.txt | 38 +++++++++++ 4 files changed, 160 insertions(+), 37 deletions(-) create mode 100644 go/vt/vtgate/planbuilder/dptable.go create mode 100644 go/vt/vtgate/planbuilder/dptable_test.go diff --git a/go/vt/vtgate/planbuilder/dptable.go b/go/vt/vtgate/planbuilder/dptable.go new file mode 100644 index 00000000000..c65870cddc2 --- /dev/null +++ b/go/vt/vtgate/planbuilder/dptable.go @@ -0,0 +1,59 @@ +/* +Copyright 2020 The Vitess 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 planbuilder + +import "vitess.io/vitess/go/vt/vtgate/semantics" + +// dpTable, is the hashmap we store results in during +// the dynamic programming part of query planning +type dpTable struct { + // hash map of the best solution for each seen table + m map[semantics.TableSet]joinTree + + highest semantics.TableSet +} + +func makeDPTable() *dpTable { + return &dpTable{ + m: map[semantics.TableSet]joinTree{}, + } +} + +func (dpt *dpTable) add(tree joinTree) { + solved := tree.solves() + if dpt.highest < solved { + dpt.highest = solved + } + dpt.m[solved] = tree +} + +func (dpt *dpTable) planFor(id semantics.TableSet) joinTree { + return dpt.m[id] +} + +func (dpt *dpTable) bitSetsOfSize(wanted int) []joinTree { + var result []joinTree + for x := semantics.TableSet(1); x <= dpt.highest; x++ { + if x.NumberOfTables() == wanted { + t, ok := dpt.m[x] + if ok { + result = append(result, t) + } + } + } + return result +} diff --git a/go/vt/vtgate/planbuilder/dptable_test.go b/go/vt/vtgate/planbuilder/dptable_test.go new file mode 100644 index 00000000000..e0e10e1b153 --- /dev/null +++ b/go/vt/vtgate/planbuilder/dptable_test.go @@ -0,0 +1,63 @@ +/* +Copyright 2020 The Vitess 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 planbuilder + +import ( + "testing" + + "github.com/stretchr/testify/assert" + + "vitess.io/vitess/go/vt/vtgate/semantics" +) + +type fakePlan struct { + solve semantics.TableSet +} + +func (f *fakePlan) solves() semantics.TableSet { + return f.solve +} + +func (f *fakePlan) cost() int { + return 1 +} + +var _ joinTree = (*fakePlan)(nil) + +func TestDpTableSizeOf(t *testing.T) { + dpTable := makeDPTable() + + a := semantics.TableSet(1) + b := semantics.TableSet(2) + + t1 := &fakePlan{solve: a} + t2 := &fakePlan{solve: b} + t3 := &fakePlan{solve: a.Merge(b)} + + dpTable.add(t1) + dpTable.add(t2) + dpTable.add(t3) + + size1 := dpTable.bitSetsOfSize(1) + assert.Equal(t, []joinTree{t1, t2}, size1, "size 1") + + size2 := dpTable.bitSetsOfSize(2) + assert.Equal(t, []joinTree{t3}, size2, "size 2") + assert.Equal(t, t1, dpTable.planFor(a)) + assert.Equal(t, t2, dpTable.planFor(b)) + assert.Equal(t, t3, dpTable.planFor(a.Merge(b))) +} diff --git a/go/vt/vtgate/planbuilder/route_planning.go b/go/vt/vtgate/planbuilder/route_planning.go index 9dc37fc283d..60144023284 100644 --- a/go/vt/vtgate/planbuilder/route_planning.go +++ b/go/vt/vtgate/planbuilder/route_planning.go @@ -112,45 +112,8 @@ type ( lhs, rhs joinTree } - dpTableT struct { - // hash map of the best solution for each seen table - m map[semantics.TableSet]joinTree - - highest semantics.TableSet - } ) -func makeDPTable() *dpTableT { - return &dpTableT{ - m: map[semantics.TableSet]joinTree{}, - } -} - -func (dpt *dpTableT) add(tree joinTree) { - solved := tree.solves() - if dpt.highest < solved { - dpt.highest = solved - } - dpt.m[solved] = tree -} - -func (dpt *dpTableT) planFor(id semantics.TableSet) joinTree { - return dpt.m[id] -} - -func (dpt *dpTableT) bitSetsOfSize(wanted int) []joinTree { - var result []joinTree - for x := semantics.TableSet(1); x < dpt.highest; x++ { - if x.NumberOfTables() == wanted { - t, ok := dpt.m[x] - if ok { - result = append(result, t) - } - } - } - return result -} - func (rp *routePlan) solves() semantics.TableSet { return rp.solved } diff --git a/go/vt/vtgate/planbuilder/testdata/onecase.txt b/go/vt/vtgate/planbuilder/testdata/onecase.txt index e819513f354..0ce1a45f0c9 100644 --- a/go/vt/vtgate/planbuilder/testdata/onecase.txt +++ b/go/vt/vtgate/planbuilder/testdata/onecase.txt @@ -1 +1,39 @@ # Add your test case here for debugging and run go test -run=One. +# ',' join +"select music.col from user, music" +{ + "QueryType": "SELECT", + "Original": "select music.col from user, music", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "1", + "TableName": "user_music", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from user where 1 != 1", + "Query": "select 1 from user", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select music.col from music where 1 != 1", + "Query": "select music.col from music", + "Table": "music" + } + ] + } +} +{ +} From 44de903d0701ef35bf734f2d8b6c2410c3cde85c Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Wed, 23 Dec 2020 21:54:42 +0100 Subject: [PATCH 11/40] add greedy option for large queries Signed-off-by: Andres Taylor --- go/vt/vtgate/planbuilder/querygraph.go | 2 +- go/vt/vtgate/planbuilder/route_planning.go | 78 ++++++++++++++++--- go/vt/vtgate/planbuilder/testdata/onecase.txt | 40 +--------- 3 files changed, 70 insertions(+), 50 deletions(-) diff --git a/go/vt/vtgate/planbuilder/querygraph.go b/go/vt/vtgate/planbuilder/querygraph.go index 550cd88a900..f24427c178d 100644 --- a/go/vt/vtgate/planbuilder/querygraph.go +++ b/go/vt/vtgate/planbuilder/querygraph.go @@ -162,7 +162,7 @@ func (qg *queryGraph) addNoDepsPredicate(predicate sqlparser.Expr) { } } -func (qg *queryGraph) tryMerge(a, b joinTree, joinPredicates []sqlparser.Expr) joinTree { +func tryMerge(a, b joinTree, joinPredicates []sqlparser.Expr) joinTree { aRoute, ok := a.(*routePlan) if !ok { return nil diff --git a/go/vt/vtgate/planbuilder/route_planning.go b/go/vt/vtgate/planbuilder/route_planning.go index 60144023284..8c2785ff4a7 100644 --- a/go/vt/vtgate/planbuilder/route_planning.go +++ b/go/vt/vtgate/planbuilder/route_planning.go @@ -42,7 +42,12 @@ func newBuildSelectPlan(sel *sqlparser.Select, vschema ContextVSchema) (engine.P return nil, err } - tree, err := solve(qgraph, semTable, vschema) + var tree joinTree + if len(qgraph.tables) <= 10 { + tree, err = dpSolve(qgraph, semTable, vschema) + } else { + tree, err = greedySolve(qgraph, semTable, vschema) + } if err != nil { return nil, err } @@ -218,7 +223,7 @@ func (jp *joinPlan) cost() int { we use dynamic programming to find the cheapest route/join tree possible, where the cost of a plan is the number of joins */ -func solve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVSchema) (joinTree, error) { +func dpSolve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVSchema) (joinTree, error) { size := len(qg.tables) dpTable := makeDPTable() @@ -251,14 +256,7 @@ func solve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVSchema) continue } joinPredicates := qg.crossTable[solves] - newPlan := qg.tryMerge(lhs, rhs, joinPredicates) - if newPlan == nil { - newPlan = &joinPlan{ - lhs: lhs, - rhs: rhs, - predicates: joinPredicates, - } - } + newPlan := createJoin(lhs, rhs, joinPredicates) if oldPlan == nil || newPlan.cost() < oldPlan.cost() { dpTable.add(newPlan) } @@ -269,6 +267,66 @@ func solve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVSchema) return dpTable.planFor(allTables), nil } +func createJoin(lhs joinTree, rhs joinTree, joinPredicates []sqlparser.Expr) joinTree { + newPlan := tryMerge(lhs, rhs, joinPredicates) + if newPlan == nil { + newPlan = &joinPlan{ + lhs: lhs, + rhs: rhs, + predicates: joinPredicates, + } + } + return newPlan +} + +/* + + */ +func greedySolve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVSchema) (joinTree, error) { + plans := make([]joinTree, len(qg.tables)) + planCache := map[semantics.TableSet]joinTree{} + + // we start by seeding the table with the single routes + for i, table := range qg.tables { + solves := semTable.TableSetFor(table.alias) + plan, err := createRoutePlan(table, solves, vschema) + if err != nil { + return nil, err + } + plans[i] = plan + } + + // loop while we have un-joined query parts left + for len(plans) > 1 { + var lIdx, rIdx int + var bestPlan joinTree + for i, lhs := range plans { + for j := i + 1; j < len(plans); j++ { + rhs := plans[j] + solves := lhs.solves() | rhs.solves() + joinPredicates := qg.crossTable[solves] + plan := planCache[solves] + if plan == nil { + plan = createJoin(lhs, rhs, joinPredicates) + planCache[solves] = plan + } + + if bestPlan == nil || plan.cost() < bestPlan.cost() { + bestPlan = plan + // remember which plans we based on, so we can remove them later + lIdx = i + rIdx = j + } + } + } + plans = append(plans[:rIdx], plans[rIdx+1:]...) + plans = append(plans[:lIdx], plans[lIdx+1:]...) + plans = append(plans, bestPlan) + } + + return plans[0], nil +} + func createRoutePlan(table *queryTable, solves semantics.TableSet, vschema ContextVSchema) (*routePlan, error) { vschemaTable, _, _, _, _, err := vschema.FindTableOrVindex(table.table) if err != nil { diff --git a/go/vt/vtgate/planbuilder/testdata/onecase.txt b/go/vt/vtgate/planbuilder/testdata/onecase.txt index 0ce1a45f0c9..16a368ddafd 100644 --- a/go/vt/vtgate/planbuilder/testdata/onecase.txt +++ b/go/vt/vtgate/planbuilder/testdata/onecase.txt @@ -1,39 +1 @@ -# Add your test case here for debugging and run go test -run=One. -# ',' join -"select music.col from user, music" -{ - "QueryType": "SELECT", - "Original": "select music.col from user, music", - "Instructions": { - "OperatorType": "Join", - "Variant": "Join", - "JoinColumnIndexes": "1", - "TableName": "user_music", - "Inputs": [ - { - "OperatorType": "Route", - "Variant": "SelectScatter", - "Keyspace": { - "Name": "user", - "Sharded": true - }, - "FieldQuery": "select 1 from user where 1 != 1", - "Query": "select 1 from user", - "Table": "user" - }, - { - "OperatorType": "Route", - "Variant": "SelectScatter", - "Keyspace": { - "Name": "user", - "Sharded": true - }, - "FieldQuery": "select music.col from music where 1 != 1", - "Query": "select music.col from music", - "Table": "music" - } - ] - } -} -{ -} +# Add your test case here for debugging and run go test -run=One. \ No newline at end of file From 14f4680afd01edd8ee5ed2dd0ba1d3cfd1d3676d Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Sun, 27 Dec 2020 12:38:36 +0100 Subject: [PATCH 12/40] refactor: extract method Signed-off-by: Andres Taylor --- go/vt/vtgate/planbuilder/route_planning.go | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/go/vt/vtgate/planbuilder/route_planning.go b/go/vt/vtgate/planbuilder/route_planning.go index 8c2785ff4a7..ff8afe5eb4c 100644 --- a/go/vt/vtgate/planbuilder/route_planning.go +++ b/go/vt/vtgate/planbuilder/route_planning.go @@ -319,14 +319,18 @@ func greedySolve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVS } } } - plans = append(plans[:rIdx], plans[rIdx+1:]...) - plans = append(plans[:lIdx], plans[lIdx+1:]...) + plans = removeAt(plans, rIdx) + plans = removeAt(plans, lIdx) plans = append(plans, bestPlan) } return plans[0], nil } +func removeAt(plans []joinTree, idx int) []joinTree { + return append(plans[:idx], plans[idx+1:]...) +} + func createRoutePlan(table *queryTable, solves semantics.TableSet, vschema ContextVSchema) (*routePlan, error) { vschemaTable, _, _, _, _, err := vschema.FindTableOrVindex(table.table) if err != nil { From 9224b3aa53d0c9a37f001c361b68b59626c071eb Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Sun, 27 Dec 2020 13:01:04 +0100 Subject: [PATCH 13/40] refactor: querygraph and test Signed-off-by: Andres Taylor --- go/vt/vtgate/planbuilder/querygraph.go | 12 +-- go/vt/vtgate/planbuilder/querygraph_test.go | 100 +++++++++++--------- 2 files changed, 62 insertions(+), 50 deletions(-) diff --git a/go/vt/vtgate/planbuilder/querygraph.go b/go/vt/vtgate/planbuilder/querygraph.go index f24427c178d..b0d5ef8cb93 100644 --- a/go/vt/vtgate/planbuilder/querygraph.go +++ b/go/vt/vtgate/planbuilder/querygraph.go @@ -46,10 +46,10 @@ type ( // queryTable is a single FROM table, including all predicates particular to this table queryTable struct { - tableIdentifier semantics.TableSet - alias *sqlparser.AliasedTableExpr - table sqlparser.TableName - predicates []sqlparser.Expr + tableID semantics.TableSet + alias *sqlparser.AliasedTableExpr + table sqlparser.TableName + predicates []sqlparser.Expr } ) @@ -78,7 +78,7 @@ func (qg *queryGraph) collectTable(t sqlparser.TableExpr, semTable *semantics.Se switch table := t.(type) { case *sqlparser.AliasedTableExpr: tableName := table.Expr.(sqlparser.TableName) - qt := &queryTable{alias: table, table: tableName, tableIdentifier: semTable.TableSetFor(table)} + qt := &queryTable{alias: table, table: tableName, tableID: semTable.TableSetFor(table)} qg.tables = append(qg.tables, qt) case *sqlparser.JoinTableExpr: if err := qg.collectTable(table.LeftExpr, semTable); err != nil { @@ -143,7 +143,7 @@ func (qg *queryGraph) collectPredicate(predicate sqlparser.Expr, semTable *seman func (qg *queryGraph) addToSingleTable(table semantics.TableSet, predicate sqlparser.Expr) bool { for _, t := range qg.tables { - if table == t.tableIdentifier { + if table == t.tableID { t.predicates = append(t.predicates, predicate) return true } diff --git a/go/vt/vtgate/planbuilder/querygraph_test.go b/go/vt/vtgate/planbuilder/querygraph_test.go index 9d46180c09b..670bf141c59 100644 --- a/go/vt/vtgate/planbuilder/querygraph_test.go +++ b/go/vt/vtgate/planbuilder/querygraph_test.go @@ -36,58 +36,45 @@ var tcases = []tcase{ { input: "select * from t", output: &queryGraph{ - tables: []*queryTable{ - { - tableIdentifier: 1, - alias: &sqlparser.AliasedTableExpr{ - Expr: sqlparser.TableName{ - Name: sqlparser.NewTableIdent("t"), - }, - }, - table: sqlparser.TableName{Name: sqlparser.NewTableIdent("t")}, - predicates: nil, - }, - }, + tables: []*queryTable{{ + tableID: 1, + alias: tableAlias("t"), + table: tableName("t"), + }}, crossTable: map[semantics.TableSet][]sqlparser.Expr{}, }, }, { input: "select t.c from t,y,z where t.c = y.c and (t.a = z.a or t.a = y.a) and 1 < 2", output: &queryGraph{ - tables: []*queryTable{ - {tableIdentifier: 1, alias: &sqlparser.AliasedTableExpr{Expr: sqlparser.TableName{Name: sqlparser.NewTableIdent("t")}}, table: sqlparser.TableName{Name: sqlparser.NewTableIdent("t")}}, - { - tableIdentifier: 2, - alias: &sqlparser.AliasedTableExpr{Expr: sqlparser.TableName{Name: sqlparser.NewTableIdent("y")}}, - table: sqlparser.TableName{Name: sqlparser.NewTableIdent("y")}, - }, - { - tableIdentifier: 4, - alias: &sqlparser.AliasedTableExpr{Expr: sqlparser.TableName{Name: sqlparser.NewTableIdent("z")}}, - table: sqlparser.TableName{Name: sqlparser.NewTableIdent("z")}, - }, - }, + tables: []*queryTable{{ + tableID: 1, + alias: tableAlias("t"), + table: tableName("t"), + }, { + tableID: 2, + alias: tableAlias("y"), + table: tableName("y"), + }, { + tableID: 4, + alias: tableAlias("z"), + table: tableName("z"), + }}, crossTable: map[semantics.TableSet][]sqlparser.Expr{ - 3: { - &sqlparser.ComparisonExpr{ - Left: &sqlparser.ColName{Name: sqlparser.NewColIdent("c"), Qualifier: sqlparser.TableName{Name: sqlparser.NewTableIdent("t")}}, - Right: &sqlparser.ColName{Name: sqlparser.NewColIdent("c"), Qualifier: sqlparser.TableName{Name: sqlparser.NewTableIdent("y")}}, - }, - }, - 7: { - &sqlparser.OrExpr{ - Left: &sqlparser.ComparisonExpr{ - Left: &sqlparser.ColName{Name: sqlparser.NewColIdent("a"), Qualifier: sqlparser.TableName{Name: sqlparser.NewTableIdent("t")}}, - Right: &sqlparser.ColName{Name: sqlparser.NewColIdent("a"), Qualifier: sqlparser.TableName{Name: sqlparser.NewTableIdent("z")}}, - }, - Right: &sqlparser.ComparisonExpr{ - Left: &sqlparser.ColName{Name: sqlparser.NewColIdent("a"), Qualifier: sqlparser.TableName{Name: sqlparser.NewTableIdent("t")}}, - Right: &sqlparser.ColName{Name: sqlparser.NewColIdent("a"), Qualifier: sqlparser.TableName{Name: sqlparser.NewTableIdent("y")}}, - }, - }, - }, + 1 | 2: { + equals( + colName("t", "c"), + colName("y", "c"))}, + 1 | 2 | 4: { + or( + equals( + colName("t", "a"), + colName("z", "a")), + equals( + colName("t", "a"), + colName("y", "a")))}, }, noDeps: &sqlparser.ComparisonExpr{ - Operator: 1, + Operator: sqlparser.LessThanOp, Left: &sqlparser.Literal{Type: 1, Val: []uint8{0x31}}, Right: &sqlparser.Literal{Type: 1, Val: []uint8{0x32}}, }, @@ -95,6 +82,31 @@ var tcases = []tcase{ }, } +func or(left, right sqlparser.Expr) sqlparser.Expr { + return &sqlparser.OrExpr{ + Left: left, + Right: right, + } +} +func equals(left, right sqlparser.Expr) sqlparser.Expr { + return &sqlparser.ComparisonExpr{ + Operator: sqlparser.EqualOp, + Left: left, + Right: right, + } +} +func colName(table, column string) *sqlparser.ColName { + return &sqlparser.ColName{Name: sqlparser.NewColIdent(column), Qualifier: tableName(table)} +} + +func tableAlias(name string) *sqlparser.AliasedTableExpr { + return &sqlparser.AliasedTableExpr{Expr: sqlparser.TableName{Name: sqlparser.NewTableIdent(name)}} +} + +func tableName(name string) sqlparser.TableName { + return sqlparser.TableName{Name: sqlparser.NewTableIdent(name)} +} + type schemaInf struct{} func (node *schemaInf) FindTable(tablename sqlparser.TableName) (*vindexes.Table, error) { From 9aa2331a6edee457d0b6d10bdfae1c246750b753 Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Sun, 27 Dec 2020 20:34:59 +0100 Subject: [PATCH 14/40] moved code to where it belongs Signed-off-by: Andres Taylor --- go/vt/vtgate/planbuilder/querygraph.go | 44 ------ go/vt/vtgate/planbuilder/querygraph_test.go | 143 ++++++++++++++------ go/vt/vtgate/planbuilder/route_planning.go | 43 ++++++ 3 files changed, 142 insertions(+), 88 deletions(-) diff --git a/go/vt/vtgate/planbuilder/querygraph.go b/go/vt/vtgate/planbuilder/querygraph.go index b0d5ef8cb93..6746e568530 100644 --- a/go/vt/vtgate/planbuilder/querygraph.go +++ b/go/vt/vtgate/planbuilder/querygraph.go @@ -20,7 +20,6 @@ import ( vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vterrors" - "vitess.io/vitess/go/vt/vtgate/engine" "vitess.io/vitess/go/vt/vtgate/semantics" ) @@ -161,46 +160,3 @@ func (qg *queryGraph) addNoDepsPredicate(predicate sqlparser.Expr) { } } } - -func tryMerge(a, b joinTree, joinPredicates []sqlparser.Expr) joinTree { - aRoute, ok := a.(*routePlan) - if !ok { - return nil - } - bRoute, ok := b.(*routePlan) - if !ok { - return nil - } - if aRoute.keyspace != bRoute.keyspace { - return nil - } - - switch aRoute.routeOpCode { - case engine.SelectUnsharded, engine.SelectDBA: - if aRoute.routeOpCode != bRoute.routeOpCode { - return nil - } - case engine.SelectEqualUnique: - return nil - case engine.SelectScatter: - //if len(joinPredicates) == 0 { - // If we are doing two Scatters, we have to make sure that the - // joins are on the correct vindex to allow them to be merged - // no join predicates - no vindex - return nil - //} - } - - newTabletSet := aRoute.solved | bRoute.solved - r := &routePlan{ - routeOpCode: aRoute.routeOpCode, - solved: newTabletSet, - tables: append(aRoute.tables, bRoute.tables...), - extraPredicates: append(aRoute.extraPredicates, bRoute.extraPredicates...), - keyspace: aRoute.keyspace, - vindexPlusPredicates: append(aRoute.vindexPlusPredicates, bRoute.vindexPlusPredicates...), - } - - r.extraPredicates = append(r.extraPredicates, joinPredicates...) - return r -} diff --git a/go/vt/vtgate/planbuilder/querygraph_test.go b/go/vt/vtgate/planbuilder/querygraph_test.go index 670bf141c59..6f42ab62f44 100644 --- a/go/vt/vtgate/planbuilder/querygraph_test.go +++ b/go/vt/vtgate/planbuilder/querygraph_test.go @@ -17,6 +17,7 @@ limitations under the License. package planbuilder import ( + "fmt" "testing" "vitess.io/vitess/go/test/utils" @@ -32,54 +33,106 @@ type tcase struct { output *queryGraph } -var tcases = []tcase{ - { - input: "select * from t", - output: &queryGraph{ - tables: []*queryTable{{ - tableID: 1, - alias: tableAlias("t"), - table: tableName("t"), - }}, - crossTable: map[semantics.TableSet][]sqlparser.Expr{}, - }, +var threeWayJoin = &queryGraph{ + tables: []*queryTable{{ + tableID: 1, + alias: tableAlias("t"), + table: tableName("t"), + predicates: []sqlparser.Expr{equals(colName("t", "name"), literalString("foo"))}, + }, { + tableID: 2, + alias: tableAlias("y"), + table: tableName("y"), + predicates: []sqlparser.Expr{equals(colName("y", "col"), literalInt(42))}, }, { - input: "select t.c from t,y,z where t.c = y.c and (t.a = z.a or t.a = y.a) and 1 < 2", - output: &queryGraph{ - tables: []*queryTable{{ - tableID: 1, - alias: tableAlias("t"), - table: tableName("t"), - }, { - tableID: 2, - alias: tableAlias("y"), - table: tableName("y"), - }, { - tableID: 4, - alias: tableAlias("z"), - table: tableName("z"), - }}, - crossTable: map[semantics.TableSet][]sqlparser.Expr{ - 1 | 2: { + tableID: 4, + alias: tableAlias("z"), + table: tableName("z"), + predicates: []sqlparser.Expr{equals(colName("z", "baz"), literalInt(101))}, + }}, + crossTable: map[semantics.TableSet][]sqlparser.Expr{ + 1 | 2: { + equals( + colName("t", "id"), + colName("y", "t_id"))}, + 1 | 4: { + equals( + colName("t", "id"), + colName("z", "t_id"))}}} + +var tcases = []tcase{{ + input: "select * from t", + output: &queryGraph{ + tables: []*queryTable{{ + tableID: 1, + alias: tableAlias("t"), + table: tableName("t"), + }}, + crossTable: map[semantics.TableSet][]sqlparser.Expr{}, + }, +}, { + input: "select t.c from t,y,z where t.c = y.c and (t.a = z.a or t.a = y.a) and 1 < 2", + output: &queryGraph{ + tables: []*queryTable{{ + tableID: 1, + alias: tableAlias("t"), + table: tableName("t"), + }, { + tableID: 2, + alias: tableAlias("y"), + table: tableName("y"), + }, { + tableID: 4, + alias: tableAlias("z"), + table: tableName("z"), + }}, + crossTable: map[semantics.TableSet][]sqlparser.Expr{ + 1 | 2: { + equals( + colName("t", "c"), + colName("y", "c"))}, + 1 | 2 | 4: { + or( equals( - colName("t", "c"), - colName("y", "c"))}, - 1 | 2 | 4: { - or( - equals( - colName("t", "a"), - colName("z", "a")), - equals( - colName("t", "a"), - colName("y", "a")))}, - }, - noDeps: &sqlparser.ComparisonExpr{ - Operator: sqlparser.LessThanOp, - Left: &sqlparser.Literal{Type: 1, Val: []uint8{0x31}}, - Right: &sqlparser.Literal{Type: 1, Val: []uint8{0x32}}, - }, + colName("t", "a"), + colName("z", "a")), + equals( + colName("t", "a"), + colName("y", "a")))}, + }, + noDeps: &sqlparser.ComparisonExpr{ + Operator: sqlparser.LessThanOp, + Left: literalInt(1), + Right: literalInt(2)}, + }, +}, { + input: "select t.c from t join y on t.id = y.t_id join z on t.id = z.t_id where t.name = 'foo' and y.col = 42 and z.baz = 101", + output: threeWayJoin, +}, { + input: "select t.c from t,y,z where t.name = 'foo' and y.col = 42 and z.baz = 101 and t.id = y.t_id and t.id = z.t_id", + output: threeWayJoin, +}, { + input: "select 1 from t where '1' = 1 and 12 = '12'", + output: &queryGraph{ + tables: []*queryTable{{ + tableID: 1, + alias: tableAlias("t"), + table: tableName("t"), + }}, + crossTable: map[semantics.TableSet][]sqlparser.Expr{}, + noDeps: &sqlparser.AndExpr{ + Left: equals(literalString("1"), literalInt(1)), + Right: equals(literalInt(12), literalString("12")), }, }, +}} + +func literalInt(i int) *sqlparser.Literal { + return &sqlparser.Literal{Type: sqlparser.IntVal, Val: []byte(fmt.Sprintf("%d", i))} +} + +func literalString(s string) *sqlparser.Literal { + return &sqlparser.Literal{Type: sqlparser.StrVal, Val: []byte(s)} } func or(left, right sqlparser.Expr) sqlparser.Expr { @@ -88,6 +141,7 @@ func or(left, right sqlparser.Expr) sqlparser.Expr { Right: right, } } + func equals(left, right sqlparser.Expr) sqlparser.Expr { return &sqlparser.ComparisonExpr{ Operator: sqlparser.EqualOp, @@ -95,6 +149,7 @@ func equals(left, right sqlparser.Expr) sqlparser.Expr { Right: right, } } + func colName(table, column string) *sqlparser.ColName { return &sqlparser.ColName{Name: sqlparser.NewColIdent(column), Qualifier: tableName(table)} } diff --git a/go/vt/vtgate/planbuilder/route_planning.go b/go/vt/vtgate/planbuilder/route_planning.go index ff8afe5eb4c..78fa22a85ce 100644 --- a/go/vt/vtgate/planbuilder/route_planning.go +++ b/go/vt/vtgate/planbuilder/route_planning.go @@ -432,3 +432,46 @@ func transformToLogicalPlan(tree joinTree) (logicalPlan, error) { return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "BUG: unknown type encountered: %T", tree) } + +func tryMerge(a, b joinTree, joinPredicates []sqlparser.Expr) joinTree { + aRoute, ok := a.(*routePlan) + if !ok { + return nil + } + bRoute, ok := b.(*routePlan) + if !ok { + return nil + } + if aRoute.keyspace != bRoute.keyspace { + return nil + } + + switch aRoute.routeOpCode { + case engine.SelectUnsharded, engine.SelectDBA: + if aRoute.routeOpCode != bRoute.routeOpCode { + return nil + } + case engine.SelectEqualUnique: + return nil + case engine.SelectScatter: + //if len(joinPredicates) == 0 { + // If we are doing two Scatters, we have to make sure that the + // joins are on the correct vindex to allow them to be merged + // no join predicates - no vindex + return nil + //} + } + + newTabletSet := aRoute.solved | bRoute.solved + r := &routePlan{ + routeOpCode: aRoute.routeOpCode, + solved: newTabletSet, + tables: append(aRoute.tables, bRoute.tables...), + extraPredicates: append(aRoute.extraPredicates, bRoute.extraPredicates...), + keyspace: aRoute.keyspace, + vindexPlusPredicates: append(aRoute.vindexPlusPredicates, bRoute.vindexPlusPredicates...), + } + + r.extraPredicates = append(r.extraPredicates, joinPredicates...) + return r +} From e500d3aa5b03bee95c978f1ccd1262ba1a0e1c3b Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Mon, 28 Dec 2020 07:07:27 +0100 Subject: [PATCH 15/40] simplify routePlan Signed-off-by: Andres Taylor --- go/vt/vtgate/planbuilder/route_planning.go | 44 ++++++++--------- .../vtgate/planbuilder/route_planning_test.go | 49 +++++++++++++++++++ 2 files changed, 70 insertions(+), 23 deletions(-) create mode 100644 go/vt/vtgate/planbuilder/route_planning_test.go diff --git a/go/vt/vtgate/planbuilder/route_planning.go b/go/vt/vtgate/planbuilder/route_planning.go index 78fa22a85ce..841b8943a23 100644 --- a/go/vt/vtgate/planbuilder/route_planning.go +++ b/go/vt/vtgate/planbuilder/route_planning.go @@ -107,10 +107,7 @@ type ( vindex vindexes.Vindex conditions []sqlparser.Expr - // this state keeps track of which vindexes are available and - // whether we have seen enough predicates to satisfy the vindex - vindexPlusPredicates []*vindexPlusPredicates - vtable *vindexes.Table + vtable *vindexes.Table } joinPlan struct { predicates []sqlparser.Expr @@ -139,11 +136,11 @@ func (rp *routePlan) addPredicate(predicates ...sqlparser.Expr) error { return vterrors.Errorf(vtrpcpb.Code_INTERNAL, "addPredicate should only be called when the route has a single table") } - if rp.vindexPlusPredicates == nil { - // Add all the column vindexes to the list of vindexPlusPredicates - for _, columnVindex := range rp.vtable.ColumnVindexes { - rp.vindexPlusPredicates = append(rp.vindexPlusPredicates, &vindexPlusPredicates{vindex: columnVindex}) - } + vindexPreds := []*vindexPlusPredicates{} + + // Add all the column vindexes to the list of vindexPlusPredicates + for _, columnVindex := range rp.vtable.ColumnVindexes { + vindexPreds = append(vindexPreds, &vindexPlusPredicates{vindex: columnVindex}) } for _, filter := range predicates { @@ -152,7 +149,7 @@ func (rp *routePlan) addPredicate(predicates ...sqlparser.Expr) error { switch node.Operator { case sqlparser.EqualOp: // TODO(Manan,Andres): Remove the predicates that are repeated eg. Id=1 AND Id=1 - for _, v := range rp.vindexPlusPredicates { + for _, v := range vindexPreds { column := node.Left.(*sqlparser.ColName) for _, col := range v.vindex.Columns { // If the column for the predicate matches any column in the vindex add it to the list @@ -168,7 +165,7 @@ func (rp *routePlan) addPredicate(predicates ...sqlparser.Expr) error { } //TODO (Manan,Andres): Improve cost metric for vindexes - for _, v := range rp.vindexPlusPredicates { + for _, v := range vindexPreds { if !v.covered { continue } @@ -280,8 +277,10 @@ func createJoin(lhs joinTree, rhs joinTree, joinPredicates []sqlparser.Expr) joi } /* - - */ + The greedy planner will plan a query by finding first finding the best route plan for every table, and then + finding the cheapest join, and using that. Then it searches for the next cheapest joinTree that can be produced, + and keeps doing this until all tables have been joined +*/ func greedySolve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVSchema) (joinTree, error) { plans := make([]joinTree, len(qg.tables)) planCache := map[semantics.TableSet]joinTree{} @@ -358,10 +357,10 @@ func createRoutePlan(table *queryTable, solves semantics.TableSet, vschema Conte plan.routeOpCode = engine.SelectEqualUnique default: plan.routeOpCode = engine.SelectScatter - err := plan.addPredicate(table.predicates...) - if err != nil { - return nil, err - } + } + err = plan.addPredicate(table.predicates...) + if err != nil { + return nil, err } return plan, nil @@ -464,12 +463,11 @@ func tryMerge(a, b joinTree, joinPredicates []sqlparser.Expr) joinTree { newTabletSet := aRoute.solved | bRoute.solved r := &routePlan{ - routeOpCode: aRoute.routeOpCode, - solved: newTabletSet, - tables: append(aRoute.tables, bRoute.tables...), - extraPredicates: append(aRoute.extraPredicates, bRoute.extraPredicates...), - keyspace: aRoute.keyspace, - vindexPlusPredicates: append(aRoute.vindexPlusPredicates, bRoute.vindexPlusPredicates...), + routeOpCode: aRoute.routeOpCode, + solved: newTabletSet, + tables: append(aRoute.tables, bRoute.tables...), + extraPredicates: append(aRoute.extraPredicates, bRoute.extraPredicates...), + keyspace: aRoute.keyspace, } r.extraPredicates = append(r.extraPredicates, joinPredicates...) diff --git a/go/vt/vtgate/planbuilder/route_planning_test.go b/go/vt/vtgate/planbuilder/route_planning_test.go new file mode 100644 index 00000000000..8b5473e0db7 --- /dev/null +++ b/go/vt/vtgate/planbuilder/route_planning_test.go @@ -0,0 +1,49 @@ +/* +Copyright 2020 The Vitess 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 planbuilder + +import ( + "testing" + + "github.com/stretchr/testify/assert" + "vitess.io/vitess/go/vt/sqlparser" + "vitess.io/vitess/go/vt/vtgate/engine" + "vitess.io/vitess/go/vt/vtgate/vindexes" +) + +func TestMergeUnshardedJoins(t *testing.T) { + ks := &vindexes.Keyspace{Name: "apa", Sharded: false} + ks2 := &vindexes.Keyspace{Name: "banan", Sharded: false} + r1 := &routePlan{ + routeOpCode: engine.SelectUnsharded, + solved: 1, + keyspace: ks, + } + r2 := &routePlan{ + routeOpCode: engine.SelectUnsharded, + solved: 2, + keyspace: ks, + } + r3 := &routePlan{ + routeOpCode: engine.SelectUnsharded, + solved: 4, + keyspace: ks2, + } + assert.NotNil(t, tryMerge(r1, r2, []sqlparser.Expr{})) + assert.Nil(t, tryMerge(r1, r3, []sqlparser.Expr{})) + assert.Nil(t, tryMerge(r2, r3, []sqlparser.Expr{})) +} From 63d4339711c3a5e7e2d44a90ef7dab738e494c15 Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Mon, 28 Dec 2020 10:41:23 +0100 Subject: [PATCH 16/40] added route planning unit tests Signed-off-by: Andres Taylor --- .../vtgate/planbuilder/route_planning_test.go | 79 +++++++++++++++---- 1 file changed, 62 insertions(+), 17 deletions(-) diff --git a/go/vt/vtgate/planbuilder/route_planning_test.go b/go/vt/vtgate/planbuilder/route_planning_test.go index 8b5473e0db7..e179c6b4b1a 100644 --- a/go/vt/vtgate/planbuilder/route_planning_test.go +++ b/go/vt/vtgate/planbuilder/route_planning_test.go @@ -17,33 +17,78 @@ limitations under the License. package planbuilder import ( + "fmt" "testing" + "vitess.io/vitess/go/vt/vtgate/semantics" + "github.com/stretchr/testify/assert" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vtgate/engine" "vitess.io/vitess/go/vt/vtgate/vindexes" ) -func TestMergeUnshardedJoins(t *testing.T) { - ks := &vindexes.Keyspace{Name: "apa", Sharded: false} - ks2 := &vindexes.Keyspace{Name: "banan", Sharded: false} - r1 := &routePlan{ +func unsharded(solved semantics.TableSet, keyspace *vindexes.Keyspace) *routePlan { + return &routePlan{ routeOpCode: engine.SelectUnsharded, - solved: 1, - keyspace: ks, + solved: solved, + keyspace: keyspace, } - r2 := &routePlan{ - routeOpCode: engine.SelectUnsharded, - solved: 2, - keyspace: ks, +} +func selectDBA(solved semantics.TableSet, keyspace *vindexes.Keyspace) *routePlan { + return &routePlan{ + routeOpCode: engine.SelectDBA, + solved: solved, + keyspace: keyspace, } - r3 := &routePlan{ - routeOpCode: engine.SelectUnsharded, - solved: 4, - keyspace: ks2, +} + +func TestMergeJoins(t *testing.T) { + ks := &vindexes.Keyspace{Name: "apa", Sharded: false} + ks2 := &vindexes.Keyspace{Name: "banan", Sharded: false} + + type testCase struct { + l, r, expected joinTree + predicates []sqlparser.Expr + } + + tests := []testCase{{ + l: unsharded(1, ks), + r: unsharded(2, ks), + expected: unsharded(1|2, ks), + }, { + l: unsharded(1, ks), + r: unsharded(2, ks2), + expected: nil, + }, { + l: unsharded(2, ks), + r: unsharded(1, ks2), + expected: nil, + }, { + l: selectDBA(1, ks), + r: selectDBA(2, ks), + expected: selectDBA(1|2, ks), + }, { + l: selectDBA(1, ks), + r: selectDBA(2, ks2), + expected: nil, + }, { + l: selectDBA(2, ks), + r: selectDBA(1, ks2), + expected: nil, + }, { + l: unsharded(1, ks), + r: selectDBA(2, ks), + expected: nil, + }, { + l: selectDBA(1, ks), + r: unsharded(2, ks), + expected: nil, + }} + for i, tc := range tests { + t.Run(fmt.Sprintf("%d", i), func(t *testing.T) { + result := tryMerge(tc.l, tc.r, tc.predicates) + assert.Equal(t, tc.expected, result) + }) } - assert.NotNil(t, tryMerge(r1, r2, []sqlparser.Expr{})) - assert.Nil(t, tryMerge(r1, r3, []sqlparser.Expr{})) - assert.Nil(t, tryMerge(r2, r3, []sqlparser.Expr{})) } From b1a5df83a35515fb29bd28e9d718dd914f8dc23b Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Mon, 28 Dec 2020 11:54:42 +0100 Subject: [PATCH 17/40] added flag to control the planner version Signed-off-by: Andres Taylor --- go/vt/vtgate/executor.go | 1 - go/vt/vtgate/planbuilder/builder.go | 14 +++++++++++++- go/vt/vtgate/planbuilder/plan_test.go | 10 +++++----- go/vt/vtgate/planbuilder/route_planning.go | 6 +++--- go/vt/vtgate/planbuilder/select.go | 2 +- go/vt/vtgate/vcursor_impl.go | 21 ++++++++++++++++++--- go/vt/vtgate/vtgate.go | 2 ++ 7 files changed, 42 insertions(+), 14 deletions(-) diff --git a/go/vt/vtgate/executor.go b/go/vt/vtgate/executor.go index 0c71cdfc2f5..acfe6b7ed77 100644 --- a/go/vt/vtgate/executor.go +++ b/go/vt/vtgate/executor.go @@ -107,7 +107,6 @@ type Executor struct { var executorOnce sync.Once const pathQueryPlans = "/debug/query_plans" - const pathScatterStats = "/debug/scatter_stats" const pathVSchema = "/debug/vschema" diff --git a/go/vt/vtgate/planbuilder/builder.go b/go/vt/vtgate/planbuilder/builder.go index 1f98569ef67..277f8d25a07 100644 --- a/go/vt/vtgate/planbuilder/builder.go +++ b/go/vt/vtgate/planbuilder/builder.go @@ -50,9 +50,21 @@ type ContextVSchema interface { KeyspaceExists(keyspace string) bool AllKeyspace() ([]*vindexes.Keyspace, error) GetSemTable() *semantics.SemTable - NewPlanner() bool + Planner() PlannerVersion } +// PlannerVersion is the version of the planner +type PlannerVersion int + +const ( + // V3 is also the default planner + V3 PlannerVersion = iota + // V4 is the new planner + V4 + // V4GreedyOnly uses only the faster greedy planner + V4GreedyOnly +) + type truncater interface { SetTruncateColumnCount(int) } diff --git a/go/vt/vtgate/planbuilder/plan_test.go b/go/vt/vtgate/planbuilder/plan_test.go index 5e84d65c76a..a3bb297eeaa 100644 --- a/go/vt/vtgate/planbuilder/plan_test.go +++ b/go/vt/vtgate/planbuilder/plan_test.go @@ -285,7 +285,7 @@ type vschemaWrapper struct { tabletType topodatapb.TabletType dest key.Destination sysVarEnabled bool - newPlanner bool + version PlannerVersion } func (vw *vschemaWrapper) AllKeyspace() ([]*vindexes.Keyspace, error) { @@ -295,8 +295,8 @@ func (vw *vschemaWrapper) AllKeyspace() ([]*vindexes.Keyspace, error) { return []*vindexes.Keyspace{vw.keyspace}, nil } -func (vw *vschemaWrapper) NewPlanner() bool { - return vw.newPlanner +func (vw *vschemaWrapper) Planner() PlannerVersion { + return vw.version } func (vw *vschemaWrapper) GetSemTable() *semantics.SemTable { return nil @@ -391,7 +391,7 @@ func testFile(t *testing.T, filename, tempDir string, vschema *vschemaWrapper) { fail := false for tcase := range iterateExecFile(filename) { t.Run(tcase.comments, func(t *testing.T) { - vschema.newPlanner = false + vschema.version = V3 plan, err := TestBuilder(tcase.input, vschema) out := getPlanOrErrorOutput(err, plan) @@ -414,7 +414,7 @@ func testFile(t *testing.T, filename, tempDir string, vschema *vschemaWrapper) { empty = true tcase.output2ndPlanner = tcase.output } - vschema.newPlanner = true + vschema.version = V4 out, err := getPlanOutput(tcase, vschema) if out != tcase.output2ndPlanner { fail = true diff --git a/go/vt/vtgate/planbuilder/route_planning.go b/go/vt/vtgate/planbuilder/route_planning.go index 841b8943a23..4bbb191e24c 100644 --- a/go/vt/vtgate/planbuilder/route_planning.go +++ b/go/vt/vtgate/planbuilder/route_planning.go @@ -43,10 +43,10 @@ func newBuildSelectPlan(sel *sqlparser.Select, vschema ContextVSchema) (engine.P } var tree joinTree - if len(qgraph.tables) <= 10 { - tree, err = dpSolve(qgraph, semTable, vschema) - } else { + if vschema.Planner() == V4GreedyOnly || len(qgraph.tables) > 10 { tree, err = greedySolve(qgraph, semTable, vschema) + } else { + tree, err = dpSolve(qgraph, semTable, vschema) } if err != nil { return nil, err diff --git a/go/vt/vtgate/planbuilder/select.go b/go/vt/vtgate/planbuilder/select.go index 5b1e020fb61..2f13184517d 100644 --- a/go/vt/vtgate/planbuilder/select.go +++ b/go/vt/vtgate/planbuilder/select.go @@ -39,7 +39,7 @@ func buildSelectPlan(query string) func(sqlparser.Statement, ContextVSchema) (en return func(stmt sqlparser.Statement, vschema ContextVSchema) (engine.Primitive, error) { sel := stmt.(*sqlparser.Select) - if vschema.NewPlanner() { + if vschema.Planner() != V3 { return newBuildSelectPlan(sel, vschema) } diff --git a/go/vt/vtgate/vcursor_impl.go b/go/vt/vtgate/vcursor_impl.go index 2e96fe80310..dd1f6d83712 100644 --- a/go/vt/vtgate/vcursor_impl.go +++ b/go/vt/vtgate/vcursor_impl.go @@ -23,6 +23,8 @@ import ( "sync/atomic" "time" + "github.com/prometheus/common/log" + "vitess.io/vitess/go/vt/vtgate/semantics" "golang.org/x/sync/errgroup" @@ -100,7 +102,6 @@ type vcursorImpl struct { vschema *vindexes.VSchema vm VSchemaOperator semTable *semantics.SemTable - newPlanner bool } func (vc *vcursorImpl) GetKeyspace() string { @@ -336,6 +337,7 @@ func (vc *vcursorImpl) KeyspaceExists(ks string) bool { return vc.vschema.Keyspaces[ks] != nil } +// AllKeyspace implements the ContextVSchema interface func (vc *vcursorImpl) AllKeyspace() ([]*vindexes.Keyspace, error) { if len(vc.vschema.Keyspaces) == 0 { return nil, vterrors.New(vtrpcpb.Code_FAILED_PRECONDITION, "no keyspaces available") @@ -347,9 +349,22 @@ func (vc *vcursorImpl) AllKeyspace() ([]*vindexes.Keyspace, error) { return kss, nil } -func (vc *vcursorImpl) NewPlanner() bool { - return vc.newPlanner +// Planner implements the ContextVSchema interface +func (vc *vcursorImpl) Planner() planbuilder.PlannerVersion { + switch strings.ToLower(*plannerVersion) { + case "v3": + return planbuilder.V3 + case "v4": + return planbuilder.V4 + case "v4greedy", "greedy": + return planbuilder.V4GreedyOnly + } + + log.Warn("unknown planner version configured. using the default") + return planbuilder.V3 } + +// GetSemTable implements the ContextVSchema interface func (vc *vcursorImpl) GetSemTable() *semantics.SemTable { return vc.semTable } diff --git a/go/vt/vtgate/vtgate.go b/go/vt/vtgate/vtgate.go index 84870af5339..feeb47ada3b 100644 --- a/go/vt/vtgate/vtgate.go +++ b/go/vt/vtgate/vtgate.go @@ -74,6 +74,8 @@ var ( // Put set-passthrough under a flag. sysVarSetEnabled = flag.Bool("enable_system_settings", true, "This will enable the system settings to be changed per session at the database connection level") + plannerVersion = flag.String("planner_version", "v3", "Sets the default planner to use when the session has not changed it. Valid values are: V3, V4 and V4Greedy. All V4 versions should be considered experimental!") + // lockHeartbeatTime is used to set the next heartbeat time. lockHeartbeatTime = flag.Duration("lock_heartbeat_time", 5*time.Second, "If there is lock function used. This will keep the lock connection active by using this heartbeat") ) From 4ecd166c7185733cc041b2ce9206a4e4449a3d9c Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Mon, 28 Dec 2020 12:16:21 +0100 Subject: [PATCH 18/40] added left to right planner Signed-off-by: Andres Taylor --- go/vt/vtgate/planbuilder/builder.go | 2 ++ go/vt/vtgate/planbuilder/route_planning.go | 35 ++++++++++++++++++++-- 2 files changed, 35 insertions(+), 2 deletions(-) diff --git a/go/vt/vtgate/planbuilder/builder.go b/go/vt/vtgate/planbuilder/builder.go index 277f8d25a07..ecc91c2022f 100644 --- a/go/vt/vtgate/planbuilder/builder.go +++ b/go/vt/vtgate/planbuilder/builder.go @@ -63,6 +63,8 @@ const ( V4 // V4GreedyOnly uses only the faster greedy planner V4GreedyOnly + // V4Left2Right tries to emulate the V3 planner by only joining plans in the order they are listed in the FROM-clause + V4Left2Right ) type truncater interface { diff --git a/go/vt/vtgate/planbuilder/route_planning.go b/go/vt/vtgate/planbuilder/route_planning.go index 4bbb191e24c..c0c24f445c2 100644 --- a/go/vt/vtgate/planbuilder/route_planning.go +++ b/go/vt/vtgate/planbuilder/route_planning.go @@ -43,11 +43,16 @@ func newBuildSelectPlan(sel *sqlparser.Select, vschema ContextVSchema) (engine.P } var tree joinTree - if vschema.Planner() == V4GreedyOnly || len(qgraph.tables) > 10 { + + switch { + case vschema.Planner() == V4GreedyOnly || len(qgraph.tables) > 10: tree, err = greedySolve(qgraph, semTable, vschema) - } else { + case vschema.Planner() == V4Left2Right: + tree, err = leftToRightSolve(qgraph, semTable, vschema) + default: tree, err = dpSolve(qgraph, semTable, vschema) } + if err != nil { return nil, err } @@ -325,6 +330,32 @@ func greedySolve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVS return plans[0], nil } +func leftToRightSolve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVSchema) (joinTree, error) { + plans := make([]joinTree, len(qg.tables)) + + // we start by seeding the table with the single routes + for i, table := range qg.tables { + solves := semTable.TableSetFor(table.alias) + plan, err := createRoutePlan(table, solves, vschema) + if err != nil { + return nil, err + } + plans[i] = plan + } + + var acc joinTree + for _, plan := range plans { + if acc == nil { + acc = plan + continue + } + solves := acc.solves() | plan.solves() + joinPredicates := qg.crossTable[solves] + acc = createJoin(acc, plan, joinPredicates) + } + + return acc, nil +} func removeAt(plans []joinTree, idx int) []joinTree { return append(plans[:idx], plans[idx+1:]...) From 33f5ca5048c036c8a2417892852450a60c5a1967 Mon Sep 17 00:00:00 2001 From: Harshit Gangal Date: Tue, 29 Dec 2020 11:34:13 +0530 Subject: [PATCH 19/40] add planner benchmark Signed-off-by: Harshit Gangal --- go/vt/vtgate/planbuilder/plan_test.go | 35 ++++++++++++++++++++++++++- 1 file changed, 34 insertions(+), 1 deletion(-) diff --git a/go/vt/vtgate/planbuilder/plan_test.go b/go/vt/vtgate/planbuilder/plan_test.go index a3bb297eeaa..c7393465481 100644 --- a/go/vt/vtgate/planbuilder/plan_test.go +++ b/go/vt/vtgate/planbuilder/plan_test.go @@ -260,7 +260,7 @@ func TestOtherPlanningFromFile(t *testing.T) { testFile(t, "other_admin_cases.txt", testOutputTempDir, vschema) } -func loadSchema(t *testing.T, filename string) *vindexes.VSchema { +func loadSchema(t testing.TB, filename string) *vindexes.VSchema { formal, err := vindexes.LoadFormal(locateFile(filename)) if err != nil { t.Fatal(err) @@ -576,3 +576,36 @@ func iterateExecFile(name string) (testCaseIterator chan testCase) { func locateFile(name string) string { return "testdata/" + name } + +func BenchmarkPlanner(b *testing.B) { + filenames := []string{"from_cases.txt", "filter_cases.txt", "aggr_cases.txt", "memory_sort_cases.txt", "select_cases.txt", "union_cases.txt", "wireup_cases.txt"} + vschema := &vschemaWrapper{ + v: loadSchema(b, "schema_test.json"), + sysVarEnabled: true, + } + for _, filename := range filenames { + b.Run(filename+"v3", func(b *testing.B) { + benchmarkPlanner(b, V3, filename, vschema) + }) + b.Run(filename+"v4", func(b *testing.B) { + benchmarkPlanner(b, V4, filename, vschema) + }) + b.Run(filename+"v4greedy", func(b *testing.B) { + benchmarkPlanner(b, V4GreedyOnly, filename, vschema) + }) + b.Run(filename+"v4left2right", func(b *testing.B) { + benchmarkPlanner(b, V4Left2Right, filename, vschema) + }) + } +} + +func benchmarkPlanner(b *testing.B, version PlannerVersion, filename string, vschema *vschemaWrapper) { + for n := 0; n < b.N; n++ { + for tcase := range iterateExecFile(filename) { + if tcase.output2ndPlanner != "" { + vschema.version = version + _, _ = TestBuilder(tcase.input, vschema) + } + } + } +} From 74e6d7ea729311aa6934606e6c313a3b618d3dc3 Mon Sep 17 00:00:00 2001 From: GuptaManan100 Date: Tue, 29 Dec 2020 13:23:08 +0530 Subject: [PATCH 20/40] Supported merging joins in selectScatter queries Signed-off-by: GuptaManan100 --- go/vt/vtgate/planbuilder/plan_test.go | 8 +- go/vt/vtgate/planbuilder/route_planning.go | 112 ++++++++++++++---- .../vtgate/planbuilder/route_planning_test.go | 17 ++- .../planbuilder/testdata/from_cases.txt | 45 +++++++ 4 files changed, 151 insertions(+), 31 deletions(-) diff --git a/go/vt/vtgate/planbuilder/plan_test.go b/go/vt/vtgate/planbuilder/plan_test.go index c7393465481..5c02c11d4cd 100644 --- a/go/vt/vtgate/planbuilder/plan_test.go +++ b/go/vt/vtgate/planbuilder/plan_test.go @@ -388,7 +388,7 @@ func testFile(t *testing.T, filename, tempDir string, vschema *vschemaWrapper) { var checkAllTests = false t.Run(filename, func(t *testing.T) { expected := &strings.Builder{} - fail := false + fail := checkAllTests for tcase := range iterateExecFile(filename) { t.Run(tcase.comments, func(t *testing.T) { vschema.version = V3 @@ -397,7 +397,7 @@ func testFile(t *testing.T, filename, tempDir string, vschema *vschemaWrapper) { if out != tcase.output { fail = true - t.Errorf("Legacy Planner - File: %s, Line: %d\nDiff:\n%s\n[%s] \n[%s]", filename, tcase.lineno, cmp.Diff(tcase.output, out), tcase.output, out) + t.Errorf("V3 - File: %s, Line: %d\nDiff:\n%s\n[%s] \n[%s]", filename, tcase.lineno, cmp.Diff(tcase.output, out), tcase.output, out) } if err != nil { out = `"` + out + `"` @@ -407,7 +407,7 @@ func testFile(t *testing.T, filename, tempDir string, vschema *vschemaWrapper) { }) if tcase.output2ndPlanner != "" || checkAllTests { - t.Run("New Planner: "+tcase.comments, func(t *testing.T) { + t.Run("V4: "+tcase.comments, func(t *testing.T) { expectedVal := "{\n}\n" empty := false if tcase.output2ndPlanner == "" { @@ -419,7 +419,7 @@ func testFile(t *testing.T, filename, tempDir string, vschema *vschemaWrapper) { if out != tcase.output2ndPlanner { fail = true expectedVal = "" - t.Errorf("New Planner - File: %s, Line: %d\nDiff:\n%s\n[%s] \n[%s]", filename, tcase.lineno, cmp.Diff(tcase.output2ndPlanner, out), tcase.output, out) + t.Errorf("V4 - File: %s, Line: %d\nDiff:\n%s\n[%s] \n[%s]", filename, tcase.lineno, cmp.Diff(tcase.output2ndPlanner, out), tcase.output, out) } if err != nil { out = `"` + out + `"` diff --git a/go/vt/vtgate/planbuilder/route_planning.go b/go/vt/vtgate/planbuilder/route_planning.go index c0c24f445c2..aef4574cfc0 100644 --- a/go/vt/vtgate/planbuilder/route_planning.go +++ b/go/vt/vtgate/planbuilder/route_planning.go @@ -101,18 +101,21 @@ type ( solves() semantics.TableSet cost() int } + routeTable struct { + qtable *queryTable + vtable *vindexes.Table + } routePlan struct { routeOpCode engine.RouteOpcode solved semantics.TableSet - tables []*queryTable + tables []*routeTable extraPredicates []sqlparser.Expr - keyspace *vindexes.Keyspace + keyspace *vindexes.Keyspace // vindex and conditions is set if a vindex will be used for this route. - vindex vindexes.Vindex - conditions []sqlparser.Expr + vindex vindexes.Vindex - vtable *vindexes.Table + conditions []sqlparser.Expr } joinPlan struct { predicates []sqlparser.Expr @@ -144,7 +147,7 @@ func (rp *routePlan) addPredicate(predicates ...sqlparser.Expr) error { vindexPreds := []*vindexPlusPredicates{} // Add all the column vindexes to the list of vindexPlusPredicates - for _, columnVindex := range rp.vtable.ColumnVindexes { + for _, columnVindex := range rp.tables[0].vtable.ColumnVindexes { vindexPreds = append(vindexPreds, &vindexPlusPredicates{vindex: columnVindex}) } @@ -204,7 +207,7 @@ func (rp *routePlan) Predicates() sqlparser.Expr { } } for _, t := range rp.tables { - for _, predicate := range t.predicates { + for _, predicate := range t.qtable.predicates { add(predicate) } } @@ -258,7 +261,7 @@ func dpSolve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVSchem continue } joinPredicates := qg.crossTable[solves] - newPlan := createJoin(lhs, rhs, joinPredicates) + newPlan := createJoin(lhs, rhs, joinPredicates, semTable) if oldPlan == nil || newPlan.cost() < oldPlan.cost() { dpTable.add(newPlan) } @@ -269,8 +272,8 @@ func dpSolve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVSchem return dpTable.planFor(allTables), nil } -func createJoin(lhs joinTree, rhs joinTree, joinPredicates []sqlparser.Expr) joinTree { - newPlan := tryMerge(lhs, rhs, joinPredicates) +func createJoin(lhs joinTree, rhs joinTree, joinPredicates []sqlparser.Expr, semTable *semantics.SemTable) joinTree { + newPlan := tryMerge(lhs, rhs, joinPredicates, semTable) if newPlan == nil { newPlan = &joinPlan{ lhs: lhs, @@ -311,7 +314,7 @@ func greedySolve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVS joinPredicates := qg.crossTable[solves] plan := planCache[solves] if plan == nil { - plan = createJoin(lhs, rhs, joinPredicates) + plan = createJoin(lhs, rhs, joinPredicates, semTable) planCache[solves] = plan } @@ -351,7 +354,7 @@ func leftToRightSolve(qg *queryGraph, semTable *semantics.SemTable, vschema Cont } solves := acc.solves() | plan.solves() joinPredicates := qg.crossTable[solves] - acc = createJoin(acc, plan, joinPredicates) + acc = createJoin(acc, plan, joinPredicates, semTable) } return acc, nil @@ -367,10 +370,12 @@ func createRoutePlan(table *queryTable, solves semantics.TableSet, vschema Conte return nil, err } plan := &routePlan{ - solved: solves, - tables: []*queryTable{table}, + solved: solves, + tables: []*routeTable{{ + qtable: table, + vtable: vschemaTable, + }}, keyspace: vschemaTable.Keyspace, - vtable: vschemaTable, } switch { @@ -404,8 +409,8 @@ func transformToLogicalPlan(tree joinTree) (logicalPlan, error) { tableNameMap := map[string]interface{}{} for _, t := range n.tables { - tablesForSelect = append(tablesForSelect, t.alias) - tableNameMap[sqlparser.String(t.alias.Expr)] = nil + tablesForSelect = append(tablesForSelect, t.qtable.alias) + tableNameMap[sqlparser.String(t.qtable.alias.Expr)] = nil } predicates := n.Predicates() var where *sqlparser.Where @@ -463,7 +468,61 @@ func transformToLogicalPlan(tree joinTree) (logicalPlan, error) { return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "BUG: unknown type encountered: %T", tree) } -func tryMerge(a, b joinTree, joinPredicates []sqlparser.Expr) joinTree { +func findColumnVindex(a *routePlan, exp sqlparser.Expr, sem *semantics.SemTable) vindexes.SingleColumn { + left, isCol := exp.(*sqlparser.ColName) + if !isCol { + return nil + } + leftDep := sem.Dependencies(left) + for _, table := range a.tables { + if semantics.IsContainedBy(table.qtable.tableID, leftDep) { + for _, vindex := range table.vtable.ColumnVindexes { + singCol, isSingle := vindex.Vindex.(vindexes.SingleColumn) + if isSingle && vindex.Columns[0].Equal(left.Name) { + return singCol + } + } + } + } + return nil +} + +func canMergeOnFilter(a, b *routePlan, predicate sqlparser.Expr, sem *semantics.SemTable) bool { + comparison, ok := predicate.(*sqlparser.ComparisonExpr) + if !ok { + return false + } + if comparison.Operator != sqlparser.EqualOp { + return false + } + left := comparison.Left + right := comparison.Right + + lVindex := findColumnVindex(a, left, sem) + if lVindex == nil { + left, right = right, left + lVindex = findColumnVindex(a, left, sem) + } + if lVindex == nil || !lVindex.IsUnique() { + return false + } + rVindex := findColumnVindex(b, right, sem) + if rVindex == nil { + return false + } + return rVindex == lVindex +} + +func canMergeScatter(a, b *routePlan, joinPredicates []sqlparser.Expr, semTable *semantics.SemTable) bool { + for _, predicate := range joinPredicates { + if canMergeOnFilter(a, b, predicate, semTable) { + return true + } + } + return false +} + +func tryMerge(a, b joinTree, joinPredicates []sqlparser.Expr, semTable *semantics.SemTable) joinTree { aRoute, ok := a.(*routePlan) if !ok { return nil @@ -484,12 +543,17 @@ func tryMerge(a, b joinTree, joinPredicates []sqlparser.Expr) joinTree { case engine.SelectEqualUnique: return nil case engine.SelectScatter: - //if len(joinPredicates) == 0 { - // If we are doing two Scatters, we have to make sure that the - // joins are on the correct vindex to allow them to be merged - // no join predicates - no vindex - return nil - //} + if len(joinPredicates) == 0 { + // If we are doing two Scatters, we have to make sure that the + // joins are on the correct vindex to allow them to be merged + // no join predicates - no vindex + return nil + } + + canMerge := canMergeScatter(aRoute, bRoute, joinPredicates, semTable) + if !canMerge { + return nil + } } newTabletSet := aRoute.solved | bRoute.solved diff --git a/go/vt/vtgate/planbuilder/route_planning_test.go b/go/vt/vtgate/planbuilder/route_planning_test.go index e179c6b4b1a..725d0760b59 100644 --- a/go/vt/vtgate/planbuilder/route_planning_test.go +++ b/go/vt/vtgate/planbuilder/route_planning_test.go @@ -43,6 +43,14 @@ func selectDBA(solved semantics.TableSet, keyspace *vindexes.Keyspace) *routePla } } +func selectScatter(solved semantics.TableSet, keyspace *vindexes.Keyspace) *routePlan { + return &routePlan{ + routeOpCode: engine.SelectScatter, + solved: solved, + keyspace: keyspace, + } +} + func TestMergeJoins(t *testing.T) { ks := &vindexes.Keyspace{Name: "apa", Sharded: false} ks2 := &vindexes.Keyspace{Name: "banan", Sharded: false} @@ -81,13 +89,16 @@ func TestMergeJoins(t *testing.T) { r: selectDBA(2, ks), expected: nil, }, { - l: selectDBA(1, ks), - r: unsharded(2, ks), + l: selectScatter(1, ks), + r: selectScatter(2, ks), + predicates: []sqlparser.Expr{ + equals(colName("t1", "id"), colName("t2", "id")), + }, expected: nil, }} for i, tc := range tests { t.Run(fmt.Sprintf("%d", i), func(t *testing.T) { - result := tryMerge(tc.l, tc.r, tc.predicates) + result := tryMerge(tc.l, tc.r, tc.predicates, &semantics.SemTable{}) assert.Equal(t, tc.expected, result) }) } diff --git a/go/vt/vtgate/planbuilder/testdata/from_cases.txt b/go/vt/vtgate/planbuilder/testdata/from_cases.txt index f0999b9d2b2..4b1b7b53fd6 100644 --- a/go/vt/vtgate/planbuilder/testdata/from_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/from_cases.txt @@ -761,6 +761,21 @@ "Table": "user" } } +{ + "QueryType": "SELECT", + "Original": "select user.col from user join user_extra on user.id = user_extra.user_id", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.col from user join user_extra on user.id = user_extra.user_id where 1 != 1", + "Query": "select user.col from user join user_extra on user.id = user_extra.user_id", + "Table": "user, user_extra" + } +} # mergeable sharded join on unique vindex (parenthesized ON clause) "select user.col from user join user_extra on (user.id = user_extra.user_id)" @@ -779,6 +794,21 @@ "Table": "user" } } +{ + "QueryType": "SELECT", + "Original": "select user.col from user join user_extra on (user.id = user_extra.user_id)", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.col from user join user_extra on user.id = user_extra.user_id where 1 != 1", + "Query": "select user.col from user join user_extra on user.id = user_extra.user_id", + "Table": "user, user_extra" + } +} # mergeable sharded join on unique vindex, with a stray condition "select user.col from user join user_extra on user.col between 1 and 2 and user.id = user_extra.user_id" @@ -815,6 +845,21 @@ "Table": "user" } } +{ + "QueryType": "SELECT", + "Original": "select user.col from user join user_extra on user_extra.user_id = user.id", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.col from user join user_extra on user_extra.user_id = user.id where 1 != 1", + "Query": "select user.col from user join user_extra on user_extra.user_id = user.id", + "Table": "user, user_extra" + } +} # mergeable sharded join on unique vindex, and condition "select user.col from user join user_extra on user.id = 5 and user.id = user_extra.user_id" From 4613d6cb6b1301b63a0a261c249d65c3fe5f2993 Mon Sep 17 00:00:00 2001 From: GuptaManan100 Date: Tue, 29 Dec 2020 16:42:02 +0530 Subject: [PATCH 21/40] pushed predicates to the correct place in join queries Signed-off-by: GuptaManan100 --- go/vt/sqlparser/ast_funcs.go | 8 ++ go/vt/vtgate/planbuilder/join2.go | 2 + go/vt/vtgate/planbuilder/jointab.go | 6 +- go/vt/vtgate/planbuilder/route_planning.go | 58 +++++++++- go/vt/vtgate/planbuilder/select.go | 73 +++++++++++++ .../planbuilder/testdata/from_cases.txt | 102 ++++++++++++++++++ 6 files changed, 240 insertions(+), 9 deletions(-) diff --git a/go/vt/sqlparser/ast_funcs.go b/go/vt/sqlparser/ast_funcs.go index 9a69e7ca451..18328b5ef4d 100644 --- a/go/vt/sqlparser/ast_funcs.go +++ b/go/vt/sqlparser/ast_funcs.go @@ -1226,6 +1226,14 @@ func (ty ShowCommandType) ToString() string { } } +// CompliantName is used to get the name of the bind variable to use for this column name +func (node *ColName) CompliantName(suffix string) string { + if !node.Qualifier.IsEmpty() { + return node.Qualifier.Name.CompliantName() + "_" + node.Name.CompliantName() + suffix + } + return node.Name.CompliantName() + suffix +} + // AtCount represents the '@' count in ColIdent type AtCount int diff --git a/go/vt/vtgate/planbuilder/join2.go b/go/vt/vtgate/planbuilder/join2.go index 7043257b6b2..c6a6d6ce788 100644 --- a/go/vt/vtgate/planbuilder/join2.go +++ b/go/vt/vtgate/planbuilder/join2.go @@ -31,6 +31,7 @@ type join2 struct { // Left and Right are the nodes for the join. Left, Right logicalPlan Cols []int + Vars map[string]int } // Order implements the logicalPlan interface @@ -83,6 +84,7 @@ func (j *join2) Primitive() engine.Primitive { Left: j.Left.Primitive(), Right: j.Right.Primitive(), Cols: j.Cols, + Vars: j.Vars, } } diff --git a/go/vt/vtgate/planbuilder/jointab.go b/go/vt/vtgate/planbuilder/jointab.go index 472ab4d0016..77d82a8525c 100644 --- a/go/vt/vtgate/planbuilder/jointab.go +++ b/go/vt/vtgate/planbuilder/jointab.go @@ -50,11 +50,7 @@ func (jt *jointab) Procure(plan logicalPlan, col *sqlparser.ColName, to int) str suffix := "" i := 0 for { - if !col.Qualifier.IsEmpty() { - joinVar = col.Qualifier.Name.CompliantName() + "_" + col.Name.CompliantName() + suffix - } else { - joinVar = col.Name.CompliantName() + suffix - } + joinVar = col.CompliantName(suffix) if _, ok := jt.vars[joinVar]; !ok { break } diff --git a/go/vt/vtgate/planbuilder/route_planning.go b/go/vt/vtgate/planbuilder/route_planning.go index aef4574cfc0..228ee00b756 100644 --- a/go/vt/vtgate/planbuilder/route_planning.go +++ b/go/vt/vtgate/planbuilder/route_planning.go @@ -17,6 +17,7 @@ limitations under the License. package planbuilder import ( + "sort" "strings" "vitess.io/vitess/go/sqltypes" @@ -57,7 +58,7 @@ func newBuildSelectPlan(sel *sqlparser.Select, vschema ContextVSchema) (engine.P return nil, err } - plan, err := transformToLogicalPlan(tree) + plan, err := transformToLogicalPlan(tree, semTable) if err != nil { return nil, err } @@ -402,7 +403,7 @@ func createRoutePlan(table *queryTable, solves semantics.TableSet, vschema Conte return plan, nil } -func transformToLogicalPlan(tree joinTree) (logicalPlan, error) { +func transformToLogicalPlan(tree joinTree, semTable *semantics.SemTable) (logicalPlan, error) { switch n := tree.(type) { case *routePlan: var tablesForSelect sqlparser.TableExprs @@ -434,6 +435,7 @@ func transformToLogicalPlan(tree joinTree) (logicalPlan, error) { for name := range tableNameMap { tableNames = append(tableNames, name) } + sort.Strings(tableNames) return &route{ eroute: &engine.Route{ @@ -451,17 +453,65 @@ func transformToLogicalPlan(tree joinTree) (logicalPlan, error) { }, nil case *joinPlan: - lhs, err := transformToLogicalPlan(n.lhs) + + lhsSolves := n.lhs.solves() + lhsColMap := map[*sqlparser.ColName]sqlparser.Argument{} + for _, predicate := range n.predicates { + sqlparser.Rewrite(predicate, func(cursor *sqlparser.Cursor) bool { + switch node := cursor.Node().(type) { + case *sqlparser.ColName: + if semantics.IsContainedBy(lhsSolves, semTable.Dependencies(node)) { + arg := sqlparser.NewArgument([]byte(":" + node.CompliantName(""))) + lhsColMap[node] = arg + cursor.Replace(arg) + } + } + return true + }, nil) + } + + var lhsColList []*sqlparser.ColName + for col := range lhsColMap { + lhsColList = append(lhsColList, col) + } + + var lhsColExpr []*sqlparser.AliasedExpr + for _, col := range lhsColList { + lhsColExpr = append(lhsColExpr, &sqlparser.AliasedExpr{ + Expr: col, + }) + } + + lhs, err := transformToLogicalPlan(n.lhs, semTable) if err != nil { return nil, err } - rhs, err := transformToLogicalPlan(n.rhs) + offset, err := pushProjection(lhsColExpr, lhs, semTable) if err != nil { return nil, err } + + vars := map[string]int{} + + for _, col := range lhsColList { + vars[col.CompliantName("")] = offset + offset++ + } + + rhs, err := transformToLogicalPlan(n.rhs, semTable) + if err != nil { + return nil, err + } + + err = pushPredicate(n.predicates, rhs, semTable) + if err != nil { + return nil, err + } + return &join2{ Left: lhs, Right: rhs, + Vars: vars, }, nil } diff --git a/go/vt/vtgate/planbuilder/select.go b/go/vt/vtgate/planbuilder/select.go index 2f13184517d..b14d3d5492c 100644 --- a/go/vt/vtgate/planbuilder/select.go +++ b/go/vt/vtgate/planbuilder/select.go @@ -116,6 +116,79 @@ func pushProjection(expr []*sqlparser.AliasedExpr, plan logicalPlan, semTable *s } } +func pushPredicate(exprs []sqlparser.Expr, plan logicalPlan, semTable *semantics.SemTable) (err error) { + if len(exprs) == 0 { + return nil + } + switch node := plan.(type) { + case *route: + sel := node.Select.(*sqlparser.Select) + finalExpr := reorderExpression(exprs[0], node.solvedTables, semTable) + for i, expr := range exprs { + if i == 0 { + continue + } + finalExpr = &sqlparser.AndExpr{ + Left: finalExpr, + Right: reorderExpression(expr, node.solvedTables, semTable), + } + } + if sel.Where != nil { + finalExpr = &sqlparser.AndExpr{ + Left: sel.Where.Expr, + Right: finalExpr, + } + } + sel.Where = &sqlparser.Where{ + Type: sqlparser.WhereClause, + Expr: finalExpr, + } + return nil + case *join2: + var lhs, rhs []sqlparser.Expr + lhsSolves := node.Left.Solves() + rhsSolves := node.Right.Solves() + for _, expr := range exprs { + deps := semTable.Dependencies(expr) + switch { + case semantics.IsContainedBy(deps, lhsSolves): + lhs = append(lhs, expr) + case semantics.IsContainedBy(deps, rhsSolves): + rhs = append(rhs, expr) + default: + return vterrors.Errorf(vtrpcpb.Code_INTERNAL, "unknown dependencies for %s", sqlparser.String(expr)) + } + } + err := pushPredicate(lhs, node.Left, semTable) + if err != nil { + return err + } + err = pushPredicate(rhs, node.Right, semTable) + return err + default: + return vterrors.Errorf(vtrpcpb.Code_INTERNAL, "not yet supported %T", node) + } +} + +func reorderExpression(expr sqlparser.Expr, solves semantics.TableSet, semTable *semantics.SemTable) sqlparser.Expr { + switch compExpr := expr.(type) { + case *sqlparser.ComparisonExpr: + if compExpr.Operator == sqlparser.EqualOp { + if !dependsOnRoute(solves, compExpr.Left, semTable) && dependsOnRoute(solves, compExpr.Right, semTable) { + compExpr.Left, compExpr.Right = compExpr.Right, compExpr.Left + } + } + } + return expr +} + +func dependsOnRoute(solves semantics.TableSet, expr sqlparser.Expr, semTable *semantics.SemTable) bool { + if node, ok := expr.(*sqlparser.ColName); ok { + return semantics.IsContainedBy(solves, semTable.Dependencies(node)) + } + return !sqlparser.IsValue(expr) +} + // processSelect builds a primitive tree for the given query or subquery. // The tree built by this function has the following general structure: // diff --git a/go/vt/vtgate/planbuilder/testdata/from_cases.txt b/go/vt/vtgate/planbuilder/testdata/from_cases.txt index 4b1b7b53fd6..24eb7841a02 100644 --- a/go/vt/vtgate/planbuilder/testdata/from_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/from_cases.txt @@ -631,6 +631,40 @@ ] } } +{ + "QueryType": "SELECT", + "Original": "select user.col from user join unsharded as m1 join unsharded as m2", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "1", + "TableName": "unsharded_user", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select 1 from unsharded as m1, unsharded as m2 where 1 != 1", + "Query": "select 1 from unsharded as m1, unsharded as m2", + "Table": "unsharded" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.col from user where 1 != 1", + "Query": "select user.col from user", + "Table": "user" + } + ] + } +} # Parenthesized, single chunk "select user.col from user join (unsharded as m1 join unsharded as m2)" @@ -919,6 +953,40 @@ ] } } +{ + "QueryType": "SELECT", + "Original": "select user.col from user join user_extra on user.id \u003c user_extra.user_id", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "-2", + "TableName": "user_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.id, user.col from user where 1 != 1", + "Query": "select user.id, user.col from user", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from user_extra where 1 != 1", + "Query": "select 1 from user_extra where :user_id \u003c user_extra.user_id", + "Table": "user_extra" + } + ] + } +} # sharded join, non-col reference RHS "select user.col from user join user_extra on user.id = 5" @@ -1038,6 +1106,40 @@ ] } } +{ + "QueryType": "SELECT", + "Original": "select user.col from user join user_extra on user.id = user_extra.col", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "-2", + "TableName": "user_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.id, user.col from user where 1 != 1", + "Query": "select user.id, user.col from user", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from user_extra where 1 != 1", + "Query": "select 1 from user_extra where user_extra.col = :user_id", + "Table": "user_extra" + } + ] + } +} # sharded join, non-unique vindex "select user.col from user_extra join user on user_extra.user_id = user.name" From c7aecd39b2f8c76f04c70361e61caede4013100a Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Tue, 29 Dec 2020 14:54:02 +0100 Subject: [PATCH 22/40] added vtgate flag and system variable to control the planner used Signed-off-by: Andres Taylor --- go/vt/proto/query/query.pb.go | 456 +++++++++++++---------- go/vt/sqlparser/ast_rewriting.go | 1 + go/vt/sqlparser/ast_rewriting_test.go | 7 +- go/vt/sysvars/sysvars.go | 1 + go/vt/vtgate/engine/fake_vcursor_test.go | 8 + go/vt/vtgate/engine/primitive.go | 1 + go/vt/vtgate/executor.go | 6 + go/vt/vtgate/planbuilder/builder.go | 12 +- go/vt/vtgate/vcursor_impl.go | 11 + proto/query.proto | 12 + 10 files changed, 305 insertions(+), 210 deletions(-) diff --git a/go/vt/proto/query/query.pb.go b/go/vt/proto/query/query.pb.go index 2cc6e6680ed..b6d553da9e9 100644 --- a/go/vt/proto/query/query.pb.go +++ b/go/vt/proto/query/query.pb.go @@ -462,6 +462,40 @@ func (ExecuteOptions_TransactionIsolation) EnumDescriptor() ([]byte, []int) { return fileDescriptor_5c6ac9b241082464, []int{6, 2} } +type ExecuteOptions_PlannerVersion int32 + +const ( + ExecuteOptions_DEFAULT_PLANNER ExecuteOptions_PlannerVersion = 0 + ExecuteOptions_V3 ExecuteOptions_PlannerVersion = 1 + ExecuteOptions_V4 ExecuteOptions_PlannerVersion = 2 + ExecuteOptions_V4Greedy ExecuteOptions_PlannerVersion = 3 + ExecuteOptions_V4Left2Right ExecuteOptions_PlannerVersion = 4 +) + +var ExecuteOptions_PlannerVersion_name = map[int32]string{ + 0: "DEFAULT_PLANNER", + 1: "V3", + 2: "V4", + 3: "V4Greedy", + 4: "V4Left2Right", +} + +var ExecuteOptions_PlannerVersion_value = map[string]int32{ + "DEFAULT_PLANNER": 0, + "V3": 1, + "V4": 2, + "V4Greedy": 3, + "V4Left2Right": 4, +} + +func (x ExecuteOptions_PlannerVersion) String() string { + return proto.EnumName(ExecuteOptions_PlannerVersion_name, int32(x)) +} + +func (ExecuteOptions_PlannerVersion) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_5c6ac9b241082464, []int{6, 3} +} + // The category of one statement. type StreamEvent_Statement_Category int32 @@ -857,10 +891,13 @@ type ExecuteOptions struct { TransactionIsolation ExecuteOptions_TransactionIsolation `protobuf:"varint,9,opt,name=transaction_isolation,json=transactionIsolation,proto3,enum=query.ExecuteOptions_TransactionIsolation" json:"transaction_isolation,omitempty"` // skip_query_plan_cache specifies if the query plan should be cached by vitess. // By default all query plans are cached. - SkipQueryPlanCache bool `protobuf:"varint,10,opt,name=skip_query_plan_cache,json=skipQueryPlanCache,proto3" json:"skip_query_plan_cache,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + SkipQueryPlanCache bool `protobuf:"varint,10,opt,name=skip_query_plan_cache,json=skipQueryPlanCache,proto3" json:"skip_query_plan_cache,omitempty"` + // PlannerVersion specifies which planner to use. + // If DEFAULT is chosen, whatever vtgate was started with will be used + PlannerVersion ExecuteOptions_PlannerVersion `protobuf:"varint,11,opt,name=planner_version,json=plannerVersion,proto3,enum=query.ExecuteOptions_PlannerVersion" json:"planner_version,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *ExecuteOptions) Reset() { *m = ExecuteOptions{} } @@ -930,6 +967,13 @@ func (m *ExecuteOptions) GetSkipQueryPlanCache() bool { return false } +func (m *ExecuteOptions) GetPlannerVersion() ExecuteOptions_PlannerVersion { + if m != nil { + return m.PlannerVersion + } + return ExecuteOptions_DEFAULT_PLANNER +} + // Field describes a single column returned by a query type Field struct { // name of the field as returned by mysql C API @@ -4257,6 +4301,7 @@ func init() { proto.RegisterEnum("query.ExecuteOptions_IncludedFields", ExecuteOptions_IncludedFields_name, ExecuteOptions_IncludedFields_value) proto.RegisterEnum("query.ExecuteOptions_Workload", ExecuteOptions_Workload_name, ExecuteOptions_Workload_value) proto.RegisterEnum("query.ExecuteOptions_TransactionIsolation", ExecuteOptions_TransactionIsolation_name, ExecuteOptions_TransactionIsolation_value) + proto.RegisterEnum("query.ExecuteOptions_PlannerVersion", ExecuteOptions_PlannerVersion_name, ExecuteOptions_PlannerVersion_value) proto.RegisterEnum("query.StreamEvent_Statement_Category", StreamEvent_Statement_Category_name, StreamEvent_Statement_Category_value) proto.RegisterType((*Target)(nil), "query.Target") proto.RegisterType((*VTGateCallerID)(nil), "query.VTGateCallerID") @@ -4325,203 +4370,208 @@ func init() { func init() { proto.RegisterFile("query.proto", fileDescriptor_5c6ac9b241082464) } var fileDescriptor_5c6ac9b241082464 = []byte{ - // 3158 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x5a, 0x4b, 0x70, 0x1b, 0x47, - 0x7a, 0xd6, 0xe0, 0x45, 0xe0, 0x07, 0x01, 0x36, 0x9b, 0xa4, 0x04, 0x51, 0x7e, 0xd0, 0x63, 0xcb, - 0x66, 0x98, 0x84, 0x92, 0x28, 0x59, 0x51, 0x6c, 0x27, 0xd1, 0x10, 0x1c, 0xca, 0x90, 0x80, 0x01, - 0xd4, 0x18, 0x48, 0x96, 0x2a, 0x55, 0x53, 0x43, 0xa0, 0x05, 0x4e, 0x71, 0x80, 0x81, 0x66, 0x86, - 0x94, 0x78, 0x53, 0xe2, 0x38, 0xce, 0x3b, 0xce, 0x3b, 0x8e, 0x2b, 0xae, 0x54, 0xe5, 0x90, 0xca, - 0x65, 0xcf, 0x3e, 0xef, 0xc1, 0x87, 0x3d, 0x6c, 0xd5, 0x1e, 0x77, 0xf7, 0xb0, 0xbb, 0x87, 0xad, - 0xdd, 0x93, 0x6b, 0x6b, 0x0f, 0x7b, 0xd8, 0xc3, 0xd6, 0x56, 0x3f, 0x66, 0x00, 0x90, 0xb0, 0x44, - 0xcb, 0xeb, 0xda, 0x92, 0xac, 0x5b, 0xff, 0x8f, 0x7e, 0x7c, 0x5f, 0xff, 0xf3, 0xf7, 0x63, 0x1a, - 0xf2, 0x77, 0x77, 0xa9, 0xbf, 0xbf, 0x3a, 0xf0, 0xbd, 0xd0, 0xc3, 0x69, 0x2e, 0x2c, 0x16, 0x43, - 0x6f, 0xe0, 0x75, 0xec, 0xd0, 0x16, 0xea, 0xc5, 0xfc, 0x5e, 0xe8, 0x0f, 0xda, 0x42, 0x50, 0xdf, - 0x53, 0x20, 0x63, 0xda, 0x7e, 0x97, 0x86, 0x78, 0x11, 0xb2, 0x3b, 0x74, 0x3f, 0x18, 0xd8, 0x6d, - 0x5a, 0x52, 0x96, 0x94, 0xe5, 0x1c, 0x89, 0x65, 0x3c, 0x0f, 0xe9, 0x60, 0xdb, 0xf6, 0x3b, 0xa5, - 0x04, 0x37, 0x08, 0x01, 0xbf, 0x0e, 0xf9, 0xd0, 0xde, 0x72, 0x69, 0x68, 0x85, 0xfb, 0x03, 0x5a, - 0x4a, 0x2e, 0x29, 0xcb, 0xc5, 0xb5, 0xf9, 0xd5, 0xb8, 0x3f, 0x93, 0x1b, 0xcd, 0xfd, 0x01, 0x25, - 0x10, 0xc6, 0x65, 0x8c, 0x21, 0xd5, 0xa6, 0xae, 0x5b, 0x4a, 0xf1, 0xb6, 0x78, 0x59, 0xdd, 0x80, - 0xe2, 0x0d, 0xf3, 0x8a, 0x1d, 0xd2, 0xb2, 0xed, 0xba, 0xd4, 0xaf, 0x6c, 0xb0, 0xe1, 0xec, 0x06, - 0xd4, 0xef, 0xdb, 0xbd, 0x78, 0x38, 0x91, 0x8c, 0x8f, 0x43, 0xa6, 0xeb, 0x7b, 0xbb, 0x83, 0xa0, - 0x94, 0x58, 0x4a, 0x2e, 0xe7, 0x88, 0x94, 0xd4, 0x3f, 0x05, 0xd0, 0xf7, 0x68, 0x3f, 0x34, 0xbd, - 0x1d, 0xda, 0xc7, 0xcf, 0x41, 0x2e, 0x74, 0x7a, 0x34, 0x08, 0xed, 0xde, 0x80, 0x37, 0x91, 0x24, - 0x43, 0xc5, 0xe7, 0x40, 0x5a, 0x84, 0xec, 0xc0, 0x0b, 0x9c, 0xd0, 0xf1, 0xfa, 0x1c, 0x4f, 0x8e, - 0xc4, 0xb2, 0xfa, 0xc7, 0x90, 0xbe, 0x61, 0xbb, 0xbb, 0x14, 0xbf, 0x08, 0x29, 0x0e, 0x58, 0xe1, - 0x80, 0xf3, 0xab, 0x82, 0x74, 0x8e, 0x93, 0x1b, 0x58, 0xdb, 0x7b, 0xcc, 0x93, 0xb7, 0x3d, 0x4d, - 0x84, 0xa0, 0xee, 0xc0, 0xf4, 0xba, 0xd3, 0xef, 0xdc, 0xb0, 0x7d, 0x87, 0x91, 0xf1, 0x98, 0xcd, - 0xe0, 0x57, 0x20, 0xc3, 0x0b, 0x41, 0x29, 0xb9, 0x94, 0x5c, 0xce, 0xaf, 0x4d, 0xcb, 0x8a, 0x7c, - 0x6c, 0x44, 0xda, 0xd4, 0x6f, 0x2a, 0x00, 0xeb, 0xde, 0x6e, 0xbf, 0x73, 0x9d, 0x19, 0x31, 0x82, - 0x64, 0x70, 0xd7, 0x95, 0x44, 0xb2, 0x22, 0xbe, 0x06, 0xc5, 0x2d, 0xa7, 0xdf, 0xb1, 0xf6, 0xe4, - 0x70, 0x04, 0x97, 0xf9, 0xb5, 0x57, 0x64, 0x73, 0xc3, 0xca, 0xab, 0xa3, 0xa3, 0x0e, 0xf4, 0x7e, - 0xe8, 0xef, 0x93, 0xc2, 0xd6, 0xa8, 0x6e, 0xb1, 0x05, 0xf8, 0xb0, 0x13, 0xeb, 0x74, 0x87, 0xee, - 0x47, 0x9d, 0xee, 0xd0, 0x7d, 0xfc, 0x3b, 0xa3, 0x88, 0xf2, 0x6b, 0x73, 0x51, 0x5f, 0x23, 0x75, - 0x25, 0xcc, 0x37, 0x12, 0x97, 0x14, 0xf5, 0x93, 0x34, 0x14, 0xf5, 0xfb, 0xb4, 0xbd, 0x1b, 0xd2, - 0xfa, 0x80, 0xcd, 0x41, 0x80, 0x6b, 0x30, 0xe3, 0xf4, 0xdb, 0xee, 0x6e, 0x87, 0x76, 0xac, 0x3b, - 0x0e, 0x75, 0x3b, 0x01, 0x8f, 0xa3, 0x62, 0x3c, 0xee, 0x71, 0xff, 0xd5, 0x8a, 0x74, 0xde, 0xe4, - 0xbe, 0xa4, 0xe8, 0x8c, 0xc9, 0x78, 0x05, 0x66, 0xdb, 0xae, 0x43, 0xfb, 0xa1, 0x75, 0x87, 0xe1, - 0xb5, 0x7c, 0xef, 0x5e, 0x50, 0x4a, 0x2f, 0x29, 0xcb, 0x59, 0x32, 0x23, 0x0c, 0x9b, 0x4c, 0x4f, - 0xbc, 0x7b, 0x01, 0x7e, 0x03, 0xb2, 0xf7, 0x3c, 0x7f, 0xc7, 0xf5, 0xec, 0x4e, 0x29, 0xc3, 0xfb, - 0x7c, 0x61, 0x72, 0x9f, 0x37, 0xa5, 0x17, 0x89, 0xfd, 0xf1, 0x32, 0xa0, 0xe0, 0xae, 0x6b, 0x05, - 0xd4, 0xa5, 0xed, 0xd0, 0x72, 0x9d, 0x9e, 0x13, 0x96, 0xb2, 0x3c, 0x24, 0x8b, 0xc1, 0x5d, 0xb7, - 0xc9, 0xd5, 0x55, 0xa6, 0xc5, 0x16, 0x2c, 0x84, 0xbe, 0xdd, 0x0f, 0xec, 0x36, 0x6b, 0xcc, 0x72, - 0x02, 0xcf, 0xb5, 0x79, 0x38, 0xe6, 0x78, 0x97, 0x2b, 0x93, 0xbb, 0x34, 0x87, 0x55, 0x2a, 0x51, - 0x0d, 0x32, 0x1f, 0x4e, 0xd0, 0xe2, 0x73, 0xb0, 0x10, 0xec, 0x38, 0x03, 0x8b, 0xb7, 0x63, 0x0d, - 0x5c, 0xbb, 0x6f, 0xb5, 0xed, 0xf6, 0x36, 0x2d, 0x01, 0x87, 0x8d, 0x99, 0x91, 0xcf, 0x7b, 0xc3, - 0xb5, 0xfb, 0x65, 0x66, 0x51, 0xdf, 0x84, 0xe2, 0x38, 0x8f, 0x78, 0x16, 0x0a, 0xe6, 0xad, 0x86, - 0x6e, 0x69, 0xc6, 0x86, 0x65, 0x68, 0x35, 0x1d, 0x1d, 0xc3, 0x05, 0xc8, 0x71, 0x55, 0xdd, 0xa8, - 0xde, 0x42, 0x0a, 0x9e, 0x82, 0xa4, 0x56, 0xad, 0xa2, 0x84, 0x7a, 0x09, 0xb2, 0x11, 0x21, 0x78, - 0x06, 0xf2, 0x2d, 0xa3, 0xd9, 0xd0, 0xcb, 0x95, 0xcd, 0x8a, 0xbe, 0x81, 0x8e, 0xe1, 0x2c, 0xa4, - 0xea, 0x55, 0xb3, 0x81, 0x14, 0x51, 0xd2, 0x1a, 0x28, 0xc1, 0x6a, 0x6e, 0xac, 0x6b, 0x28, 0xa9, - 0xfe, 0x9f, 0x02, 0xf3, 0x93, 0x80, 0xe1, 0x3c, 0x4c, 0x6d, 0xe8, 0x9b, 0x5a, 0xab, 0x6a, 0xa2, - 0x63, 0x78, 0x0e, 0x66, 0x88, 0xde, 0xd0, 0x35, 0x53, 0x5b, 0xaf, 0xea, 0x16, 0xd1, 0xb5, 0x0d, - 0xa4, 0x60, 0x0c, 0x45, 0x56, 0xb2, 0xca, 0xf5, 0x5a, 0xad, 0x62, 0x9a, 0xfa, 0x06, 0x4a, 0xe0, - 0x79, 0x40, 0x5c, 0xd7, 0x32, 0x86, 0xda, 0x24, 0x46, 0x30, 0xdd, 0xd4, 0x49, 0x45, 0xab, 0x56, - 0x6e, 0xb3, 0x06, 0x50, 0x0a, 0xbf, 0x04, 0xcf, 0x97, 0xeb, 0x46, 0xb3, 0xd2, 0x34, 0x75, 0xc3, - 0xb4, 0x9a, 0x86, 0xd6, 0x68, 0xbe, 0x5d, 0x37, 0x79, 0xcb, 0x02, 0x5c, 0x1a, 0x17, 0x01, 0xb4, - 0x96, 0x59, 0x17, 0xed, 0xa0, 0xcc, 0xd5, 0x54, 0x56, 0x41, 0x89, 0xab, 0xa9, 0x6c, 0x02, 0x25, - 0xaf, 0xa6, 0xb2, 0x49, 0x94, 0x52, 0x3f, 0x49, 0x40, 0x9a, 0x73, 0xc5, 0xd2, 0xdd, 0x48, 0x12, - 0xe3, 0xe5, 0xf8, 0xd3, 0x4f, 0x3c, 0xe4, 0xd3, 0xe7, 0x19, 0x53, 0x26, 0x21, 0x21, 0xe0, 0x53, - 0x90, 0xf3, 0xfc, 0xae, 0x25, 0x2c, 0x22, 0x7d, 0x66, 0x3d, 0xbf, 0xcb, 0xf3, 0x2c, 0x4b, 0x5d, - 0x2c, 0xeb, 0x6e, 0xd9, 0x01, 0xe5, 0x11, 0x9c, 0x23, 0xb1, 0x8c, 0x4f, 0x02, 0xf3, 0xb3, 0xf8, - 0x38, 0x32, 0xdc, 0x36, 0xe5, 0xf9, 0x5d, 0x83, 0x0d, 0xe5, 0x65, 0x28, 0xb4, 0x3d, 0x77, 0xb7, - 0xd7, 0xb7, 0x5c, 0xda, 0xef, 0x86, 0xdb, 0xa5, 0xa9, 0x25, 0x65, 0xb9, 0x40, 0xa6, 0x85, 0xb2, - 0xca, 0x75, 0xb8, 0x04, 0x53, 0xed, 0x6d, 0xdb, 0x0f, 0xa8, 0x88, 0xda, 0x02, 0x89, 0x44, 0xde, - 0x2b, 0x6d, 0x3b, 0x3d, 0xdb, 0x0d, 0x78, 0x84, 0x16, 0x48, 0x2c, 0x33, 0x10, 0x77, 0x5c, 0xbb, - 0x1b, 0xf0, 0xc8, 0x2a, 0x10, 0x21, 0xe0, 0x17, 0x21, 0x2f, 0x3b, 0xe4, 0x14, 0xe4, 0xf9, 0x70, - 0x40, 0xa8, 0x18, 0x03, 0xea, 0x1f, 0x40, 0x92, 0x78, 0xf7, 0x58, 0x9f, 0x62, 0x44, 0x41, 0x49, - 0x59, 0x4a, 0x2e, 0x63, 0x12, 0x89, 0x2c, 0xfd, 0xcb, 0x0c, 0x28, 0x12, 0x63, 0x94, 0xf3, 0x3e, - 0x52, 0x20, 0xcf, 0x23, 0x97, 0xd0, 0x60, 0xd7, 0x0d, 0x59, 0xa6, 0x94, 0x29, 0x42, 0x19, 0xcb, - 0x94, 0x7c, 0x5e, 0x88, 0xb4, 0x31, 0x02, 0xd8, 0x57, 0x6f, 0xd9, 0x77, 0xee, 0xd0, 0x76, 0x48, - 0xc5, 0x82, 0x90, 0x22, 0xd3, 0x4c, 0xa9, 0x49, 0x1d, 0x63, 0xde, 0xe9, 0x07, 0xd4, 0x0f, 0x2d, - 0xa7, 0xc3, 0xe7, 0x24, 0x45, 0xb2, 0x42, 0x51, 0xe9, 0xe0, 0x17, 0x20, 0xc5, 0xf3, 0x46, 0x8a, - 0xf7, 0x02, 0xb2, 0x17, 0xe2, 0xdd, 0x23, 0x5c, 0x7f, 0x35, 0x95, 0x4d, 0xa3, 0x8c, 0xfa, 0x16, - 0x4c, 0xf3, 0xc1, 0xdd, 0xb4, 0xfd, 0xbe, 0xd3, 0xef, 0xf2, 0x65, 0xd0, 0xeb, 0x88, 0xb8, 0x28, - 0x10, 0x5e, 0x66, 0x98, 0x7b, 0x34, 0x08, 0xec, 0x2e, 0x95, 0xcb, 0x52, 0x24, 0xaa, 0xff, 0x93, - 0x84, 0x7c, 0x33, 0xf4, 0xa9, 0xdd, 0xe3, 0x2b, 0x1c, 0x7e, 0x0b, 0x20, 0x08, 0xed, 0x90, 0xf6, - 0x68, 0x3f, 0x8c, 0xf0, 0x3d, 0x27, 0x7b, 0x1e, 0xf1, 0x5b, 0x6d, 0x46, 0x4e, 0x64, 0xc4, 0x1f, - 0xaf, 0x41, 0x9e, 0x32, 0xb3, 0x15, 0xb2, 0x95, 0x52, 0x66, 0xe3, 0xd9, 0x28, 0xb5, 0xc4, 0x4b, - 0x28, 0x01, 0x1a, 0x97, 0x17, 0x3f, 0x4e, 0x40, 0x2e, 0x6e, 0x0d, 0x6b, 0x90, 0x6d, 0xdb, 0x21, - 0xed, 0x7a, 0xfe, 0xbe, 0x5c, 0xc0, 0x4e, 0x3f, 0xac, 0xf7, 0xd5, 0xb2, 0x74, 0x26, 0x71, 0x35, - 0xfc, 0x3c, 0x88, 0x5d, 0x81, 0x08, 0x4b, 0x81, 0x37, 0xc7, 0x35, 0x3c, 0x30, 0xdf, 0x00, 0x3c, - 0xf0, 0x9d, 0x9e, 0xed, 0xef, 0x5b, 0x3b, 0x74, 0x3f, 0x4a, 0xf6, 0xc9, 0x09, 0x33, 0x89, 0xa4, - 0xdf, 0x35, 0xba, 0x2f, 0xd3, 0xd3, 0xa5, 0xf1, 0xba, 0x32, 0x5a, 0x0e, 0xcf, 0xcf, 0x48, 0x4d, - 0xbe, 0x7c, 0x06, 0xd1, 0x42, 0x99, 0xe6, 0x81, 0xc5, 0x8a, 0xea, 0x6b, 0x90, 0x8d, 0x06, 0x8f, - 0x73, 0x90, 0xd6, 0x7d, 0xdf, 0xf3, 0xd1, 0x31, 0x9e, 0xa5, 0x6a, 0x55, 0x91, 0xe8, 0x36, 0x36, - 0x58, 0xa2, 0xfb, 0x51, 0x22, 0x5e, 0xad, 0x08, 0xbd, 0xbb, 0x4b, 0x83, 0x10, 0xff, 0x09, 0xcc, - 0x51, 0x1e, 0x42, 0xce, 0x1e, 0xb5, 0xda, 0x7c, 0x6b, 0xc3, 0x02, 0x48, 0xe1, 0x7c, 0xcf, 0xac, - 0x8a, 0x9d, 0x58, 0xb4, 0xe5, 0x21, 0xb3, 0xb1, 0xaf, 0x54, 0x75, 0xb0, 0x0e, 0x73, 0x4e, 0xaf, - 0x47, 0x3b, 0x8e, 0x1d, 0x8e, 0x36, 0x20, 0x26, 0x6c, 0x21, 0x5a, 0xf9, 0xc7, 0x76, 0x4e, 0x64, - 0x36, 0xae, 0x11, 0x37, 0x73, 0x1a, 0x32, 0x21, 0xdf, 0xe5, 0xf1, 0xd8, 0xcd, 0xaf, 0x15, 0xa2, - 0x8c, 0xc3, 0x95, 0x44, 0x1a, 0xf1, 0x6b, 0x20, 0xf6, 0x8c, 0x3c, 0xb7, 0x0c, 0x03, 0x62, 0xb8, - 0x15, 0x20, 0xc2, 0x8e, 0x4f, 0x43, 0x71, 0x6c, 0x91, 0xea, 0x70, 0xc2, 0x92, 0xa4, 0x30, 0xba, - 0xe2, 0x74, 0xf0, 0x19, 0x98, 0xf2, 0xc4, 0x02, 0xc5, 0xb3, 0xce, 0x70, 0xc4, 0xe3, 0xab, 0x17, - 0x89, 0xbc, 0x58, 0x6e, 0xf0, 0x69, 0x40, 0xfd, 0x3d, 0xda, 0x61, 0x8d, 0x4e, 0xf1, 0x46, 0x21, - 0x52, 0x55, 0x3a, 0xea, 0x1f, 0xc1, 0x4c, 0x4c, 0x71, 0x30, 0xf0, 0xfa, 0x01, 0xc5, 0x2b, 0x90, - 0xf1, 0xf9, 0xf7, 0x2e, 0x69, 0xc5, 0xb2, 0x8f, 0x91, 0x4c, 0x40, 0xa4, 0x87, 0xda, 0x81, 0x19, - 0xa1, 0xb9, 0xe9, 0x84, 0xdb, 0x7c, 0x26, 0xf1, 0x69, 0x48, 0x53, 0x56, 0x38, 0x30, 0x29, 0xa4, - 0x51, 0xe6, 0x76, 0x22, 0xac, 0x23, 0xbd, 0x24, 0x1e, 0xd9, 0xcb, 0xcf, 0x12, 0x30, 0x27, 0x47, - 0xb9, 0x6e, 0x87, 0xed, 0xed, 0x27, 0x34, 0x1a, 0x7e, 0x17, 0xa6, 0x98, 0xde, 0x89, 0xbf, 0x9c, - 0x09, 0xf1, 0x10, 0x79, 0xb0, 0x88, 0xb0, 0x03, 0x6b, 0x64, 0xfa, 0xe5, 0x2e, 0xaa, 0x60, 0x07, - 0x23, 0x4b, 0xf8, 0x84, 0xc0, 0xc9, 0x3c, 0x22, 0x70, 0xa6, 0x8e, 0x12, 0x38, 0xea, 0x06, 0xcc, - 0x8f, 0x33, 0x2e, 0x83, 0xe3, 0xf7, 0x60, 0x4a, 0x4c, 0x4a, 0x94, 0x23, 0x27, 0xcd, 0x5b, 0xe4, - 0xa2, 0x7e, 0x9a, 0x80, 0x79, 0x99, 0xbe, 0xbe, 0x1e, 0xdf, 0xf1, 0x08, 0xcf, 0xe9, 0x23, 0x7d, - 0xa0, 0x47, 0x9b, 0x3f, 0xb5, 0x0c, 0x0b, 0x07, 0x78, 0x7c, 0x8c, 0x8f, 0xf5, 0x33, 0x05, 0xa6, - 0xd7, 0x69, 0xd7, 0xe9, 0x3f, 0xa1, 0xb3, 0x30, 0x42, 0x6e, 0xea, 0x48, 0x41, 0x3c, 0x80, 0x82, - 0xc4, 0x2b, 0xd9, 0x3a, 0xcc, 0xb6, 0x32, 0xe9, 0x6b, 0xb9, 0x04, 0xd3, 0xf2, 0x1c, 0x6e, 0xbb, - 0x8e, 0x1d, 0xc4, 0x78, 0x0e, 0x1c, 0xc4, 0x35, 0x66, 0x24, 0xf2, 0xc8, 0xce, 0x05, 0xf5, 0xc7, - 0x0a, 0x14, 0xca, 0x5e, 0xaf, 0xe7, 0x84, 0x4f, 0x28, 0xc7, 0x87, 0x19, 0x4a, 0x4d, 0x8a, 0xc7, - 0x73, 0x50, 0x8c, 0x60, 0x4a, 0x6a, 0x0f, 0xac, 0x34, 0xca, 0xa1, 0x95, 0xe6, 0x27, 0x0a, 0xcc, - 0x10, 0xcf, 0x75, 0xb7, 0xec, 0xf6, 0xce, 0xd3, 0x4d, 0xce, 0x79, 0x40, 0x43, 0xa0, 0x47, 0xa5, - 0xe7, 0x97, 0x0a, 0x14, 0x1b, 0x3e, 0x1d, 0xd8, 0x3e, 0x7d, 0xaa, 0xd9, 0x61, 0xdb, 0xf4, 0x4e, - 0x28, 0x37, 0x38, 0x39, 0xc2, 0xcb, 0xea, 0x2c, 0xcc, 0xc4, 0xd8, 0x05, 0x61, 0xea, 0xf7, 0x14, - 0x58, 0x10, 0x21, 0x26, 0x2d, 0x9d, 0x27, 0x94, 0x96, 0x08, 0x6f, 0x6a, 0x04, 0x6f, 0x09, 0x8e, - 0x1f, 0xc4, 0x26, 0x61, 0xbf, 0x9b, 0x80, 0x13, 0x51, 0xf0, 0x3c, 0xe1, 0xc0, 0xbf, 0x44, 0x3c, - 0x2c, 0x42, 0xe9, 0x30, 0x09, 0x92, 0xa1, 0x0f, 0x12, 0x50, 0x2a, 0xfb, 0xd4, 0x0e, 0xe9, 0xc8, - 0x3e, 0xe8, 0xe9, 0x89, 0x0d, 0x7c, 0x0e, 0xa6, 0x07, 0xb6, 0x1f, 0x3a, 0x6d, 0x67, 0x60, 0xb3, - 0xa3, 0x68, 0x9a, 0x6f, 0xb3, 0x0e, 0x34, 0x30, 0xe6, 0xa2, 0x9e, 0x82, 0x93, 0x13, 0x18, 0x91, - 0x7c, 0xfd, 0x4a, 0x01, 0xdc, 0x0c, 0x6d, 0x3f, 0xfc, 0x1a, 0xac, 0x4b, 0x13, 0x83, 0x69, 0x01, - 0xe6, 0xc6, 0xf0, 0x8f, 0xf2, 0x42, 0xc3, 0xaf, 0xc5, 0x92, 0xf4, 0xb9, 0xbc, 0x8c, 0xe2, 0x97, - 0xbc, 0xfc, 0x40, 0x81, 0xc5, 0xb2, 0x27, 0x6e, 0x27, 0x9f, 0xca, 0x2f, 0x4c, 0x7d, 0x1e, 0x4e, - 0x4d, 0x04, 0x28, 0x09, 0xf8, 0xbe, 0x02, 0xc7, 0x09, 0xb5, 0x3b, 0x4f, 0x27, 0xf8, 0xeb, 0x70, - 0xe2, 0x10, 0x38, 0xb9, 0x47, 0xb9, 0x08, 0xd9, 0x1e, 0x0d, 0x6d, 0xb6, 0xc3, 0x95, 0x90, 0x16, - 0xa3, 0x76, 0x87, 0xde, 0x35, 0xe9, 0x41, 0x62, 0x5f, 0xf5, 0x87, 0x09, 0x98, 0xe3, 0xfb, 0xec, - 0x67, 0x87, 0xbc, 0x23, 0xdd, 0xc2, 0x64, 0x0e, 0x6e, 0xfe, 0x98, 0xc3, 0xc0, 0xa7, 0x56, 0x74, - 0x3b, 0x30, 0xc5, 0x7f, 0xc2, 0xc1, 0xc0, 0xa7, 0xd7, 0x85, 0x46, 0xfd, 0x96, 0x02, 0xf3, 0xe3, - 0x14, 0xc7, 0x27, 0x9a, 0xdf, 0xf4, 0x6d, 0xcb, 0x84, 0x94, 0x92, 0x3c, 0xca, 0x21, 0x29, 0x75, - 0xe4, 0x43, 0xd2, 0xb7, 0x13, 0x50, 0x1a, 0x05, 0xf3, 0xec, 0x4e, 0x67, 0xfc, 0x4e, 0xe7, 0x8b, - 0xde, 0xf2, 0xa9, 0xdf, 0x51, 0xe0, 0xe4, 0x04, 0x42, 0xbf, 0x58, 0x88, 0x8c, 0xdc, 0xec, 0x24, - 0x1e, 0x79, 0xb3, 0xf3, 0xd5, 0x07, 0xc9, 0x77, 0x15, 0x98, 0xaf, 0x89, 0xbb, 0x7a, 0x71, 0xf3, - 0xf1, 0xe4, 0xe6, 0x60, 0x7e, 0x1d, 0x9f, 0x1a, 0xfe, 0xad, 0x52, 0xcb, 0xb0, 0x70, 0x00, 0xda, - 0x63, 0xdc, 0xe6, 0xfc, 0x42, 0x81, 0x59, 0xd9, 0x8a, 0xf6, 0xc4, 0x6e, 0x5f, 0x26, 0xb0, 0x83, - 0x5f, 0x80, 0xa4, 0xd3, 0x89, 0xf6, 0xbd, 0xe3, 0x3f, 0xe3, 0x99, 0x41, 0xbd, 0x0c, 0x78, 0x14, - 0xf7, 0x63, 0x50, 0xf7, 0xd3, 0x04, 0x2c, 0x10, 0x91, 0x7d, 0x9f, 0xfd, 0x5f, 0xf8, 0xb2, 0xff, - 0x17, 0x1e, 0xbe, 0x70, 0x7d, 0xca, 0x37, 0x53, 0xe3, 0x54, 0x7f, 0x75, 0x4b, 0xd7, 0x81, 0x85, - 0x36, 0x79, 0x68, 0xa1, 0x7d, 0xfc, 0x7c, 0xf4, 0x69, 0x02, 0x16, 0x25, 0x90, 0x67, 0x7b, 0x9d, - 0xa3, 0x47, 0x44, 0xe6, 0x50, 0x44, 0xfc, 0x5c, 0x81, 0x53, 0x13, 0x89, 0xfc, 0xad, 0xef, 0x68, - 0x0e, 0x44, 0x4f, 0xea, 0x91, 0xd1, 0x93, 0x3e, 0x72, 0xf4, 0xbc, 0x9f, 0x80, 0x22, 0xa1, 0x2e, - 0xb5, 0x83, 0xa7, 0xfc, 0x76, 0xef, 0x00, 0x87, 0xe9, 0x43, 0xf7, 0x9c, 0xb3, 0x30, 0x13, 0x13, - 0x21, 0x0f, 0x5c, 0xfc, 0x80, 0xce, 0xd6, 0xc1, 0xb7, 0xa9, 0xed, 0x86, 0xd1, 0x4e, 0x50, 0xfd, - 0xdf, 0x04, 0x14, 0x08, 0xd3, 0x38, 0x3d, 0xda, 0x0c, 0xed, 0x30, 0xc0, 0x2f, 0xc1, 0xf4, 0x36, - 0x77, 0xb1, 0x86, 0x11, 0x92, 0x23, 0x79, 0xa1, 0x13, 0x7f, 0x1f, 0xd7, 0x60, 0x21, 0xa0, 0x6d, - 0xaf, 0xdf, 0x09, 0xac, 0x2d, 0xba, 0xed, 0xf4, 0x3b, 0x56, 0xcf, 0x0e, 0x42, 0xea, 0x73, 0x5a, - 0x0a, 0x64, 0x4e, 0x1a, 0xd7, 0xb9, 0xad, 0xc6, 0x4d, 0xf8, 0x2c, 0xcc, 0x6f, 0x39, 0x7d, 0xd7, - 0xeb, 0x5a, 0x03, 0xd7, 0xde, 0xa7, 0x7e, 0x60, 0xb5, 0xbd, 0xdd, 0xbe, 0xe0, 0x23, 0x4d, 0xb0, - 0xb0, 0x35, 0x84, 0xa9, 0xcc, 0x2c, 0xf8, 0x36, 0xac, 0x4c, 0xec, 0xc5, 0xba, 0xe3, 0xb8, 0x21, - 0xf5, 0x69, 0xc7, 0xf2, 0xe9, 0xc0, 0x75, 0xda, 0xe2, 0xa1, 0x91, 0x20, 0xea, 0xd5, 0x09, 0x5d, - 0x6f, 0x4a, 0x77, 0x32, 0xf4, 0xc6, 0xa7, 0x20, 0xd7, 0x1e, 0xec, 0x5a, 0xbb, 0xfc, 0xd1, 0x02, - 0xe3, 0x4f, 0x21, 0xd9, 0xf6, 0x60, 0xb7, 0xc5, 0x64, 0x8c, 0x20, 0x79, 0x77, 0x20, 0x92, 0xb3, - 0x42, 0x58, 0x51, 0xfd, 0x4c, 0x81, 0xa2, 0xd6, 0xed, 0xfa, 0xb4, 0x6b, 0x87, 0x92, 0xa6, 0xb3, - 0x30, 0x2f, 0x28, 0xd9, 0xb7, 0x64, 0xb8, 0x0a, 0x3c, 0x8a, 0xc0, 0x23, 0x6d, 0x22, 0x56, 0x05, - 0x9e, 0x0b, 0x70, 0x7c, 0xb7, 0x3f, 0xb1, 0x4e, 0x82, 0xd7, 0x99, 0x8f, 0xad, 0xa3, 0xb5, 0xfe, - 0x10, 0x4e, 0x4e, 0x66, 0xa1, 0xe7, 0x88, 0xc7, 0x7e, 0x05, 0x72, 0x7c, 0x02, 0xe8, 0x9a, 0xd3, - 0x7f, 0x48, 0x55, 0xfb, 0x3e, 0xe7, 0xeb, 0x73, 0xaa, 0xda, 0xf7, 0xd5, 0xff, 0x8f, 0xff, 0x29, - 0x46, 0xe1, 0x12, 0x27, 0x8e, 0x28, 0x90, 0x95, 0x87, 0x05, 0x72, 0x09, 0xa6, 0x58, 0x30, 0x3a, - 0xfd, 0x2e, 0x07, 0x97, 0x25, 0x91, 0x88, 0x9b, 0xf0, 0xaa, 0xc4, 0x4e, 0xef, 0x87, 0xd4, 0xef, - 0xdb, 0xae, 0xbb, 0x6f, 0x89, 0xeb, 0xc7, 0x7e, 0x48, 0x3b, 0xd6, 0xf0, 0xf1, 0xa3, 0x48, 0x1f, - 0x2f, 0x0b, 0x6f, 0x3d, 0x76, 0x26, 0xb1, 0xaf, 0x19, 0x3f, 0x8b, 0x7c, 0x13, 0x8a, 0xbe, 0x0c, - 0x62, 0x2b, 0x60, 0xd3, 0x23, 0x53, 0xee, 0x7c, 0xf4, 0x6a, 0x62, 0x34, 0xc2, 0x49, 0xc1, 0x1f, - 0x0b, 0xf8, 0xc7, 0x4e, 0x38, 0x57, 0x53, 0xd9, 0x0c, 0x9a, 0x52, 0xbf, 0xa1, 0xc0, 0xdc, 0x84, - 0xb3, 0x7b, 0x7c, 0x31, 0xa0, 0x8c, 0xdc, 0x3b, 0xfe, 0x3e, 0xa4, 0xf9, 0x83, 0x16, 0xf9, 0x86, - 0xea, 0xc4, 0xe1, 0xa3, 0x3f, 0x7f, 0x7c, 0x42, 0x84, 0x17, 0xfb, 0x16, 0x39, 0xa6, 0x36, 0xbf, - 0x78, 0x8c, 0x32, 0x6a, 0x9e, 0xe9, 0xc4, 0x5d, 0xe4, 0xe1, 0x9b, 0xcc, 0xd4, 0x23, 0x6f, 0x32, - 0x57, 0xfe, 0x39, 0x09, 0xb9, 0xda, 0x7e, 0xf3, 0xae, 0xbb, 0xe9, 0xda, 0x5d, 0xfe, 0x3a, 0xa4, - 0xd6, 0x30, 0x6f, 0xa1, 0x63, 0x78, 0x16, 0x0a, 0x46, 0xdd, 0xb4, 0x8c, 0x56, 0xb5, 0x6a, 0x6d, - 0x56, 0xb5, 0x2b, 0x48, 0xc1, 0x08, 0xa6, 0x1b, 0xa4, 0x62, 0x5d, 0xd3, 0x6f, 0x09, 0x4d, 0x02, - 0xcf, 0xc1, 0x4c, 0xcb, 0xa8, 0x5c, 0x6f, 0xe9, 0x43, 0x65, 0x0a, 0x2f, 0xc0, 0x6c, 0xad, 0x55, - 0x35, 0x2b, 0x8d, 0xea, 0x88, 0x3a, 0x8b, 0x0b, 0x90, 0x5b, 0xaf, 0xd6, 0xd7, 0x85, 0x88, 0x58, - 0xfb, 0x2d, 0xa3, 0x59, 0xb9, 0x62, 0xe8, 0x1b, 0x42, 0xb5, 0xc4, 0x54, 0xb7, 0x75, 0x52, 0xdf, - 0xac, 0x44, 0x5d, 0x5e, 0xc6, 0x08, 0xf2, 0xeb, 0x15, 0x43, 0x23, 0xb2, 0x95, 0x07, 0x0a, 0x2e, - 0x42, 0x4e, 0x37, 0x5a, 0x35, 0x29, 0x27, 0x70, 0x09, 0xe6, 0xb4, 0x96, 0x59, 0xb7, 0x2a, 0x46, - 0x99, 0xe8, 0x35, 0xdd, 0x30, 0xa5, 0x25, 0x85, 0xe7, 0xa0, 0x68, 0x56, 0x6a, 0x7a, 0xd3, 0xd4, - 0x6a, 0x0d, 0xa9, 0x64, 0xa3, 0xc8, 0x36, 0xf5, 0xc8, 0x07, 0xe1, 0x45, 0x58, 0x30, 0xea, 0x96, - 0x7c, 0x8a, 0x67, 0xdd, 0xd0, 0xaa, 0x2d, 0x5d, 0xda, 0x96, 0xf0, 0x09, 0xc0, 0x75, 0xc3, 0x6a, - 0x35, 0x36, 0x34, 0x53, 0xb7, 0x8c, 0xfa, 0x4d, 0x69, 0xb8, 0x8c, 0x8b, 0x90, 0x1d, 0x8e, 0xe0, - 0x01, 0x63, 0xa1, 0xd0, 0xd0, 0x88, 0x39, 0x04, 0xfb, 0xe0, 0x01, 0x23, 0x0b, 0xae, 0x90, 0x7a, - 0xab, 0x31, 0x74, 0x9b, 0x85, 0xbc, 0x24, 0x4b, 0xaa, 0x52, 0x4c, 0xb5, 0x5e, 0x31, 0xca, 0xf1, - 0xf8, 0x1e, 0x64, 0x17, 0x13, 0x48, 0x59, 0xd9, 0x81, 0x14, 0x9f, 0x8e, 0x2c, 0xa4, 0x8c, 0xba, - 0xa1, 0xa3, 0x63, 0x78, 0x06, 0xa0, 0xd2, 0xac, 0x18, 0xa6, 0x7e, 0x85, 0x68, 0x55, 0x06, 0x9b, - 0x2b, 0x22, 0x02, 0x19, 0xda, 0x69, 0x98, 0xaa, 0x34, 0x37, 0xab, 0x75, 0xcd, 0x94, 0x30, 0x2b, - 0xcd, 0xeb, 0xad, 0xba, 0xc9, 0x8c, 0x08, 0xe7, 0x21, 0x53, 0x69, 0x9a, 0xfa, 0x3b, 0x26, 0xc3, - 0xc5, 0x6d, 0x82, 0x55, 0xf4, 0xe0, 0xf2, 0xca, 0x87, 0x49, 0x48, 0xf1, 0x57, 0xcd, 0x05, 0xc8, - 0xf1, 0xd9, 0x36, 0x6f, 0x35, 0x58, 0x97, 0x39, 0x48, 0x55, 0x0c, 0xf3, 0x12, 0xfa, 0xb3, 0x04, - 0x06, 0x48, 0xb7, 0x78, 0xf9, 0xcf, 0x33, 0xac, 0x5c, 0x31, 0xcc, 0x73, 0x17, 0xd1, 0xbb, 0x09, - 0xd6, 0x6c, 0x4b, 0x08, 0x7f, 0x11, 0x19, 0xd6, 0x2e, 0xa0, 0xf7, 0x62, 0xc3, 0xda, 0x05, 0xf4, - 0x97, 0x91, 0xe1, 0xfc, 0x1a, 0x7a, 0x3f, 0x36, 0x9c, 0x5f, 0x43, 0x7f, 0x15, 0x19, 0x2e, 0x5e, - 0x40, 0x7f, 0x1d, 0x1b, 0x2e, 0x5e, 0x40, 0x7f, 0x93, 0x61, 0x58, 0x38, 0x92, 0xf3, 0x6b, 0xe8, - 0x6f, 0xb3, 0xb1, 0x74, 0xf1, 0x02, 0xfa, 0xbb, 0x2c, 0x9b, 0xff, 0x78, 0x56, 0xd1, 0xdf, 0x23, - 0x36, 0x4c, 0x36, 0x41, 0xe8, 0x1f, 0x78, 0x91, 0x99, 0xd0, 0x3f, 0x22, 0x86, 0x91, 0x69, 0xb9, - 0xf8, 0x01, 0xb7, 0xdc, 0xd2, 0x35, 0x82, 0xfe, 0x29, 0x23, 0x5e, 0x5e, 0x96, 0x2b, 0x35, 0xad, - 0x8a, 0x30, 0xaf, 0xc1, 0x58, 0xf9, 0x97, 0xb3, 0xac, 0xc8, 0xc2, 0x13, 0xfd, 0x6b, 0x83, 0x75, - 0x78, 0x43, 0x23, 0xe5, 0xb7, 0x35, 0x82, 0xfe, 0xed, 0x2c, 0xeb, 0xf0, 0x86, 0x46, 0x24, 0x5f, - 0xff, 0xde, 0x60, 0x8e, 0xdc, 0xf4, 0x1f, 0x67, 0xd9, 0xa0, 0xa5, 0xfe, 0x3f, 0x1b, 0x38, 0x0b, - 0xc9, 0xf5, 0x8a, 0x89, 0x3e, 0xe4, 0xbd, 0xb1, 0x10, 0x45, 0xff, 0x85, 0x98, 0xb2, 0xa9, 0x9b, - 0xe8, 0x23, 0xa6, 0x4c, 0x9b, 0xad, 0x46, 0x55, 0x47, 0xcf, 0xb1, 0xc1, 0x5d, 0xd1, 0xeb, 0x35, - 0xdd, 0x24, 0xb7, 0xd0, 0x7f, 0x73, 0xf7, 0xab, 0xcd, 0xba, 0x81, 0x3e, 0x46, 0xb8, 0x08, 0xa0, - 0xbf, 0xd3, 0x20, 0x7a, 0xb3, 0x59, 0xa9, 0x1b, 0xe8, 0xc5, 0x95, 0x4d, 0x40, 0x07, 0xd3, 0x01, - 0x03, 0xd0, 0x32, 0xae, 0x19, 0xf5, 0x9b, 0x06, 0x3a, 0xc6, 0x84, 0x06, 0xd1, 0x1b, 0x1a, 0xd1, - 0x91, 0x82, 0x01, 0x32, 0xf2, 0x3d, 0x67, 0x02, 0x4f, 0x43, 0x96, 0xd4, 0xab, 0xd5, 0x75, 0xad, - 0x7c, 0x0d, 0x25, 0xd7, 0x5f, 0x87, 0x19, 0xc7, 0x5b, 0xdd, 0x73, 0x42, 0x1a, 0x04, 0xe2, 0xdd, - 0xfc, 0x6d, 0x55, 0x4a, 0x8e, 0x77, 0x46, 0x94, 0xce, 0x74, 0xbd, 0x33, 0x7b, 0xe1, 0x19, 0x6e, - 0x3d, 0xc3, 0x33, 0xc6, 0x56, 0x86, 0x0b, 0xe7, 0x7f, 0x1d, 0x00, 0x00, 0xff, 0xff, 0xd7, 0x1b, - 0xa7, 0xfb, 0x95, 0x2f, 0x00, 0x00, + // 3235 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x5a, 0x4b, 0x70, 0x1b, 0x5b, + 0x5a, 0x76, 0xb7, 0x1e, 0x96, 0x7e, 0x59, 0xf2, 0xf1, 0xb1, 0x9d, 0xab, 0xeb, 0xdc, 0x87, 0xa7, + 0x67, 0x32, 0x63, 0x0c, 0x38, 0x89, 0xe3, 0x09, 0xe1, 0xce, 0x00, 0x69, 0xcb, 0xed, 0x5c, 0x25, + 0x52, 0x4b, 0x39, 0x6a, 0x39, 0x93, 0x14, 0x55, 0x5d, 0x6d, 0xe9, 0x44, 0xee, 0x72, 0x4b, 0xad, + 0x74, 0xb7, 0x9d, 0x68, 0x17, 0x18, 0x86, 0xe1, 0xcd, 0xf0, 0x66, 0x98, 0x62, 0x8a, 0x2a, 0x16, + 0x14, 0x1b, 0xd6, 0xac, 0x59, 0xdc, 0x05, 0x0b, 0xaa, 0x58, 0x02, 0x55, 0x3c, 0x16, 0x14, 0xac, + 0xa6, 0x28, 0x16, 0x2c, 0x58, 0x50, 0xd4, 0x79, 0x74, 0x4b, 0xb2, 0x75, 0x13, 0x4f, 0x86, 0x29, + 0x2a, 0xb9, 0x59, 0xe9, 0xfc, 0x8f, 0xf3, 0xf8, 0xbf, 0xf3, 0x9f, 0xff, 0x3f, 0xfd, 0xeb, 0x40, + 0xe1, 0xc9, 0x09, 0x0d, 0x46, 0x5b, 0xc3, 0xc0, 0x8f, 0x7c, 0x9c, 0xe1, 0xc4, 0x5a, 0x29, 0xf2, + 0x87, 0x7e, 0xd7, 0x89, 0x1c, 0xc1, 0x5e, 0x2b, 0x9c, 0x46, 0xc1, 0xb0, 0x23, 0x08, 0xed, 0x1b, + 0x0a, 0x64, 0x2d, 0x27, 0xe8, 0xd1, 0x08, 0xaf, 0x41, 0xee, 0x98, 0x8e, 0xc2, 0xa1, 0xd3, 0xa1, + 0x65, 0x65, 0x5d, 0xd9, 0xc8, 0x93, 0x84, 0xc6, 0x2b, 0x90, 0x09, 0x8f, 0x9c, 0xa0, 0x5b, 0x56, + 0xb9, 0x40, 0x10, 0xf8, 0xcb, 0x50, 0x88, 0x9c, 0x43, 0x8f, 0x46, 0x76, 0x34, 0x1a, 0xd2, 0x72, + 0x6a, 0x5d, 0xd9, 0x28, 0x6d, 0xaf, 0x6c, 0x25, 0xf3, 0x59, 0x5c, 0x68, 0x8d, 0x86, 0x94, 0x40, + 0x94, 0xb4, 0x31, 0x86, 0x74, 0x87, 0x7a, 0x5e, 0x39, 0xcd, 0xc7, 0xe2, 0x6d, 0x6d, 0x0f, 0x4a, + 0x07, 0xd6, 0x1d, 0x27, 0xa2, 0x15, 0xc7, 0xf3, 0x68, 0x50, 0xdd, 0x63, 0xcb, 0x39, 0x09, 0x69, + 0x30, 0x70, 0xfa, 0xc9, 0x72, 0x62, 0x1a, 0x5f, 0x82, 0x6c, 0x2f, 0xf0, 0x4f, 0x86, 0x61, 0x59, + 0x5d, 0x4f, 0x6d, 0xe4, 0x89, 0xa4, 0xb4, 0x9f, 0x05, 0x30, 0x4e, 0xe9, 0x20, 0xb2, 0xfc, 0x63, + 0x3a, 0xc0, 0xef, 0x41, 0x3e, 0x72, 0xfb, 0x34, 0x8c, 0x9c, 0xfe, 0x90, 0x0f, 0x91, 0x22, 0x63, + 0xc6, 0xa7, 0x98, 0xb4, 0x06, 0xb9, 0xa1, 0x1f, 0xba, 0x91, 0xeb, 0x0f, 0xb8, 0x3d, 0x79, 0x92, + 0xd0, 0xda, 0x4f, 0x43, 0xe6, 0xc0, 0xf1, 0x4e, 0x28, 0xfe, 0x10, 0xd2, 0xdc, 0x60, 0x85, 0x1b, + 0x5c, 0xd8, 0x12, 0xa0, 0x73, 0x3b, 0xb9, 0x80, 0x8d, 0x7d, 0xca, 0x34, 0xf9, 0xd8, 0x0b, 0x44, + 0x10, 0xda, 0x31, 0x2c, 0xec, 0xba, 0x83, 0xee, 0x81, 0x13, 0xb8, 0x0c, 0x8c, 0x57, 0x1c, 0x06, + 0x7f, 0x01, 0xb2, 0xbc, 0x11, 0x96, 0x53, 0xeb, 0xa9, 0x8d, 0xc2, 0xf6, 0x82, 0xec, 0xc8, 0xd7, + 0x46, 0xa4, 0x4c, 0xfb, 0x2b, 0x05, 0x60, 0xd7, 0x3f, 0x19, 0x74, 0xef, 0x33, 0x21, 0x46, 0x90, + 0x0a, 0x9f, 0x78, 0x12, 0x48, 0xd6, 0xc4, 0xf7, 0xa0, 0x74, 0xe8, 0x0e, 0xba, 0xf6, 0xa9, 0x5c, + 0x8e, 0xc0, 0xb2, 0xb0, 0xfd, 0x05, 0x39, 0xdc, 0xb8, 0xf3, 0xd6, 0xe4, 0xaa, 0x43, 0x63, 0x10, + 0x05, 0x23, 0x52, 0x3c, 0x9c, 0xe4, 0xad, 0xb5, 0x01, 0x9f, 0x57, 0x62, 0x93, 0x1e, 0xd3, 0x51, + 0x3c, 0xe9, 0x31, 0x1d, 0xe1, 0x1f, 0x99, 0xb4, 0xa8, 0xb0, 0xbd, 0x1c, 0xcf, 0x35, 0xd1, 0x57, + 0x9a, 0xf9, 0x91, 0x7a, 0x4b, 0xd1, 0xfe, 0x31, 0x0b, 0x25, 0xe3, 0x19, 0xed, 0x9c, 0x44, 0xb4, + 0x31, 0x64, 0x7b, 0x10, 0xe2, 0x3a, 0x2c, 0xba, 0x83, 0x8e, 0x77, 0xd2, 0xa5, 0x5d, 0xfb, 0xb1, + 0x4b, 0xbd, 0x6e, 0xc8, 0xfd, 0xa8, 0x94, 0xac, 0x7b, 0x5a, 0x7f, 0xab, 0x2a, 0x95, 0xf7, 0xb9, + 0x2e, 0x29, 0xb9, 0x53, 0x34, 0xde, 0x84, 0xa5, 0x8e, 0xe7, 0xd2, 0x41, 0x64, 0x3f, 0x66, 0xf6, + 0xda, 0x81, 0xff, 0x34, 0x2c, 0x67, 0xd6, 0x95, 0x8d, 0x1c, 0x59, 0x14, 0x82, 0x7d, 0xc6, 0x27, + 0xfe, 0xd3, 0x10, 0x7f, 0x04, 0xb9, 0xa7, 0x7e, 0x70, 0xec, 0xf9, 0x4e, 0xb7, 0x9c, 0xe5, 0x73, + 0x7e, 0x30, 0x7b, 0xce, 0x07, 0x52, 0x8b, 0x24, 0xfa, 0x78, 0x03, 0x50, 0xf8, 0xc4, 0xb3, 0x43, + 0xea, 0xd1, 0x4e, 0x64, 0x7b, 0x6e, 0xdf, 0x8d, 0xca, 0x39, 0xee, 0x92, 0xa5, 0xf0, 0x89, 0xd7, + 0xe2, 0xec, 0x1a, 0xe3, 0x62, 0x1b, 0x56, 0xa3, 0xc0, 0x19, 0x84, 0x4e, 0x87, 0x0d, 0x66, 0xbb, + 0xa1, 0xef, 0x39, 0xdc, 0x1d, 0xf3, 0x7c, 0xca, 0xcd, 0xd9, 0x53, 0x5a, 0xe3, 0x2e, 0xd5, 0xb8, + 0x07, 0x59, 0x89, 0x66, 0x70, 0xf1, 0x75, 0x58, 0x0d, 0x8f, 0xdd, 0xa1, 0xcd, 0xc7, 0xb1, 0x87, + 0x9e, 0x33, 0xb0, 0x3b, 0x4e, 0xe7, 0x88, 0x96, 0x81, 0x9b, 0x8d, 0x99, 0x90, 0xef, 0x7b, 0xd3, + 0x73, 0x06, 0x15, 0x26, 0x61, 0xa0, 0x33, 0xbd, 0x01, 0x0d, 0xec, 0x53, 0x1a, 0x84, 0x6c, 0x35, + 0x85, 0x17, 0x81, 0xde, 0x14, 0xca, 0x07, 0x42, 0x97, 0x94, 0x86, 0x53, 0xb4, 0xf6, 0x15, 0x28, + 0x4d, 0x6f, 0x0b, 0x5e, 0x82, 0xa2, 0xf5, 0xb0, 0x69, 0xd8, 0xba, 0xb9, 0x67, 0x9b, 0x7a, 0xdd, + 0x40, 0x73, 0xb8, 0x08, 0x79, 0xce, 0x6a, 0x98, 0xb5, 0x87, 0x48, 0xc1, 0xf3, 0x90, 0xd2, 0x6b, + 0x35, 0xa4, 0x6a, 0xb7, 0x20, 0x17, 0xe3, 0x8b, 0x17, 0xa1, 0xd0, 0x36, 0x5b, 0x4d, 0xa3, 0x52, + 0xdd, 0xaf, 0x1a, 0x7b, 0x68, 0x0e, 0xe7, 0x20, 0xdd, 0xa8, 0x59, 0x4d, 0xa4, 0x88, 0x96, 0xde, + 0x44, 0x2a, 0xeb, 0xb9, 0xb7, 0xab, 0xa3, 0x94, 0xf6, 0x67, 0x0a, 0xac, 0xcc, 0xc2, 0x09, 0x17, + 0x60, 0x7e, 0xcf, 0xd8, 0xd7, 0xdb, 0x35, 0x0b, 0xcd, 0xe1, 0x65, 0x58, 0x24, 0x46, 0xd3, 0xd0, + 0x2d, 0x7d, 0xb7, 0x66, 0xd8, 0xc4, 0xd0, 0xf7, 0x90, 0x82, 0x31, 0x94, 0x58, 0xcb, 0xae, 0x34, + 0xea, 0xf5, 0xaa, 0x65, 0x19, 0x7b, 0x48, 0xc5, 0x2b, 0x80, 0x38, 0xaf, 0x6d, 0x8e, 0xb9, 0x29, + 0x8c, 0x60, 0xa1, 0x65, 0x90, 0xaa, 0x5e, 0xab, 0x3e, 0x62, 0x03, 0xa0, 0x34, 0xfe, 0x1c, 0xbc, + 0x5f, 0x69, 0x98, 0xad, 0x6a, 0xcb, 0x32, 0x4c, 0xcb, 0x6e, 0x99, 0x7a, 0xb3, 0xf5, 0x71, 0xc3, + 0xe2, 0x23, 0x0b, 0xe3, 0x32, 0xb8, 0x04, 0xa0, 0xb7, 0xad, 0x86, 0x18, 0x07, 0x65, 0xb5, 0x36, + 0x94, 0xa6, 0x21, 0x64, 0xab, 0x92, 0x4b, 0xb4, 0x9b, 0x35, 0xdd, 0x34, 0x0d, 0x82, 0xe6, 0x70, + 0x16, 0xd4, 0x83, 0x1b, 0x48, 0xe1, 0xbf, 0x3b, 0x48, 0xc5, 0x0b, 0x90, 0x3b, 0xd8, 0xb9, 0x13, + 0x50, 0xda, 0x1d, 0x89, 0x95, 0x1c, 0xec, 0xd4, 0xe8, 0xe3, 0x68, 0x9b, 0xb8, 0xbd, 0xa3, 0x08, + 0xa5, 0xef, 0xa6, 0x73, 0x0a, 0x52, 0xef, 0xa6, 0x73, 0x2a, 0x4a, 0xdd, 0x4d, 0xe7, 0x52, 0x28, + 0xad, 0xfd, 0xa5, 0x0a, 0x19, 0xbe, 0x05, 0x2c, 0x28, 0x4f, 0x84, 0x5a, 0xde, 0x4e, 0x02, 0x94, + 0xfa, 0x82, 0x00, 0xc5, 0xe3, 0xba, 0x0c, 0x95, 0x82, 0xc0, 0x97, 0x21, 0xef, 0x07, 0x3d, 0x5b, + 0x48, 0x44, 0x90, 0xcf, 0xf9, 0x41, 0x8f, 0x67, 0x03, 0x16, 0x60, 0x59, 0x6e, 0x38, 0x74, 0x42, + 0xca, 0xcf, 0x59, 0x9e, 0x24, 0x34, 0x7e, 0x17, 0x98, 0x9e, 0xcd, 0xd7, 0x91, 0xe5, 0xb2, 0x79, + 0x3f, 0xe8, 0x99, 0x6c, 0x29, 0x9f, 0x87, 0x62, 0xc7, 0xf7, 0x4e, 0xfa, 0x03, 0xdb, 0xa3, 0x83, + 0x5e, 0x74, 0x54, 0x9e, 0x5f, 0x57, 0x36, 0x8a, 0x64, 0x41, 0x30, 0x6b, 0x9c, 0x87, 0xcb, 0x30, + 0xdf, 0x39, 0x72, 0x82, 0x90, 0x8a, 0xb3, 0x55, 0x24, 0x31, 0xc9, 0x67, 0xa5, 0x1d, 0xb7, 0xef, + 0x78, 0x21, 0x3f, 0x47, 0x45, 0x92, 0xd0, 0xcc, 0x88, 0xc7, 0x9e, 0xd3, 0x0b, 0xb9, 0xff, 0x17, + 0x89, 0x20, 0xf0, 0x87, 0x50, 0x90, 0x13, 0x72, 0x08, 0x0a, 0x7c, 0x39, 0x20, 0x58, 0x0c, 0x01, + 0xed, 0x27, 0x20, 0x45, 0xfc, 0xa7, 0x6c, 0x4e, 0xb1, 0xa2, 0xb0, 0xac, 0xac, 0xa7, 0x36, 0x30, + 0x89, 0x49, 0x96, 0xa4, 0x64, 0x9c, 0x16, 0xe1, 0x3b, 0x8e, 0xcc, 0xdf, 0x51, 0xa0, 0xc0, 0xcf, + 0x17, 0xa1, 0xe1, 0x89, 0x17, 0xb1, 0x78, 0x2e, 0x03, 0x99, 0x32, 0x15, 0xcf, 0xf9, 0xbe, 0x10, + 0x29, 0x63, 0x00, 0xb0, 0xd8, 0x64, 0x3b, 0x8f, 0x1f, 0xd3, 0x4e, 0x44, 0x45, 0xda, 0x4a, 0x93, + 0x05, 0xc6, 0xd4, 0x25, 0x8f, 0x21, 0xef, 0x0e, 0x42, 0x1a, 0x44, 0xb6, 0xdb, 0xe5, 0x7b, 0x92, + 0x26, 0x39, 0xc1, 0xa8, 0x76, 0xf1, 0x07, 0x90, 0xe6, 0xd1, 0x2d, 0xcd, 0x67, 0x01, 0x39, 0x0b, + 0xf1, 0x9f, 0x12, 0xce, 0xbf, 0x9b, 0xce, 0x65, 0x50, 0x56, 0xfb, 0x2a, 0x2c, 0xf0, 0xc5, 0x3d, + 0x70, 0x82, 0x81, 0x3b, 0xe8, 0xf1, 0x64, 0xed, 0x77, 0x85, 0x5f, 0x14, 0x09, 0x6f, 0x33, 0x9b, + 0xfb, 0x34, 0x0c, 0x9d, 0x1e, 0x95, 0xc9, 0x33, 0x26, 0xb5, 0x3f, 0x49, 0x41, 0xa1, 0x15, 0x05, + 0xd4, 0xe9, 0xf3, 0x3c, 0x8c, 0xbf, 0x0a, 0x10, 0x46, 0x4e, 0x44, 0xfb, 0x74, 0x10, 0xc5, 0xf6, + 0xbd, 0x27, 0x67, 0x9e, 0xd0, 0xdb, 0x6a, 0xc5, 0x4a, 0x64, 0x42, 0x1f, 0x6f, 0x43, 0x81, 0x32, + 0xb1, 0x1d, 0xb1, 0x7c, 0x2e, 0x73, 0xc6, 0x52, 0x1c, 0x72, 0x92, 0x44, 0x4f, 0x80, 0x26, 0xed, + 0xb5, 0xef, 0xaa, 0x90, 0x4f, 0x46, 0xc3, 0x3a, 0xe4, 0x3a, 0x4e, 0x44, 0x7b, 0x7e, 0x30, 0x92, + 0x69, 0xf6, 0xca, 0x8b, 0x66, 0xdf, 0xaa, 0x48, 0x65, 0x92, 0x74, 0xc3, 0xef, 0x83, 0xb8, 0xbb, + 0x08, 0xb7, 0x14, 0xf6, 0xe6, 0x39, 0x87, 0x3b, 0xe6, 0x47, 0x80, 0x87, 0x81, 0xdb, 0x77, 0x82, + 0x91, 0x7d, 0x4c, 0x47, 0x71, 0x4a, 0x4a, 0xcd, 0xd8, 0x49, 0x24, 0xf5, 0xee, 0xd1, 0x91, 0x8c, + 0x7a, 0xb7, 0xa6, 0xfb, 0x4a, 0x6f, 0x39, 0xbf, 0x3f, 0x13, 0x3d, 0x79, 0x92, 0x0f, 0xe3, 0x74, + 0x9e, 0xe1, 0x8e, 0xc5, 0x9a, 0xda, 0x97, 0x20, 0x17, 0x2f, 0x1e, 0xe7, 0x21, 0x63, 0x04, 0x81, + 0x1f, 0xa0, 0x39, 0x1e, 0xfc, 0xea, 0x35, 0x11, 0x3f, 0xf7, 0xf6, 0x58, 0xfc, 0xfc, 0x17, 0x35, + 0xc9, 0xa9, 0x84, 0x3e, 0x39, 0xa1, 0x61, 0x84, 0x7f, 0x06, 0x96, 0x29, 0x77, 0x21, 0xf7, 0x94, + 0xda, 0x1d, 0x7e, 0x01, 0x63, 0x0e, 0xa4, 0x70, 0xbc, 0x17, 0xb7, 0xc4, 0x7d, 0x31, 0xbe, 0x98, + 0x91, 0xa5, 0x44, 0x57, 0xb2, 0xba, 0xd8, 0x80, 0x65, 0xb7, 0xdf, 0xa7, 0x5d, 0xd7, 0x89, 0x26, + 0x07, 0x10, 0x1b, 0xb6, 0x1a, 0xdf, 0x4f, 0xa6, 0xee, 0x77, 0x64, 0x29, 0xe9, 0x91, 0x0c, 0x73, + 0x05, 0xb2, 0x11, 0xbf, 0x8b, 0x72, 0xdf, 0x2d, 0x6c, 0x17, 0xe3, 0x88, 0xc3, 0x99, 0x44, 0x0a, + 0xf1, 0x97, 0x40, 0xdc, 0x6c, 0x79, 0x6c, 0x19, 0x3b, 0xc4, 0xf8, 0xc2, 0x42, 0x84, 0x1c, 0x5f, + 0x81, 0xd2, 0x54, 0x2a, 0xed, 0x72, 0xc0, 0x52, 0xa4, 0x38, 0x99, 0x17, 0xbb, 0xf8, 0x2a, 0xcc, + 0xfb, 0x22, 0x71, 0xf1, 0xa8, 0x33, 0x5e, 0xf1, 0x74, 0x56, 0x23, 0xb1, 0x16, 0x8b, 0x0d, 0x01, + 0x0d, 0x69, 0x70, 0x4a, 0xbb, 0x6c, 0xd0, 0x79, 0x3e, 0x28, 0xc4, 0xac, 0x6a, 0x57, 0xfb, 0x29, + 0x58, 0x4c, 0x20, 0x0e, 0x87, 0xfe, 0x20, 0xa4, 0x78, 0x13, 0xb2, 0x01, 0x3f, 0xef, 0x12, 0x56, + 0x2c, 0xe7, 0x98, 0x88, 0x04, 0x44, 0x6a, 0x68, 0x5d, 0x58, 0x14, 0x9c, 0x07, 0x6e, 0x74, 0xc4, + 0x77, 0x12, 0x5f, 0x81, 0x0c, 0x65, 0x8d, 0x33, 0x9b, 0x42, 0x9a, 0x15, 0x2e, 0x27, 0x42, 0x3a, + 0x31, 0x8b, 0xfa, 0xd2, 0x59, 0xfe, 0x43, 0x85, 0x65, 0xb9, 0xca, 0x5d, 0x27, 0xea, 0x1c, 0xbd, + 0xa6, 0xde, 0xf0, 0xa3, 0x30, 0xcf, 0xf8, 0x6e, 0x72, 0x72, 0x66, 0xf8, 0x43, 0xac, 0xc1, 0x3c, + 0xc2, 0x09, 0xed, 0x89, 0xed, 0x97, 0x77, 0xbd, 0xa2, 0x13, 0x4e, 0xdc, 0x0c, 0x66, 0x38, 0x4e, + 0xf6, 0x25, 0x8e, 0x33, 0x7f, 0x11, 0xc7, 0xd1, 0xf6, 0x60, 0x65, 0x1a, 0x71, 0xe9, 0x1c, 0x3f, + 0x06, 0xf3, 0x62, 0x53, 0xe2, 0x18, 0x39, 0x6b, 0xdf, 0x62, 0x15, 0xed, 0x13, 0x15, 0x56, 0x64, + 0xf8, 0xfa, 0x6c, 0x9c, 0xe3, 0x09, 0x9c, 0x33, 0x17, 0x3a, 0xa0, 0x17, 0xdb, 0x3f, 0xad, 0x02, + 0xab, 0x67, 0x70, 0x7c, 0x85, 0xc3, 0xfa, 0x3d, 0x05, 0x16, 0x76, 0x69, 0xcf, 0x1d, 0xbc, 0xa6, + 0xbb, 0x30, 0x01, 0x6e, 0xfa, 0x42, 0x4e, 0x3c, 0x84, 0xa2, 0xb4, 0x57, 0xa2, 0x75, 0x1e, 0x6d, + 0x65, 0xd6, 0x69, 0xb9, 0x05, 0x0b, 0xb2, 0x5a, 0xe0, 0x78, 0xae, 0x13, 0x26, 0xf6, 0x9c, 0x29, + 0x17, 0xe8, 0x4c, 0x48, 0x64, 0x61, 0x81, 0x13, 0xda, 0xbf, 0x2a, 0x50, 0xac, 0xf8, 0xfd, 0xbe, + 0x1b, 0xbd, 0xa6, 0x18, 0x9f, 0x47, 0x28, 0x3d, 0xcb, 0x1f, 0xaf, 0x43, 0x29, 0x36, 0x53, 0x42, + 0x7b, 0x26, 0xd3, 0x28, 0xe7, 0x32, 0xcd, 0xbf, 0x29, 0xb0, 0x48, 0x7c, 0xcf, 0x3b, 0x74, 0x3a, + 0xc7, 0x6f, 0x36, 0x38, 0x37, 0x00, 0x8d, 0x0d, 0xbd, 0x28, 0x3c, 0xff, 0xad, 0x40, 0xa9, 0x19, + 0xd0, 0xa1, 0x13, 0xd0, 0x37, 0x1a, 0x1d, 0x76, 0x4d, 0xef, 0x46, 0xf2, 0x82, 0x93, 0x27, 0xbc, + 0xad, 0x2d, 0xc1, 0x62, 0x62, 0xbb, 0x00, 0x4c, 0xfb, 0x7b, 0x05, 0x56, 0x85, 0x8b, 0x49, 0x49, + 0xf7, 0x35, 0x85, 0x25, 0xb6, 0x37, 0x3d, 0x61, 0x6f, 0x19, 0x2e, 0x9d, 0xb5, 0x4d, 0x9a, 0xfd, + 0x75, 0x15, 0xde, 0x89, 0x9d, 0xe7, 0x35, 0x37, 0xfc, 0x07, 0xf0, 0x87, 0x35, 0x28, 0x9f, 0x07, + 0x41, 0x22, 0xf4, 0x2d, 0x15, 0xca, 0x95, 0x80, 0x3a, 0x11, 0x9d, 0xb8, 0x07, 0xbd, 0x39, 0xbe, + 0x81, 0xaf, 0xc3, 0xc2, 0xd0, 0x09, 0x22, 0xb7, 0xe3, 0x0e, 0x1d, 0xf6, 0x29, 0x9a, 0xe1, 0xd7, + 0xac, 0x33, 0x03, 0x4c, 0xa9, 0x68, 0x97, 0xe1, 0xdd, 0x19, 0x88, 0x48, 0xbc, 0xfe, 0x47, 0x01, + 0xdc, 0x8a, 0x9c, 0x20, 0xfa, 0x0c, 0xe4, 0xa5, 0x99, 0xce, 0xb4, 0x0a, 0xcb, 0x53, 0xf6, 0x4f, + 0xe2, 0x42, 0xa3, 0xcf, 0x44, 0x4a, 0xfa, 0x54, 0x5c, 0x26, 0xed, 0x97, 0xb8, 0xfc, 0x93, 0x02, + 0x6b, 0x15, 0x5f, 0x14, 0x3d, 0xdf, 0xc8, 0x13, 0xa6, 0xbd, 0x0f, 0x97, 0x67, 0x1a, 0x28, 0x01, + 0xf8, 0x07, 0x05, 0x2e, 0x11, 0xea, 0x74, 0xdf, 0x4c, 0xe3, 0xef, 0xc3, 0x3b, 0xe7, 0x8c, 0x93, + 0x77, 0x94, 0x9b, 0x90, 0xeb, 0xd3, 0xc8, 0x61, 0x37, 0x5c, 0x69, 0xd2, 0x5a, 0x3c, 0xee, 0x58, + 0xbb, 0x2e, 0x35, 0x48, 0xa2, 0xab, 0xfd, 0xb3, 0x0a, 0xcb, 0xfc, 0x9e, 0xfd, 0xf6, 0x23, 0xef, + 0x42, 0x55, 0x98, 0xec, 0xd9, 0xcb, 0x1f, 0x53, 0x18, 0x06, 0xd4, 0x8e, 0xab, 0x03, 0xf3, 0xfc, + 0xaf, 0x42, 0x18, 0x06, 0xf4, 0xbe, 0xe0, 0x68, 0x7f, 0xad, 0xc0, 0xca, 0x34, 0xc4, 0xc9, 0x17, + 0xcd, 0xff, 0x75, 0xb5, 0x65, 0x46, 0x48, 0x49, 0x5d, 0xe4, 0x23, 0x29, 0x7d, 0xe1, 0x8f, 0xa4, + 0xbf, 0x51, 0xa1, 0x3c, 0x69, 0xcc, 0xdb, 0x9a, 0xce, 0x74, 0x4d, 0xe7, 0xfb, 0xad, 0xf2, 0x69, + 0x7f, 0xab, 0xc0, 0xbb, 0x33, 0x00, 0xfd, 0xfe, 0x5c, 0x64, 0xa2, 0xb2, 0xa3, 0xbe, 0xb4, 0xb2, + 0xf3, 0xc3, 0x77, 0x92, 0xbf, 0x53, 0x60, 0xa5, 0x2e, 0x6a, 0xf5, 0xa2, 0xf2, 0xf1, 0xfa, 0xc6, + 0x60, 0x5e, 0x8e, 0x4f, 0x8f, 0xff, 0xad, 0xd2, 0x2a, 0xb0, 0x7a, 0xc6, 0xb4, 0x57, 0xa8, 0xe6, + 0xfc, 0x97, 0x02, 0x4b, 0x72, 0x14, 0xfd, 0xb5, 0xbd, 0xbe, 0xcc, 0x40, 0x07, 0x7f, 0x00, 0x29, + 0xb7, 0x1b, 0xdf, 0x7b, 0xa7, 0x9f, 0x0c, 0x30, 0x81, 0x76, 0x1b, 0xf0, 0xa4, 0xdd, 0xaf, 0x00, + 0xdd, 0xbf, 0xab, 0xb0, 0x4a, 0x44, 0xf4, 0x7d, 0xfb, 0xff, 0xc2, 0x0f, 0xfa, 0xff, 0xc2, 0x8b, + 0x13, 0xd7, 0x27, 0xfc, 0x32, 0x35, 0x0d, 0xf5, 0x0f, 0x2f, 0x75, 0x9d, 0x49, 0xb4, 0xa9, 0x73, + 0x89, 0xf6, 0xd5, 0xe3, 0xd1, 0x27, 0x2a, 0xac, 0x49, 0x43, 0xde, 0xde, 0x75, 0x2e, 0xee, 0x11, + 0xd9, 0x73, 0x1e, 0xf1, 0x9f, 0x0a, 0x5c, 0x9e, 0x09, 0xe4, 0xff, 0xfb, 0x8d, 0xe6, 0x8c, 0xf7, + 0xa4, 0x5f, 0xea, 0x3d, 0x99, 0x0b, 0x7b, 0xcf, 0x37, 0x55, 0x28, 0x11, 0xea, 0x51, 0x27, 0x7c, + 0xc3, 0xab, 0x7b, 0x67, 0x30, 0xcc, 0x9c, 0xab, 0x73, 0x2e, 0xc1, 0x62, 0x02, 0x84, 0xfc, 0xe0, + 0xe2, 0x1f, 0xe8, 0x2c, 0x0f, 0x7e, 0x4c, 0x1d, 0x2f, 0x8a, 0x6f, 0x82, 0xda, 0x9f, 0xaa, 0x50, + 0x24, 0x8c, 0xe3, 0xf6, 0x69, 0x2b, 0x72, 0xa2, 0x10, 0x7f, 0x0e, 0x16, 0x8e, 0xb8, 0x8a, 0x3d, + 0xf6, 0x90, 0x3c, 0x29, 0x08, 0x9e, 0xf8, 0xf7, 0x71, 0x1b, 0x56, 0x43, 0xda, 0xf1, 0x07, 0xdd, + 0xd0, 0x3e, 0xa4, 0x47, 0xee, 0xa0, 0x6b, 0xf7, 0x9d, 0x30, 0xa2, 0x01, 0x87, 0xa5, 0x48, 0x96, + 0xa5, 0x70, 0x97, 0xcb, 0xea, 0x5c, 0x84, 0xaf, 0xc1, 0xca, 0xa1, 0x3b, 0xf0, 0xfc, 0x9e, 0x3d, + 0xf4, 0x9c, 0x11, 0x0d, 0x42, 0xbb, 0xe3, 0x9f, 0x0c, 0x04, 0x1e, 0x19, 0x82, 0x85, 0xac, 0x29, + 0x44, 0x15, 0x26, 0xc1, 0x8f, 0x60, 0x73, 0xe6, 0x2c, 0xf6, 0x63, 0xd7, 0x8b, 0x68, 0x40, 0xbb, + 0x76, 0x40, 0x87, 0x9e, 0xdb, 0x11, 0xcf, 0xa1, 0x04, 0x50, 0x5f, 0x9c, 0x31, 0xf5, 0xbe, 0x54, + 0x27, 0x63, 0x6d, 0x7c, 0x19, 0xf2, 0x9d, 0xe1, 0x89, 0x7d, 0xc2, 0x1f, 0x2d, 0x30, 0xfc, 0x14, + 0x92, 0xeb, 0x0c, 0x4f, 0xda, 0x8c, 0xc6, 0x08, 0x52, 0x4f, 0x86, 0x22, 0x38, 0x2b, 0x84, 0x35, + 0xb5, 0xef, 0x29, 0x50, 0xd2, 0x7b, 0xbd, 0x80, 0xf6, 0x9c, 0x48, 0xc2, 0x74, 0x0d, 0x56, 0x04, + 0x24, 0x23, 0x5b, 0xba, 0xab, 0xb0, 0x47, 0x11, 0xf6, 0x48, 0x99, 0xf0, 0x55, 0x61, 0xcf, 0x0e, + 0x5c, 0x3a, 0x19, 0xcc, 0xec, 0xa3, 0xf2, 0x3e, 0x2b, 0x89, 0x74, 0xb2, 0xd7, 0x4f, 0xc2, 0xbb, + 0xb3, 0x51, 0xe8, 0xbb, 0xe2, 0x49, 0x62, 0x91, 0x5c, 0x9a, 0x61, 0x74, 0xdd, 0x1d, 0xbc, 0xa0, + 0xab, 0xf3, 0x8c, 0xe3, 0xf5, 0x29, 0x5d, 0x9d, 0x67, 0xda, 0x9f, 0x27, 0xff, 0x29, 0xc6, 0xee, + 0x92, 0x04, 0x8e, 0xd8, 0x91, 0x95, 0x17, 0x39, 0x72, 0x19, 0xe6, 0x99, 0x33, 0xba, 0x83, 0x1e, + 0x37, 0x2e, 0x47, 0x62, 0x12, 0xb7, 0xe0, 0x8b, 0xd2, 0x76, 0xfa, 0x2c, 0xa2, 0xc1, 0xc0, 0xf1, + 0xbc, 0x91, 0x2d, 0xca, 0x8f, 0x83, 0x88, 0x76, 0xed, 0xf1, 0x13, 0x4d, 0x11, 0x3e, 0x3e, 0x2f, + 0xb4, 0x8d, 0x44, 0x99, 0x24, 0xba, 0x56, 0xf2, 0x78, 0xf3, 0x2b, 0x50, 0x0a, 0xa4, 0x13, 0xdb, + 0x21, 0xdb, 0x1e, 0x19, 0x72, 0x57, 0xe2, 0x57, 0x13, 0x93, 0x1e, 0x4e, 0x8a, 0xc1, 0x94, 0xc3, + 0xbf, 0x72, 0xc0, 0xb9, 0x9b, 0xce, 0x65, 0xd1, 0xbc, 0xf6, 0x17, 0x0a, 0x2c, 0xcf, 0xf8, 0x76, + 0x4f, 0x0a, 0x03, 0xca, 0x44, 0xdd, 0xf1, 0xc7, 0x21, 0xc3, 0x1f, 0xb4, 0xc8, 0x37, 0x54, 0xef, + 0x9c, 0xff, 0xf4, 0xe7, 0x8f, 0x4f, 0x88, 0xd0, 0x62, 0x67, 0x91, 0xdb, 0xd4, 0xe1, 0x85, 0xc7, + 0x38, 0xa2, 0x16, 0x18, 0x4f, 0xd4, 0x22, 0xcf, 0x57, 0x32, 0xd3, 0x2f, 0xad, 0x64, 0x6e, 0xfe, + 0x76, 0x0a, 0xf2, 0xf5, 0x51, 0xeb, 0x89, 0xb7, 0xef, 0x39, 0x3d, 0xfe, 0x3a, 0xa4, 0xde, 0xb4, + 0x1e, 0xa2, 0x39, 0xbc, 0x04, 0x45, 0xb3, 0x61, 0xd9, 0x66, 0xbb, 0x56, 0xb3, 0xf7, 0x6b, 0xfa, + 0x1d, 0xa4, 0x60, 0x04, 0x0b, 0x4d, 0x52, 0xb5, 0xef, 0x19, 0x0f, 0x05, 0x47, 0xc5, 0xcb, 0xb0, + 0xd8, 0x36, 0xab, 0xf7, 0xdb, 0xc6, 0x98, 0x99, 0xc6, 0xab, 0xb0, 0x54, 0x6f, 0xd7, 0xac, 0x6a, + 0xb3, 0x36, 0xc1, 0xce, 0xe1, 0x22, 0xe4, 0x77, 0x6b, 0x8d, 0x5d, 0x41, 0x22, 0x36, 0x7e, 0xdb, + 0x6c, 0x55, 0xef, 0x98, 0xc6, 0x9e, 0x60, 0xad, 0x33, 0xd6, 0x23, 0x83, 0x34, 0xf6, 0xab, 0xf1, + 0x94, 0xb7, 0x31, 0x82, 0xc2, 0x6e, 0xd5, 0xd4, 0x89, 0x1c, 0xe5, 0xb9, 0x82, 0x4b, 0x90, 0x37, + 0xcc, 0x76, 0x5d, 0xd2, 0x2a, 0x2e, 0xc3, 0xb2, 0xde, 0xb6, 0x1a, 0x76, 0xd5, 0xac, 0x10, 0xa3, + 0x6e, 0x98, 0x96, 0x94, 0xa4, 0xf1, 0x32, 0x94, 0xac, 0x6a, 0xdd, 0x68, 0x59, 0x7a, 0xbd, 0x29, + 0x99, 0x6c, 0x15, 0xb9, 0x96, 0x11, 0xeb, 0x20, 0xbc, 0x06, 0xab, 0x66, 0xc3, 0x8e, 0x9f, 0xcf, + 0x1d, 0xe8, 0xb5, 0xb6, 0x21, 0x65, 0xeb, 0xf8, 0x1d, 0xc0, 0x0d, 0xd3, 0x6e, 0x37, 0xf7, 0x74, + 0xcb, 0xb0, 0xcd, 0xc6, 0x03, 0x29, 0xb8, 0x8d, 0x4b, 0x90, 0x1b, 0xaf, 0xe0, 0x39, 0x43, 0xa1, + 0xd8, 0xd4, 0x89, 0x35, 0x36, 0xf6, 0xf9, 0x73, 0x06, 0x16, 0xdc, 0x21, 0x8d, 0x76, 0x73, 0xac, + 0xb6, 0x04, 0x05, 0x09, 0x96, 0x64, 0xa5, 0x19, 0x6b, 0xb7, 0x6a, 0x56, 0x92, 0xf5, 0x3d, 0xcf, + 0xad, 0xa9, 0x48, 0xd9, 0x3c, 0x86, 0x34, 0xdf, 0x8e, 0x1c, 0xa4, 0xcd, 0x86, 0x69, 0xa0, 0x39, + 0xbc, 0x08, 0x50, 0x6d, 0x55, 0x4d, 0xcb, 0xb8, 0x43, 0xf4, 0x1a, 0x33, 0x9b, 0x33, 0x62, 0x00, + 0x99, 0xb5, 0x0b, 0x30, 0x5f, 0x6d, 0xed, 0xd7, 0x1a, 0xba, 0x25, 0xcd, 0xac, 0xb6, 0xee, 0xb7, + 0x1b, 0x16, 0x13, 0x22, 0x5c, 0x80, 0x6c, 0xb5, 0x65, 0x19, 0x5f, 0xb3, 0x98, 0x5d, 0x5c, 0x26, + 0x50, 0x45, 0xcf, 0x6f, 0x6f, 0x7e, 0x3b, 0x05, 0x69, 0xfe, 0xf6, 0xba, 0x08, 0x79, 0xbe, 0xdb, + 0xd6, 0xc3, 0x26, 0x9b, 0x32, 0x0f, 0xe9, 0xaa, 0x69, 0xdd, 0x42, 0x3f, 0xa7, 0x62, 0x80, 0x4c, + 0x9b, 0xb7, 0x7f, 0x3e, 0xcb, 0xda, 0x55, 0xd3, 0xba, 0x7e, 0x13, 0x7d, 0x5d, 0x65, 0xc3, 0xb6, + 0x05, 0xf1, 0x0b, 0xb1, 0x60, 0x7b, 0x07, 0x7d, 0x23, 0x11, 0x6c, 0xef, 0xa0, 0x5f, 0x8c, 0x05, + 0x37, 0xb6, 0xd1, 0x37, 0x13, 0xc1, 0x8d, 0x6d, 0xf4, 0x4b, 0xb1, 0xe0, 0xe6, 0x0e, 0xfa, 0xe5, + 0x44, 0x70, 0x73, 0x07, 0xfd, 0x4a, 0x96, 0xd9, 0xc2, 0x2d, 0xb9, 0xb1, 0x8d, 0x7e, 0x35, 0x97, + 0x50, 0x37, 0x77, 0xd0, 0xaf, 0xe5, 0xd8, 0xfe, 0x27, 0xbb, 0x8a, 0x7e, 0x1d, 0xb1, 0x65, 0xb2, + 0x0d, 0x42, 0xbf, 0xc1, 0x9b, 0x4c, 0x84, 0x7e, 0x13, 0x31, 0x1b, 0x19, 0x97, 0x93, 0xdf, 0xe2, + 0x92, 0x87, 0x86, 0x4e, 0xd0, 0x6f, 0x65, 0xc5, 0x83, 0xce, 0x4a, 0xb5, 0xae, 0xd7, 0x10, 0xe6, + 0x3d, 0x18, 0x2a, 0xbf, 0x73, 0x8d, 0x35, 0x99, 0x7b, 0xa2, 0xdf, 0x6d, 0xb2, 0x09, 0x0f, 0x74, + 0x52, 0xf9, 0x58, 0x27, 0xe8, 0xf7, 0xae, 0xb1, 0x09, 0x0f, 0x74, 0x22, 0xf1, 0xfa, 0xfd, 0x26, + 0x53, 0xe4, 0xa2, 0x3f, 0xb8, 0xc6, 0x16, 0x2d, 0xf9, 0x7f, 0xd8, 0xc4, 0x39, 0x48, 0xed, 0x56, + 0x2d, 0xf4, 0x6d, 0x3e, 0x1b, 0x73, 0x51, 0xf4, 0x47, 0x88, 0x31, 0x5b, 0x86, 0x85, 0xbe, 0xc3, + 0x98, 0x19, 0xab, 0xdd, 0xac, 0x19, 0xe8, 0x3d, 0xb6, 0xb8, 0x3b, 0x46, 0xa3, 0x6e, 0x58, 0xe4, + 0x21, 0xfa, 0x63, 0xae, 0x7e, 0xb7, 0xd5, 0x30, 0xd1, 0x77, 0x11, 0x2e, 0x01, 0x18, 0x5f, 0x6b, + 0x12, 0xa3, 0xd5, 0xaa, 0x36, 0x4c, 0xf4, 0xe1, 0xe6, 0x3e, 0xa0, 0xb3, 0xe1, 0x80, 0x19, 0xd0, + 0x36, 0xef, 0x99, 0x8d, 0x07, 0x26, 0x9a, 0x63, 0x44, 0x93, 0x18, 0x4d, 0x9d, 0x18, 0x48, 0xc1, + 0x00, 0x59, 0xf9, 0x4c, 0x94, 0xbf, 0xf7, 0x24, 0x8d, 0x5a, 0x6d, 0x57, 0xaf, 0xdc, 0x43, 0xa9, + 0xdd, 0x2f, 0xc3, 0xa2, 0xeb, 0x6f, 0x9d, 0xba, 0x11, 0x0d, 0x43, 0xf1, 0xba, 0xff, 0x91, 0x26, + 0x29, 0xd7, 0xbf, 0x2a, 0x5a, 0x57, 0x7b, 0xfe, 0xd5, 0xd3, 0xe8, 0x2a, 0x97, 0x5e, 0xe5, 0x11, + 0xe3, 0x30, 0xcb, 0x89, 0x1b, 0xff, 0x1b, 0x00, 0x00, 0xff, 0xff, 0xbd, 0xb2, 0x28, 0xeb, 0x3b, + 0x30, 0x00, 0x00, } diff --git a/go/vt/sqlparser/ast_rewriting.go b/go/vt/sqlparser/ast_rewriting.go index 23ad0758c79..96de3d54ebc 100644 --- a/go/vt/sqlparser/ast_rewriting.go +++ b/go/vt/sqlparser/ast_rewriting.go @@ -233,6 +233,7 @@ func (er *expressionRewriter) sysVarRewrite(cursor *Cursor, node *ColName) { sysvars.SQLSelectLimit.Name, sysvars.TransactionMode.Name, sysvars.Workload.Name, + sysvars.PlannerVersion.Name, sysvars.DDLStrategy.Name, sysvars.ReadAfterWriteGTID.Name, sysvars.ReadAfterWriteTimeOut.Name, diff --git a/go/vt/sqlparser/ast_rewriting_test.go b/go/vt/sqlparser/ast_rewriting_test.go index 29ac9431c81..53b2ebafcac 100644 --- a/go/vt/sqlparser/ast_rewriting_test.go +++ b/go/vt/sqlparser/ast_rewriting_test.go @@ -31,7 +31,7 @@ type myTestCase struct { ddlStrategy bool udv int autocommit, clientFoundRows, skipQueryPlanCache bool - sqlSelectLimit, transactionMode, workload bool + sqlSelectLimit, transactionMode, workload, plannerVersion bool } func TestRewrites(in *testing.T) { @@ -164,6 +164,10 @@ func TestRewrites(in *testing.T) { // SELECT * behaves different depending the join type used, so if that has been used, we won't rewrite in: "SELECT * FROM A JOIN B USING (id1,id2,id3)", expected: "SELECT * FROM A JOIN B USING (id1,id2,id3)", + }, { + in: "SELECT @@planner_version", + expected: "SELECT :__vtplanner_version as `@@planner_version`", + plannerVersion: true, }} for _, tc := range tests { @@ -196,6 +200,7 @@ func TestRewrites(in *testing.T) { assert.Equal(tc.rawGTID, result.NeedsSysVar(sysvars.ReadAfterWriteGTID.Name), "should need rawGTID") assert.Equal(tc.rawTimeout, result.NeedsSysVar(sysvars.ReadAfterWriteTimeOut.Name), "should need rawTimeout") assert.Equal(tc.sessTrackGTID, result.NeedsSysVar(sysvars.SessionTrackGTIDs.Name), "should need sessTrackGTID") + assert.Equal(tc.plannerVersion, result.NeedsSysVar(sysvars.PlannerVersion.Name), "should need :__vtplanner_version") }) } } diff --git a/go/vt/sysvars/sysvars.go b/go/vt/sysvars/sysvars.go index 2fa219fa2f6..8af378d2074 100644 --- a/go/vt/sysvars/sysvars.go +++ b/go/vt/sysvars/sysvars.go @@ -51,6 +51,7 @@ var ( SQLSelectLimit = SystemVariable{Name: "sql_select_limit", Default: off} TransactionMode = SystemVariable{Name: "transaction_mode", IdentifierAsString: true} Workload = SystemVariable{Name: "workload", IdentifierAsString: true} + PlannerVersion = SystemVariable{Name: "planner_version", IdentifierAsString: true} Charset = SystemVariable{Name: "charset", Default: utf8, IdentifierAsString: true} Names = SystemVariable{Name: "names", Default: utf8, IdentifierAsString: true} // Online DDL diff --git a/go/vt/vtgate/engine/fake_vcursor_test.go b/go/vt/vtgate/engine/fake_vcursor_test.go index 25af8de7e84..ed01d292c84 100644 --- a/go/vt/vtgate/engine/fake_vcursor_test.go +++ b/go/vt/vtgate/engine/fake_vcursor_test.go @@ -148,6 +148,10 @@ func (t noopVCursor) SetWorkload(querypb.ExecuteOptions_Workload) { panic("implement me") } +func (t noopVCursor) SetPlannerVersion(querypb.ExecuteOptions_PlannerVersion) { + panic("implement me") +} + func (t noopVCursor) SetTarget(string) error { panic("implement me") } @@ -479,6 +483,10 @@ func (f *loggingVCursor) SetWorkload(querypb.ExecuteOptions_Workload) { panic("implement me") } +func (f *loggingVCursor) SetPlannerVersion(querypb.ExecuteOptions_PlannerVersion) { + panic("implement me") +} + func (f *loggingVCursor) FindRoutedTable(tbl sqlparser.TableName) (*vindexes.Table, error) { f.log = append(f.log, fmt.Sprintf("FindTable(%s)", sqlparser.String(tbl))) return f.tableRoutes.tbl, nil diff --git a/go/vt/vtgate/engine/primitive.go b/go/vt/vtgate/engine/primitive.go index fd8be8044e8..396cde32457 100644 --- a/go/vt/vtgate/engine/primitive.go +++ b/go/vt/vtgate/engine/primitive.go @@ -125,6 +125,7 @@ type ( SetSQLSelectLimit(int64) error SetTransactionMode(vtgatepb.TransactionMode) SetWorkload(querypb.ExecuteOptions_Workload) + SetPlannerVersion(querypb.ExecuteOptions_PlannerVersion) SetFoundRows(uint64) SetDDLStrategy(string) diff --git a/go/vt/vtgate/executor.go b/go/vt/vtgate/executor.go index acfe6b7ed77..766b6c7f992 100644 --- a/go/vt/vtgate/executor.go +++ b/go/vt/vtgate/executor.go @@ -312,6 +312,12 @@ func (e *Executor) addNeededBindVars(bindVarNeeds *sqlparser.BindVarNeeds, bindV } }) bindVars[key] = sqltypes.StringBindVariable(v) + case sysvars.PlannerVersion.Name: + v := *plannerVersion + ifOptionsExist(session, func(options *querypb.ExecuteOptions) { + v = options.GetPlannerVersion().String() + }) + bindVars[key] = sqltypes.StringBindVariable(v) } } diff --git a/go/vt/vtgate/planbuilder/builder.go b/go/vt/vtgate/planbuilder/builder.go index ecc91c2022f..c7d38fe2e3b 100644 --- a/go/vt/vtgate/planbuilder/builder.go +++ b/go/vt/vtgate/planbuilder/builder.go @@ -53,18 +53,18 @@ type ContextVSchema interface { Planner() PlannerVersion } -// PlannerVersion is the version of the planner -type PlannerVersion int +// PlannerVersion is an alias here to make the code more readable +type PlannerVersion = querypb.ExecuteOptions_PlannerVersion const ( // V3 is also the default planner - V3 PlannerVersion = iota + V3 = querypb.ExecuteOptions_V3 // V4 is the new planner - V4 + V4 = querypb.ExecuteOptions_V4 // V4GreedyOnly uses only the faster greedy planner - V4GreedyOnly + V4GreedyOnly = querypb.ExecuteOptions_V4Greedy // V4Left2Right tries to emulate the V3 planner by only joining plans in the order they are listed in the FROM-clause - V4Left2Right + V4Left2Right = querypb.ExecuteOptions_V4Left2Right ) type truncater interface { diff --git a/go/vt/vtgate/vcursor_impl.go b/go/vt/vtgate/vcursor_impl.go index dd1f6d83712..403d04a390f 100644 --- a/go/vt/vtgate/vcursor_impl.go +++ b/go/vt/vtgate/vcursor_impl.go @@ -351,6 +351,10 @@ func (vc *vcursorImpl) AllKeyspace() ([]*vindexes.Keyspace, error) { // Planner implements the ContextVSchema interface func (vc *vcursorImpl) Planner() planbuilder.PlannerVersion { + if vc.safeSession.Options != nil && + vc.safeSession.Options.PlannerVersion != querypb.ExecuteOptions_DEFAULT_PLANNER { + return vc.safeSession.Options.PlannerVersion + } switch strings.ToLower(*plannerVersion) { case "v3": return planbuilder.V3 @@ -358,6 +362,8 @@ func (vc *vcursorImpl) Planner() planbuilder.PlannerVersion { return planbuilder.V4 case "v4greedy", "greedy": return planbuilder.V4GreedyOnly + case "left2right": + return planbuilder.V4Left2Right } log.Warn("unknown planner version configured. using the default") @@ -629,6 +635,11 @@ func (vc *vcursorImpl) SetWorkload(workload querypb.ExecuteOptions_Workload) { vc.safeSession.GetOrCreateOptions().Workload = workload } +// SetPlannerVersion implements the SessionActions interface +func (vc *vcursorImpl) SetPlannerVersion(v planbuilder.PlannerVersion) { + vc.safeSession.GetOrCreateOptions().PlannerVersion = v +} + // SetFoundRows implements the SessionActions interface func (vc *vcursorImpl) SetFoundRows(foundRows uint64) { vc.safeSession.FoundRows = foundRows diff --git a/proto/query.proto b/proto/query.proto index 9b8d21bc0e0..69e889a0610 100644 --- a/proto/query.proto +++ b/proto/query.proto @@ -297,6 +297,18 @@ message ExecuteOptions { // skip_query_plan_cache specifies if the query plan should be cached by vitess. // By default all query plans are cached. bool skip_query_plan_cache = 10; + + enum PlannerVersion { + DEFAULT_PLANNER = 0; + V3 = 1; + V4 = 2; + V4Greedy = 3; + V4Left2Right = 4; + } + + // PlannerVersion specifies which planner to use. + // If DEFAULT is chosen, whatever vtgate was started with will be used + PlannerVersion planner_version = 11; } // Field describes a single column returned by a query From b4ac6de5f24cffa4c02e1601a637374285bb14c2 Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Tue, 29 Dec 2020 16:50:03 +0100 Subject: [PATCH 23/40] added helpful comments Signed-off-by: Andres Taylor --- go/vt/vtgate/planbuilder/route_planning.go | 33 +++++++++++++++------- 1 file changed, 23 insertions(+), 10 deletions(-) diff --git a/go/vt/vtgate/planbuilder/route_planning.go b/go/vt/vtgate/planbuilder/route_planning.go index 228ee00b756..48e4045d02b 100644 --- a/go/vt/vtgate/planbuilder/route_planning.go +++ b/go/vt/vtgate/planbuilder/route_planning.go @@ -99,7 +99,10 @@ func planProjections(sel *sqlparser.Select, plan logicalPlan, semTable *semantic type ( joinTree interface { + // solves returns the table identifiers that are solved by this plan solves() semantics.TableSet + + // cost is simply the number of routes in the joinTree cost() int } routeTable struct { @@ -107,27 +110,33 @@ type ( vtable *vindexes.Table } routePlan struct { - routeOpCode engine.RouteOpcode - solved semantics.TableSet - tables []*routeTable + routeOpCode engine.RouteOpcode + solved semantics.TableSet + keyspace *vindexes.Keyspace + + // tables contains all the tables that are solved by this plan. + // the tables also contain any predicates that only depend on that particular table + tables []*routeTable + + // extraPredicates are the predicates that depend on multiple tables extraPredicates []sqlparser.Expr - keyspace *vindexes.Keyspace // vindex and conditions is set if a vindex will be used for this route. - vindex vindexes.Vindex - + vindex vindexes.Vindex conditions []sqlparser.Expr } joinPlan struct { predicates []sqlparser.Expr - - lhs, rhs joinTree + lhs, rhs joinTree } ) +// solves implements the joinTree interface func (rp *routePlan) solves() semantics.TableSet { return rp.solved } + +// cost implements the joinTree interface func (*routePlan) cost() int { return 1 } @@ -140,12 +149,11 @@ type vindexPlusPredicates struct { } func (rp *routePlan) addPredicate(predicates ...sqlparser.Expr) error { - if len(rp.tables) != 1 { return vterrors.Errorf(vtrpcpb.Code_INTERNAL, "addPredicate should only be called when the route has a single table") } - vindexPreds := []*vindexPlusPredicates{} + var vindexPreds []*vindexPlusPredicates // Add all the column vindexes to the list of vindexPlusPredicates for _, columnVindex := range rp.tables[0].vtable.ColumnVindexes { @@ -246,6 +254,10 @@ func dpSolve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVSchem dpTable.add(plan) } + /* + Next we'll solve bigger and bigger joins, using smaller plans to build larger ones, + until we have a join tree covering all tables in the FROM clause + */ for currentSize := 2; currentSize <= size; currentSize++ { lefts := dpTable.bitSetsOfSize(currentSize - 1) rights := dpTable.bitSetsOfSize(1) @@ -334,6 +346,7 @@ func greedySolve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVS return plans[0], nil } + func leftToRightSolve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVSchema) (joinTree, error) { plans := make([]joinTree, len(qg.tables)) From cf1ad11342317d639568ed50f2a5d392d71c5c5b Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Tue, 29 Dec 2020 16:54:45 +0100 Subject: [PATCH 24/40] add a shortcut to the greedy planner to prefer joins with predicates Signed-off-by: Andres Taylor --- go/vt/vtgate/planbuilder/route_planning.go | 84 +++++++++++++++------- 1 file changed, 58 insertions(+), 26 deletions(-) diff --git a/go/vt/vtgate/planbuilder/route_planning.go b/go/vt/vtgate/planbuilder/route_planning.go index 48e4045d02b..c046043427b 100644 --- a/go/vt/vtgate/planbuilder/route_planning.go +++ b/go/vt/vtgate/planbuilder/route_planning.go @@ -298,9 +298,10 @@ func createJoin(lhs joinTree, rhs joinTree, joinPredicates []sqlparser.Expr, sem } /* - The greedy planner will plan a query by finding first finding the best route plan for every table, and then - finding the cheapest join, and using that. Then it searches for the next cheapest joinTree that can be produced, - and keeps doing this until all tables have been joined + The greedy planner will plan a query by finding first finding the best route plan for every table. + Then, iteratively, it finds the cheapest join that can be produced between the remaining plans, + and removes the two inputs to this cheapest plan and instead adds the join. + As an optimization, it first only considers joining tables that have predicates defined between them */ func greedySolve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVSchema) (joinTree, error) { plans := make([]joinTree, len(qg.tables)) @@ -316,35 +317,66 @@ func greedySolve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVS plans[i] = plan } - // loop while we have un-joined query parts left + crossJoinsOK := false for len(plans) > 1 { - var lIdx, rIdx int - var bestPlan joinTree - for i, lhs := range plans { - for j := i + 1; j < len(plans); j++ { - rhs := plans[j] - solves := lhs.solves() | rhs.solves() - joinPredicates := qg.crossTable[solves] - plan := planCache[solves] - if plan == nil { - plan = createJoin(lhs, rhs, joinPredicates, semTable) - planCache[solves] = plan - } + bestPlan, lIdx, rIdx := findBestJoin(qg, semTable, plans, planCache, crossJoinsOK) + if bestPlan != nil { + // if we found a best plan, we'll replace the two plans that were joined with the join plan created + plans = removeAt(plans, rIdx) + plans = removeAt(plans, lIdx) + plans = append(plans, bestPlan) + } else { + // we will only fail to find a join plan when there are only cross joins left + // when that happens, we switch over to allow cross joins as well. + // this way we prioritize joining plans with predicates first + crossJoinsOK = true + } + } - if bestPlan == nil || plan.cost() < bestPlan.cost() { - bestPlan = plan - // remember which plans we based on, so we can remove them later - lIdx = i - rIdx = j + return plans[0], nil +} + +func findBestJoin( + qg *queryGraph, + semTable *semantics.SemTable, + plans []joinTree, + planCache map[semantics.TableSet]joinTree, + crossJoinsOK bool, +) (joinTree, int, int) { + var lIdx, rIdx int + var bestPlan joinTree + + for i, lhs := range plans { + for j := i + 1; j < len(plans); j++ { + rhs := plans[j] + solves := lhs.solves() | rhs.solves() + joinPredicates := qg.crossTable[solves] + if len(joinPredicates) == 0 && !crossJoinsOK { + // if there are no predicates joining the to tables, + // creating a join between them would produce a + // cartesian product, which is almost always a bad idea + continue + } + plan := planCache[solves] + if plan == nil { + plan = createJoin(lhs, rhs, joinPredicates, semTable) + planCache[solves] = plan + if plan.cost() == 1 { + // if we are able to merge the two inputs into a single route, + // we shortcut here and pick this plan. this limits the search space + return plan, i, j } } + + if bestPlan == nil || plan.cost() < bestPlan.cost() { + bestPlan = plan + // remember which plans we based on, so we can remove them later + lIdx = i + rIdx = j + } } - plans = removeAt(plans, rIdx) - plans = removeAt(plans, lIdx) - plans = append(plans, bestPlan) } - - return plans[0], nil + return bestPlan, lIdx, rIdx } func leftToRightSolve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVSchema) (joinTree, error) { From 000426e03b1e2eaf581b54006abe2fff51ab22fa Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Tue, 29 Dec 2020 19:32:14 +0100 Subject: [PATCH 25/40] change planner benchmark to only read the input file once Signed-off-by: Andres Taylor --- go/vt/vtgate/planbuilder/plan_test.go | 24 ++++++++++++++---------- 1 file changed, 14 insertions(+), 10 deletions(-) diff --git a/go/vt/vtgate/planbuilder/plan_test.go b/go/vt/vtgate/planbuilder/plan_test.go index 5c02c11d4cd..10d8d1cf157 100644 --- a/go/vt/vtgate/planbuilder/plan_test.go +++ b/go/vt/vtgate/planbuilder/plan_test.go @@ -584,24 +584,28 @@ func BenchmarkPlanner(b *testing.B) { sysVarEnabled: true, } for _, filename := range filenames { - b.Run(filename+"v3", func(b *testing.B) { - benchmarkPlanner(b, V3, filename, vschema) + var testCases []testCase + for tc := range iterateExecFile(filename) { + testCases = append(testCases, tc) + } + b.Run(filename+"-v3", func(b *testing.B) { + benchmarkPlanner(b, V3, testCases, vschema) }) - b.Run(filename+"v4", func(b *testing.B) { - benchmarkPlanner(b, V4, filename, vschema) + b.Run(filename+"-v4", func(b *testing.B) { + benchmarkPlanner(b, V4, testCases, vschema) }) - b.Run(filename+"v4greedy", func(b *testing.B) { - benchmarkPlanner(b, V4GreedyOnly, filename, vschema) + b.Run(filename+"-v4greedy", func(b *testing.B) { + benchmarkPlanner(b, V4GreedyOnly, testCases, vschema) }) - b.Run(filename+"v4left2right", func(b *testing.B) { - benchmarkPlanner(b, V4Left2Right, filename, vschema) + b.Run(filename+"-v4left2right", func(b *testing.B) { + benchmarkPlanner(b, V4Left2Right, testCases, vschema) }) } } -func benchmarkPlanner(b *testing.B, version PlannerVersion, filename string, vschema *vschemaWrapper) { +func benchmarkPlanner(b *testing.B, version PlannerVersion, testCases []testCase, vschema *vschemaWrapper) { for n := 0; n < b.N; n++ { - for tcase := range iterateExecFile(filename) { + for _, tcase := range testCases { if tcase.output2ndPlanner != "" { vschema.version = version _, _ = TestBuilder(tcase.input, vschema) From 1c9c5ebc9b55f641b4e28d854009377198321f33 Mon Sep 17 00:00:00 2001 From: GuptaManan100 Date: Wed, 30 Dec 2020 13:50:35 +0530 Subject: [PATCH 26/40] added a new version of greedy optimizer using priority queue Signed-off-by: GuptaManan100 --- go/vt/proto/query/query.pb.go | 427 +++++++++++---------- go/vt/vtgate/planbuilder/builder.go | 2 + go/vt/vtgate/planbuilder/plan_test.go | 4 + go/vt/vtgate/planbuilder/route_planning.go | 120 ++++++ proto/query.proto | 1 + 5 files changed, 342 insertions(+), 212 deletions(-) diff --git a/go/vt/proto/query/query.pb.go b/go/vt/proto/query/query.pb.go index b6d553da9e9..24b17a9b244 100644 --- a/go/vt/proto/query/query.pb.go +++ b/go/vt/proto/query/query.pb.go @@ -465,11 +465,12 @@ func (ExecuteOptions_TransactionIsolation) EnumDescriptor() ([]byte, []int) { type ExecuteOptions_PlannerVersion int32 const ( - ExecuteOptions_DEFAULT_PLANNER ExecuteOptions_PlannerVersion = 0 - ExecuteOptions_V3 ExecuteOptions_PlannerVersion = 1 - ExecuteOptions_V4 ExecuteOptions_PlannerVersion = 2 - ExecuteOptions_V4Greedy ExecuteOptions_PlannerVersion = 3 - ExecuteOptions_V4Left2Right ExecuteOptions_PlannerVersion = 4 + ExecuteOptions_DEFAULT_PLANNER ExecuteOptions_PlannerVersion = 0 + ExecuteOptions_V3 ExecuteOptions_PlannerVersion = 1 + ExecuteOptions_V4 ExecuteOptions_PlannerVersion = 2 + ExecuteOptions_V4Greedy ExecuteOptions_PlannerVersion = 3 + ExecuteOptions_V4Left2Right ExecuteOptions_PlannerVersion = 4 + ExecuteOptions_V4GreedyOptimized ExecuteOptions_PlannerVersion = 5 ) var ExecuteOptions_PlannerVersion_name = map[int32]string{ @@ -478,14 +479,16 @@ var ExecuteOptions_PlannerVersion_name = map[int32]string{ 2: "V4", 3: "V4Greedy", 4: "V4Left2Right", + 5: "V4GreedyOptimized", } var ExecuteOptions_PlannerVersion_value = map[string]int32{ - "DEFAULT_PLANNER": 0, - "V3": 1, - "V4": 2, - "V4Greedy": 3, - "V4Left2Right": 4, + "DEFAULT_PLANNER": 0, + "V3": 1, + "V4": 2, + "V4Greedy": 3, + "V4Left2Right": 4, + "V4GreedyOptimized": 5, } func (x ExecuteOptions_PlannerVersion) String() string { @@ -4370,208 +4373,208 @@ func init() { func init() { proto.RegisterFile("query.proto", fileDescriptor_5c6ac9b241082464) } var fileDescriptor_5c6ac9b241082464 = []byte{ - // 3235 bytes of a gzipped FileDescriptorProto + // 3248 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x5a, 0x4b, 0x70, 0x1b, 0x5b, 0x5a, 0x76, 0xb7, 0x1e, 0x96, 0x7e, 0x59, 0xf2, 0xf1, 0xb1, 0x9d, 0xab, 0xeb, 0xdc, 0x87, 0xa7, - 0x67, 0x32, 0x63, 0x0c, 0x38, 0x89, 0xe3, 0x09, 0xe1, 0xce, 0x00, 0x69, 0xcb, 0xed, 0x5c, 0x25, - 0x52, 0x4b, 0x39, 0x6a, 0x39, 0x93, 0x14, 0x55, 0x5d, 0x6d, 0xe9, 0x44, 0xee, 0x72, 0x4b, 0xad, - 0x74, 0xb7, 0x9d, 0x68, 0x17, 0x18, 0x86, 0xe1, 0xcd, 0xf0, 0x66, 0x98, 0x62, 0x8a, 0x2a, 0x16, - 0x14, 0x1b, 0xd6, 0xac, 0x59, 0xdc, 0x05, 0x0b, 0xaa, 0x58, 0x02, 0x55, 0x3c, 0x16, 0x14, 0xac, - 0xa6, 0x28, 0x16, 0x2c, 0x58, 0x50, 0xd4, 0x79, 0x74, 0x4b, 0xb2, 0x75, 0x13, 0x4f, 0x86, 0x29, - 0x2a, 0xb9, 0x59, 0xe9, 0xfc, 0x8f, 0xf3, 0xf8, 0xbf, 0xf3, 0x9f, 0xff, 0x3f, 0xfd, 0xeb, 0x40, - 0xe1, 0xc9, 0x09, 0x0d, 0x46, 0x5b, 0xc3, 0xc0, 0x8f, 0x7c, 0x9c, 0xe1, 0xc4, 0x5a, 0x29, 0xf2, - 0x87, 0x7e, 0xd7, 0x89, 0x1c, 0xc1, 0x5e, 0x2b, 0x9c, 0x46, 0xc1, 0xb0, 0x23, 0x08, 0xed, 0x1b, - 0x0a, 0x64, 0x2d, 0x27, 0xe8, 0xd1, 0x08, 0xaf, 0x41, 0xee, 0x98, 0x8e, 0xc2, 0xa1, 0xd3, 0xa1, - 0x65, 0x65, 0x5d, 0xd9, 0xc8, 0x93, 0x84, 0xc6, 0x2b, 0x90, 0x09, 0x8f, 0x9c, 0xa0, 0x5b, 0x56, - 0xb9, 0x40, 0x10, 0xf8, 0xcb, 0x50, 0x88, 0x9c, 0x43, 0x8f, 0x46, 0x76, 0x34, 0x1a, 0xd2, 0x72, - 0x6a, 0x5d, 0xd9, 0x28, 0x6d, 0xaf, 0x6c, 0x25, 0xf3, 0x59, 0x5c, 0x68, 0x8d, 0x86, 0x94, 0x40, - 0x94, 0xb4, 0x31, 0x86, 0x74, 0x87, 0x7a, 0x5e, 0x39, 0xcd, 0xc7, 0xe2, 0x6d, 0x6d, 0x0f, 0x4a, - 0x07, 0xd6, 0x1d, 0x27, 0xa2, 0x15, 0xc7, 0xf3, 0x68, 0x50, 0xdd, 0x63, 0xcb, 0x39, 0x09, 0x69, - 0x30, 0x70, 0xfa, 0xc9, 0x72, 0x62, 0x1a, 0x5f, 0x82, 0x6c, 0x2f, 0xf0, 0x4f, 0x86, 0x61, 0x59, - 0x5d, 0x4f, 0x6d, 0xe4, 0x89, 0xa4, 0xb4, 0x9f, 0x05, 0x30, 0x4e, 0xe9, 0x20, 0xb2, 0xfc, 0x63, - 0x3a, 0xc0, 0xef, 0x41, 0x3e, 0x72, 0xfb, 0x34, 0x8c, 0x9c, 0xfe, 0x90, 0x0f, 0x91, 0x22, 0x63, - 0xc6, 0xa7, 0x98, 0xb4, 0x06, 0xb9, 0xa1, 0x1f, 0xba, 0x91, 0xeb, 0x0f, 0xb8, 0x3d, 0x79, 0x92, - 0xd0, 0xda, 0x4f, 0x43, 0xe6, 0xc0, 0xf1, 0x4e, 0x28, 0xfe, 0x10, 0xd2, 0xdc, 0x60, 0x85, 0x1b, - 0x5c, 0xd8, 0x12, 0xa0, 0x73, 0x3b, 0xb9, 0x80, 0x8d, 0x7d, 0xca, 0x34, 0xf9, 0xd8, 0x0b, 0x44, - 0x10, 0xda, 0x31, 0x2c, 0xec, 0xba, 0x83, 0xee, 0x81, 0x13, 0xb8, 0x0c, 0x8c, 0x57, 0x1c, 0x06, - 0x7f, 0x01, 0xb2, 0xbc, 0x11, 0x96, 0x53, 0xeb, 0xa9, 0x8d, 0xc2, 0xf6, 0x82, 0xec, 0xc8, 0xd7, - 0x46, 0xa4, 0x4c, 0xfb, 0x2b, 0x05, 0x60, 0xd7, 0x3f, 0x19, 0x74, 0xef, 0x33, 0x21, 0x46, 0x90, - 0x0a, 0x9f, 0x78, 0x12, 0x48, 0xd6, 0xc4, 0xf7, 0xa0, 0x74, 0xe8, 0x0e, 0xba, 0xf6, 0xa9, 0x5c, - 0x8e, 0xc0, 0xb2, 0xb0, 0xfd, 0x05, 0x39, 0xdc, 0xb8, 0xf3, 0xd6, 0xe4, 0xaa, 0x43, 0x63, 0x10, - 0x05, 0x23, 0x52, 0x3c, 0x9c, 0xe4, 0xad, 0xb5, 0x01, 0x9f, 0x57, 0x62, 0x93, 0x1e, 0xd3, 0x51, - 0x3c, 0xe9, 0x31, 0x1d, 0xe1, 0x1f, 0x99, 0xb4, 0xa8, 0xb0, 0xbd, 0x1c, 0xcf, 0x35, 0xd1, 0x57, - 0x9a, 0xf9, 0x91, 0x7a, 0x4b, 0xd1, 0xfe, 0x31, 0x0b, 0x25, 0xe3, 0x19, 0xed, 0x9c, 0x44, 0xb4, - 0x31, 0x64, 0x7b, 0x10, 0xe2, 0x3a, 0x2c, 0xba, 0x83, 0x8e, 0x77, 0xd2, 0xa5, 0x5d, 0xfb, 0xb1, - 0x4b, 0xbd, 0x6e, 0xc8, 0xfd, 0xa8, 0x94, 0xac, 0x7b, 0x5a, 0x7f, 0xab, 0x2a, 0x95, 0xf7, 0xb9, - 0x2e, 0x29, 0xb9, 0x53, 0x34, 0xde, 0x84, 0xa5, 0x8e, 0xe7, 0xd2, 0x41, 0x64, 0x3f, 0x66, 0xf6, - 0xda, 0x81, 0xff, 0x34, 0x2c, 0x67, 0xd6, 0x95, 0x8d, 0x1c, 0x59, 0x14, 0x82, 0x7d, 0xc6, 0x27, - 0xfe, 0xd3, 0x10, 0x7f, 0x04, 0xb9, 0xa7, 0x7e, 0x70, 0xec, 0xf9, 0x4e, 0xb7, 0x9c, 0xe5, 0x73, - 0x7e, 0x30, 0x7b, 0xce, 0x07, 0x52, 0x8b, 0x24, 0xfa, 0x78, 0x03, 0x50, 0xf8, 0xc4, 0xb3, 0x43, - 0xea, 0xd1, 0x4e, 0x64, 0x7b, 0x6e, 0xdf, 0x8d, 0xca, 0x39, 0xee, 0x92, 0xa5, 0xf0, 0x89, 0xd7, - 0xe2, 0xec, 0x1a, 0xe3, 0x62, 0x1b, 0x56, 0xa3, 0xc0, 0x19, 0x84, 0x4e, 0x87, 0x0d, 0x66, 0xbb, - 0xa1, 0xef, 0x39, 0xdc, 0x1d, 0xf3, 0x7c, 0xca, 0xcd, 0xd9, 0x53, 0x5a, 0xe3, 0x2e, 0xd5, 0xb8, - 0x07, 0x59, 0x89, 0x66, 0x70, 0xf1, 0x75, 0x58, 0x0d, 0x8f, 0xdd, 0xa1, 0xcd, 0xc7, 0xb1, 0x87, - 0x9e, 0x33, 0xb0, 0x3b, 0x4e, 0xe7, 0x88, 0x96, 0x81, 0x9b, 0x8d, 0x99, 0x90, 0xef, 0x7b, 0xd3, - 0x73, 0x06, 0x15, 0x26, 0x61, 0xa0, 0x33, 0xbd, 0x01, 0x0d, 0xec, 0x53, 0x1a, 0x84, 0x6c, 0x35, - 0x85, 0x17, 0x81, 0xde, 0x14, 0xca, 0x07, 0x42, 0x97, 0x94, 0x86, 0x53, 0xb4, 0xf6, 0x15, 0x28, - 0x4d, 0x6f, 0x0b, 0x5e, 0x82, 0xa2, 0xf5, 0xb0, 0x69, 0xd8, 0xba, 0xb9, 0x67, 0x9b, 0x7a, 0xdd, - 0x40, 0x73, 0xb8, 0x08, 0x79, 0xce, 0x6a, 0x98, 0xb5, 0x87, 0x48, 0xc1, 0xf3, 0x90, 0xd2, 0x6b, - 0x35, 0xa4, 0x6a, 0xb7, 0x20, 0x17, 0xe3, 0x8b, 0x17, 0xa1, 0xd0, 0x36, 0x5b, 0x4d, 0xa3, 0x52, - 0xdd, 0xaf, 0x1a, 0x7b, 0x68, 0x0e, 0xe7, 0x20, 0xdd, 0xa8, 0x59, 0x4d, 0xa4, 0x88, 0x96, 0xde, - 0x44, 0x2a, 0xeb, 0xb9, 0xb7, 0xab, 0xa3, 0x94, 0xf6, 0x67, 0x0a, 0xac, 0xcc, 0xc2, 0x09, 0x17, - 0x60, 0x7e, 0xcf, 0xd8, 0xd7, 0xdb, 0x35, 0x0b, 0xcd, 0xe1, 0x65, 0x58, 0x24, 0x46, 0xd3, 0xd0, - 0x2d, 0x7d, 0xb7, 0x66, 0xd8, 0xc4, 0xd0, 0xf7, 0x90, 0x82, 0x31, 0x94, 0x58, 0xcb, 0xae, 0x34, - 0xea, 0xf5, 0xaa, 0x65, 0x19, 0x7b, 0x48, 0xc5, 0x2b, 0x80, 0x38, 0xaf, 0x6d, 0x8e, 0xb9, 0x29, - 0x8c, 0x60, 0xa1, 0x65, 0x90, 0xaa, 0x5e, 0xab, 0x3e, 0x62, 0x03, 0xa0, 0x34, 0xfe, 0x1c, 0xbc, - 0x5f, 0x69, 0x98, 0xad, 0x6a, 0xcb, 0x32, 0x4c, 0xcb, 0x6e, 0x99, 0x7a, 0xb3, 0xf5, 0x71, 0xc3, - 0xe2, 0x23, 0x0b, 0xe3, 0x32, 0xb8, 0x04, 0xa0, 0xb7, 0xad, 0x86, 0x18, 0x07, 0x65, 0xb5, 0x36, - 0x94, 0xa6, 0x21, 0x64, 0xab, 0x92, 0x4b, 0xb4, 0x9b, 0x35, 0xdd, 0x34, 0x0d, 0x82, 0xe6, 0x70, - 0x16, 0xd4, 0x83, 0x1b, 0x48, 0xe1, 0xbf, 0x3b, 0x48, 0xc5, 0x0b, 0x90, 0x3b, 0xd8, 0xb9, 0x13, - 0x50, 0xda, 0x1d, 0x89, 0x95, 0x1c, 0xec, 0xd4, 0xe8, 0xe3, 0x68, 0x9b, 0xb8, 0xbd, 0xa3, 0x08, - 0xa5, 0xef, 0xa6, 0x73, 0x0a, 0x52, 0xef, 0xa6, 0x73, 0x2a, 0x4a, 0xdd, 0x4d, 0xe7, 0x52, 0x28, - 0xad, 0xfd, 0xa5, 0x0a, 0x19, 0xbe, 0x05, 0x2c, 0x28, 0x4f, 0x84, 0x5a, 0xde, 0x4e, 0x02, 0x94, - 0xfa, 0x82, 0x00, 0xc5, 0xe3, 0xba, 0x0c, 0x95, 0x82, 0xc0, 0x97, 0x21, 0xef, 0x07, 0x3d, 0x5b, - 0x48, 0x44, 0x90, 0xcf, 0xf9, 0x41, 0x8f, 0x67, 0x03, 0x16, 0x60, 0x59, 0x6e, 0x38, 0x74, 0x42, - 0xca, 0xcf, 0x59, 0x9e, 0x24, 0x34, 0x7e, 0x17, 0x98, 0x9e, 0xcd, 0xd7, 0x91, 0xe5, 0xb2, 0x79, - 0x3f, 0xe8, 0x99, 0x6c, 0x29, 0x9f, 0x87, 0x62, 0xc7, 0xf7, 0x4e, 0xfa, 0x03, 0xdb, 0xa3, 0x83, - 0x5e, 0x74, 0x54, 0x9e, 0x5f, 0x57, 0x36, 0x8a, 0x64, 0x41, 0x30, 0x6b, 0x9c, 0x87, 0xcb, 0x30, - 0xdf, 0x39, 0x72, 0x82, 0x90, 0x8a, 0xb3, 0x55, 0x24, 0x31, 0xc9, 0x67, 0xa5, 0x1d, 0xb7, 0xef, - 0x78, 0x21, 0x3f, 0x47, 0x45, 0x92, 0xd0, 0xcc, 0x88, 0xc7, 0x9e, 0xd3, 0x0b, 0xb9, 0xff, 0x17, - 0x89, 0x20, 0xf0, 0x87, 0x50, 0x90, 0x13, 0x72, 0x08, 0x0a, 0x7c, 0x39, 0x20, 0x58, 0x0c, 0x01, - 0xed, 0x27, 0x20, 0x45, 0xfc, 0xa7, 0x6c, 0x4e, 0xb1, 0xa2, 0xb0, 0xac, 0xac, 0xa7, 0x36, 0x30, - 0x89, 0x49, 0x96, 0xa4, 0x64, 0x9c, 0x16, 0xe1, 0x3b, 0x8e, 0xcc, 0xdf, 0x51, 0xa0, 0xc0, 0xcf, - 0x17, 0xa1, 0xe1, 0x89, 0x17, 0xb1, 0x78, 0x2e, 0x03, 0x99, 0x32, 0x15, 0xcf, 0xf9, 0xbe, 0x10, - 0x29, 0x63, 0x00, 0xb0, 0xd8, 0x64, 0x3b, 0x8f, 0x1f, 0xd3, 0x4e, 0x44, 0x45, 0xda, 0x4a, 0x93, - 0x05, 0xc6, 0xd4, 0x25, 0x8f, 0x21, 0xef, 0x0e, 0x42, 0x1a, 0x44, 0xb6, 0xdb, 0xe5, 0x7b, 0x92, - 0x26, 0x39, 0xc1, 0xa8, 0x76, 0xf1, 0x07, 0x90, 0xe6, 0xd1, 0x2d, 0xcd, 0x67, 0x01, 0x39, 0x0b, - 0xf1, 0x9f, 0x12, 0xce, 0xbf, 0x9b, 0xce, 0x65, 0x50, 0x56, 0xfb, 0x2a, 0x2c, 0xf0, 0xc5, 0x3d, - 0x70, 0x82, 0x81, 0x3b, 0xe8, 0xf1, 0x64, 0xed, 0x77, 0x85, 0x5f, 0x14, 0x09, 0x6f, 0x33, 0x9b, - 0xfb, 0x34, 0x0c, 0x9d, 0x1e, 0x95, 0xc9, 0x33, 0x26, 0xb5, 0x3f, 0x49, 0x41, 0xa1, 0x15, 0x05, - 0xd4, 0xe9, 0xf3, 0x3c, 0x8c, 0xbf, 0x0a, 0x10, 0x46, 0x4e, 0x44, 0xfb, 0x74, 0x10, 0xc5, 0xf6, - 0xbd, 0x27, 0x67, 0x9e, 0xd0, 0xdb, 0x6a, 0xc5, 0x4a, 0x64, 0x42, 0x1f, 0x6f, 0x43, 0x81, 0x32, - 0xb1, 0x1d, 0xb1, 0x7c, 0x2e, 0x73, 0xc6, 0x52, 0x1c, 0x72, 0x92, 0x44, 0x4f, 0x80, 0x26, 0xed, - 0xb5, 0xef, 0xaa, 0x90, 0x4f, 0x46, 0xc3, 0x3a, 0xe4, 0x3a, 0x4e, 0x44, 0x7b, 0x7e, 0x30, 0x92, - 0x69, 0xf6, 0xca, 0x8b, 0x66, 0xdf, 0xaa, 0x48, 0x65, 0x92, 0x74, 0xc3, 0xef, 0x83, 0xb8, 0xbb, - 0x08, 0xb7, 0x14, 0xf6, 0xe6, 0x39, 0x87, 0x3b, 0xe6, 0x47, 0x80, 0x87, 0x81, 0xdb, 0x77, 0x82, - 0x91, 0x7d, 0x4c, 0x47, 0x71, 0x4a, 0x4a, 0xcd, 0xd8, 0x49, 0x24, 0xf5, 0xee, 0xd1, 0x91, 0x8c, - 0x7a, 0xb7, 0xa6, 0xfb, 0x4a, 0x6f, 0x39, 0xbf, 0x3f, 0x13, 0x3d, 0x79, 0x92, 0x0f, 0xe3, 0x74, - 0x9e, 0xe1, 0x8e, 0xc5, 0x9a, 0xda, 0x97, 0x20, 0x17, 0x2f, 0x1e, 0xe7, 0x21, 0x63, 0x04, 0x81, - 0x1f, 0xa0, 0x39, 0x1e, 0xfc, 0xea, 0x35, 0x11, 0x3f, 0xf7, 0xf6, 0x58, 0xfc, 0xfc, 0x17, 0x35, - 0xc9, 0xa9, 0x84, 0x3e, 0x39, 0xa1, 0x61, 0x84, 0x7f, 0x06, 0x96, 0x29, 0x77, 0x21, 0xf7, 0x94, - 0xda, 0x1d, 0x7e, 0x01, 0x63, 0x0e, 0xa4, 0x70, 0xbc, 0x17, 0xb7, 0xc4, 0x7d, 0x31, 0xbe, 0x98, - 0x91, 0xa5, 0x44, 0x57, 0xb2, 0xba, 0xd8, 0x80, 0x65, 0xb7, 0xdf, 0xa7, 0x5d, 0xd7, 0x89, 0x26, - 0x07, 0x10, 0x1b, 0xb6, 0x1a, 0xdf, 0x4f, 0xa6, 0xee, 0x77, 0x64, 0x29, 0xe9, 0x91, 0x0c, 0x73, - 0x05, 0xb2, 0x11, 0xbf, 0x8b, 0x72, 0xdf, 0x2d, 0x6c, 0x17, 0xe3, 0x88, 0xc3, 0x99, 0x44, 0x0a, - 0xf1, 0x97, 0x40, 0xdc, 0x6c, 0x79, 0x6c, 0x19, 0x3b, 0xc4, 0xf8, 0xc2, 0x42, 0x84, 0x1c, 0x5f, - 0x81, 0xd2, 0x54, 0x2a, 0xed, 0x72, 0xc0, 0x52, 0xa4, 0x38, 0x99, 0x17, 0xbb, 0xf8, 0x2a, 0xcc, - 0xfb, 0x22, 0x71, 0xf1, 0xa8, 0x33, 0x5e, 0xf1, 0x74, 0x56, 0x23, 0xb1, 0x16, 0x8b, 0x0d, 0x01, - 0x0d, 0x69, 0x70, 0x4a, 0xbb, 0x6c, 0xd0, 0x79, 0x3e, 0x28, 0xc4, 0xac, 0x6a, 0x57, 0xfb, 0x29, - 0x58, 0x4c, 0x20, 0x0e, 0x87, 0xfe, 0x20, 0xa4, 0x78, 0x13, 0xb2, 0x01, 0x3f, 0xef, 0x12, 0x56, - 0x2c, 0xe7, 0x98, 0x88, 0x04, 0x44, 0x6a, 0x68, 0x5d, 0x58, 0x14, 0x9c, 0x07, 0x6e, 0x74, 0xc4, - 0x77, 0x12, 0x5f, 0x81, 0x0c, 0x65, 0x8d, 0x33, 0x9b, 0x42, 0x9a, 0x15, 0x2e, 0x27, 0x42, 0x3a, - 0x31, 0x8b, 0xfa, 0xd2, 0x59, 0xfe, 0x43, 0x85, 0x65, 0xb9, 0xca, 0x5d, 0x27, 0xea, 0x1c, 0xbd, - 0xa6, 0xde, 0xf0, 0xa3, 0x30, 0xcf, 0xf8, 0x6e, 0x72, 0x72, 0x66, 0xf8, 0x43, 0xac, 0xc1, 0x3c, - 0xc2, 0x09, 0xed, 0x89, 0xed, 0x97, 0x77, 0xbd, 0xa2, 0x13, 0x4e, 0xdc, 0x0c, 0x66, 0x38, 0x4e, - 0xf6, 0x25, 0x8e, 0x33, 0x7f, 0x11, 0xc7, 0xd1, 0xf6, 0x60, 0x65, 0x1a, 0x71, 0xe9, 0x1c, 0x3f, - 0x06, 0xf3, 0x62, 0x53, 0xe2, 0x18, 0x39, 0x6b, 0xdf, 0x62, 0x15, 0xed, 0x13, 0x15, 0x56, 0x64, - 0xf8, 0xfa, 0x6c, 0x9c, 0xe3, 0x09, 0x9c, 0x33, 0x17, 0x3a, 0xa0, 0x17, 0xdb, 0x3f, 0xad, 0x02, - 0xab, 0x67, 0x70, 0x7c, 0x85, 0xc3, 0xfa, 0x3d, 0x05, 0x16, 0x76, 0x69, 0xcf, 0x1d, 0xbc, 0xa6, - 0xbb, 0x30, 0x01, 0x6e, 0xfa, 0x42, 0x4e, 0x3c, 0x84, 0xa2, 0xb4, 0x57, 0xa2, 0x75, 0x1e, 0x6d, - 0x65, 0xd6, 0x69, 0xb9, 0x05, 0x0b, 0xb2, 0x5a, 0xe0, 0x78, 0xae, 0x13, 0x26, 0xf6, 0x9c, 0x29, - 0x17, 0xe8, 0x4c, 0x48, 0x64, 0x61, 0x81, 0x13, 0xda, 0xbf, 0x2a, 0x50, 0xac, 0xf8, 0xfd, 0xbe, - 0x1b, 0xbd, 0xa6, 0x18, 0x9f, 0x47, 0x28, 0x3d, 0xcb, 0x1f, 0xaf, 0x43, 0x29, 0x36, 0x53, 0x42, - 0x7b, 0x26, 0xd3, 0x28, 0xe7, 0x32, 0xcd, 0xbf, 0x29, 0xb0, 0x48, 0x7c, 0xcf, 0x3b, 0x74, 0x3a, - 0xc7, 0x6f, 0x36, 0x38, 0x37, 0x00, 0x8d, 0x0d, 0xbd, 0x28, 0x3c, 0xff, 0xad, 0x40, 0xa9, 0x19, - 0xd0, 0xa1, 0x13, 0xd0, 0x37, 0x1a, 0x1d, 0x76, 0x4d, 0xef, 0x46, 0xf2, 0x82, 0x93, 0x27, 0xbc, - 0xad, 0x2d, 0xc1, 0x62, 0x62, 0xbb, 0x00, 0x4c, 0xfb, 0x7b, 0x05, 0x56, 0x85, 0x8b, 0x49, 0x49, - 0xf7, 0x35, 0x85, 0x25, 0xb6, 0x37, 0x3d, 0x61, 0x6f, 0x19, 0x2e, 0x9d, 0xb5, 0x4d, 0x9a, 0xfd, - 0x75, 0x15, 0xde, 0x89, 0x9d, 0xe7, 0x35, 0x37, 0xfc, 0x07, 0xf0, 0x87, 0x35, 0x28, 0x9f, 0x07, - 0x41, 0x22, 0xf4, 0x2d, 0x15, 0xca, 0x95, 0x80, 0x3a, 0x11, 0x9d, 0xb8, 0x07, 0xbd, 0x39, 0xbe, - 0x81, 0xaf, 0xc3, 0xc2, 0xd0, 0x09, 0x22, 0xb7, 0xe3, 0x0e, 0x1d, 0xf6, 0x29, 0x9a, 0xe1, 0xd7, - 0xac, 0x33, 0x03, 0x4c, 0xa9, 0x68, 0x97, 0xe1, 0xdd, 0x19, 0x88, 0x48, 0xbc, 0xfe, 0x47, 0x01, - 0xdc, 0x8a, 0x9c, 0x20, 0xfa, 0x0c, 0xe4, 0xa5, 0x99, 0xce, 0xb4, 0x0a, 0xcb, 0x53, 0xf6, 0x4f, - 0xe2, 0x42, 0xa3, 0xcf, 0x44, 0x4a, 0xfa, 0x54, 0x5c, 0x26, 0xed, 0x97, 0xb8, 0xfc, 0x93, 0x02, - 0x6b, 0x15, 0x5f, 0x14, 0x3d, 0xdf, 0xc8, 0x13, 0xa6, 0xbd, 0x0f, 0x97, 0x67, 0x1a, 0x28, 0x01, - 0xf8, 0x07, 0x05, 0x2e, 0x11, 0xea, 0x74, 0xdf, 0x4c, 0xe3, 0xef, 0xc3, 0x3b, 0xe7, 0x8c, 0x93, - 0x77, 0x94, 0x9b, 0x90, 0xeb, 0xd3, 0xc8, 0x61, 0x37, 0x5c, 0x69, 0xd2, 0x5a, 0x3c, 0xee, 0x58, - 0xbb, 0x2e, 0x35, 0x48, 0xa2, 0xab, 0xfd, 0xb3, 0x0a, 0xcb, 0xfc, 0x9e, 0xfd, 0xf6, 0x23, 0xef, - 0x42, 0x55, 0x98, 0xec, 0xd9, 0xcb, 0x1f, 0x53, 0x18, 0x06, 0xd4, 0x8e, 0xab, 0x03, 0xf3, 0xfc, - 0xaf, 0x42, 0x18, 0x06, 0xf4, 0xbe, 0xe0, 0x68, 0x7f, 0xad, 0xc0, 0xca, 0x34, 0xc4, 0xc9, 0x17, - 0xcd, 0xff, 0x75, 0xb5, 0x65, 0x46, 0x48, 0x49, 0x5d, 0xe4, 0x23, 0x29, 0x7d, 0xe1, 0x8f, 0xa4, - 0xbf, 0x51, 0xa1, 0x3c, 0x69, 0xcc, 0xdb, 0x9a, 0xce, 0x74, 0x4d, 0xe7, 0xfb, 0xad, 0xf2, 0x69, - 0x7f, 0xab, 0xc0, 0xbb, 0x33, 0x00, 0xfd, 0xfe, 0x5c, 0x64, 0xa2, 0xb2, 0xa3, 0xbe, 0xb4, 0xb2, - 0xf3, 0xc3, 0x77, 0x92, 0xbf, 0x53, 0x60, 0xa5, 0x2e, 0x6a, 0xf5, 0xa2, 0xf2, 0xf1, 0xfa, 0xc6, - 0x60, 0x5e, 0x8e, 0x4f, 0x8f, 0xff, 0xad, 0xd2, 0x2a, 0xb0, 0x7a, 0xc6, 0xb4, 0x57, 0xa8, 0xe6, - 0xfc, 0x97, 0x02, 0x4b, 0x72, 0x14, 0xfd, 0xb5, 0xbd, 0xbe, 0xcc, 0x40, 0x07, 0x7f, 0x00, 0x29, - 0xb7, 0x1b, 0xdf, 0x7b, 0xa7, 0x9f, 0x0c, 0x30, 0x81, 0x76, 0x1b, 0xf0, 0xa4, 0xdd, 0xaf, 0x00, - 0xdd, 0xbf, 0xab, 0xb0, 0x4a, 0x44, 0xf4, 0x7d, 0xfb, 0xff, 0xc2, 0x0f, 0xfa, 0xff, 0xc2, 0x8b, - 0x13, 0xd7, 0x27, 0xfc, 0x32, 0x35, 0x0d, 0xf5, 0x0f, 0x2f, 0x75, 0x9d, 0x49, 0xb4, 0xa9, 0x73, - 0x89, 0xf6, 0xd5, 0xe3, 0xd1, 0x27, 0x2a, 0xac, 0x49, 0x43, 0xde, 0xde, 0x75, 0x2e, 0xee, 0x11, - 0xd9, 0x73, 0x1e, 0xf1, 0x9f, 0x0a, 0x5c, 0x9e, 0x09, 0xe4, 0xff, 0xfb, 0x8d, 0xe6, 0x8c, 0xf7, - 0xa4, 0x5f, 0xea, 0x3d, 0x99, 0x0b, 0x7b, 0xcf, 0x37, 0x55, 0x28, 0x11, 0xea, 0x51, 0x27, 0x7c, - 0xc3, 0xab, 0x7b, 0x67, 0x30, 0xcc, 0x9c, 0xab, 0x73, 0x2e, 0xc1, 0x62, 0x02, 0x84, 0xfc, 0xe0, - 0xe2, 0x1f, 0xe8, 0x2c, 0x0f, 0x7e, 0x4c, 0x1d, 0x2f, 0x8a, 0x6f, 0x82, 0xda, 0x9f, 0xaa, 0x50, - 0x24, 0x8c, 0xe3, 0xf6, 0x69, 0x2b, 0x72, 0xa2, 0x10, 0x7f, 0x0e, 0x16, 0x8e, 0xb8, 0x8a, 0x3d, - 0xf6, 0x90, 0x3c, 0x29, 0x08, 0x9e, 0xf8, 0xf7, 0x71, 0x1b, 0x56, 0x43, 0xda, 0xf1, 0x07, 0xdd, - 0xd0, 0x3e, 0xa4, 0x47, 0xee, 0xa0, 0x6b, 0xf7, 0x9d, 0x30, 0xa2, 0x01, 0x87, 0xa5, 0x48, 0x96, - 0xa5, 0x70, 0x97, 0xcb, 0xea, 0x5c, 0x84, 0xaf, 0xc1, 0xca, 0xa1, 0x3b, 0xf0, 0xfc, 0x9e, 0x3d, - 0xf4, 0x9c, 0x11, 0x0d, 0x42, 0xbb, 0xe3, 0x9f, 0x0c, 0x04, 0x1e, 0x19, 0x82, 0x85, 0xac, 0x29, - 0x44, 0x15, 0x26, 0xc1, 0x8f, 0x60, 0x73, 0xe6, 0x2c, 0xf6, 0x63, 0xd7, 0x8b, 0x68, 0x40, 0xbb, - 0x76, 0x40, 0x87, 0x9e, 0xdb, 0x11, 0xcf, 0xa1, 0x04, 0x50, 0x5f, 0x9c, 0x31, 0xf5, 0xbe, 0x54, - 0x27, 0x63, 0x6d, 0x7c, 0x19, 0xf2, 0x9d, 0xe1, 0x89, 0x7d, 0xc2, 0x1f, 0x2d, 0x30, 0xfc, 0x14, - 0x92, 0xeb, 0x0c, 0x4f, 0xda, 0x8c, 0xc6, 0x08, 0x52, 0x4f, 0x86, 0x22, 0x38, 0x2b, 0x84, 0x35, - 0xb5, 0xef, 0x29, 0x50, 0xd2, 0x7b, 0xbd, 0x80, 0xf6, 0x9c, 0x48, 0xc2, 0x74, 0x0d, 0x56, 0x04, - 0x24, 0x23, 0x5b, 0xba, 0xab, 0xb0, 0x47, 0x11, 0xf6, 0x48, 0x99, 0xf0, 0x55, 0x61, 0xcf, 0x0e, - 0x5c, 0x3a, 0x19, 0xcc, 0xec, 0xa3, 0xf2, 0x3e, 0x2b, 0x89, 0x74, 0xb2, 0xd7, 0x4f, 0xc2, 0xbb, - 0xb3, 0x51, 0xe8, 0xbb, 0xe2, 0x49, 0x62, 0x91, 0x5c, 0x9a, 0x61, 0x74, 0xdd, 0x1d, 0xbc, 0xa0, - 0xab, 0xf3, 0x8c, 0xe3, 0xf5, 0x29, 0x5d, 0x9d, 0x67, 0xda, 0x9f, 0x27, 0xff, 0x29, 0xc6, 0xee, - 0x92, 0x04, 0x8e, 0xd8, 0x91, 0x95, 0x17, 0x39, 0x72, 0x19, 0xe6, 0x99, 0x33, 0xba, 0x83, 0x1e, - 0x37, 0x2e, 0x47, 0x62, 0x12, 0xb7, 0xe0, 0x8b, 0xd2, 0x76, 0xfa, 0x2c, 0xa2, 0xc1, 0xc0, 0xf1, - 0xbc, 0x91, 0x2d, 0xca, 0x8f, 0x83, 0x88, 0x76, 0xed, 0xf1, 0x13, 0x4d, 0x11, 0x3e, 0x3e, 0x2f, - 0xb4, 0x8d, 0x44, 0x99, 0x24, 0xba, 0x56, 0xf2, 0x78, 0xf3, 0x2b, 0x50, 0x0a, 0xa4, 0x13, 0xdb, - 0x21, 0xdb, 0x1e, 0x19, 0x72, 0x57, 0xe2, 0x57, 0x13, 0x93, 0x1e, 0x4e, 0x8a, 0xc1, 0x94, 0xc3, - 0xbf, 0x72, 0xc0, 0xb9, 0x9b, 0xce, 0x65, 0xd1, 0xbc, 0xf6, 0x17, 0x0a, 0x2c, 0xcf, 0xf8, 0x76, - 0x4f, 0x0a, 0x03, 0xca, 0x44, 0xdd, 0xf1, 0xc7, 0x21, 0xc3, 0x1f, 0xb4, 0xc8, 0x37, 0x54, 0xef, - 0x9c, 0xff, 0xf4, 0xe7, 0x8f, 0x4f, 0x88, 0xd0, 0x62, 0x67, 0x91, 0xdb, 0xd4, 0xe1, 0x85, 0xc7, - 0x38, 0xa2, 0x16, 0x18, 0x4f, 0xd4, 0x22, 0xcf, 0x57, 0x32, 0xd3, 0x2f, 0xad, 0x64, 0x6e, 0xfe, - 0x76, 0x0a, 0xf2, 0xf5, 0x51, 0xeb, 0x89, 0xb7, 0xef, 0x39, 0x3d, 0xfe, 0x3a, 0xa4, 0xde, 0xb4, - 0x1e, 0xa2, 0x39, 0xbc, 0x04, 0x45, 0xb3, 0x61, 0xd9, 0x66, 0xbb, 0x56, 0xb3, 0xf7, 0x6b, 0xfa, - 0x1d, 0xa4, 0x60, 0x04, 0x0b, 0x4d, 0x52, 0xb5, 0xef, 0x19, 0x0f, 0x05, 0x47, 0xc5, 0xcb, 0xb0, - 0xd8, 0x36, 0xab, 0xf7, 0xdb, 0xc6, 0x98, 0x99, 0xc6, 0xab, 0xb0, 0x54, 0x6f, 0xd7, 0xac, 0x6a, - 0xb3, 0x36, 0xc1, 0xce, 0xe1, 0x22, 0xe4, 0x77, 0x6b, 0x8d, 0x5d, 0x41, 0x22, 0x36, 0x7e, 0xdb, - 0x6c, 0x55, 0xef, 0x98, 0xc6, 0x9e, 0x60, 0xad, 0x33, 0xd6, 0x23, 0x83, 0x34, 0xf6, 0xab, 0xf1, - 0x94, 0xb7, 0x31, 0x82, 0xc2, 0x6e, 0xd5, 0xd4, 0x89, 0x1c, 0xe5, 0xb9, 0x82, 0x4b, 0x90, 0x37, - 0xcc, 0x76, 0x5d, 0xd2, 0x2a, 0x2e, 0xc3, 0xb2, 0xde, 0xb6, 0x1a, 0x76, 0xd5, 0xac, 0x10, 0xa3, - 0x6e, 0x98, 0x96, 0x94, 0xa4, 0xf1, 0x32, 0x94, 0xac, 0x6a, 0xdd, 0x68, 0x59, 0x7a, 0xbd, 0x29, - 0x99, 0x6c, 0x15, 0xb9, 0x96, 0x11, 0xeb, 0x20, 0xbc, 0x06, 0xab, 0x66, 0xc3, 0x8e, 0x9f, 0xcf, - 0x1d, 0xe8, 0xb5, 0xb6, 0x21, 0x65, 0xeb, 0xf8, 0x1d, 0xc0, 0x0d, 0xd3, 0x6e, 0x37, 0xf7, 0x74, - 0xcb, 0xb0, 0xcd, 0xc6, 0x03, 0x29, 0xb8, 0x8d, 0x4b, 0x90, 0x1b, 0xaf, 0xe0, 0x39, 0x43, 0xa1, - 0xd8, 0xd4, 0x89, 0x35, 0x36, 0xf6, 0xf9, 0x73, 0x06, 0x16, 0xdc, 0x21, 0x8d, 0x76, 0x73, 0xac, - 0xb6, 0x04, 0x05, 0x09, 0x96, 0x64, 0xa5, 0x19, 0x6b, 0xb7, 0x6a, 0x56, 0x92, 0xf5, 0x3d, 0xcf, - 0xad, 0xa9, 0x48, 0xd9, 0x3c, 0x86, 0x34, 0xdf, 0x8e, 0x1c, 0xa4, 0xcd, 0x86, 0x69, 0xa0, 0x39, - 0xbc, 0x08, 0x50, 0x6d, 0x55, 0x4d, 0xcb, 0xb8, 0x43, 0xf4, 0x1a, 0x33, 0x9b, 0x33, 0x62, 0x00, - 0x99, 0xb5, 0x0b, 0x30, 0x5f, 0x6d, 0xed, 0xd7, 0x1a, 0xba, 0x25, 0xcd, 0xac, 0xb6, 0xee, 0xb7, - 0x1b, 0x16, 0x13, 0x22, 0x5c, 0x80, 0x6c, 0xb5, 0x65, 0x19, 0x5f, 0xb3, 0x98, 0x5d, 0x5c, 0x26, - 0x50, 0x45, 0xcf, 0x6f, 0x6f, 0x7e, 0x3b, 0x05, 0x69, 0xfe, 0xf6, 0xba, 0x08, 0x79, 0xbe, 0xdb, - 0xd6, 0xc3, 0x26, 0x9b, 0x32, 0x0f, 0xe9, 0xaa, 0x69, 0xdd, 0x42, 0x3f, 0xa7, 0x62, 0x80, 0x4c, - 0x9b, 0xb7, 0x7f, 0x3e, 0xcb, 0xda, 0x55, 0xd3, 0xba, 0x7e, 0x13, 0x7d, 0x5d, 0x65, 0xc3, 0xb6, - 0x05, 0xf1, 0x0b, 0xb1, 0x60, 0x7b, 0x07, 0x7d, 0x23, 0x11, 0x6c, 0xef, 0xa0, 0x5f, 0x8c, 0x05, - 0x37, 0xb6, 0xd1, 0x37, 0x13, 0xc1, 0x8d, 0x6d, 0xf4, 0x4b, 0xb1, 0xe0, 0xe6, 0x0e, 0xfa, 0xe5, - 0x44, 0x70, 0x73, 0x07, 0xfd, 0x4a, 0x96, 0xd9, 0xc2, 0x2d, 0xb9, 0xb1, 0x8d, 0x7e, 0x35, 0x97, - 0x50, 0x37, 0x77, 0xd0, 0xaf, 0xe5, 0xd8, 0xfe, 0x27, 0xbb, 0x8a, 0x7e, 0x1d, 0xb1, 0x65, 0xb2, - 0x0d, 0x42, 0xbf, 0xc1, 0x9b, 0x4c, 0x84, 0x7e, 0x13, 0x31, 0x1b, 0x19, 0x97, 0x93, 0xdf, 0xe2, - 0x92, 0x87, 0x86, 0x4e, 0xd0, 0x6f, 0x65, 0xc5, 0x83, 0xce, 0x4a, 0xb5, 0xae, 0xd7, 0x10, 0xe6, - 0x3d, 0x18, 0x2a, 0xbf, 0x73, 0x8d, 0x35, 0x99, 0x7b, 0xa2, 0xdf, 0x6d, 0xb2, 0x09, 0x0f, 0x74, - 0x52, 0xf9, 0x58, 0x27, 0xe8, 0xf7, 0xae, 0xb1, 0x09, 0x0f, 0x74, 0x22, 0xf1, 0xfa, 0xfd, 0x26, - 0x53, 0xe4, 0xa2, 0x3f, 0xb8, 0xc6, 0x16, 0x2d, 0xf9, 0x7f, 0xd8, 0xc4, 0x39, 0x48, 0xed, 0x56, - 0x2d, 0xf4, 0x6d, 0x3e, 0x1b, 0x73, 0x51, 0xf4, 0x47, 0x88, 0x31, 0x5b, 0x86, 0x85, 0xbe, 0xc3, - 0x98, 0x19, 0xab, 0xdd, 0xac, 0x19, 0xe8, 0x3d, 0xb6, 0xb8, 0x3b, 0x46, 0xa3, 0x6e, 0x58, 0xe4, - 0x21, 0xfa, 0x63, 0xae, 0x7e, 0xb7, 0xd5, 0x30, 0xd1, 0x77, 0x11, 0x2e, 0x01, 0x18, 0x5f, 0x6b, - 0x12, 0xa3, 0xd5, 0xaa, 0x36, 0x4c, 0xf4, 0xe1, 0xe6, 0x3e, 0xa0, 0xb3, 0xe1, 0x80, 0x19, 0xd0, - 0x36, 0xef, 0x99, 0x8d, 0x07, 0x26, 0x9a, 0x63, 0x44, 0x93, 0x18, 0x4d, 0x9d, 0x18, 0x48, 0xc1, - 0x00, 0x59, 0xf9, 0x4c, 0x94, 0xbf, 0xf7, 0x24, 0x8d, 0x5a, 0x6d, 0x57, 0xaf, 0xdc, 0x43, 0xa9, - 0xdd, 0x2f, 0xc3, 0xa2, 0xeb, 0x6f, 0x9d, 0xba, 0x11, 0x0d, 0x43, 0xf1, 0xba, 0xff, 0x91, 0x26, - 0x29, 0xd7, 0xbf, 0x2a, 0x5a, 0x57, 0x7b, 0xfe, 0xd5, 0xd3, 0xe8, 0x2a, 0x97, 0x5e, 0xe5, 0x11, - 0xe3, 0x30, 0xcb, 0x89, 0x1b, 0xff, 0x1b, 0x00, 0x00, 0xff, 0xff, 0xbd, 0xb2, 0x28, 0xeb, 0x3b, - 0x30, 0x00, 0x00, + 0x67, 0xee, 0x8c, 0x31, 0xe0, 0x24, 0x8e, 0x27, 0x84, 0x3b, 0x03, 0xa4, 0x2d, 0xb7, 0x73, 0x95, + 0x48, 0x2d, 0xe5, 0xa8, 0xe5, 0x4c, 0x52, 0x54, 0x75, 0xb5, 0xa5, 0x13, 0xb9, 0xcb, 0x2d, 0xb5, + 0xd2, 0xdd, 0x76, 0x22, 0x56, 0x81, 0x61, 0x18, 0xde, 0x0c, 0x6f, 0x86, 0x29, 0xa6, 0xa8, 0x62, + 0x41, 0xb1, 0x61, 0xcd, 0x9a, 0xc5, 0x5d, 0xb0, 0xa0, 0x8a, 0x25, 0xb0, 0x00, 0x16, 0x14, 0xac, + 0xa6, 0x28, 0xaa, 0x60, 0xc1, 0x82, 0xa2, 0xce, 0xa3, 0x5b, 0x92, 0xad, 0x9b, 0x78, 0x32, 0xdc, + 0x9a, 0x4a, 0x6e, 0x56, 0x3a, 0xff, 0xe3, 0x3c, 0xfe, 0xef, 0xfc, 0xe7, 0xff, 0x4f, 0xff, 0x3a, + 0x50, 0x78, 0x7c, 0x42, 0x83, 0xd1, 0xd6, 0x30, 0xf0, 0x23, 0x1f, 0x67, 0x38, 0xb1, 0x56, 0x8a, + 0xfc, 0xa1, 0xdf, 0x75, 0x22, 0x47, 0xb0, 0xd7, 0x0a, 0xa7, 0x51, 0x30, 0xec, 0x08, 0x42, 0xfb, + 0x86, 0x02, 0x59, 0xcb, 0x09, 0x7a, 0x34, 0xc2, 0x6b, 0x90, 0x3b, 0xa6, 0xa3, 0x70, 0xe8, 0x74, + 0x68, 0x59, 0x59, 0x57, 0x36, 0xf2, 0x24, 0xa1, 0xf1, 0x0a, 0x64, 0xc2, 0x23, 0x27, 0xe8, 0x96, + 0x55, 0x2e, 0x10, 0x04, 0xfe, 0x32, 0x14, 0x22, 0xe7, 0xd0, 0xa3, 0x91, 0x1d, 0x8d, 0x86, 0xb4, + 0x9c, 0x5a, 0x57, 0x36, 0x4a, 0xdb, 0x2b, 0x5b, 0xc9, 0x7c, 0x16, 0x17, 0x5a, 0xa3, 0x21, 0x25, + 0x10, 0x25, 0x6d, 0x8c, 0x21, 0xdd, 0xa1, 0x9e, 0x57, 0x4e, 0xf3, 0xb1, 0x78, 0x5b, 0xdb, 0x83, + 0xd2, 0x81, 0x75, 0xdb, 0x89, 0x68, 0xc5, 0xf1, 0x3c, 0x1a, 0x54, 0xf7, 0xd8, 0x72, 0x4e, 0x42, + 0x1a, 0x0c, 0x9c, 0x7e, 0xb2, 0x9c, 0x98, 0xc6, 0x97, 0x20, 0xdb, 0x0b, 0xfc, 0x93, 0x61, 0x58, + 0x56, 0xd7, 0x53, 0x1b, 0x79, 0x22, 0x29, 0xed, 0x67, 0x01, 0x8c, 0x53, 0x3a, 0x88, 0x2c, 0xff, + 0x98, 0x0e, 0xf0, 0x3b, 0x90, 0x8f, 0xdc, 0x3e, 0x0d, 0x23, 0xa7, 0x3f, 0xe4, 0x43, 0xa4, 0xc8, + 0x98, 0xf1, 0x09, 0x26, 0xad, 0x41, 0x6e, 0xe8, 0x87, 0x6e, 0xe4, 0xfa, 0x03, 0x6e, 0x4f, 0x9e, + 0x24, 0xb4, 0xf6, 0xd3, 0x90, 0x39, 0x70, 0xbc, 0x13, 0x8a, 0xdf, 0x87, 0x34, 0x37, 0x58, 0xe1, + 0x06, 0x17, 0xb6, 0x04, 0xe8, 0xdc, 0x4e, 0x2e, 0x60, 0x63, 0x9f, 0x32, 0x4d, 0x3e, 0xf6, 0x02, + 0x11, 0x84, 0x76, 0x0c, 0x0b, 0xbb, 0xee, 0xa0, 0x7b, 0xe0, 0x04, 0x2e, 0x03, 0xe3, 0x25, 0x87, + 0xc1, 0x5f, 0x80, 0x2c, 0x6f, 0x84, 0xe5, 0xd4, 0x7a, 0x6a, 0xa3, 0xb0, 0xbd, 0x20, 0x3b, 0xf2, + 0xb5, 0x11, 0x29, 0xd3, 0xfe, 0x5a, 0x01, 0xd8, 0xf5, 0x4f, 0x06, 0xdd, 0x7b, 0x4c, 0x88, 0x11, + 0xa4, 0xc2, 0xc7, 0x9e, 0x04, 0x92, 0x35, 0xf1, 0x5d, 0x28, 0x1d, 0xba, 0x83, 0xae, 0x7d, 0x2a, + 0x97, 0x23, 0xb0, 0x2c, 0x6c, 0x7f, 0x41, 0x0e, 0x37, 0xee, 0xbc, 0x35, 0xb9, 0xea, 0xd0, 0x18, + 0x44, 0xc1, 0x88, 0x14, 0x0f, 0x27, 0x79, 0x6b, 0x6d, 0xc0, 0xe7, 0x95, 0xd8, 0xa4, 0xc7, 0x74, + 0x14, 0x4f, 0x7a, 0x4c, 0x47, 0xf8, 0x47, 0x26, 0x2d, 0x2a, 0x6c, 0x2f, 0xc7, 0x73, 0x4d, 0xf4, + 0x95, 0x66, 0x7e, 0xa8, 0xde, 0x54, 0xb4, 0xff, 0xca, 0x42, 0xc9, 0x78, 0x4a, 0x3b, 0x27, 0x11, + 0x6d, 0x0c, 0xd9, 0x1e, 0x84, 0xb8, 0x0e, 0x8b, 0xee, 0xa0, 0xe3, 0x9d, 0x74, 0x69, 0xd7, 0x7e, + 0xe4, 0x52, 0xaf, 0x1b, 0x72, 0x3f, 0x2a, 0x25, 0xeb, 0x9e, 0xd6, 0xdf, 0xaa, 0x4a, 0xe5, 0x7d, + 0xae, 0x4b, 0x4a, 0xee, 0x14, 0x8d, 0x37, 0x61, 0xa9, 0xe3, 0xb9, 0x74, 0x10, 0xd9, 0x8f, 0x98, + 0xbd, 0x76, 0xe0, 0x3f, 0x09, 0xcb, 0x99, 0x75, 0x65, 0x23, 0x47, 0x16, 0x85, 0x60, 0x9f, 0xf1, + 0x89, 0xff, 0x24, 0xc4, 0x1f, 0x42, 0xee, 0x89, 0x1f, 0x1c, 0x7b, 0xbe, 0xd3, 0x2d, 0x67, 0xf9, + 0x9c, 0xef, 0xcd, 0x9e, 0xf3, 0xbe, 0xd4, 0x22, 0x89, 0x3e, 0xde, 0x00, 0x14, 0x3e, 0xf6, 0xec, + 0x90, 0x7a, 0xb4, 0x13, 0xd9, 0x9e, 0xdb, 0x77, 0xa3, 0x72, 0x8e, 0xbb, 0x64, 0x29, 0x7c, 0xec, + 0xb5, 0x38, 0xbb, 0xc6, 0xb8, 0xd8, 0x86, 0xd5, 0x28, 0x70, 0x06, 0xa1, 0xd3, 0x61, 0x83, 0xd9, + 0x6e, 0xe8, 0x7b, 0x0e, 0x77, 0xc7, 0x3c, 0x9f, 0x72, 0x73, 0xf6, 0x94, 0xd6, 0xb8, 0x4b, 0x35, + 0xee, 0x41, 0x56, 0xa2, 0x19, 0x5c, 0x7c, 0x0d, 0x56, 0xc3, 0x63, 0x77, 0x68, 0xf3, 0x71, 0xec, + 0xa1, 0xe7, 0x0c, 0xec, 0x8e, 0xd3, 0x39, 0xa2, 0x65, 0xe0, 0x66, 0x63, 0x26, 0xe4, 0xfb, 0xde, + 0xf4, 0x9c, 0x41, 0x85, 0x49, 0x18, 0xe8, 0x4c, 0x6f, 0x40, 0x03, 0xfb, 0x94, 0x06, 0x21, 0x5b, + 0x4d, 0xe1, 0x79, 0xa0, 0x37, 0x85, 0xf2, 0x81, 0xd0, 0x25, 0xa5, 0xe1, 0x14, 0xad, 0x7d, 0x05, + 0x4a, 0xd3, 0xdb, 0x82, 0x97, 0xa0, 0x68, 0x3d, 0x68, 0x1a, 0xb6, 0x6e, 0xee, 0xd9, 0xa6, 0x5e, + 0x37, 0xd0, 0x1c, 0x2e, 0x42, 0x9e, 0xb3, 0x1a, 0x66, 0xed, 0x01, 0x52, 0xf0, 0x3c, 0xa4, 0xf4, + 0x5a, 0x0d, 0xa9, 0xda, 0x4d, 0xc8, 0xc5, 0xf8, 0xe2, 0x45, 0x28, 0xb4, 0xcd, 0x56, 0xd3, 0xa8, + 0x54, 0xf7, 0xab, 0xc6, 0x1e, 0x9a, 0xc3, 0x39, 0x48, 0x37, 0x6a, 0x56, 0x13, 0x29, 0xa2, 0xa5, + 0x37, 0x91, 0xca, 0x7a, 0xee, 0xed, 0xea, 0x28, 0xa5, 0xfd, 0xb9, 0x02, 0x2b, 0xb3, 0x70, 0xc2, + 0x05, 0x98, 0xdf, 0x33, 0xf6, 0xf5, 0x76, 0xcd, 0x42, 0x73, 0x78, 0x19, 0x16, 0x89, 0xd1, 0x34, + 0x74, 0x4b, 0xdf, 0xad, 0x19, 0x36, 0x31, 0xf4, 0x3d, 0xa4, 0x60, 0x0c, 0x25, 0xd6, 0xb2, 0x2b, + 0x8d, 0x7a, 0xbd, 0x6a, 0x59, 0xc6, 0x1e, 0x52, 0xf1, 0x0a, 0x20, 0xce, 0x6b, 0x9b, 0x63, 0x6e, + 0x0a, 0x23, 0x58, 0x68, 0x19, 0xa4, 0xaa, 0xd7, 0xaa, 0x0f, 0xd9, 0x00, 0x28, 0x8d, 0x3f, 0x07, + 0xef, 0x56, 0x1a, 0x66, 0xab, 0xda, 0xb2, 0x0c, 0xd3, 0xb2, 0x5b, 0xa6, 0xde, 0x6c, 0x7d, 0xd4, + 0xb0, 0xf8, 0xc8, 0xc2, 0xb8, 0x0c, 0x2e, 0x01, 0xe8, 0x6d, 0xab, 0x21, 0xc6, 0x41, 0x59, 0xcd, + 0x83, 0xd2, 0x34, 0x84, 0x6c, 0x55, 0x72, 0x89, 0x76, 0xb3, 0xa6, 0x9b, 0xa6, 0x41, 0xd0, 0x1c, + 0xce, 0x82, 0x7a, 0x70, 0x1d, 0x29, 0xfc, 0x77, 0x07, 0xa9, 0x78, 0x01, 0x72, 0x07, 0x3b, 0xb7, + 0x03, 0x4a, 0xbb, 0x23, 0xb1, 0x92, 0x83, 0x9d, 0x1a, 0x7d, 0x14, 0x6d, 0x13, 0xb7, 0x77, 0x14, + 0xa1, 0x34, 0x5e, 0x85, 0xa5, 0x58, 0xce, 0x76, 0xaa, 0xef, 0xfe, 0x1c, 0xed, 0xa2, 0xcc, 0x9d, + 0x74, 0x4e, 0x41, 0xea, 0x9d, 0x74, 0x4e, 0x45, 0xa9, 0x3b, 0xe9, 0x5c, 0x0a, 0xa5, 0xb5, 0xbf, + 0x52, 0x21, 0xc3, 0x77, 0x86, 0xc5, 0xea, 0x89, 0x08, 0xcc, 0xdb, 0x49, 0xdc, 0x52, 0x9f, 0x13, + 0xb7, 0x78, 0xb8, 0x97, 0x11, 0x54, 0x10, 0xf8, 0x32, 0xe4, 0xfd, 0xa0, 0x67, 0x0b, 0x89, 0x88, + 0xfd, 0x39, 0x3f, 0xe8, 0xf1, 0x24, 0xc1, 0xe2, 0x2e, 0x4b, 0x19, 0x87, 0x4e, 0x48, 0xf9, 0xf1, + 0xcb, 0x93, 0x84, 0xc6, 0x6f, 0x03, 0xd3, 0xb3, 0xf9, 0x3a, 0xb2, 0x5c, 0x36, 0xef, 0x07, 0x3d, + 0x93, 0x2d, 0xe5, 0xf3, 0x50, 0xec, 0xf8, 0xde, 0x49, 0x7f, 0x60, 0x7b, 0x74, 0xd0, 0x8b, 0x8e, + 0xca, 0xf3, 0xeb, 0xca, 0x46, 0x91, 0x2c, 0x08, 0x66, 0x8d, 0xf3, 0x70, 0x19, 0xe6, 0x3b, 0x47, + 0x4e, 0x10, 0x52, 0x71, 0xe4, 0x8a, 0x24, 0x26, 0xf9, 0xac, 0xb4, 0xe3, 0xf6, 0x1d, 0x2f, 0xe4, + 0xc7, 0xab, 0x48, 0x12, 0x9a, 0x19, 0xf1, 0xc8, 0x73, 0x7a, 0x21, 0x3f, 0x16, 0x45, 0x22, 0x08, + 0xfc, 0x3e, 0x14, 0xe4, 0x84, 0x1c, 0x82, 0x02, 0x5f, 0x0e, 0x08, 0x16, 0x43, 0x40, 0xfb, 0x09, + 0x48, 0x11, 0xff, 0x09, 0x9b, 0x53, 0xac, 0x28, 0x2c, 0x2b, 0xeb, 0xa9, 0x0d, 0x4c, 0x62, 0x92, + 0xe5, 0x2e, 0x19, 0xbe, 0x45, 0x54, 0x8f, 0x03, 0xf6, 0x77, 0x14, 0x28, 0xf0, 0x63, 0x47, 0x68, + 0x78, 0xe2, 0x45, 0x2c, 0xcc, 0xcb, 0xf8, 0xa6, 0x4c, 0x85, 0x79, 0xbe, 0x2f, 0x44, 0xca, 0x18, + 0x00, 0x2c, 0x64, 0xd9, 0xce, 0xa3, 0x47, 0xb4, 0x13, 0x51, 0x91, 0xcd, 0xd2, 0x64, 0x81, 0x31, + 0x75, 0xc9, 0x63, 0xc8, 0xbb, 0x83, 0x90, 0x06, 0x91, 0xed, 0x76, 0xf9, 0x9e, 0xa4, 0x49, 0x4e, + 0x30, 0xaa, 0x5d, 0xfc, 0x1e, 0xa4, 0x79, 0xd0, 0x4b, 0xf3, 0x59, 0x40, 0xce, 0x42, 0xfc, 0x27, + 0x84, 0xf3, 0xef, 0xa4, 0x73, 0x19, 0x94, 0xd5, 0xbe, 0x0a, 0x0b, 0x7c, 0x71, 0xf7, 0x9d, 0x60, + 0xe0, 0x0e, 0x7a, 0x3c, 0x87, 0xfb, 0x5d, 0xe1, 0x17, 0x45, 0xc2, 0xdb, 0xcc, 0xe6, 0x3e, 0x0d, + 0x43, 0xa7, 0x47, 0x65, 0x4e, 0x8d, 0x49, 0xed, 0x4f, 0x53, 0x50, 0x68, 0x45, 0x01, 0x75, 0xfa, + 0x3c, 0x3d, 0xe3, 0xaf, 0x02, 0x84, 0x91, 0x13, 0xd1, 0x3e, 0x1d, 0x44, 0xb1, 0x7d, 0xef, 0xc8, + 0x99, 0x27, 0xf4, 0xb6, 0x5a, 0xb1, 0x12, 0x99, 0xd0, 0xc7, 0xdb, 0x50, 0xa0, 0x4c, 0x6c, 0x47, + 0x2c, 0xcd, 0xcb, 0x54, 0xb2, 0x14, 0x47, 0xa2, 0x24, 0xff, 0x13, 0xa0, 0x49, 0x7b, 0xed, 0xbb, + 0x2a, 0xe4, 0x93, 0xd1, 0xb0, 0x0e, 0xb9, 0x8e, 0x13, 0xd1, 0x9e, 0x1f, 0x8c, 0x64, 0xf6, 0xfd, + 0xe0, 0x79, 0xb3, 0x6f, 0x55, 0xa4, 0x32, 0x49, 0xba, 0xe1, 0x77, 0x41, 0x5c, 0x69, 0x84, 0x5b, + 0x0a, 0x7b, 0xf3, 0x9c, 0xc3, 0x1d, 0xf3, 0x43, 0xc0, 0xc3, 0xc0, 0xed, 0x3b, 0xc1, 0xc8, 0x3e, + 0xa6, 0xa3, 0x38, 0x53, 0xa5, 0x66, 0xec, 0x24, 0x92, 0x7a, 0x77, 0xe9, 0x48, 0x06, 0xc3, 0x9b, + 0xd3, 0x7d, 0xa5, 0xb7, 0x9c, 0xdf, 0x9f, 0x89, 0x9e, 0x3c, 0xf7, 0x87, 0x71, 0x96, 0xcf, 0x70, + 0xc7, 0x62, 0x4d, 0xed, 0x4b, 0x90, 0x8b, 0x17, 0x8f, 0xf3, 0x90, 0x31, 0x82, 0xc0, 0x0f, 0xd0, + 0x1c, 0x8f, 0x89, 0xf5, 0x9a, 0x08, 0xab, 0x7b, 0x7b, 0x2c, 0xac, 0xfe, 0x8b, 0x9a, 0xa4, 0x5a, + 0x42, 0x1f, 0x9f, 0xd0, 0x30, 0xc2, 0x3f, 0x03, 0xcb, 0x94, 0xbb, 0x90, 0x7b, 0x4a, 0xed, 0x0e, + 0xbf, 0x97, 0x31, 0x07, 0x52, 0x38, 0xde, 0x8b, 0x5b, 0xe2, 0x1a, 0x19, 0xdf, 0xd7, 0xc8, 0x52, + 0xa2, 0x2b, 0x59, 0x5d, 0x6c, 0xc0, 0xb2, 0xdb, 0xef, 0xd3, 0xae, 0xeb, 0x44, 0x93, 0x03, 0x88, + 0x0d, 0x5b, 0x8d, 0xaf, 0x2d, 0x53, 0xd7, 0x3e, 0xb2, 0x94, 0xf4, 0x48, 0x86, 0xf9, 0x00, 0xb2, + 0x11, 0xbf, 0xa2, 0x72, 0xdf, 0x2d, 0x6c, 0x17, 0xe3, 0x88, 0xc3, 0x99, 0x44, 0x0a, 0xf1, 0x97, + 0x40, 0x5c, 0x78, 0x79, 0x6c, 0x19, 0x3b, 0xc4, 0xf8, 0x1e, 0x43, 0x84, 0x1c, 0x7f, 0x00, 0xa5, + 0xa9, 0x0c, 0xdb, 0xe5, 0x80, 0xa5, 0x48, 0x71, 0x32, 0x5d, 0x76, 0xf1, 0x15, 0x98, 0xf7, 0x45, + 0x3e, 0xe3, 0x51, 0x67, 0xbc, 0xe2, 0xe9, 0x64, 0x47, 0x62, 0x2d, 0x16, 0x1b, 0x02, 0x1a, 0xd2, + 0xe0, 0x94, 0x76, 0xd9, 0xa0, 0xf3, 0x7c, 0x50, 0x88, 0x59, 0xd5, 0xae, 0xf6, 0x53, 0xb0, 0x98, + 0x40, 0x1c, 0x0e, 0xfd, 0x41, 0x48, 0xf1, 0x26, 0x64, 0x03, 0x7e, 0xde, 0x25, 0xac, 0x58, 0xce, + 0x31, 0x11, 0x09, 0x88, 0xd4, 0xd0, 0xba, 0xb0, 0x28, 0x38, 0xf7, 0xdd, 0xe8, 0x88, 0xef, 0x24, + 0xfe, 0x00, 0x32, 0x94, 0x35, 0xce, 0x6c, 0x0a, 0x69, 0x56, 0xb8, 0x9c, 0x08, 0xe9, 0xc4, 0x2c, + 0xea, 0x0b, 0x67, 0xf9, 0x0f, 0x15, 0x96, 0xe5, 0x2a, 0x77, 0x9d, 0xa8, 0x73, 0xf4, 0x8a, 0x7a, + 0xc3, 0x8f, 0xc2, 0x3c, 0xe3, 0xbb, 0xc9, 0xc9, 0x99, 0xe1, 0x0f, 0xb1, 0x06, 0xf3, 0x08, 0x27, + 0xb4, 0x27, 0xb6, 0x5f, 0x5e, 0x01, 0x8b, 0x4e, 0x38, 0x71, 0x61, 0x98, 0xe1, 0x38, 0xd9, 0x17, + 0x38, 0xce, 0xfc, 0x45, 0x1c, 0x47, 0xdb, 0x83, 0x95, 0x69, 0xc4, 0xa5, 0x73, 0xfc, 0x18, 0xcc, + 0x8b, 0x4d, 0x89, 0x63, 0xe4, 0xac, 0x7d, 0x8b, 0x55, 0xb4, 0x8f, 0x55, 0x58, 0x91, 0xe1, 0xeb, + 0xb3, 0x71, 0x8e, 0x27, 0x70, 0xce, 0x5c, 0xe8, 0x80, 0x5e, 0x6c, 0xff, 0xb4, 0x0a, 0xac, 0x9e, + 0xc1, 0xf1, 0x25, 0x0e, 0xeb, 0xf7, 0x14, 0x58, 0xd8, 0xa5, 0x3d, 0x77, 0xf0, 0x8a, 0xee, 0xc2, + 0x04, 0xb8, 0xe9, 0x0b, 0x39, 0xf1, 0x10, 0x8a, 0xd2, 0x5e, 0x89, 0xd6, 0x79, 0xb4, 0x95, 0x59, + 0xa7, 0xe5, 0x26, 0x2c, 0xc8, 0x22, 0x82, 0xe3, 0xb9, 0x4e, 0x98, 0xd8, 0x73, 0xa6, 0x8a, 0xa0, + 0x33, 0x21, 0x91, 0xf5, 0x06, 0x4e, 0x68, 0xff, 0xaa, 0x40, 0xb1, 0xe2, 0xf7, 0xfb, 0x6e, 0xf4, + 0x8a, 0x62, 0x7c, 0x1e, 0xa1, 0xf4, 0x2c, 0x7f, 0xbc, 0x06, 0xa5, 0xd8, 0x4c, 0x09, 0xed, 0x99, + 0x4c, 0xa3, 0x9c, 0xcb, 0x34, 0xff, 0xa6, 0xc0, 0x22, 0xf1, 0x3d, 0xef, 0xd0, 0xe9, 0x1c, 0xbf, + 0xde, 0xe0, 0x5c, 0x07, 0x34, 0x36, 0xf4, 0xa2, 0xf0, 0xfc, 0x8f, 0x02, 0xa5, 0x66, 0x40, 0x87, + 0x4e, 0x40, 0x5f, 0x6b, 0x74, 0xd8, 0x35, 0xbd, 0x1b, 0xc9, 0x0b, 0x4e, 0x9e, 0xf0, 0xb6, 0xb6, + 0x04, 0x8b, 0x89, 0xed, 0x02, 0x30, 0xed, 0x1f, 0x14, 0x58, 0x15, 0x2e, 0x26, 0x25, 0xdd, 0x57, + 0x14, 0x96, 0xd8, 0xde, 0xf4, 0x84, 0xbd, 0x65, 0xb8, 0x74, 0xd6, 0x36, 0x69, 0xf6, 0xd7, 0x55, + 0x78, 0x2b, 0x76, 0x9e, 0x57, 0xdc, 0xf0, 0x1f, 0xc0, 0x1f, 0xd6, 0xa0, 0x7c, 0x1e, 0x04, 0x89, + 0xd0, 0xb7, 0x54, 0x28, 0x57, 0x02, 0xea, 0x44, 0x74, 0xe2, 0x1e, 0xf4, 0xfa, 0xf8, 0x06, 0xbe, + 0x06, 0x0b, 0x43, 0x27, 0x88, 0xdc, 0x8e, 0x3b, 0x74, 0xd8, 0xa7, 0x68, 0x86, 0x5f, 0xb3, 0xce, + 0x0c, 0x30, 0xa5, 0xa2, 0x5d, 0x86, 0xb7, 0x67, 0x20, 0x22, 0xf1, 0xfa, 0x5f, 0x05, 0x70, 0x2b, + 0x72, 0x82, 0xe8, 0x33, 0x90, 0x97, 0x66, 0x3a, 0xd3, 0x2a, 0x2c, 0x4f, 0xd9, 0x3f, 0x89, 0x0b, + 0x8d, 0x3e, 0x13, 0x29, 0xe9, 0x13, 0x71, 0x99, 0xb4, 0x5f, 0xe2, 0xf2, 0x4f, 0x0a, 0xac, 0x55, + 0x7c, 0x51, 0x0b, 0x7d, 0x2d, 0x4f, 0x98, 0xf6, 0x2e, 0x5c, 0x9e, 0x69, 0xa0, 0x04, 0xe0, 0x1f, + 0x15, 0xb8, 0x44, 0xa8, 0xd3, 0x7d, 0x3d, 0x8d, 0xbf, 0x07, 0x6f, 0x9d, 0x33, 0x4e, 0xde, 0x51, + 0x6e, 0x40, 0xae, 0x4f, 0x23, 0x87, 0xdd, 0x70, 0xa5, 0x49, 0x6b, 0xf1, 0xb8, 0x63, 0xed, 0xba, + 0xd4, 0x20, 0x89, 0xae, 0xf6, 0xcf, 0x2a, 0x2c, 0xf3, 0x7b, 0xf6, 0x9b, 0x8f, 0xbc, 0x0b, 0x55, + 0x61, 0xb2, 0x67, 0x2f, 0x7f, 0x4c, 0x61, 0x18, 0x50, 0x3b, 0xae, 0x0e, 0xcc, 0xf3, 0x7f, 0x10, + 0x61, 0x18, 0xd0, 0x7b, 0x82, 0xa3, 0xfd, 0x8d, 0x02, 0x2b, 0xd3, 0x10, 0x27, 0x5f, 0x34, 0xff, + 0xdf, 0xd5, 0x96, 0x19, 0x21, 0x25, 0x75, 0x91, 0x8f, 0xa4, 0xf4, 0x85, 0x3f, 0x92, 0xfe, 0x56, + 0x85, 0xf2, 0xa4, 0x31, 0x6f, 0x6a, 0x3a, 0xd3, 0x35, 0x9d, 0xef, 0xb7, 0xca, 0xa7, 0xfd, 0x9d, + 0x02, 0x6f, 0xcf, 0x00, 0xf4, 0xfb, 0x73, 0x91, 0x89, 0xca, 0x8e, 0xfa, 0xc2, 0xca, 0xce, 0xa7, + 0xef, 0x24, 0x7f, 0xaf, 0xc0, 0x4a, 0x5d, 0xd4, 0xea, 0x45, 0xe5, 0xe3, 0xd5, 0x8d, 0xc1, 0xbc, + 0x1c, 0x9f, 0x1e, 0xff, 0x5b, 0xa5, 0x55, 0x60, 0xf5, 0x8c, 0x69, 0x2f, 0x51, 0xcd, 0xf9, 0x6f, + 0x05, 0x96, 0xe4, 0x28, 0xfa, 0x2b, 0x7b, 0x7d, 0x99, 0x81, 0x0e, 0x7e, 0x0f, 0x52, 0x6e, 0x37, + 0xbe, 0xf7, 0x4e, 0xbf, 0x24, 0x60, 0x02, 0xed, 0x16, 0xe0, 0x49, 0xbb, 0x5f, 0x02, 0xba, 0x7f, + 0x57, 0x61, 0x95, 0x88, 0xe8, 0xfb, 0xe6, 0xff, 0x85, 0x1f, 0xf4, 0xff, 0x85, 0xe7, 0x27, 0xae, + 0x8f, 0xf9, 0x65, 0x6a, 0x1a, 0xea, 0x4f, 0x2f, 0x75, 0x9d, 0x49, 0xb4, 0xa9, 0x73, 0x89, 0xf6, + 0xe5, 0xe3, 0xd1, 0xc7, 0x2a, 0xac, 0x49, 0x43, 0xde, 0xdc, 0x75, 0x2e, 0xee, 0x11, 0xd9, 0x73, + 0x1e, 0xf1, 0x9f, 0x0a, 0x5c, 0x9e, 0x09, 0xe4, 0x0f, 0xfd, 0x46, 0x73, 0xc6, 0x7b, 0xd2, 0x2f, + 0xf4, 0x9e, 0xcc, 0x85, 0xbd, 0xe7, 0x9b, 0x2a, 0x94, 0x08, 0xf5, 0xa8, 0x13, 0xbe, 0xe6, 0xd5, + 0xbd, 0x33, 0x18, 0x66, 0xce, 0xd5, 0x39, 0x97, 0x60, 0x31, 0x01, 0x42, 0x7e, 0x70, 0xf1, 0x0f, + 0x74, 0x96, 0x07, 0x3f, 0xa2, 0x8e, 0x17, 0xc5, 0x37, 0x41, 0xed, 0xcf, 0x54, 0x28, 0x12, 0xc6, + 0x71, 0xfb, 0xb4, 0x15, 0x39, 0x51, 0x88, 0x3f, 0x07, 0x0b, 0x47, 0x5c, 0xc5, 0x1e, 0x7b, 0x48, + 0x9e, 0x14, 0x04, 0x4f, 0xfc, 0xfb, 0xb8, 0x0d, 0xab, 0x21, 0xed, 0xf8, 0x83, 0x6e, 0x68, 0x1f, + 0xd2, 0x23, 0x77, 0xd0, 0xb5, 0xfb, 0x4e, 0x18, 0xd1, 0x80, 0xc3, 0x52, 0x24, 0xcb, 0x52, 0xb8, + 0xcb, 0x65, 0x75, 0x2e, 0xc2, 0x57, 0x61, 0xe5, 0xd0, 0x1d, 0x78, 0x7e, 0xcf, 0x1e, 0x7a, 0xce, + 0x88, 0x06, 0xa1, 0xdd, 0xf1, 0x4f, 0x06, 0x02, 0x8f, 0x0c, 0xc1, 0x42, 0xd6, 0x14, 0xa2, 0x0a, + 0x93, 0xe0, 0x87, 0xb0, 0x39, 0x73, 0x16, 0xfb, 0x91, 0xeb, 0x45, 0x34, 0xa0, 0x5d, 0x3b, 0xa0, + 0x43, 0xcf, 0xed, 0x88, 0x57, 0x52, 0x02, 0xa8, 0x2f, 0xce, 0x98, 0x7a, 0x5f, 0xaa, 0x93, 0xb1, + 0x36, 0xbe, 0x0c, 0xf9, 0xce, 0xf0, 0xc4, 0x3e, 0xe1, 0x8f, 0x16, 0x18, 0x7e, 0x0a, 0xc9, 0x75, + 0x86, 0x27, 0x6d, 0x46, 0x63, 0x04, 0xa9, 0xc7, 0x43, 0x11, 0x9c, 0x15, 0xc2, 0x9a, 0xda, 0xf7, + 0x14, 0x28, 0xe9, 0xbd, 0x5e, 0x40, 0x7b, 0x4e, 0x24, 0x61, 0xba, 0x0a, 0x2b, 0x02, 0x92, 0x91, + 0x2d, 0xdd, 0x55, 0xd8, 0xa3, 0x08, 0x7b, 0xa4, 0x4c, 0xf8, 0xaa, 0xb0, 0x67, 0x07, 0x2e, 0x9d, + 0x0c, 0x66, 0xf6, 0x51, 0x79, 0x9f, 0x95, 0x44, 0x3a, 0xd9, 0xeb, 0x27, 0xe1, 0xed, 0xd9, 0x28, + 0xf4, 0x5d, 0xf1, 0x52, 0xb1, 0x48, 0x2e, 0xcd, 0x30, 0xba, 0xee, 0x0e, 0x9e, 0xd3, 0xd5, 0x79, + 0xca, 0xf1, 0xfa, 0x84, 0xae, 0xce, 0x53, 0xed, 0x2f, 0x92, 0xff, 0x14, 0x63, 0x77, 0x49, 0x02, + 0x47, 0xec, 0xc8, 0xca, 0xf3, 0x1c, 0xb9, 0x0c, 0xf3, 0xcc, 0x19, 0xdd, 0x41, 0x8f, 0x1b, 0x97, + 0x23, 0x31, 0x89, 0x5b, 0xf0, 0x45, 0x69, 0x3b, 0x7d, 0x1a, 0xd1, 0x60, 0xe0, 0x78, 0xde, 0xc8, + 0x16, 0xe5, 0xc7, 0x41, 0x44, 0xbb, 0xf6, 0xf8, 0xe5, 0xa6, 0x08, 0x1f, 0x9f, 0x17, 0xda, 0x46, + 0xa2, 0x4c, 0x12, 0x5d, 0x2b, 0x79, 0xd3, 0xf9, 0x15, 0x28, 0x05, 0xd2, 0x89, 0xed, 0x90, 0x6d, + 0x8f, 0x0c, 0xb9, 0x2b, 0xf1, 0xab, 0x89, 0x49, 0x0f, 0x27, 0xc5, 0x60, 0xca, 0xe1, 0x5f, 0x3a, + 0xe0, 0xdc, 0x49, 0xe7, 0xb2, 0x68, 0x5e, 0xfb, 0x4b, 0x05, 0x96, 0x67, 0x7c, 0xbb, 0x27, 0x85, + 0x01, 0x65, 0xa2, 0xee, 0xf8, 0xe3, 0x90, 0xe1, 0x0f, 0x5a, 0xe4, 0x1b, 0xaa, 0xb7, 0xce, 0x7f, + 0xfa, 0xf3, 0xc7, 0x27, 0x44, 0x68, 0xb1, 0xb3, 0xc8, 0x6d, 0xea, 0xf0, 0xc2, 0x63, 0x1c, 0x51, + 0x0b, 0x8c, 0x27, 0x6a, 0x91, 0xe7, 0x2b, 0x99, 0xe9, 0x17, 0x56, 0x32, 0x37, 0x7f, 0x27, 0x05, + 0xf9, 0xfa, 0xa8, 0xf5, 0xd8, 0xdb, 0xf7, 0x9c, 0x1e, 0x7f, 0x1d, 0x52, 0x6f, 0x5a, 0x0f, 0xd0, + 0x1c, 0x5e, 0x82, 0xa2, 0xd9, 0xb0, 0x6c, 0xb3, 0x5d, 0xab, 0xd9, 0xfb, 0x35, 0xfd, 0x36, 0x52, + 0x30, 0x82, 0x85, 0x26, 0xa9, 0xda, 0x77, 0x8d, 0x07, 0x82, 0xa3, 0xe2, 0x65, 0x58, 0x6c, 0x9b, + 0xd5, 0x7b, 0x6d, 0x63, 0xcc, 0xe4, 0x2f, 0xcc, 0xea, 0xed, 0x9a, 0x55, 0x6d, 0xd6, 0x26, 0xd8, + 0x39, 0x5c, 0x84, 0xfc, 0x6e, 0xad, 0xb1, 0x2b, 0x48, 0xc4, 0xc6, 0x6f, 0x9b, 0xad, 0xea, 0x6d, + 0xd3, 0xd8, 0x13, 0xac, 0x75, 0xc6, 0x7a, 0x68, 0x90, 0xc6, 0x7e, 0x35, 0x9e, 0xf2, 0x16, 0x46, + 0x50, 0xd8, 0xad, 0x9a, 0x3a, 0x91, 0xa3, 0x3c, 0x53, 0x70, 0x09, 0xf2, 0x86, 0xd9, 0xae, 0x4b, + 0x5a, 0xc5, 0x65, 0x58, 0xd6, 0xdb, 0x56, 0xc3, 0xae, 0x9a, 0x15, 0x62, 0xd4, 0x0d, 0xd3, 0x92, + 0x92, 0x34, 0x5e, 0x86, 0x92, 0x55, 0xad, 0x1b, 0x2d, 0x4b, 0xaf, 0x37, 0x25, 0x93, 0xad, 0x22, + 0xd7, 0x32, 0x62, 0x1d, 0x84, 0xd7, 0x60, 0xd5, 0x6c, 0xd8, 0xf1, 0xab, 0xba, 0x03, 0xbd, 0xd6, + 0x36, 0xa4, 0x6c, 0x1d, 0xbf, 0x05, 0xb8, 0x61, 0xda, 0xed, 0xe6, 0x9e, 0x6e, 0x19, 0xb6, 0xd9, + 0xb8, 0x2f, 0x05, 0xb7, 0x70, 0x09, 0x72, 0xe3, 0x15, 0x3c, 0x63, 0x28, 0x14, 0x9b, 0x3a, 0xb1, + 0xc6, 0xc6, 0x3e, 0x7b, 0xc6, 0xc0, 0x82, 0xdb, 0xa4, 0xd1, 0x6e, 0x8e, 0xd5, 0x96, 0xa0, 0x20, + 0xc1, 0x92, 0xac, 0x34, 0x63, 0xed, 0x56, 0xcd, 0x4a, 0xb2, 0xbe, 0x67, 0xb9, 0x35, 0x15, 0x29, + 0x9b, 0xc7, 0x90, 0xe6, 0xdb, 0x91, 0x83, 0xb4, 0xd9, 0x30, 0x0d, 0x34, 0x87, 0x17, 0x01, 0xaa, + 0xad, 0xaa, 0x69, 0x19, 0xb7, 0x89, 0x5e, 0x63, 0x66, 0x73, 0x46, 0x0c, 0x20, 0xb3, 0x76, 0x01, + 0xe6, 0xab, 0xad, 0xfd, 0x5a, 0x43, 0xb7, 0xa4, 0x99, 0xd5, 0xd6, 0xbd, 0x76, 0xc3, 0x62, 0x42, + 0x84, 0x0b, 0x90, 0xad, 0xb6, 0x2c, 0xe3, 0x6b, 0x16, 0xb3, 0x8b, 0xcb, 0x04, 0xaa, 0xe8, 0xd9, + 0xad, 0xcd, 0x6f, 0xa7, 0x20, 0xcd, 0x9f, 0x64, 0x17, 0x21, 0xcf, 0x77, 0xdb, 0x7a, 0xd0, 0x64, + 0x53, 0xe6, 0x21, 0x5d, 0x35, 0xad, 0x9b, 0xe8, 0xe7, 0x55, 0x0c, 0x90, 0x69, 0xf3, 0xf6, 0x2f, + 0x64, 0x59, 0xbb, 0x6a, 0x5a, 0xd7, 0x6e, 0xa0, 0xaf, 0xab, 0x6c, 0xd8, 0xb6, 0x20, 0x7e, 0x31, + 0x16, 0x6c, 0xef, 0xa0, 0x6f, 0x24, 0x82, 0xed, 0x1d, 0xf4, 0x4b, 0xb1, 0xe0, 0xfa, 0x36, 0xfa, + 0x66, 0x22, 0xb8, 0xbe, 0x8d, 0x7e, 0x39, 0x16, 0xdc, 0xd8, 0x41, 0xbf, 0x92, 0x08, 0x6e, 0xec, + 0xa0, 0x5f, 0xcd, 0x32, 0x5b, 0xb8, 0x25, 0xd7, 0xb7, 0xd1, 0xaf, 0xe5, 0x12, 0xea, 0xc6, 0x0e, + 0xfa, 0xf5, 0x1c, 0xdb, 0xff, 0x64, 0x57, 0xd1, 0x6f, 0x20, 0xb6, 0x4c, 0xb6, 0x41, 0xe8, 0x37, + 0x79, 0x93, 0x89, 0xd0, 0x6f, 0x21, 0x66, 0x23, 0xe3, 0x72, 0xf2, 0x5b, 0x5c, 0xf2, 0xc0, 0xd0, + 0x09, 0xfa, 0xed, 0xac, 0x78, 0xe7, 0x59, 0xa9, 0xd6, 0xf5, 0x1a, 0xc2, 0xbc, 0x07, 0x43, 0xe5, + 0x77, 0xaf, 0xb2, 0x26, 0x73, 0x4f, 0xf4, 0x7b, 0x4d, 0x36, 0xe1, 0x81, 0x4e, 0x2a, 0x1f, 0xe9, + 0x04, 0xfd, 0xfe, 0x55, 0x36, 0xe1, 0x81, 0x4e, 0x24, 0x5e, 0x7f, 0xd0, 0x64, 0x8a, 0x5c, 0xf4, + 0x87, 0x57, 0xd9, 0xa2, 0x25, 0xff, 0x8f, 0x9a, 0x38, 0x07, 0xa9, 0xdd, 0xaa, 0x85, 0xbe, 0xcd, + 0x67, 0x63, 0x2e, 0x8a, 0xfe, 0x18, 0x31, 0x66, 0xcb, 0xb0, 0xd0, 0x77, 0x18, 0x33, 0x63, 0xb5, + 0x9b, 0x35, 0x03, 0xbd, 0xc3, 0x16, 0x77, 0xdb, 0x68, 0xd4, 0x0d, 0x8b, 0x3c, 0x40, 0x7f, 0xc2, + 0xd5, 0xef, 0xb4, 0x1a, 0x26, 0xfa, 0x2e, 0xc2, 0x25, 0x00, 0xe3, 0x6b, 0x4d, 0x62, 0xb4, 0x5a, + 0xd5, 0x86, 0x89, 0xde, 0xdf, 0xdc, 0x07, 0x74, 0x36, 0x1c, 0x30, 0x03, 0xda, 0xe6, 0x5d, 0xb3, + 0x71, 0xdf, 0x44, 0x73, 0x8c, 0x68, 0x12, 0xa3, 0xa9, 0x13, 0x03, 0x29, 0x18, 0x20, 0x2b, 0x5f, + 0x8f, 0xf2, 0x67, 0xa0, 0xa4, 0x51, 0xab, 0xed, 0xea, 0x95, 0xbb, 0x28, 0xb5, 0xfb, 0x65, 0x58, + 0x74, 0xfd, 0xad, 0x53, 0x37, 0xa2, 0x61, 0x28, 0x1e, 0xfd, 0x3f, 0xd4, 0x24, 0xe5, 0xfa, 0x57, + 0x44, 0xeb, 0x4a, 0xcf, 0xbf, 0x72, 0x1a, 0x5d, 0xe1, 0xd2, 0x2b, 0x3c, 0x62, 0x1c, 0x66, 0x39, + 0x71, 0xfd, 0xff, 0x02, 0x00, 0x00, 0xff, 0xff, 0x59, 0xe4, 0x3a, 0xdc, 0x52, 0x30, 0x00, 0x00, } diff --git a/go/vt/vtgate/planbuilder/builder.go b/go/vt/vtgate/planbuilder/builder.go index c7d38fe2e3b..cfd2c2c1a19 100644 --- a/go/vt/vtgate/planbuilder/builder.go +++ b/go/vt/vtgate/planbuilder/builder.go @@ -65,6 +65,8 @@ const ( V4GreedyOnly = querypb.ExecuteOptions_V4Greedy // V4Left2Right tries to emulate the V3 planner by only joining plans in the order they are listed in the FROM-clause V4Left2Right = querypb.ExecuteOptions_V4Left2Right + // V4GreedyOptimized uses only the faster greedy planner + V4GreedyOptimized = querypb.ExecuteOptions_V4GreedyOptimized ) type truncater interface { diff --git a/go/vt/vtgate/planbuilder/plan_test.go b/go/vt/vtgate/planbuilder/plan_test.go index 10d8d1cf157..a8fd9083bf2 100644 --- a/go/vt/vtgate/planbuilder/plan_test.go +++ b/go/vt/vtgate/planbuilder/plan_test.go @@ -600,10 +600,14 @@ func BenchmarkPlanner(b *testing.B) { b.Run(filename+"-v4left2right", func(b *testing.B) { benchmarkPlanner(b, V4Left2Right, testCases, vschema) }) + b.Run(filename+"-v4greedyOptimized", func(b *testing.B) { + benchmarkPlanner(b, V4GreedyOptimized, testCases, vschema) + }) } } func benchmarkPlanner(b *testing.B, version PlannerVersion, testCases []testCase, vschema *vschemaWrapper) { + b.ReportAllocs() for n := 0; n < b.N; n++ { for _, tcase := range testCases { if tcase.output2ndPlanner != "" { diff --git a/go/vt/vtgate/planbuilder/route_planning.go b/go/vt/vtgate/planbuilder/route_planning.go index c046043427b..c07612a4c33 100644 --- a/go/vt/vtgate/planbuilder/route_planning.go +++ b/go/vt/vtgate/planbuilder/route_planning.go @@ -17,6 +17,7 @@ limitations under the License. package planbuilder import ( + "container/heap" "sort" "strings" @@ -48,6 +49,8 @@ func newBuildSelectPlan(sel *sqlparser.Select, vschema ContextVSchema) (engine.P switch { case vschema.Planner() == V4GreedyOnly || len(qgraph.tables) > 10: tree, err = greedySolve(qgraph, semTable, vschema) + case vschema.Planner() == V4GreedyOptimized: + tree, err = greedySolveOptimized(qgraph, semTable, vschema) case vschema.Planner() == V4Left2Right: tree, err = leftToRightSolve(qgraph, semTable, vschema) default: @@ -297,6 +300,123 @@ func createJoin(lhs joinTree, rhs joinTree, joinPredicates []sqlparser.Expr, sem return newPlan } +type priorityQueueItem struct { + plan joinTree + cost int + lhsSolve semantics.TableSet + rhsSolve semantics.TableSet +} + +type priorityQueuePlans []*priorityQueueItem + +// Len implements the Heap interface +func (pq priorityQueuePlans) Len() int { return len(pq) } + +// Less implements the Heap interface +func (pq priorityQueuePlans) Less(i, j int) bool { + // We want Pop to give us the lowest cost so we use lesser than here. + return pq[i].cost < pq[j].cost +} + +// Swap implements the Heap interface +func (pq priorityQueuePlans) Swap(i, j int) { + pq[i], pq[j] = pq[j], pq[i] +} + +// Push implements the Heap interface +func (pq *priorityQueuePlans) Push(x interface{}) { + item := x.(*priorityQueueItem) + *pq = append(*pq, item) +} + +// Pop implements the Heap interface +func (pq *priorityQueuePlans) Pop() interface{} { + old := *pq + n := len(old) + item := old[n-1] + old[n-1] = nil // avoid memory leak + *pq = old[0 : n-1] + return item +} + +func greedySolveOptimized(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVSchema) (joinTree, error) { + routePlans := make([]*routePlan, len(qg.tables)) + intermediatePlans := map[semantics.TableSet]*priorityQueueItem{} + + for i, table := range qg.tables { + solves := semTable.TableSetFor(table.alias) + plan, err := createRoutePlan(table, solves, vschema) + if err != nil { + return nil, err + } + routePlans[i] = plan + intermediatePlans[solves] = &priorityQueueItem{ + plan: plan, + cost: plan.cost(), + lhsSolve: 0, + } + } + + if len(qg.tables) == 1 { + return routePlans[0], nil + } + + pq := priorityQueuePlans{} + + for i, lhs := range routePlans { + for j := i + 1; j < len(routePlans); j++ { + rhs := routePlans[j] + solves := lhs.solves() | rhs.solves() + joinPredicates := qg.crossTable[solves] + plan := createJoin(lhs, rhs, joinPredicates, semTable) + pq.Push(&priorityQueueItem{ + plan: plan, + cost: plan.cost(), + lhsSolve: lhs.solves(), + rhsSolve: rhs.solves(), + }) + } + } + + heap.Init(&pq) + + for pq.Len() > 0 { + item := heap.Pop(&pq).(*priorityQueueItem) + _, isLeftAvail := intermediatePlans[item.lhsSolve] + _, isRightAvail := intermediatePlans[item.rhsSolve] + if !isLeftAvail || !isRightAvail { + continue + } + delete(intermediatePlans, item.lhsSolve) + delete(intermediatePlans, item.rhsSolve) + solves := item.lhsSolve | item.rhsSolve + plan := item.plan + + for tableSet, intermPlan := range intermediatePlans { + totalSolved := solves | tableSet + newPlan := createJoin(intermPlan.plan, plan, qg.crossTable[totalSolved], semTable) + heap.Push(&pq, &priorityQueueItem{ + plan: newPlan, + cost: newPlan.cost(), + lhsSolve: tableSet, + rhsSolve: solves, + }) + } + intermediatePlans[solves] = item + } + + // intermediatePlans should only have 1 value now + if len(intermediatePlans) != 1 { + return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "their should be only 1 intermediate planner now") + } + + for _, item := range intermediatePlans { + return item.plan, nil + } + + return nil, nil +} + /* The greedy planner will plan a query by finding first finding the best route plan for every table. Then, iteratively, it finds the cheapest join that can be produced between the remaining plans, diff --git a/proto/query.proto b/proto/query.proto index 69e889a0610..b371c8f1e12 100644 --- a/proto/query.proto +++ b/proto/query.proto @@ -304,6 +304,7 @@ message ExecuteOptions { V4 = 2; V4Greedy = 3; V4Left2Right = 4; + V4GreedyOptimized = 5; } // PlannerVersion specifies which planner to use. From 9c16d63a3f8ca3debfa9707f80dbd0cd14cf9e79 Mon Sep 17 00:00:00 2001 From: GuptaManan100 Date: Wed, 30 Dec 2020 15:01:51 +0530 Subject: [PATCH 27/40] fixed join predicate collection issue Signed-off-by: GuptaManan100 --- go/vt/vtgate/planbuilder/plan_test.go | 5 +- go/vt/vtgate/planbuilder/querygraph.go | 12 + go/vt/vtgate/planbuilder/route_planning.go | 26 +- go/vt/vtgate/planbuilder/select.go | 10 +- .../planbuilder/testdata/from_cases.txt | 26 +- .../planbuilder/testdata/large_cases.txt | 226 ++++++++++++++++++ go/vt/vtgate/semantics/semantic_state.go | 38 +-- go/vt/vtgate/semantics/tabletset_test.go | 20 +- 8 files changed, 303 insertions(+), 60 deletions(-) create mode 100644 go/vt/vtgate/planbuilder/testdata/large_cases.txt diff --git a/go/vt/vtgate/planbuilder/plan_test.go b/go/vt/vtgate/planbuilder/plan_test.go index a8fd9083bf2..afda76bae73 100644 --- a/go/vt/vtgate/planbuilder/plan_test.go +++ b/go/vt/vtgate/planbuilder/plan_test.go @@ -184,6 +184,7 @@ func TestPlan(t *testing.T) { testFile(t, "union_cases.txt", testOutputTempDir, vschemaWrapper) testFile(t, "transaction_cases.txt", testOutputTempDir, vschemaWrapper) testFile(t, "lock_cases.txt", testOutputTempDir, vschemaWrapper) + testFile(t, "large_cases.txt", testOutputTempDir, vschemaWrapper) } func TestSysVarSetDisabled(t *testing.T) { @@ -414,7 +415,7 @@ func testFile(t *testing.T, filename, tempDir string, vschema *vschemaWrapper) { empty = true tcase.output2ndPlanner = tcase.output } - vschema.version = V4 + vschema.version = V4GreedyOptimized out, err := getPlanOutput(tcase, vschema) if out != tcase.output2ndPlanner { fail = true @@ -578,7 +579,7 @@ func locateFile(name string) string { } func BenchmarkPlanner(b *testing.B) { - filenames := []string{"from_cases.txt", "filter_cases.txt", "aggr_cases.txt", "memory_sort_cases.txt", "select_cases.txt", "union_cases.txt", "wireup_cases.txt"} + filenames := []string{"from_cases.txt", "filter_cases.txt", "large_cases.txt", "aggr_cases.txt", "memory_sort_cases.txt", "select_cases.txt", "union_cases.txt", "wireup_cases.txt"} vschema := &vschemaWrapper{ v: loadSchema(b, "schema_test.json"), sysVarEnabled: true, diff --git a/go/vt/vtgate/planbuilder/querygraph.go b/go/vt/vtgate/planbuilder/querygraph.go index 6746e568530..d80c313cbf5 100644 --- a/go/vt/vtgate/planbuilder/querygraph.go +++ b/go/vt/vtgate/planbuilder/querygraph.go @@ -52,6 +52,18 @@ type ( } ) +func (qg *queryGraph) getPredicates(lhs, rhs semantics.TableSet) []sqlparser.Expr { + var allExprs []sqlparser.Expr + for tableSet, exprs := range qg.crossTable { + if tableSet.IsSolvedBy(lhs|rhs) && + tableSet.IsOverlapping(rhs) && + tableSet.IsOverlapping(lhs) { + allExprs = append(allExprs, exprs...) + } + } + return allExprs +} + func createQGFromSelect(sel *sqlparser.Select, semTable *semantics.SemTable) (*queryGraph, error) { qg := newQueryGraph() if err := qg.collectTables(sel.From, semTable); err != nil { diff --git a/go/vt/vtgate/planbuilder/route_planning.go b/go/vt/vtgate/planbuilder/route_planning.go index c07612a4c33..ad9fcde0340 100644 --- a/go/vt/vtgate/planbuilder/route_planning.go +++ b/go/vt/vtgate/planbuilder/route_planning.go @@ -266,7 +266,7 @@ func dpSolve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVSchem rights := dpTable.bitSetsOfSize(1) for _, lhs := range lefts { for _, rhs := range rights { - if semantics.IsOverlapping(lhs.solves(), rhs.solves()) { + if lhs.solves().IsOverlapping(rhs.solves()) { // at least one of the tables is solved on both sides continue } @@ -276,7 +276,7 @@ func dpSolve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVSchem // we already have the perfect plan. keep it continue } - joinPredicates := qg.crossTable[solves] + joinPredicates := qg.getPredicates(lhs.solves(), rhs.solves()) newPlan := createJoin(lhs, rhs, joinPredicates, semTable) if oldPlan == nil || newPlan.cost() < oldPlan.cost() { dpTable.add(newPlan) @@ -307,6 +307,10 @@ type priorityQueueItem struct { rhsSolve semantics.TableSet } +func (pqi *priorityQueueItem) solves() semantics.TableSet { + return pqi.lhsSolve | pqi.rhsSolve +} + type priorityQueuePlans []*priorityQueueItem // Len implements the Heap interface @@ -315,6 +319,9 @@ func (pq priorityQueuePlans) Len() int { return len(pq) } // Less implements the Heap interface func (pq priorityQueuePlans) Less(i, j int) bool { // We want Pop to give us the lowest cost so we use lesser than here. + if pq[i].cost == pq[j].cost { + return pq[i].solves() < pq[j].solves() + } return pq[i].cost < pq[j].cost } @@ -366,8 +373,7 @@ func greedySolveOptimized(qg *queryGraph, semTable *semantics.SemTable, vschema for i, lhs := range routePlans { for j := i + 1; j < len(routePlans); j++ { rhs := routePlans[j] - solves := lhs.solves() | rhs.solves() - joinPredicates := qg.crossTable[solves] + joinPredicates := qg.getPredicates(lhs.solves(), rhs.solves()) plan := createJoin(lhs, rhs, joinPredicates, semTable) pq.Push(&priorityQueueItem{ plan: plan, @@ -393,8 +399,7 @@ func greedySolveOptimized(qg *queryGraph, semTable *semantics.SemTable, vschema plan := item.plan for tableSet, intermPlan := range intermediatePlans { - totalSolved := solves | tableSet - newPlan := createJoin(intermPlan.plan, plan, qg.crossTable[totalSolved], semTable) + newPlan := createJoin(intermPlan.plan, plan, qg.getPredicates(solves, tableSet), semTable) heap.Push(&pq, &priorityQueueItem{ plan: newPlan, cost: newPlan.cost(), @@ -470,7 +475,7 @@ func findBestJoin( for j := i + 1; j < len(plans); j++ { rhs := plans[j] solves := lhs.solves() | rhs.solves() - joinPredicates := qg.crossTable[solves] + joinPredicates := qg.getPredicates(lhs.solves(), rhs.solves()) if len(joinPredicates) == 0 && !crossJoinsOK { // if there are no predicates joining the to tables, // creating a join between them would produce a @@ -518,8 +523,7 @@ func leftToRightSolve(qg *queryGraph, semTable *semantics.SemTable, vschema Cont acc = plan continue } - solves := acc.solves() | plan.solves() - joinPredicates := qg.crossTable[solves] + joinPredicates := qg.getPredicates(acc.solves(), plan.solves()) acc = createJoin(acc, plan, joinPredicates, semTable) } @@ -625,7 +629,7 @@ func transformToLogicalPlan(tree joinTree, semTable *semantics.SemTable) (logica sqlparser.Rewrite(predicate, func(cursor *sqlparser.Cursor) bool { switch node := cursor.Node().(type) { case *sqlparser.ColName: - if semantics.IsContainedBy(lhsSolves, semTable.Dependencies(node)) { + if semTable.Dependencies(node).IsSolvedBy(lhsSolves) { arg := sqlparser.NewArgument([]byte(":" + node.CompliantName(""))) lhsColMap[node] = arg cursor.Replace(arg) @@ -690,7 +694,7 @@ func findColumnVindex(a *routePlan, exp sqlparser.Expr, sem *semantics.SemTable) } leftDep := sem.Dependencies(left) for _, table := range a.tables { - if semantics.IsContainedBy(table.qtable.tableID, leftDep) { + if leftDep.IsSolvedBy(table.qtable.tableID) { for _, vindex := range table.vtable.ColumnVindexes { singCol, isSingle := vindex.Vindex.(vindexes.SingleColumn) if isSingle && vindex.Columns[0].Equal(left.Name) { diff --git a/go/vt/vtgate/planbuilder/select.go b/go/vt/vtgate/planbuilder/select.go index b14d3d5492c..7df3376662d 100644 --- a/go/vt/vtgate/planbuilder/select.go +++ b/go/vt/vtgate/planbuilder/select.go @@ -79,10 +79,10 @@ func pushProjection(expr []*sqlparser.AliasedExpr, plan logicalPlan, semTable *s for i, e := range expr { deps := semTable.Dependencies(e.Expr) switch { - case semantics.IsContainedBy(deps, lhsSolves): + case deps.IsSolvedBy(lhsSolves): lhs = append(lhs, e) cols[i] = -1 - case semantics.IsContainedBy(deps, rhsSolves): + case deps.IsSolvedBy(rhsSolves): rhs = append(rhs, e) cols[i] = 1 default: @@ -151,9 +151,9 @@ func pushPredicate(exprs []sqlparser.Expr, plan logicalPlan, semTable *semantics for _, expr := range exprs { deps := semTable.Dependencies(expr) switch { - case semantics.IsContainedBy(deps, lhsSolves): + case deps.IsSolvedBy(lhsSolves): lhs = append(lhs, expr) - case semantics.IsContainedBy(deps, rhsSolves): + case deps.IsSolvedBy(rhsSolves): rhs = append(rhs, expr) default: return vterrors.Errorf(vtrpcpb.Code_INTERNAL, "unknown dependencies for %s", sqlparser.String(expr)) @@ -184,7 +184,7 @@ func reorderExpression(expr sqlparser.Expr, solves semantics.TableSet, semTable func dependsOnRoute(solves semantics.TableSet, expr sqlparser.Expr, semTable *semantics.SemTable) bool { if node, ok := expr.(*sqlparser.ColName); ok { - return semantics.IsContainedBy(solves, semTable.Dependencies(node)) + return semTable.Dependencies(node).IsSolvedBy(solves) } return !sqlparser.IsValue(expr) } diff --git a/go/vt/vtgate/planbuilder/testdata/from_cases.txt b/go/vt/vtgate/planbuilder/testdata/from_cases.txt index 24eb7841a02..058105b14a8 100644 --- a/go/vt/vtgate/planbuilder/testdata/from_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/from_cases.txt @@ -637,20 +637,9 @@ "Instructions": { "OperatorType": "Join", "Variant": "Join", - "JoinColumnIndexes": "1", - "TableName": "unsharded_user", + "JoinColumnIndexes": "-1", + "TableName": "user_unsharded", "Inputs": [ - { - "OperatorType": "Route", - "Variant": "SelectUnsharded", - "Keyspace": { - "Name": "main", - "Sharded": false - }, - "FieldQuery": "select 1 from unsharded as m1, unsharded as m2 where 1 != 1", - "Query": "select 1 from unsharded as m1, unsharded as m2", - "Table": "unsharded" - }, { "OperatorType": "Route", "Variant": "SelectScatter", @@ -661,6 +650,17 @@ "FieldQuery": "select user.col from user where 1 != 1", "Query": "select user.col from user", "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select 1 from unsharded as m1, unsharded as m2 where 1 != 1", + "Query": "select 1 from unsharded as m1, unsharded as m2", + "Table": "unsharded" } ] } diff --git a/go/vt/vtgate/planbuilder/testdata/large_cases.txt b/go/vt/vtgate/planbuilder/testdata/large_cases.txt new file mode 100644 index 00000000000..4c246a88bdf --- /dev/null +++ b/go/vt/vtgate/planbuilder/testdata/large_cases.txt @@ -0,0 +1,226 @@ +"select user.id from user, user_extra, user_metadata, music, unsharded, unsharded_a, unsharded_b, unsharded_auto, music_extra where user.id = user_extra.user_id and user_metadata.user_id = user_extra.user_id and music.id = music_extra.music_id and unsharded.x = unsharded_a.y" +{ + "QueryType": "SELECT", + "Original": "select user.id from user, user_extra, user_metadata, music, unsharded, unsharded_a, unsharded_b, unsharded_auto, music_extra where user.id = user_extra.user_id and user_metadata.user_id = user_extra.user_id and music.id = music_extra.music_id and unsharded.x = unsharded_a.y", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "-1", + "TableName": "user_user_extra_user_metadata_music_unsharded_unsharded_a_unsharded_b_unsharded_auto_music_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.id from user where 1 != 1", + "Query": "select user.id from user", + "Table": "user" + }, + { + "OperatorType": "Join", + "Variant": "Join", + "TableName": "user_extra_user_metadata_music_unsharded_unsharded_a_unsharded_b_unsharded_auto_music_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user_extra.user_id from user_extra where 1 != 1", + "Query": "select user_extra.user_id from user_extra where user_extra.user_id = :user_id", + "Table": "user_extra", + "Values": [ + ":user_id" + ], + "Vindex": "user_index" + }, + { + "OperatorType": "Join", + "Variant": "Join", + "TableName": "user_metadata_music_unsharded_unsharded_a_unsharded_b_unsharded_auto_music_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from user_metadata where 1 != 1", + "Query": "select 1 from user_metadata where user_metadata.user_id = :user_extra_user_id", + "Table": "user_metadata", + "Values": [ + ":user_extra_user_id" + ], + "Vindex": "user_index" + }, + { + "OperatorType": "Join", + "Variant": "Join", + "TableName": "music_unsharded_unsharded_a_unsharded_b_unsharded_auto_music_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select music.id from music where 1 != 1", + "Query": "select music.id from music", + "Table": "music" + }, + { + "OperatorType": "Join", + "Variant": "Join", + "TableName": "unsharded_unsharded_a_unsharded_b_unsharded_auto_music_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select unsharded.x from unsharded where 1 != 1", + "Query": "select unsharded.x from unsharded", + "Table": "unsharded" + }, + { + "OperatorType": "Join", + "Variant": "Join", + "TableName": "unsharded_a_unsharded_b_unsharded_auto_music_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select 1 from unsharded_a where 1 != 1", + "Query": "select 1 from unsharded_a where unsharded_a.y = :unsharded_x", + "Table": "unsharded_a" + }, + { + "OperatorType": "Join", + "Variant": "Join", + "TableName": "unsharded_b_unsharded_auto_music_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select 1 from unsharded_b where 1 != 1", + "Query": "select 1 from unsharded_b", + "Table": "unsharded_b" + }, + { + "OperatorType": "Join", + "Variant": "Join", + "TableName": "unsharded_auto_music_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select 1 from unsharded_auto where 1 != 1", + "Query": "select 1 from unsharded_auto", + "Table": "unsharded_auto" + }, + { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from music_extra where 1 != 1", + "Query": "select 1 from music_extra where music_extra.music_id = :music_id", + "Table": "music_extra", + "Values": [ + ":music_id" + ], + "Vindex": "music_user_map" + } + ] + } + ] + } + ] + } + ] + } + ] + } + ] + } + ] + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select user.id from user, user_extra, user_metadata, music, unsharded, unsharded_a, unsharded_b, unsharded_auto, music_extra where user.id = user_extra.user_id and user_metadata.user_id = user_extra.user_id and music.id = music_extra.music_id and unsharded.x = unsharded_a.y", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "1", + "TableName": "music, music_extra_user, user_extra, user_metadata_unsharded, unsharded_a, unsharded_auto, unsharded_b", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from music, music_extra where 1 != 1", + "Query": "select 1 from music, music_extra where music.id = music_extra.music_id", + "Table": "music, music_extra" + }, + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "-1", + "TableName": "user, user_extra, user_metadata_unsharded, unsharded_a, unsharded_auto, unsharded_b", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.id from user_metadata, user, user_extra where 1 != 1", + "Query": "select user.id from user_metadata, user, user_extra where user.id = user_extra.user_id and user_metadata.user_id = user_extra.user_id", + "Table": "user, user_extra, user_metadata" + }, + { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select 1 from unsharded_auto, unsharded_b, unsharded, unsharded_a where 1 != 1", + "Query": "select 1 from unsharded_auto, unsharded_b, unsharded, unsharded_a where unsharded.x = unsharded_a.y", + "Table": "unsharded, unsharded_a, unsharded_auto, unsharded_b" + } + ] + } + ] + } +} diff --git a/go/vt/vtgate/semantics/semantic_state.go b/go/vt/vtgate/semantics/semantic_state.go index 8ebbbf8347e..2b265f6afa9 100644 --- a/go/vt/vtgate/semantics/semantic_state.go +++ b/go/vt/vtgate/semantics/semantic_state.go @@ -48,22 +48,6 @@ type ( } ) -// NumberOfTables returns the number of bits set -func (ts TableSet) NumberOfTables() int { - // Brian Kernighan’s Algorithm - count := 0 - for ts > 0 { - ts &= ts - 1 - count++ - } - return count -} - -// Merge creates a TableSet that contains both inputs -func (ts TableSet) Merge(other TableSet) TableSet { - return ts | other -} - // TableSetFor returns the bitmask for this particular tableshoe func (st *SemTable) TableSetFor(t table) TableSet { for idx, t2 := range st.Tables { @@ -126,7 +110,23 @@ func log(node sqlparser.SQLNode, format string, args ...interface{}) { } // IsOverlapping returns true if at least one table exists in both sets -func IsOverlapping(a, b TableSet) bool { return a&b != 0 } +func (ts TableSet) IsOverlapping(b TableSet) bool { return ts&b != 0 } -// IsContainedBy returns true if all of `b` is contained in `a` -func IsContainedBy(a, b TableSet) bool { return a&b == a } +// IsSolvedBy returns true if all of `ts` is contained in `b` +func (ts TableSet) IsSolvedBy(b TableSet) bool { return ts&b == ts } + +// NumberOfTables returns the number of bits set +func (ts TableSet) NumberOfTables() int { + // Brian Kernighan’s Algorithm + count := 0 + for ts > 0 { + ts &= ts - 1 + count++ + } + return count +} + +// Merge creates a TableSet that contains both inputs +func (ts TableSet) Merge(other TableSet) TableSet { + return ts | other +} diff --git a/go/vt/vtgate/semantics/tabletset_test.go b/go/vt/vtgate/semantics/tabletset_test.go index 15bc041f1c7..c3d1da5b95c 100644 --- a/go/vt/vtgate/semantics/tabletset_test.go +++ b/go/vt/vtgate/semantics/tabletset_test.go @@ -30,14 +30,14 @@ const ( ) func TestTableSet(t *testing.T) { - assert.True(t, IsOverlapping(F1|F2, F1|F2)) - assert.True(t, IsOverlapping(F1|F2, F1)) - assert.True(t, IsOverlapping(F1, F1|F2)) - assert.False(t, IsOverlapping(F1|F2, F3)) - assert.False(t, IsOverlapping(F3, F1|F2)) - - assert.False(t, IsContainedBy(F1|F2, F1)) - assert.True(t, IsContainedBy(F1, F1|F2)) - assert.False(t, IsContainedBy(F1|F2, F3)) - assert.False(t, IsContainedBy(F3, F1|F2)) + assert.True(t, (F1 | F2).IsOverlapping(F1|F2)) + assert.True(t, F1.IsOverlapping(F1|F2)) + assert.True(t, (F1 | F2).IsOverlapping(F1)) + assert.False(t, F3.IsOverlapping(F1|F2)) + assert.False(t, (F1 | F2).IsOverlapping(F3)) + + assert.True(t, F1.IsSolvedBy(F1|F2)) + assert.False(t, (F1 | F2).IsSolvedBy(F1)) + assert.False(t, F3.IsSolvedBy(F1|F2)) + assert.False(t, (F1 | F2).IsSolvedBy(F3)) } From 23d653548312720dd7a3630ef991fe2eef881139 Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Wed, 30 Dec 2020 10:54:49 +0100 Subject: [PATCH 28/40] added more supported queries Signed-off-by: Andres Taylor --- go/vt/vtgate/planbuilder/testdata/filter_cases.txt | 4 ++++ go/vt/vtgate/planbuilder/testdata/from_cases.txt | 2 ++ go/vt/vtgate/planbuilder/testdata/unsupported_cases.txt | 2 ++ 3 files changed, 8 insertions(+) diff --git a/go/vt/vtgate/planbuilder/testdata/filter_cases.txt b/go/vt/vtgate/planbuilder/testdata/filter_cases.txt index 7c3a9bbd777..b5b94e3dd0b 100644 --- a/go/vt/vtgate/planbuilder/testdata/filter_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/filter_cases.txt @@ -672,6 +672,8 @@ ] } } +{ +} # Multi-route unique vindex route on both routes "select user_extra.id from user join user_extra on user.col = user_extra.col where user.id = 5 and user_extra.user_id = 5" @@ -998,6 +1000,8 @@ ] } } +{ +} # routing rules: choose the redirected table "select col from route1 where id = 1" diff --git a/go/vt/vtgate/planbuilder/testdata/from_cases.txt b/go/vt/vtgate/planbuilder/testdata/from_cases.txt index 058105b14a8..4dba107cb0e 100644 --- a/go/vt/vtgate/planbuilder/testdata/from_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/from_cases.txt @@ -1028,6 +1028,8 @@ ] } } +{ +} # sharded join, non-col reference LHS "select user.col from user join user_extra on 5 = user.id" diff --git a/go/vt/vtgate/planbuilder/testdata/unsupported_cases.txt b/go/vt/vtgate/planbuilder/testdata/unsupported_cases.txt index c5971f890e5..5b6981243d4 100644 --- a/go/vt/vtgate/planbuilder/testdata/unsupported_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/unsupported_cases.txt @@ -485,6 +485,8 @@ # create view with join that cannot be served in each shard separately "create view user.view_a as select user_extra.id from user join user_extra" "Complex select queries are not supported in create view statements" +{ +} # create view with sharded limit "create view user.view_a as select id from user order by id limit 10" From 57e14ca81eec07c2d0c6cb69ec60aa319889de19 Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Wed, 30 Dec 2020 16:15:09 +0100 Subject: [PATCH 29/40] handle null comparisons in the V4 planner Signed-off-by: Andres Taylor --- go/vt/vtgate/planbuilder/route_planning.go | 7 +++++++ go/vt/vtgate/planbuilder/testdata/filter_cases.txt | 6 ++++++ 2 files changed, 13 insertions(+) diff --git a/go/vt/vtgate/planbuilder/route_planning.go b/go/vt/vtgate/planbuilder/route_planning.go index ad9fcde0340..05a74e9fad7 100644 --- a/go/vt/vtgate/planbuilder/route_planning.go +++ b/go/vt/vtgate/planbuilder/route_planning.go @@ -168,6 +168,13 @@ func (rp *routePlan) addPredicate(predicates ...sqlparser.Expr) error { case *sqlparser.ComparisonExpr: switch node.Operator { case sqlparser.EqualOp: + if sqlparser.IsNull(node.Left) || sqlparser.IsNull(node.Right) { + // we are looking at ANDed predicates in the WHERE clause. + // since we know that nothing returns true when compared to NULL, + // so we can safely bail out here + rp.routeOpCode = engine.SelectNone + return nil + } // TODO(Manan,Andres): Remove the predicates that are repeated eg. Id=1 AND Id=1 for _, v := range vindexPreds { column := node.Left.(*sqlparser.ColName) diff --git a/go/vt/vtgate/planbuilder/testdata/filter_cases.txt b/go/vt/vtgate/planbuilder/testdata/filter_cases.txt index b5b94e3dd0b..1534f090161 100644 --- a/go/vt/vtgate/planbuilder/testdata/filter_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/filter_cases.txt @@ -35,6 +35,8 @@ "Table": "user" } } +{ +} # Single table unique vindex route "select id from user where user.id = 5" @@ -1614,6 +1616,8 @@ "Table": "music" } } +{ +} # SELECT with IS NULL "select id from music where id is null" @@ -1674,6 +1678,8 @@ "Table": "music" } } +{ +} # Single table with unique vindex match and IN (null) "select id from music where user_id = 4 and id IN (null)" From 4b837034c003ceb2fd910db2ecbb0f0e4d9f6418 Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Wed, 30 Dec 2020 17:01:00 +0100 Subject: [PATCH 30/40] fail plan tests if the v4 planner unexpectedly produces the same plan as the v3 Signed-off-by: Andres Taylor --- go/vt/vtgate/planbuilder/plan_test.go | 79 ++++++++++--------- .../planbuilder/testdata/bypass_cases.txt | 18 +++++ 2 files changed, 61 insertions(+), 36 deletions(-) diff --git a/go/vt/vtgate/planbuilder/plan_test.go b/go/vt/vtgate/planbuilder/plan_test.go index afda76bae73..8a6e26ada6f 100644 --- a/go/vt/vtgate/planbuilder/plan_test.go +++ b/go/vt/vtgate/planbuilder/plan_test.go @@ -168,23 +168,23 @@ func TestPlan(t *testing.T) { // the column is named as Id. This is to make sure that // column names are case-preserved, but treated as // case-insensitive even if they come from the vschema. - testFile(t, "aggr_cases.txt", testOutputTempDir, vschemaWrapper) - testFile(t, "dml_cases.txt", testOutputTempDir, vschemaWrapper) - testFile(t, "from_cases.txt", testOutputTempDir, vschemaWrapper) - testFile(t, "filter_cases.txt", testOutputTempDir, vschemaWrapper) - testFile(t, "postprocess_cases.txt", testOutputTempDir, vschemaWrapper) - testFile(t, "select_cases.txt", testOutputTempDir, vschemaWrapper) - testFile(t, "symtab_cases.txt", testOutputTempDir, vschemaWrapper) - testFile(t, "unsupported_cases.txt", testOutputTempDir, vschemaWrapper) - testFile(t, "vindex_func_cases.txt", testOutputTempDir, vschemaWrapper) - testFile(t, "wireup_cases.txt", testOutputTempDir, vschemaWrapper) - testFile(t, "memory_sort_cases.txt", testOutputTempDir, vschemaWrapper) - testFile(t, "use_cases.txt", testOutputTempDir, vschemaWrapper) - testFile(t, "set_cases.txt", testOutputTempDir, vschemaWrapper) - testFile(t, "union_cases.txt", testOutputTempDir, vschemaWrapper) - testFile(t, "transaction_cases.txt", testOutputTempDir, vschemaWrapper) - testFile(t, "lock_cases.txt", testOutputTempDir, vschemaWrapper) - testFile(t, "large_cases.txt", testOutputTempDir, vschemaWrapper) + testFile(t, "aggr_cases.txt", testOutputTempDir, vschemaWrapper, true) + testFile(t, "dml_cases.txt", testOutputTempDir, vschemaWrapper, true) + testFile(t, "from_cases.txt", testOutputTempDir, vschemaWrapper, true) + testFile(t, "filter_cases.txt", testOutputTempDir, vschemaWrapper, true) + testFile(t, "postprocess_cases.txt", testOutputTempDir, vschemaWrapper, true) + testFile(t, "select_cases.txt", testOutputTempDir, vschemaWrapper, true) + testFile(t, "symtab_cases.txt", testOutputTempDir, vschemaWrapper, true) + testFile(t, "unsupported_cases.txt", testOutputTempDir, vschemaWrapper, true) + testFile(t, "vindex_func_cases.txt", testOutputTempDir, vschemaWrapper, true) + testFile(t, "wireup_cases.txt", testOutputTempDir, vschemaWrapper, true) + testFile(t, "memory_sort_cases.txt", testOutputTempDir, vschemaWrapper, true) + testFile(t, "use_cases.txt", testOutputTempDir, vschemaWrapper, true) + testFile(t, "set_cases.txt", testOutputTempDir, vschemaWrapper, true) + testFile(t, "union_cases.txt", testOutputTempDir, vschemaWrapper, true) + testFile(t, "transaction_cases.txt", testOutputTempDir, vschemaWrapper, true) + testFile(t, "lock_cases.txt", testOutputTempDir, vschemaWrapper, true) + testFile(t, "large_cases.txt", testOutputTempDir, vschemaWrapper, true) } func TestSysVarSetDisabled(t *testing.T) { @@ -196,7 +196,7 @@ func TestSysVarSetDisabled(t *testing.T) { testOutputTempDir, err := ioutil.TempDir("", "plan_test") require.NoError(t, err) defer os.RemoveAll(testOutputTempDir) - testFile(t, "set_sysvar_disabled_cases.txt", testOutputTempDir, vschemaWrapper) + testFile(t, "set_sysvar_disabled_cases.txt", testOutputTempDir, vschemaWrapper, false) } func TestOne(t *testing.T) { @@ -204,7 +204,7 @@ func TestOne(t *testing.T) { v: loadSchema(t, "schema_test.json"), } - testFile(t, "onecase.txt", "", vschema) + testFile(t, "onecase.txt", "", vschema, true) } func TestBypassPlanningFromFile(t *testing.T) { @@ -221,7 +221,7 @@ func TestBypassPlanningFromFile(t *testing.T) { dest: key.DestinationShard("-80"), } - testFile(t, "bypass_cases.txt", testOutputTempDir, vschema) + testFile(t, "bypass_cases.txt", testOutputTempDir, vschema, true) } func TestWithDefaultKeyspaceFromFile(t *testing.T) { @@ -238,9 +238,9 @@ func TestWithDefaultKeyspaceFromFile(t *testing.T) { tabletType: topodatapb.TabletType_MASTER, } - testFile(t, "alterVschema_cases.txt", testOutputTempDir, vschema) - testFile(t, "ddl_cases.txt", testOutputTempDir, vschema) - testFile(t, "show_cases.txt", testOutputTempDir, vschema) + testFile(t, "alterVschema_cases.txt", testOutputTempDir, vschema, false) + testFile(t, "ddl_cases.txt", testOutputTempDir, vschema, false) + testFile(t, "show_cases.txt", testOutputTempDir, vschema, false) } func TestOtherPlanningFromFile(t *testing.T) { @@ -257,8 +257,8 @@ func TestOtherPlanningFromFile(t *testing.T) { tabletType: topodatapb.TabletType_MASTER, } - testFile(t, "other_read_cases.txt", testOutputTempDir, vschema) - testFile(t, "other_admin_cases.txt", testOutputTempDir, vschema) + testFile(t, "other_read_cases.txt", testOutputTempDir, vschema, false) + testFile(t, "other_admin_cases.txt", testOutputTempDir, vschema, false) } func loadSchema(t testing.TB, filename string) *vindexes.VSchema { @@ -385,7 +385,7 @@ func escapeNewLines(in string) string { return strings.ReplaceAll(in, "\n", "\\n") } -func testFile(t *testing.T, filename, tempDir string, vschema *vschemaWrapper) { +func testFile(t *testing.T, filename, tempDir string, vschema *vschemaWrapper, checkV4equalPlan bool) { var checkAllTests = false t.Run(filename, func(t *testing.T) { expected := &strings.Builder{} @@ -407,20 +407,21 @@ func testFile(t *testing.T, filename, tempDir string, vschema *vschemaWrapper) { expected.WriteString(fmt.Sprintf("%s\"%s\"\n%s\n", tcase.comments, escapeNewLines(tcase.input), out)) }) - if tcase.output2ndPlanner != "" || checkAllTests { + expectedVal := "{\n}\n" + empty := false + if tcase.output2ndPlanner == "" { + empty = true + tcase.output2ndPlanner = tcase.output + } + vschema.version = V4GreedyOptimized + out, err := getPlanOutput(tcase, vschema) + + if !empty || checkAllTests { t.Run("V4: "+tcase.comments, func(t *testing.T) { - expectedVal := "{\n}\n" - empty := false - if tcase.output2ndPlanner == "" { - empty = true - tcase.output2ndPlanner = tcase.output - } - vschema.version = V4GreedyOptimized - out, err := getPlanOutput(tcase, vschema) if out != tcase.output2ndPlanner { fail = true expectedVal = "" - t.Errorf("V4 - File: %s, Line: %d\nDiff:\n%s\n[%s] \n[%s]", filename, tcase.lineno, cmp.Diff(tcase.output2ndPlanner, out), tcase.output, out) + t.Errorf("V4 - %s:%d\nDiff:\n%s\n[%s] \n[%s]", filename, tcase.lineno, cmp.Diff(tcase.output2ndPlanner, out), tcase.output, out) } if err != nil { out = `"` + out + `"` @@ -437,6 +438,12 @@ func testFile(t *testing.T, filename, tempDir string, vschema *vschemaWrapper) { expected.WriteString(out) } }) + } else { + if out == tcase.output && checkV4equalPlan { + t.Run("V4: "+tcase.comments, func(t *testing.T) { + t.Errorf("V4 - %s:%d\nplanner produces same output as V3", filename, tcase.lineno) + }) + } } expected.WriteString("\n") diff --git a/go/vt/vtgate/planbuilder/testdata/bypass_cases.txt b/go/vt/vtgate/planbuilder/testdata/bypass_cases.txt index daafda9d360..fb495774662 100644 --- a/go/vt/vtgate/planbuilder/testdata/bypass_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/bypass_cases.txt @@ -15,6 +15,8 @@ "SingleShardOnly": false } } +{ +} # update bypass "update user set val = 1 where id = 18446744073709551616 and id = 1" @@ -33,6 +35,8 @@ "SingleShardOnly": false } } +{ +} # delete bypass "DELETE FROM USER WHERE ID = 42" @@ -51,6 +55,8 @@ "SingleShardOnly": false } } +{ +} # insert bypass "INSERT INTO USER (ID, NAME) VALUES (42, 'ms X')" @@ -69,6 +75,8 @@ "SingleShardOnly": false } } +{ +} # insert bypass with sequence: sequences ignored "insert into user(nonid) values (2)" @@ -87,6 +95,8 @@ "SingleShardOnly": false } } +{ +} # bypass query for into outfile s3 "select count(*), col from unsharded into outfile S3 'x.txt'" @@ -105,6 +115,8 @@ "SingleShardOnly": false } } +{ +} "select * from user into outfile S3 'x.txt'" { @@ -122,6 +134,8 @@ "SingleShardOnly": false } } +{ +} "load data from s3 'x.txt' into table x" { @@ -139,6 +153,8 @@ "SingleShardOnly": true } } +{ +} "load data from s3 'x.txt'" { @@ -156,3 +172,5 @@ "SingleShardOnly": true } } +{ +} From b9fa57e8562ce95b01458c16493e23ea66a1df61 Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Fri, 1 Jan 2021 17:18:06 +0100 Subject: [PATCH 31/40] merge SelectEqualUnique plans Signed-off-by: Andres Taylor --- go/vt/vtgate/planbuilder/querygraph.go | 9 ++- go/vt/vtgate/planbuilder/route_planning.go | 36 ++++++---- .../planbuilder/testdata/from_cases.txt | 68 +++++++++++++++++++ 3 files changed, 96 insertions(+), 17 deletions(-) diff --git a/go/vt/vtgate/planbuilder/querygraph.go b/go/vt/vtgate/planbuilder/querygraph.go index d80c313cbf5..c1cbb795d4b 100644 --- a/go/vt/vtgate/planbuilder/querygraph.go +++ b/go/vt/vtgate/planbuilder/querygraph.go @@ -98,8 +98,13 @@ func (qg *queryGraph) collectTable(t sqlparser.TableExpr, semTable *semantics.Se if err := qg.collectTable(table.RightExpr, semTable); err != nil { return err } - if err := qg.collectPredicate(table.Condition.On, semTable); err != nil { - return err + if table.Condition.On != nil { + for _, predicate := range splitAndExpression(nil, table.Condition.On) { + err := qg.collectPredicate(predicate, semTable) + if err != nil { + return err + } + } } case *sqlparser.ParenTableExpr: if err := qg.collectTables(table.Exprs, semTable); err != nil { diff --git a/go/vt/vtgate/planbuilder/route_planning.go b/go/vt/vtgate/planbuilder/route_planning.go index 05a74e9fad7..b5da4ea311c 100644 --- a/go/vt/vtgate/planbuilder/route_planning.go +++ b/go/vt/vtgate/planbuilder/route_planning.go @@ -739,7 +739,7 @@ func canMergeOnFilter(a, b *routePlan, predicate sqlparser.Expr, sem *semantics. return rVindex == lVindex } -func canMergeScatter(a, b *routePlan, joinPredicates []sqlparser.Expr, semTable *semantics.SemTable) bool { +func canMergeOnFilters(a, b *routePlan, joinPredicates []sqlparser.Expr, semTable *semantics.SemTable) bool { for _, predicate := range joinPredicates { if canMergeOnFilter(a, b, predicate, semTable) { return true @@ -761,14 +761,22 @@ func tryMerge(a, b joinTree, joinPredicates []sqlparser.Expr, semTable *semantic return nil } + newTabletSet := aRoute.solved | bRoute.solved + r := &routePlan{ + routeOpCode: aRoute.routeOpCode, + solved: newTabletSet, + tables: append(aRoute.tables, bRoute.tables...), + extraPredicates: append(aRoute.extraPredicates, bRoute.extraPredicates...), + keyspace: aRoute.keyspace, + } + r.extraPredicates = append(r.extraPredicates, joinPredicates...) + switch aRoute.routeOpCode { case engine.SelectUnsharded, engine.SelectDBA: if aRoute.routeOpCode != bRoute.routeOpCode { return nil } - case engine.SelectEqualUnique: - return nil - case engine.SelectScatter: + case engine.SelectScatter, engine.SelectEqualUnique: if len(joinPredicates) == 0 { // If we are doing two Scatters, we have to make sure that the // joins are on the correct vindex to allow them to be merged @@ -776,21 +784,19 @@ func tryMerge(a, b joinTree, joinPredicates []sqlparser.Expr, semTable *semantic return nil } - canMerge := canMergeScatter(aRoute, bRoute, joinPredicates, semTable) + canMerge := canMergeOnFilters(aRoute, bRoute, joinPredicates, semTable) if !canMerge { return nil } + if aRoute.routeOpCode == engine.SelectEqualUnique { + r.vindex = aRoute.vindex + r.conditions = aRoute.conditions + } else if bRoute.routeOpCode == engine.SelectEqualUnique { + r.routeOpCode = bRoute.routeOpCode + r.vindex = bRoute.vindex + r.conditions = bRoute.conditions + } } - newTabletSet := aRoute.solved | bRoute.solved - r := &routePlan{ - routeOpCode: aRoute.routeOpCode, - solved: newTabletSet, - tables: append(aRoute.tables, bRoute.tables...), - extraPredicates: append(aRoute.extraPredicates, bRoute.extraPredicates...), - keyspace: aRoute.keyspace, - } - - r.extraPredicates = append(r.extraPredicates, joinPredicates...) return r } diff --git a/go/vt/vtgate/planbuilder/testdata/from_cases.txt b/go/vt/vtgate/planbuilder/testdata/from_cases.txt index 4dba107cb0e..f66e873ba4d 100644 --- a/go/vt/vtgate/planbuilder/testdata/from_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/from_cases.txt @@ -702,6 +702,40 @@ ] } } +{ + "QueryType": "SELECT", + "Original": "select user.col from user join (unsharded as m1 join unsharded as m2)", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "-1", + "TableName": "user_unsharded", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.col from user where 1 != 1", + "Query": "select user.col from user", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select 1 from unsharded as m1, unsharded as m2 where 1 != 1", + "Query": "select 1 from unsharded as m1, unsharded as m2", + "Table": "unsharded" + } + ] + } +} # Parenthesized, multi-chunk "select user.col from user join (user as u1 join unsharded)" @@ -861,6 +895,21 @@ "Table": "user" } } +{ + "QueryType": "SELECT", + "Original": "select user.col from user join user_extra on user.col between 1 and 2 and user.id = user_extra.user_id", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.col from user join user_extra on user.col between 1 and 2 and user.id = user_extra.user_id where 1 != 1", + "Query": "select user.col from user join user_extra on user.col between 1 and 2 and user.id = user_extra.user_id", + "Table": "user, user_extra" + } +} # mergeable sharded join on unique vindex, swapped operands "select user.col from user join user_extra on user_extra.user_id = user.id" @@ -916,6 +965,25 @@ "Vindex": "user_index" } } +{ + "QueryType": "SELECT", + "Original": "select user.col from user join user_extra on user.id = 5 and user.id = user_extra.user_id", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.col from user join user_extra on user.id = 5 and user.id = user_extra.user_id where 1 != 1", + "Query": "select user.col from user join user_extra on user.id = 5 and user.id = user_extra.user_id", + "Table": "user, user_extra", + "Values": [ + 5 + ], + "Vindex": "user_index" + } +} # sharded join on unique vindex, inequality "select user.col from user join user_extra on user.id < user_extra.user_id" From ecd4936de818f3036b24e624b08a239f93f48ea1 Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Fri, 1 Jan 2021 20:53:21 +0100 Subject: [PATCH 32/40] refactored route planning code Signed-off-by: Andres Taylor --- go/vt/vtgate/planbuilder/route_planning.go | 216 +++++++++--------- .../planbuilder/testdata/from_cases.txt | 34 +++ 2 files changed, 148 insertions(+), 102 deletions(-) diff --git a/go/vt/vtgate/planbuilder/route_planning.go b/go/vt/vtgate/planbuilder/route_planning.go index b5da4ea311c..b1db9ec7242 100644 --- a/go/vt/vtgate/planbuilder/route_planning.go +++ b/go/vt/vtgate/planbuilder/route_planning.go @@ -582,116 +582,127 @@ func createRoutePlan(table *queryTable, solves semantics.TableSet, vschema Conte func transformToLogicalPlan(tree joinTree, semTable *semantics.SemTable) (logicalPlan, error) { switch n := tree.(type) { case *routePlan: - var tablesForSelect sqlparser.TableExprs - tableNameMap := map[string]interface{}{} + return transformRoutePlan(n) - for _, t := range n.tables { - tablesForSelect = append(tablesForSelect, t.qtable.alias) - tableNameMap[sqlparser.String(t.qtable.alias.Expr)] = nil - } - predicates := n.Predicates() - var where *sqlparser.Where - if predicates != nil { - where = &sqlparser.Where{Expr: predicates, Type: sqlparser.WhereClause} - } - var values []sqltypes.PlanValue - if len(n.conditions) == 1 { - value, err := sqlparser.NewPlanValue(n.conditions[0].(*sqlparser.ComparisonExpr).Right) - if err != nil { - return nil, err - } - values = []sqltypes.PlanValue{value} - } - var singleColumn vindexes.SingleColumn - if n.vindex != nil { - singleColumn = n.vindex.(vindexes.SingleColumn) - } + case *joinPlan: + return transformJoinPlan(n, semTable) + } - var tableNames []string - for name := range tableNameMap { - tableNames = append(tableNames, name) - } - sort.Strings(tableNames) - - return &route{ - eroute: &engine.Route{ - Opcode: n.routeOpCode, - TableName: strings.Join(tableNames, ", "), - Keyspace: n.keyspace, - Vindex: singleColumn, - Values: values, - }, - Select: &sqlparser.Select{ - From: tablesForSelect, - Where: where, - }, - solvedTables: n.solved, - }, nil + return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "BUG: unknown type encountered: %T", tree) +} - case *joinPlan: +func transformJoinPlan(n *joinPlan, semTable *semantics.SemTable) (*join2, error) { + lhsColList := extractColumnsNeededFromLHS(n, semTable, n.lhs.solves()) - lhsSolves := n.lhs.solves() - lhsColMap := map[*sqlparser.ColName]sqlparser.Argument{} - for _, predicate := range n.predicates { - sqlparser.Rewrite(predicate, func(cursor *sqlparser.Cursor) bool { - switch node := cursor.Node().(type) { - case *sqlparser.ColName: - if semTable.Dependencies(node).IsSolvedBy(lhsSolves) { - arg := sqlparser.NewArgument([]byte(":" + node.CompliantName(""))) - lhsColMap[node] = arg - cursor.Replace(arg) - } - } - return true - }, nil) - } + var lhsColExpr []*sqlparser.AliasedExpr + for _, col := range lhsColList { + lhsColExpr = append(lhsColExpr, &sqlparser.AliasedExpr{ + Expr: col, + }) + } - var lhsColList []*sqlparser.ColName - for col := range lhsColMap { - lhsColList = append(lhsColList, col) - } + lhs, err := transformToLogicalPlan(n.lhs, semTable) + if err != nil { + return nil, err + } + offset, err := pushProjection(lhsColExpr, lhs, semTable) + if err != nil { + return nil, err + } - var lhsColExpr []*sqlparser.AliasedExpr - for _, col := range lhsColList { - lhsColExpr = append(lhsColExpr, &sqlparser.AliasedExpr{ - Expr: col, - }) - } + vars := map[string]int{} - lhs, err := transformToLogicalPlan(n.lhs, semTable) - if err != nil { - return nil, err - } - offset, err := pushProjection(lhsColExpr, lhs, semTable) - if err != nil { - return nil, err - } + for _, col := range lhsColList { + vars[col.CompliantName("")] = offset + offset++ + } - vars := map[string]int{} + rhs, err := transformToLogicalPlan(n.rhs, semTable) + if err != nil { + return nil, err + } - for _, col := range lhsColList { - vars[col.CompliantName("")] = offset - offset++ - } + err = pushPredicate(n.predicates, rhs, semTable) + if err != nil { + return nil, err + } - rhs, err := transformToLogicalPlan(n.rhs, semTable) - if err != nil { - return nil, err - } + return &join2{ + Left: lhs, + Right: rhs, + Vars: vars, + }, nil +} - err = pushPredicate(n.predicates, rhs, semTable) - if err != nil { - return nil, err - } +func extractColumnsNeededFromLHS(n *joinPlan, semTable *semantics.SemTable, lhsSolves semantics.TableSet) []*sqlparser.ColName { + lhsColMap := map[*sqlparser.ColName]sqlparser.Argument{} + for _, predicate := range n.predicates { + sqlparser.Rewrite(predicate, func(cursor *sqlparser.Cursor) bool { + switch node := cursor.Node().(type) { + case *sqlparser.ColName: + if semTable.Dependencies(node).IsSolvedBy(lhsSolves) { + arg := sqlparser.NewArgument([]byte(":" + node.CompliantName(""))) + lhsColMap[node] = arg + cursor.Replace(arg) + } + } + return true + }, nil) + } - return &join2{ - Left: lhs, - Right: rhs, - Vars: vars, - }, nil + var lhsColList []*sqlparser.ColName + for col := range lhsColMap { + lhsColList = append(lhsColList, col) } + return lhsColList +} - return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "BUG: unknown type encountered: %T", tree) +func transformRoutePlan(n *routePlan) (*route, error) { + var tablesForSelect sqlparser.TableExprs + tableNameMap := map[string]interface{}{} + + for _, t := range n.tables { + tablesForSelect = append(tablesForSelect, t.qtable.alias) + tableNameMap[sqlparser.String(t.qtable.table.Name)] = nil + } + predicates := n.Predicates() + var where *sqlparser.Where + if predicates != nil { + where = &sqlparser.Where{Expr: predicates, Type: sqlparser.WhereClause} + } + var values []sqltypes.PlanValue + if len(n.conditions) == 1 { + value, err := sqlparser.NewPlanValue(n.conditions[0].(*sqlparser.ComparisonExpr).Right) + if err != nil { + return nil, err + } + values = []sqltypes.PlanValue{value} + } + var singleColumn vindexes.SingleColumn + if n.vindex != nil { + singleColumn = n.vindex.(vindexes.SingleColumn) + } + + var tableNames []string + for name := range tableNameMap { + tableNames = append(tableNames, name) + } + sort.Strings(tableNames) + + return &route{ + eroute: &engine.Route{ + Opcode: n.routeOpCode, + TableName: strings.Join(tableNames, ", "), + Keyspace: n.keyspace, + Vindex: singleColumn, + Values: values, + }, + Select: &sqlparser.Select{ + From: tablesForSelect, + Where: where, + }, + solvedTables: n.solved, + }, nil } func findColumnVindex(a *routePlan, exp sqlparser.Expr, sem *semantics.SemTable) vindexes.SingleColumn { @@ -763,13 +774,14 @@ func tryMerge(a, b joinTree, joinPredicates []sqlparser.Expr, semTable *semantic newTabletSet := aRoute.solved | bRoute.solved r := &routePlan{ - routeOpCode: aRoute.routeOpCode, - solved: newTabletSet, - tables: append(aRoute.tables, bRoute.tables...), - extraPredicates: append(aRoute.extraPredicates, bRoute.extraPredicates...), - keyspace: aRoute.keyspace, + routeOpCode: aRoute.routeOpCode, + solved: newTabletSet, + tables: append(aRoute.tables, bRoute.tables...), + extraPredicates: append( + append(aRoute.extraPredicates, bRoute.extraPredicates...), + joinPredicates...), + keyspace: aRoute.keyspace, } - r.extraPredicates = append(r.extraPredicates, joinPredicates...) switch aRoute.routeOpCode { case engine.SelectUnsharded, engine.SelectDBA: diff --git a/go/vt/vtgate/planbuilder/testdata/from_cases.txt b/go/vt/vtgate/planbuilder/testdata/from_cases.txt index f66e873ba4d..e7d00349f3e 100644 --- a/go/vt/vtgate/planbuilder/testdata/from_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/from_cases.txt @@ -2366,6 +2366,40 @@ ] } } +{ + "QueryType": "SELECT", + "Original": "select user.id from user join user_extra using(id)", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "-2", + "TableName": "user_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.id, user.id from user where 1 != 1", + "Query": "select user.id, user.id from user", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from user_extra where 1 != 1", + "Query": "select 1 from user_extra where user_extra.id = :user_id", + "Table": "user_extra" + } + ] + } +} # verify ',' vs JOIN precedence "select u1.a from unsharded u1, unsharded u2 join unsharded u3 on u1.a = u2.a" From 3bafc1089f099bc73fc8f4b633c569373bb59f3e Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Fri, 1 Jan 2021 21:11:17 +0100 Subject: [PATCH 33/40] Remove the assumption that A join B has the same cost as B join A Signed-off-by: Andres Taylor --- go/vt/vtgate/planbuilder/route_planning.go | 43 +++++++++++++--------- 1 file changed, 26 insertions(+), 17 deletions(-) diff --git a/go/vt/vtgate/planbuilder/route_planning.go b/go/vt/vtgate/planbuilder/route_planning.go index b1db9ec7242..2c6584d6444 100644 --- a/go/vt/vtgate/planbuilder/route_planning.go +++ b/go/vt/vtgate/planbuilder/route_planning.go @@ -295,7 +295,7 @@ func dpSolve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVSchem return dpTable.planFor(allTables), nil } -func createJoin(lhs joinTree, rhs joinTree, joinPredicates []sqlparser.Expr, semTable *semantics.SemTable) joinTree { +func createJoin(lhs, rhs joinTree, joinPredicates []sqlparser.Expr, semTable *semantics.SemTable) joinTree { newPlan := tryMerge(lhs, rhs, joinPredicates, semTable) if newPlan == nil { newPlan = &joinPlan{ @@ -429,6 +429,13 @@ func greedySolveOptimized(qg *queryGraph, semTable *semantics.SemTable, vschema return nil, nil } +type ( + tableSetPair struct { + left, right semantics.TableSet + } + cacheMap map[tableSetPair]joinTree +) + /* The greedy planner will plan a query by finding first finding the best route plan for every table. Then, iteratively, it finds the cheapest join that can be produced between the remaining plans, @@ -437,7 +444,7 @@ func greedySolveOptimized(qg *queryGraph, semTable *semantics.SemTable, vschema */ func greedySolve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVSchema) (joinTree, error) { plans := make([]joinTree, len(qg.tables)) - planCache := map[semantics.TableSet]joinTree{} + planCache := cacheMap{} // we start by seeding the table with the single routes for i, table := range qg.tables { @@ -468,37 +475,39 @@ func greedySolve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVS return plans[0], nil } +func (cm cacheMap) getJoinFor(lhs, rhs joinTree, joinPredicates []sqlparser.Expr, semTable *semantics.SemTable) joinTree { + solves := tableSetPair{left: lhs.solves(), right: rhs.solves()} + plan := cm[solves] + if plan == nil { + plan = createJoin(lhs, rhs, joinPredicates, semTable) + cm[solves] = plan + } + return plan +} + func findBestJoin( qg *queryGraph, semTable *semantics.SemTable, plans []joinTree, - planCache map[semantics.TableSet]joinTree, + planCache cacheMap, crossJoinsOK bool, ) (joinTree, int, int) { var lIdx, rIdx int var bestPlan joinTree for i, lhs := range plans { - for j := i + 1; j < len(plans); j++ { - rhs := plans[j] - solves := lhs.solves() | rhs.solves() + for j, rhs := range plans { + if i == j { + continue + } joinPredicates := qg.getPredicates(lhs.solves(), rhs.solves()) if len(joinPredicates) == 0 && !crossJoinsOK { - // if there are no predicates joining the to tables, + // if there are no predicates joining the two tables, // creating a join between them would produce a // cartesian product, which is almost always a bad idea continue } - plan := planCache[solves] - if plan == nil { - plan = createJoin(lhs, rhs, joinPredicates, semTable) - planCache[solves] = plan - if plan.cost() == 1 { - // if we are able to merge the two inputs into a single route, - // we shortcut here and pick this plan. this limits the search space - return plan, i, j - } - } + plan := planCache.getJoinFor(lhs, rhs, joinPredicates, semTable) if bestPlan == nil || plan.cost() < bestPlan.cost() { bestPlan = plan From 2d4dd7234612e20c7fe45edd60008f04ea6078f4 Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Sat, 2 Jan 2021 12:11:12 +0100 Subject: [PATCH 34/40] don't copy table qualifier and only copy some fields if the full plan is a single route Signed-off-by: Andres Taylor --- go/vt/vtgate/planbuilder/plan_test.go | 22 +++--- go/vt/vtgate/planbuilder/route_planning.go | 19 ++++- .../planbuilder/testdata/filter_cases.txt | 16 ---- .../planbuilder/testdata/from_cases.txt | 73 ++++++++++++++----- go/vt/vtgate/planbuilder/testdata/onecase.txt | 7 +- .../testdata/postprocess_cases.txt | 2 - .../planbuilder/testdata/select_cases.txt | 10 ++- 7 files changed, 97 insertions(+), 52 deletions(-) diff --git a/go/vt/vtgate/planbuilder/plan_test.go b/go/vt/vtgate/planbuilder/plan_test.go index 8a6e26ada6f..5ebc51646bd 100644 --- a/go/vt/vtgate/planbuilder/plan_test.go +++ b/go/vt/vtgate/planbuilder/plan_test.go @@ -407,33 +407,37 @@ func testFile(t *testing.T, filename, tempDir string, vschema *vschemaWrapper, c expected.WriteString(fmt.Sprintf("%s\"%s\"\n%s\n", tcase.comments, escapeNewLines(tcase.input), out)) }) - expectedVal := "{\n}\n" empty := false if tcase.output2ndPlanner == "" { empty = true + } + if tcase.output2ndPlanner == "{\n}\n" { tcase.output2ndPlanner = tcase.output } + vschema.version = V4GreedyOptimized out, err := getPlanOutput(tcase, vschema) + // our expectation for the new planner on this query is one of three + // - it produces the same plan as V3 - this is shown using empty brackets: {\n} + // - it produces a different but accepted plan - this is shown using the accepted plan + // - or it produces a different plan that has not yet been accepted, or it fails to produce a plan + // this is shown by not having any info at all after the result for the V3 planner + // with this last expectation, it is an error if the V4 planner + // produces the same plan as the V3 planner does if !empty || checkAllTests { t.Run("V4: "+tcase.comments, func(t *testing.T) { if out != tcase.output2ndPlanner { fail = true - expectedVal = "" t.Errorf("V4 - %s:%d\nDiff:\n%s\n[%s] \n[%s]", filename, tcase.lineno, cmp.Diff(tcase.output2ndPlanner, out), tcase.output, out) + } if err != nil { out = `"` + out + `"` } - if tcase.output == tcase.output2ndPlanner { - if empty { - expected.WriteString(expectedVal) - } else { - // produce empty brackets when the planners agree - expected.WriteString("{\n}\n") - } + if tcase.output == out { + expected.WriteString("{\n}\n") } else { expected.WriteString(out) } diff --git a/go/vt/vtgate/planbuilder/route_planning.go b/go/vt/vtgate/planbuilder/route_planning.go index 2c6584d6444..e83556cba9f 100644 --- a/go/vt/vtgate/planbuilder/route_planning.go +++ b/go/vt/vtgate/planbuilder/route_planning.go @@ -79,7 +79,13 @@ func newBuildSelectPlan(sel *sqlparser.Select, vschema ContextVSchema) (engine.P func planProjections(sel *sqlparser.Select, plan logicalPlan, semTable *semantics.SemTable) error { rb, ok := plan.(*route) if ok { - rb.Select = sel + ast := rb.Select.(*sqlparser.Select) + ast.Distinct = sel.Distinct + ast.GroupBy = sel.GroupBy + ast.OrderBy = sel.OrderBy + ast.Limit = sel.Limit + ast.SelectExprs = sel.SelectExprs + ast.Comments = sel.Comments } else { var projections []*sqlparser.AliasedExpr @@ -671,9 +677,18 @@ func transformRoutePlan(n *routePlan) (*route, error) { tableNameMap := map[string]interface{}{} for _, t := range n.tables { - tablesForSelect = append(tablesForSelect, t.qtable.alias) + alias := sqlparser.AliasedTableExpr{ + Expr: sqlparser.TableName{ + Name: t.vtable.Name, + }, + Partitions: nil, + As: t.qtable.alias.As, + Hints: nil, + } + tablesForSelect = append(tablesForSelect, &alias) tableNameMap[sqlparser.String(t.qtable.table.Name)] = nil } + predicates := n.Predicates() var where *sqlparser.Where if predicates != nil { diff --git a/go/vt/vtgate/planbuilder/testdata/filter_cases.txt b/go/vt/vtgate/planbuilder/testdata/filter_cases.txt index 1534f090161..126602ac3d7 100644 --- a/go/vt/vtgate/planbuilder/testdata/filter_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/filter_cases.txt @@ -413,8 +413,6 @@ "Table": "user" } } -{ -} # Composite IN: RHS has no simple values "select id from user where (col1, name) in (('aa', 1+1))" @@ -453,8 +451,6 @@ "Table": "user" } } -{ -} # Single table complex in clause "select id from user where name in (col, 'bb')" @@ -1190,8 +1186,6 @@ "Table": "user" } } -{ -} # outer and inner subquery route by same int val "select id from user where id = 5 and user.col in (select user_extra.col from user_extra where user_extra.user_id = 5)" @@ -1214,8 +1208,6 @@ "Vindex": "user_index" } } -{ -} # outer and inner subquery route by same str val "select id from user where id = 'aa' and user.col in (select user_extra.col from user_extra where user_extra.user_id = 'aa')" @@ -1238,8 +1230,6 @@ "Vindex": "user_index" } } -{ -} # outer and inner subquery route by same val arg "select id from user where id = :a and user.col in (select user_extra.col from user_extra where user_extra.user_id = :a)" @@ -1262,8 +1252,6 @@ "Vindex": "user_index" } } -{ -} # unresolved symbol in inner subquery. "select id from user where id = :a and user.col in (select user_extra.col from user_extra where user_extra.user_id = :a and foo.id = 1)" @@ -1286,8 +1274,6 @@ "Table": "user" } } -{ -} # cross-shard subquery in IN clause. # Note the improved Underlying plan as SelectIN. @@ -1590,8 +1576,6 @@ "Table": "user" } } -{ -} # outer and inner subquery route reference the same "uu.id" name # but they refer to different things. The first reference is to the outermost query, diff --git a/go/vt/vtgate/planbuilder/testdata/from_cases.txt b/go/vt/vtgate/planbuilder/testdata/from_cases.txt index e7d00349f3e..5273576c94d 100644 --- a/go/vt/vtgate/planbuilder/testdata/from_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/from_cases.txt @@ -113,6 +113,19 @@ } } { + "QueryType": "SELECT", + "Original": "select m1.col from unsharded as m1 join unsharded as m2", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select m1.col from unsharded as m1, unsharded as m2 where 1 != 1", + "Query": "select m1.col from unsharded as m1, unsharded as m2", + "Table": "unsharded" + } } # Multi-table, multi-chunk @@ -171,6 +184,8 @@ "Table": "user" } } +{ +} # routing rules where table name matches, and there's an alias. "select * from second_user.user as a" @@ -189,6 +204,8 @@ "Table": "user" } } +{ +} # routing rules where table name does not match, and there's no alias. "select * from route1" @@ -350,6 +367,19 @@ } } { + "QueryType": "SELECT", + "Original": "select u1.a, u2.a from unsharded u1, unsharded u2, unsharded u3", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select u1.a, u2.a from unsharded as u3, unsharded as u1, unsharded as u2 where 1 != 1", + "Query": "select u1.a, u2.a from unsharded as u3, unsharded as u1, unsharded as u2", + "Table": "unsharded" + } } # Left join, single chunk @@ -369,8 +399,6 @@ "Table": "unsharded" } } -{ -} # Left join, multi-chunk "select u.col from user u left join unsharded m on u.a = m.b" @@ -573,8 +601,6 @@ "Table": "unsharded" } } -{ -} # Three-way join "select user.col from user join unsharded as m1 join unsharded as m2" @@ -809,8 +835,6 @@ "Table": "user" } } -{ -} # mergeable sharded join on unique vindex "select user.col from user join user_extra on user.id = user_extra.user_id" @@ -839,8 +863,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select user.col from user join user_extra on user.id = user_extra.user_id where 1 != 1", - "Query": "select user.col from user join user_extra on user.id = user_extra.user_id", + "FieldQuery": "select user.col from user, user_extra where 1 != 1", + "Query": "select user.col from user, user_extra where user.id = user_extra.user_id", "Table": "user, user_extra" } } @@ -872,8 +896,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select user.col from user join user_extra on user.id = user_extra.user_id where 1 != 1", - "Query": "select user.col from user join user_extra on user.id = user_extra.user_id", + "FieldQuery": "select user.col from user, user_extra where 1 != 1", + "Query": "select user.col from user, user_extra where user.id = user_extra.user_id", "Table": "user, user_extra" } } @@ -905,8 +929,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select user.col from user join user_extra on user.col between 1 and 2 and user.id = user_extra.user_id where 1 != 1", - "Query": "select user.col from user join user_extra on user.col between 1 and 2 and user.id = user_extra.user_id", + "FieldQuery": "select user.col from user, user_extra where 1 != 1", + "Query": "select user.col from user, user_extra where user.col between 1 and 2 and user.id = user_extra.user_id", "Table": "user, user_extra" } } @@ -938,8 +962,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select user.col from user join user_extra on user_extra.user_id = user.id where 1 != 1", - "Query": "select user.col from user join user_extra on user_extra.user_id = user.id", + "FieldQuery": "select user.col from user, user_extra where 1 != 1", + "Query": "select user.col from user, user_extra where user_extra.user_id = user.id", "Table": "user, user_extra" } } @@ -975,8 +999,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select user.col from user join user_extra on user.id = 5 and user.id = user_extra.user_id where 1 != 1", - "Query": "select user.col from user join user_extra on user.id = 5 and user.id = user_extra.user_id", + "FieldQuery": "select user.col from user, user_extra where 1 != 1", + "Query": "select user.col from user, user_extra where user.id = 5 and user.id = user_extra.user_id", "Table": "user, user_extra", "Values": [ 5 @@ -1288,6 +1312,19 @@ } } { + "QueryType": "SELECT", + "Original": "select r1.col from ref r1 join ref", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectReference", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select r1.col from ref as r1, ref where 1 != 1", + "Query": "select r1.col from ref as r1, ref", + "Table": "ref" + } } # reference table can merge with other opcodes left to right. @@ -2268,8 +2305,6 @@ "Table": "unsharded" } } -{ -} # last_insert_id for dual "select last_insert_id()" @@ -2329,6 +2364,8 @@ "Table": "unsharded" } } +{ +} # join with USING construct "select user.id from user join user_extra using(id)" diff --git a/go/vt/vtgate/planbuilder/testdata/onecase.txt b/go/vt/vtgate/planbuilder/testdata/onecase.txt index 16a368ddafd..6c068d32119 100644 --- a/go/vt/vtgate/planbuilder/testdata/onecase.txt +++ b/go/vt/vtgate/planbuilder/testdata/onecase.txt @@ -1 +1,6 @@ -# Add your test case here for debugging and run go test -run=One. \ No newline at end of file +# Add your test case here for debugging and run go test -run=One. +# routing rules bad table +"select * from bad_table" +"keyspace noks not found in vschema" +{ +} diff --git a/go/vt/vtgate/planbuilder/testdata/postprocess_cases.txt b/go/vt/vtgate/planbuilder/testdata/postprocess_cases.txt index 794f72864d3..6cace35e97e 100644 --- a/go/vt/vtgate/planbuilder/testdata/postprocess_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/postprocess_cases.txt @@ -15,8 +15,6 @@ "Table": "user" } } -{ -} # ambiguous symbol reference "select user.col1, user_extra.col1 from user join user_extra having col1 = 2" diff --git a/go/vt/vtgate/planbuilder/testdata/select_cases.txt b/go/vt/vtgate/planbuilder/testdata/select_cases.txt index 84db2518a42..ead579b8b07 100644 --- a/go/vt/vtgate/planbuilder/testdata/select_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/select_cases.txt @@ -457,6 +457,8 @@ "Table": "unsharded" } } +{ +} # select from dual on unqualified keyspace "select @@session.auto_increment_increment from dual" @@ -517,6 +519,8 @@ "Table": "dual" } } +{ +} # RHS route referenced "select user_extra.id from user join user_extra" @@ -1290,8 +1294,6 @@ "Table": "unsharded" } } -{ -} "(select id from unsharded) union (select id from unsharded_auto) order by id limit 5" { @@ -1425,8 +1427,6 @@ "Table": "dual" } } -{ -} # testing SingleRow Projection with arithmetics "select 42+2" @@ -1470,6 +1470,8 @@ "Vindex": "user_index" } } +{ +} # sql_calc_found_rows with limit "select sql_calc_found_rows * from music limit 100" From ff4adaea7ef565ac8200acfb205b7baf07373eac Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Sat, 2 Jan 2021 12:18:37 +0100 Subject: [PATCH 35/40] keep tables in FROM according to original query Signed-off-by: Andres Taylor --- go/vt/vtgate/planbuilder/route_planning.go | 18 +++++++++++++++++- .../vtgate/planbuilder/testdata/from_cases.txt | 4 ++-- .../planbuilder/testdata/large_cases.txt | 8 ++++---- 3 files changed, 23 insertions(+), 7 deletions(-) diff --git a/go/vt/vtgate/planbuilder/route_planning.go b/go/vt/vtgate/planbuilder/route_planning.go index e83556cba9f..5f647c7d6b1 100644 --- a/go/vt/vtgate/planbuilder/route_planning.go +++ b/go/vt/vtgate/planbuilder/route_planning.go @@ -125,7 +125,7 @@ type ( // tables contains all the tables that are solved by this plan. // the tables also contain any predicates that only depend on that particular table - tables []*routeTable + tables routeTables // extraPredicates are the predicates that depend on multiple tables extraPredicates []sqlparser.Expr @@ -138,6 +138,7 @@ type ( predicates []sqlparser.Expr lhs, rhs joinTree } + routeTables []*routeTable ) // solves implements the joinTree interface @@ -676,6 +677,7 @@ func transformRoutePlan(n *routePlan) (*route, error) { var tablesForSelect sqlparser.TableExprs tableNameMap := map[string]interface{}{} + sort.Sort(n.tables) for _, t := range n.tables { alias := sqlparser.AliasedTableExpr{ Expr: sqlparser.TableName{ @@ -836,3 +838,17 @@ func tryMerge(a, b joinTree, joinPredicates []sqlparser.Expr, semTable *semantic return r } + +var _ sort.Interface = (routeTables)(nil) + +func (r routeTables) Len() int { + return len(r) +} + +func (r routeTables) Less(i, j int) bool { + return r[i].qtable.tableID < r[j].qtable.tableID +} + +func (r routeTables) Swap(i, j int) { + r[i], r[j] = r[j], r[i] +} diff --git a/go/vt/vtgate/planbuilder/testdata/from_cases.txt b/go/vt/vtgate/planbuilder/testdata/from_cases.txt index 5273576c94d..9305de7fe24 100644 --- a/go/vt/vtgate/planbuilder/testdata/from_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/from_cases.txt @@ -376,8 +376,8 @@ "Name": "main", "Sharded": false }, - "FieldQuery": "select u1.a, u2.a from unsharded as u3, unsharded as u1, unsharded as u2 where 1 != 1", - "Query": "select u1.a, u2.a from unsharded as u3, unsharded as u1, unsharded as u2", + "FieldQuery": "select u1.a, u2.a from unsharded as u1, unsharded as u2, unsharded as u3 where 1 != 1", + "Query": "select u1.a, u2.a from unsharded as u1, unsharded as u2, unsharded as u3", "Table": "unsharded" } } diff --git a/go/vt/vtgate/planbuilder/testdata/large_cases.txt b/go/vt/vtgate/planbuilder/testdata/large_cases.txt index 4c246a88bdf..064a60a4792 100644 --- a/go/vt/vtgate/planbuilder/testdata/large_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/large_cases.txt @@ -204,8 +204,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select user.id from user_metadata, user, user_extra where 1 != 1", - "Query": "select user.id from user_metadata, user, user_extra where user.id = user_extra.user_id and user_metadata.user_id = user_extra.user_id", + "FieldQuery": "select user.id from user, user_extra, user_metadata where 1 != 1", + "Query": "select user.id from user, user_extra, user_metadata where user.id = user_extra.user_id and user_metadata.user_id = user_extra.user_id", "Table": "user, user_extra, user_metadata" }, { @@ -215,8 +215,8 @@ "Name": "main", "Sharded": false }, - "FieldQuery": "select 1 from unsharded_auto, unsharded_b, unsharded, unsharded_a where 1 != 1", - "Query": "select 1 from unsharded_auto, unsharded_b, unsharded, unsharded_a where unsharded.x = unsharded_a.y", + "FieldQuery": "select 1 from unsharded, unsharded_a, unsharded_b, unsharded_auto where 1 != 1", + "Query": "select 1 from unsharded, unsharded_a, unsharded_b, unsharded_auto where unsharded.x = unsharded_a.y", "Table": "unsharded, unsharded_a, unsharded_auto, unsharded_b" } ] From 027e154dee80fea9e7ee0915517cea5eee21d468 Mon Sep 17 00:00:00 2001 From: Shlomi Noach <2607934+shlomi-noach@users.noreply.github.com> Date: Thu, 7 Jan 2021 16:44:05 +0200 Subject: [PATCH 36/40] fix lint on go/vt/srvtopo/resilient_server_test.go Signed-off-by: Shlomi Noach <2607934+shlomi-noach@users.noreply.github.com> --- go/vt/srvtopo/resilient_server_test.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/go/vt/srvtopo/resilient_server_test.go b/go/vt/srvtopo/resilient_server_test.go index 40e50c200db..76f6d18dcbe 100644 --- a/go/vt/srvtopo/resilient_server_test.go +++ b/go/vt/srvtopo/resilient_server_test.go @@ -310,7 +310,8 @@ func TestGetSrvKeyspace(t *testing.T) { time.Sleep(*srvTopoCacheTTL) - timeoutCtx, _ := context.WithTimeout(context.Background(), *srvTopoCacheRefresh*2) //nolint + timeoutCtx, cancel := context.WithTimeout(context.Background(), *srvTopoCacheRefresh*2) //nolint + defer cancel() _, err = rs.GetSrvKeyspace(timeoutCtx, "test_cell", "test_ks") wantErr := "timed out waiting for keyspace" if err == nil || err.Error() != wantErr { @@ -614,7 +615,8 @@ func TestGetSrvKeyspaceNames(t *testing.T) { time.Sleep(*srvTopoCacheTTL) - timeoutCtx, _ := context.WithTimeout(context.Background(), *srvTopoCacheRefresh*2) //nolint + timeoutCtx, cancel := context.WithTimeout(context.Background(), *srvTopoCacheRefresh*2) //nolint + defer cancel() _, err = rs.GetSrvKeyspaceNames(timeoutCtx, "test_cell", false) wantErr := "timed out waiting for keyspace names" if err == nil || err.Error() != wantErr { From 01a1ef12292294fcf865b64add499e102759361d Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Thu, 7 Jan 2021 15:40:42 +0100 Subject: [PATCH 37/40] imports Signed-off-by: Andres Taylor --- go/vt/vtgate/planbuilder/querygraph_test.go | 1 + go/vt/vtgate/planbuilder/route_planning_test.go | 1 + go/vt/vtgate/planbuilder/testdata/from_cases.txt | 2 +- go/vt/vtgate/planbuilder/testdata/onecase.txt | 5 ----- go/vt/vtgate/semantics/analyzer_test.go | 1 + 5 files changed, 4 insertions(+), 6 deletions(-) diff --git a/go/vt/vtgate/planbuilder/querygraph_test.go b/go/vt/vtgate/planbuilder/querygraph_test.go index 6f42ab62f44..daade48a5ec 100644 --- a/go/vt/vtgate/planbuilder/querygraph_test.go +++ b/go/vt/vtgate/planbuilder/querygraph_test.go @@ -23,6 +23,7 @@ import ( "vitess.io/vitess/go/test/utils" "github.com/stretchr/testify/require" + "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vtgate/semantics" "vitess.io/vitess/go/vt/vtgate/vindexes" diff --git a/go/vt/vtgate/planbuilder/route_planning_test.go b/go/vt/vtgate/planbuilder/route_planning_test.go index 725d0760b59..f3335d9d1e5 100644 --- a/go/vt/vtgate/planbuilder/route_planning_test.go +++ b/go/vt/vtgate/planbuilder/route_planning_test.go @@ -23,6 +23,7 @@ import ( "vitess.io/vitess/go/vt/vtgate/semantics" "github.com/stretchr/testify/assert" + "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vtgate/engine" "vitess.io/vitess/go/vt/vtgate/vindexes" diff --git a/go/vt/vtgate/planbuilder/testdata/from_cases.txt b/go/vt/vtgate/planbuilder/testdata/from_cases.txt index 9305de7fe24..00b6c72008e 100644 --- a/go/vt/vtgate/planbuilder/testdata/from_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/from_cases.txt @@ -1165,7 +1165,7 @@ } # sharded join, non-vindex col -"select user.col from user join user_extra on user.id = user_extra.col" +"select user.col from user join user_extra on user.id = user_extra.col where user.id = 35" { "QueryType": "SELECT", "Original": "select user.col from user join user_extra on user.id = user_extra.col", diff --git a/go/vt/vtgate/planbuilder/testdata/onecase.txt b/go/vt/vtgate/planbuilder/testdata/onecase.txt index 6c068d32119..e819513f354 100644 --- a/go/vt/vtgate/planbuilder/testdata/onecase.txt +++ b/go/vt/vtgate/planbuilder/testdata/onecase.txt @@ -1,6 +1 @@ # Add your test case here for debugging and run go test -run=One. -# routing rules bad table -"select * from bad_table" -"keyspace noks not found in vschema" -{ -} diff --git a/go/vt/vtgate/semantics/analyzer_test.go b/go/vt/vtgate/semantics/analyzer_test.go index 15be7796674..094b24e331e 100644 --- a/go/vt/vtgate/semantics/analyzer_test.go +++ b/go/vt/vtgate/semantics/analyzer_test.go @@ -22,6 +22,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "vitess.io/vitess/go/vt/sqlparser" ) From 5dc6497d8745a26ebce11fbaea0f20611f43793a Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Thu, 7 Jan 2021 15:58:55 +0100 Subject: [PATCH 38/40] cleaned out code Signed-off-by: Andres Taylor --- go/vt/proto/query/query.pb.go | 427 +++++++++--------- go/vt/vtgate/planbuilder/builder.go | 4 +- go/vt/vtgate/planbuilder/dptable.go | 59 --- go/vt/vtgate/planbuilder/dptable_test.go | 63 --- go/vt/vtgate/planbuilder/plan_test.go | 12 +- go/vt/vtgate/planbuilder/route_planning.go | 181 +------- .../planbuilder/testdata/from_cases.txt | 2 +- .../planbuilder/testdata/large_cases.txt | 28 +- proto/query.proto | 1 - 9 files changed, 232 insertions(+), 545 deletions(-) delete mode 100644 go/vt/vtgate/planbuilder/dptable.go delete mode 100644 go/vt/vtgate/planbuilder/dptable_test.go diff --git a/go/vt/proto/query/query.pb.go b/go/vt/proto/query/query.pb.go index 4d2c8ebafdf..fccfb46ffe5 100644 --- a/go/vt/proto/query/query.pb.go +++ b/go/vt/proto/query/query.pb.go @@ -466,12 +466,11 @@ func (ExecuteOptions_TransactionIsolation) EnumDescriptor() ([]byte, []int) { type ExecuteOptions_PlannerVersion int32 const ( - ExecuteOptions_DEFAULT_PLANNER ExecuteOptions_PlannerVersion = 0 - ExecuteOptions_V3 ExecuteOptions_PlannerVersion = 1 - ExecuteOptions_V4 ExecuteOptions_PlannerVersion = 2 - ExecuteOptions_V4Greedy ExecuteOptions_PlannerVersion = 3 - ExecuteOptions_V4Left2Right ExecuteOptions_PlannerVersion = 4 - ExecuteOptions_V4GreedyOptimized ExecuteOptions_PlannerVersion = 5 + ExecuteOptions_DEFAULT_PLANNER ExecuteOptions_PlannerVersion = 0 + ExecuteOptions_V3 ExecuteOptions_PlannerVersion = 1 + ExecuteOptions_V4 ExecuteOptions_PlannerVersion = 2 + ExecuteOptions_V4Greedy ExecuteOptions_PlannerVersion = 3 + ExecuteOptions_V4Left2Right ExecuteOptions_PlannerVersion = 4 ) var ExecuteOptions_PlannerVersion_name = map[int32]string{ @@ -480,16 +479,14 @@ var ExecuteOptions_PlannerVersion_name = map[int32]string{ 2: "V4", 3: "V4Greedy", 4: "V4Left2Right", - 5: "V4GreedyOptimized", } var ExecuteOptions_PlannerVersion_value = map[string]int32{ - "DEFAULT_PLANNER": 0, - "V3": 1, - "V4": 2, - "V4Greedy": 3, - "V4Left2Right": 4, - "V4GreedyOptimized": 5, + "DEFAULT_PLANNER": 0, + "V3": 1, + "V4": 2, + "V4Greedy": 3, + "V4Left2Right": 4, } func (x ExecuteOptions_PlannerVersion) String() string { @@ -4374,208 +4371,208 @@ func init() { func init() { proto.RegisterFile("query.proto", fileDescriptor_5c6ac9b241082464) } var fileDescriptor_5c6ac9b241082464 = []byte{ - // 3248 bytes of a gzipped FileDescriptorProto + // 3235 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x5a, 0x4b, 0x70, 0x1b, 0x5b, 0x5a, 0x76, 0xb7, 0x1e, 0x96, 0x7e, 0x59, 0xf2, 0xf1, 0xb1, 0x9d, 0xab, 0xeb, 0xdc, 0x87, 0xa7, - 0x67, 0xee, 0x8c, 0x31, 0xe0, 0x24, 0x8e, 0x27, 0x84, 0x3b, 0x03, 0xa4, 0x2d, 0xb7, 0x73, 0x95, - 0x48, 0x2d, 0xe5, 0xa8, 0xe5, 0x4c, 0x52, 0x54, 0x75, 0xb5, 0xa5, 0x13, 0xb9, 0xcb, 0x2d, 0xb5, - 0xd2, 0xdd, 0x76, 0x22, 0x56, 0x81, 0x61, 0x18, 0xde, 0x0c, 0x6f, 0x86, 0x29, 0xa6, 0xa8, 0x62, - 0x41, 0xb1, 0x61, 0xcd, 0x9a, 0xc5, 0x5d, 0xb0, 0xa0, 0x8a, 0x25, 0xb0, 0x00, 0x16, 0x14, 0xac, - 0xa6, 0x28, 0xaa, 0x60, 0xc1, 0x82, 0xa2, 0xce, 0xa3, 0x5b, 0x92, 0xad, 0x9b, 0x78, 0x32, 0xdc, - 0x9a, 0x4a, 0x6e, 0x56, 0x3a, 0xff, 0xe3, 0x3c, 0xfe, 0xef, 0xfc, 0xe7, 0xff, 0x4f, 0xff, 0x3a, - 0x50, 0x78, 0x7c, 0x42, 0x83, 0xd1, 0xd6, 0x30, 0xf0, 0x23, 0x1f, 0x67, 0x38, 0xb1, 0x56, 0x8a, - 0xfc, 0xa1, 0xdf, 0x75, 0x22, 0x47, 0xb0, 0xd7, 0x0a, 0xa7, 0x51, 0x30, 0xec, 0x08, 0x42, 0xfb, - 0x86, 0x02, 0x59, 0xcb, 0x09, 0x7a, 0x34, 0xc2, 0x6b, 0x90, 0x3b, 0xa6, 0xa3, 0x70, 0xe8, 0x74, - 0x68, 0x59, 0x59, 0x57, 0x36, 0xf2, 0x24, 0xa1, 0xf1, 0x0a, 0x64, 0xc2, 0x23, 0x27, 0xe8, 0x96, - 0x55, 0x2e, 0x10, 0x04, 0xfe, 0x32, 0x14, 0x22, 0xe7, 0xd0, 0xa3, 0x91, 0x1d, 0x8d, 0x86, 0xb4, - 0x9c, 0x5a, 0x57, 0x36, 0x4a, 0xdb, 0x2b, 0x5b, 0xc9, 0x7c, 0x16, 0x17, 0x5a, 0xa3, 0x21, 0x25, - 0x10, 0x25, 0x6d, 0x8c, 0x21, 0xdd, 0xa1, 0x9e, 0x57, 0x4e, 0xf3, 0xb1, 0x78, 0x5b, 0xdb, 0x83, - 0xd2, 0x81, 0x75, 0xdb, 0x89, 0x68, 0xc5, 0xf1, 0x3c, 0x1a, 0x54, 0xf7, 0xd8, 0x72, 0x4e, 0x42, - 0x1a, 0x0c, 0x9c, 0x7e, 0xb2, 0x9c, 0x98, 0xc6, 0x97, 0x20, 0xdb, 0x0b, 0xfc, 0x93, 0x61, 0x58, - 0x56, 0xd7, 0x53, 0x1b, 0x79, 0x22, 0x29, 0xed, 0x67, 0x01, 0x8c, 0x53, 0x3a, 0x88, 0x2c, 0xff, - 0x98, 0x0e, 0xf0, 0x3b, 0x90, 0x8f, 0xdc, 0x3e, 0x0d, 0x23, 0xa7, 0x3f, 0xe4, 0x43, 0xa4, 0xc8, - 0x98, 0xf1, 0x09, 0x26, 0xad, 0x41, 0x6e, 0xe8, 0x87, 0x6e, 0xe4, 0xfa, 0x03, 0x6e, 0x4f, 0x9e, - 0x24, 0xb4, 0xf6, 0xd3, 0x90, 0x39, 0x70, 0xbc, 0x13, 0x8a, 0xdf, 0x87, 0x34, 0x37, 0x58, 0xe1, - 0x06, 0x17, 0xb6, 0x04, 0xe8, 0xdc, 0x4e, 0x2e, 0x60, 0x63, 0x9f, 0x32, 0x4d, 0x3e, 0xf6, 0x02, - 0x11, 0x84, 0x76, 0x0c, 0x0b, 0xbb, 0xee, 0xa0, 0x7b, 0xe0, 0x04, 0x2e, 0x03, 0xe3, 0x25, 0x87, - 0xc1, 0x5f, 0x80, 0x2c, 0x6f, 0x84, 0xe5, 0xd4, 0x7a, 0x6a, 0xa3, 0xb0, 0xbd, 0x20, 0x3b, 0xf2, - 0xb5, 0x11, 0x29, 0xd3, 0xfe, 0x5a, 0x01, 0xd8, 0xf5, 0x4f, 0x06, 0xdd, 0x7b, 0x4c, 0x88, 0x11, - 0xa4, 0xc2, 0xc7, 0x9e, 0x04, 0x92, 0x35, 0xf1, 0x5d, 0x28, 0x1d, 0xba, 0x83, 0xae, 0x7d, 0x2a, - 0x97, 0x23, 0xb0, 0x2c, 0x6c, 0x7f, 0x41, 0x0e, 0x37, 0xee, 0xbc, 0x35, 0xb9, 0xea, 0xd0, 0x18, - 0x44, 0xc1, 0x88, 0x14, 0x0f, 0x27, 0x79, 0x6b, 0x6d, 0xc0, 0xe7, 0x95, 0xd8, 0xa4, 0xc7, 0x74, - 0x14, 0x4f, 0x7a, 0x4c, 0x47, 0xf8, 0x47, 0x26, 0x2d, 0x2a, 0x6c, 0x2f, 0xc7, 0x73, 0x4d, 0xf4, - 0x95, 0x66, 0x7e, 0xa8, 0xde, 0x54, 0xb4, 0xff, 0xca, 0x42, 0xc9, 0x78, 0x4a, 0x3b, 0x27, 0x11, - 0x6d, 0x0c, 0xd9, 0x1e, 0x84, 0xb8, 0x0e, 0x8b, 0xee, 0xa0, 0xe3, 0x9d, 0x74, 0x69, 0xd7, 0x7e, - 0xe4, 0x52, 0xaf, 0x1b, 0x72, 0x3f, 0x2a, 0x25, 0xeb, 0x9e, 0xd6, 0xdf, 0xaa, 0x4a, 0xe5, 0x7d, - 0xae, 0x4b, 0x4a, 0xee, 0x14, 0x8d, 0x37, 0x61, 0xa9, 0xe3, 0xb9, 0x74, 0x10, 0xd9, 0x8f, 0x98, - 0xbd, 0x76, 0xe0, 0x3f, 0x09, 0xcb, 0x99, 0x75, 0x65, 0x23, 0x47, 0x16, 0x85, 0x60, 0x9f, 0xf1, - 0x89, 0xff, 0x24, 0xc4, 0x1f, 0x42, 0xee, 0x89, 0x1f, 0x1c, 0x7b, 0xbe, 0xd3, 0x2d, 0x67, 0xf9, - 0x9c, 0xef, 0xcd, 0x9e, 0xf3, 0xbe, 0xd4, 0x22, 0x89, 0x3e, 0xde, 0x00, 0x14, 0x3e, 0xf6, 0xec, - 0x90, 0x7a, 0xb4, 0x13, 0xd9, 0x9e, 0xdb, 0x77, 0xa3, 0x72, 0x8e, 0xbb, 0x64, 0x29, 0x7c, 0xec, - 0xb5, 0x38, 0xbb, 0xc6, 0xb8, 0xd8, 0x86, 0xd5, 0x28, 0x70, 0x06, 0xa1, 0xd3, 0x61, 0x83, 0xd9, - 0x6e, 0xe8, 0x7b, 0x0e, 0x77, 0xc7, 0x3c, 0x9f, 0x72, 0x73, 0xf6, 0x94, 0xd6, 0xb8, 0x4b, 0x35, - 0xee, 0x41, 0x56, 0xa2, 0x19, 0x5c, 0x7c, 0x0d, 0x56, 0xc3, 0x63, 0x77, 0x68, 0xf3, 0x71, 0xec, - 0xa1, 0xe7, 0x0c, 0xec, 0x8e, 0xd3, 0x39, 0xa2, 0x65, 0xe0, 0x66, 0x63, 0x26, 0xe4, 0xfb, 0xde, - 0xf4, 0x9c, 0x41, 0x85, 0x49, 0x18, 0xe8, 0x4c, 0x6f, 0x40, 0x03, 0xfb, 0x94, 0x06, 0x21, 0x5b, - 0x4d, 0xe1, 0x79, 0xa0, 0x37, 0x85, 0xf2, 0x81, 0xd0, 0x25, 0xa5, 0xe1, 0x14, 0xad, 0x7d, 0x05, - 0x4a, 0xd3, 0xdb, 0x82, 0x97, 0xa0, 0x68, 0x3d, 0x68, 0x1a, 0xb6, 0x6e, 0xee, 0xd9, 0xa6, 0x5e, - 0x37, 0xd0, 0x1c, 0x2e, 0x42, 0x9e, 0xb3, 0x1a, 0x66, 0xed, 0x01, 0x52, 0xf0, 0x3c, 0xa4, 0xf4, - 0x5a, 0x0d, 0xa9, 0xda, 0x4d, 0xc8, 0xc5, 0xf8, 0xe2, 0x45, 0x28, 0xb4, 0xcd, 0x56, 0xd3, 0xa8, - 0x54, 0xf7, 0xab, 0xc6, 0x1e, 0x9a, 0xc3, 0x39, 0x48, 0x37, 0x6a, 0x56, 0x13, 0x29, 0xa2, 0xa5, - 0x37, 0x91, 0xca, 0x7a, 0xee, 0xed, 0xea, 0x28, 0xa5, 0xfd, 0xb9, 0x02, 0x2b, 0xb3, 0x70, 0xc2, - 0x05, 0x98, 0xdf, 0x33, 0xf6, 0xf5, 0x76, 0xcd, 0x42, 0x73, 0x78, 0x19, 0x16, 0x89, 0xd1, 0x34, - 0x74, 0x4b, 0xdf, 0xad, 0x19, 0x36, 0x31, 0xf4, 0x3d, 0xa4, 0x60, 0x0c, 0x25, 0xd6, 0xb2, 0x2b, - 0x8d, 0x7a, 0xbd, 0x6a, 0x59, 0xc6, 0x1e, 0x52, 0xf1, 0x0a, 0x20, 0xce, 0x6b, 0x9b, 0x63, 0x6e, - 0x0a, 0x23, 0x58, 0x68, 0x19, 0xa4, 0xaa, 0xd7, 0xaa, 0x0f, 0xd9, 0x00, 0x28, 0x8d, 0x3f, 0x07, - 0xef, 0x56, 0x1a, 0x66, 0xab, 0xda, 0xb2, 0x0c, 0xd3, 0xb2, 0x5b, 0xa6, 0xde, 0x6c, 0x7d, 0xd4, - 0xb0, 0xf8, 0xc8, 0xc2, 0xb8, 0x0c, 0x2e, 0x01, 0xe8, 0x6d, 0xab, 0x21, 0xc6, 0x41, 0x59, 0xcd, - 0x83, 0xd2, 0x34, 0x84, 0x6c, 0x55, 0x72, 0x89, 0x76, 0xb3, 0xa6, 0x9b, 0xa6, 0x41, 0xd0, 0x1c, - 0xce, 0x82, 0x7a, 0x70, 0x1d, 0x29, 0xfc, 0x77, 0x07, 0xa9, 0x78, 0x01, 0x72, 0x07, 0x3b, 0xb7, - 0x03, 0x4a, 0xbb, 0x23, 0xb1, 0x92, 0x83, 0x9d, 0x1a, 0x7d, 0x14, 0x6d, 0x13, 0xb7, 0x77, 0x14, - 0xa1, 0x34, 0x5e, 0x85, 0xa5, 0x58, 0xce, 0x76, 0xaa, 0xef, 0xfe, 0x1c, 0xed, 0xa2, 0xcc, 0x9d, - 0x74, 0x4e, 0x41, 0xea, 0x9d, 0x74, 0x4e, 0x45, 0xa9, 0x3b, 0xe9, 0x5c, 0x0a, 0xa5, 0xb5, 0xbf, - 0x52, 0x21, 0xc3, 0x77, 0x86, 0xc5, 0xea, 0x89, 0x08, 0xcc, 0xdb, 0x49, 0xdc, 0x52, 0x9f, 0x13, - 0xb7, 0x78, 0xb8, 0x97, 0x11, 0x54, 0x10, 0xf8, 0x32, 0xe4, 0xfd, 0xa0, 0x67, 0x0b, 0x89, 0x88, - 0xfd, 0x39, 0x3f, 0xe8, 0xf1, 0x24, 0xc1, 0xe2, 0x2e, 0x4b, 0x19, 0x87, 0x4e, 0x48, 0xf9, 0xf1, - 0xcb, 0x93, 0x84, 0xc6, 0x6f, 0x03, 0xd3, 0xb3, 0xf9, 0x3a, 0xb2, 0x5c, 0x36, 0xef, 0x07, 0x3d, - 0x93, 0x2d, 0xe5, 0xf3, 0x50, 0xec, 0xf8, 0xde, 0x49, 0x7f, 0x60, 0x7b, 0x74, 0xd0, 0x8b, 0x8e, - 0xca, 0xf3, 0xeb, 0xca, 0x46, 0x91, 0x2c, 0x08, 0x66, 0x8d, 0xf3, 0x70, 0x19, 0xe6, 0x3b, 0x47, - 0x4e, 0x10, 0x52, 0x71, 0xe4, 0x8a, 0x24, 0x26, 0xf9, 0xac, 0xb4, 0xe3, 0xf6, 0x1d, 0x2f, 0xe4, - 0xc7, 0xab, 0x48, 0x12, 0x9a, 0x19, 0xf1, 0xc8, 0x73, 0x7a, 0x21, 0x3f, 0x16, 0x45, 0x22, 0x08, - 0xfc, 0x3e, 0x14, 0xe4, 0x84, 0x1c, 0x82, 0x02, 0x5f, 0x0e, 0x08, 0x16, 0x43, 0x40, 0xfb, 0x09, - 0x48, 0x11, 0xff, 0x09, 0x9b, 0x53, 0xac, 0x28, 0x2c, 0x2b, 0xeb, 0xa9, 0x0d, 0x4c, 0x62, 0x92, - 0xe5, 0x2e, 0x19, 0xbe, 0x45, 0x54, 0x8f, 0x03, 0xf6, 0x77, 0x14, 0x28, 0xf0, 0x63, 0x47, 0x68, - 0x78, 0xe2, 0x45, 0x2c, 0xcc, 0xcb, 0xf8, 0xa6, 0x4c, 0x85, 0x79, 0xbe, 0x2f, 0x44, 0xca, 0x18, - 0x00, 0x2c, 0x64, 0xd9, 0xce, 0xa3, 0x47, 0xb4, 0x13, 0x51, 0x91, 0xcd, 0xd2, 0x64, 0x81, 0x31, - 0x75, 0xc9, 0x63, 0xc8, 0xbb, 0x83, 0x90, 0x06, 0x91, 0xed, 0x76, 0xf9, 0x9e, 0xa4, 0x49, 0x4e, - 0x30, 0xaa, 0x5d, 0xfc, 0x1e, 0xa4, 0x79, 0xd0, 0x4b, 0xf3, 0x59, 0x40, 0xce, 0x42, 0xfc, 0x27, - 0x84, 0xf3, 0xef, 0xa4, 0x73, 0x19, 0x94, 0xd5, 0xbe, 0x0a, 0x0b, 0x7c, 0x71, 0xf7, 0x9d, 0x60, - 0xe0, 0x0e, 0x7a, 0x3c, 0x87, 0xfb, 0x5d, 0xe1, 0x17, 0x45, 0xc2, 0xdb, 0xcc, 0xe6, 0x3e, 0x0d, - 0x43, 0xa7, 0x47, 0x65, 0x4e, 0x8d, 0x49, 0xed, 0x4f, 0x53, 0x50, 0x68, 0x45, 0x01, 0x75, 0xfa, - 0x3c, 0x3d, 0xe3, 0xaf, 0x02, 0x84, 0x91, 0x13, 0xd1, 0x3e, 0x1d, 0x44, 0xb1, 0x7d, 0xef, 0xc8, - 0x99, 0x27, 0xf4, 0xb6, 0x5a, 0xb1, 0x12, 0x99, 0xd0, 0xc7, 0xdb, 0x50, 0xa0, 0x4c, 0x6c, 0x47, - 0x2c, 0xcd, 0xcb, 0x54, 0xb2, 0x14, 0x47, 0xa2, 0x24, 0xff, 0x13, 0xa0, 0x49, 0x7b, 0xed, 0xbb, - 0x2a, 0xe4, 0x93, 0xd1, 0xb0, 0x0e, 0xb9, 0x8e, 0x13, 0xd1, 0x9e, 0x1f, 0x8c, 0x64, 0xf6, 0xfd, - 0xe0, 0x79, 0xb3, 0x6f, 0x55, 0xa4, 0x32, 0x49, 0xba, 0xe1, 0x77, 0x41, 0x5c, 0x69, 0x84, 0x5b, - 0x0a, 0x7b, 0xf3, 0x9c, 0xc3, 0x1d, 0xf3, 0x43, 0xc0, 0xc3, 0xc0, 0xed, 0x3b, 0xc1, 0xc8, 0x3e, - 0xa6, 0xa3, 0x38, 0x53, 0xa5, 0x66, 0xec, 0x24, 0x92, 0x7a, 0x77, 0xe9, 0x48, 0x06, 0xc3, 0x9b, - 0xd3, 0x7d, 0xa5, 0xb7, 0x9c, 0xdf, 0x9f, 0x89, 0x9e, 0x3c, 0xf7, 0x87, 0x71, 0x96, 0xcf, 0x70, - 0xc7, 0x62, 0x4d, 0xed, 0x4b, 0x90, 0x8b, 0x17, 0x8f, 0xf3, 0x90, 0x31, 0x82, 0xc0, 0x0f, 0xd0, - 0x1c, 0x8f, 0x89, 0xf5, 0x9a, 0x08, 0xab, 0x7b, 0x7b, 0x2c, 0xac, 0xfe, 0x8b, 0x9a, 0xa4, 0x5a, - 0x42, 0x1f, 0x9f, 0xd0, 0x30, 0xc2, 0x3f, 0x03, 0xcb, 0x94, 0xbb, 0x90, 0x7b, 0x4a, 0xed, 0x0e, - 0xbf, 0x97, 0x31, 0x07, 0x52, 0x38, 0xde, 0x8b, 0x5b, 0xe2, 0x1a, 0x19, 0xdf, 0xd7, 0xc8, 0x52, - 0xa2, 0x2b, 0x59, 0x5d, 0x6c, 0xc0, 0xb2, 0xdb, 0xef, 0xd3, 0xae, 0xeb, 0x44, 0x93, 0x03, 0x88, - 0x0d, 0x5b, 0x8d, 0xaf, 0x2d, 0x53, 0xd7, 0x3e, 0xb2, 0x94, 0xf4, 0x48, 0x86, 0xf9, 0x00, 0xb2, - 0x11, 0xbf, 0xa2, 0x72, 0xdf, 0x2d, 0x6c, 0x17, 0xe3, 0x88, 0xc3, 0x99, 0x44, 0x0a, 0xf1, 0x97, - 0x40, 0x5c, 0x78, 0x79, 0x6c, 0x19, 0x3b, 0xc4, 0xf8, 0x1e, 0x43, 0x84, 0x1c, 0x7f, 0x00, 0xa5, - 0xa9, 0x0c, 0xdb, 0xe5, 0x80, 0xa5, 0x48, 0x71, 0x32, 0x5d, 0x76, 0xf1, 0x15, 0x98, 0xf7, 0x45, - 0x3e, 0xe3, 0x51, 0x67, 0xbc, 0xe2, 0xe9, 0x64, 0x47, 0x62, 0x2d, 0x16, 0x1b, 0x02, 0x1a, 0xd2, - 0xe0, 0x94, 0x76, 0xd9, 0xa0, 0xf3, 0x7c, 0x50, 0x88, 0x59, 0xd5, 0xae, 0xf6, 0x53, 0xb0, 0x98, - 0x40, 0x1c, 0x0e, 0xfd, 0x41, 0x48, 0xf1, 0x26, 0x64, 0x03, 0x7e, 0xde, 0x25, 0xac, 0x58, 0xce, - 0x31, 0x11, 0x09, 0x88, 0xd4, 0xd0, 0xba, 0xb0, 0x28, 0x38, 0xf7, 0xdd, 0xe8, 0x88, 0xef, 0x24, - 0xfe, 0x00, 0x32, 0x94, 0x35, 0xce, 0x6c, 0x0a, 0x69, 0x56, 0xb8, 0x9c, 0x08, 0xe9, 0xc4, 0x2c, - 0xea, 0x0b, 0x67, 0xf9, 0x0f, 0x15, 0x96, 0xe5, 0x2a, 0x77, 0x9d, 0xa8, 0x73, 0xf4, 0x8a, 0x7a, - 0xc3, 0x8f, 0xc2, 0x3c, 0xe3, 0xbb, 0xc9, 0xc9, 0x99, 0xe1, 0x0f, 0xb1, 0x06, 0xf3, 0x08, 0x27, - 0xb4, 0x27, 0xb6, 0x5f, 0x5e, 0x01, 0x8b, 0x4e, 0x38, 0x71, 0x61, 0x98, 0xe1, 0x38, 0xd9, 0x17, - 0x38, 0xce, 0xfc, 0x45, 0x1c, 0x47, 0xdb, 0x83, 0x95, 0x69, 0xc4, 0xa5, 0x73, 0xfc, 0x18, 0xcc, - 0x8b, 0x4d, 0x89, 0x63, 0xe4, 0xac, 0x7d, 0x8b, 0x55, 0xb4, 0x8f, 0x55, 0x58, 0x91, 0xe1, 0xeb, - 0xb3, 0x71, 0x8e, 0x27, 0x70, 0xce, 0x5c, 0xe8, 0x80, 0x5e, 0x6c, 0xff, 0xb4, 0x0a, 0xac, 0x9e, - 0xc1, 0xf1, 0x25, 0x0e, 0xeb, 0xf7, 0x14, 0x58, 0xd8, 0xa5, 0x3d, 0x77, 0xf0, 0x8a, 0xee, 0xc2, - 0x04, 0xb8, 0xe9, 0x0b, 0x39, 0xf1, 0x10, 0x8a, 0xd2, 0x5e, 0x89, 0xd6, 0x79, 0xb4, 0x95, 0x59, - 0xa7, 0xe5, 0x26, 0x2c, 0xc8, 0x22, 0x82, 0xe3, 0xb9, 0x4e, 0x98, 0xd8, 0x73, 0xa6, 0x8a, 0xa0, - 0x33, 0x21, 0x91, 0xf5, 0x06, 0x4e, 0x68, 0xff, 0xaa, 0x40, 0xb1, 0xe2, 0xf7, 0xfb, 0x6e, 0xf4, - 0x8a, 0x62, 0x7c, 0x1e, 0xa1, 0xf4, 0x2c, 0x7f, 0xbc, 0x06, 0xa5, 0xd8, 0x4c, 0x09, 0xed, 0x99, - 0x4c, 0xa3, 0x9c, 0xcb, 0x34, 0xff, 0xa6, 0xc0, 0x22, 0xf1, 0x3d, 0xef, 0xd0, 0xe9, 0x1c, 0xbf, - 0xde, 0xe0, 0x5c, 0x07, 0x34, 0x36, 0xf4, 0xa2, 0xf0, 0xfc, 0x8f, 0x02, 0xa5, 0x66, 0x40, 0x87, - 0x4e, 0x40, 0x5f, 0x6b, 0x74, 0xd8, 0x35, 0xbd, 0x1b, 0xc9, 0x0b, 0x4e, 0x9e, 0xf0, 0xb6, 0xb6, - 0x04, 0x8b, 0x89, 0xed, 0x02, 0x30, 0xed, 0x1f, 0x14, 0x58, 0x15, 0x2e, 0x26, 0x25, 0xdd, 0x57, - 0x14, 0x96, 0xd8, 0xde, 0xf4, 0x84, 0xbd, 0x65, 0xb8, 0x74, 0xd6, 0x36, 0x69, 0xf6, 0xd7, 0x55, - 0x78, 0x2b, 0x76, 0x9e, 0x57, 0xdc, 0xf0, 0x1f, 0xc0, 0x1f, 0xd6, 0xa0, 0x7c, 0x1e, 0x04, 0x89, - 0xd0, 0xb7, 0x54, 0x28, 0x57, 0x02, 0xea, 0x44, 0x74, 0xe2, 0x1e, 0xf4, 0xfa, 0xf8, 0x06, 0xbe, - 0x06, 0x0b, 0x43, 0x27, 0x88, 0xdc, 0x8e, 0x3b, 0x74, 0xd8, 0xa7, 0x68, 0x86, 0x5f, 0xb3, 0xce, - 0x0c, 0x30, 0xa5, 0xa2, 0x5d, 0x86, 0xb7, 0x67, 0x20, 0x22, 0xf1, 0xfa, 0x5f, 0x05, 0x70, 0x2b, - 0x72, 0x82, 0xe8, 0x33, 0x90, 0x97, 0x66, 0x3a, 0xd3, 0x2a, 0x2c, 0x4f, 0xd9, 0x3f, 0x89, 0x0b, - 0x8d, 0x3e, 0x13, 0x29, 0xe9, 0x13, 0x71, 0x99, 0xb4, 0x5f, 0xe2, 0xf2, 0x4f, 0x0a, 0xac, 0x55, - 0x7c, 0x51, 0x0b, 0x7d, 0x2d, 0x4f, 0x98, 0xf6, 0x2e, 0x5c, 0x9e, 0x69, 0xa0, 0x04, 0xe0, 0x1f, - 0x15, 0xb8, 0x44, 0xa8, 0xd3, 0x7d, 0x3d, 0x8d, 0xbf, 0x07, 0x6f, 0x9d, 0x33, 0x4e, 0xde, 0x51, - 0x6e, 0x40, 0xae, 0x4f, 0x23, 0x87, 0xdd, 0x70, 0xa5, 0x49, 0x6b, 0xf1, 0xb8, 0x63, 0xed, 0xba, - 0xd4, 0x20, 0x89, 0xae, 0xf6, 0xcf, 0x2a, 0x2c, 0xf3, 0x7b, 0xf6, 0x9b, 0x8f, 0xbc, 0x0b, 0x55, - 0x61, 0xb2, 0x67, 0x2f, 0x7f, 0x4c, 0x61, 0x18, 0x50, 0x3b, 0xae, 0x0e, 0xcc, 0xf3, 0x7f, 0x10, - 0x61, 0x18, 0xd0, 0x7b, 0x82, 0xa3, 0xfd, 0x8d, 0x02, 0x2b, 0xd3, 0x10, 0x27, 0x5f, 0x34, 0xff, - 0xdf, 0xd5, 0x96, 0x19, 0x21, 0x25, 0x75, 0x91, 0x8f, 0xa4, 0xf4, 0x85, 0x3f, 0x92, 0xfe, 0x56, - 0x85, 0xf2, 0xa4, 0x31, 0x6f, 0x6a, 0x3a, 0xd3, 0x35, 0x9d, 0xef, 0xb7, 0xca, 0xa7, 0xfd, 0x9d, - 0x02, 0x6f, 0xcf, 0x00, 0xf4, 0xfb, 0x73, 0x91, 0x89, 0xca, 0x8e, 0xfa, 0xc2, 0xca, 0xce, 0xa7, - 0xef, 0x24, 0x7f, 0xaf, 0xc0, 0x4a, 0x5d, 0xd4, 0xea, 0x45, 0xe5, 0xe3, 0xd5, 0x8d, 0xc1, 0xbc, - 0x1c, 0x9f, 0x1e, 0xff, 0x5b, 0xa5, 0x55, 0x60, 0xf5, 0x8c, 0x69, 0x2f, 0x51, 0xcd, 0xf9, 0x6f, - 0x05, 0x96, 0xe4, 0x28, 0xfa, 0x2b, 0x7b, 0x7d, 0x99, 0x81, 0x0e, 0x7e, 0x0f, 0x52, 0x6e, 0x37, - 0xbe, 0xf7, 0x4e, 0xbf, 0x24, 0x60, 0x02, 0xed, 0x16, 0xe0, 0x49, 0xbb, 0x5f, 0x02, 0xba, 0x7f, - 0x57, 0x61, 0x95, 0x88, 0xe8, 0xfb, 0xe6, 0xff, 0x85, 0x1f, 0xf4, 0xff, 0x85, 0xe7, 0x27, 0xae, - 0x8f, 0xf9, 0x65, 0x6a, 0x1a, 0xea, 0x4f, 0x2f, 0x75, 0x9d, 0x49, 0xb4, 0xa9, 0x73, 0x89, 0xf6, - 0xe5, 0xe3, 0xd1, 0xc7, 0x2a, 0xac, 0x49, 0x43, 0xde, 0xdc, 0x75, 0x2e, 0xee, 0x11, 0xd9, 0x73, - 0x1e, 0xf1, 0x9f, 0x0a, 0x5c, 0x9e, 0x09, 0xe4, 0x0f, 0xfd, 0x46, 0x73, 0xc6, 0x7b, 0xd2, 0x2f, - 0xf4, 0x9e, 0xcc, 0x85, 0xbd, 0xe7, 0x9b, 0x2a, 0x94, 0x08, 0xf5, 0xa8, 0x13, 0xbe, 0xe6, 0xd5, - 0xbd, 0x33, 0x18, 0x66, 0xce, 0xd5, 0x39, 0x97, 0x60, 0x31, 0x01, 0x42, 0x7e, 0x70, 0xf1, 0x0f, - 0x74, 0x96, 0x07, 0x3f, 0xa2, 0x8e, 0x17, 0xc5, 0x37, 0x41, 0xed, 0xcf, 0x54, 0x28, 0x12, 0xc6, - 0x71, 0xfb, 0xb4, 0x15, 0x39, 0x51, 0x88, 0x3f, 0x07, 0x0b, 0x47, 0x5c, 0xc5, 0x1e, 0x7b, 0x48, - 0x9e, 0x14, 0x04, 0x4f, 0xfc, 0xfb, 0xb8, 0x0d, 0xab, 0x21, 0xed, 0xf8, 0x83, 0x6e, 0x68, 0x1f, - 0xd2, 0x23, 0x77, 0xd0, 0xb5, 0xfb, 0x4e, 0x18, 0xd1, 0x80, 0xc3, 0x52, 0x24, 0xcb, 0x52, 0xb8, - 0xcb, 0x65, 0x75, 0x2e, 0xc2, 0x57, 0x61, 0xe5, 0xd0, 0x1d, 0x78, 0x7e, 0xcf, 0x1e, 0x7a, 0xce, - 0x88, 0x06, 0xa1, 0xdd, 0xf1, 0x4f, 0x06, 0x02, 0x8f, 0x0c, 0xc1, 0x42, 0xd6, 0x14, 0xa2, 0x0a, - 0x93, 0xe0, 0x87, 0xb0, 0x39, 0x73, 0x16, 0xfb, 0x91, 0xeb, 0x45, 0x34, 0xa0, 0x5d, 0x3b, 0xa0, - 0x43, 0xcf, 0xed, 0x88, 0x57, 0x52, 0x02, 0xa8, 0x2f, 0xce, 0x98, 0x7a, 0x5f, 0xaa, 0x93, 0xb1, - 0x36, 0xbe, 0x0c, 0xf9, 0xce, 0xf0, 0xc4, 0x3e, 0xe1, 0x8f, 0x16, 0x18, 0x7e, 0x0a, 0xc9, 0x75, - 0x86, 0x27, 0x6d, 0x46, 0x63, 0x04, 0xa9, 0xc7, 0x43, 0x11, 0x9c, 0x15, 0xc2, 0x9a, 0xda, 0xf7, - 0x14, 0x28, 0xe9, 0xbd, 0x5e, 0x40, 0x7b, 0x4e, 0x24, 0x61, 0xba, 0x0a, 0x2b, 0x02, 0x92, 0x91, - 0x2d, 0xdd, 0x55, 0xd8, 0xa3, 0x08, 0x7b, 0xa4, 0x4c, 0xf8, 0xaa, 0xb0, 0x67, 0x07, 0x2e, 0x9d, - 0x0c, 0x66, 0xf6, 0x51, 0x79, 0x9f, 0x95, 0x44, 0x3a, 0xd9, 0xeb, 0x27, 0xe1, 0xed, 0xd9, 0x28, - 0xf4, 0x5d, 0xf1, 0x52, 0xb1, 0x48, 0x2e, 0xcd, 0x30, 0xba, 0xee, 0x0e, 0x9e, 0xd3, 0xd5, 0x79, - 0xca, 0xf1, 0xfa, 0x84, 0xae, 0xce, 0x53, 0xed, 0x2f, 0x92, 0xff, 0x14, 0x63, 0x77, 0x49, 0x02, - 0x47, 0xec, 0xc8, 0xca, 0xf3, 0x1c, 0xb9, 0x0c, 0xf3, 0xcc, 0x19, 0xdd, 0x41, 0x8f, 0x1b, 0x97, - 0x23, 0x31, 0x89, 0x5b, 0xf0, 0x45, 0x69, 0x3b, 0x7d, 0x1a, 0xd1, 0x60, 0xe0, 0x78, 0xde, 0xc8, - 0x16, 0xe5, 0xc7, 0x41, 0x44, 0xbb, 0xf6, 0xf8, 0xe5, 0xa6, 0x08, 0x1f, 0x9f, 0x17, 0xda, 0x46, - 0xa2, 0x4c, 0x12, 0x5d, 0x2b, 0x79, 0xd3, 0xf9, 0x15, 0x28, 0x05, 0xd2, 0x89, 0xed, 0x90, 0x6d, - 0x8f, 0x0c, 0xb9, 0x2b, 0xf1, 0xab, 0x89, 0x49, 0x0f, 0x27, 0xc5, 0x60, 0xca, 0xe1, 0x5f, 0x3a, - 0xe0, 0xdc, 0x49, 0xe7, 0xb2, 0x68, 0x5e, 0xfb, 0x4b, 0x05, 0x96, 0x67, 0x7c, 0xbb, 0x27, 0x85, - 0x01, 0x65, 0xa2, 0xee, 0xf8, 0xe3, 0x90, 0xe1, 0x0f, 0x5a, 0xe4, 0x1b, 0xaa, 0xb7, 0xce, 0x7f, - 0xfa, 0xf3, 0xc7, 0x27, 0x44, 0x68, 0xb1, 0xb3, 0xc8, 0x6d, 0xea, 0xf0, 0xc2, 0x63, 0x1c, 0x51, - 0x0b, 0x8c, 0x27, 0x6a, 0x91, 0xe7, 0x2b, 0x99, 0xe9, 0x17, 0x56, 0x32, 0x37, 0x7f, 0x27, 0x05, - 0xf9, 0xfa, 0xa8, 0xf5, 0xd8, 0xdb, 0xf7, 0x9c, 0x1e, 0x7f, 0x1d, 0x52, 0x6f, 0x5a, 0x0f, 0xd0, - 0x1c, 0x5e, 0x82, 0xa2, 0xd9, 0xb0, 0x6c, 0xb3, 0x5d, 0xab, 0xd9, 0xfb, 0x35, 0xfd, 0x36, 0x52, - 0x30, 0x82, 0x85, 0x26, 0xa9, 0xda, 0x77, 0x8d, 0x07, 0x82, 0xa3, 0xe2, 0x65, 0x58, 0x6c, 0x9b, - 0xd5, 0x7b, 0x6d, 0x63, 0xcc, 0xe4, 0x2f, 0xcc, 0xea, 0xed, 0x9a, 0x55, 0x6d, 0xd6, 0x26, 0xd8, - 0x39, 0x5c, 0x84, 0xfc, 0x6e, 0xad, 0xb1, 0x2b, 0x48, 0xc4, 0xc6, 0x6f, 0x9b, 0xad, 0xea, 0x6d, - 0xd3, 0xd8, 0x13, 0xac, 0x75, 0xc6, 0x7a, 0x68, 0x90, 0xc6, 0x7e, 0x35, 0x9e, 0xf2, 0x16, 0x46, - 0x50, 0xd8, 0xad, 0x9a, 0x3a, 0x91, 0xa3, 0x3c, 0x53, 0x70, 0x09, 0xf2, 0x86, 0xd9, 0xae, 0x4b, - 0x5a, 0xc5, 0x65, 0x58, 0xd6, 0xdb, 0x56, 0xc3, 0xae, 0x9a, 0x15, 0x62, 0xd4, 0x0d, 0xd3, 0x92, - 0x92, 0x34, 0x5e, 0x86, 0x92, 0x55, 0xad, 0x1b, 0x2d, 0x4b, 0xaf, 0x37, 0x25, 0x93, 0xad, 0x22, - 0xd7, 0x32, 0x62, 0x1d, 0x84, 0xd7, 0x60, 0xd5, 0x6c, 0xd8, 0xf1, 0xab, 0xba, 0x03, 0xbd, 0xd6, - 0x36, 0xa4, 0x6c, 0x1d, 0xbf, 0x05, 0xb8, 0x61, 0xda, 0xed, 0xe6, 0x9e, 0x6e, 0x19, 0xb6, 0xd9, - 0xb8, 0x2f, 0x05, 0xb7, 0x70, 0x09, 0x72, 0xe3, 0x15, 0x3c, 0x63, 0x28, 0x14, 0x9b, 0x3a, 0xb1, - 0xc6, 0xc6, 0x3e, 0x7b, 0xc6, 0xc0, 0x82, 0xdb, 0xa4, 0xd1, 0x6e, 0x8e, 0xd5, 0x96, 0xa0, 0x20, - 0xc1, 0x92, 0xac, 0x34, 0x63, 0xed, 0x56, 0xcd, 0x4a, 0xb2, 0xbe, 0x67, 0xb9, 0x35, 0x15, 0x29, - 0x9b, 0xc7, 0x90, 0xe6, 0xdb, 0x91, 0x83, 0xb4, 0xd9, 0x30, 0x0d, 0x34, 0x87, 0x17, 0x01, 0xaa, - 0xad, 0xaa, 0x69, 0x19, 0xb7, 0x89, 0x5e, 0x63, 0x66, 0x73, 0x46, 0x0c, 0x20, 0xb3, 0x76, 0x01, - 0xe6, 0xab, 0xad, 0xfd, 0x5a, 0x43, 0xb7, 0xa4, 0x99, 0xd5, 0xd6, 0xbd, 0x76, 0xc3, 0x62, 0x42, - 0x84, 0x0b, 0x90, 0xad, 0xb6, 0x2c, 0xe3, 0x6b, 0x16, 0xb3, 0x8b, 0xcb, 0x04, 0xaa, 0xe8, 0xd9, - 0xad, 0xcd, 0x6f, 0xa7, 0x20, 0xcd, 0x9f, 0x64, 0x17, 0x21, 0xcf, 0x77, 0xdb, 0x7a, 0xd0, 0x64, - 0x53, 0xe6, 0x21, 0x5d, 0x35, 0xad, 0x9b, 0xe8, 0xe7, 0x55, 0x0c, 0x90, 0x69, 0xf3, 0xf6, 0x2f, - 0x64, 0x59, 0xbb, 0x6a, 0x5a, 0xd7, 0x6e, 0xa0, 0xaf, 0xab, 0x6c, 0xd8, 0xb6, 0x20, 0x7e, 0x31, - 0x16, 0x6c, 0xef, 0xa0, 0x6f, 0x24, 0x82, 0xed, 0x1d, 0xf4, 0x4b, 0xb1, 0xe0, 0xfa, 0x36, 0xfa, - 0x66, 0x22, 0xb8, 0xbe, 0x8d, 0x7e, 0x39, 0x16, 0xdc, 0xd8, 0x41, 0xbf, 0x92, 0x08, 0x6e, 0xec, - 0xa0, 0x5f, 0xcd, 0x32, 0x5b, 0xb8, 0x25, 0xd7, 0xb7, 0xd1, 0xaf, 0xe5, 0x12, 0xea, 0xc6, 0x0e, - 0xfa, 0xf5, 0x1c, 0xdb, 0xff, 0x64, 0x57, 0xd1, 0x6f, 0x20, 0xb6, 0x4c, 0xb6, 0x41, 0xe8, 0x37, - 0x79, 0x93, 0x89, 0xd0, 0x6f, 0x21, 0x66, 0x23, 0xe3, 0x72, 0xf2, 0x5b, 0x5c, 0xf2, 0xc0, 0xd0, - 0x09, 0xfa, 0xed, 0xac, 0x78, 0xe7, 0x59, 0xa9, 0xd6, 0xf5, 0x1a, 0xc2, 0xbc, 0x07, 0x43, 0xe5, - 0x77, 0xaf, 0xb2, 0x26, 0x73, 0x4f, 0xf4, 0x7b, 0x4d, 0x36, 0xe1, 0x81, 0x4e, 0x2a, 0x1f, 0xe9, - 0x04, 0xfd, 0xfe, 0x55, 0x36, 0xe1, 0x81, 0x4e, 0x24, 0x5e, 0x7f, 0xd0, 0x64, 0x8a, 0x5c, 0xf4, - 0x87, 0x57, 0xd9, 0xa2, 0x25, 0xff, 0x8f, 0x9a, 0x38, 0x07, 0xa9, 0xdd, 0xaa, 0x85, 0xbe, 0xcd, - 0x67, 0x63, 0x2e, 0x8a, 0xfe, 0x18, 0x31, 0x66, 0xcb, 0xb0, 0xd0, 0x77, 0x18, 0x33, 0x63, 0xb5, - 0x9b, 0x35, 0x03, 0xbd, 0xc3, 0x16, 0x77, 0xdb, 0x68, 0xd4, 0x0d, 0x8b, 0x3c, 0x40, 0x7f, 0xc2, - 0xd5, 0xef, 0xb4, 0x1a, 0x26, 0xfa, 0x2e, 0xc2, 0x25, 0x00, 0xe3, 0x6b, 0x4d, 0x62, 0xb4, 0x5a, - 0xd5, 0x86, 0x89, 0xde, 0xdf, 0xdc, 0x07, 0x74, 0x36, 0x1c, 0x30, 0x03, 0xda, 0xe6, 0x5d, 0xb3, - 0x71, 0xdf, 0x44, 0x73, 0x8c, 0x68, 0x12, 0xa3, 0xa9, 0x13, 0x03, 0x29, 0x18, 0x20, 0x2b, 0x5f, - 0x8f, 0xf2, 0x67, 0xa0, 0xa4, 0x51, 0xab, 0xed, 0xea, 0x95, 0xbb, 0x28, 0xb5, 0xfb, 0x65, 0x58, - 0x74, 0xfd, 0xad, 0x53, 0x37, 0xa2, 0x61, 0x28, 0x1e, 0xfd, 0x3f, 0xd4, 0x24, 0xe5, 0xfa, 0x57, - 0x44, 0xeb, 0x4a, 0xcf, 0xbf, 0x72, 0x1a, 0x5d, 0xe1, 0xd2, 0x2b, 0x3c, 0x62, 0x1c, 0x66, 0x39, - 0x71, 0xfd, 0xff, 0x02, 0x00, 0x00, 0xff, 0xff, 0x59, 0xe4, 0x3a, 0xdc, 0x52, 0x30, 0x00, 0x00, + 0x67, 0x32, 0x63, 0x0c, 0x38, 0x89, 0xe3, 0x09, 0xe1, 0xce, 0x00, 0x69, 0xcb, 0xed, 0x5c, 0x25, + 0x52, 0x4b, 0x39, 0x6a, 0x39, 0x93, 0x14, 0x55, 0x5d, 0x6d, 0xe9, 0x44, 0xee, 0x72, 0x4b, 0xad, + 0x74, 0xb7, 0x9d, 0x68, 0x17, 0x18, 0x86, 0xe1, 0xcd, 0xf0, 0x66, 0x98, 0x62, 0x8a, 0x2a, 0x16, + 0x14, 0x1b, 0xd6, 0xac, 0x59, 0xdc, 0x05, 0x0b, 0xaa, 0x58, 0x02, 0x55, 0x3c, 0x16, 0x14, 0xac, + 0xa6, 0x28, 0x16, 0x2c, 0x58, 0x50, 0xd4, 0x79, 0x74, 0x4b, 0xb2, 0x75, 0x13, 0x4f, 0x86, 0x29, + 0x2a, 0xb9, 0x59, 0xe9, 0xfc, 0x8f, 0xf3, 0xf8, 0xbf, 0xf3, 0x9f, 0xff, 0x3f, 0xfd, 0xeb, 0x40, + 0xe1, 0xc9, 0x09, 0x0d, 0x46, 0x5b, 0xc3, 0xc0, 0x8f, 0x7c, 0x9c, 0xe1, 0xc4, 0x5a, 0x29, 0xf2, + 0x87, 0x7e, 0xd7, 0x89, 0x1c, 0xc1, 0x5e, 0x2b, 0x9c, 0x46, 0xc1, 0xb0, 0x23, 0x08, 0xed, 0x1b, + 0x0a, 0x64, 0x2d, 0x27, 0xe8, 0xd1, 0x08, 0xaf, 0x41, 0xee, 0x98, 0x8e, 0xc2, 0xa1, 0xd3, 0xa1, + 0x65, 0x65, 0x5d, 0xd9, 0xc8, 0x93, 0x84, 0xc6, 0x2b, 0x90, 0x09, 0x8f, 0x9c, 0xa0, 0x5b, 0x56, + 0xb9, 0x40, 0x10, 0xf8, 0xcb, 0x50, 0x88, 0x9c, 0x43, 0x8f, 0x46, 0x76, 0x34, 0x1a, 0xd2, 0x72, + 0x6a, 0x5d, 0xd9, 0x28, 0x6d, 0xaf, 0x6c, 0x25, 0xf3, 0x59, 0x5c, 0x68, 0x8d, 0x86, 0x94, 0x40, + 0x94, 0xb4, 0x31, 0x86, 0x74, 0x87, 0x7a, 0x5e, 0x39, 0xcd, 0xc7, 0xe2, 0x6d, 0x6d, 0x0f, 0x4a, + 0x07, 0xd6, 0x1d, 0x27, 0xa2, 0x15, 0xc7, 0xf3, 0x68, 0x50, 0xdd, 0x63, 0xcb, 0x39, 0x09, 0x69, + 0x30, 0x70, 0xfa, 0xc9, 0x72, 0x62, 0x1a, 0x5f, 0x82, 0x6c, 0x2f, 0xf0, 0x4f, 0x86, 0x61, 0x59, + 0x5d, 0x4f, 0x6d, 0xe4, 0x89, 0xa4, 0xb4, 0x9f, 0x05, 0x30, 0x4e, 0xe9, 0x20, 0xb2, 0xfc, 0x63, + 0x3a, 0xc0, 0xef, 0x41, 0x3e, 0x72, 0xfb, 0x34, 0x8c, 0x9c, 0xfe, 0x90, 0x0f, 0x91, 0x22, 0x63, + 0xc6, 0xa7, 0x98, 0xb4, 0x06, 0xb9, 0xa1, 0x1f, 0xba, 0x91, 0xeb, 0x0f, 0xb8, 0x3d, 0x79, 0x92, + 0xd0, 0xda, 0x4f, 0x43, 0xe6, 0xc0, 0xf1, 0x4e, 0x28, 0xfe, 0x10, 0xd2, 0xdc, 0x60, 0x85, 0x1b, + 0x5c, 0xd8, 0x12, 0xa0, 0x73, 0x3b, 0xb9, 0x80, 0x8d, 0x7d, 0xca, 0x34, 0xf9, 0xd8, 0x0b, 0x44, + 0x10, 0xda, 0x31, 0x2c, 0xec, 0xba, 0x83, 0xee, 0x81, 0x13, 0xb8, 0x0c, 0x8c, 0x57, 0x1c, 0x06, + 0x7f, 0x01, 0xb2, 0xbc, 0x11, 0x96, 0x53, 0xeb, 0xa9, 0x8d, 0xc2, 0xf6, 0x82, 0xec, 0xc8, 0xd7, + 0x46, 0xa4, 0x4c, 0xfb, 0x2b, 0x05, 0x60, 0xd7, 0x3f, 0x19, 0x74, 0xef, 0x33, 0x21, 0x46, 0x90, + 0x0a, 0x9f, 0x78, 0x12, 0x48, 0xd6, 0xc4, 0xf7, 0xa0, 0x74, 0xe8, 0x0e, 0xba, 0xf6, 0xa9, 0x5c, + 0x8e, 0xc0, 0xb2, 0xb0, 0xfd, 0x05, 0x39, 0xdc, 0xb8, 0xf3, 0xd6, 0xe4, 0xaa, 0x43, 0x63, 0x10, + 0x05, 0x23, 0x52, 0x3c, 0x9c, 0xe4, 0xad, 0xb5, 0x01, 0x9f, 0x57, 0x62, 0x93, 0x1e, 0xd3, 0x51, + 0x3c, 0xe9, 0x31, 0x1d, 0xe1, 0x1f, 0x99, 0xb4, 0xa8, 0xb0, 0xbd, 0x1c, 0xcf, 0x35, 0xd1, 0x57, + 0x9a, 0xf9, 0x91, 0x7a, 0x4b, 0xd1, 0xfe, 0x31, 0x0b, 0x25, 0xe3, 0x19, 0xed, 0x9c, 0x44, 0xb4, + 0x31, 0x64, 0x7b, 0x10, 0xe2, 0x3a, 0x2c, 0xba, 0x83, 0x8e, 0x77, 0xd2, 0xa5, 0x5d, 0xfb, 0xb1, + 0x4b, 0xbd, 0x6e, 0xc8, 0xfd, 0xa8, 0x94, 0xac, 0x7b, 0x5a, 0x7f, 0xab, 0x2a, 0x95, 0xf7, 0xb9, + 0x2e, 0x29, 0xb9, 0x53, 0x34, 0xde, 0x84, 0xa5, 0x8e, 0xe7, 0xd2, 0x41, 0x64, 0x3f, 0x66, 0xf6, + 0xda, 0x81, 0xff, 0x34, 0x2c, 0x67, 0xd6, 0x95, 0x8d, 0x1c, 0x59, 0x14, 0x82, 0x7d, 0xc6, 0x27, + 0xfe, 0xd3, 0x10, 0x7f, 0x04, 0xb9, 0xa7, 0x7e, 0x70, 0xec, 0xf9, 0x4e, 0xb7, 0x9c, 0xe5, 0x73, + 0x7e, 0x30, 0x7b, 0xce, 0x07, 0x52, 0x8b, 0x24, 0xfa, 0x78, 0x03, 0x50, 0xf8, 0xc4, 0xb3, 0x43, + 0xea, 0xd1, 0x4e, 0x64, 0x7b, 0x6e, 0xdf, 0x8d, 0xca, 0x39, 0xee, 0x92, 0xa5, 0xf0, 0x89, 0xd7, + 0xe2, 0xec, 0x1a, 0xe3, 0x62, 0x1b, 0x56, 0xa3, 0xc0, 0x19, 0x84, 0x4e, 0x87, 0x0d, 0x66, 0xbb, + 0xa1, 0xef, 0x39, 0xdc, 0x1d, 0xf3, 0x7c, 0xca, 0xcd, 0xd9, 0x53, 0x5a, 0xe3, 0x2e, 0xd5, 0xb8, + 0x07, 0x59, 0x89, 0x66, 0x70, 0xf1, 0x75, 0x58, 0x0d, 0x8f, 0xdd, 0xa1, 0xcd, 0xc7, 0xb1, 0x87, + 0x9e, 0x33, 0xb0, 0x3b, 0x4e, 0xe7, 0x88, 0x96, 0x81, 0x9b, 0x8d, 0x99, 0x90, 0xef, 0x7b, 0xd3, + 0x73, 0x06, 0x15, 0x26, 0x61, 0xa0, 0x33, 0xbd, 0x01, 0x0d, 0xec, 0x53, 0x1a, 0x84, 0x6c, 0x35, + 0x85, 0x17, 0x81, 0xde, 0x14, 0xca, 0x07, 0x42, 0x97, 0x94, 0x86, 0x53, 0xb4, 0xf6, 0x15, 0x28, + 0x4d, 0x6f, 0x0b, 0x5e, 0x82, 0xa2, 0xf5, 0xb0, 0x69, 0xd8, 0xba, 0xb9, 0x67, 0x9b, 0x7a, 0xdd, + 0x40, 0x73, 0xb8, 0x08, 0x79, 0xce, 0x6a, 0x98, 0xb5, 0x87, 0x48, 0xc1, 0xf3, 0x90, 0xd2, 0x6b, + 0x35, 0xa4, 0x6a, 0xb7, 0x20, 0x17, 0xe3, 0x8b, 0x17, 0xa1, 0xd0, 0x36, 0x5b, 0x4d, 0xa3, 0x52, + 0xdd, 0xaf, 0x1a, 0x7b, 0x68, 0x0e, 0xe7, 0x20, 0xdd, 0xa8, 0x59, 0x4d, 0xa4, 0x88, 0x96, 0xde, + 0x44, 0x2a, 0xeb, 0xb9, 0xb7, 0xab, 0xa3, 0x94, 0xf6, 0x67, 0x0a, 0xac, 0xcc, 0xc2, 0x09, 0x17, + 0x60, 0x7e, 0xcf, 0xd8, 0xd7, 0xdb, 0x35, 0x0b, 0xcd, 0xe1, 0x65, 0x58, 0x24, 0x46, 0xd3, 0xd0, + 0x2d, 0x7d, 0xb7, 0x66, 0xd8, 0xc4, 0xd0, 0xf7, 0x90, 0x82, 0x31, 0x94, 0x58, 0xcb, 0xae, 0x34, + 0xea, 0xf5, 0xaa, 0x65, 0x19, 0x7b, 0x48, 0xc5, 0x2b, 0x80, 0x38, 0xaf, 0x6d, 0x8e, 0xb9, 0x29, + 0x8c, 0x60, 0xa1, 0x65, 0x90, 0xaa, 0x5e, 0xab, 0x3e, 0x62, 0x03, 0xa0, 0x34, 0xfe, 0x1c, 0xbc, + 0x5f, 0x69, 0x98, 0xad, 0x6a, 0xcb, 0x32, 0x4c, 0xcb, 0x6e, 0x99, 0x7a, 0xb3, 0xf5, 0x71, 0xc3, + 0xe2, 0x23, 0x0b, 0xe3, 0x32, 0xb8, 0x04, 0xa0, 0xb7, 0xad, 0x86, 0x18, 0x07, 0x65, 0xb5, 0x36, + 0x94, 0xa6, 0x21, 0x64, 0xab, 0x92, 0x4b, 0xb4, 0x9b, 0x35, 0xdd, 0x34, 0x0d, 0x82, 0xe6, 0x70, + 0x16, 0xd4, 0x83, 0x1b, 0x48, 0xe1, 0xbf, 0x3b, 0x48, 0xc5, 0x0b, 0x90, 0x3b, 0xd8, 0xb9, 0x13, + 0x50, 0xda, 0x1d, 0x89, 0x95, 0x1c, 0xec, 0xd4, 0xe8, 0xe3, 0x68, 0x9b, 0xb8, 0xbd, 0xa3, 0x08, + 0xa5, 0xef, 0xa6, 0x73, 0x0a, 0x52, 0xef, 0xa6, 0x73, 0x2a, 0x4a, 0xdd, 0x4d, 0xe7, 0x52, 0x28, + 0xad, 0xfd, 0xa5, 0x0a, 0x19, 0xbe, 0x05, 0x2c, 0x28, 0x4f, 0x84, 0x5a, 0xde, 0x4e, 0x02, 0x94, + 0xfa, 0x82, 0x00, 0xc5, 0xe3, 0xba, 0x0c, 0x95, 0x82, 0xc0, 0x97, 0x21, 0xef, 0x07, 0x3d, 0x5b, + 0x48, 0x44, 0x90, 0xcf, 0xf9, 0x41, 0x8f, 0x67, 0x03, 0x16, 0x60, 0x59, 0x6e, 0x38, 0x74, 0x42, + 0xca, 0xcf, 0x59, 0x9e, 0x24, 0x34, 0x7e, 0x17, 0x98, 0x9e, 0xcd, 0xd7, 0x91, 0xe5, 0xb2, 0x79, + 0x3f, 0xe8, 0x99, 0x6c, 0x29, 0x9f, 0x87, 0x62, 0xc7, 0xf7, 0x4e, 0xfa, 0x03, 0xdb, 0xa3, 0x83, + 0x5e, 0x74, 0x54, 0x9e, 0x5f, 0x57, 0x36, 0x8a, 0x64, 0x41, 0x30, 0x6b, 0x9c, 0x87, 0xcb, 0x30, + 0xdf, 0x39, 0x72, 0x82, 0x90, 0x8a, 0xb3, 0x55, 0x24, 0x31, 0xc9, 0x67, 0xa5, 0x1d, 0xb7, 0xef, + 0x78, 0x21, 0x3f, 0x47, 0x45, 0x92, 0xd0, 0xcc, 0x88, 0xc7, 0x9e, 0xd3, 0x0b, 0xb9, 0xff, 0x17, + 0x89, 0x20, 0xf0, 0x87, 0x50, 0x90, 0x13, 0x72, 0x08, 0x0a, 0x7c, 0x39, 0x20, 0x58, 0x0c, 0x01, + 0xed, 0x27, 0x20, 0x45, 0xfc, 0xa7, 0x6c, 0x4e, 0xb1, 0xa2, 0xb0, 0xac, 0xac, 0xa7, 0x36, 0x30, + 0x89, 0x49, 0x96, 0xa4, 0x64, 0x9c, 0x16, 0xe1, 0x3b, 0x8e, 0xcc, 0xdf, 0x51, 0xa0, 0xc0, 0xcf, + 0x17, 0xa1, 0xe1, 0x89, 0x17, 0xb1, 0x78, 0x2e, 0x03, 0x99, 0x32, 0x15, 0xcf, 0xf9, 0xbe, 0x10, + 0x29, 0x63, 0x00, 0xb0, 0xd8, 0x64, 0x3b, 0x8f, 0x1f, 0xd3, 0x4e, 0x44, 0x45, 0xda, 0x4a, 0x93, + 0x05, 0xc6, 0xd4, 0x25, 0x8f, 0x21, 0xef, 0x0e, 0x42, 0x1a, 0x44, 0xb6, 0xdb, 0xe5, 0x7b, 0x92, + 0x26, 0x39, 0xc1, 0xa8, 0x76, 0xf1, 0x07, 0x90, 0xe6, 0xd1, 0x2d, 0xcd, 0x67, 0x01, 0x39, 0x0b, + 0xf1, 0x9f, 0x12, 0xce, 0xbf, 0x9b, 0xce, 0x65, 0x50, 0x56, 0xfb, 0x2a, 0x2c, 0xf0, 0xc5, 0x3d, + 0x70, 0x82, 0x81, 0x3b, 0xe8, 0xf1, 0x64, 0xed, 0x77, 0x85, 0x5f, 0x14, 0x09, 0x6f, 0x33, 0x9b, + 0xfb, 0x34, 0x0c, 0x9d, 0x1e, 0x95, 0xc9, 0x33, 0x26, 0xb5, 0x3f, 0x49, 0x41, 0xa1, 0x15, 0x05, + 0xd4, 0xe9, 0xf3, 0x3c, 0x8c, 0xbf, 0x0a, 0x10, 0x46, 0x4e, 0x44, 0xfb, 0x74, 0x10, 0xc5, 0xf6, + 0xbd, 0x27, 0x67, 0x9e, 0xd0, 0xdb, 0x6a, 0xc5, 0x4a, 0x64, 0x42, 0x1f, 0x6f, 0x43, 0x81, 0x32, + 0xb1, 0x1d, 0xb1, 0x7c, 0x2e, 0x73, 0xc6, 0x52, 0x1c, 0x72, 0x92, 0x44, 0x4f, 0x80, 0x26, 0xed, + 0xb5, 0xef, 0xaa, 0x90, 0x4f, 0x46, 0xc3, 0x3a, 0xe4, 0x3a, 0x4e, 0x44, 0x7b, 0x7e, 0x30, 0x92, + 0x69, 0xf6, 0xca, 0x8b, 0x66, 0xdf, 0xaa, 0x48, 0x65, 0x92, 0x74, 0xc3, 0xef, 0x83, 0xb8, 0xbb, + 0x08, 0xb7, 0x14, 0xf6, 0xe6, 0x39, 0x87, 0x3b, 0xe6, 0x47, 0x80, 0x87, 0x81, 0xdb, 0x77, 0x82, + 0x91, 0x7d, 0x4c, 0x47, 0x71, 0x4a, 0x4a, 0xcd, 0xd8, 0x49, 0x24, 0xf5, 0xee, 0xd1, 0x91, 0x8c, + 0x7a, 0xb7, 0xa6, 0xfb, 0x4a, 0x6f, 0x39, 0xbf, 0x3f, 0x13, 0x3d, 0x79, 0x92, 0x0f, 0xe3, 0x74, + 0x9e, 0xe1, 0x8e, 0xc5, 0x9a, 0xda, 0x97, 0x20, 0x17, 0x2f, 0x1e, 0xe7, 0x21, 0x63, 0x04, 0x81, + 0x1f, 0xa0, 0x39, 0x1e, 0xfc, 0xea, 0x35, 0x11, 0x3f, 0xf7, 0xf6, 0x58, 0xfc, 0xfc, 0x17, 0x35, + 0xc9, 0xa9, 0x84, 0x3e, 0x39, 0xa1, 0x61, 0x84, 0x7f, 0x06, 0x96, 0x29, 0x77, 0x21, 0xf7, 0x94, + 0xda, 0x1d, 0x7e, 0x01, 0x63, 0x0e, 0xa4, 0x70, 0xbc, 0x17, 0xb7, 0xc4, 0x7d, 0x31, 0xbe, 0x98, + 0x91, 0xa5, 0x44, 0x57, 0xb2, 0xba, 0xd8, 0x80, 0x65, 0xb7, 0xdf, 0xa7, 0x5d, 0xd7, 0x89, 0x26, + 0x07, 0x10, 0x1b, 0xb6, 0x1a, 0xdf, 0x4f, 0xa6, 0xee, 0x77, 0x64, 0x29, 0xe9, 0x91, 0x0c, 0x73, + 0x05, 0xb2, 0x11, 0xbf, 0x8b, 0x72, 0xdf, 0x2d, 0x6c, 0x17, 0xe3, 0x88, 0xc3, 0x99, 0x44, 0x0a, + 0xf1, 0x97, 0x40, 0xdc, 0x6c, 0x79, 0x6c, 0x19, 0x3b, 0xc4, 0xf8, 0xc2, 0x42, 0x84, 0x1c, 0x5f, + 0x81, 0xd2, 0x54, 0x2a, 0xed, 0x72, 0xc0, 0x52, 0xa4, 0x38, 0x99, 0x17, 0xbb, 0xf8, 0x2a, 0xcc, + 0xfb, 0x22, 0x71, 0xf1, 0xa8, 0x33, 0x5e, 0xf1, 0x74, 0x56, 0x23, 0xb1, 0x16, 0x8b, 0x0d, 0x01, + 0x0d, 0x69, 0x70, 0x4a, 0xbb, 0x6c, 0xd0, 0x79, 0x3e, 0x28, 0xc4, 0xac, 0x6a, 0x57, 0xfb, 0x29, + 0x58, 0x4c, 0x20, 0x0e, 0x87, 0xfe, 0x20, 0xa4, 0x78, 0x13, 0xb2, 0x01, 0x3f, 0xef, 0x12, 0x56, + 0x2c, 0xe7, 0x98, 0x88, 0x04, 0x44, 0x6a, 0x68, 0x5d, 0x58, 0x14, 0x9c, 0x07, 0x6e, 0x74, 0xc4, + 0x77, 0x12, 0x5f, 0x81, 0x0c, 0x65, 0x8d, 0x33, 0x9b, 0x42, 0x9a, 0x15, 0x2e, 0x27, 0x42, 0x3a, + 0x31, 0x8b, 0xfa, 0xd2, 0x59, 0xfe, 0x43, 0x85, 0x65, 0xb9, 0xca, 0x5d, 0x27, 0xea, 0x1c, 0xbd, + 0xa6, 0xde, 0xf0, 0xa3, 0x30, 0xcf, 0xf8, 0x6e, 0x72, 0x72, 0x66, 0xf8, 0x43, 0xac, 0xc1, 0x3c, + 0xc2, 0x09, 0xed, 0x89, 0xed, 0x97, 0x77, 0xbd, 0xa2, 0x13, 0x4e, 0xdc, 0x0c, 0x66, 0x38, 0x4e, + 0xf6, 0x25, 0x8e, 0x33, 0x7f, 0x11, 0xc7, 0xd1, 0xf6, 0x60, 0x65, 0x1a, 0x71, 0xe9, 0x1c, 0x3f, + 0x06, 0xf3, 0x62, 0x53, 0xe2, 0x18, 0x39, 0x6b, 0xdf, 0x62, 0x15, 0xed, 0x13, 0x15, 0x56, 0x64, + 0xf8, 0xfa, 0x6c, 0x9c, 0xe3, 0x09, 0x9c, 0x33, 0x17, 0x3a, 0xa0, 0x17, 0xdb, 0x3f, 0xad, 0x02, + 0xab, 0x67, 0x70, 0x7c, 0x85, 0xc3, 0xfa, 0x3d, 0x05, 0x16, 0x76, 0x69, 0xcf, 0x1d, 0xbc, 0xa6, + 0xbb, 0x30, 0x01, 0x6e, 0xfa, 0x42, 0x4e, 0x3c, 0x84, 0xa2, 0xb4, 0x57, 0xa2, 0x75, 0x1e, 0x6d, + 0x65, 0xd6, 0x69, 0xb9, 0x05, 0x0b, 0xb2, 0x5a, 0xe0, 0x78, 0xae, 0x13, 0x26, 0xf6, 0x9c, 0x29, + 0x17, 0xe8, 0x4c, 0x48, 0x64, 0x61, 0x81, 0x13, 0xda, 0xbf, 0x2a, 0x50, 0xac, 0xf8, 0xfd, 0xbe, + 0x1b, 0xbd, 0xa6, 0x18, 0x9f, 0x47, 0x28, 0x3d, 0xcb, 0x1f, 0xaf, 0x43, 0x29, 0x36, 0x53, 0x42, + 0x7b, 0x26, 0xd3, 0x28, 0xe7, 0x32, 0xcd, 0xbf, 0x29, 0xb0, 0x48, 0x7c, 0xcf, 0x3b, 0x74, 0x3a, + 0xc7, 0x6f, 0x36, 0x38, 0x37, 0x00, 0x8d, 0x0d, 0xbd, 0x28, 0x3c, 0xff, 0xad, 0x40, 0xa9, 0x19, + 0xd0, 0xa1, 0x13, 0xd0, 0x37, 0x1a, 0x1d, 0x76, 0x4d, 0xef, 0x46, 0xf2, 0x82, 0x93, 0x27, 0xbc, + 0xad, 0x2d, 0xc1, 0x62, 0x62, 0xbb, 0x00, 0x4c, 0xfb, 0x7b, 0x05, 0x56, 0x85, 0x8b, 0x49, 0x49, + 0xf7, 0x35, 0x85, 0x25, 0xb6, 0x37, 0x3d, 0x61, 0x6f, 0x19, 0x2e, 0x9d, 0xb5, 0x4d, 0x9a, 0xfd, + 0x75, 0x15, 0xde, 0x89, 0x9d, 0xe7, 0x35, 0x37, 0xfc, 0x07, 0xf0, 0x87, 0x35, 0x28, 0x9f, 0x07, + 0x41, 0x22, 0xf4, 0x2d, 0x15, 0xca, 0x95, 0x80, 0x3a, 0x11, 0x9d, 0xb8, 0x07, 0xbd, 0x39, 0xbe, + 0x81, 0xaf, 0xc3, 0xc2, 0xd0, 0x09, 0x22, 0xb7, 0xe3, 0x0e, 0x1d, 0xf6, 0x29, 0x9a, 0xe1, 0xd7, + 0xac, 0x33, 0x03, 0x4c, 0xa9, 0x68, 0x97, 0xe1, 0xdd, 0x19, 0x88, 0x48, 0xbc, 0xfe, 0x47, 0x01, + 0xdc, 0x8a, 0x9c, 0x20, 0xfa, 0x0c, 0xe4, 0xa5, 0x99, 0xce, 0xb4, 0x0a, 0xcb, 0x53, 0xf6, 0x4f, + 0xe2, 0x42, 0xa3, 0xcf, 0x44, 0x4a, 0xfa, 0x54, 0x5c, 0x26, 0xed, 0x97, 0xb8, 0xfc, 0x93, 0x02, + 0x6b, 0x15, 0x5f, 0x14, 0x3d, 0xdf, 0xc8, 0x13, 0xa6, 0xbd, 0x0f, 0x97, 0x67, 0x1a, 0x28, 0x01, + 0xf8, 0x07, 0x05, 0x2e, 0x11, 0xea, 0x74, 0xdf, 0x4c, 0xe3, 0xef, 0xc3, 0x3b, 0xe7, 0x8c, 0x93, + 0x77, 0x94, 0x9b, 0x90, 0xeb, 0xd3, 0xc8, 0x61, 0x37, 0x5c, 0x69, 0xd2, 0x5a, 0x3c, 0xee, 0x58, + 0xbb, 0x2e, 0x35, 0x48, 0xa2, 0xab, 0xfd, 0xb3, 0x0a, 0xcb, 0xfc, 0x9e, 0xfd, 0xf6, 0x23, 0xef, + 0x42, 0x55, 0x98, 0xec, 0xd9, 0xcb, 0x1f, 0x53, 0x18, 0x06, 0xd4, 0x8e, 0xab, 0x03, 0xf3, 0xfc, + 0xaf, 0x42, 0x18, 0x06, 0xf4, 0xbe, 0xe0, 0x68, 0x7f, 0xad, 0xc0, 0xca, 0x34, 0xc4, 0xc9, 0x17, + 0xcd, 0xff, 0x75, 0xb5, 0x65, 0x46, 0x48, 0x49, 0x5d, 0xe4, 0x23, 0x29, 0x7d, 0xe1, 0x8f, 0xa4, + 0xbf, 0x51, 0xa1, 0x3c, 0x69, 0xcc, 0xdb, 0x9a, 0xce, 0x74, 0x4d, 0xe7, 0xfb, 0xad, 0xf2, 0x69, + 0x7f, 0xab, 0xc0, 0xbb, 0x33, 0x00, 0xfd, 0xfe, 0x5c, 0x64, 0xa2, 0xb2, 0xa3, 0xbe, 0xb4, 0xb2, + 0xf3, 0xc3, 0x77, 0x92, 0xbf, 0x53, 0x60, 0xa5, 0x2e, 0x6a, 0xf5, 0xa2, 0xf2, 0xf1, 0xfa, 0xc6, + 0x60, 0x5e, 0x8e, 0x4f, 0x8f, 0xff, 0xad, 0xd2, 0x2a, 0xb0, 0x7a, 0xc6, 0xb4, 0x57, 0xa8, 0xe6, + 0xfc, 0x97, 0x02, 0x4b, 0x72, 0x14, 0xfd, 0xb5, 0xbd, 0xbe, 0xcc, 0x40, 0x07, 0x7f, 0x00, 0x29, + 0xb7, 0x1b, 0xdf, 0x7b, 0xa7, 0x9f, 0x0c, 0x30, 0x81, 0x76, 0x1b, 0xf0, 0xa4, 0xdd, 0xaf, 0x00, + 0xdd, 0xbf, 0xab, 0xb0, 0x4a, 0x44, 0xf4, 0x7d, 0xfb, 0xff, 0xc2, 0x0f, 0xfa, 0xff, 0xc2, 0x8b, + 0x13, 0xd7, 0x27, 0xfc, 0x32, 0x35, 0x0d, 0xf5, 0x0f, 0x2f, 0x75, 0x9d, 0x49, 0xb4, 0xa9, 0x73, + 0x89, 0xf6, 0xd5, 0xe3, 0xd1, 0x27, 0x2a, 0xac, 0x49, 0x43, 0xde, 0xde, 0x75, 0x2e, 0xee, 0x11, + 0xd9, 0x73, 0x1e, 0xf1, 0x9f, 0x0a, 0x5c, 0x9e, 0x09, 0xe4, 0xff, 0xfb, 0x8d, 0xe6, 0x8c, 0xf7, + 0xa4, 0x5f, 0xea, 0x3d, 0x99, 0x0b, 0x7b, 0xcf, 0x37, 0x55, 0x28, 0x11, 0xea, 0x51, 0x27, 0x7c, + 0xc3, 0xab, 0x7b, 0x67, 0x30, 0xcc, 0x9c, 0xab, 0x73, 0x2e, 0xc1, 0x62, 0x02, 0x84, 0xfc, 0xe0, + 0xe2, 0x1f, 0xe8, 0x2c, 0x0f, 0x7e, 0x4c, 0x1d, 0x2f, 0x8a, 0x6f, 0x82, 0xda, 0x9f, 0xaa, 0x50, + 0x24, 0x8c, 0xe3, 0xf6, 0x69, 0x2b, 0x72, 0xa2, 0x10, 0x7f, 0x0e, 0x16, 0x8e, 0xb8, 0x8a, 0x3d, + 0xf6, 0x90, 0x3c, 0x29, 0x08, 0x9e, 0xf8, 0xf7, 0x71, 0x1b, 0x56, 0x43, 0xda, 0xf1, 0x07, 0xdd, + 0xd0, 0x3e, 0xa4, 0x47, 0xee, 0xa0, 0x6b, 0xf7, 0x9d, 0x30, 0xa2, 0x01, 0x87, 0xa5, 0x48, 0x96, + 0xa5, 0x70, 0x97, 0xcb, 0xea, 0x5c, 0x84, 0xaf, 0xc1, 0xca, 0xa1, 0x3b, 0xf0, 0xfc, 0x9e, 0x3d, + 0xf4, 0x9c, 0x11, 0x0d, 0x42, 0xbb, 0xe3, 0x9f, 0x0c, 0x04, 0x1e, 0x19, 0x82, 0x85, 0xac, 0x29, + 0x44, 0x15, 0x26, 0xc1, 0x8f, 0x60, 0x73, 0xe6, 0x2c, 0xf6, 0x63, 0xd7, 0x8b, 0x68, 0x40, 0xbb, + 0x76, 0x40, 0x87, 0x9e, 0xdb, 0x11, 0xcf, 0xa1, 0x04, 0x50, 0x5f, 0x9c, 0x31, 0xf5, 0xbe, 0x54, + 0x27, 0x63, 0x6d, 0x7c, 0x19, 0xf2, 0x9d, 0xe1, 0x89, 0x7d, 0xc2, 0x1f, 0x2d, 0x30, 0xfc, 0x14, + 0x92, 0xeb, 0x0c, 0x4f, 0xda, 0x8c, 0xc6, 0x08, 0x52, 0x4f, 0x86, 0x22, 0x38, 0x2b, 0x84, 0x35, + 0xb5, 0xef, 0x29, 0x50, 0xd2, 0x7b, 0xbd, 0x80, 0xf6, 0x9c, 0x48, 0xc2, 0x74, 0x0d, 0x56, 0x04, + 0x24, 0x23, 0x5b, 0xba, 0xab, 0xb0, 0x47, 0x11, 0xf6, 0x48, 0x99, 0xf0, 0x55, 0x61, 0xcf, 0x0e, + 0x5c, 0x3a, 0x19, 0xcc, 0xec, 0xa3, 0xf2, 0x3e, 0x2b, 0x89, 0x74, 0xb2, 0xd7, 0x4f, 0xc2, 0xbb, + 0xb3, 0x51, 0xe8, 0xbb, 0xe2, 0x49, 0x62, 0x91, 0x5c, 0x9a, 0x61, 0x74, 0xdd, 0x1d, 0xbc, 0xa0, + 0xab, 0xf3, 0x8c, 0xe3, 0xf5, 0x29, 0x5d, 0x9d, 0x67, 0xda, 0x9f, 0x27, 0xff, 0x29, 0xc6, 0xee, + 0x92, 0x04, 0x8e, 0xd8, 0x91, 0x95, 0x17, 0x39, 0x72, 0x19, 0xe6, 0x99, 0x33, 0xba, 0x83, 0x1e, + 0x37, 0x2e, 0x47, 0x62, 0x12, 0xb7, 0xe0, 0x8b, 0xd2, 0x76, 0xfa, 0x2c, 0xa2, 0xc1, 0xc0, 0xf1, + 0xbc, 0x91, 0x2d, 0xca, 0x8f, 0x83, 0x88, 0x76, 0xed, 0xf1, 0x13, 0x4d, 0x11, 0x3e, 0x3e, 0x2f, + 0xb4, 0x8d, 0x44, 0x99, 0x24, 0xba, 0x56, 0xf2, 0x78, 0xf3, 0x2b, 0x50, 0x0a, 0xa4, 0x13, 0xdb, + 0x21, 0xdb, 0x1e, 0x19, 0x72, 0x57, 0xe2, 0x57, 0x13, 0x93, 0x1e, 0x4e, 0x8a, 0xc1, 0x94, 0xc3, + 0xbf, 0x72, 0xc0, 0xb9, 0x9b, 0xce, 0x65, 0xd1, 0xbc, 0xf6, 0x17, 0x0a, 0x2c, 0xcf, 0xf8, 0x76, + 0x4f, 0x0a, 0x03, 0xca, 0x44, 0xdd, 0xf1, 0xc7, 0x21, 0xc3, 0x1f, 0xb4, 0xc8, 0x37, 0x54, 0xef, + 0x9c, 0xff, 0xf4, 0xe7, 0x8f, 0x4f, 0x88, 0xd0, 0x62, 0x67, 0x91, 0xdb, 0xd4, 0xe1, 0x85, 0xc7, + 0x38, 0xa2, 0x16, 0x18, 0x4f, 0xd4, 0x22, 0xcf, 0x57, 0x32, 0xd3, 0x2f, 0xad, 0x64, 0x6e, 0xfe, + 0x76, 0x0a, 0xf2, 0xf5, 0x51, 0xeb, 0x89, 0xb7, 0xef, 0x39, 0x3d, 0xfe, 0x3a, 0xa4, 0xde, 0xb4, + 0x1e, 0xa2, 0x39, 0xbc, 0x04, 0x45, 0xb3, 0x61, 0xd9, 0x66, 0xbb, 0x56, 0xb3, 0xf7, 0x6b, 0xfa, + 0x1d, 0xa4, 0x60, 0x04, 0x0b, 0x4d, 0x52, 0xb5, 0xef, 0x19, 0x0f, 0x05, 0x47, 0xc5, 0xcb, 0xb0, + 0xd8, 0x36, 0xab, 0xf7, 0xdb, 0xc6, 0x98, 0x99, 0xc6, 0xab, 0xb0, 0x54, 0x6f, 0xd7, 0xac, 0x6a, + 0xb3, 0x36, 0xc1, 0xce, 0xe1, 0x22, 0xe4, 0x77, 0x6b, 0x8d, 0x5d, 0x41, 0x22, 0x36, 0x7e, 0xdb, + 0x6c, 0x55, 0xef, 0x98, 0xc6, 0x9e, 0x60, 0xad, 0x33, 0xd6, 0x23, 0x83, 0x34, 0xf6, 0xab, 0xf1, + 0x94, 0xb7, 0x31, 0x82, 0xc2, 0x6e, 0xd5, 0xd4, 0x89, 0x1c, 0xe5, 0xb9, 0x82, 0x4b, 0x90, 0x37, + 0xcc, 0x76, 0x5d, 0xd2, 0x2a, 0x2e, 0xc3, 0xb2, 0xde, 0xb6, 0x1a, 0x76, 0xd5, 0xac, 0x10, 0xa3, + 0x6e, 0x98, 0x96, 0x94, 0xa4, 0xf1, 0x32, 0x94, 0xac, 0x6a, 0xdd, 0x68, 0x59, 0x7a, 0xbd, 0x29, + 0x99, 0x6c, 0x15, 0xb9, 0x96, 0x11, 0xeb, 0x20, 0xbc, 0x06, 0xab, 0x66, 0xc3, 0x8e, 0x9f, 0xcf, + 0x1d, 0xe8, 0xb5, 0xb6, 0x21, 0x65, 0xeb, 0xf8, 0x1d, 0xc0, 0x0d, 0xd3, 0x6e, 0x37, 0xf7, 0x74, + 0xcb, 0xb0, 0xcd, 0xc6, 0x03, 0x29, 0xb8, 0x8d, 0x4b, 0x90, 0x1b, 0xaf, 0xe0, 0x39, 0x43, 0xa1, + 0xd8, 0xd4, 0x89, 0x35, 0x36, 0xf6, 0xf9, 0x73, 0x06, 0x16, 0xdc, 0x21, 0x8d, 0x76, 0x73, 0xac, + 0xb6, 0x04, 0x05, 0x09, 0x96, 0x64, 0xa5, 0x19, 0x6b, 0xb7, 0x6a, 0x56, 0x92, 0xf5, 0x3d, 0xcf, + 0xad, 0xa9, 0x48, 0xd9, 0x3c, 0x86, 0x34, 0xdf, 0x8e, 0x1c, 0xa4, 0xcd, 0x86, 0x69, 0xa0, 0x39, + 0xbc, 0x08, 0x50, 0x6d, 0x55, 0x4d, 0xcb, 0xb8, 0x43, 0xf4, 0x1a, 0x33, 0x9b, 0x33, 0x62, 0x00, + 0x99, 0xb5, 0x0b, 0x30, 0x5f, 0x6d, 0xed, 0xd7, 0x1a, 0xba, 0x25, 0xcd, 0xac, 0xb6, 0xee, 0xb7, + 0x1b, 0x16, 0x13, 0x22, 0x5c, 0x80, 0x6c, 0xb5, 0x65, 0x19, 0x5f, 0xb3, 0x98, 0x5d, 0x5c, 0x26, + 0x50, 0x45, 0xcf, 0x6f, 0x6f, 0x7e, 0x3b, 0x05, 0x69, 0xfe, 0xf6, 0xba, 0x08, 0x79, 0xbe, 0xdb, + 0xd6, 0xc3, 0x26, 0x9b, 0x32, 0x0f, 0xe9, 0xaa, 0x69, 0xdd, 0x42, 0x3f, 0xa7, 0x62, 0x80, 0x4c, + 0x9b, 0xb7, 0x7f, 0x3e, 0xcb, 0xda, 0x55, 0xd3, 0xba, 0x7e, 0x13, 0x7d, 0x5d, 0x65, 0xc3, 0xb6, + 0x05, 0xf1, 0x0b, 0xb1, 0x60, 0x7b, 0x07, 0x7d, 0x23, 0x11, 0x6c, 0xef, 0xa0, 0x5f, 0x8c, 0x05, + 0x37, 0xb6, 0xd1, 0x37, 0x13, 0xc1, 0x8d, 0x6d, 0xf4, 0x4b, 0xb1, 0xe0, 0xe6, 0x0e, 0xfa, 0xe5, + 0x44, 0x70, 0x73, 0x07, 0xfd, 0x4a, 0x96, 0xd9, 0xc2, 0x2d, 0xb9, 0xb1, 0x8d, 0x7e, 0x35, 0x97, + 0x50, 0x37, 0x77, 0xd0, 0xaf, 0xe5, 0xd8, 0xfe, 0x27, 0xbb, 0x8a, 0x7e, 0x1d, 0xb1, 0x65, 0xb2, + 0x0d, 0x42, 0xbf, 0xc1, 0x9b, 0x4c, 0x84, 0x7e, 0x13, 0x31, 0x1b, 0x19, 0x97, 0x93, 0xdf, 0xe2, + 0x92, 0x87, 0x86, 0x4e, 0xd0, 0x6f, 0x65, 0xc5, 0x83, 0xce, 0x4a, 0xb5, 0xae, 0xd7, 0x10, 0xe6, + 0x3d, 0x18, 0x2a, 0xbf, 0x73, 0x8d, 0x35, 0x99, 0x7b, 0xa2, 0xdf, 0x6d, 0xb2, 0x09, 0x0f, 0x74, + 0x52, 0xf9, 0x58, 0x27, 0xe8, 0xf7, 0xae, 0xb1, 0x09, 0x0f, 0x74, 0x22, 0xf1, 0xfa, 0xfd, 0x26, + 0x53, 0xe4, 0xa2, 0x3f, 0xb8, 0xc6, 0x16, 0x2d, 0xf9, 0x7f, 0xd8, 0xc4, 0x39, 0x48, 0xed, 0x56, + 0x2d, 0xf4, 0x6d, 0x3e, 0x1b, 0x73, 0x51, 0xf4, 0x47, 0x88, 0x31, 0x5b, 0x86, 0x85, 0xbe, 0xc3, + 0x98, 0x19, 0xab, 0xdd, 0xac, 0x19, 0xe8, 0x3d, 0xb6, 0xb8, 0x3b, 0x46, 0xa3, 0x6e, 0x58, 0xe4, + 0x21, 0xfa, 0x63, 0xae, 0x7e, 0xb7, 0xd5, 0x30, 0xd1, 0x77, 0x11, 0x2e, 0x01, 0x18, 0x5f, 0x6b, + 0x12, 0xa3, 0xd5, 0xaa, 0x36, 0x4c, 0xf4, 0xe1, 0xe6, 0x3e, 0xa0, 0xb3, 0xe1, 0x80, 0x19, 0xd0, + 0x36, 0xef, 0x99, 0x8d, 0x07, 0x26, 0x9a, 0x63, 0x44, 0x93, 0x18, 0x4d, 0x9d, 0x18, 0x48, 0xc1, + 0x00, 0x59, 0xf9, 0x4c, 0x94, 0xbf, 0xf7, 0x24, 0x8d, 0x5a, 0x6d, 0x57, 0xaf, 0xdc, 0x43, 0xa9, + 0xdd, 0x2f, 0xc3, 0xa2, 0xeb, 0x6f, 0x9d, 0xba, 0x11, 0x0d, 0x43, 0xf1, 0xba, 0xff, 0x91, 0x26, + 0x29, 0xd7, 0xbf, 0x2a, 0x5a, 0x57, 0x7b, 0xfe, 0xd5, 0xd3, 0xe8, 0x2a, 0x97, 0x5e, 0xe5, 0x11, + 0xe3, 0x30, 0xcb, 0x89, 0x1b, 0xff, 0x1b, 0x00, 0x00, 0xff, 0xff, 0xbd, 0xb2, 0x28, 0xeb, 0x3b, + 0x30, 0x00, 0x00, } diff --git a/go/vt/vtgate/planbuilder/builder.go b/go/vt/vtgate/planbuilder/builder.go index cfd2c2c1a19..2eb3a21fbc7 100644 --- a/go/vt/vtgate/planbuilder/builder.go +++ b/go/vt/vtgate/planbuilder/builder.go @@ -59,14 +59,12 @@ type PlannerVersion = querypb.ExecuteOptions_PlannerVersion const ( // V3 is also the default planner V3 = querypb.ExecuteOptions_V3 - // V4 is the new planner + // V4 uses the default V4 planner, which is the greedy planner V4 = querypb.ExecuteOptions_V4 // V4GreedyOnly uses only the faster greedy planner V4GreedyOnly = querypb.ExecuteOptions_V4Greedy // V4Left2Right tries to emulate the V3 planner by only joining plans in the order they are listed in the FROM-clause V4Left2Right = querypb.ExecuteOptions_V4Left2Right - // V4GreedyOptimized uses only the faster greedy planner - V4GreedyOptimized = querypb.ExecuteOptions_V4GreedyOptimized ) type truncater interface { diff --git a/go/vt/vtgate/planbuilder/dptable.go b/go/vt/vtgate/planbuilder/dptable.go deleted file mode 100644 index c65870cddc2..00000000000 --- a/go/vt/vtgate/planbuilder/dptable.go +++ /dev/null @@ -1,59 +0,0 @@ -/* -Copyright 2020 The Vitess 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 planbuilder - -import "vitess.io/vitess/go/vt/vtgate/semantics" - -// dpTable, is the hashmap we store results in during -// the dynamic programming part of query planning -type dpTable struct { - // hash map of the best solution for each seen table - m map[semantics.TableSet]joinTree - - highest semantics.TableSet -} - -func makeDPTable() *dpTable { - return &dpTable{ - m: map[semantics.TableSet]joinTree{}, - } -} - -func (dpt *dpTable) add(tree joinTree) { - solved := tree.solves() - if dpt.highest < solved { - dpt.highest = solved - } - dpt.m[solved] = tree -} - -func (dpt *dpTable) planFor(id semantics.TableSet) joinTree { - return dpt.m[id] -} - -func (dpt *dpTable) bitSetsOfSize(wanted int) []joinTree { - var result []joinTree - for x := semantics.TableSet(1); x <= dpt.highest; x++ { - if x.NumberOfTables() == wanted { - t, ok := dpt.m[x] - if ok { - result = append(result, t) - } - } - } - return result -} diff --git a/go/vt/vtgate/planbuilder/dptable_test.go b/go/vt/vtgate/planbuilder/dptable_test.go deleted file mode 100644 index e0e10e1b153..00000000000 --- a/go/vt/vtgate/planbuilder/dptable_test.go +++ /dev/null @@ -1,63 +0,0 @@ -/* -Copyright 2020 The Vitess 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 planbuilder - -import ( - "testing" - - "github.com/stretchr/testify/assert" - - "vitess.io/vitess/go/vt/vtgate/semantics" -) - -type fakePlan struct { - solve semantics.TableSet -} - -func (f *fakePlan) solves() semantics.TableSet { - return f.solve -} - -func (f *fakePlan) cost() int { - return 1 -} - -var _ joinTree = (*fakePlan)(nil) - -func TestDpTableSizeOf(t *testing.T) { - dpTable := makeDPTable() - - a := semantics.TableSet(1) - b := semantics.TableSet(2) - - t1 := &fakePlan{solve: a} - t2 := &fakePlan{solve: b} - t3 := &fakePlan{solve: a.Merge(b)} - - dpTable.add(t1) - dpTable.add(t2) - dpTable.add(t3) - - size1 := dpTable.bitSetsOfSize(1) - assert.Equal(t, []joinTree{t1, t2}, size1, "size 1") - - size2 := dpTable.bitSetsOfSize(2) - assert.Equal(t, []joinTree{t3}, size2, "size 2") - assert.Equal(t, t1, dpTable.planFor(a)) - assert.Equal(t, t2, dpTable.planFor(b)) - assert.Equal(t, t3, dpTable.planFor(a.Merge(b))) -} diff --git a/go/vt/vtgate/planbuilder/plan_test.go b/go/vt/vtgate/planbuilder/plan_test.go index 6e927508c96..d3d0affb694 100644 --- a/go/vt/vtgate/planbuilder/plan_test.go +++ b/go/vt/vtgate/planbuilder/plan_test.go @@ -185,8 +185,8 @@ func TestPlan(t *testing.T) { testFile(t, "transaction_cases.txt", testOutputTempDir, vschemaWrapper, true) testFile(t, "lock_cases.txt", testOutputTempDir, vschemaWrapper, true) testFile(t, "large_cases.txt", testOutputTempDir, vschemaWrapper, true) - testFile(t, "ddl_cases_no_default_keyspace.txt", testOutputTempDir, vschemaWrapper, true) - testFile(t, "show_cases_no_default_keyspace.txt", testOutputTempDir, vschemaWrapper, true) + testFile(t, "ddl_cases_no_default_keyspace.txt", testOutputTempDir, vschemaWrapper, false) + testFile(t, "show_cases_no_default_keyspace.txt", testOutputTempDir, vschemaWrapper, false) } func TestSysVarSetDisabled(t *testing.T) { @@ -420,7 +420,7 @@ func testFile(t *testing.T, filename, tempDir string, vschema *vschemaWrapper, c tcase.output2ndPlanner = tcase.output } - vschema.version = V4GreedyOptimized + vschema.version = V4 out, err := getPlanOutput(tcase, vschema) // our expectation for the new planner on this query is one of three @@ -611,15 +611,9 @@ func BenchmarkPlanner(b *testing.B) { b.Run(filename+"-v4", func(b *testing.B) { benchmarkPlanner(b, V4, testCases, vschema) }) - b.Run(filename+"-v4greedy", func(b *testing.B) { - benchmarkPlanner(b, V4GreedyOnly, testCases, vschema) - }) b.Run(filename+"-v4left2right", func(b *testing.B) { benchmarkPlanner(b, V4Left2Right, testCases, vschema) }) - b.Run(filename+"-v4greedyOptimized", func(b *testing.B) { - benchmarkPlanner(b, V4GreedyOptimized, testCases, vschema) - }) } } diff --git a/go/vt/vtgate/planbuilder/route_planning.go b/go/vt/vtgate/planbuilder/route_planning.go index 5f647c7d6b1..bb9df3655b5 100644 --- a/go/vt/vtgate/planbuilder/route_planning.go +++ b/go/vt/vtgate/planbuilder/route_planning.go @@ -17,7 +17,6 @@ limitations under the License. package planbuilder import ( - "container/heap" "sort" "strings" @@ -47,14 +46,10 @@ func newBuildSelectPlan(sel *sqlparser.Select, vschema ContextVSchema) (engine.P var tree joinTree switch { - case vschema.Planner() == V4GreedyOnly || len(qgraph.tables) > 10: - tree, err = greedySolve(qgraph, semTable, vschema) - case vschema.Planner() == V4GreedyOptimized: - tree, err = greedySolveOptimized(qgraph, semTable, vschema) case vschema.Planner() == V4Left2Right: tree, err = leftToRightSolve(qgraph, semTable, vschema) default: - tree, err = dpSolve(qgraph, semTable, vschema) + tree, err = greedySolve(qgraph, semTable, vschema) } if err != nil { @@ -250,58 +245,6 @@ func (jp *joinPlan) cost() int { return jp.lhs.cost() + jp.rhs.cost() } -/* - we use dynamic programming to find the cheapest route/join tree possible, - where the cost of a plan is the number of joins -*/ -func dpSolve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVSchema) (joinTree, error) { - size := len(qg.tables) - dpTable := makeDPTable() - - var allTables semantics.TableSet - - // we start by seeding the table with the single routes - for _, table := range qg.tables { - solves := semTable.TableSetFor(table.alias) - allTables |= solves - plan, err := createRoutePlan(table, solves, vschema) - if err != nil { - return nil, err - } - dpTable.add(plan) - } - - /* - Next we'll solve bigger and bigger joins, using smaller plans to build larger ones, - until we have a join tree covering all tables in the FROM clause - */ - for currentSize := 2; currentSize <= size; currentSize++ { - lefts := dpTable.bitSetsOfSize(currentSize - 1) - rights := dpTable.bitSetsOfSize(1) - for _, lhs := range lefts { - for _, rhs := range rights { - if lhs.solves().IsOverlapping(rhs.solves()) { - // at least one of the tables is solved on both sides - continue - } - solves := lhs.solves() | rhs.solves() - oldPlan := dpTable.planFor(solves) - if oldPlan != nil && oldPlan.cost() == 1 { - // we already have the perfect plan. keep it - continue - } - joinPredicates := qg.getPredicates(lhs.solves(), rhs.solves()) - newPlan := createJoin(lhs, rhs, joinPredicates, semTable) - if oldPlan == nil || newPlan.cost() < oldPlan.cost() { - dpTable.add(newPlan) - } - } - } - } - - return dpTable.planFor(allTables), nil -} - func createJoin(lhs, rhs joinTree, joinPredicates []sqlparser.Expr, semTable *semantics.SemTable) joinTree { newPlan := tryMerge(lhs, rhs, joinPredicates, semTable) if newPlan == nil { @@ -314,128 +257,6 @@ func createJoin(lhs, rhs joinTree, joinPredicates []sqlparser.Expr, semTable *se return newPlan } -type priorityQueueItem struct { - plan joinTree - cost int - lhsSolve semantics.TableSet - rhsSolve semantics.TableSet -} - -func (pqi *priorityQueueItem) solves() semantics.TableSet { - return pqi.lhsSolve | pqi.rhsSolve -} - -type priorityQueuePlans []*priorityQueueItem - -// Len implements the Heap interface -func (pq priorityQueuePlans) Len() int { return len(pq) } - -// Less implements the Heap interface -func (pq priorityQueuePlans) Less(i, j int) bool { - // We want Pop to give us the lowest cost so we use lesser than here. - if pq[i].cost == pq[j].cost { - return pq[i].solves() < pq[j].solves() - } - return pq[i].cost < pq[j].cost -} - -// Swap implements the Heap interface -func (pq priorityQueuePlans) Swap(i, j int) { - pq[i], pq[j] = pq[j], pq[i] -} - -// Push implements the Heap interface -func (pq *priorityQueuePlans) Push(x interface{}) { - item := x.(*priorityQueueItem) - *pq = append(*pq, item) -} - -// Pop implements the Heap interface -func (pq *priorityQueuePlans) Pop() interface{} { - old := *pq - n := len(old) - item := old[n-1] - old[n-1] = nil // avoid memory leak - *pq = old[0 : n-1] - return item -} - -func greedySolveOptimized(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVSchema) (joinTree, error) { - routePlans := make([]*routePlan, len(qg.tables)) - intermediatePlans := map[semantics.TableSet]*priorityQueueItem{} - - for i, table := range qg.tables { - solves := semTable.TableSetFor(table.alias) - plan, err := createRoutePlan(table, solves, vschema) - if err != nil { - return nil, err - } - routePlans[i] = plan - intermediatePlans[solves] = &priorityQueueItem{ - plan: plan, - cost: plan.cost(), - lhsSolve: 0, - } - } - - if len(qg.tables) == 1 { - return routePlans[0], nil - } - - pq := priorityQueuePlans{} - - for i, lhs := range routePlans { - for j := i + 1; j < len(routePlans); j++ { - rhs := routePlans[j] - joinPredicates := qg.getPredicates(lhs.solves(), rhs.solves()) - plan := createJoin(lhs, rhs, joinPredicates, semTable) - pq.Push(&priorityQueueItem{ - plan: plan, - cost: plan.cost(), - lhsSolve: lhs.solves(), - rhsSolve: rhs.solves(), - }) - } - } - - heap.Init(&pq) - - for pq.Len() > 0 { - item := heap.Pop(&pq).(*priorityQueueItem) - _, isLeftAvail := intermediatePlans[item.lhsSolve] - _, isRightAvail := intermediatePlans[item.rhsSolve] - if !isLeftAvail || !isRightAvail { - continue - } - delete(intermediatePlans, item.lhsSolve) - delete(intermediatePlans, item.rhsSolve) - solves := item.lhsSolve | item.rhsSolve - plan := item.plan - - for tableSet, intermPlan := range intermediatePlans { - newPlan := createJoin(intermPlan.plan, plan, qg.getPredicates(solves, tableSet), semTable) - heap.Push(&pq, &priorityQueueItem{ - plan: newPlan, - cost: newPlan.cost(), - lhsSolve: tableSet, - rhsSolve: solves, - }) - } - intermediatePlans[solves] = item - } - - // intermediatePlans should only have 1 value now - if len(intermediatePlans) != 1 { - return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "their should be only 1 intermediate planner now") - } - - for _, item := range intermediatePlans { - return item.plan, nil - } - - return nil, nil -} - type ( tableSetPair struct { left, right semantics.TableSet diff --git a/go/vt/vtgate/planbuilder/testdata/from_cases.txt b/go/vt/vtgate/planbuilder/testdata/from_cases.txt index 00b6c72008e..9305de7fe24 100644 --- a/go/vt/vtgate/planbuilder/testdata/from_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/from_cases.txt @@ -1165,7 +1165,7 @@ } # sharded join, non-vindex col -"select user.col from user join user_extra on user.id = user_extra.col where user.id = 35" +"select user.col from user join user_extra on user.id = user_extra.col" { "QueryType": "SELECT", "Original": "select user.col from user join user_extra on user.id = user_extra.col", diff --git a/go/vt/vtgate/planbuilder/testdata/large_cases.txt b/go/vt/vtgate/planbuilder/testdata/large_cases.txt index 064a60a4792..3ec24ac86c6 100644 --- a/go/vt/vtgate/planbuilder/testdata/large_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/large_cases.txt @@ -178,24 +178,24 @@ "OperatorType": "Join", "Variant": "Join", "JoinColumnIndexes": "1", - "TableName": "music, music_extra_user, user_extra, user_metadata_unsharded, unsharded_a, unsharded_auto, unsharded_b", + "TableName": "unsharded, unsharded_a, unsharded_auto, unsharded_b_user, user_extra, user_metadata_music, music_extra", "Inputs": [ { "OperatorType": "Route", - "Variant": "SelectScatter", + "Variant": "SelectUnsharded", "Keyspace": { - "Name": "user", - "Sharded": true + "Name": "main", + "Sharded": false }, - "FieldQuery": "select 1 from music, music_extra where 1 != 1", - "Query": "select 1 from music, music_extra where music.id = music_extra.music_id", - "Table": "music, music_extra" + "FieldQuery": "select 1 from unsharded, unsharded_a, unsharded_b, unsharded_auto where 1 != 1", + "Query": "select 1 from unsharded, unsharded_a, unsharded_b, unsharded_auto where unsharded.x = unsharded_a.y", + "Table": "unsharded, unsharded_a, unsharded_auto, unsharded_b" }, { "OperatorType": "Join", "Variant": "Join", "JoinColumnIndexes": "-1", - "TableName": "user, user_extra, user_metadata_unsharded, unsharded_a, unsharded_auto, unsharded_b", + "TableName": "user, user_extra, user_metadata_music, music_extra", "Inputs": [ { "OperatorType": "Route", @@ -210,14 +210,14 @@ }, { "OperatorType": "Route", - "Variant": "SelectUnsharded", + "Variant": "SelectScatter", "Keyspace": { - "Name": "main", - "Sharded": false + "Name": "user", + "Sharded": true }, - "FieldQuery": "select 1 from unsharded, unsharded_a, unsharded_b, unsharded_auto where 1 != 1", - "Query": "select 1 from unsharded, unsharded_a, unsharded_b, unsharded_auto where unsharded.x = unsharded_a.y", - "Table": "unsharded, unsharded_a, unsharded_auto, unsharded_b" + "FieldQuery": "select 1 from music, music_extra where 1 != 1", + "Query": "select 1 from music, music_extra where music.id = music_extra.music_id", + "Table": "music, music_extra" } ] } diff --git a/proto/query.proto b/proto/query.proto index b371c8f1e12..69e889a0610 100644 --- a/proto/query.proto +++ b/proto/query.proto @@ -304,7 +304,6 @@ message ExecuteOptions { V4 = 2; V4Greedy = 3; V4Left2Right = 4; - V4GreedyOptimized = 5; } // PlannerVersion specifies which planner to use. From d285bfecf96e48ae63abe78eb803cf7c058402db Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Mon, 11 Jan 2021 15:20:59 +0100 Subject: [PATCH 39/40] removed planner-version sysvar Signed-off-by: Andres Taylor --- go/vt/sqlparser/ast_rewriting.go | 1 - go/vt/sqlparser/ast_rewriting_test.go | 7 +------ go/vt/sysvars/sysvars.go | 1 - go/vt/vtgate/executor.go | 6 ------ 4 files changed, 1 insertion(+), 14 deletions(-) diff --git a/go/vt/sqlparser/ast_rewriting.go b/go/vt/sqlparser/ast_rewriting.go index 36a42f91b1c..7791f1cfd8f 100644 --- a/go/vt/sqlparser/ast_rewriting.go +++ b/go/vt/sqlparser/ast_rewriting.go @@ -233,7 +233,6 @@ func (er *expressionRewriter) sysVarRewrite(cursor *Cursor, node *ColName) { sysvars.SQLSelectLimit.Name, sysvars.TransactionMode.Name, sysvars.Workload.Name, - sysvars.PlannerVersion.Name, sysvars.DDLStrategy.Name, sysvars.SessionUUID.Name, sysvars.ReadAfterWriteGTID.Name, diff --git a/go/vt/sqlparser/ast_rewriting_test.go b/go/vt/sqlparser/ast_rewriting_test.go index 13ae1d33ed2..802f632f627 100644 --- a/go/vt/sqlparser/ast_rewriting_test.go +++ b/go/vt/sqlparser/ast_rewriting_test.go @@ -32,7 +32,7 @@ type myTestCase struct { ddlStrategy, sessionUUID bool udv int autocommit, clientFoundRows, skipQueryPlanCache bool - sqlSelectLimit, transactionMode, workload, plannerVersion bool + sqlSelectLimit, transactionMode, workload bool } func TestRewrites(in *testing.T) { @@ -165,10 +165,6 @@ func TestRewrites(in *testing.T) { // SELECT * behaves different depending the join type used, so if that has been used, we won't rewrite in: "SELECT * FROM A JOIN B USING (id1,id2,id3)", expected: "SELECT * FROM A JOIN B USING (id1,id2,id3)", - }, { - in: "SELECT @@planner_version", - expected: "SELECT :__vtplanner_version as `@@planner_version`", - plannerVersion: true, }} for _, tc := range tests { @@ -202,7 +198,6 @@ func TestRewrites(in *testing.T) { assert.Equal(tc.rawGTID, result.NeedsSysVar(sysvars.ReadAfterWriteGTID.Name), "should need rawGTID") assert.Equal(tc.rawTimeout, result.NeedsSysVar(sysvars.ReadAfterWriteTimeOut.Name), "should need rawTimeout") assert.Equal(tc.sessTrackGTID, result.NeedsSysVar(sysvars.SessionTrackGTIDs.Name), "should need sessTrackGTID") - assert.Equal(tc.plannerVersion, result.NeedsSysVar(sysvars.PlannerVersion.Name), "should need :__vtplanner_version") }) } } diff --git a/go/vt/sysvars/sysvars.go b/go/vt/sysvars/sysvars.go index d8d8f5144f2..b0e60b61d6d 100644 --- a/go/vt/sysvars/sysvars.go +++ b/go/vt/sysvars/sysvars.go @@ -51,7 +51,6 @@ var ( SQLSelectLimit = SystemVariable{Name: "sql_select_limit", Default: off} TransactionMode = SystemVariable{Name: "transaction_mode", IdentifierAsString: true} Workload = SystemVariable{Name: "workload", IdentifierAsString: true} - PlannerVersion = SystemVariable{Name: "planner_version", IdentifierAsString: true} Charset = SystemVariable{Name: "charset", Default: utf8, IdentifierAsString: true} Names = SystemVariable{Name: "names", Default: utf8, IdentifierAsString: true} SessionUUID = SystemVariable{Name: "session_uuid", IdentifierAsString: true} diff --git a/go/vt/vtgate/executor.go b/go/vt/vtgate/executor.go index b109702cd66..265a13821b7 100644 --- a/go/vt/vtgate/executor.go +++ b/go/vt/vtgate/executor.go @@ -315,12 +315,6 @@ func (e *Executor) addNeededBindVars(bindVarNeeds *sqlparser.BindVarNeeds, bindV } }) bindVars[key] = sqltypes.StringBindVariable(v) - case sysvars.PlannerVersion.Name: - v := *plannerVersion - ifOptionsExist(session, func(options *querypb.ExecuteOptions) { - v = options.GetPlannerVersion().String() - }) - bindVars[key] = sqltypes.StringBindVariable(v) } } From 26b008b02f9b7d3a8de80b242cb886156d8aacea Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Mon, 11 Jan 2021 15:57:43 +0100 Subject: [PATCH 40/40] adress peer review comments Signed-off-by: Andres Taylor --- go/vt/vtgate/planbuilder/concatenate.go | 10 +- go/vt/vtgate/planbuilder/join.go | 10 +- go/vt/vtgate/planbuilder/join2.go | 39 +++--- go/vt/vtgate/planbuilder/logical_plan.go | 20 +-- go/vt/vtgate/planbuilder/memory_sort.go | 4 +- go/vt/vtgate/planbuilder/merge_sort.go | 4 +- go/vt/vtgate/planbuilder/ordered_aggregate.go | 4 +- go/vt/vtgate/planbuilder/pullout_subquery.go | 10 +- go/vt/vtgate/planbuilder/querygraph.go | 6 +- go/vt/vtgate/planbuilder/querygraph_test.go | 2 +- go/vt/vtgate/planbuilder/route.go | 10 +- go/vt/vtgate/planbuilder/route_planning.go | 116 +++++++++--------- go/vt/vtgate/planbuilder/select.go | 16 +-- .../vtgate/planbuilder/sql_calc_found_rows.go | 10 +- go/vt/vtgate/planbuilder/vindex_func.go | 6 +- go/vt/vtgate/semantics/analyzer.go | 15 +-- go/vt/vtgate/semantics/analyzer_test.go | 6 +- go/vt/vtgate/semantics/semantic_state.go | 23 +--- 18 files changed, 147 insertions(+), 164 deletions(-) diff --git a/go/vt/vtgate/planbuilder/concatenate.go b/go/vt/vtgate/planbuilder/concatenate.go index 1e0db368fa9..b482e795845 100644 --- a/go/vt/vtgate/planbuilder/concatenate.go +++ b/go/vt/vtgate/planbuilder/concatenate.go @@ -54,12 +54,12 @@ func (c *concatenate) Wireup(plan logicalPlan, jt *jointab) error { return c.rhs.Wireup(plan, jt) } -func (c *concatenate) Wireup2(semTable *semantics.SemTable) error { - err := c.lhs.Wireup2(semTable) +func (c *concatenate) WireupV4(semTable *semantics.SemTable) error { + err := c.lhs.WireupV4(semTable) if err != nil { return err } - return c.rhs.Wireup2(semTable) + return c.rhs.WireupV4(semTable) } func (c *concatenate) SupplyVar(from, to int, col *sqlparser.ColName, varname string) { @@ -93,8 +93,8 @@ func (c *concatenate) Rewrite(inputs ...logicalPlan) error { return nil } -func (c *concatenate) Solves() semantics.TableSet { - return c.lhs.Solves().Merge(c.rhs.Solves()) +func (c *concatenate) ContainsTables() semantics.TableSet { + return c.lhs.ContainsTables().Merge(c.rhs.ContainsTables()) } // Inputs implements the logicalPlan interface diff --git a/go/vt/vtgate/planbuilder/join.go b/go/vt/vtgate/planbuilder/join.go index 55fa449bca7..d8e7429a900 100644 --- a/go/vt/vtgate/planbuilder/join.go +++ b/go/vt/vtgate/planbuilder/join.go @@ -154,12 +154,12 @@ func (jb *join) Wireup(plan logicalPlan, jt *jointab) error { } // Wireup2 implements the logicalPlan interface -func (jb *join) Wireup2(semTable *semantics.SemTable) error { - err := jb.Right.Wireup2(semTable) +func (jb *join) WireupV4(semTable *semantics.SemTable) error { + err := jb.Right.WireupV4(semTable) if err != nil { return err } - return jb.Left.Wireup2(semTable) + return jb.Left.WireupV4(semTable) } // SupplyVar implements the logicalPlan interface @@ -247,8 +247,8 @@ func (jb *join) Rewrite(inputs ...logicalPlan) error { } // Solves implements the logicalPlan interface -func (jb *join) Solves() semantics.TableSet { - return jb.Left.Solves().Merge(jb.Right.Solves()) +func (jb *join) ContainsTables() semantics.TableSet { + return jb.Left.ContainsTables().Merge(jb.Right.ContainsTables()) } // Inputs implements the logicalPlan interface diff --git a/go/vt/vtgate/planbuilder/join2.go b/go/vt/vtgate/planbuilder/join2.go index c6a6d6ce788..7b273c88865 100644 --- a/go/vt/vtgate/planbuilder/join2.go +++ b/go/vt/vtgate/planbuilder/join2.go @@ -22,12 +22,11 @@ import ( "vitess.io/vitess/go/vt/vtgate/semantics" ) -var _ logicalPlan = (*join2)(nil) +var _ logicalPlan = (*joinV4)(nil) -// join is used to build a Join primitive. -// It's used to build a normal join or a left join -// operation. -type join2 struct { +// joinV4 is used to build a Join primitive. +// It's used to build an inner join and only used by the V4 planner +type joinV4 struct { // Left and Right are the nodes for the join. Left, Right logicalPlan Cols []int @@ -35,51 +34,51 @@ type join2 struct { } // Order implements the logicalPlan interface -func (j *join2) Order() int { +func (j *joinV4) Order() int { panic("implement me") } // ResultColumns implements the logicalPlan interface -func (j *join2) ResultColumns() []*resultColumn { +func (j *joinV4) ResultColumns() []*resultColumn { panic("implement me") } // Reorder implements the logicalPlan interface -func (j *join2) Reorder(i int) { +func (j *joinV4) Reorder(i int) { panic("implement me") } // Wireup implements the logicalPlan interface -func (j *join2) Wireup(lp logicalPlan, jt *jointab) error { +func (j *joinV4) Wireup(lp logicalPlan, jt *jointab) error { panic("implement me") } // Wireup2 implements the logicalPlan interface -func (j *join2) Wireup2(semTable *semantics.SemTable) error { - err := j.Left.Wireup2(semTable) +func (j *joinV4) WireupV4(semTable *semantics.SemTable) error { + err := j.Left.WireupV4(semTable) if err != nil { return err } - return j.Right.Wireup2(semTable) + return j.Right.WireupV4(semTable) } // SupplyVar implements the logicalPlan interface -func (j *join2) SupplyVar(from, to int, col *sqlparser.ColName, varname string) { +func (j *joinV4) SupplyVar(from, to int, col *sqlparser.ColName, varname string) { panic("implement me") } // SupplyCol implements the logicalPlan interface -func (j *join2) SupplyCol(col *sqlparser.ColName) (rc *resultColumn, colNumber int) { +func (j *joinV4) SupplyCol(col *sqlparser.ColName) (rc *resultColumn, colNumber int) { panic("implement me") } // SupplyWeightString implements the logicalPlan interface -func (j *join2) SupplyWeightString(colNumber int) (weightcolNumber int, err error) { +func (j *joinV4) SupplyWeightString(colNumber int) (weightcolNumber int, err error) { panic("implement me") } // Primitive implements the logicalPlan interface -func (j *join2) Primitive() engine.Primitive { +func (j *joinV4) Primitive() engine.Primitive { return &engine.Join{ Left: j.Left.Primitive(), Right: j.Right.Primitive(), @@ -89,16 +88,16 @@ func (j *join2) Primitive() engine.Primitive { } // Inputs implements the logicalPlan interface -func (j *join2) Inputs() []logicalPlan { +func (j *joinV4) Inputs() []logicalPlan { panic("implement me") } // Rewrite implements the logicalPlan interface -func (j *join2) Rewrite(inputs ...logicalPlan) error { +func (j *joinV4) Rewrite(inputs ...logicalPlan) error { panic("implement me") } // Solves implements the logicalPlan interface -func (j *join2) Solves() semantics.TableSet { - return j.Left.Solves().Merge(j.Right.Solves()) +func (j *joinV4) ContainsTables() semantics.TableSet { + return j.Left.ContainsTables().Merge(j.Right.ContainsTables()) } diff --git a/go/vt/vtgate/planbuilder/logical_plan.go b/go/vt/vtgate/planbuilder/logical_plan.go index bb0ba8004e6..bf17fca74c9 100644 --- a/go/vt/vtgate/planbuilder/logical_plan.go +++ b/go/vt/vtgate/planbuilder/logical_plan.go @@ -49,8 +49,8 @@ type logicalPlan interface { // the lhs nodes. Wireup(lp logicalPlan, jt *jointab) error - // Wireup2 does the wire up work for the new planner - Wireup2(semTable *semantics.SemTable) error + // WireupV4 does the wire up work for the V4 planner + WireupV4(semTable *semantics.SemTable) error // SupplyVar finds the common root between from and to. If it's // the common root, it supplies the requested var to the rhs tree. @@ -76,9 +76,15 @@ type logicalPlan interface { // This function should only be called after Wireup is finished. Primitive() engine.Primitive + // Inputs are the children of this plan Inputs() []logicalPlan + + // Rewrite replaces the inputs of this plan with the ones provided Rewrite(inputs ...logicalPlan) error - Solves() semantics.TableSet + + // ContainsTables keeps track which query tables are being solved by this logical plan + // This is only applicable for plans that have been built with the V4 planner + ContainsTables() semantics.TableSet } //------------------------------------------------------------------------- @@ -152,8 +158,8 @@ func (bc *logicalPlanCommon) Wireup(plan logicalPlan, jt *jointab) error { return bc.input.Wireup(plan, jt) } -func (bc *logicalPlanCommon) Wireup2(semTable *semantics.SemTable) error { - return bc.input.Wireup2(semTable) +func (bc *logicalPlanCommon) WireupV4(semTable *semantics.SemTable) error { + return bc.input.WireupV4(semTable) } func (bc *logicalPlanCommon) SupplyVar(from, to int, col *sqlparser.ColName, varname string) { @@ -183,8 +189,8 @@ func (bc *logicalPlanCommon) Inputs() []logicalPlan { } // Solves implements the logicalPlan interface -func (bc *logicalPlanCommon) Solves() semantics.TableSet { - return bc.input.Solves() +func (bc *logicalPlanCommon) ContainsTables() semantics.TableSet { + return bc.input.ContainsTables() } //------------------------------------------------------------------------- diff --git a/go/vt/vtgate/planbuilder/memory_sort.go b/go/vt/vtgate/planbuilder/memory_sort.go index ae972fe6047..053c5ad5e87 100644 --- a/go/vt/vtgate/planbuilder/memory_sort.go +++ b/go/vt/vtgate/planbuilder/memory_sort.go @@ -128,6 +128,6 @@ func (ms *memorySort) Wireup(plan logicalPlan, jt *jointab) error { return ms.input.Wireup(plan, jt) } -func (ms *memorySort) Wireup2(semTable *semantics.SemTable) error { - return ms.input.Wireup2(semTable) +func (ms *memorySort) WireupV4(semTable *semantics.SemTable) error { + return ms.input.WireupV4(semTable) } diff --git a/go/vt/vtgate/planbuilder/merge_sort.go b/go/vt/vtgate/planbuilder/merge_sort.go index 91df0296cc9..ee28f7fec71 100644 --- a/go/vt/vtgate/planbuilder/merge_sort.go +++ b/go/vt/vtgate/planbuilder/merge_sort.go @@ -84,6 +84,6 @@ func (ms *mergeSort) Wireup(plan logicalPlan, jt *jointab) error { return ms.input.Wireup(plan, jt) } -func (ms *mergeSort) Wireup2(semTable *semantics.SemTable) error { - return ms.input.Wireup2(semTable) +func (ms *mergeSort) WireupV4(semTable *semantics.SemTable) error { + return ms.input.WireupV4(semTable) } diff --git a/go/vt/vtgate/planbuilder/ordered_aggregate.go b/go/vt/vtgate/planbuilder/ordered_aggregate.go index 566eeb9723a..7e2065b7f64 100644 --- a/go/vt/vtgate/planbuilder/ordered_aggregate.go +++ b/go/vt/vtgate/planbuilder/ordered_aggregate.go @@ -350,6 +350,6 @@ func (oa *orderedAggregate) Wireup(plan logicalPlan, jt *jointab) error { return oa.input.Wireup(plan, jt) } -func (oa *orderedAggregate) Wireup2(semTable *semantics.SemTable) error { - return oa.input.Wireup2(semTable) +func (oa *orderedAggregate) WireupV4(semTable *semantics.SemTable) error { + return oa.input.WireupV4(semTable) } diff --git a/go/vt/vtgate/planbuilder/pullout_subquery.go b/go/vt/vtgate/planbuilder/pullout_subquery.go index 1cf6267c777..ec0862bcf0d 100644 --- a/go/vt/vtgate/planbuilder/pullout_subquery.go +++ b/go/vt/vtgate/planbuilder/pullout_subquery.go @@ -88,11 +88,11 @@ func (ps *pulloutSubquery) Wireup(plan logicalPlan, jt *jointab) error { } // Wireup2 implements the logicalPlan interface -func (ps *pulloutSubquery) Wireup2(semTable *semantics.SemTable) error { - if err := ps.underlying.Wireup2(semTable); err != nil { +func (ps *pulloutSubquery) WireupV4(semTable *semantics.SemTable) error { + if err := ps.underlying.WireupV4(semTable); err != nil { return err } - return ps.subquery.Wireup2(semTable) + return ps.subquery.WireupV4(semTable) } // SupplyVar implements the logicalPlan interface @@ -125,8 +125,8 @@ func (ps *pulloutSubquery) Rewrite(inputs ...logicalPlan) error { } // Solves implements the logicalPlan interface -func (ps *pulloutSubquery) Solves() semantics.TableSet { - return ps.underlying.Solves().Merge(ps.subquery.Solves()) +func (ps *pulloutSubquery) ContainsTables() semantics.TableSet { + return ps.underlying.ContainsTables().Merge(ps.subquery.ContainsTables()) } // Inputs implements the logicalPlan interface diff --git a/go/vt/vtgate/planbuilder/querygraph.go b/go/vt/vtgate/planbuilder/querygraph.go index c1cbb795d4b..28641f495b0 100644 --- a/go/vt/vtgate/planbuilder/querygraph.go +++ b/go/vt/vtgate/planbuilder/querygraph.go @@ -107,8 +107,10 @@ func (qg *queryGraph) collectTable(t sqlparser.TableExpr, semTable *semantics.Se } } case *sqlparser.ParenTableExpr: - if err := qg.collectTables(table.Exprs, semTable); err != nil { - return err + for _, expr := range table.Exprs { + if err := qg.collectTable(expr, semTable); err != nil { + return err + } } } return nil diff --git a/go/vt/vtgate/planbuilder/querygraph_test.go b/go/vt/vtgate/planbuilder/querygraph_test.go index daade48a5ec..df16ab022e5 100644 --- a/go/vt/vtgate/planbuilder/querygraph_test.go +++ b/go/vt/vtgate/planbuilder/querygraph_test.go @@ -175,7 +175,7 @@ func TestQueryGraph(t *testing.T) { t.Run(sql, func(t *testing.T) { tree, err := sqlparser.Parse(sql) require.NoError(t, err) - semTable, err := semantics.Analyse(tree, &schemaInf{}) + semTable, err := semantics.Analyse(tree) require.NoError(t, err) qgraph, err := createQGFromSelect(tree.(*sqlparser.Select), semTable) require.NoError(t, err) diff --git a/go/vt/vtgate/planbuilder/route.go b/go/vt/vtgate/planbuilder/route.go index 898341cf9eb..2a357845ec5 100644 --- a/go/vt/vtgate/planbuilder/route.go +++ b/go/vt/vtgate/planbuilder/route.go @@ -65,8 +65,8 @@ type route struct { // eroute is the primitive being built. eroute *engine.Route - // solvedTables keeps track of which tables this route is covering - solvedTables semantics.TableSet + // tables keeps track of which tables this route is covering + tables semantics.TableSet } type tableSubstitution struct { @@ -133,7 +133,7 @@ func (rb *route) SetLimit(limit *sqlparser.Limit) { } // Wireup2 implements the logicalPlan interface -func (rb *route) Wireup2(semTable *semantics.SemTable) error { +func (rb *route) WireupV4(semTable *semantics.SemTable) error { rb.prepareTheAST() rb.eroute.Query = sqlparser.String(rb.Select) @@ -145,8 +145,8 @@ func (rb *route) Wireup2(semTable *semantics.SemTable) error { } // Solves implements the logicalPlan interface -func (rb *route) Solves() semantics.TableSet { - return rb.solvedTables +func (rb *route) ContainsTables() semantics.TableSet { + return rb.tables } // Wireup implements the logicalPlan interface diff --git a/go/vt/vtgate/planbuilder/route_planning.go b/go/vt/vtgate/planbuilder/route_planning.go index bb9df3655b5..92277985a2e 100644 --- a/go/vt/vtgate/planbuilder/route_planning.go +++ b/go/vt/vtgate/planbuilder/route_planning.go @@ -33,7 +33,7 @@ import ( ) func newBuildSelectPlan(sel *sqlparser.Select, vschema ContextVSchema) (engine.Primitive, error) { - semTable, err := semantics.Analyse(sel, nil) // TODO no nil no + semTable, err := semantics.Analyse(sel) // TODO no nil no if err != nil { return nil, err } @@ -65,7 +65,7 @@ func newBuildSelectPlan(sel *sqlparser.Select, vschema ContextVSchema) (engine.P return nil, err } - if err := plan.Wireup2(semTable); err != nil { + if err := plan.WireupV4(semTable); err != nil { return nil, err } return plan.Primitive(), nil @@ -103,8 +103,8 @@ func planProjections(sel *sqlparser.Select, plan logicalPlan, semTable *semantic type ( joinTree interface { - // solves returns the table identifiers that are solved by this plan - solves() semantics.TableSet + // tables returns the table identifiers that are solved by this plan + tables() semantics.TableSet // cost is simply the number of routes in the joinTree cost() int @@ -118,9 +118,9 @@ type ( solved semantics.TableSet keyspace *vindexes.Keyspace - // tables contains all the tables that are solved by this plan. + // _tables contains all the tables that are solved by this plan. // the tables also contain any predicates that only depend on that particular table - tables routeTables + _tables routeTables // extraPredicates are the predicates that depend on multiple tables extraPredicates []sqlparser.Expr @@ -136,8 +136,8 @@ type ( routeTables []*routeTable ) -// solves implements the joinTree interface -func (rp *routePlan) solves() semantics.TableSet { +// tables implements the joinTree interface +func (rp *routePlan) tables() semantics.TableSet { return rp.solved } @@ -154,14 +154,14 @@ type vindexPlusPredicates struct { } func (rp *routePlan) addPredicate(predicates ...sqlparser.Expr) error { - if len(rp.tables) != 1 { + if len(rp._tables) != 1 { return vterrors.Errorf(vtrpcpb.Code_INTERNAL, "addPredicate should only be called when the route has a single table") } var vindexPreds []*vindexPlusPredicates // Add all the column vindexes to the list of vindexPlusPredicates - for _, columnVindex := range rp.tables[0].vtable.ColumnVindexes { + for _, columnVindex := range rp._tables[0].vtable.ColumnVindexes { vindexPreds = append(vindexPreds, &vindexPlusPredicates{vindex: columnVindex}) } @@ -227,7 +227,7 @@ func (rp *routePlan) Predicates() sqlparser.Expr { Right: e, } } - for _, t := range rp.tables { + for _, t := range rp._tables { for _, predicate := range t.qtable.predicates { add(predicate) } @@ -238,8 +238,8 @@ func (rp *routePlan) Predicates() sqlparser.Expr { return result } -func (jp *joinPlan) solves() semantics.TableSet { - return jp.lhs.solves() | jp.rhs.solves() +func (jp *joinPlan) tables() semantics.TableSet { + return jp.lhs.tables() | jp.rhs.tables() } func (jp *joinPlan) cost() int { return jp.lhs.cost() + jp.rhs.cost() @@ -271,40 +271,33 @@ type ( As an optimization, it first only considers joining tables that have predicates defined between them */ func greedySolve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVSchema) (joinTree, error) { - plans := make([]joinTree, len(qg.tables)) + joinTrees, err := seedPlanList(qg, semTable, vschema) planCache := cacheMap{} - - // we start by seeding the table with the single routes - for i, table := range qg.tables { - solves := semTable.TableSetFor(table.alias) - plan, err := createRoutePlan(table, solves, vschema) - if err != nil { - return nil, err - } - plans[i] = plan + if err != nil { + return nil, err } crossJoinsOK := false - for len(plans) > 1 { - bestPlan, lIdx, rIdx := findBestJoin(qg, semTable, plans, planCache, crossJoinsOK) - if bestPlan != nil { - // if we found a best plan, we'll replace the two plans that were joined with the join plan created - plans = removeAt(plans, rIdx) - plans = removeAt(plans, lIdx) - plans = append(plans, bestPlan) + for len(joinTrees) > 1 { + bestTree, lIdx, rIdx := findBestJoinTree(qg, semTable, joinTrees, planCache, crossJoinsOK) + if bestTree != nil { + // if we found a best plan, we'll replace the two joinTrees that were joined with the join plan created + joinTrees = removeAt(joinTrees, rIdx) + joinTrees = removeAt(joinTrees, lIdx) + joinTrees = append(joinTrees, bestTree) } else { // we will only fail to find a join plan when there are only cross joins left // when that happens, we switch over to allow cross joins as well. - // this way we prioritize joining plans with predicates first + // this way we prioritize joining joinTrees with predicates first crossJoinsOK = true } } - return plans[0], nil + return joinTrees[0], nil } -func (cm cacheMap) getJoinFor(lhs, rhs joinTree, joinPredicates []sqlparser.Expr, semTable *semantics.SemTable) joinTree { - solves := tableSetPair{left: lhs.solves(), right: rhs.solves()} +func (cm cacheMap) getJoinTreeFor(lhs, rhs joinTree, joinPredicates []sqlparser.Expr, semTable *semantics.SemTable) joinTree { + solves := tableSetPair{left: lhs.tables(), right: rhs.tables()} plan := cm[solves] if plan == nil { plan = createJoin(lhs, rhs, joinPredicates, semTable) @@ -313,7 +306,7 @@ func (cm cacheMap) getJoinFor(lhs, rhs joinTree, joinPredicates []sqlparser.Expr return plan } -func findBestJoin( +func findBestJoinTree( qg *queryGraph, semTable *semantics.SemTable, plans []joinTree, @@ -328,14 +321,14 @@ func findBestJoin( if i == j { continue } - joinPredicates := qg.getPredicates(lhs.solves(), rhs.solves()) + joinPredicates := qg.getPredicates(lhs.tables(), rhs.tables()) if len(joinPredicates) == 0 && !crossJoinsOK { // if there are no predicates joining the two tables, // creating a join between them would produce a // cartesian product, which is almost always a bad idea continue } - plan := planCache.getJoinFor(lhs, rhs, joinPredicates, semTable) + plan := planCache.getJoinTreeFor(lhs, rhs, joinPredicates, semTable) if bestPlan == nil || plan.cost() < bestPlan.cost() { bestPlan = plan @@ -349,16 +342,9 @@ func findBestJoin( } func leftToRightSolve(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVSchema) (joinTree, error) { - plans := make([]joinTree, len(qg.tables)) - - // we start by seeding the table with the single routes - for i, table := range qg.tables { - solves := semTable.TableSetFor(table.alias) - plan, err := createRoutePlan(table, solves, vschema) - if err != nil { - return nil, err - } - plans[i] = plan + plans, err := seedPlanList(qg, semTable, vschema) + if err != nil { + return nil, err } var acc joinTree @@ -367,13 +353,29 @@ func leftToRightSolve(qg *queryGraph, semTable *semantics.SemTable, vschema Cont acc = plan continue } - joinPredicates := qg.getPredicates(acc.solves(), plan.solves()) + joinPredicates := qg.getPredicates(acc.tables(), plan.tables()) acc = createJoin(acc, plan, joinPredicates, semTable) } return acc, nil } +// seedPlanList returns a routePlan for each table in the qg +func seedPlanList(qg *queryGraph, semTable *semantics.SemTable, vschema ContextVSchema) ([]joinTree, error) { + plans := make([]joinTree, len(qg.tables)) + + // we start by seeding the table with the single routes + for i, table := range qg.tables { + solves := semTable.TableSetFor(table.alias) + plan, err := createRoutePlan(table, solves, vschema) + if err != nil { + return nil, err + } + plans[i] = plan + } + return plans, nil +} + func removeAt(plans []joinTree, idx int) []joinTree { return append(plans[:idx], plans[idx+1:]...) } @@ -385,7 +387,7 @@ func createRoutePlan(table *queryTable, solves semantics.TableSet, vschema Conte } plan := &routePlan{ solved: solves, - tables: []*routeTable{{ + _tables: []*routeTable{{ qtable: table, vtable: vschemaTable, }}, @@ -428,8 +430,8 @@ func transformToLogicalPlan(tree joinTree, semTable *semantics.SemTable) (logica return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "BUG: unknown type encountered: %T", tree) } -func transformJoinPlan(n *joinPlan, semTable *semantics.SemTable) (*join2, error) { - lhsColList := extractColumnsNeededFromLHS(n, semTable, n.lhs.solves()) +func transformJoinPlan(n *joinPlan, semTable *semantics.SemTable) (*joinV4, error) { + lhsColList := extractColumnsNeededFromLHS(n, semTable, n.lhs.tables()) var lhsColExpr []*sqlparser.AliasedExpr for _, col := range lhsColList { @@ -464,7 +466,7 @@ func transformJoinPlan(n *joinPlan, semTable *semantics.SemTable) (*join2, error return nil, err } - return &join2{ + return &joinV4{ Left: lhs, Right: rhs, Vars: vars, @@ -498,8 +500,8 @@ func transformRoutePlan(n *routePlan) (*route, error) { var tablesForSelect sqlparser.TableExprs tableNameMap := map[string]interface{}{} - sort.Sort(n.tables) - for _, t := range n.tables { + sort.Sort(n._tables) + for _, t := range n._tables { alias := sqlparser.AliasedTableExpr{ Expr: sqlparser.TableName{ Name: t.vtable.Name, @@ -548,7 +550,7 @@ func transformRoutePlan(n *routePlan) (*route, error) { From: tablesForSelect, Where: where, }, - solvedTables: n.solved, + tables: n.solved, }, nil } @@ -558,7 +560,7 @@ func findColumnVindex(a *routePlan, exp sqlparser.Expr, sem *semantics.SemTable) return nil } leftDep := sem.Dependencies(left) - for _, table := range a.tables { + for _, table := range a._tables { if leftDep.IsSolvedBy(table.qtable.tableID) { for _, vindex := range table.vtable.ColumnVindexes { singCol, isSingle := vindex.Vindex.(vindexes.SingleColumn) @@ -623,7 +625,7 @@ func tryMerge(a, b joinTree, joinPredicates []sqlparser.Expr, semTable *semantic r := &routePlan{ routeOpCode: aRoute.routeOpCode, solved: newTabletSet, - tables: append(aRoute.tables, bRoute.tables...), + _tables: append(aRoute._tables, bRoute._tables...), extraPredicates: append( append(aRoute.extraPredicates, bRoute.extraPredicates...), joinPredicates...), diff --git a/go/vt/vtgate/planbuilder/select.go b/go/vt/vtgate/planbuilder/select.go index 7df3376662d..79900def0c7 100644 --- a/go/vt/vtgate/planbuilder/select.go +++ b/go/vt/vtgate/planbuilder/select.go @@ -71,11 +71,11 @@ func pushProjection(expr []*sqlparser.AliasedExpr, plan logicalPlan, semTable *s sel.SelectExprs = append(sel.SelectExprs, e) } return offset, nil - case *join2: + case *joinV4: cols := make([]int, len(expr)) var lhs, rhs []*sqlparser.AliasedExpr - lhsSolves := node.Left.Solves() - rhsSolves := node.Right.Solves() + lhsSolves := node.Left.ContainsTables() + rhsSolves := node.Right.ContainsTables() for i, e := range expr { deps := semTable.Dependencies(e.Expr) switch { @@ -123,14 +123,14 @@ func pushPredicate(exprs []sqlparser.Expr, plan logicalPlan, semTable *semantics switch node := plan.(type) { case *route: sel := node.Select.(*sqlparser.Select) - finalExpr := reorderExpression(exprs[0], node.solvedTables, semTable) + finalExpr := reorderExpression(exprs[0], node.tables, semTable) for i, expr := range exprs { if i == 0 { continue } finalExpr = &sqlparser.AndExpr{ Left: finalExpr, - Right: reorderExpression(expr, node.solvedTables, semTable), + Right: reorderExpression(expr, node.tables, semTable), } } if sel.Where != nil { @@ -144,10 +144,10 @@ func pushPredicate(exprs []sqlparser.Expr, plan logicalPlan, semTable *semantics Expr: finalExpr, } return nil - case *join2: + case *joinV4: var lhs, rhs []sqlparser.Expr - lhsSolves := node.Left.Solves() - rhsSolves := node.Right.Solves() + lhsSolves := node.Left.ContainsTables() + rhsSolves := node.Right.ContainsTables() for _, expr := range exprs { deps := semTable.Dependencies(expr) switch { diff --git a/go/vt/vtgate/planbuilder/sql_calc_found_rows.go b/go/vt/vtgate/planbuilder/sql_calc_found_rows.go index 96f5f602502..f9971b0d2da 100644 --- a/go/vt/vtgate/planbuilder/sql_calc_found_rows.go +++ b/go/vt/vtgate/planbuilder/sql_calc_found_rows.go @@ -41,17 +41,17 @@ func (s *sqlCalcFoundRows) Wireup(logicalPlan, *jointab) error { } //Wireup2 implements the logicalPlan interface -func (s *sqlCalcFoundRows) Wireup2(semTable *semantics.SemTable) error { - err := s.LimitQuery.Wireup2(semTable) +func (s *sqlCalcFoundRows) WireupV4(semTable *semantics.SemTable) error { + err := s.LimitQuery.WireupV4(semTable) if err != nil { return err } - return s.CountQuery.Wireup2(semTable) + return s.CountQuery.WireupV4(semTable) } // Solves implements the logicalPlan interface -func (s *sqlCalcFoundRows) Solves() semantics.TableSet { - return s.LimitQuery.Solves() +func (s *sqlCalcFoundRows) ContainsTables() semantics.TableSet { + return s.LimitQuery.ContainsTables() } //Primitive implements the logicalPlan interface diff --git a/go/vt/vtgate/planbuilder/vindex_func.go b/go/vt/vtgate/planbuilder/vindex_func.go index 17c6a24939d..1c0073c39aa 100644 --- a/go/vt/vtgate/planbuilder/vindex_func.go +++ b/go/vt/vtgate/planbuilder/vindex_func.go @@ -99,7 +99,7 @@ func (vf *vindexFunc) Wireup(logicalPlan, *jointab) error { } // Wireup2 implements the logicalPlan interface -func (vf *vindexFunc) Wireup2(*semantics.SemTable) error { +func (vf *vindexFunc) WireupV4(*semantics.SemTable) error { return nil } @@ -144,8 +144,8 @@ func (vf *vindexFunc) Rewrite(inputs ...logicalPlan) error { return nil } -// Solves implements the logicalPlan interface -func (vf *vindexFunc) Solves() semantics.TableSet { +// ContainsTables implements the logicalPlan interface +func (vf *vindexFunc) ContainsTables() semantics.TableSet { return 0 } diff --git a/go/vt/vtgate/semantics/analyzer.go b/go/vt/vtgate/semantics/analyzer.go index bb24521139d..097164380d4 100644 --- a/go/vt/vtgate/semantics/analyzer.go +++ b/go/vt/vtgate/semantics/analyzer.go @@ -23,8 +23,6 @@ import ( "vitess.io/vitess/go/vt/vterrors" ) -var debug = false - type ( // analyzer is a struct to work with analyzing the query. analyzer struct { @@ -32,16 +30,14 @@ type ( scopes []*scope exprDeps map[sqlparser.Expr]TableSet - si schemaInformation err error } ) // newAnalyzer create the semantic analyzer -func newAnalyzer(si schemaInformation) *analyzer { +func newAnalyzer() *analyzer { return &analyzer{ exprDeps: map[sqlparser.Expr]TableSet{}, - si: si, } } @@ -50,7 +46,6 @@ func newAnalyzer(si schemaInformation) *analyzer { func (a *analyzer) analyzeDown(cursor *sqlparser.Cursor) bool { current := a.currentScope() n := cursor.Node() - log(n, "%p analyzeDown %T", current, n) switch node := n.(type) { case *sqlparser.Select: a.push(newScope(current)) @@ -80,7 +75,7 @@ func (a *analyzer) resolveColumn(colName *sqlparser.ColName, current *scope) (Ta var t table var err error if colName.Qualifier.IsEmpty() { - t, err = a.resolveUnQualifiedColumn(current, colName) + t, err = a.resolveUnQualifiedColumn(current) } else { t, err = a.resolveQualifiedColumn(current, colName) } @@ -100,7 +95,6 @@ func (a *analyzer) analyzeTableExprs(tablExprs sqlparser.TableExprs) error { } func (a *analyzer) analyzeTableExpr(tableExpr sqlparser.TableExpr) error { - log(tableExpr, "analyzeTableExpr %T", tableExpr) switch table := tableExpr.(type) { case *sqlparser.AliasedTableExpr: return a.bindTable(table, table.Expr) @@ -133,7 +127,7 @@ func (a *analyzer) resolveQualifiedColumn(current *scope, expr *sqlparser.ColNam } // resolveUnQualifiedColumn -func (a *analyzer) resolveUnQualifiedColumn(current *scope, expr *sqlparser.ColName) (table, error) { +func (a *analyzer) resolveUnQualifiedColumn(current *scope) (table, error) { if len(current.tables) == 1 { for _, tableExpr := range current.tables { return tableExpr, nil @@ -173,7 +167,6 @@ func (a *analyzer) bindTable(alias *sqlparser.AliasedTableExpr, expr sqlparser.S } func (a *analyzer) analyze(statement sqlparser.Statement) error { - log(statement, "analyse %T", statement) _ = sqlparser.Rewrite(statement, a.analyzeDown, a.analyzeUp) return a.err @@ -192,12 +185,10 @@ func (a *analyzer) shouldContinue() bool { } func (a *analyzer) push(s *scope) { - log(nil, "enter new scope") a.scopes = append(a.scopes, s) } func (a *analyzer) popScope() { - log(nil, "exit scope") l := len(a.scopes) - 1 a.scopes = a.scopes[:l] } diff --git a/go/vt/vtgate/semantics/analyzer_test.go b/go/vt/vtgate/semantics/analyzer_test.go index 094b24e331e..710c4551dd7 100644 --- a/go/vt/vtgate/semantics/analyzer_test.go +++ b/go/vt/vtgate/semantics/analyzer_test.go @@ -158,7 +158,7 @@ func TestNotUniqueTableName(t *testing.T) { for _, query := range queries { t.Run(query, func(t *testing.T) { parse, _ := sqlparser.Parse(query) - _, err := Analyse(parse, nil) + _, err := Analyse(parse) require.Error(t, err) require.Contains(t, err.Error(), "Not unique table/alias") }) @@ -173,7 +173,7 @@ func TestMissingTable(t *testing.T) { for _, query := range queries { t.Run(query, func(t *testing.T) { parse, _ := sqlparser.Parse(query) - _, err := Analyse(parse, nil) + _, err := Analyse(parse) require.Error(t, err) require.Contains(t, err.Error(), "Unknown table") }) @@ -183,7 +183,7 @@ func TestMissingTable(t *testing.T) { func parseAndAnalyze(t *testing.T, query string) (sqlparser.Statement, *SemTable) { parse, err := sqlparser.Parse(query) require.NoError(t, err) - semTable, err := Analyse(parse, nil) + semTable, err := Analyse(parse) require.NoError(t, err) return parse, semTable } diff --git a/go/vt/vtgate/semantics/semantic_state.go b/go/vt/vtgate/semantics/semantic_state.go index 2b265f6afa9..555748d4f14 100644 --- a/go/vt/vtgate/semantics/semantic_state.go +++ b/go/vt/vtgate/semantics/semantic_state.go @@ -17,12 +17,8 @@ limitations under the License. package semantics import ( - "fmt" - "vitess.io/vitess/go/mysql" - "vitess.io/vitess/go/vt/vtgate/vindexes" - "vitess.io/vitess/go/vt/sqlparser" ) @@ -39,9 +35,7 @@ type ( Tables []table exprDependencies map[sqlparser.Expr]TableSet } - schemaInformation interface { - FindTable(tablename sqlparser.TableName) (*vindexes.Table, error) - } + scope struct { parent *scope tables map[string]*sqlparser.AliasedTableExpr @@ -88,8 +82,8 @@ func (s *scope) addTable(name string, table *sqlparser.AliasedTableExpr) error { } // Analyse analyzes the parsed query. -func Analyse(statement sqlparser.Statement, si schemaInformation) (*SemTable, error) { - analyzer := newAnalyzer(si) +func Analyse(statement sqlparser.Statement) (*SemTable, error) { + analyzer := newAnalyzer() // Initial scope err := analyzer.analyze(statement) if err != nil { @@ -98,17 +92,6 @@ func Analyse(statement sqlparser.Statement, si schemaInformation) (*SemTable, er return &SemTable{exprDependencies: analyzer.exprDeps, Tables: analyzer.Tables}, nil } -func log(node sqlparser.SQLNode, format string, args ...interface{}) { - if debug { - fmt.Printf(format, args...) - if node == nil { - fmt.Println() - } else { - fmt.Println(" - " + sqlparser.String(node)) - } - } -} - // IsOverlapping returns true if at least one table exists in both sets func (ts TableSet) IsOverlapping(b TableSet) bool { return ts&b != 0 }