From 2bac5f9617c9fe8efb3e6ee5c7a06f619f8299e5 Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Mon, 20 Jan 2025 13:06:18 +0100 Subject: [PATCH 1/3] performance: less allocations Signed-off-by: Andres Taylor --- .../vtgate/planbuilder/operators/ast_to_op.go | 7 +-- go/vt/vtgate/planbuilder/operators/helpers.go | 7 +-- .../planbuilder/operators/querygraph.go | 6 +-- go/vt/vtgate/planbuilder/operators/route.go | 6 +-- .../operators/subquery_planning.go | 8 ++-- go/vt/vtgate/semantics/analyzer.go | 2 +- go/vt/vtgate/semantics/binder.go | 18 ++++---- go/vt/vtgate/semantics/bitset/mutable.go | 46 +++++++++++++++++++ go/vt/vtgate/semantics/cte_table.go | 7 +-- go/vt/vtgate/semantics/derived_table.go | 5 +- go/vt/vtgate/semantics/scoper.go | 7 ++- go/vt/vtgate/semantics/semantic_table.go | 6 ++- go/vt/vtgate/semantics/table_collector.go | 10 ++-- go/vt/vtgate/semantics/table_set.go | 13 ++++-- go/vt/vtgate/semantics/vtable.go | 4 +- 15 files changed, 110 insertions(+), 42 deletions(-) create mode 100644 go/vt/vtgate/semantics/bitset/mutable.go diff --git a/go/vt/vtgate/planbuilder/operators/ast_to_op.go b/go/vt/vtgate/planbuilder/operators/ast_to_op.go index 2e3781c94db..da6a7688d46 100644 --- a/go/vt/vtgate/planbuilder/operators/ast_to_op.go +++ b/go/vt/vtgate/planbuilder/operators/ast_to_op.go @@ -120,17 +120,18 @@ func cloneASTAndSemState[T sqlparser.SQLNode](ctx *plancontext.PlanningContext, } // findTablesContained returns the TableSet of all the contained -func findTablesContained(ctx *plancontext.PlanningContext, node sqlparser.SQLNode) (result semantics.TableSet) { +func findTablesContained(ctx *plancontext.PlanningContext, node sqlparser.SQLNode) semantics.TableSet { + var tables semantics.MutableTableSet _ = sqlparser.Walk(func(node sqlparser.SQLNode) (bool, error) { t, ok := node.(*sqlparser.AliasedTableExpr) if !ok { return true, nil } ts := ctx.SemTable.TableSetFor(t) - result = result.Merge(ts) + tables.MergeInPlace(ts) return true, nil }, node) - return + return tables.ToImmutable() } // joinPredicateCollector is used to inspect the predicates inside the subquery, looking for any diff --git a/go/vt/vtgate/planbuilder/operators/helpers.go b/go/vt/vtgate/planbuilder/operators/helpers.go index 36b10c96ae1..af8e2607b18 100644 --- a/go/vt/vtgate/planbuilder/operators/helpers.go +++ b/go/vt/vtgate/planbuilder/operators/helpers.go @@ -71,14 +71,15 @@ type tableIDIntroducer interface { introducesTableID() semantics.TableSet } -func TableID(op Operator) (result semantics.TableSet) { +func TableID(op Operator) semantics.TableSet { + var tables semantics.MutableTableSet _ = Visit(op, func(this Operator) error { if tbl, ok := this.(tableIDIntroducer); ok { - result = result.Merge(tbl.introducesTableID()) + tables.MergeInPlace(tbl.introducesTableID()) } return nil }) - return + return tables.ToImmutable() } // TableUser is used to signal that this operator directly interacts with one or more tables diff --git a/go/vt/vtgate/planbuilder/operators/querygraph.go b/go/vt/vtgate/planbuilder/operators/querygraph.go index 98cd9ada64a..ac885383801 100644 --- a/go/vt/vtgate/planbuilder/operators/querygraph.go +++ b/go/vt/vtgate/planbuilder/operators/querygraph.go @@ -66,11 +66,11 @@ var _ Operator = (*QueryGraph)(nil) // Introduces implements the tableIDIntroducer interface func (qg *QueryGraph) introducesTableID() semantics.TableSet { - var ts semantics.TableSet + var ts semantics.MutableTableSet for _, table := range qg.Tables { - ts = ts.Merge(table.ID) + ts.MergeInPlace(table.ID) } - return ts + return ts.ToImmutable() } // GetPredicates returns the predicates that are applicable for the two given TableSets diff --git a/go/vt/vtgate/planbuilder/operators/route.go b/go/vt/vtgate/planbuilder/operators/route.go index e398fb05607..95abc54e833 100644 --- a/go/vt/vtgate/planbuilder/operators/route.go +++ b/go/vt/vtgate/planbuilder/operators/route.go @@ -806,9 +806,9 @@ func (r *Route) getTruncateColumnCount() int { } func (r *Route) introducesTableID() semantics.TableSet { - id := semantics.EmptyTableSet() + var tables semantics.MutableTableSet for _, route := range r.MergedWith { - id = id.Merge(TableID(route)) + tables.MergeInPlace(TableID(route)) } - return id + return tables.ToImmutable() } diff --git a/go/vt/vtgate/planbuilder/operators/subquery_planning.go b/go/vt/vtgate/planbuilder/operators/subquery_planning.go index 06ca69dd7f3..e7def099456 100644 --- a/go/vt/vtgate/planbuilder/operators/subquery_planning.go +++ b/go/vt/vtgate/planbuilder/operators/subquery_planning.go @@ -211,7 +211,7 @@ We transform it to: └─────┘ └─────┘ We are rewriting all expressions in the subquery to use arguments any columns coming from the LHS. The join predicate is not affected, but we are adding -any new columns needed by the inner subquery to the JoinVars that the join +any new columns needed by the inner subquery to the JoinVars that the join1 will handle. */ func tryPushSubQueryInJoin( @@ -228,11 +228,11 @@ func tryPushSubQueryInJoin( // we want to push the subquery as close to its needs // as possible, so that we can potentially merge them together // TODO: we need to check dependencies and break apart all expressions in the subquery, not just the merge predicates - deps := semantics.EmptyTableSet() + var tables semantics.MutableTableSet for _, predicate := range inner.GetMergePredicates() { - deps = deps.Merge(ctx.SemTable.RecursiveDeps(predicate)) + tables.MergeInPlace(ctx.SemTable.RecursiveDeps(predicate)) } - deps = deps.Remove(innerID) + deps := tables.ToImmutable().Remove(innerID) // in general, we don't want to push down uncorrelated subqueries into the RHS of a join, // since this side is executed once per row from the LHS, so we would unnecessarily execute diff --git a/go/vt/vtgate/semantics/analyzer.go b/go/vt/vtgate/semantics/analyzer.go index c4e7dc55866..32b3e8930ac 100644 --- a/go/vt/vtgate/semantics/analyzer.go +++ b/go/vt/vtgate/semantics/analyzer.go @@ -174,7 +174,7 @@ func (a *analyzer) newSemTable( Direct: a.binder.direct, ExprTypes: a.typer.m, Tables: a.tables.Tables, - DMLTargets: a.binder.targets, + DMLTargets: a.binder.targets.ToImmutable(), NotSingleRouteErr: a.notSingleRouteErr, NotUnshardedErr: a.unshardedErr, Warning: a.warning, diff --git a/go/vt/vtgate/semantics/binder.go b/go/vt/vtgate/semantics/binder.go index 78148f4bb1f..106b593591d 100644 --- a/go/vt/vtgate/semantics/binder.go +++ b/go/vt/vtgate/semantics/binder.go @@ -31,7 +31,7 @@ import ( type binder struct { recursive ExprDependencies direct ExprDependencies - targets TableSet + targets MutableTableSet scoper *scoper tc *tableCollector org originable @@ -81,7 +81,7 @@ func (b *binder) bindUpdateExpr(ue *sqlparser.UpdateExpr) error { if !ok { return nil } - b.targets = b.targets.Merge(ts) + b.targets.MergeInPlace(ts) return nil } @@ -96,7 +96,7 @@ func (b *binder) bindTableNames(cursor *sqlparser.Cursor, tables sqlparser.Table if err != nil { return err } - b.targets = b.targets.Merge(finalDep.direct) + b.targets.MergeInPlace(finalDep.direct) } return nil } @@ -184,19 +184,20 @@ func (b *binder) findDependentTableSet(current *scope, target sqlparser.TableNam func (b *binder) bindCountStar(node *sqlparser.CountStar) error { scope := b.scoper.currentScope() - var ts TableSet + var tables MutableTableSet for _, tbl := range scope.tables { switch tbl := tbl.(type) { case *vTableInfo: for _, col := range tbl.cols { if sqlparser.Equals.Expr(node, col) { - ts = ts.Merge(b.recursive[col]) + tables.MergeInPlace(b.recursive[col]) } } default: - ts = ts.Merge(tbl.getTableSet(b.org)) + tables.MergeInPlace(tbl.getTableSet(b.org)) } } + ts := tables.ToImmutable() b.recursive[node] = ts b.direct[node] = ts return nil @@ -246,15 +247,16 @@ func (b *binder) setSubQueryDependencies(subq *sqlparser.Subquery) error { subqRecursiveDeps := b.recursive.dependencies(subq) subqDirectDeps := b.direct.dependencies(subq) - tablesToKeep := EmptyTableSet() + var tables MutableTableSet sco := currScope for sco != nil { for _, table := range sco.tables { - tablesToKeep = tablesToKeep.Merge(table.getTableSet(b.org)) + tables.MergeInPlace(table.getTableSet(b.org)) } sco = sco.parent } + tablesToKeep := tables.ToImmutable() b.recursive[subq] = subqRecursiveDeps.KeepOnly(tablesToKeep) b.direct[subq] = subqDirectDeps.KeepOnly(tablesToKeep) return nil diff --git a/go/vt/vtgate/semantics/bitset/mutable.go b/go/vt/vtgate/semantics/bitset/mutable.go new file mode 100644 index 00000000000..7782ad4224b --- /dev/null +++ b/go/vt/vtgate/semantics/bitset/mutable.go @@ -0,0 +1,46 @@ +/* +Copyright 2024 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 bitset + +// Mutable is a growable, in-place version that we can OR bits into +// and eventually turn into a stable (immutable) TableSet. +type Mutable struct { + data []byte +} + +// Or merges another TableSet into this Mutable, resizing if needed. +func (m *Mutable) Or(ts Bitset) { + // If ts is longer than our current data, grow to accommodate it. + if len(ts) > len(m.data) { + oldData := m.data + m.data = make([]byte, len(ts)) + copy(m.data, oldData) + } + // Merge in-place. + for i := 0; i < len(ts); i++ { + m.data[i] |= ts[i] + } +} + +// AsImmutable finalizes the Mutable into a TableSet, trimming trailing zeros. +func (m *Mutable) AsImmutable() Bitset { + trim := len(m.data) + for trim > 0 && m.data[trim-1] == 0 { + trim-- + } + return toBitset(m.data[:trim]) +} diff --git a/go/vt/vtgate/semantics/cte_table.go b/go/vt/vtgate/semantics/cte_table.go index 7083fea10d2..77da0a04113 100644 --- a/go/vt/vtgate/semantics/cte_table.go +++ b/go/vt/vtgate/semantics/cte_table.go @@ -163,10 +163,11 @@ type CTE struct { Merged bool } -func (cte *CTE) recursive(org originable) (id TableSet) { +func (cte *CTE) recursive(org originable) TableSet { if cte.recursiveDeps != nil { return *cte.recursiveDeps } + var id MutableTableSet // We need to find the recursive dependencies of the CTE // We'll do this by walking the inner query and finding all the tables @@ -175,8 +176,8 @@ func (cte *CTE) recursive(org originable) (id TableSet) { if !ok { return true, nil } - id = id.Merge(org.tableSetFor(ate)) + id.MergeInPlace(org.tableSetFor(ate)) return true, nil }, cte.Query) - return + return id.ToImmutable() } diff --git a/go/vt/vtgate/semantics/derived_table.go b/go/vt/vtgate/semantics/derived_table.go index fc7e1cb391c..38169514cd9 100644 --- a/go/vt/vtgate/semantics/derived_table.go +++ b/go/vt/vtgate/semantics/derived_table.go @@ -92,12 +92,15 @@ func handleAliasedExpr(vTbl *DerivedTable, expr *sqlparser.AliasedExpr, cols sql } func handleUnexpandedStarExpression(tables []TableInfo, vTbl *DerivedTable, org originable) { + var tableSets MutableTableSet for _, table := range tables { - vTbl.tables = vTbl.tables.Merge(table.getTableSet(org)) + ts := table.getTableSet(org) + tableSets.MergeInPlace(ts) if !table.authoritative() { vTbl.isAuthoritative = false } } + vTbl.tables = tableSets.ToImmutable() } // dependencies implements the TableInfo interface diff --git a/go/vt/vtgate/semantics/scoper.go b/go/vt/vtgate/semantics/scoper.go index e6df3c3a5b0..27fbff9ce0c 100644 --- a/go/vt/vtgate/semantics/scoper.go +++ b/go/vt/vtgate/semantics/scoper.go @@ -228,12 +228,11 @@ func (s *scoper) up(cursor *sqlparser.Cursor) error { s.popScope() } case *sqlparser.Select, *sqlparser.GroupBy, *sqlparser.Update, *sqlparser.Insert, *sqlparser.Union, *sqlparser.Delete: - id := EmptyTableSet() + var id MutableTableSet for _, tableInfo := range s.currentScope().tables { - set := tableInfo.getTableSet(s.org) - id = id.Merge(set) + id.MergeInPlace(tableInfo.getTableSet(s.org)) } - s.statementIDs[s.currentScope().stmt] = id + s.statementIDs[s.currentScope().stmt] = id.ToImmutable() s.popScope() case *sqlparser.Where: if node.Type != sqlparser.HavingClause { diff --git a/go/vt/vtgate/semantics/semantic_table.go b/go/vt/vtgate/semantics/semantic_table.go index e3eead71c90..c18067b8316 100644 --- a/go/vt/vtgate/semantics/semantic_table.go +++ b/go/vt/vtgate/semantics/semantic_table.go @@ -697,6 +697,7 @@ func (d ExprDependencies) dependencies(expr sqlparser.Expr) (deps TableSet) { if found { return deps } + // If we did not find the expression in the cache, we'll add it after calculating it defer func() { d[expr] = deps }() @@ -705,6 +706,7 @@ func (d ExprDependencies) dependencies(expr sqlparser.Expr) (deps TableSet) { // During the original semantic analysis, all ColNames were found and bound to the corresponding tables // Here, we'll walk the expression tree and look to see if we can find any sub-expressions // that have already set dependencies. + var depsCalc MutableTableSet _ = sqlparser.Walk(func(node sqlparser.SQLNode) (kontinue bool, err error) { expr, ok := node.(sqlparser.Expr) if !ok || !ValidAsMapKey(expr) { @@ -714,12 +716,14 @@ func (d ExprDependencies) dependencies(expr sqlparser.Expr) (deps TableSet) { } set, found := d[expr] - deps = deps.Merge(set) + depsCalc.MergeInPlace(set) // if we found a cached value, there is no need to continue down to visit children return !found, nil }, expr) + deps = depsCalc.ToImmutable() + return deps } diff --git a/go/vt/vtgate/semantics/table_collector.go b/go/vt/vtgate/semantics/table_collector.go index 329ebcef254..5edf350dc08 100644 --- a/go/vt/vtgate/semantics/table_collector.go +++ b/go/vt/vtgate/semantics/table_collector.go @@ -169,7 +169,7 @@ func (tc *tableCollector) visitUnion(union *sqlparser.Union) error { } size := firstSelect.GetColumnCount() - info.recursive = make([]TableSet, size) + recursiveDeps := make([]MutableTableSet, size) typers := make([]evalengine.TypeAggregator, size) collations := tc.org.collationEnv() @@ -179,8 +179,8 @@ func (tc *tableCollector) visitUnion(union *sqlparser.Union) error { if !ok { continue } - _, recursiveDeps, qt := tc.org.depsForExpr(ae.Expr) - info.recursive[i] = info.recursive[i].Merge(recursiveDeps) + _, deps, qt := tc.org.depsForExpr(ae.Expr) + recursiveDeps[i].MergeInPlace(deps) if err := typers[i].Add(qt, collations); err != nil { return err } @@ -191,8 +191,10 @@ func (tc *tableCollector) visitUnion(union *sqlparser.Union) error { return err } - for _, ts := range typers { + info.recursive = make([]TableSet, size) + for i, ts := range typers { info.types = append(info.types, ts.Type()) + info.recursive[i] = recursiveDeps[i].ToImmutable() } tc.unionInfo[union] = info return nil diff --git a/go/vt/vtgate/semantics/table_set.go b/go/vt/vtgate/semantics/table_set.go index acc83306869..4425a51117a 100644 --- a/go/vt/vtgate/semantics/table_set.go +++ b/go/vt/vtgate/semantics/table_set.go @@ -123,7 +123,14 @@ func MergeTableSets(tss ...TableSet) TableSet { return TableSet(result) } -// TableSetFromIds returns TableSet for all the id passed in argument. -func TableSetFromIds(tids ...int) (ts TableSet) { - return TableSet(bitset.Build(tids...)) +type MutableTableSet struct { + bitset bitset.Mutable +} + +func (ts *MutableTableSet) MergeInPlace(other TableSet) { + ts.bitset.Or(bitset.Bitset(other)) +} + +func (ts *MutableTableSet) ToImmutable() TableSet { + return TableSet(ts.bitset.AsImmutable()) } diff --git a/go/vt/vtgate/semantics/vtable.go b/go/vt/vtgate/semantics/vtable.go index 6cd7e34aecc..a6c8a3eff8c 100644 --- a/go/vt/vtgate/semantics/vtable.go +++ b/go/vt/vtgate/semantics/vtable.go @@ -165,12 +165,14 @@ func selectExprsToInfos( colNames = append(colNames, expr.As.String()) } case *sqlparser.StarExpr: + var tableSets MutableTableSet for _, table := range tables { - ts = ts.Merge(table.getTableSet(org)) + tableSets.MergeInPlace(table.getTableSet(org)) if !table.authoritative() { isAuthoritative = false } } + ts = tableSets.ToImmutable() } } return From 76b1b0133d3db88425ebf4596d9ed55ea71c2239 Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Mon, 20 Jan 2025 13:23:33 +0100 Subject: [PATCH 2/3] only copy already calculated deps Signed-off-by: Andres Taylor --- go/vt/vtgate/semantics/semantic_table.go | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/go/vt/vtgate/semantics/semantic_table.go b/go/vt/vtgate/semantics/semantic_table.go index c18067b8316..326603798aa 100644 --- a/go/vt/vtgate/semantics/semantic_table.go +++ b/go/vt/vtgate/semantics/semantic_table.go @@ -190,13 +190,15 @@ var ErrNotSingleTable = vterrors.VT13001("should only be used for single tables" // CopyDependencies copies the dependencies from one expression into the other func (st *SemTable) CopyDependencies(from, to sqlparser.Expr) { - if ValidAsMapKey(to) { - st.Recursive[to] = st.RecursiveDeps(from) - st.Direct[to] = st.DirectDeps(from) - if ValidAsMapKey(from) { - if typ, found := st.ExprTypes[from]; found { - st.ExprTypes[to] = typ - } + if ValidAsMapKey(to) && ValidAsMapKey(from) { + if deps, found := st.Recursive[from]; found { + st.Recursive[to] = deps + } + if deps, found := st.Direct[from]; found { + st.Direct[to] = deps + } + if typ, found := st.ExprTypes[from]; found { + st.ExprTypes[to] = typ } } } From 833b859ce4ed635c3f9edf0d10157cb1573b6912 Mon Sep 17 00:00:00 2001 From: Andres Taylor Date: Fri, 17 Jan 2025 11:26:30 +0100 Subject: [PATCH 3/3] add benchmark Signed-off-by: Andres Taylor --- go/vt/vtgate/planbuilder/plan_test.go | 38 + go/vt/vtgate/planbuilder/testdata/my.json | 740 ++++++++++++++++++ .../testdata/vschemas/my_schema.json | 110 +++ 3 files changed, 888 insertions(+) create mode 100644 go/vt/vtgate/planbuilder/testdata/my.json create mode 100644 go/vt/vtgate/planbuilder/testdata/vschemas/my_schema.json diff --git a/go/vt/vtgate/planbuilder/plan_test.go b/go/vt/vtgate/planbuilder/plan_test.go index 8a8d55279a6..de68a76f017 100644 --- a/go/vt/vtgate/planbuilder/plan_test.go +++ b/go/vt/vtgate/planbuilder/plan_test.go @@ -872,3 +872,41 @@ func benchmarkPlanner(b *testing.B, version plancontext.PlannerVersion, testCase } } } + +func (s *planTestSuite) TestMy() { + vschema, err := vschemawrapper.NewVschemaWrapper( + vtenv.NewTestEnv(), + loadSchema(s.T(), "vschemas/my_schema.json", true), + TestBuilder, + ) + require.NoError(s.T(), err) + // vschema := &vschemawrapper.VSchemaWrapper{ + // V: loadSchema(s.T(), "vschemas/my_schema.json", true), + // SysVarEnabled: true, + // Version: Gen4, + // Env: vtenv.NewTestEnv(), + // } + + s.testFile("my.json", vschema, false) +} + +func BenchmarkMine(b *testing.B) { + vschema, err := vschemawrapper.NewVschemaWrapper( + vtenv.NewTestEnv(), + loadSchema(b, "vschemas/my_schema.json", true), + TestBuilder, + ) + require.NoError(b, err) + testCases := readJSONTests("my.json") + + b.ReportAllocs() + b.ResetTimer() + for range b.N { + for _, tcase := range testCases { + plan, _ := TestBuilder(tcase.Query, vschema, vschema.CurrentDb()) + if plan == nil { + panic("") + } + } + } +} diff --git a/go/vt/vtgate/planbuilder/testdata/my.json b/go/vt/vtgate/planbuilder/testdata/my.json new file mode 100644 index 00000000000..ae8403be495 --- /dev/null +++ b/go/vt/vtgate/planbuilder/testdata/my.json @@ -0,0 +1,740 @@ +[ + { + "comment": "Add your test case here for debugging and run go test -run=One.", + "query": "select col04, col03, col01, col02 from my_table where id in ((select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '2XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'9FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'VXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'AFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '6XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'6FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'ZXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'5FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'GXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'DFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'JXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'DFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '8XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'DFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'FXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'0FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'FXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'EFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'RXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'7FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'JXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'5FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'YXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'2FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'RXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'AFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'YXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'9FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '1XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'7FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'RXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'CFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'XXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'CFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'WXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'AFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '2XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'BFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'RXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'4FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'HXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'7FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'QXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'0FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'ZXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'EFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'NXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'BFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'XXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'BFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'WXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'AFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'PXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'3FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'BXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'4FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'XXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'5FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'TXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'4FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '4XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'DFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'KXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'4FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '3XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'FFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '0XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'AFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '1XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'7FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'JXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'3FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'SXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'7FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '5XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'2FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'KXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'5FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'XXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'CFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'RXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'7FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'XXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'FFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'HXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'2FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '5XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'BFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'CXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'DFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'NXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'EFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'TXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'CFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'DXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'9FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'GXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'4FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'RXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'BFFF' order by col02 desc limit 0, 10000)) order by col01 desc, sdKey desc, col02 desc limit 0, 500000;", + "plan": { + "QueryType": "SELECT", + "Original": "select col04, col03, col01, col02 from my_table where id in ((select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '2XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'9FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'VXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'AFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '6XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'6FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'ZXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'5FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'GXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'DFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'JXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'DFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '8XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'DFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'FXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'0FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'FXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'EFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'RXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'7FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'JXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'5FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'YXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'2FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'RXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'AFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'YXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'9FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '1XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'7FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'RXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'CFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'XXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'CFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'WXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'AFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '2XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'BFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'RXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'4FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'HXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'7FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'QXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'0FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'ZXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'EFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'NXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'BFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'XXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'BFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'WXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'AFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'PXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'3FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'BXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'4FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'XXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'5FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'TXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'4FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '4XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'DFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'KXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'4FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '3XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'FFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '0XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'AFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '1XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'7FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'JXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'3FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'SXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'7FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '5XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'2FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'KXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'5FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'XXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'CFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'RXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'7FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'XXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'FFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'HXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'2FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '5XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'BFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'CXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'DFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'NXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'EFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'TXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'CFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'DXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'9FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'GXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'4FFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'RXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'BFFF' order by col02 desc limit 0, 10000)) order by col01 desc, sdKey desc, col02 desc limit 0, 500000;", + "Instructions": { + "OperatorType": "Limit", + "Count": "500000", + "Offset": "0", + "Inputs": [ + { + "OperatorType": "UncorrelatedSubquery", + "Variant": "PulloutIn", + "PulloutVars": [ + "__sq_has_values", + "__sq1" + ], + "Inputs": [ + { + "InputName": "SubQuery", + "OperatorType": "Concatenate", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '2XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'9FFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'\ufffd\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'VXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'AFFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'\ufffd\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '6XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'6FFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'o\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'ZXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'5FFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'_\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "(select id from my_table force index (MY_INDEX) where 1 != 1) union all (select id from my_table force index (MY_INDEX) where 1 != 1) union all (select id from my_table force index (MY_INDEX) where 1 != 1)", + "Query": "(select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'GXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'DFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'JXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'DFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '8XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'DFFF' order by col02 desc limit 0, 10000)", + "Table": "my_table", + "Values": [ + "_binary'\ufffd\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'FXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'0FFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'\u000f\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'FXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'EFFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'\ufffd\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'RXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'7FFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'JXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'5FFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'_\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'YXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'2FFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'/\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'RXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'AFFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'\ufffd\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'YXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'9FFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'\ufffd\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '1XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'7FFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "(select id from my_table force index (MY_INDEX) where 1 != 1) union all (select id from my_table force index (MY_INDEX) where 1 != 1)", + "Query": "(select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'RXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'CFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'XXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'CFFF' order by col02 desc limit 0, 10000)", + "Table": "my_table", + "Values": [ + "_binary'\ufffd\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'WXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'AFFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'\ufffd\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '2XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'BFFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'\ufffd\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'RXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'4FFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'O\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'HXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'7FFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'QXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'0FFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'\u000f\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'ZXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'EFFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'\ufffd\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "(select id from my_table force index (MY_INDEX) where 1 != 1) union all (select id from my_table force index (MY_INDEX) where 1 != 1)", + "Query": "(select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'NXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'BFFF' order by col02 desc limit 0, 10000) union all (select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'XXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'BFFF' order by col02 desc limit 0, 10000)", + "Table": "my_table", + "Values": [ + "_binary'\ufffd\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'WXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'AFFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'\ufffd\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'PXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'3FFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'?\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'BXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'4FFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'O\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'XXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'5FFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'_\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'TXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'4FFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'O\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '4XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'DFFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'\ufffd\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'KXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'4FFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'O\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '3XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'FFFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'\ufffd\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '0XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'AFFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'\ufffd\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '1XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'7FFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'JXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'3FFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'?\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'SXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'7FFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '5XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'2FFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'/\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'KXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'5FFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'_\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'XXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'CFFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'\ufffd\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'RXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'7FFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'XXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'FFFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'\ufffd\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'HXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'2FFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'/\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = '5XxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'BFFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'\ufffd\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'CXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'DFFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'\ufffd\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'NXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'EFFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'\ufffd\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'TXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'CFFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'\ufffd\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'DXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'9FFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'\ufffd\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'GXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'4FFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'O\ufffd'" + ], + "Vindex": "binary_vdx" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select id from my_table force index (MY_INDEX) where 1 != 1", + "Query": "select id from my_table force index (MY_INDEX) where col02 <= 1234567890000 and col10 = 1234567890000000000 and col06 = 0 and col02 >= 1000000000000 and col01 = 'RXxXxXxXxXxXxXxXxXxXxXx' and sdKey = X'BFFF' order by col02 desc limit 0, 10000", + "Table": "my_table", + "Values": [ + "_binary'\ufffd\ufffd'" + ], + "Vindex": "binary_vdx" + } + ] + }, + { + "InputName": "Outer", + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "main", + "Sharded": true + }, + "FieldQuery": "select col04, col03, col01, col02, sdKey from my_table where 1 != 1", + "OrderBy": "2 DESC COLLATE latin1_swedish_ci, 4 DESC, 3 DESC", + "Query": "select col04, col03, col01, col02, sdKey from my_table where :__sq_has_values and id in ::__sq1 order by my_table.col01 desc, sdKey desc, my_table.col02 desc", + "ResultColumns": 4, + "Table": "my_table" + } + ] + } + ] + }, + "TablesUsed": [ + "main.my_table" + ] + } + } +] diff --git a/go/vt/vtgate/planbuilder/testdata/vschemas/my_schema.json b/go/vt/vtgate/planbuilder/testdata/vschemas/my_schema.json new file mode 100644 index 00000000000..f87c0cd5b60 --- /dev/null +++ b/go/vt/vtgate/planbuilder/testdata/vschemas/my_schema.json @@ -0,0 +1,110 @@ +{ + "keyspaces": { + "unsharded_ns": { + "sharded": false, + "tables": { + "my_table_seq": { + "type": "sequence", + "columns": [ + { + "name": "next_id", + "type": "UINT64" + }, + { + "name": "cache", + "type": "UINT32" + }, + { + "name": "id", + "type": "UINT64" + } + ], + "column_list_authoritative": true + } + } + }, + "main": { + "sharded": true, + "vindexes": { + "binary_vdx": { + "type": "binary" + } + }, + "tables": { + "my_table": { + "column_vindexes": [ + { + "columns": [ + "sdKey" + ], + "name": "binary_vdx" + } + ], + "auto_increment": { + "column": "id", + "sequence": "my_table_seq" + }, + "columns": [ + { + "name": "col01", + "type": "VARCHAR" + }, + { + "name": "col02", + "type": "INT64" + }, + { + "name": "col03", + "type": "BLOB" + }, + { + "name": "col04", + "type": "VARCHAR" + }, + { + "name": "col05", + "type": "VARBINARY" + }, + { + "name": "col06", + "type": "INT8" + }, + { + "name": "col07", + "type": "INT64" + }, + { + "name": "col08", + "type": "VARBINARY" + }, + { + "name": "sdKey", + "type": "VARBINARY" + }, + { + "name": "col09", + "type": "INT64" + }, + { + "name": "col10", + "type": "INT64" + }, + { + "name": "col11", + "type": "INT64" + }, + { + "name": "col12", + "type": "INT64" + }, + { + "name": "id", + "type": "UINT64" + } + ], + "column_list_authoritative": true + } + } + } + } +}