diff --git a/.github/workflows/maven_test.yml b/.github/workflows/maven_test.yml index 0d03f85..3595424 100644 --- a/.github/workflows/maven_test.yml +++ b/.github/workflows/maven_test.yml @@ -138,6 +138,27 @@ jobs: - name: Test charEncoding run: mvn clean test -Dtest=\ com.jd.jdbc.engine.CharEncodingTest + - name: Test gen4 + run: mvn clean test -Dtest=\ + com.jd.jdbc.engine.gen4.ConcatenateGen4EngineTest,\ + com.jd.jdbc.engine.gen4.DistinctGen4EngineTest,\ + com.jd.jdbc.engine.gen4.FilterGen4EngineTest,\ + com.jd.jdbc.engine.gen4.JoinGen4EngineTest,\ + com.jd.jdbc.engine.gen4.LimitGen4EngineTest,\ + com.jd.jdbc.engine.gen4.MemorySortGen4EngineTest,\ + com.jd.jdbc.engine.gen4.OrderedAggregateGen4EngineTest,\ + com.jd.jdbc.engine.gen4.RouteGen4EngineTest,\ + com.jd.jdbc.engine.gen4.ScalarAggregateGen4EngineTest,\ + com.jd.jdbc.engine.gen4.VitessCompareTest,\ + com.jd.jdbc.planbuilder.Gen4PlanTest,\ + com.jd.jdbc.planbuilder.gen4.AnalyzerTest,\ + com.jd.jdbc.planbuilder.gen4.HorizonPlanningTest,\ + com.jd.jdbc.planbuilder.gen4.OperatorTest,\ + com.jd.jdbc.planbuilder.gen4.QueryProjectionTest,\ + com.jd.jdbc.planbuilder.gen4.RewriterTest,\ + com.jd.jdbc.planbuilder.gen4.operator.physical.RoutePlanningTest,\ + com.jd.jdbc.planbuilder.semantics.EarlyRewriterTest,\ + com.jd.jdbc.planbuilder.semantics.TableSetTest - name: install vtdriver run: mvn clean install -Dmaven.test.skip=true # - name: Test starter diff --git a/.gitignore b/.gitignore index 2d496b3..fc78cb9 100644 --- a/.gitignore +++ b/.gitignore @@ -170,3 +170,8 @@ hs_err_pid* /src/test/resources/vitess_env/build_vitess/ dependency-reduced-pom.xml + + +# vscode + +.vscode diff --git a/src/main/java/com/jd/jdbc/common/Constant.java b/src/main/java/com/jd/jdbc/common/Constant.java index 64718ff..57ee48e 100644 --- a/src/main/java/com/jd/jdbc/common/Constant.java +++ b/src/main/java/com/jd/jdbc/common/Constant.java @@ -45,4 +45,5 @@ public class Constant { public static final String DEFAULT_SPLIT_TABLE_CONFIG_PATH = "vtdriver-split-table.yml"; + public static final String GEN4_PLAN_ENABLE = "vtdriver.gen4plan.enable"; } diff --git a/src/main/java/com/jd/jdbc/context/PlanningContext.java b/src/main/java/com/jd/jdbc/context/PlanningContext.java new file mode 100644 index 0000000..9a950a1 --- /dev/null +++ b/src/main/java/com/jd/jdbc/context/PlanningContext.java @@ -0,0 +1,61 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.context; + +import com.jd.jdbc.planbuilder.semantics.SemTable; +import com.jd.jdbc.planbuilder.semantics.VSchema; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.vindexes.VKeyspace; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import lombok.Getter; + +@Getter +public class PlanningContext { + + private Object reservedVars; + + private SemTable semTable; + + private VKeyspace keyspace; + + private VSchema vschema; + + // here we add all predicates that were created because of a join condition + // e.g. [FROM tblA JOIN tblB ON a.colA = b.colB] will be rewritten to [FROM tblB WHERE :a_colA = b.colB], + // if we assume that tblB is on the RHS of the join. This last predicate in the WHERE clause is added to the + // map below + + private Map> joinPredicates; + + private Map skipPredicates; + + private boolean rewriteDerivedExpr; + + public PlanningContext(Object reservedVars, SemTable semTable, VSchema vschema, VKeyspace keyspace) { + this.reservedVars = reservedVars; + this.semTable = semTable; + this.vschema = vschema; + this.keyspace = keyspace; + this.skipPredicates = new HashMap<>(); + this.joinPredicates = new HashMap<>(); + this.rewriteDerivedExpr = false; + } +} diff --git a/src/main/java/com/jd/jdbc/engine/Engine.java b/src/main/java/com/jd/jdbc/engine/Engine.java index 5f50024..1c615f4 100644 --- a/src/main/java/com/jd/jdbc/engine/Engine.java +++ b/src/main/java/com/jd/jdbc/engine/Engine.java @@ -20,6 +20,19 @@ import com.google.common.collect.Lists; import com.jd.jdbc.IExecute; +import static com.jd.jdbc.engine.Engine.AggregateOpcode.AggregateAvg; +import static com.jd.jdbc.engine.Engine.AggregateOpcode.AggregateAvgCount; +import static com.jd.jdbc.engine.Engine.AggregateOpcode.AggregateAvgSum; +import static com.jd.jdbc.engine.Engine.AggregateOpcode.AggregateCount; +import static com.jd.jdbc.engine.Engine.AggregateOpcode.AggregateCountDistinct; +import static com.jd.jdbc.engine.Engine.AggregateOpcode.AggregateMax; +import static com.jd.jdbc.engine.Engine.AggregateOpcode.AggregateMin; +import static com.jd.jdbc.engine.Engine.AggregateOpcode.AggregateSum; +import static com.jd.jdbc.engine.Engine.AggregateOpcode.AggregateSumDistinct; +import static com.jd.jdbc.engine.Engine.PulloutOpcode.PulloutExists; +import static com.jd.jdbc.engine.Engine.PulloutOpcode.PulloutIn; +import static com.jd.jdbc.engine.Engine.PulloutOpcode.PulloutNotIn; +import static com.jd.jdbc.engine.Engine.PulloutOpcode.PulloutValue; import com.jd.jdbc.key.Destination; import com.jd.jdbc.sqlparser.ast.SQLObject; import com.jd.jdbc.sqlparser.dialect.mysql.visitor.VtRestoreVisitor; @@ -43,20 +56,6 @@ import java.util.stream.Collectors; import lombok.Getter; -import static com.jd.jdbc.engine.Engine.AggregateOpcode.AggregateAvg; -import static com.jd.jdbc.engine.Engine.AggregateOpcode.AggregateAvgCount; -import static com.jd.jdbc.engine.Engine.AggregateOpcode.AggregateAvgSum; -import static com.jd.jdbc.engine.Engine.AggregateOpcode.AggregateCount; -import static com.jd.jdbc.engine.Engine.AggregateOpcode.AggregateCountDistinct; -import static com.jd.jdbc.engine.Engine.AggregateOpcode.AggregateMax; -import static com.jd.jdbc.engine.Engine.AggregateOpcode.AggregateMin; -import static com.jd.jdbc.engine.Engine.AggregateOpcode.AggregateSum; -import static com.jd.jdbc.engine.Engine.AggregateOpcode.AggregateSumDistinct; -import static com.jd.jdbc.engine.Engine.PulloutOpcode.PulloutExists; -import static com.jd.jdbc.engine.Engine.PulloutOpcode.PulloutIn; -import static com.jd.jdbc.engine.Engine.PulloutOpcode.PulloutNotIn; -import static com.jd.jdbc.engine.Engine.PulloutOpcode.PulloutValue; - public class Engine { public static final String LIST_VAR_NAME = "__vals"; @@ -182,6 +181,20 @@ public static List getQueries(SQLObject query, List getQueriesGen4(SQLObject query, List> bindVariableMapList, String charEncoding) throws SQLException { + List queries = new ArrayList<>(bindVariableMapList.size()); + for (Map bindVariableMap : bindVariableMapList) { + StringBuilder realQueryOutput = new StringBuilder(); + VtRestoreVisitor vtRestoreVisitor = new VtRestoreVisitor(realQueryOutput, bindVariableMap, charEncoding); + query.accept(vtRestoreVisitor); + if (vtRestoreVisitor.getException() != null) { + throw vtRestoreVisitor.getException(); + } + queries.add(new BoundQuery(realQueryOutput.toString(), bindVariableMap)); + } + return queries; + } + /** * @param query * @param bindVariableMapList @@ -289,10 +302,17 @@ public enum RouteOpcode { * SelectNone is used for queries that always return empty values */ SelectNone(8), + + /** + * ByDestination is to route explicitly to a given target destination. + * Is used when the query explicitly sets a target destination: + * in the clause e.g: UPDATE `keyspace[-]`.x1 SET foo=1 + */ + ByDestination(9), /** * NumRouteOpcodes is the number of opcodes */ - NumRouteOpcodes(9); + NumRouteOpcodes(10); @Getter private final Integer value; @@ -363,7 +383,8 @@ public enum AggregateOpcode { AggregateSumDistinct(5), AggregateAvg(6), AggregateAvgSum(7), - AggregateAvgCount(8); + AggregateAvgCount(8), + AggregateRandom(9); @Getter private final Integer value; @@ -373,6 +394,31 @@ public enum AggregateOpcode { } } + + /** + * These constants list the possible aggregate opcodes. + */ + public enum AggregateOpcodeG4 { + /***/ + AggregateUnassigned(0), + AggregateCount(1), + AggregateSum(2), + AggregateMin(3), + AggregateMax(4), + AggregateCountDistinct(5), + AggregateSumDistinct(6), + AggregateGtid(7), + AggregateRandom(8), + AggregateCountStar(9); + + @Getter + private final Integer value; + + AggregateOpcodeG4(Integer value) { + this.value = value; + } + } + /** * This is the list of InsertOpcode values. */ diff --git a/src/main/java/com/jd/jdbc/engine/OrderedAggregateEngine.java b/src/main/java/com/jd/jdbc/engine/OrderedAggregateEngine.java index 6ddbaf5..e0a4749 100644 --- a/src/main/java/com/jd/jdbc/engine/OrderedAggregateEngine.java +++ b/src/main/java/com/jd/jdbc/engine/OrderedAggregateEngine.java @@ -302,7 +302,6 @@ private IExecute.ExecuteMultiShardResponse orderedAggregateExecute(IContext ctx, * @throws SQLException */ private rowProcessResponse merge(Query.Field[] fields, List row1, List row2, VtResultValue curDistinct) throws SQLException { - // result := sqltypes.CopyRow(row1) List result = new ArrayList<>(row1); for (AggregateParams aggr : this.aggregateParamsList) { if (aggr.isDistinct()) { diff --git a/src/main/java/com/jd/jdbc/engine/ProjectionEngine.java b/src/main/java/com/jd/jdbc/engine/ProjectionEngine.java index 409b054..af3bd9d 100644 --- a/src/main/java/com/jd/jdbc/engine/ProjectionEngine.java +++ b/src/main/java/com/jd/jdbc/engine/ProjectionEngine.java @@ -21,6 +21,7 @@ import com.jd.jdbc.IExecute; import com.jd.jdbc.context.IContext; import com.jd.jdbc.evalengine.EvalEngine; +import com.jd.jdbc.evalengine.EvalResult; import com.jd.jdbc.sqlparser.utils.StringUtils; import com.jd.jdbc.sqltypes.VtResultSet; import com.jd.jdbc.sqltypes.VtResultValue; @@ -122,7 +123,7 @@ public VtRowList fetch(boolean wantFields) throws SQLException { for (List row : vtResultSet.getRows()) { env.setRow(row); for (EvalEngine.Expr expr : exprs) { - EvalEngine.EvalResult res = expr.evaluate(env); + EvalResult res = expr.evaluate(env); row.add(convertToVtResultValue(res, vcursor.getCharEncoding())); } rows.add(row); @@ -232,7 +233,7 @@ private IExecute.ExecuteMultiShardResponse getExecuteMultiShardResponse(VtResult for (List row : resultSet.getRows()) { env.setRow(row); for (EvalEngine.Expr expr : this.exprs) { - EvalEngine.EvalResult res = expr.evaluate(env); + EvalResult res = expr.evaluate(env); row.add(convertToVtResultValue(res, charEncoding)); } rows.add(row); @@ -241,15 +242,15 @@ private IExecute.ExecuteMultiShardResponse getExecuteMultiShardResponse(VtResult return new IExecute.ExecuteMultiShardResponse(resultSet); } - private static VtResultValue convertToVtResultValue(EvalEngine.EvalResult res, String charEncoding) throws SQLException { + private static VtResultValue convertToVtResultValue(EvalResult res, String charEncoding) throws SQLException { VtResultValue resultValue; switch (res.getType()) { case FLOAT64: - EvalEngine.EvalResult evalResult1 = new EvalEngine.EvalResult(BigDecimal.valueOf(res.getFval()).setScale(4, RoundingMode.HALF_UP), Query.Type.DECIMAL); + EvalResult evalResult1 = new EvalResult(BigDecimal.valueOf(res.getFval()).setScale(4, RoundingMode.HALF_UP), Query.Type.DECIMAL); resultValue = evalResult1.resultValue(); break; case VARBINARY: - EvalEngine.EvalResult evalResult2 = new EvalEngine.EvalResult(res.getBytes(), Query.Type.VARBINARY); + EvalResult evalResult2 = new EvalResult(res.getBytes(), Query.Type.VARBINARY); Charset cs = StringUtils.isEmpty(charEncoding) ? Charset.defaultCharset() : Charset.forName(charEncoding); resultValue = VtResultValue.newVtResultValue(Query.Type.VARBINARY, new String(evalResult2.getBytes(), cs)); break; diff --git a/src/main/java/com/jd/jdbc/engine/RouteEngine.java b/src/main/java/com/jd/jdbc/engine/RouteEngine.java index f384968..c8ecffa 100644 --- a/src/main/java/com/jd/jdbc/engine/RouteEngine.java +++ b/src/main/java/com/jd/jdbc/engine/RouteEngine.java @@ -22,6 +22,7 @@ import com.jd.jdbc.IExecute.ExecuteMultiShardResponse; import com.jd.jdbc.context.IContext; import com.jd.jdbc.evalengine.EvalEngine; +import com.jd.jdbc.evalengine.EvalResult; import com.jd.jdbc.key.Destination; import com.jd.jdbc.key.DestinationAllShard; import com.jd.jdbc.key.DestinationAnyShard; @@ -336,7 +337,7 @@ public ParamsResponse paramsSystemQuery(Vcursor vcursor, Map SUPPORTED_AGGREGATES = new HashMap<>(9); + + static { + SUPPORTED_AGGREGATES.put("count", AggregateOpcodeG4.AggregateCount); + SUPPORTED_AGGREGATES.put("sum", AggregateOpcodeG4.AggregateSum); + SUPPORTED_AGGREGATES.put("min", AggregateOpcodeG4.AggregateMin); + SUPPORTED_AGGREGATES.put("max", AggregateOpcodeG4.AggregateMax); + + // These functions don't exist in mysql, but are used to display the plan. + SUPPORTED_AGGREGATES.put("count_distinct", AggregateOpcodeG4.AggregateCountDistinct); + SUPPORTED_AGGREGATES.put("sum_distinct", AggregateOpcodeG4.AggregateSumDistinct); + SUPPORTED_AGGREGATES.put("vgtid", AggregateOpcodeG4.AggregateGtid); + SUPPORTED_AGGREGATES.put("count_star", AggregateOpcodeG4.AggregateCountStar); + SUPPORTED_AGGREGATES.put("random", AggregateOpcodeG4.AggregateRandom); + } + + public static Map OPCODE_TYPE = new HashMap<>(6); + + static { + OPCODE_TYPE.put(AggregateOpcodeG4.AggregateCountDistinct, Query.Type.INT64); + OPCODE_TYPE.put(AggregateOpcodeG4.AggregateCount, Query.Type.INT64); + OPCODE_TYPE.put(AggregateOpcodeG4.AggregateCountStar, Query.Type.INT64); + OPCODE_TYPE.put(AggregateOpcodeG4.AggregateSumDistinct, Query.Type.DECIMAL); + OPCODE_TYPE.put(AggregateOpcodeG4.AggregateSum, Query.Type.DECIMAL); + OPCODE_TYPE.put(AggregateOpcodeG4.AggregateGtid, Query.Type.VARCHAR); + } + + public static VtResultValue COUNT_ZERO = null; + + public static VtResultValue COUNT_ONE = null; + + public static VtResultValue SUM_ZERO = null; + + static { + try { + COUNT_ZERO = VtResultValue.newVtResultValue(Query.Type.INT64, 0L); + COUNT_ONE = VtResultValue.newVtResultValue(Query.Type.INT64, 1L); + SUM_ZERO = VtResultValue.newVtResultValue(Query.Type.DECIMAL, new BigDecimal(0)); + } catch (SQLException ignored) { + // unreachable + logger.error("unreachable! init zero data error!", ignored); + } + } + + /** + * PreProcess is true if one of the aggregates needs preprocessing. + */ + protected boolean preProcess; + + /** + * Aggregates specifies the aggregation parameters for each + * aggregation function: function opcode and input column number. + */ + @Getter + protected List aggregates; + + protected boolean aggrOnEngine; + + /** + * TruncateColumnCount specifies the number of columns to return + * in the final result. Rest of the columns are truncated + * from the result received. If 0, no truncation happens. + */ + @Getter + protected int truncateColumnCount; + + /** + * Collations stores the collation ID per column offset. + * It is used for grouping keys and distinct aggregate functions + */ + protected Map collations; + + /** + * Input is the primitive that will feed into this Primitive. + */ + @Getter + protected PrimitiveEngine input; + + @Override + public String getKeyspaceName() { + return input.getKeyspaceName(); + } + + @Override + public String getTableName() { + return input.getTableName(); + } + + @Override + public Boolean needsTransaction() { + return input.needsTransaction(); + } + + @Override + public void setTruncateColumnCount(Integer count) { + this.truncateColumnCount = count; + } + + @Override + public VtResultSet getFields(Vcursor vcursor, Map bindValues) throws SQLException { + VtResultSet qr = this.input.getFields(vcursor, bindValues); + Query.Field[] fields = convertFields(qr.getFields(), this.preProcess, this.aggregates, this.aggrOnEngine); + qr = new VtResultSet(); + qr.setFields(fields); + return qr.truncate(this.truncateColumnCount); + } + + protected VtResultValue findComparableCurrentDistinct(List row, AggregateParams aggr) { + VtResultValue curDistinct = row.get(aggr.keyCol); + if (aggr.wAssigned && curDistinct.isComparable()) { + aggr.keyCol = aggr.getWCol(); + curDistinct = row.get(aggr.keyCol); + } + return curDistinct; + } + + public Pair, List> merge(Query.Field[] fields, List row1, List row2, List curDistincts, + Map colls, List aggregates) throws SQLException { + List result = new ArrayList<>(row1); + for (int index = 0; index < aggregates.size(); index++) { + AggregateParams aggr = aggregates.get(index); + if (aggr.isDistinct()) { + if (row2.get(aggr.keyCol).isNull()) { + continue; + } + Integer cmp = EvalEngine.nullSafeCompare(curDistincts.get(index), row2.get(aggr.getCol())); + if (cmp == 0) { + continue; + } + curDistincts.set(index, findComparableCurrentDistinct(row2, aggr)); + } + switch (aggr.getOpcode()) { + case AggregateCountStar: + result.set(aggr.col, EvalEngine.nullSafeAdd(row1.get(aggr.col), COUNT_ONE, fields[aggr.col].getType())); + break; + case AggregateCount: + VtResultValue value = COUNT_ONE; + if (row2.get(aggr.col).isNull()) { + value = COUNT_ZERO; + } + result.set(aggr.col, EvalEngine.nullSafeAdd(row1.get(aggr.col), value, fields[aggr.col].getType())); + break; + case AggregateSum: + VtResultValue v1 = row1.get(aggr.col); + VtResultValue v2 = row2.get(aggr.col); + if (v1.isNull() && v2.isNull()) { + break; + } + result.set(aggr.col, EvalEngine.nullSafeAdd(v1, v2, fields[aggr.col].getType())); + break; + case AggregateMin: + result.set(aggr.col, EvalEngine.min(row1.get(aggr.col), row2.get(aggr.col))); + break; + case AggregateMax: + result.set(aggr.col, EvalEngine.max(row1.get(aggr.col), row2.get(aggr.col))); + break; + case AggregateCountDistinct: + result.set(aggr.col, EvalEngine.nullSafeAdd(row1.get(aggr.col), COUNT_ONE, OPCODE_TYPE.get(aggr.opcode))); + break; + case AggregateSumDistinct: + result.set(aggr.col, EvalEngine.nullSafeAdd(row1.get(aggr.col), row2.get(aggr.col), OPCODE_TYPE.get(aggr.opcode))); + break; + case AggregateRandom: + // we just grab the first value per grouping. no need to do anything more complicated here + break; + default: + throw new SQLException("BUG: Unexpected opcode: " + aggr.opcode); + } + } + return new ImmutablePair<>(result, curDistincts); + } + + protected List convertFinal(List current, List aggregates) { + List result = new ArrayList<>(current); + return result; + } + + protected Query.Field[] convertFields(Query.Field[] fields, boolean preProcess, List aggrs, boolean aggrOnEngine) { + if (!preProcess) { + return fields; + } + for (AggregateParams aggr : aggrs) { + if (!aggr.preProcess() && !aggrOnEngine) { + continue; + } + Query.Field field = Query.Field.newBuilder().setName(aggr.getAlias()).setType(OPCODE_TYPE.get(aggr.getOpcode())).build(); + fields[aggr.getCol()] = field; + if (aggr.isDistinct()) { + aggr.setKeyCol(aggr.getCol()); + } + } + return fields; + } + + protected Pair, List> convertRow(List row, boolean preProcess, List aggregates, boolean aggrOnEngine) { + if (!preProcess) { + return new ImmutablePair<>(row, null); + } + List newRow = new ArrayList<>(row); + List curDistincts = new ArrayList<>(aggregates.size()); + for (int index = 0; index < aggregates.size(); index++) { + AggregateParams aggr = aggregates.get(index); + switch (aggr.getOpcode()) { + case AggregateCountStar: + newRow.set(aggr.getCol(), COUNT_ONE); + break; + case AggregateCount: + VtResultValue val = COUNT_ONE; + if (row.get(aggr.getCol()).isNull()) { + val = COUNT_ZERO; + } + newRow.set(aggr.getCol(), val); + break; + case AggregateCountDistinct: + curDistincts.add(index, findComparableCurrentDistinct(row, aggr)); + // Type is int64. Ok to call MakeTrusted. + if (row.get(aggr.getKeyCol()).isNull()) { + newRow.set(aggr.getCol(), COUNT_ZERO); + } else { + newRow.set(aggr.getCol(), COUNT_ONE); + } + break; + case AggregateSum: + if (!aggrOnEngine) { + break; + } + if (row.get(aggr.getCol()).isNull()) { + break; + } + try { + newRow.set(aggr.getCol(), EvalEngine.cast(row.get(aggr.getCol()), OPCODE_TYPE.get(aggr.getOpcode()))); + } catch (Exception e) { + newRow.set(aggr.getCol(), SUM_ZERO); + } + break; + case AggregateSumDistinct: + curDistincts.add(index, findComparableCurrentDistinct(row, aggr)); + try { + newRow.set(aggr.getCol(), EvalEngine.cast(row.get(aggr.getCol()), OPCODE_TYPE.get(aggr.getOpcode()))); + } catch (Exception e) { + newRow.set(aggr.getCol(), SUM_ZERO); + } + break; + default: + break; + } + } + return new ImmutablePair<>(newRow, curDistincts); + } + + public static String printOpcode(AggregateOpcodeG4 code) throws SQLException { + for (Map.Entry entry : SUPPORTED_AGGREGATES.entrySet()) { + if (entry.getValue() == code) { + return entry.getKey(); + } + } + throw new SQLException("unexpect AggregateOpcodeG4"); + } + + + /** + * AggregateParams specify the parameters for each aggregation. + * It contains the opcode and input column number. + **/ + @Getter + @Setter + public static class AggregateParams { + private AggregateOpcodeG4 opcode; + + private int col; + + private String alias; + + private SQLExpr expr; + + private SQLSelectItem original; + + // These are used only for distinct opcodes. + private int keyCol; + + private int wCol; + + private boolean wAssigned; + + private Integer collationId; + + /** + * This is based on the function passed in the select expression and + * not what we use to aggregate at the engine primitive level. + */ + private AggregateOpcodeG4 origOpcode; + + public void setOrigOpcode(AggregateOpcodeG4 opcode) { + this.origOpcode = opcode == null ? AggregateOpcodeG4.AggregateUnassigned : opcode; + } + + public void setOpcode(AggregateOpcodeG4 opcode) { + this.opcode = opcode == null ? AggregateOpcodeG4.AggregateUnassigned : opcode; + } + + public AggregateParams() { + + } + + public AggregateParams(AggregateOpcodeG4 opcode, Integer col) { + this.opcode = opcode == null ? AggregateOpcodeG4.AggregateUnassigned : opcode; + this.col = col; + } + + public AggregateParams(AggregateOpcodeG4 opcode, Integer col, String alias) { + this.opcode = opcode == null ? AggregateOpcodeG4.AggregateUnassigned : opcode; + this.col = col; + this.alias = alias; + } + + public AggregateParams(AggregateOpcodeG4 opcode, Integer col, String alias, SQLExpr expr, SQLSelectItem original) { + this.opcode = opcode == null ? AggregateOpcodeG4.AggregateUnassigned : opcode; + this.col = col; + this.alias = alias; + this.expr = expr; + this.original = original; + } + + public Boolean isDistinct() { + return this.opcode == AggregateOpcodeG4.AggregateCountDistinct || this.opcode == AggregateOpcodeG4.AggregateSumDistinct; + } + + public Boolean preProcess() { + return this.opcode == AggregateOpcodeG4.AggregateCountDistinct || this.opcode == AggregateOpcodeG4.AggregateSumDistinct || + this.opcode == AggregateOpcodeG4.AggregateCount; // AggregateGtid + } + } +} diff --git a/src/main/java/com/jd/jdbc/engine/gen4/CheckCol.java b/src/main/java/com/jd/jdbc/engine/gen4/CheckCol.java new file mode 100644 index 0000000..b59c557 --- /dev/null +++ b/src/main/java/com/jd/jdbc/engine/gen4/CheckCol.java @@ -0,0 +1,39 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.engine.gen4; + +import lombok.Getter; +import lombok.Setter; + +public class CheckCol { + @Getter + private int col; + + @Setter + @Getter + private Integer wsCol; + + @Getter + private int collation; + + public CheckCol(int col, int collation) { + this.col = col; + this.collation = collation; + } +} diff --git a/src/main/java/com/jd/jdbc/engine/gen4/ConcatenateGen4Engine.java b/src/main/java/com/jd/jdbc/engine/gen4/ConcatenateGen4Engine.java new file mode 100644 index 0000000..1996a48 --- /dev/null +++ b/src/main/java/com/jd/jdbc/engine/gen4/ConcatenateGen4Engine.java @@ -0,0 +1,212 @@ +/* +Copyright 2021 JD Project Authors. Licensed under Apache-2.0. + +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 com.jd.jdbc.engine.gen4; + +import com.jd.jdbc.IExecute; +import com.jd.jdbc.context.IContext; +import com.jd.jdbc.engine.PrimitiveEngine; +import com.jd.jdbc.engine.Vcursor; +import com.jd.jdbc.sqltypes.VtResultSet; +import com.jd.jdbc.sqltypes.VtResultValue; +import com.jd.jdbc.srvtopo.BindVariable; +import com.jd.jdbc.util.threadpool.impl.VtQueryExecutorService; +import io.vitess.proto.Query; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.CountDownLatch; +import lombok.Getter; + +public class ConcatenateGen4Engine implements PrimitiveEngine { + @Getter + private final List sourceList; + + private Map noNeedToTypeCheck; + + public ConcatenateGen4Engine(List sourceList, Map noNeedToTypeCheck) { + this.sourceList = sourceList; + this.noNeedToTypeCheck = noNeedToTypeCheck; + } + + // NewConcatenate creates a Concatenate primitive. The ignoreCols slice contains the offsets that + // don't need to have the same type between sources - + // weight_string() sometimes returns VARBINARY and sometimes VARCHAR + public ConcatenateGen4Engine(List sourceList, List ignoreCols) { + Map ignoreTypes = new HashMap<>(); + if (ignoreCols != null) { + for (Integer i : ignoreCols) { + ignoreTypes.put(i, null); + } + } + this.sourceList = sourceList; + this.noNeedToTypeCheck = ignoreTypes; + } + + @Override + public String getKeyspaceName() { + Set ksSet = new LinkedHashSet<>(16, 1); + for (PrimitiveEngine source : this.sourceList) { + ksSet.add(source.getKeyspaceName()); + } + List ksList = new ArrayList<>(ksSet); + Collections.sort(ksList); + return String.join("_", ksList); + } + + @Override + public String getTableName() { + List tabList = new ArrayList<>(); + for (PrimitiveEngine source : this.sourceList) { + tabList.add(source.getTableName()); + } + return String.join("_", tabList); + } + + @Override + public IExecute.ExecuteMultiShardResponse execute(IContext ctx, Vcursor vcursor, Map bindValue, boolean wantFields) throws SQLException { + List res; + try { + res = execSources(ctx, vcursor, bindValue, wantFields); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + Query.Field[] fields = getFields(res); + long rowsAffected = 0; + List> rows = new ArrayList<>(); + for (VtResultSet r : res) { + rowsAffected += r.getRowsAffected(); + if (!rows.isEmpty() && !r.getRows().isEmpty() + && rows.get(0).size() != r.getRows().get(0).size()) { + throw new SQLException("The used SELECT statements have a different number of columns"); + } + rows.addAll(r.getRows()); + } + VtResultSet resultSet = new VtResultSet(); + resultSet.setFields(fields); + resultSet.setRows(rows); + resultSet.setRowsAffected(rowsAffected); + return new IExecute.ExecuteMultiShardResponse(resultSet); + } + + public List execSources(IContext ctx, Vcursor vcursor, Map bindVars, boolean wantfields) throws SQLException, InterruptedException { + int size = sourceList.size(); + VtResultSet[] results = new VtResultSet[size]; + CountDownLatch latch = new CountDownLatch(size); + ConcurrentLinkedQueue exceptions = new ConcurrentLinkedQueue<>(); + for (int i = 0; i < size; i++) { + final int currIndex = i; + final PrimitiveEngine currSource = sourceList.get(i); + final Map vars = bindVars == null ? null : new HashMap<>(bindVars); + VtQueryExecutorService.execute(() -> { + try { + IExecute.ExecuteMultiShardResponse result = currSource.execute(ctx, vcursor, vars, wantfields); + results[currIndex] = (VtResultSet) result.getVtRowList(); + } catch (SQLException t) { + exceptions.add(t); + } finally { + latch.countDown(); + } + }); + } + latch.await(); + if (!exceptions.isEmpty()) { + throw exceptions.peek(); + } + return Arrays.asList(results); + } + + private void compareFields(Query.Field[] firstFields, Query.Field[] secondFields) throws SQLException { + if (firstFields.length != secondFields.length) { + throw new SQLException("The used SELECT statements have a different number of columns"); + } + for (int i = 0; i < secondFields.length; i++) { + Query.Field firstField = firstFields[i]; + Query.Field secondField = secondFields[i]; + if (noNeedToTypeCheck != null && noNeedToTypeCheck.containsKey(i)) { + continue; + } + if (firstField.getType() != secondField.getType()) { + throw new SQLException( + String.format("merging field of different types is not supported, name: (%s, %s) types: (%s, %s)", firstField.getName(), secondField.getName(), firstField.getType(), + secondField.getType())); + } + } + } + + @Override + public VtResultSet getFields(Vcursor vcursor, Map bindVariableMap) throws SQLException { + VtResultSet firstQr = this.sourceList.get(0).getFields(vcursor, bindVariableMap); + for (int i = 0; i < this.sourceList.size(); i++) { + PrimitiveEngine source = this.sourceList.get(i); + if (i == 0) { + continue; + } + VtResultSet qr = source.getFields(vcursor, bindVariableMap); + this.compareFields(firstQr.getFields(), qr.getFields()); + } + return firstQr; + } + + private Query.Field[] getFields(List res) throws SQLException { + if (res == null || res.isEmpty()) { + return null; + } + Query.Field[] fields = null; + for (VtResultSet r : res) { + if (r.getFields() == null) { + continue; + } + if (fields == null) { + fields = r.getFields(); + continue; + } + + compareFields(fields, r.getFields()); + } + return fields; + } + + @Override + public Boolean canResolveShardQuery() { + return Boolean.FALSE; + } + + @Override + public Boolean needsTransaction() { + for (PrimitiveEngine source : this.sourceList) { + if (source.needsTransaction()) { + return true; + } + } + return false; + } + + @Override + public List inputs() { + return this.sourceList; + } + +} diff --git a/src/main/java/com/jd/jdbc/engine/gen4/DistinctGen4Engine.java b/src/main/java/com/jd/jdbc/engine/gen4/DistinctGen4Engine.java new file mode 100644 index 0000000..586f872 --- /dev/null +++ b/src/main/java/com/jd/jdbc/engine/gen4/DistinctGen4Engine.java @@ -0,0 +1,97 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.engine.gen4; + +import com.jd.jdbc.IExecute; +import com.jd.jdbc.context.IContext; +import com.jd.jdbc.engine.PrimitiveEngine; +import com.jd.jdbc.engine.Vcursor; +import com.jd.jdbc.sqltypes.VtResultSet; +import com.jd.jdbc.sqltypes.VtResultValue; +import com.jd.jdbc.srvtopo.BindVariable; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import lombok.Getter; + +@Getter +public class DistinctGen4Engine implements PrimitiveEngine { + private PrimitiveEngine source; + + private List checkCols; + + private boolean truncate; + + private ProbeTable probeTable; + + public DistinctGen4Engine(PrimitiveEngine source, List checkCols, boolean truncate) { + this.source = source; + this.checkCols = checkCols; + this.truncate = truncate; + } + + @Override + public String getKeyspaceName() { + return this.source.getKeyspaceName(); + } + + @Override + public String getTableName() { + return this.source.getTableName(); + } + + @Override + public IExecute.ExecuteMultiShardResponse execute(IContext ctx, Vcursor vcursor, Map bindVars, boolean wantFields) throws SQLException { + IExecute.ExecuteMultiShardResponse response = this.source.execute(ctx, vcursor, bindVars, wantFields); + VtResultSet input = (VtResultSet) response.getVtRowList(); + + VtResultSet result = new VtResultSet(); + result.setFields(input.getFields()); + + ProbeTable pt = newProbeTable(this.checkCols); + + for (List row : input.getRows()) { + boolean exists = pt.exists(row); + if (!exists) { + result.getRows().add(row); + } + } + if (this.truncate) { + return new IExecute.ExecuteMultiShardResponse(result.truncate(this.checkCols.size())); + } + return new IExecute.ExecuteMultiShardResponse(result); + } + + private ProbeTable newProbeTable(List checkCols) { + List cols = new ArrayList<>(checkCols); + return new ProbeTable(new HashMap<>(), cols); + } + + @Override + public Boolean needsTransaction() { + return this.source.needsTransaction(); + } + + @Override + public VtResultSet getFields(Vcursor vcursor, Map bindValues) throws SQLException { + return this.source.getFields(vcursor, bindValues); + } +} diff --git a/src/main/java/com/jd/jdbc/engine/gen4/FilterGen4Engine.java b/src/main/java/com/jd/jdbc/engine/gen4/FilterGen4Engine.java new file mode 100644 index 0000000..52204ce --- /dev/null +++ b/src/main/java/com/jd/jdbc/engine/gen4/FilterGen4Engine.java @@ -0,0 +1,99 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.engine.gen4; + +import com.jd.jdbc.IExecute; +import com.jd.jdbc.context.IContext; +import com.jd.jdbc.engine.PrimitiveEngine; +import com.jd.jdbc.engine.Vcursor; +import com.jd.jdbc.evalengine.EvalEngine; +import com.jd.jdbc.evalengine.EvalResult; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqltypes.VtResultSet; +import com.jd.jdbc.sqltypes.VtResultValue; +import com.jd.jdbc.srvtopo.BindVariable; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import lombok.Getter; +import lombok.Setter; + +@Getter +@Setter +public class FilterGen4Engine implements PrimitiveEngine { + + private EvalEngine.Expr predicate; + + private SQLExpr astPredicate; + + private PrimitiveEngine input; + + public FilterGen4Engine() { + } + + public FilterGen4Engine(SQLExpr astPredicate, EvalEngine.Expr predicate) { + this.astPredicate = astPredicate; + this.predicate = predicate; + } + + @Override + public String getKeyspaceName() { + return this.input.getKeyspaceName(); + } + + @Override + public String getTableName() { + return this.input.getTableName(); + } + + @Override + public IExecute.ExecuteMultiShardResponse execute(IContext ctx, Vcursor vcursor, Map bindVariableMap, boolean wantFields) throws SQLException { + IExecute.ExecuteMultiShardResponse result = this.getInput().execute(ctx, vcursor, bindVariableMap, wantFields); + if (this.predicate == null) { + return result; + } + + EvalEngine.ExpressionEnv env = new EvalEngine.ExpressionEnv(bindVariableMap); + List> newResultRows = new ArrayList<>(); + + VtResultSet resultSet = (VtResultSet) result.getVtRowList(); + for (List rv : resultSet.getRows()) { + env.setRow(rv); + EvalResult evalResult = this.predicate.evaluate(env); + long intEvalResult = evalResult.value().toInt(); + if (intEvalResult == EvalEngine.TRUE_FLAG) { + newResultRows.add(rv); + } + } + resultSet.setRows(newResultRows); + return result; + } + + @Override + public List inputs() { + return Collections.singletonList(this.input); + } + + @Override + public Boolean needsTransaction() { + return false; + } +} diff --git a/src/main/java/com/jd/jdbc/engine/gen4/GroupByParams.java b/src/main/java/com/jd/jdbc/engine/gen4/GroupByParams.java new file mode 100644 index 0000000..d6e3beb --- /dev/null +++ b/src/main/java/com/jd/jdbc/engine/gen4/GroupByParams.java @@ -0,0 +1,55 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.engine.gen4; + +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import lombok.Getter; +import lombok.Setter; + +/** + * GroupByParams specify the grouping key to be used. + */ +@Getter +@Setter +public class GroupByParams { + + private Integer keyCol; + + private Integer weightStringCol; + + private SQLExpr expr; + + private Boolean fromGroupBy; + + private int collationID; + + public GroupByParams() { + + } + + public GroupByParams(SQLExpr expr, Boolean fromGroupBy) { + this.expr = expr; + this.fromGroupBy = fromGroupBy; + } + + public GroupByParams(Integer keyCol, Integer weightStringCol) { + this.keyCol = keyCol; + this.weightStringCol = weightStringCol; + } +} diff --git a/src/main/java/com/jd/jdbc/engine/gen4/JoinGen4Engine.java b/src/main/java/com/jd/jdbc/engine/gen4/JoinGen4Engine.java new file mode 100644 index 0000000..f11ea8f --- /dev/null +++ b/src/main/java/com/jd/jdbc/engine/gen4/JoinGen4Engine.java @@ -0,0 +1,297 @@ +/* +Copyright 2021 JD Project Authors. Licensed under Apache-2.0. + +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 com.jd.jdbc.engine.gen4; + +import com.jd.jdbc.IExecute; +import com.jd.jdbc.context.IContext; +import com.jd.jdbc.engine.Engine; +import com.jd.jdbc.engine.PrimitiveEngine; +import com.jd.jdbc.engine.Vcursor; +import com.jd.jdbc.sqltypes.SqlTypes; +import com.jd.jdbc.sqltypes.VtResultSet; +import com.jd.jdbc.sqltypes.VtResultValue; +import com.jd.jdbc.sqltypes.VtRowList; +import com.jd.jdbc.sqltypes.VtStreamResultSet; +import com.jd.jdbc.srvtopo.BindVariable; +import io.vitess.proto.Query; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import lombok.Getter; +import lombok.Setter; + +@Getter +@Setter +public class JoinGen4Engine implements PrimitiveEngine { + private Engine.JoinOpcode opcode; + + /** + * Left and Right are the LHS and RHS primitives + * of the Join. They can be any primitive. + */ + private PrimitiveEngine left; + + private PrimitiveEngine right; + + /** + * Cols defines which columns from the left + * or right results should be used to build the + * return result. For results coming from the + * left query, the index values go as -1, -2, etc. + * For the right query, they're 1, 2, etc. + * If Cols is {-1, -2, 1, 2}, it means that + * the returned result will be {Left0, Left1, Right0, Right1}. + */ + private List cols; + + /** + * Vars defines the list of joinVars that need to + * be built from the LHS result before invoking + * the RHS subqquery. + */ + private Map vars; + + public JoinGen4Engine(Engine.JoinOpcode opcode, Map vars) { + this.opcode = opcode; + this.vars = vars; + this.cols = new ArrayList<>(); + } + + @Override + public String getKeyspaceName() { + if (this.left.getKeyspaceName().equalsIgnoreCase(this.right.getKeyspaceName())) { + return this.left.getKeyspaceName(); + } + return this.left.getKeyspaceName() + "_" + this.right.getKeyspaceName(); + } + + @Override + public String getTableName() { + return this.left.getTableName() + "_" + this.right.getTableName(); + } + + @Override + public IExecute.ExecuteMultiShardResponse execute(IContext ctx, Vcursor vcursor, Map bindVariableMap, boolean wantFields) throws SQLException { + Map joinVars = new LinkedHashMap<>(); + + IExecute.ExecuteMultiShardResponse leftResultResponse = this.left.execute(ctx, vcursor, bindVariableMap, wantFields); + + VtRowList leftRowList = leftResultResponse.getVtRowList(); + if (leftRowList == null) { + throw new SQLException("VtRowList is null"); + } + + VtResultSet resultSet = new VtResultSet(); + + VtResultSet leftResult = (VtResultSet) leftRowList; + if ((leftResult.getRows() == null || leftResult.getRows().isEmpty()) && wantFields) { + for (Map.Entry entry : this.vars.entrySet()) { + joinVars.put(entry.getKey(), BindVariable.NULL_BIND_VARIABLE); + } + VtResultSet rightResult = this.right.getFields(vcursor, combineVars(bindVariableMap, joinVars)); + resultSet.setFields(joinFields(leftResult.getFields(), rightResult.getFields(), this.cols)); + return new IExecute.ExecuteMultiShardResponse(resultSet); + } + + if (leftResult.getRows() != null) { + for (List leftRow : leftResult.getRows()) { + for (Map.Entry entry : this.vars.entrySet()) { + joinVars.put(entry.getKey(), SqlTypes.valueBindVariable(leftRow.get(entry.getValue()))); + } + + IExecute.ExecuteMultiShardResponse rightResultResponse = this.right.execute(ctx, vcursor, combineVars(bindVariableMap, joinVars), wantFields); + + VtRowList rightRowList = rightResultResponse.getVtRowList(); + if (rightRowList == null) { + throw new SQLException("VtRowList is null"); + } + + VtResultSet rightResult = (VtResultSet) rightRowList; + + if (wantFields) { + wantFields = false; + resultSet.setFields(joinFields(leftResult.getFields(), rightResult.getFields(), this.cols)); + } + + for (List rightRow : rightResult.getRows()) { + resultSet.getRows().add(joinRows(leftRow, rightRow, this.cols)); + } + + if (this.opcode == Engine.JoinOpcode.LeftJoin && (rightResult.getRows() == null || rightResult.getRows().isEmpty())) { + resultSet.getRows().add(joinRows(leftRow, null, this.cols)); + } + if (vcursor.exceedsMaxMemoryRows(resultSet.getRows().size())) { + throw new SQLException("in-memory row count exceeded allowed limit of " + vcursor.maxMemoryRows()); + } + } + } + return new IExecute.ExecuteMultiShardResponse(resultSet); + } + + @Override + public IExecute.VtStream streamExecute(IContext ctx, Vcursor vcursor, Map bindValue, boolean wantFields) throws SQLException { + IExecute.VtStream leftStream = this.left.streamExecute(ctx, vcursor, bindValue, wantFields); + return new IExecute.VtStream() { + private IExecute.VtStream leftStreamResult = leftStream; + + private IExecute.VtStream rightStreamResult; + + @Override + public VtRowList fetch(boolean wantFields) throws SQLException { + return this.internalFetch(wantFields); + } + + private VtRowList internalFetch(boolean wantFields) throws SQLException { + Map joinVars = new HashMap<>(); + VtResultSet resultSet = new VtResultSet(); + + VtStreamResultSet leftStreamResultSet = new VtStreamResultSet(this.leftStreamResult, wantFields); + while (leftStreamResultSet.hasNext()) { + List leftRow = leftStreamResultSet.next(); + for (Map.Entry var : vars.entrySet()) { + joinVars.put(var.getKey(), SqlTypes.valueBindVariable(leftRow.get(var.getValue()))); + } + boolean rowSent = false; + this.rightStreamResult = right.streamExecute(ctx, vcursor, combineVars(bindValue, joinVars), wantFields); + VtStreamResultSet rightStreamResultSet = new VtStreamResultSet(rightStreamResult, wantFields); + if (wantFields) { + // This code is currently unreachable because the first result + // will always be just the field info, which will cause the outer + // wantfields code path to be executed. But this may change in the future. + wantFields = false; + resultSet.setFields(joinFields(leftStreamResultSet.getFields(), rightStreamResultSet.getFields(), cols)); + } + while (rightStreamResultSet.hasNext()) { + rowSent = true; + List rightRow = rightStreamResultSet.next(); + resultSet.getRows().add(joinRows(leftRow, rightRow, cols)); + } + rightStreamResultSet.close(); + if (opcode == Engine.JoinOpcode.LeftJoin && !rowSent) { + resultSet.setRows(new ArrayList>() { + { + add(new ArrayList() {{ + addAll(joinRows(leftRow, null, cols)); + }}); + } + }); + } + } + if (wantFields) { + wantFields = false; + for (Map.Entry var : vars.entrySet()) { + joinVars.put(var.getKey(), BindVariable.NULL_BIND_VARIABLE); + } + VtResultSet rightResultSet = right.getFields(vcursor, null); + resultSet.setFields(joinFields(leftStreamResultSet.getFields(), rightResultSet.getFields(), cols)); + } + return resultSet; + } + + @Override + public void close() throws SQLException { + if (this.rightStreamResult != null) { + this.rightStreamResult.close(); + this.rightStreamResult = null; + } + if (leftStream != null) { + this.leftStreamResult.close(); + this.leftStreamResult = null; + } + } + }; + } + + @Override + public Boolean canResolveShardQuery() { + return Boolean.FALSE; + } + + @Override + public VtResultSet getFields(Vcursor vcursor, Map bindvars) throws SQLException { + Map joinVars = new HashMap<>(); + VtResultSet resultSet = new VtResultSet(); + VtResultSet leftResult = this.left.getFields(vcursor, bindvars); + for (Map.Entry var : this.vars.entrySet()) { + joinVars.put(var.getKey(), BindVariable.NULL_BIND_VARIABLE); + } + VtResultSet rightResult = this.right.getFields(vcursor, combineVars(bindvars, joinVars)); + resultSet.setFields(joinFields(leftResult.getFields(), rightResult.getFields(), this.cols)); + return resultSet; + } + + @Override + public Boolean needsTransaction() { + return this.right.needsTransaction() || this.left.needsTransaction(); + } + + @Override + public List inputs() { + return new ArrayList() {{ + add(left); + add(right); + }}; + } + + private Map combineVars(Map bindVariableMap1, Map bindVariableMap2) { + Map newBindVar = new HashMap<>(16, 1); + if (bindVariableMap1 == null) { + bindVariableMap1 = new LinkedHashMap<>(); + } + if (bindVariableMap2 == null) { + bindVariableMap2 = new LinkedHashMap<>(); + } + newBindVar.putAll(bindVariableMap1); + newBindVar.putAll(bindVariableMap2); + return newBindVar; + } + + private Query.Field[] joinFields(Query.Field[] leftFields, Query.Field[] rightFields, List cols) { + Query.Field[] fields = new Query.Field[cols.size()]; + for (int i = 0; i < cols.size(); i++) { + Integer index = cols.get(i); + if (index < 0) { + fields[i] = leftFields[-index - 1]; + continue; + } + fields[i] = rightFields[index - 1]; + } + return fields; + } + + private List joinRows(List leftRow, List rightRow, List cols) { + List row = new ArrayList<>(); + for (Integer index : cols) { + if (index < 0) { + row.add(leftRow.get(-index - 1)); + continue; + } + // right row can be null on left joins + if (rightRow != null) { + row.add(rightRow.get(index - 1)); + } else { + row.add(new VtResultValue(null, Query.Type.NULL_TYPE)); + } + } + return row; + } +} diff --git a/src/main/java/com/jd/jdbc/engine/gen4/LimitGen4Engine.java b/src/main/java/com/jd/jdbc/engine/gen4/LimitGen4Engine.java new file mode 100644 index 0000000..e81553b --- /dev/null +++ b/src/main/java/com/jd/jdbc/engine/gen4/LimitGen4Engine.java @@ -0,0 +1,152 @@ +/* +Copyright 2021 JD Project Authors. Licensed under Apache-2.0. + +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 com.jd.jdbc.engine.gen4; + +import com.jd.jdbc.IExecute; +import com.jd.jdbc.context.IContext; +import com.jd.jdbc.engine.PrimitiveEngine; +import com.jd.jdbc.engine.Vcursor; +import com.jd.jdbc.evalengine.EvalEngine; +import com.jd.jdbc.evalengine.EvalResult; +import com.jd.jdbc.planbuilder.PlanBuilder; +import com.jd.jdbc.sqlparser.support.logging.Log; +import com.jd.jdbc.sqlparser.support.logging.LogFactory; +import com.jd.jdbc.sqltypes.SqlTypes; +import com.jd.jdbc.sqltypes.VtResultSet; +import com.jd.jdbc.sqltypes.VtValue; +import com.jd.jdbc.srvtopo.BindVariable; +import java.math.BigInteger; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Map; +import lombok.Getter; +import lombok.Setter; + +@Getter +@Setter +public class LimitGen4Engine implements PrimitiveEngine { + private static final Log LOGGER = LogFactory.getLog(PlanBuilder.class); + + private EvalEngine.Expr count; + + private EvalEngine.Expr offset; + + private PrimitiveEngine input; + + private Integer fetchCount; + + private Integer fetchOffset; + + public LimitGen4Engine() { + this.input = null; + } + + public LimitGen4Engine(PrimitiveEngine input, EvalEngine.Expr count, EvalEngine.Expr offset) { + this.count = count; + this.offset = offset; + this.input = input; + } + + @Override + public String getKeyspaceName() { + return this.input.getKeyspaceName(); + } + + @Override + public String getTableName() { + return this.input.getTableName(); + } + + @Override + public IExecute.ExecuteMultiShardResponse execute(IContext ctx, Vcursor vcursor, Map bindVariableMap, boolean wantFields) throws SQLException { + Integer count = getCount(vcursor, bindVariableMap); + Integer offset = getOffset(vcursor, bindVariableMap); + + // When offset is present, we hijack the limit value so we can calculate + // the offset in memory from the result of the scatter query with count + offset. + bindVariableMap.put("__upper_limit", SqlTypes.int64BindVariable((long) (count + offset))); + + IExecute.ExecuteMultiShardResponse response = this.input.execute(ctx, vcursor, bindVariableMap, wantFields); + VtResultSet result = (VtResultSet) response.getVtRowList(); + + return getExecuteMultiShardResponse(result, count, offset); + } + + private IExecute.ExecuteMultiShardResponse getExecuteMultiShardResponse(VtResultSet result, Integer count, Integer offset) { + // There are more rows in the response than limit + offset + if (count + offset <= result.getRows().size()) { + result.setRows(result.getRows().subList(offset, count + offset)); + result.setRowsAffected(count); + return new IExecute.ExecuteMultiShardResponse(result); + } + // Remove extra rows from response + if (offset <= result.getRows().size()) { + result.setRows(result.getRows().subList(offset, result.getRows().size())); + result.setRowsAffected(result.getRows().size()); + return new IExecute.ExecuteMultiShardResponse(result); + } + // offset is beyond the result set + result.setRows(new ArrayList<>()); + result.setRowsAffected(0); + return new IExecute.ExecuteMultiShardResponse(result); + } + + @Override + public VtResultSet getFields(Vcursor vcursor, Map bindValues) throws SQLException { + return this.input.getFields(vcursor, bindValues); + } + + @Override + public Boolean canResolveShardQuery() { + return this.input.canResolveShardQuery(); + } + + @Override + public Boolean needsTransaction() { + return this.input.needsTransaction(); + } + + public Integer getCount(Vcursor vcursor, Map bindValues) throws SQLException { + EvalEngine.ExpressionEnv env = new EvalEngine.ExpressionEnv(bindValues); + return getIntFrom(env, this.count); + } + + public Integer getOffset(Vcursor vcursor, Map bindValues) throws SQLException { + EvalEngine.ExpressionEnv env = new EvalEngine.ExpressionEnv(bindValues); + return getIntFrom(env, this.offset); + } + + private Integer getIntFrom(EvalEngine.ExpressionEnv env, EvalEngine.Expr expr) throws SQLException { + if (expr == null) { + return 0; + } + EvalResult evalResult = env.evaluate(expr); + VtValue value = evalResult.value(); + if (value.isNull()) { + return 0; + } + + BigInteger num = EvalEngine.toUint64(value); + int count = num.intValue(); + if (count < 0) { + throw new SQLException("requested limit is out of range: " + value); + } + return count; + } +} diff --git a/src/main/java/com/jd/jdbc/engine/gen4/MemorySortGen4Engine.java b/src/main/java/com/jd/jdbc/engine/gen4/MemorySortGen4Engine.java new file mode 100644 index 0000000..72a2578 --- /dev/null +++ b/src/main/java/com/jd/jdbc/engine/gen4/MemorySortGen4Engine.java @@ -0,0 +1,188 @@ +/* +Copyright 2021 JD Project Authors. Licensed under Apache-2.0. + +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 com.jd.jdbc.engine.gen4; + +import com.jd.jdbc.IExecute; +import com.jd.jdbc.context.IContext; +import com.jd.jdbc.engine.PrimitiveEngine; +import com.jd.jdbc.engine.Vcursor; +import com.jd.jdbc.evalengine.EvalEngine; +import com.jd.jdbc.evalengine.EvalResult; +import com.jd.jdbc.planbuilder.Truncater; +import com.jd.jdbc.sqltypes.VtResultSet; +import com.jd.jdbc.sqltypes.VtResultValue; +import com.jd.jdbc.srvtopo.BindVariable; +import java.math.BigInteger; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.PriorityQueue; +import lombok.Getter; +import lombok.Setter; + +/** + * MemorySort is a primitive that performs in-memory sorting. + */ +@Getter +@Setter +public class MemorySortGen4Engine implements PrimitiveEngine, Truncater { + + private EvalEngine.Expr upperLimit; + + private List orderByParams = new ArrayList<>(10); + + private PrimitiveEngine input = null; + + /** + * TruncateColumnCount specifies the number of columns to return + * in the final result. Rest of the columns are truncated + * from the result received. If 0, no truncation happens. + */ + private int truncateColumnCount = 0; + + public MemorySortGen4Engine() { + } + + public MemorySortGen4Engine(List orderByParams, PrimitiveEngine input) { + this.orderByParams = orderByParams; + this.input = input; + } + + @Override + public String getKeyspaceName() { + return this.input.getKeyspaceName(); + } + + @Override + public String getTableName() { + return this.input.getTableName(); + } + + @Override + public IExecute.ExecuteMultiShardResponse execute(IContext ctx, Vcursor vcursor, Map bindVariableMap, boolean wantFields) throws SQLException { + int count = this.fetchCount(bindVariableMap); + + IExecute.ExecuteMultiShardResponse response = this.input.execute(ctx, vcursor, bindVariableMap, wantFields); + + VtResultSet resultSet = (VtResultSet) response.getVtRowList(); + return getExecuteMultiShardResponse(count, resultSet); + } + + @Override + public Boolean canResolveShardQuery() { + return this.input.canResolveShardQuery(); + } + + @Override + public VtResultSet getFields(Vcursor vcursor, Map bindValues) throws SQLException { + return this.input.getFields(vcursor, bindValues); + } + + @Override + public Boolean needsTransaction() { + return this.input.needsTransaction(); + } + + @Override + public void setTruncateColumnCount(Integer count) { + this.truncateColumnCount = count; + } + + /** + * @param bindVariableMap + * @return + * @throws SQLException + */ + private Integer fetchCount(Map bindVariableMap) throws SQLException { + if (this.upperLimit == null) { + return Integer.MAX_VALUE; + } + EvalEngine.ExpressionEnv env = new EvalEngine.ExpressionEnv(bindVariableMap); + EvalResult resolved = env.evaluate(this.upperLimit); + BigInteger num = EvalEngine.toUint64(resolved.value()); + + int count = num.intValue(); + if (count < 0) { + throw new SQLException("requested limit is out of range: " + num); + } + return count; + } + + private IExecute.ExecuteMultiShardResponse getExecuteMultiShardResponse(int count, VtResultSet resultSet) throws SQLException { + List compares = VitessCompare.extractSlices(this.orderByParams); + + SortHeapComparator sh = new SortHeapComparator(compares); + PriorityQueue> queue = new PriorityQueue<>(resultSet.getRows().size(), sh); + queue.addAll(resultSet.getRows()); + if (sh.getException() != null) { + throw sh.getException(); + } + List> rows = new ArrayList<>(resultSet.getRows().size()); + while (!queue.isEmpty()) { + List poll = queue.poll(); + rows.add(poll); + } + resultSet.setRows(rows); + if (resultSet.getRows().size() > count) { + resultSet.setRows(resultSet.getRows().subList(0, count)); + } + return new IExecute.ExecuteMultiShardResponse(resultSet.truncate(this.truncateColumnCount)); + } + + private static class SortHeapComparator implements Comparator> { + + private final List comparers; + + private final boolean reverse; + + @Getter + private SQLException exception; + + SortHeapComparator(List comparers) { + this.comparers = comparers; + this.reverse = false; + } + + @Override + public int compare(List o1, List o2) { + for (VitessCompare c : this.comparers) { + if (this.exception != null) { + return -1; + } + int cmp; + try { + cmp = c.compare(o1, o2); + } catch (SQLException e) { + this.exception = e; + return -1; + } + if (cmp == 0) { + continue; + } + if (this.reverse) { + cmp = -cmp; + } + return cmp; + } + return 0; + } + } +} diff --git a/src/main/java/com/jd/jdbc/engine/gen4/OrderByParamsGen4.java b/src/main/java/com/jd/jdbc/engine/gen4/OrderByParamsGen4.java new file mode 100644 index 0000000..97d4f30 --- /dev/null +++ b/src/main/java/com/jd/jdbc/engine/gen4/OrderByParamsGen4.java @@ -0,0 +1,57 @@ +/* +Copyright 2021 JD Project Authors. Licensed under Apache-2.0. + +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 com.jd.jdbc.engine.gen4; + +import lombok.Getter; +import lombok.Setter; + +@Getter +@Setter +public class OrderByParamsGen4 { + private int col; + + private boolean desc; + + // WeightStringCol is the weight_string column that will be used for sorting. + // It is set to -1 if such a column is not added to the query + private int weightStrCol; + + private int starColFixedIndex; + + // v3 specific boolean. Used to also add weight strings originating from GroupBys to the Group by clause + private boolean fromGroupBy; + + // Collation ID for comparison using collation + private Integer collationID; + + public OrderByParamsGen4(int col, boolean desc, int weightStringCol, int starColFixedIndex, Integer collationID) { + this.col = col; + this.desc = desc; + this.weightStrCol = weightStringCol; + this.starColFixedIndex = starColFixedIndex; + this.collationID = collationID; + } + + public OrderByParamsGen4(int col, boolean desc, int weightStrCol, Integer collationID) { + this.col = col; + this.desc = desc; + this.weightStrCol = weightStrCol; + this.collationID = collationID; + } +} diff --git a/src/main/java/com/jd/jdbc/engine/gen4/OrderedAggregateGen4Engine.java b/src/main/java/com/jd/jdbc/engine/gen4/OrderedAggregateGen4Engine.java new file mode 100644 index 0000000..1aa1a1f --- /dev/null +++ b/src/main/java/com/jd/jdbc/engine/gen4/OrderedAggregateGen4Engine.java @@ -0,0 +1,107 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.engine.gen4; + +import com.jd.jdbc.IExecute; +import com.jd.jdbc.common.tuple.Pair; +import com.jd.jdbc.context.IContext; +import com.jd.jdbc.engine.PrimitiveEngine; +import com.jd.jdbc.engine.Vcursor; +import com.jd.jdbc.evalengine.EvalEngine; +import com.jd.jdbc.sqltypes.VtResultSet; +import com.jd.jdbc.sqltypes.VtResultValue; +import com.jd.jdbc.srvtopo.BindVariable; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import lombok.Getter; + +/** + * OrderedAggregate is a primitive that expects the underlying primitive + * to feed results in an order sorted by the Keys. Rows with duplicate + * keys are aggregated using the Aggregate functions. The assumption + * is that the underlying primitive is a scatter select with pre-sorted + * rows. + */ +public class OrderedAggregateGen4Engine extends AbstractAggregateGen4 { + /** + * GroupByKeys specifies the input values that must be used for + * the aggregation key. + */ + @Getter + private List groupByKeys; + + public OrderedAggregateGen4Engine(boolean preProcess, List aggregates, boolean aggrOnEngine, int truncateColumnCount, List groupByKeys, + Map collations, PrimitiveEngine input) { + super.preProcess = preProcess; + super.aggregates = aggregates; + super.aggrOnEngine = aggrOnEngine; + super.truncateColumnCount = truncateColumnCount; + super.collations = collations; + super.input = input; + this.groupByKeys = groupByKeys; + } + + @Override + public IExecute.ExecuteMultiShardResponse execute(IContext ctx, Vcursor vcursor, Map bindVariableMap, boolean wantFields) throws SQLException { + IExecute.ExecuteMultiShardResponse resultResponse = this.input.execute(ctx, vcursor, bindVariableMap, wantFields); + VtResultSet result = (VtResultSet) resultResponse.getVtRowList(); + + VtResultSet out = new VtResultSet(convertFields(result.getFields(), super.preProcess, super.aggregates, super.aggrOnEngine), new ArrayList<>()); + + // This code is similar to the one in StreamExecute. + List current = null; + List curDistincts = null; + for (List row : result.getRows()) { + if (current == null) { + Pair, List> pair = convertRow(row, super.preProcess, super.aggregates, super.aggrOnEngine); + current = pair.getLeft(); + curDistincts = pair.getRight(); + continue; + } + boolean equal = this.keysEqual(current, row); + if (equal) { + Pair, List> pair = merge(result.getFields(), current, row, curDistincts, super.collations, super.aggregates); + current = pair.getLeft(); + curDistincts = pair.getRight(); + continue; + } + out.getRows().add(current); + Pair, List> pair = convertRow(row, super.preProcess, super.aggregates, super.aggrOnEngine); + current = pair.getLeft(); + curDistincts = pair.getRight(); + } + if (current != null) { + List finalValues = convertFinal(current, super.aggregates); + out.getRows().add(finalValues); + } + return new IExecute.ExecuteMultiShardResponse(out.truncate(super.truncateColumnCount)); + } + + private boolean keysEqual(List row1, List row2) throws SQLException { + for (GroupByParams key : this.groupByKeys) { + Integer cmp = EvalEngine.nullSafeCompare(row1.get(key.getKeyCol()), row2.get(key.getKeyCol())); + if (cmp != 0) { + return false; + } + } + return true; + } +} diff --git a/src/main/java/com/jd/jdbc/engine/gen4/ProbeTable.java b/src/main/java/com/jd/jdbc/engine/gen4/ProbeTable.java new file mode 100644 index 0000000..ce37f58 --- /dev/null +++ b/src/main/java/com/jd/jdbc/engine/gen4/ProbeTable.java @@ -0,0 +1,87 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.engine.gen4; + +import com.jd.jdbc.evalengine.EvalEngine; +import com.jd.jdbc.sqltypes.VtResultValue; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class ProbeTable { + Map>> seenRows; + + List checkCols; + + public ProbeTable(Map>> seenRows, List checkCols) { + this.seenRows = seenRows; + this.checkCols = checkCols; + } + + public boolean exists(List inputRow) throws SQLException { + // the two prime numbers used here (17 and 31) are used to + // calculate hashcode from all column values in the input sqltypes.Row + long code = hashCodeForRow(inputRow); + List> existingRows = seenRows.get(code); + if (existingRows == null) { + // nothing with this hash code found, we can be sure it's a not seen sqltypes.Row + List> lists = new ArrayList<>(); + lists.add(inputRow); + seenRows.put(code, lists); + return false; + } + + // we found something in the map - still need to check all individual values + // so we don't just fall for a hash collision + for (List existingRow : existingRows) { + boolean exists = equal(existingRow, inputRow); + if (exists) { + return true; + } + } + existingRows.add(inputRow); + return false; + } + + private boolean equal(List a, List b) throws SQLException { + for (int i = 0; i < this.checkCols.size(); i++) { + CheckCol checkCol = this.checkCols.get(i); + int cmp = EvalEngine.nullSafeCompare(a.get(i), b.get(i)); + if (cmp != 0) { + return false; + } + } + return true; + } + + private long hashCodeForRow(List inputRow) throws SQLException { + long code = 17; + for (int i = 0; i < checkCols.size(); i++) { + CheckCol checkCol = checkCols.get(i); + if (i >= inputRow.size()) { + throw new RuntimeException("Distinct check cols is larger than its input row."); + } + VtResultValue col = inputRow.get(checkCol.getCol()); + long hashcode = EvalEngine.nullsafeHashcode(col, checkCol.getCollation(), col.getVtType()); + code = code * 31 + hashcode; + } + return code; + } +} diff --git a/src/main/java/com/jd/jdbc/engine/gen4/RouteGen4Engine.java b/src/main/java/com/jd/jdbc/engine/gen4/RouteGen4Engine.java new file mode 100644 index 0000000..2cfae32 --- /dev/null +++ b/src/main/java/com/jd/jdbc/engine/gen4/RouteGen4Engine.java @@ -0,0 +1,326 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.engine.gen4; + +import com.jd.jdbc.IExecute; +import com.jd.jdbc.IExecute.ExecuteMultiShardResponse; +import com.jd.jdbc.common.tuple.Pair; +import com.jd.jdbc.common.util.CollectionUtils; +import com.jd.jdbc.context.IContext; +import com.jd.jdbc.engine.Engine; +import com.jd.jdbc.engine.PrimitiveEngine; +import com.jd.jdbc.engine.Vcursor; +import com.jd.jdbc.key.Destination; +import com.jd.jdbc.key.DestinationAnyShard; +import com.jd.jdbc.planbuilder.Truncater; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectQuery; +import com.jd.jdbc.sqltypes.SqlTypes; +import com.jd.jdbc.sqltypes.VtResultSet; +import com.jd.jdbc.sqltypes.VtResultValue; +import com.jd.jdbc.sqltypes.VtValue; +import com.jd.jdbc.srvtopo.BindVariable; +import com.jd.jdbc.srvtopo.BoundQuery; +import com.jd.jdbc.srvtopo.ResolvedShard; +import com.jd.jdbc.srvtopo.Resolver; +import com.jd.jdbc.vindexes.VKeyspace; +import io.vitess.proto.Query; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.Getter; +import lombok.Setter; + +public class RouteGen4Engine implements PrimitiveEngine, Truncater { + + /** + * Query specifies the query to be executed. + */ + @Setter + @Getter + private String query = ""; + + /** + * FieldQuery specifies the query to be executed for a GetFieldInfo request. + */ + @Setter + @Getter + private String fieldQuery = ""; + + @Setter + private SQLSelectQuery selectFieldQuery; + + @Setter + private String tableName = ""; + + @Setter + private SQLSelectQuery selectQuery; + + /** + * TruncateColumnCount specifies the number of columns to return + * in the final result. Rest of the columns are truncated + * from the result received. If 0, no truncation happens. + */ + @Getter + private int truncateColumnCount = 0; + + /** + * OrderBy specifies the key order for merge sorting. This will be + * set only for scatter queries that need the results to be + * merge-sorted. + */ + @Getter + private final List orderBy = new ArrayList<>(); + + /** + * NoRoutesSpecialHandling will make the route send a query to arbitrary shard if the routing logic can't find + * the correct shard. This is important for queries where no matches does not mean empty result - examples would be: + * select count(*) from tbl where lookupColumn = 'not there' + * select exists() + */ + @Setter + private boolean noRoutesSpecialHandling; + + @Getter + private final RoutingParameters routingParameters = new RoutingParameters(); + + public RouteGen4Engine(Engine.RouteOpcode routeOpcode, VKeyspace keyspace) { + this.routingParameters.routeOpcode = routeOpcode; + this.routingParameters.keyspace = keyspace; + } + + public RouteGen4Engine(Engine.RouteOpcode routeOpcode, VKeyspace keyspace, String query, String fieldQuery, SQLSelectQuery selectQuery) { + this.routingParameters.routeOpcode = routeOpcode; + this.routingParameters.keyspace = keyspace; + this.query = query; + this.fieldQuery = fieldQuery; + this.selectQuery = selectQuery; + } + + @Override + public String getKeyspaceName() { + return this.routingParameters.keyspace.getName(); + } + + @Override + public String getTableName() { + return this.tableName; + } + + @Override + public ExecuteMultiShardResponse execute(IContext ctx, Vcursor vcursor, Map bindVariableMap, boolean wantFields) throws SQLException { + VtResultSet vtResultSet = this.exec(vcursor, bindVariableMap, wantFields); + return new ExecuteMultiShardResponse(vtResultSet.truncate(this.truncateColumnCount)); + } + + @Override + public Boolean canResolveShardQuery() { + return true; + } + + /** + * GetFields fetches the field info. + * + * @param vcursor + * @param bindVariableMap + * @return + * @throws Exception + */ + @Override + public VtResultSet getFields(Vcursor vcursor, Map bindVariableMap) throws SQLException { + Resolver.ResolveDestinationResult resolveDestinationResult = vcursor.resolveDestinations(this.routingParameters.keyspace.getName(), null, Collections.singletonList(new DestinationAnyShard())); + if (resolveDestinationResult == null) { + throw new SQLException("no shards for keyspace: " + this.routingParameters.keyspace.getName()); + } + List resolvedShardList = resolveDestinationResult.getResolvedShards(); + if (resolvedShardList == null) { + throw new SQLException("no shards for keyspace: " + this.routingParameters.keyspace.getName()); + } + if (resolvedShardList.size() != 1) { + throw new SQLException("no shards for keyspace: " + this.routingParameters.keyspace.getName()); + } + ExecuteMultiShardResponse executeMultiShardResponse = Engine.execShard(vcursor, this.selectFieldQuery, bindVariableMap, resolvedShardList.get(0), false, false); + return ((VtResultSet) executeMultiShardResponse.getVtRowList()).truncate(this.truncateColumnCount); + } + + @Override + public Boolean needsTransaction() { + return false; + } + + private VtResultSet exec(Vcursor vcursor, Map bindVariableMap, boolean wantFields) throws SQLException { + Pair, List>> pair = this.routingParameters.findRoute(vcursor, bindVariableMap); + List rss = pair.getLeft(); + List> bvs = pair.getRight(); + + // No route + if (CollectionUtils.isEmpty(rss)) { + if (!this.noRoutesSpecialHandling) { + if (wantFields) { + return this.getFields(vcursor, new HashMap<>(16, 1)); + } + return new VtResultSet(); + } + pair = this.routingParameters.paramsAnyShard(vcursor, bindVariableMap); + rss = pair.getLeft(); + bvs = pair.getRight(); + } + List queries = Engine.getQueriesGen4(this.selectQuery, bvs, null); + ExecuteMultiShardResponse executeMultiShardResponse = vcursor.executeMultiShard(rss, queries, false, false); + + VtResultSet vtResultSet = (VtResultSet) executeMultiShardResponse.getVtRowList(); + + if (CollectionUtils.isEmpty(orderBy)) { + return vtResultSet; + } + return this.sort(vtResultSet); + } + + /** + * @param in + * @return + */ + private VtResultSet sort(VtResultSet in) throws SQLException { + // Since Result is immutable, we make a copy. + // The copy can be shallow because we won't be changing + // the contents of any row. + VtResultSet out = new VtResultSet(in.getFields(), in.getRows()); + List compares = VitessCompare.extractSlices(this.orderBy); + ResultComparator comparator = new ResultComparator(compares); + out.getRows().sort(comparator); + if (comparator.getException() != null) { + throw comparator.getException(); + } + return out; + } + + public IExecute.ExecuteMultiShardResponse executeAfterLookup(IContext ctx, Vcursor vcursor, Map bindVariableMap, + boolean wantFields, VtValue[] ids, List dest) throws SQLException { + List protoIds = new ArrayList<>(ids.length); + + for (VtValue id : ids) { + protoIds.add(SqlTypes.vtValueToProto(id)); + } + List rss = vcursor.resolveDestinations(this.routingParameters.keyspace.getName(), protoIds, dest).getResolvedShards(); + + List> bvs = new ArrayList<>(rss.size()); + for (ResolvedShard rs : rss) { + bvs.add(bindVariableMap); + } + return this.executeShards(ctx, vcursor, bindVariableMap, wantFields, rss, bvs); + + } + + private IExecute.ExecuteMultiShardResponse executeShards(IContext ctx, Vcursor vcursor, Map bindVars, + boolean wantFields, List rss, List> bvs) throws SQLException { + if (this.routingParameters.routeOpcode == Engine.RouteOpcode.SelectNext) { + // TODO + } + // No route + if (rss.size() == 0) { + if (!this.noRoutesSpecialHandling) { + if (wantFields) { + return new ExecuteMultiShardResponse(this.getFields(vcursor, bindVars)); + } + return new ExecuteMultiShardResponse(new VtResultSet()); + } + // Here we were earlier returning no rows back. + // But this was incorrect for queries like select count(*) from user where name='x' + // If the lookup_vindex for name, returns no shards, we still want a result from here + // with a single row with 0 as the output. + // However, at this level it is hard to distinguish between the cases that need a result + // and the ones that don't. So, we are sending the query to any shard! This is safe because + // the query contains a predicate that make it not match any rows on that shard. (If they did, + // we should have gotten that shard back already from findRoute) + + Pair, List>> pair = this.routingParameters.paramsAnyShard(vcursor, bindVars); + rss = pair.getLeft(); + bvs = pair.getRight(); + } + // getQuery() + String charEncoding = vcursor.getCharEncoding(); + List queries = Engine.getQueriesGen4(this.selectQuery, bvs, charEncoding); + IExecute.ExecuteMultiShardResponse results = vcursor.executeMultiShard(rss, queries, false, false); + + // TODO error process + + if (this.orderBy.size() == 0) { + return results; + } + + return new ExecuteMultiShardResponse(this.sort((VtResultSet) results.getVtRowList())); + + } + + @Override + public void setTruncateColumnCount(Integer count) { + this.truncateColumnCount = count; + } + + private static class ResultComparator implements Comparator> { + + private final List comparers; + + @Getter + private SQLException exception; + + ResultComparator(List comparers) { + this.comparers = comparers; + } + + @Override + public int compare(List o1, List o2) { + if (this.exception != null) { + return -1; + } + // If there are any errors below, the function sets + // the external err and returns true. Once err is set, + // all subsequent calls return true. This will make + // Slice think that all elements are in the correct + // order and return more quickly. + for (VitessCompare c : this.comparers) { + int cmp; + try { + cmp = c.compare(o1, o2); + } catch (SQLException e) { + this.exception = e; + return -1; + } + if (cmp == 0) { + continue; + } + return cmp; + } + return 0; + } + } + + @Data + @AllArgsConstructor + public static class ParamsResponse { + private List resolvedShardList; + + private List> shardVarList; + } +} diff --git a/src/main/java/com/jd/jdbc/engine/gen4/RoutingParameters.java b/src/main/java/com/jd/jdbc/engine/gen4/RoutingParameters.java new file mode 100644 index 0000000..8a61a15 --- /dev/null +++ b/src/main/java/com/jd/jdbc/engine/gen4/RoutingParameters.java @@ -0,0 +1,252 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.engine.gen4; + +import com.jd.jdbc.common.tuple.ImmutablePair; +import com.jd.jdbc.common.tuple.Pair; +import com.jd.jdbc.engine.Engine; +import com.jd.jdbc.engine.Vcursor; +import com.jd.jdbc.evalengine.EvalEngine; +import com.jd.jdbc.evalengine.EvalResult; +import com.jd.jdbc.key.Destination; +import com.jd.jdbc.key.DestinationAllShard; +import com.jd.jdbc.key.DestinationAnyShard; +import com.jd.jdbc.planbuilder.RoutePlan; +import com.jd.jdbc.sqltypes.SqlTypes; +import com.jd.jdbc.sqltypes.VtValue; +import com.jd.jdbc.srvtopo.BindVariable; +import com.jd.jdbc.srvtopo.ResolvedShard; +import com.jd.jdbc.srvtopo.Resolver; +import com.jd.jdbc.vindexes.MultiColumn; +import com.jd.jdbc.vindexes.SingleColumn; +import com.jd.jdbc.vindexes.VKeyspace; +import io.netty.util.internal.StringUtil; +import io.vitess.proto.Query; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import lombok.Data; + +@Data +public class RoutingParameters { + /** + * RouteOpcode is a number representing the opcode + * for the Route primitve + */ + protected Engine.RouteOpcode routeOpcode; + + protected VKeyspace keyspace; + + private List systableTableSchema = new ArrayList<>(); + + private Map systableTableName = new HashMap<>(); + + /** + * TargetDestination specifies an explicit target destination to send the query to. + * This bypases the core of the v3 engine. + */ + private Destination targetDestination; + + /** + * Vindex specifies the vindex to be used. + */ + private SingleColumn vindex; + + private List values = new ArrayList<>(); + + protected boolean isQueryPinnedTable; + + protected Pair, List>> findRoute(Vcursor vcursor, Map bindVariableMap) throws SQLException { + RouteGen4Engine.ParamsResponse paramsResponse = null; + switch (routeOpcode) { + case SelectDBA: + paramsResponse = this.systemQuery(vcursor, bindVariableMap); + break; + case SelectUnsharded: + paramsResponse = this.paramsAllShard(vcursor, bindVariableMap); + break; + case SelectReference: + return this.paramsAnyShard(vcursor, bindVariableMap); + case SelectScatter: + case SelectNext: + paramsResponse = this.paramsAllShard(vcursor, bindVariableMap); + break; + case SelectEqual: + case SelectEqualUnique: + if (this.vindex instanceof MultiColumn) { + } else { + paramsResponse = this.paramsSelectEqual(vcursor, bindVariableMap); + } + break; + case SelectIN: + if (this.vindex instanceof MultiColumn) { + } else { + paramsResponse = this.paramsSelectIn(vcursor, bindVariableMap); + } + break; + case SelectNone: + paramsResponse = new RouteGen4Engine.ParamsResponse(new ArrayList<>(), new ArrayList<>()); + break; + default: + // Unreachable. + throw new SQLException("unsupported query route: " + routeOpcode); + } + return new ImmutablePair<>(paramsResponse.getResolvedShardList(), paramsResponse.getShardVarList()); + } + + public RouteGen4Engine.ParamsResponse systemQuery(Vcursor vcursor, Map bindVariableMap) throws SQLException { + if (this.systableTableName.size() == 0 && this.systableTableSchema.size() == 0) { + return new RouteGen4Engine.ParamsResponse(defaultRoute(vcursor), new ArrayList>() {{ + add(bindVariableMap); + }}); + } + + String specifiedKS = ""; + boolean schemaExists = false; + EvalEngine.ExpressionEnv env = new EvalEngine.ExpressionEnv(bindVariableMap); + for (EvalEngine.Expr expr : this.systableTableSchema) { + EvalResult result = env.evaluate(expr); + String ks = result.value().toString(); + if (StringUtil.isNullOrEmpty(specifiedKS)) { + specifiedKS = ks; + } else if (!ks.equalsIgnoreCase(specifiedKS)) { + throw new SQLException("can't use more than one keyspace per system table query - found both '" + specifiedKS + "' and '" + ks + "'"); + } + } + + if (!StringUtil.isNullOrEmpty(specifiedKS)) { + bindVariableMap.put(SqlTypes.BV_SCHEMA_NAME, SqlTypes.stringBindVariable(specifiedKS)); + } + Map tableNames = new HashMap(); + for (Map.Entry entry : this.systableTableName.entrySet()) { + String tblBvName = entry.getKey(); + EvalResult val = env.evaluate(entry.getValue()); + String tabName = val.value().toString(); + tableNames.put(tblBvName, tabName); + bindVariableMap.put(tblBvName, SqlTypes.stringBindVariable(tabName)); + } + + // if the table_schema is system schema, route to default keyspace. + if (RoutePlan.systemTable(specifiedKS)) { + return new RouteGen4Engine.ParamsResponse(defaultRoute(vcursor), new ArrayList>() {{ + add(bindVariableMap); + }}); + } + // the use has specified a table_name - let's check if it's a routed table + if (tableNames.size() > 0) { + + } + List destinations; + try { + Resolver.ResolveDestinationResult resolveDestinationResult = vcursor.resolveDestinations( + specifiedKS, null, new ArrayList() {{ + add(new DestinationAnyShard()); + }}); + destinations = resolveDestinationResult.getResolvedShards(); + if (schemaExists) { + bindVariableMap.put(SqlTypes.BV_REPLACE_SCHEMA_NAME, SqlTypes.int64BindVariable(1L)); + } + } catch (Exception e) { + try { + Resolver.ResolveDestinationResult resolveDestinationResult = vcursor.resolveDestinations( + this.keyspace.getName(), null, new ArrayList() {{ + add(new DestinationAnyShard()); + }}); + destinations = resolveDestinationResult.getResolvedShards(); + } catch (Exception e1) { + throw new SQLException("failed to find information about keyspace `" + keyspace + "`"); + } + } +// return destinations; + return new RouteGen4Engine.ParamsResponse(destinations, new ArrayList>() {{ + add(bindVariableMap); + }}); + } + + public List defaultRoute(Vcursor vcursor) throws SQLException { + Resolver.ResolveDestinationResult resolveDestinationResult = null; + try { + resolveDestinationResult = vcursor.resolveDestinations(this.keyspace.getName(), null, new ArrayList() {{ + add(new DestinationAnyShard()); + }}); + } catch (SQLException e) { + throw new SQLException("failed to find information about keyspace `" + keyspace.getName() + "`"); + } + return resolveDestinationResult.getResolvedShards(); + + } + + /** + * @param vcursor + * @return + * @throws Exception + */ + protected Pair, List>> paramsAnyShard(Vcursor vcursor, Map bindVariableMap) throws SQLException { + Resolver.ResolveDestinationResult resolveDestinationResult = vcursor.resolveDestinations(this.keyspace.getName(), null, Collections.singletonList(new DestinationAnyShard())); + List rss = resolveDestinationResult.getResolvedShards(); + return new ImmutablePair<>(rss, IntStream.range(0, rss.size()).mapToObj(i -> bindVariableMap).collect(Collectors.toList())); + } + + /** + * @param vcursor + * @return + * @throws Exception + */ + private RouteGen4Engine.ParamsResponse paramsAllShard(Vcursor vcursor, Map bindVariableMap) throws SQLException { + Resolver.ResolveDestinationResult resolveDestinationResult = vcursor.resolveDestinations(this.keyspace.getName(), null, Collections.singletonList(new DestinationAllShard())); + List rss = resolveDestinationResult.getResolvedShards(); + return new RouteGen4Engine.ParamsResponse(rss, IntStream.range(0, rss.size()).mapToObj(i -> bindVariableMap).collect(Collectors.toList())); + } + + /** + * @param vcursor + * @return + * @throws Exception + */ + private RouteGen4Engine.ParamsResponse paramsSelectEqual(Vcursor vcursor, Map bindVariableMap) throws SQLException { + EvalEngine.ExpressionEnv env = new EvalEngine.ExpressionEnv(bindVariableMap); + EvalResult value = env.evaluate(this.values.get(0)); + Resolver.ResolveDestinationResult resolveDestinationResult = Engine.resolveShards(vcursor, this.vindex, + this.keyspace, new ArrayList() {{ + add(value.value()); + }}); + List rss = resolveDestinationResult.getResolvedShards(); + return new RouteGen4Engine.ParamsResponse(rss, IntStream.range(0, rss.size()).mapToObj(i -> bindVariableMap).collect(Collectors.toList())); + } + + /** + * @param vcursor + * @return + * @throws Exception + */ + private RouteGen4Engine.ParamsResponse paramsSelectIn(Vcursor vcursor, Map bindVariableMap) throws SQLException { + EvalEngine.ExpressionEnv env = new EvalEngine.ExpressionEnv(bindVariableMap); + EvalResult value = env.evaluate(this.values.get(0)); + Resolver.ResolveDestinationResult resolveDestinationResult = Engine.resolveShards(vcursor, this.vindex, this.keyspace, value.tupleValues()); + List rss = resolveDestinationResult.getResolvedShards(); + List> values = resolveDestinationResult.getValues(); + return new RouteGen4Engine.ParamsResponse(rss, Engine.shardVars(bindVariableMap, values)); + } + +} diff --git a/src/main/java/com/jd/jdbc/engine/gen4/ScalarAggregateGen4Engine.java b/src/main/java/com/jd/jdbc/engine/gen4/ScalarAggregateGen4Engine.java new file mode 100644 index 0000000..8809462 --- /dev/null +++ b/src/main/java/com/jd/jdbc/engine/gen4/ScalarAggregateGen4Engine.java @@ -0,0 +1,120 @@ +/* +Copyright 2022 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 com.jd.jdbc.engine.gen4; + +import com.jd.jdbc.IExecute; +import com.jd.jdbc.common.tuple.Pair; +import com.jd.jdbc.context.IContext; +import com.jd.jdbc.engine.Engine.AggregateOpcodeG4; +import com.jd.jdbc.engine.PrimitiveEngine; +import com.jd.jdbc.engine.Vcursor; +import com.jd.jdbc.sqltypes.VtResultSet; +import com.jd.jdbc.sqltypes.VtResultValue; +import com.jd.jdbc.srvtopo.BindVariable; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * ScalarAggregate is a primitive used to do aggregations without grouping keys + */ +public class ScalarAggregateGen4Engine extends AbstractAggregateGen4 { + + public ScalarAggregateGen4Engine(boolean preProcess, List aggregates, boolean aggrOnEngine, Integer truncateColumnCount, Map collations, + PrimitiveEngine input) { + super.preProcess = preProcess; + super.aggregates = aggregates; + super.aggrOnEngine = aggrOnEngine; + super.truncateColumnCount = truncateColumnCount; + super.collations = collations; + super.input = input; + } + + public ScalarAggregateGen4Engine(boolean preProcess, List aggregates, PrimitiveEngine input) { + super.preProcess = preProcess; + super.aggregates = aggregates; + super.input = input; + } + + @Override + public IExecute.ExecuteMultiShardResponse execute(IContext ctx, Vcursor cursor, Map bindVariableMap, boolean wantFields) throws SQLException { + IExecute.ExecuteMultiShardResponse resultResponse = this.input.execute(ctx, cursor, bindVariableMap, wantFields); + VtResultSet queryResult = getExecuteMultiShardResponse((VtResultSet) resultResponse.getVtRowList()); + return new IExecute.ExecuteMultiShardResponse(queryResult.truncate(this.truncateColumnCount)); + } + + private VtResultSet getExecuteMultiShardResponse(VtResultSet result) throws SQLException { + VtResultSet out = new VtResultSet(convertFields(result.getFields(), this.preProcess, this.aggregates, this.aggrOnEngine), new ArrayList<>()); + List resultRow = null; + List curDistinct = null; + for (List row : result.getRows()) { + if (resultRow == null) { + Pair, List> listListPair = convertRow(row, this.preProcess, this.aggregates, this.aggrOnEngine); + resultRow = listListPair.getLeft(); + curDistinct = listListPair.getRight(); + continue; + } + Pair, List> merge = merge(result.getFields(), resultRow, row, curDistinct, collations, aggregates); + resultRow = merge.getLeft(); + curDistinct = merge.getRight(); + } + if (resultRow == null) { + // When doing aggregation without grouping keys, we need to produce a single row containing zero-value for the + // different aggregation functions + resultRow = createEmptyRow(); + } else { + resultRow = convertFinal(resultRow, this.aggregates); + } + out.getRows().add(resultRow); + return out; + } + + /** + * creates the empty row for the case when we are missing grouping keys and have empty input table + * + * @return + */ + private List createEmptyRow() throws SQLException { + List out = new ArrayList<>(); + for (AggregateParams aggr : this.aggregates) { + AggregateOpcodeG4 op = aggr.getOpcode(); + if (aggr.getOrigOpcode() != AggregateOpcodeG4.AggregateUnassigned) { + op = aggr.getOrigOpcode(); + } + VtResultValue value = createEmptyValueFor(op); + out.add(value); + } + return out; + } + + private VtResultValue createEmptyValueFor(AggregateOpcodeG4 opcode) throws SQLException { + switch (opcode) { + case AggregateCountDistinct: + case AggregateCount: + case AggregateCountStar: + return COUNT_ZERO; + case AggregateSumDistinct: + case AggregateSum: + case AggregateMin: + case AggregateMax: + return VtResultValue.NULL; + default: + throw new SQLException("unknown aggregation " + opcode); + } + } +} diff --git a/src/main/java/com/jd/jdbc/engine/gen4/SimpleConverterLookup.java b/src/main/java/com/jd/jdbc/engine/gen4/SimpleConverterLookup.java new file mode 100644 index 0000000..94c2c32 --- /dev/null +++ b/src/main/java/com/jd/jdbc/engine/gen4/SimpleConverterLookup.java @@ -0,0 +1,72 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.engine.gen4; + +import com.jd.jdbc.common.tuple.Pair; +import com.jd.jdbc.context.PlanningContext; +import com.jd.jdbc.evalengine.TranslationLookup; +import com.jd.jdbc.planbuilder.gen4.ProjectionPushing; +import com.jd.jdbc.planbuilder.gen4.logical.LogicalPlan; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLName; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import java.sql.SQLException; +import lombok.Getter; +import lombok.Setter; + +@Getter +@Setter +public class SimpleConverterLookup implements TranslationLookup { + + private PlanningContext ctx; + + private LogicalPlan plan; + + private boolean canPushProjection; + + public SimpleConverterLookup(PlanningContext ctx, LogicalPlan plan) { + this.ctx = ctx; + this.plan = plan; + } + + public SimpleConverterLookup(PlanningContext ctx, LogicalPlan plan, boolean canPushProjection) { + this.ctx = ctx; + this.plan = plan; + this.canPushProjection = canPushProjection; + } + + @Override + public int columnLookup(SQLName col) throws SQLException { + Pair result = ProjectionPushing.pushProjection(ctx, new SQLSelectItem(col), plan, true, true, false); + if (result.getRight() == 1 && !this.canPushProjection) { + throw new SQLException("column should not be pushed to projection while doing a column lookup"); + } + return result.getLeft(); + } + + @Override + public int collationForExpr(SQLExpr expr) { + return this.getCtx().getSemTable().collationForExpr(expr); + } + + @Override + public int defaultCollation() { + return this.getCtx().getSemTable().getCollation(); + } +} diff --git a/src/main/java/com/jd/jdbc/engine/gen4/SimpleProjectionGen4Engine.java b/src/main/java/com/jd/jdbc/engine/gen4/SimpleProjectionGen4Engine.java new file mode 100644 index 0000000..65e5428 --- /dev/null +++ b/src/main/java/com/jd/jdbc/engine/gen4/SimpleProjectionGen4Engine.java @@ -0,0 +1,112 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.engine.gen4; + +import com.jd.jdbc.IExecute; +import com.jd.jdbc.context.IContext; +import com.jd.jdbc.engine.PrimitiveEngine; +import com.jd.jdbc.engine.Vcursor; +import com.jd.jdbc.sqltypes.VtResultSet; +import com.jd.jdbc.sqltypes.VtResultValue; +import com.jd.jdbc.srvtopo.BindVariable; +import io.vitess.proto.Query; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import lombok.Getter; +import lombok.Setter; + +// SimpleProjection selects which columns to keep from the input +@Getter +@Setter +public class SimpleProjectionGen4Engine implements PrimitiveEngine { + // Cols defines the column numbers from the underlying primitive + // to be returned. + private List cols = new ArrayList<>(); + + private PrimitiveEngine input; + + + @Override + public String getKeyspaceName() { + return this.getInput().getKeyspaceName(); + } + + @Override + public String getTableName() { + return this.getInput().getTableName(); + } + + @Override + public IExecute.ExecuteMultiShardResponse execute(IContext ctx, Vcursor vcursor, Map bindVariableMap, boolean wantFields) throws SQLException { + IExecute.ExecuteMultiShardResponse inner = this.getInput().execute(ctx, vcursor, bindVariableMap, wantFields); + return new IExecute.ExecuteMultiShardResponse(this.buildResult((VtResultSet) inner.getVtRowList())); + } + + @Override + public Boolean needsTransaction() { + return this.getInput().needsTransaction(); + } + + @Override + public VtResultSet getFields(Vcursor vcursor, Map bindvars) throws SQLException { + VtResultSet inner = this.input.getFields(vcursor, bindvars); + VtResultSet ret = new VtResultSet(); + ret.setFields(this.buildFields(inner).toArray(new Query.Field[0])); + return ret; + } + + @Override + public List inputs() { + return Arrays.asList(this.input); + } + + + private VtResultSet buildResult(VtResultSet inner) { + VtResultSet newInner = new VtResultSet(); + newInner.setFields(this.buildFields(inner).toArray(new Query.Field[0])); + + List> rows = new ArrayList<>(inner.getRows().size()); + + for (List innerRow : inner.getRows()) { + List newRow = new ArrayList<>(this.cols.size()); + for (int col : this.cols) { + newRow.add(innerRow.get(col)); + } + rows.add(newRow); + } + newInner.setRows(rows); + newInner.setRowsAffected(inner.getRowsAffected()); + return newInner; + } + + private List buildFields(VtResultSet inner) { + if (inner.getFields().length == 0) { + return null; + } + List fields = new ArrayList<>(inner.getFields().length); + + for (int col : this.cols) { + fields.add(inner.getFields()[col]); + } + return fields; + } +} diff --git a/src/main/java/com/jd/jdbc/engine/gen4/VindexLookup.java b/src/main/java/com/jd/jdbc/engine/gen4/VindexLookup.java new file mode 100644 index 0000000..7a7d08a --- /dev/null +++ b/src/main/java/com/jd/jdbc/engine/gen4/VindexLookup.java @@ -0,0 +1,177 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.engine.gen4; + +import com.jd.jdbc.IExecute; +import com.jd.jdbc.context.IContext; +import com.jd.jdbc.engine.Engine; +import com.jd.jdbc.engine.PrimitiveEngine; +import com.jd.jdbc.engine.Vcursor; +import com.jd.jdbc.evalengine.EvalEngine; +import com.jd.jdbc.evalengine.EvalResult; +import com.jd.jdbc.key.Destination; +import com.jd.jdbc.sqltypes.SqlTypes; +import com.jd.jdbc.sqltypes.VtResultSet; +import com.jd.jdbc.sqltypes.VtValue; +import com.jd.jdbc.srvtopo.BindVariable; +import com.jd.jdbc.vindexes.LookupPlanable; +import com.jd.jdbc.vindexes.VKeyspace; +import io.vitess.proto.Query; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class VindexLookup implements PrimitiveEngine { + + Engine.RouteOpcode opcode; + + /** + * The vindex to use to do the Map + */ + LookupPlanable vindex; + + /** + * Keyspace specifies the keyspace to send the query to. + */ + VKeyspace keyspace; + + List arguments; + + /** + * Values specifies the vindex values to use for routing. + */ + List values; + + /** + * We fetch data in order to do the map from this primitive + */ + PrimitiveEngine lookup; + + /** + * This is the side that needs to be routed + */ + RouteGen4Engine sendTo; + + public VindexLookup(Engine.RouteOpcode opcode, LookupPlanable vindex, VKeyspace keyspace, + List values, RouteGen4Engine sendTo, List arguments, + PrimitiveEngine lookup) { + this.opcode = opcode; + this.vindex = vindex; + this.keyspace = keyspace; + this.values = values; + this.sendTo = sendTo; + this.arguments = arguments; + this.lookup = lookup; + } + + @Override + public String getKeyspaceName() { + return this.sendTo.getKeyspaceName(); + } + + @Override + public String getTableName() { + return this.sendTo.getTableName(); + } + + public VtResultSet getFields(Vcursor vcursor, Map bindVariableMap) throws SQLException { + return this.sendTo.getFields(vcursor, bindVariableMap); + } + + @Override + public IExecute.ExecuteMultiShardResponse execute(IContext ctx, Vcursor vcursor, Map bindVariableMap, boolean wantFields) throws SQLException { + VtValue[] ids = this.generateIds(vcursor, bindVariableMap); + VtResultSet[] results = this.lookUpFunc(ctx, vcursor, ids); + List dest = this.mapVindexToDestination(ids, results, bindVariableMap); + + return this.sendTo.executeAfterLookup(ctx, vcursor, bindVariableMap, wantFields, ids, dest); + } + + @Override + public Boolean needsTransaction() { + return this.sendTo.needsTransaction(); + } + + private VtValue[] generateIds(Vcursor vcursor, Map bindVariableMap) throws SQLException { + // vcursor. + EvalEngine.ExpressionEnv env = new EvalEngine.ExpressionEnv(bindVariableMap); + EvalEngine.Expr expr = this.values.get(0); + EvalResult value = expr.evaluate(env); + if (this.opcode == Engine.RouteOpcode.SelectEqual || this.opcode == Engine.RouteOpcode.SelectEqualUnique) { + return new VtValue[] {value.value()}; + } else if (this.opcode == Engine.RouteOpcode.SelectIN) { + // return res.resultValue() + //TODO + } + + return null; + } + + private VtResultSet[] lookUpFunc(IContext ctx, Vcursor vcursor, VtValue[] ids) throws SQLException { + if (ids[0].isIntegral() || this.vindex.AllowBatch()) { + return this.executeBatch(ctx, vcursor, ids); + } + return this.executeNonBatch(ctx, vcursor, ids); + } + + private VtResultSet[] executeNonBatch(IContext ctx, Vcursor vcursor, VtValue[] ids) throws SQLException { + ArrayList results = new ArrayList<>(ids.length); + for (VtValue id : ids) { + Map bindVariableMap = new HashMap<>(); + bindVariableMap.put(this.arguments.get(0), SqlTypes.valueBindVariable(id)); + IExecute.ExecuteMultiShardResponse executeMultiShardResponse = this.lookup.execute(ctx, vcursor, bindVariableMap, false); + VtResultSet vtResultSet = (VtResultSet) executeMultiShardResponse.getVtRowList(); + results.add(vtResultSet); + } + return results.toArray(new VtResultSet[0]); + } + + private VtResultSet[] executeBatch(IContext ctx, Vcursor vcursor, VtValue[] ids) throws SQLException { + ArrayList results = new ArrayList<>(ids.length); + Map bindVariableMap = new HashMap<>(); + + List qv = new ArrayList<>(ids.length); + + for (VtValue id : ids) { + qv.add(id.toQueryValue()); + } + bindVariableMap.put(this.arguments.get(0), new BindVariable(qv, Query.Type.TUPLE)); + IExecute.ExecuteMultiShardResponse executeMultiShardResponse = this.lookup.execute(ctx, vcursor, bindVariableMap, false); + VtResultSet vtResultSet = (VtResultSet) executeMultiShardResponse.getVtRowList(); + //TODO 结果处理 多列转多行 + results.add(vtResultSet); + return results.toArray(new VtResultSet[0]); + } + + private List mapVindexToDestination(VtValue[] ids, VtResultSet[] results, Map bindVariableMap) { + Destination[] dest = this.vindex.MapResult(ids, results); + if (this.opcode == Engine.RouteOpcode.SelectIN) { + List valuesList = new ArrayList<>(ids.length); + for (VtValue id : ids) { + valuesList.add(SqlTypes.vtValueToProto(id)); + } + BindVariable valsBv = new BindVariable(valuesList, Query.Type.TUPLE); + bindVariableMap.put(Engine.LIST_VAR_NAME, valsBv); + } + return Arrays.asList(dest); + } +} diff --git a/src/main/java/com/jd/jdbc/engine/gen4/VitessCompare.java b/src/main/java/com/jd/jdbc/engine/gen4/VitessCompare.java new file mode 100644 index 0000000..b6d779a --- /dev/null +++ b/src/main/java/com/jd/jdbc/engine/gen4/VitessCompare.java @@ -0,0 +1,83 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.engine.gen4; + +import com.jd.jdbc.evalengine.EvalEngine; +import com.jd.jdbc.sqltypes.VtResultValue; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import lombok.Getter; + +public class VitessCompare { + private int orderBy; + + private int weightString; + + private final int starColFixedIndex; + + // collationID collations.ID + + @Getter + private final boolean desc; + + public VitessCompare(int orderBy, int weightString, boolean desc, int starColFixedIndex) { + this.orderBy = orderBy; + this.weightString = weightString; + this.desc = desc; + this.starColFixedIndex = starColFixedIndex; + } + + public int compare(List r1, List r2) throws SQLException { + int colIndex; + if (this.starColFixedIndex > this.orderBy && this.starColFixedIndex < r1.size()) { + colIndex = this.starColFixedIndex; + } else { + colIndex = this.orderBy; + } + int cmp = 0; + if (this.weightString != -1) { + // in case of a comparison or collation error switch to using the weight string column for ordering + this.orderBy = this.weightString; + this.weightString = -1; + cmp = EvalEngine.nullSafeCompare(r1.get(this.orderBy), r2.get(this.orderBy)); + } else { + cmp = EvalEngine.nullSafeCompare(r1.get(colIndex), r2.get(colIndex)); + } + // change the result if descending ordering is required + if (this.desc) { + cmp = -cmp; + } + return cmp; + } + + /** + * extractSlices extracts the three fields of OrderByParams into a slice of comparers + * @param orderBy + * @return + */ + public static List extractSlices(List orderBy) { + List compares = new ArrayList<>(orderBy.size()); + for (OrderByParamsGen4 order : orderBy) { + VitessCompare vitessCompare = new VitessCompare(order.getCol(), order.getWeightStrCol(), order.isDesc(), order.getStarColFixedIndex()); + compares.add(vitessCompare); + } + return compares; + } +} diff --git a/src/main/java/com/jd/jdbc/evalengine/Arithmetic.java b/src/main/java/com/jd/jdbc/evalengine/Arithmetic.java index 8658a67..c9470ce 100644 --- a/src/main/java/com/jd/jdbc/evalengine/Arithmetic.java +++ b/src/main/java/com/jd/jdbc/evalengine/Arithmetic.java @@ -22,17 +22,16 @@ import com.jd.jdbc.sqltypes.VtType; import com.jd.jdbc.sqltypes.VtValue; import io.vitess.proto.Query; +import static io.vitess.proto.Query.Type.DECIMAL; +import static io.vitess.proto.Query.Type.FLOAT64; +import static io.vitess.proto.Query.Type.INT64; +import static io.vitess.proto.Query.Type.UINT64; +import java.math.BigDecimal; import java.math.BigInteger; import java.sql.SQLException; import java.text.DecimalFormat; import java.util.ArrayList; import java.util.List; -import java.math.BigDecimal; - -import static io.vitess.proto.Query.Type.DECIMAL; -import static io.vitess.proto.Query.Type.FLOAT64; -import static io.vitess.proto.Query.Type.INT64; -import static io.vitess.proto.Query.Type.UINT64; public class Arithmetic { @@ -41,24 +40,30 @@ public class Arithmetic { * @return * @throws SQLException */ - static EvalEngine.EvalResult newIntegralNumeric(VtValue v) throws SQLException { + static EvalResult newIntegralNumeric(VtValue v) throws SQLException { String str = v.toString(); if (v.isSigned()) { long ival = Long.parseLong(str, 10); - return new EvalEngine.EvalResult(ival, Query.Type.INT64); + return new EvalResult(ival, Query.Type.INT64); } else if (v.isUnsigned()) { long uval = Long.parseUnsignedLong(str, 10); - return new EvalEngine.EvalResult(uval, Query.Type.UINT64); + if (uval < 0) { + return new EvalResult(new BigInteger(str), Query.Type.UINT64); + } + return new EvalResult(BigInteger.valueOf(uval), Query.Type.UINT64); } // For other types, do best effort. try { long ival = Long.parseLong(str, 10); - return new EvalEngine.EvalResult(ival, Query.Type.INT64); + return new EvalResult(ival, Query.Type.INT64); } catch (NumberFormatException e) { try { long uval = Long.parseUnsignedLong(str, 10); - return new EvalEngine.EvalResult(uval, Query.Type.UINT64); + if (uval < 0) { + return new EvalResult(new BigInteger(str), Query.Type.UINT64); + } + return new EvalResult(BigInteger.valueOf(uval), Query.Type.UINT64); } catch (NumberFormatException ee) { throw new SQLException("could not parse value: '" + str + "'"); } @@ -70,7 +75,7 @@ static EvalEngine.EvalResult newIntegralNumeric(VtValue v) throws SQLException { * @param value2 * @return */ - static Integer compareNumeric(EvalEngine.EvalResult value1, EvalEngine.EvalResult value2) { + static Integer compareNumeric(EvalResult value1, EvalResult value2) { // Equalize the types. switch (value1.getType()) { @@ -80,10 +85,10 @@ static Integer compareNumeric(EvalEngine.EvalResult value1, EvalEngine.EvalResul if (value1.getIval() < 0) { return -1; } - value1 = new EvalEngine.EvalResult(BigInteger.valueOf(value1.getIval()), Query.Type.UINT64); + value1 = new EvalResult(BigInteger.valueOf(value1.getIval()), Query.Type.UINT64); break; case FLOAT64: - value1 = new EvalEngine.EvalResult(value1.getIval().doubleValue(), Query.Type.FLOAT64); + value1 = new EvalResult(value1.getIval().doubleValue(), Query.Type.FLOAT64); break; } break; @@ -93,20 +98,20 @@ static Integer compareNumeric(EvalEngine.EvalResult value1, EvalEngine.EvalResul if (value2.getIval() < 0) { return 1; } - value2 = new EvalEngine.EvalResult(BigInteger.valueOf(value2.getIval()), Query.Type.UINT64); + value2 = new EvalResult(BigInteger.valueOf(value2.getIval()), Query.Type.UINT64); break; case FLOAT64: - value1 = new EvalEngine.EvalResult(value1.getUval().doubleValue(), Query.Type.FLOAT64); + value1 = new EvalResult(value1.getUval().doubleValue(), Query.Type.FLOAT64); break; } break; case FLOAT64: switch (value2.getType()) { case INT64: - value2 = new EvalEngine.EvalResult(value2.getIval().doubleValue(), FLOAT64); + value2 = new EvalResult(value2.getIval().doubleValue(), FLOAT64); break; case UINT64: - value2 = new EvalEngine.EvalResult(value2.getUval().doubleValue(), FLOAT64); + value2 = new EvalResult(value2.getUval().doubleValue(), FLOAT64); break; } break; @@ -130,8 +135,8 @@ static Integer compareNumeric(EvalEngine.EvalResult value1, EvalEngine.EvalResul * @param value2 * @return */ - static EvalEngine.EvalResult addNumeric(EvalEngine.EvalResult value1, EvalEngine.EvalResult value2) throws SQLException { - List resultValues = makeNumericAndPrioritize(value1, value2); + static EvalResult addNumeric(EvalResult value1, EvalResult value2) throws SQLException { + List resultValues = makeNumericAndPrioritize(value1, value2); value1 = resultValues.get(0); value2 = resultValues.get(1); switch (value1.getType()) { @@ -151,8 +156,8 @@ static EvalEngine.EvalResult addNumeric(EvalEngine.EvalResult value1, EvalEngine throw new SQLException("unreachable"); } - static EvalEngine.EvalResult addNumericWithError(EvalEngine.EvalResult value1, EvalEngine.EvalResult value2) throws SQLException { - List resultValues = makeNumericAndPrioritize(value1, value2); + static EvalResult addNumericWithError(EvalResult value1, EvalResult value2) throws SQLException { + List resultValues = makeNumericAndPrioritize(value1, value2); value1 = resultValues.get(0); value2 = resultValues.get(1); switch (value1.getType()) { @@ -174,9 +179,9 @@ static EvalEngine.EvalResult addNumericWithError(EvalEngine.EvalResult value1, E throw new SQLException("invalid arithmetic between:" + value1.value().toString() + " " + value2.value().toString()); } - static EvalEngine.EvalResult subtractNumericWithError(EvalEngine.EvalResult i1, EvalEngine.EvalResult i2) throws SQLException { - EvalEngine.EvalResult v1 = makeNumeric(i1); - EvalEngine.EvalResult v2 = makeNumeric(i2); + static EvalResult subtractNumericWithError(EvalResult i1, EvalResult i2) throws SQLException { + EvalResult v1 = makeNumeric(i1); + EvalResult v2 = makeNumeric(i2); switch (v1.getType()) { case INT64: @@ -205,8 +210,8 @@ static EvalEngine.EvalResult subtractNumericWithError(EvalEngine.EvalResult i1, throw new SQLException("invalid arithmetic between:" + v1.value().toString() + " " + v2.value().toString()); } - static EvalEngine.EvalResult multiplyNumericWithError(EvalEngine.EvalResult value1, EvalEngine.EvalResult value2) throws SQLException { - List resultValues = makeNumericAndPrioritize(value1, value2); + static EvalResult multiplyNumericWithError(EvalResult value1, EvalResult value2) throws SQLException { + List resultValues = makeNumericAndPrioritize(value1, value2); value1 = resultValues.get(0); value2 = resultValues.get(1); switch (value1.getType()) { @@ -226,8 +231,8 @@ static EvalEngine.EvalResult multiplyNumericWithError(EvalEngine.EvalResult valu throw new SQLException("invalid arithmetic between:" + value1.value().toString() + " " + value2.value().toString()); } - static EvalEngine.EvalResult divideNumericWithError(EvalEngine.EvalResult value1, EvalEngine.EvalResult value2) throws SQLException { - List resultValues = makeNumericAndPrioritize(value1, value2); + static EvalResult divideNumericWithError(EvalResult value1, EvalResult value2) throws SQLException { + List resultValues = makeNumericAndPrioritize(value1, value2); value1 = resultValues.get(0); value2 = resultValues.get(1); switch (value1.getType()) { @@ -241,38 +246,38 @@ static EvalEngine.EvalResult divideNumericWithError(EvalEngine.EvalResult value1 throw new SQLException("invalid arithmetic between: " + value1.value().toString() + " " + value2.value().toString()); } - static EvalEngine.EvalResult intPlusIntWithError(Long v1, Long v2) throws SQLException { + static EvalResult intPlusIntWithError(Long v1, Long v2) throws SQLException { Long result = v1 + v2; if ((result > v1) != (v2 > 0)) { throw new SQLException("BIGINT value is out of range in " + v1 + " + " + v2); } - return new EvalEngine.EvalResult(result, INT64); + return new EvalResult(result, INT64); } - static EvalEngine.EvalResult intMinusIntWithError(Long v1, Long v2) throws SQLException { + static EvalResult intMinusIntWithError(Long v1, Long v2) throws SQLException { Long result = v1 - v2; if ((result < v1) != (v2 > 0)) { throw new SQLException("BIGINT value is out of range in " + v1 + " - " + v2); } - return new EvalEngine.EvalResult(result, INT64); + return new EvalResult(result, INT64); } - static EvalEngine.EvalResult intTimesIntWithError(Long v1, Long v2) throws SQLException { + static EvalResult intTimesIntWithError(Long v1, Long v2) throws SQLException { Long result = v1 * v2; if (v1 != 0 && result / v1 != v2) { throw new SQLException("BIGINT value is out of range in " + v1 + " * " + v2); } - return new EvalEngine.EvalResult(result, INT64); + return new EvalResult(result, INT64); } - static EvalEngine.EvalResult intMinusUintWithError(Long v1, BigInteger v2) throws SQLException { + static EvalResult intMinusUintWithError(Long v1, BigInteger v2) throws SQLException { if (v1 < 0 && v1 < v2.longValue()) { throw new SQLException("BIGINT value is out of range in " + v1 + " * " + v2); } return uintMinusUintWithError(BigInteger.valueOf(v1), v2); } - static EvalEngine.EvalResult uintMinusIntWithError(BigInteger v1, Long v2) throws SQLException { + static EvalResult uintMinusIntWithError(BigInteger v1, Long v2) throws SQLException { if (v1.longValue() < v2 && v2 > 0) { throw new SQLException("BIGINT UNSIGNED value is out of range in " + v1 + " - " + v2); } @@ -282,45 +287,45 @@ static EvalEngine.EvalResult uintMinusIntWithError(BigInteger v1, Long v2) throw return uintMinusUintWithError(v1, BigInteger.valueOf(v2)); } - static EvalEngine.EvalResult uintPlusIntWithError(BigInteger v1, Long v2) throws SQLException { + static EvalResult uintPlusIntWithError(BigInteger v1, Long v2) throws SQLException { if (v2 < 0 && v1.compareTo(BigInteger.valueOf(v2)) == -1) { throw new SQLException("BIGINT value is out of range in " + v1 + " + " + v2); } return uintPlusUintWithError(v1, BigInteger.valueOf(v2)); } - static EvalEngine.EvalResult uintMinusUintWithError(BigInteger v1, BigInteger v2) throws SQLException { + static EvalResult uintMinusUintWithError(BigInteger v1, BigInteger v2) throws SQLException { BigInteger result = v1.subtract(v2); if (v2.compareTo(v1) == 1) { throw new SQLException("BIGINT UNSIGNED value is out of range in " + v1 + " - " + v2); } - return new EvalEngine.EvalResult(result, UINT64); + return new EvalResult(result, UINT64); } - static EvalEngine.EvalResult uintPlusUintWithError(BigInteger v1, BigInteger v2) throws SQLException { + static EvalResult uintPlusUintWithError(BigInteger v1, BigInteger v2) throws SQLException { BigInteger result = v1.add(v2); if (result.compareTo(v2) == -1) { throw new SQLException("BIGINT value is out of range in " + v1 + " + " + v2); } - return new EvalEngine.EvalResult(result, UINT64); + return new EvalResult(result, UINT64); } - static EvalEngine.EvalResult uintTimesIntWithError(BigInteger v1, Long v2) throws SQLException { + static EvalResult uintTimesIntWithError(BigInteger v1, Long v2) throws SQLException { if (v2 < 0 || v1.longValue() < 0) { throw new SQLException("BIGINT value is out of range in " + v1 + " * " + v2); } return uintTimesUintWithError(v1, BigInteger.valueOf(v2)); } - static EvalEngine.EvalResult uintTimesUintWithError(BigInteger v1, BigInteger v2) throws SQLException { + static EvalResult uintTimesUintWithError(BigInteger v1, BigInteger v2) throws SQLException { BigInteger result = v1.multiply(v2); if (result.compareTo(v2) == -1 || result.compareTo(v1) == -1) { throw new SQLException("BIGINT value is out of range in " + v1 + " * " + v2); } - return new EvalEngine.EvalResult(result, UINT64); + return new EvalResult(result, UINT64); } - static EvalEngine.EvalResult anyMinusFloat(EvalEngine.EvalResult v1, Double v2) { + static EvalResult anyMinusFloat(EvalResult v1, Double v2) { switch (v1.getType()) { case INT64: v1.setFval(Double.parseDouble(v1.getIval().toString())); @@ -329,10 +334,10 @@ static EvalEngine.EvalResult anyMinusFloat(EvalEngine.EvalResult v1, Double v2) v1.setFval(Double.parseDouble(v1.getUval().toString())); break; } - return new EvalEngine.EvalResult(v1.getFval() - v2, FLOAT64); + return new EvalResult(v1.getFval() - v2, FLOAT64); } - static EvalEngine.EvalResult floatMinusAny(Double v1, EvalEngine.EvalResult v2) { + static EvalResult floatMinusAny(Double v1, EvalResult v2) { switch (v2.getType()) { case INT64: v2.setFval(Double.valueOf(v2.getIval().toString())); @@ -341,10 +346,10 @@ static EvalEngine.EvalResult floatMinusAny(Double v1, EvalEngine.EvalResult v2) v2.setFval(Double.valueOf(v2.getUval().toString())); break; } - return new EvalEngine.EvalResult(v1 - v2.getFval(), FLOAT64); + return new EvalResult(v1 - v2.getFval(), FLOAT64); } - static EvalEngine.EvalResult floatTimesAny(Double v1, EvalEngine.EvalResult v2) { + static EvalResult floatTimesAny(Double v1, EvalResult v2) { switch (v2.getType()) { case INT64: v2.setFval(Double.valueOf(v2.getIval().toString())); @@ -353,10 +358,10 @@ static EvalEngine.EvalResult floatTimesAny(Double v1, EvalEngine.EvalResult v2) v2.setFval(Double.valueOf(v2.getUval().toString())); break; } - return new EvalEngine.EvalResult(v1 * v2.getFval(), FLOAT64); + return new EvalResult(v1 * v2.getFval(), FLOAT64); } - static EvalEngine.EvalResult floatDivideAnyWithError(Double value1, EvalEngine.EvalResult value2) throws SQLException { + static EvalResult floatDivideAnyWithError(Double value1, EvalResult value2) throws SQLException { switch (value2.getType()) { case INT64: value2.setFval(Double.valueOf(value2.getIval().toString())); @@ -373,7 +378,7 @@ static EvalEngine.EvalResult floatDivideAnyWithError(Double value1, EvalEngine.E throw new SQLException("BIGINT is out of range in " + value1 + " " + value2.getFval()); } - return new EvalEngine.EvalResult(value1 / value2.getFval(), FLOAT64); + return new EvalResult(value1 / value2.getFval(), FLOAT64); } /** @@ -384,10 +389,10 @@ static EvalEngine.EvalResult floatDivideAnyWithError(Double value1, EvalEngine.E * @param inputValue2 * @return */ - private static List makeNumericAndPrioritize(EvalEngine.EvalResult inputValue1, EvalEngine.EvalResult inputValue2) { - EvalEngine.EvalResult value1 = makeNumeric(inputValue1); - EvalEngine.EvalResult value2 = makeNumeric(inputValue2); - List resultValues = new ArrayList<>(); + private static List makeNumericAndPrioritize(EvalResult inputValue1, EvalResult inputValue2) { + EvalResult value1 = makeNumeric(inputValue1); + EvalResult value2 = makeNumeric(inputValue2); + List resultValues = new ArrayList<>(); switch (value1.getType()) { case INT64: if (value2.getType() == UINT64 || value2.getType() == FLOAT64 || value2.getType() == DECIMAL) { @@ -415,24 +420,24 @@ private static List makeNumericAndPrioritize(EvalEngine.E * @param value * @return */ - private static EvalEngine.EvalResult makeNumeric(EvalEngine.EvalResult value) { + private static EvalResult makeNumeric(EvalResult value) { if (VtType.isNumber(value.getType())) { return value; } try { Long ival = Long.parseLong(new String(value.getBytes())); - return new EvalEngine.EvalResult(ival, INT64); + return new EvalResult(ival, INT64); } catch (Exception e) { } try { Double fval = Double.parseDouble(new String(value.getBytes())); - return new EvalEngine.EvalResult(fval, FLOAT64); + return new EvalResult(fval, FLOAT64); } catch (Exception e) { } - return new EvalEngine.EvalResult(new Long(0), Query.Type.INT64); + return new EvalResult(new Long(0), Query.Type.INT64); } /** @@ -461,20 +466,20 @@ static Boolean isByteComparable(VtValue value) { * @param value2 * @return */ - private static EvalEngine.EvalResult intPlusInt(Long value1, Long value2) { + private static EvalResult intPlusInt(Long value1, Long value2) { long result = value1 + value2; if (value1 > 0 && value2 > 0 && result < 0) { Double dv1 = value1.doubleValue(); Double dv2 = value2.doubleValue(); - return new EvalEngine.EvalResult(dv1 + dv2, FLOAT64); + return new EvalResult(dv1 + dv2, FLOAT64); } if (value1 < 0 && value2 < 0 && result > 0) { Double dv1 = value1.doubleValue(); Double dv2 = value2.doubleValue(); - return new EvalEngine.EvalResult(dv1 + dv2, FLOAT64); + return new EvalResult(dv1 + dv2, FLOAT64); } - return new EvalEngine.EvalResult(result, INT64); + return new EvalResult(result, INT64); } /** @@ -482,7 +487,7 @@ private static EvalEngine.EvalResult intPlusInt(Long value1, Long value2) { * @param value2 * @return */ - private static EvalEngine.EvalResult uintPlusInt(BigInteger value1, Long value2) { + private static EvalResult uintPlusInt(BigInteger value1, Long value2) { return uintPlusUint(value1, new BigInteger(String.valueOf(value2))); } @@ -491,15 +496,15 @@ private static EvalEngine.EvalResult uintPlusInt(BigInteger value1, Long value2) * @param value2 * @return */ - private static EvalEngine.EvalResult uintPlusUint(BigInteger value1, BigInteger value2) { + private static EvalResult uintPlusUint(BigInteger value1, BigInteger value2) { BigInteger result = value1.add(value2); //if result < v2 if (result.compareTo(value2) == -1) { Double v1 = value1.doubleValue(); Double v2 = value2.doubleValue(); - return new EvalEngine.EvalResult(v1 + v2, FLOAT64); + return new EvalResult(v1 + v2, FLOAT64); } - return new EvalEngine.EvalResult(result, UINT64); + return new EvalResult(result, UINT64); } /** @@ -507,7 +512,7 @@ private static EvalEngine.EvalResult uintPlusUint(BigInteger value1, BigInteger * @param value2 * @return */ - private static EvalEngine.EvalResult floatPlusAny(Double value1, EvalEngine.EvalResult value2) { + private static EvalResult floatPlusAny(Double value1, EvalResult value2) { switch (value2.getType()) { case INT64: value2.setFval(value2.getIval().doubleValue()); @@ -516,10 +521,10 @@ private static EvalEngine.EvalResult floatPlusAny(Double value1, EvalEngine.Eval value2.setFval(value2.getUval().doubleValue()); break; } - return new EvalEngine.EvalResult(value1 + value2.getFval(), FLOAT64); + return new EvalResult(value1 + value2.getFval(), FLOAT64); } - private static EvalEngine.EvalResult decimalPlusAny(BigDecimal value1, EvalEngine.EvalResult value2) { + private static EvalResult decimalPlusAny(BigDecimal value1, EvalResult value2) { switch (value2.getType()) { case INT64: value2.setBigDecimal(BigDecimal.valueOf(value2.getIval())); @@ -528,7 +533,7 @@ private static EvalEngine.EvalResult decimalPlusAny(BigDecimal value1, EvalEngin value2.setBigDecimal(new BigDecimal(value2.getUval().toString())); break; } - return new EvalEngine.EvalResult(value1.add(value2.getBigDecimal()), DECIMAL); + return new EvalResult(value1.add(value2.getBigDecimal()), DECIMAL); } /** @@ -536,7 +541,7 @@ private static EvalEngine.EvalResult decimalPlusAny(BigDecimal value1, EvalEngin * @param resultType * @return */ - public static VtValue castFromNumeric(EvalEngine.EvalResult value, Query.Type resultType) throws SQLException { + public static VtValue castFromNumeric(EvalResult value, Query.Type resultType) throws SQLException { if (VtType.isSigned(resultType)) { switch (value.getType()) { case INT64: @@ -579,7 +584,7 @@ public static VtValue castFromNumeric(EvalEngine.EvalResult value, Query.Type re return VtValue.NULL; } - public static VtResultValue castFromNum(EvalEngine.EvalResult value, Query.Type resultType) throws SQLException { + public static VtResultValue castFromNum(EvalResult value, Query.Type resultType) throws SQLException { if (VtType.isSigned(resultType)) { switch (value.getType()) { case INT64: diff --git a/src/main/java/com/jd/jdbc/evalengine/Comparisons.java b/src/main/java/com/jd/jdbc/evalengine/Comparisons.java new file mode 100644 index 0000000..4e49a5b --- /dev/null +++ b/src/main/java/com/jd/jdbc/evalengine/Comparisons.java @@ -0,0 +1,218 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.evalengine; + +import static com.jd.jdbc.evalengine.EvalEngine.TRUE_FLAG; +import com.jd.jdbc.srvtopo.BindVariable; +import io.vitess.proto.Query; +import static io.vitess.proto.Query.Type.NULL_TYPE; +import java.sql.SQLException; +import java.util.Map; +import lombok.AllArgsConstructor; +import lombok.Getter; + +public class Comparisons { + + public interface ComparisonOp { + /** + * @param left + * @param right + * @return + * @throws SQLException + */ + Boolean compare(EvalResult left, EvalResult right) throws SQLException; + + + /** + * @return + */ + String string(); + } + + @Getter + @AllArgsConstructor + public static class ComparisonExpr implements EvalEngine.Expr { + + private final ComparisonOp op; + + private final EvalEngine.Expr left; + + private final EvalEngine.Expr right; + + @Override + public EvalResult evaluate(EvalEngine.ExpressionEnv env) throws SQLException { + EvalResult lval; + if (this.left != null) { + lval = this.left.evaluate(env); + } else { + lval = new EvalResult(NULL_TYPE); + } + EvalResult rval; + if (this.right != null) { + rval = this.right.evaluate(env); + } else { + rval = new EvalResult(NULL_TYPE); + } + Boolean cmp = op.compare(lval, rval); + long value = 0; + if (cmp) { + value = TRUE_FLAG; + } + return new EvalResult(value, Query.Type.INT16); + } + + @Override + public EvalResult eval(EvalEngine.ExpressionEnv env, EvalResult result) throws SQLException { + EvalResult lval; + if (this.left != null) { + lval = this.left.evaluate(env); + } else { + lval = new EvalResult(NULL_TYPE); + } + EvalResult rval; + if (this.right != null) { + rval = this.right.evaluate(env); + } else { + rval = new EvalResult(NULL_TYPE); + } + Boolean cmp = op.compare(lval, rval); + long value = 0; + if (cmp) { + value = TRUE_FLAG; + } + return new EvalResult(value, Query.Type.INT16); + } + + @Override + public Query.Type type(EvalEngine.ExpressionEnv env) throws SQLException { + // return null; + return Query.Type.INT16; + } + + @Override + public int getFlags() { + return 0; + } + + @Override + public String string() { + return left.string() + op.string() + right.string(); + } + + @Override + public boolean constant() { + return false; + } + + @Override + public void output(StringBuilder builder, boolean wrap, Map bindVariableMap) throws SQLException { + + } + } + + public static class CompareEQ implements ComparisonOp { + @Override + public Boolean compare(EvalResult left, EvalResult right) throws SQLException { + return EvalEngine.allCompare(left.resultValue(), right.resultValue()) == 0; + } + + @Override + public String string() { + return "="; + } + } + + public static class CompareNE implements ComparisonOp { + @Override + public Boolean compare(EvalResult left, EvalResult right) throws SQLException { + return EvalEngine.allCompare(left.resultValue(), right.resultValue()) != 0; + } + + @Override + public String string() { + return "!="; + } + } + + public static class CompareLT implements ComparisonOp { + @Override + public Boolean compare(EvalResult left, EvalResult right) throws SQLException { + // left < right + return EvalEngine.allCompare(left.resultValue(), right.resultValue()) < 0; + } + + @Override + public String string() { + return "<"; + } + } + + public static class CompareLE implements ComparisonOp { + @Override + public Boolean compare(EvalResult left, EvalResult right) throws SQLException { + // left <= right + return EvalEngine.allCompare(left.resultValue(), right.resultValue()) <= 0; + } + + @Override + public String string() { + return "<="; + } + } + + public static class CompareGT implements ComparisonOp { + @Override + public Boolean compare(EvalResult left, EvalResult right) throws SQLException { + // left > right + return EvalEngine.allCompare(left.resultValue(), right.resultValue()) > 0; + } + + @Override + public String string() { + return ">"; + } + } + + public static class CompareGE implements ComparisonOp { + @Override + public Boolean compare(EvalResult left, EvalResult right) throws SQLException { + // left >= right + return EvalEngine.allCompare(left.resultValue(), right.resultValue()) >= 0; + } + + @Override + public String string() { + return ">="; + } + } + + public static class CompareNullSafeEQ implements ComparisonOp { + @Override + public Boolean compare(EvalResult left, EvalResult right) throws SQLException { + return EvalEngine.nullSafeCompare(left.resultValue(), right.resultValue()) == 0; + } + + @Override + public String string() { + return "<=>"; + } + } + + +} diff --git a/src/main/java/com/jd/jdbc/evalengine/EvalEngine.java b/src/main/java/com/jd/jdbc/evalengine/EvalEngine.java index c2491ef..ef8f8b7 100644 --- a/src/main/java/com/jd/jdbc/evalengine/EvalEngine.java +++ b/src/main/java/com/jd/jdbc/evalengine/EvalEngine.java @@ -18,9 +18,20 @@ package com.jd.jdbc.evalengine; +import com.jd.jdbc.common.util.CollectionUtils; import com.jd.jdbc.key.Bytes; -import com.jd.jdbc.sqlparser.support.logging.Log; -import com.jd.jdbc.sqlparser.support.logging.LogFactory; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLName; +import com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOpExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOperator; +import com.jd.jdbc.sqlparser.ast.expr.SQLCharExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLIdentifierExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLIntegerExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLNullExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLPropertyExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLVariantRefExpr; +import com.jd.jdbc.sqlparser.ast.expr.VtOffset; +import com.jd.jdbc.sqltypes.VtNumberRange; import com.jd.jdbc.sqltypes.VtResultValue; import com.jd.jdbc.sqltypes.VtValue; import com.jd.jdbc.vitess.resultset.ResultSetUtil; @@ -28,8 +39,8 @@ import java.math.BigDecimal; import java.math.BigInteger; import java.sql.SQLException; +import java.sql.SQLFeatureNotSupportedException; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Objects; @@ -39,6 +50,10 @@ public class EvalEngine { + public static final int TRUE_FLAG = 1; + + private static final Literal NULL_EXPR = new Literal(new EvalResult()); + /** * Cast converts a Value to the target type. * @@ -91,6 +106,13 @@ public static Integer nullSafeCompare(final VtResultValue value1, final VtResult throw new SQLException("types are not comparable: " + value1.getVtType() + " vs " + value2.getVtType()); } + public static Integer allCompare(final VtResultValue value1, final VtResultValue value2) throws SQLException { + if (value1.isNull() || value2.isNull()) { + return 0; + } + return nullSafeCompare(value1, value2); + } + private static Boolean isByteComparable(final VtResultValue value) { switch (value.getVtType()) { case BLOB: @@ -204,8 +226,16 @@ public static BigInteger toUint64(VtValue v) throws SQLException { } public static Expr newLiteralIntFromBytes(byte[] val) { - Long ival = Long.parseLong(new String(val)); - return newLiteralInt(ival); + BigInteger uval = new BigInteger(val); + if (uval.compareTo(new BigInteger(String.valueOf(VtNumberRange.INT64_MAX))) <= 0) { + Long ival = Long.parseLong(new String(val)); + return newLiteralInt(ival); + } + return newLiteralUint(uval); + } + + public static Expr newLiteralUint(BigInteger i) { + return new Literal(new EvalResult(i, Query.Type.UINT64)); } public static Expr newLiteralInt(Long i) { @@ -244,8 +274,37 @@ private static EvalResult evaluateByType(VtValue val) throws SQLException { return new EvalResult(val.getVtValue(), Query.Type.VARBINARY); case NULL_TYPE: return new EvalResult(Query.Type.NULL_TYPE); + default: + throw new SQLException("Type is not supported: " + val.getVtType()); + } + } + + private static EvalResult evaluateByType(VtResultValue val) throws SQLException { + switch (val.getVtType()) { + case INT64: + Long lval = Long.valueOf(val.toString()); + return new EvalResult(lval, Query.Type.INT64); + case INT32: + Long ival = Long.valueOf(val.toString()); + return new EvalResult(ival, Query.Type.INT32); + case UINT64: + BigInteger uval = new BigInteger(val.toString()); + return new EvalResult(uval, Query.Type.UINT64); + case FLOAT64: + Double fval = Double.valueOf(val.toString()); + return new EvalResult(fval, Query.Type.FLOAT64); + case DECIMAL: + BigDecimal bigDecimal = new BigDecimal(val.toString()); + return new EvalResult(bigDecimal, Query.Type.DECIMAL); + case VARCHAR: + case TEXT: + case VARBINARY: + return new EvalResult(val.toBytes(), Query.Type.VARBINARY); + case NULL_TYPE: + return new EvalResult(Query.Type.NULL_TYPE); + default: + throw new SQLException("Type is not supported: " + val.getVtType()); } - throw new SQLException("Type is not supported: " + val.getVtType()); } private static Query.Type mergeNumericalTypes(Query.Type ltype, Query.Type rtype) { @@ -254,20 +313,163 @@ private static Query.Type mergeNumericalTypes(Query.Type ltype, Query.Type rtype if (rtype == Query.Type.UINT32 || rtype == Query.Type.INT64 || rtype == Query.Type.UINT64 || rtype == Query.Type.FLOAT64 || rtype == Query.Type.DECIMAL) { return rtype; } + break; case INT64: if (rtype == Query.Type.UINT64 || rtype == Query.Type.FLOAT64 || rtype == Query.Type.DECIMAL) { return rtype; } + break; case UINT64: if (rtype == Query.Type.FLOAT64 || rtype == Query.Type.DECIMAL) { return rtype; } + break; case DECIMAL: break; + default: + throw new RuntimeException(); } return ltype; } + public static Expr translate(SQLExpr e, TranslationLookup translationLookup) throws SQLException { + return translateEx(e, translationLookup, true); + } + + public static Expr translate(List targetList, TranslationLookup lookup) throws SQLException { + List exprList = new ArrayList<>(); + for (SQLExpr expr : targetList) { + Expr translate = translate(expr, lookup); + if (translate == null) { + return null; + } + exprList.add(translate); + } + return new TupleExpr(exprList); + } + + public static Expr translateEx(SQLExpr e, TranslationLookup translationLookup, boolean simplify) throws SQLException { + Expr expr = translateExpr(e, translationLookup); + if (expr == null) { + return expr; + } + if (simplify) { + expr = Simplify.simplifyExpr(new EvalEngine.ExpressionEnv(), expr); + } + return expr; + } + + public static Expr translateEx(List targetList, TranslationLookup lookup, boolean simplify) throws SQLException { + Expr expr = translate(targetList, lookup); + if (expr == null) { + return expr; + } + if (simplify) { + expr = Simplify.simplifyExpr(new EvalEngine.ExpressionEnv(), expr); + } + return expr; + } + + private static Expr translateExpr(SQLExpr e, TranslationLookup translationLookup) throws SQLException { + if (e instanceof SQLIdentifierExpr || e instanceof SQLPropertyExpr) { + try { + int idx = translationLookup.columnLookup((SQLName) e); + if (idx >= 0) { + return new Column(idx); + } + return null; + } catch (SQLException exception) { + return null; + } + } else if (e instanceof SQLBinaryOpExpr) { + return translateComparisonExpr(e, translationLookup); + } else if (e instanceof VtOffset) { + return new Column(((VtOffset) e).getValue()); + } else if (e instanceof SQLVariantRefExpr) { + int index = ((SQLVariantRefExpr) e).getIndex(); + if (Integer.valueOf(-1).equals(index)) { + // default value + return new EvalEngine.BindVariable(((SQLVariantRefExpr) e).getName()); + } + return new EvalEngine.BindVariable(String.valueOf(index)); + } else if (e instanceof SQLNullExpr) { + return NULL_EXPR; + } else if (e instanceof SQLIntegerExpr) { + return EvalEngine.newLiteralIntFromBytes(((SQLIntegerExpr) e).getNumber().toString().getBytes()); + } else if (e instanceof SQLCharExpr) { + return new Literal(new EvalResult(((SQLCharExpr) e).getText().getBytes(), Query.Type.VARCHAR)); + } else { + // todo + return new AnyExpr(e); + } + } + + private static Expr translateComparisonExpr(SQLExpr e, TranslationLookup translationLookup) throws SQLException { + SQLExpr left = ((SQLBinaryOpExpr) e).getLeft(); + SQLExpr right = ((SQLBinaryOpExpr) e).getRight(); + Expr leftE = translateExpr(left, translationLookup); + Expr rightE = translateExpr(right, translationLookup); + BinaryExpr bExpr; + SQLBinaryOperator op = ((SQLBinaryOpExpr) e).getOperator(); + switch (op) { + case Equality: + return new Comparisons.ComparisonExpr(new Comparisons.CompareEQ(), leftE, rightE); + case NotEqual: + return new Comparisons.ComparisonExpr(new Comparisons.CompareNE(), leftE, rightE); + case LessThan: + return new Comparisons.ComparisonExpr(new Comparisons.CompareLT(), leftE, rightE); + case LessThanOrEqual: + return new Comparisons.ComparisonExpr(new Comparisons.CompareLE(), leftE, rightE); + case GreaterThan: + return new Comparisons.ComparisonExpr(new Comparisons.CompareGT(), leftE, rightE); + case GreaterThanOrEqual: + return new Comparisons.ComparisonExpr(new Comparisons.CompareGE(), leftE, rightE); + case LessThanOrEqualOrGreaterThan: + return new Comparisons.ComparisonExpr(new Comparisons.CompareNullSafeEQ(), leftE, rightE); + case BooleanAnd: + case BooleanXor: + case BooleanOr: + bExpr = new Logical.LogicalExpr(op); + return new BinaryOp(bExpr, leftE, rightE); + case Add: + bExpr = new EvalEngine.Addition(); + return new BinaryOp(bExpr, leftE, rightE); + case Subtract: + bExpr = new EvalEngine.Subtraction(); + return new BinaryOp(bExpr, leftE, rightE); + case Multiply: + bExpr = new EvalEngine.Multiplication(); + return new BinaryOp(bExpr, leftE, rightE); + case Divide: + bExpr = new EvalEngine.Division(); + return new BinaryOp(bExpr, leftE, rightE); + case Is: + bExpr = new Logical.IsExpr(false); + return new BinaryOp(bExpr, leftE, rightE); + case IsNot: + bExpr = new Logical.IsExpr(true); + return new BinaryOp(bExpr, leftE, rightE); + default: + throw new SQLFeatureNotSupportedException(op.name); + } + } + + /** + * NullsafeHashcode returns an int64 hashcode that is guaranteed to be the same + * for two values that are considered equal by `NullsafeCompare`. + * + * @param v + * @param collation + * @param coerceType + * @return + */ + public static long nullsafeHashcode(VtResultValue v, int collation, Query.Type coerceType) throws SQLException { + EvalResult cast = new EvalResult(); + cast.setValueCast(v, coerceType); +// cast.setCollation(collation); + return cast.nullSafeHashcode(); + } + /** * Expr is the interface that all evaluating expressions must implement */ @@ -278,17 +480,23 @@ public interface Expr { */ EvalResult evaluate(ExpressionEnv env) throws SQLException; + EvalResult eval(ExpressionEnv env, EvalResult result) throws SQLException; + /** * @param env * @return */ Query.Type type(ExpressionEnv env) throws SQLException; + int getFlags(); + /** * @return */ String string(); + boolean constant(); + void output(StringBuilder builder, boolean wrap, Map bindVariableMap) throws SQLException; } @@ -316,88 +524,6 @@ public interface BinaryExpr { String string(); } - @Getter - @Setter - public static class EvalResult { - private Query.Type type; - - private Long ival; - - private BigInteger uval; - - private Double fval; - - private BigDecimal bigDecimal; - - private byte[] bytes; - - public EvalResult() { - } - - public EvalResult(byte[] bytes, Query.Type type) { - this.bytes = bytes; - this.type = type; - } - - public EvalResult(Long ival, Query.Type type) { - this.ival = ival; - this.type = type; - } - - public EvalResult(BigInteger uval, Query.Type type) { - this.uval = uval; - this.type = type; - } - - public EvalResult(Double fval, Query.Type type) { - this.fval = fval; - this.type = type; - } - - public EvalResult(BigDecimal bigDecimal, Query.Type type) { - this.bigDecimal = bigDecimal; - this.type = type; - } - - public EvalResult(Query.Type type) { - this.type = type; - } - - @Override - public String toString() { - switch (this.type) { - case INT64: - return "Type: " + this.type + ", Value: " + this.ival; - case UINT64: - return "Type: " + this.type + ", Value: " + this.uval.toString(); - case FLOAT64: - return "Type: " + this.type + ", Value: " + this.fval.toString(); - case DECIMAL: - return "Type: " + this.type + ", Value: " + this.bigDecimal.toString(); - default: - return "Type: " + this.type + ", Value: " + Arrays.toString(this.bytes); - } - } - - /** - * Value allows for retrieval of the value we expose for public consumption - * - * @return - */ - public VtValue value() throws SQLException { - if (this.type == Query.Type.INT32 || this.type == Query.Type.INT24 || this.type == Query.Type.INT16) { - this.type = Query.Type.INT64; - } - return Arithmetic.castFromNumeric(this, this.type); - } - - public VtResultValue resultValue() throws SQLException { - if (this.type == Query.Type.INT32 || this.type == Query.Type.INT24 || this.type == Query.Type.INT16) { - this.type = Query.Type.INT64; - } - return Arithmetic.castFromNum(this, this.type); - } - } /** * ExpressionEnv contains the `environment that the expression @@ -415,6 +541,18 @@ public ExpressionEnv(Map bindVariableM this.bindVariableMap = bindVariableMap; this.row = new ArrayList<>(); } + + public ExpressionEnv() { + } + + public EvalResult evaluate(Expr expr) throws SQLException { + typecheck(expr); + return expr.eval(this, new EvalResult()); + } + + public void typecheck(Expr expr) { + //todo + } } public static class BindVariable implements Expr { @@ -432,6 +570,14 @@ public EvalResult evaluate(ExpressionEnv env) throws SQLException { return evaluateByType(VtValue.newVtValue(env.bindVariableMap.get(this.key))); } + @Override + public EvalResult eval(ExpressionEnv env, EvalResult result) throws SQLException { + if (!env.bindVariableMap.containsKey(this.key)) { + throw new SQLException("Bind variable not found"); + } + return evaluateByType(VtValue.newVtValue(env.bindVariableMap.get(this.key))); + } + @Override public Query.Type type(ExpressionEnv env) throws SQLException { if (!env.bindVariableMap.containsKey(this.key)) { @@ -440,9 +586,19 @@ public Query.Type type(ExpressionEnv env) throws SQLException { return env.bindVariableMap.get(this.key).getType(); } + @Override + public int getFlags() { + return 0; + } + @Override public String string() { - return ":" + this.key; + return this.key; + } + + @Override + public boolean constant() { + return false; } @Override @@ -451,9 +607,45 @@ public void output(final StringBuilder builder, final boolean wrap, final Map bindVariableMap) throws SQLException { + + } + } + + public static class Literal implements Expr { @Getter private final EvalResult val; @@ -466,21 +658,46 @@ public EvalResult evaluate(ExpressionEnv env) { return this.val; } + @Override + public EvalResult eval(ExpressionEnv env, EvalResult result) throws SQLException { + return this.val; + } + @Override public Query.Type type(ExpressionEnv env) { return this.val.getType(); } + @Override + public int getFlags() { + return 0; + } + @Override public String string() { try { - return this.val.value().toString(); + if (CollectionUtils.isNotEmpty(this.val.getTuple())) { + List strings = new ArrayList<>(); + for (VtValue tupleValue : this.val.tupleValues()) { + String string = tupleValue.string(); + strings.add(string); + } + return "(" + String.join(", ", strings) + ")"; + } + + Query.Type type = this.val.getType(); + String string = this.val.value().string(); + return string; } catch (SQLException e) { - logger.error(e.getMessage(), e); return ""; } } + @Override + public boolean constant() { + return true; + } + @Override public void output(final StringBuilder builder, final boolean wrap, final Map bindVariableMap) throws SQLException { builder.append(val.value().toString()); @@ -513,6 +730,22 @@ public EvalResult evaluate(ExpressionEnv env) throws SQLException { return this.expr.evaluate(lval, rval); } + @Override + public EvalResult eval(ExpressionEnv env, EvalResult result) throws SQLException { + EvalResult lval = this.left.evaluate(env); + EvalResult rval = this.right.evaluate(env); + + if (lval.type == Query.Type.INT32 || lval.type == Query.Type.INT24 || lval.type == Query.Type.INT16 || lval.type == Query.Type.INT8) { + lval.type = Query.Type.INT64; + } + + if (rval.type == Query.Type.INT32 || rval.type == Query.Type.INT24 || rval.type == Query.Type.INT16 || rval.type == Query.Type.INT8) { + rval.type = Query.Type.INT64; + } + + return this.expr.evaluate(lval, rval); + } + @Override public Query.Type type(ExpressionEnv env) throws SQLException { Query.Type ltype = this.left.type(env); @@ -521,11 +754,21 @@ public Query.Type type(ExpressionEnv env) throws SQLException { return this.expr.type(type); } + @Override + public int getFlags() { + return 0; + } + @Override public String string() { return null; } + @Override + public boolean constant() { + return this.left.constant() && this.right.constant(); + } + @Override public void output(final StringBuilder builder, final boolean wrap, final Map bindVariableMap) throws SQLException { if (wrap) { @@ -615,4 +858,155 @@ public String string() { return "/"; } } + + public static class AnyExpr implements Expr { + + private SQLExpr expr; + + public AnyExpr(SQLExpr expr) { + this.expr = expr; + } + + @Override + public EvalResult evaluate(ExpressionEnv env) throws SQLException { + return null; + } + + @Override + public EvalResult eval(ExpressionEnv env, EvalResult result) throws SQLException { + return null; + } + + @Override + public Query.Type type(ExpressionEnv env) throws SQLException { + return null; + } + + @Override + public int getFlags() { + return 0; + } + + @Override + public String string() { + return expr.toString(); + } + + @Override + public boolean constant() { + return false; + } + + @Override + public void output(StringBuilder builder, boolean wrap, Map bindVariableMap) throws SQLException { + + } + } + + @Getter + @AllArgsConstructor + public static class Column implements Expr { + + private int offset; + + @Override + public EvalResult evaluate(ExpressionEnv env) throws SQLException { + VtResultValue value = env.getRow().get(this.offset); + if (value.isNull()) { + return new EvalResult(Query.Type.NULL_TYPE); + } + return evaluateByType(value); + } + + @Override + public EvalResult eval(ExpressionEnv env, EvalResult result) throws SQLException { + VtResultValue value = env.getRow().get(this.offset); + if (value.isNull()) { + return new EvalResult(Query.Type.NULL_TYPE); + } + return evaluateByType(value); + } + + @Override + public Query.Type type(ExpressionEnv env) throws SQLException { + VtResultValue value = env.getRow().get(this.offset); + return value.getVtType(); + } + + @Override + public int getFlags() { + return 0; + } + + @Override + public String string() { + return null; + } + + @Override + public boolean constant() { + return false; + } + + @Override + public void output(StringBuilder builder, boolean wrap, Map bindVariableMap) throws SQLException { + + } + } + + @Getter + @AllArgsConstructor + public static class TupleExpr implements Expr { + + private List tupleExpr; + + @Override + public EvalResult evaluate(ExpressionEnv env) throws SQLException { + return null; + } + + @Override + public EvalResult eval(ExpressionEnv env, EvalResult result) throws SQLException { + List tup = new ArrayList<>(); + for (Expr expr : tupleExpr) { + tup.add(EvalResult.init(env, expr)); + } + result.setTuple(tup); + return result; + } + + @Override + public Query.Type type(ExpressionEnv env) throws SQLException { + return Query.Type.TUPLE; + } + + @Override + public int getFlags() { + return 0; + } + + @Override + public String string() { + List strings = new ArrayList<>(); + for (Expr expr : tupleExpr) { + strings.add(expr.string()); + } + return "(" + String.join(", ", strings) + ")"; + } + + @Override + public boolean constant() { + for (Expr subexpr : tupleExpr) { + if (!subexpr.constant()) { + return false; + } + } + return true; + } + + @Override + public void output(StringBuilder builder, boolean wrap, Map bindVariableMap) throws SQLException { + + } + } } diff --git a/src/main/java/com/jd/jdbc/evalengine/EvalResult.java b/src/main/java/com/jd/jdbc/evalengine/EvalResult.java new file mode 100644 index 0000000..6bb7b68 --- /dev/null +++ b/src/main/java/com/jd/jdbc/evalengine/EvalResult.java @@ -0,0 +1,403 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.evalengine; + +import com.jd.jdbc.sqltypes.VtNumberRange; +import com.jd.jdbc.sqltypes.VtResultValue; +import com.jd.jdbc.sqltypes.VtType; +import com.jd.jdbc.sqltypes.VtValue; +import io.vitess.proto.Query; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; +import lombok.Getter; +import lombok.Setter; + +@Getter +@Setter +public class EvalResult { + + // flagNull marks that this value is null; implies flagNullable + private final int flagNull = 1 << 0; + + // flagNullable marks that this value CAN be null + private final int flagNullable = 1 << 1; + + // flagIntegerUdf marks that this value is math.MinInt64, and will underflow if negated + private final int flagIntegerUdf = 1 << 5; + + // flagIntegerCap marks that this value is (-math.MinInt64), + // and should be promoted to flagIntegerUdf if negated + private final int flagIntegerCap = 1 << 6; + + // flagIntegerOvf marks that this value will overflow if negated + private final int flagIntegerOvf = 1 << 7; + + // flagHex marks that this value originated from a hex literal + private final int flagHex = 1 << 8; + + // flagBit marks that this value originated from a bit literal + private final int flagBit = 1 << 9; + + // flagExplicitCollation marks that this value has an explicit collation + private final int flagExplicitCollation = 1 << 10; + + private final int flagIntegerRange = flagIntegerOvf | flagIntegerCap | flagIntegerUdf; + + /** + * // expr is the expression that will be eventually evaluated to fill the other fields. + * // If expr is set, it means that this EvalResult has not been evaluated yet, and the + * // remaining fields are not valid. Once the evaluation engine calls EvalResult.resolve(), + * // the other fields will be set based on the evaluation result of expr and expr will be + * // set to nil, to mark this result as fully resolved. + */ + private EvalEngine.Expr expr; + + /** + * // env is the ExpressionEnv in which the expr is being evaluated + */ + private EvalEngine.ExpressionEnv env; + + /** + * // type_ is the SQL type of this result. + * // Must not be accessed directly: call EvalResult.typeof() instead. + * // For most expression types, this is known ahead of time and calling typeof() does not require + * // an evaluation, so the type of an expression can be known without evaluating it. + */ + protected Query.Type type = Query.Type.NULL_TYPE; + + private int flags; + + protected Long ival; + + protected BigInteger uval; + + private Double fval; + + private BigDecimal bigDecimal; + + private byte[] bytes; + + /** + * tuple_ is the list of all results contained in this result, if the result is a tuple. + * It may be uninitialized. + * Must not be accessed directly: call EvalResult.tuple() instead. + */ + //nolint + private List tuple; + + private final boolean typecheckEval = false; + + public EvalResult() { + } + + public EvalResult(byte[] bytes, Query.Type type) { + this.bytes = bytes; + this.type = type; + } + + public EvalResult(Long ival, Query.Type type) { + this.ival = ival; + this.type = type; + } + + public EvalResult(BigInteger uval, Query.Type type) { + this.uval = uval; + this.type = type; + } + + public EvalResult(Double fval, Query.Type type) { + this.fval = fval; + this.type = type; + } + + public EvalResult(BigDecimal bigDecimal, Query.Type type) { + this.bigDecimal = bigDecimal; + this.type = type; + } + + public EvalResult(EvalEngine.Expr expr, Query.Type type) { + this.expr = expr; + this.type = type; + } + + public EvalResult(Query.Type type) { + this.type = type; + } + + /** + * init initializes this EvalResult with the given expr. The actual value of this result will be + * calculated lazily when required, and will be the output of evaluating the expr. + * + * @param env + * @param expr + * @return + */ + public static EvalResult init(EvalEngine.ExpressionEnv env, EvalEngine.Expr expr) throws SQLException { + EvalResult er = new EvalResult(); + er.expr = expr; + er.type = expr.type(env); + er.flags = expr.getFlags(); + return er; + } + + @Override + public String toString() { + switch (this.type) { + case INT64: + return "Type: " + this.type + ", Value: " + this.ival; + case UINT64: + return "Type: " + this.type + ", Value: " + this.uval.toString(); + case FLOAT64: + return "Type: " + this.type + ", Value: " + this.fval.toString(); + case DECIMAL: + return "Type: " + this.type + ", Value: " + this.bigDecimal.toString(); + case EXPRESSION: + return "Type: " + this.type + ", Value: " + this.expr.toString(); + default: + return "Type: " + this.type + ", Value: " + Arrays.toString(this.bytes); + } + } + + /** + * Value allows for retrieval of the value we expose for public consumption + * + * @return + */ + public VtValue value() throws SQLException { + if (this.type == Query.Type.INT32 || this.type == Query.Type.INT24 || this.type == Query.Type.INT16) { + this.type = Query.Type.INT64; + } + return Arithmetic.castFromNumeric(this, this.type); + } + + public VtResultValue resultValue() throws SQLException { + if (this.type == Query.Type.INT32 || this.type == Query.Type.INT24 || this.type == Query.Type.INT16) { + this.type = Query.Type.INT64; + } + return Arithmetic.castFromNum(this, this.type); + } + + /** + * TupleValues allows for retrieval of the value we expose for public consumption + * + * @return + * @throws SQLException + */ + public List tupleValues() throws SQLException { + if (expr != null) { + throw new SQLException("did not resolve EvalResult after evaluation"); + } + if (tuple == null || tuple.size() == 0) { + return null; + } + List res = new ArrayList<>(); + for (EvalResult vt : tuple) { + res.add(vt.resolveValue().value()); + } + return res; + } + + public void setTuple(List tuple) { + this.tuple = tuple; + this.type = Query.Type.INT16; + } + + private void setNull() { + this.flags |= flagNullable | flagNull; + } + + public void setInt64(Long i) { + this.type = Query.Type.INT64; + this.uval = BigInteger.valueOf(i); + if (i == VtNumberRange.INT64_MIN) { + flags |= flagIntegerUdf; + } + } + + public void setUint64(BigInteger u) { + this.type = Query.Type.UINT64; + this.uval = u; + BigInteger bigInteger = BigInteger.valueOf(VtNumberRange.INT64_MAX + 1); + if (Objects.equals(u, bigInteger)) { + flags |= flagIntegerCap; + } + if (u.compareTo(bigInteger) > 0) { + flags |= flagIntegerOvf; + } + } + + public void setFloat(double f) { + this.type = Query.Type.FLOAT64; + this.uval = BigInteger.valueOf(Double.doubleToLongBits(f)); + } + + public void setRaw(Query.Type type, byte[] raw, Object coll) { + this.type = type; + this.bytes = raw; + } + + /** + * resolve computes the final value of this EvalResult by evaluating the expr embedded in it. + * This function should not be called directly: it will be called by the evaluation engine + * lazily when it needs to know the value of this result and not earlier. + */ + private void resolve() throws SQLException { + if (this.expr != null) { + if (typecheckEval) { + Query.Type before = this.type; + expr.eval(env, this); + if (before != this.type) { + throw new SQLException("did not pre-compute the right type: " + before + " before evaluation, " + this.type + " after"); + } + } else { + this.expr.eval(this.env, this); + } + this.expr = null; + } + } + + private EvalResult resolveValue() throws SQLException { + EvalResult eval = null; + if (this.expr != null) { + if (typecheckEval) { + Query.Type before = this.type; + expr.eval(env, this); + if (before != this.type) { + throw new SQLException("did not pre-compute the right type: " + before + " before evaluation, " + this.type + " after"); + } + } else { + eval = this.expr.eval(this.env, this); + } + eval.expr = null; + } + return eval; + } + + public void setValueCast(VtResultValue v, Query.Type coerceType) throws SQLException { + if (coerceType == Query.Type.NULL_TYPE) { + this.setNull(); + } else if (VtType.isFloat(coerceType)) { + if (VtType.isSigned(v.getVtType())) { + VtValue vtValue = VtValue.newVtValue(v.getVtType(), v.toBytes()); + BigInteger ival = EvalEngine.toUint64(vtValue); + this.setFloat(ival.floatValue()); + } else if (VtType.isUnsigned(v.getVtType())) { + VtValue vtValue = VtValue.newVtValue(v.getVtType(), v.toBytes()); + BigInteger ival = EvalEngine.toUint64(vtValue); + this.setFloat(ival.floatValue()); + } else if (VtType.isFloat(v.getVtType()) || Objects.equals(v.getVtType(), Query.Type.DECIMAL)) { + VtValue vtValue = VtValue.newVtValue(v.getVtType(), v.toBytes()); + float v1 = vtValue.toFloat(); + this.setFloat(v1); + } else if (VtType.isText(v.getVtType()) || VtType.isBinary(v.getVtType())) { + + } else { + throw new SQLException("coercion should not try to coerce this value to a float: " + v); + } + } else if (coerceType == Query.Type.DECIMAL) { + + } else if (VtType.isSigned(coerceType)) { + Query.Type vtType = v.getVtType(); + if (VtType.isSigned(vtType)) { + VtValue vtValue = VtValue.newVtValue(vtType, v.toBytes()); + BigInteger ival = EvalEngine.toUint64(vtValue); + this.setInt64(ival.longValue()); + } else if (VtType.isUnsigned(vtType)) { + VtValue vtValue = VtValue.newVtValue(vtType, v.toBytes()); + BigInteger ival = EvalEngine.toUint64(vtValue); + this.setInt64(ival.longValue()); + } else { + throw new SQLException("coercion should not try to coerce this value to a signed int: " + v); + } + } else if (VtType.isUnsigned(coerceType)) { + Query.Type vtType = v.getVtType(); + if (VtType.isSigned(vtType)) { + VtValue vtValue = VtValue.newVtValue(vtType, v.toBytes()); + BigInteger ival = EvalEngine.toUint64(vtValue); + this.setUint64(ival); + } else if (VtType.isUnsigned(vtType)) { + VtValue vtValue = VtValue.newVtValue(vtType, v.toBytes()); + BigInteger ival = EvalEngine.toUint64(vtValue); + this.setUint64(ival); + } else { + throw new SQLException("coercion should not try to coerce this value to a unsigned int: " + v); + } + } else if (VtType.isText(coerceType) || VtType.isBinary(coerceType)) { + if (VtType.isText(v.getVtType()) || VtType.isBinary(v.getVtType())) { + VtValue vtValue = VtValue.newVtValue(v.getVtType(), v.toBytes()); + this.setRaw(v.getVtType(), vtValue.raw(), null); + } else { + throw new SQLException("coercion should not try to coerce this value to a text: " + v); + } + } else { + throw new SQLException("coercion should not try to coerce this value: " + v); + } + } + + public long nullSafeHashcode() throws SQLException { + this.resolve(); + + if (this.isNull()) { + return Long.MAX_VALUE; + } else if (isNumeric()) { + return uint64().longValue(); + } else if (isTextual()) { + throw new SQLException("text type with an unknown/unsupported collation cannot be hashed"); + } else if (VtType.isDate(type)) { + + } else { + throw new SQLException("types does not support hashcode yet: " + type); + } + return -1; + } + + public boolean hasFlag(int f) { + return (flags & f) != 0; + } + + public boolean isNull() throws SQLException { + if (!hasFlag(flagNullable)) { + return false; + } + if (hasFlag(flagNull)) { + return true; + } + resolve(); + return hasFlag(flagNull); + } + + public boolean isNumeric() { + return VtType.isNumber(type); + } + + public boolean isTextual() { + return VtType.isText(type) || VtType.isBinary(type); + } + + public BigInteger uint64() throws SQLException { + resolve(); + return uval; + } + +} diff --git a/src/main/java/com/jd/jdbc/evalengine/Logical.java b/src/main/java/com/jd/jdbc/evalengine/Logical.java new file mode 100644 index 0000000..307cf1c --- /dev/null +++ b/src/main/java/com/jd/jdbc/evalengine/Logical.java @@ -0,0 +1,125 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.evalengine; + +import static com.jd.jdbc.evalengine.EvalEngine.TRUE_FLAG; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOperator; +import io.vitess.proto.Query; +import java.sql.SQLException; +import java.sql.SQLFeatureNotSupportedException; +import lombok.AllArgsConstructor; +import lombok.Getter; + +public class Logical { + @Getter + @AllArgsConstructor + public static class LogicalExpr implements EvalEngine.BinaryExpr { + SQLBinaryOperator op; + + @Override + public EvalResult evaluate(EvalResult left, EvalResult right) throws SQLException { + Boolean cmp = false; + if (op.equals(SQLBinaryOperator.BooleanAnd)) { + long leftEvalResult = left.value().toLong(); + long rightEvalResult = right.value().toLong(); + cmp = (leftEvalResult == TRUE_FLAG) && (rightEvalResult == TRUE_FLAG); + } else if (op.equals(SQLBinaryOperator.BooleanOr)) { + long leftEvalResult = left.value().toLong(); + long rightEvalResult = right.value().toLong(); + cmp = (leftEvalResult == TRUE_FLAG) || (rightEvalResult == TRUE_FLAG); + } else if (op.equals(SQLBinaryOperator.BooleanXor)) { + long leftEvalResult = left.value().toLong(); + long rightEvalResult = right.value().toLong(); + cmp = leftEvalResult != rightEvalResult; + } else { + throw new SQLFeatureNotSupportedException(op.name); + } + long value = 0; + if (cmp) { + value = TRUE_FLAG; + } + return new EvalResult(value, Query.Type.INT16); + } + + @Override + public Query.Type type(Query.Type left) { + return null; + } + + @Override + public String string() { + return op.name; + } + } + + public static class IsExpr implements EvalEngine.BinaryExpr { + boolean isNot; + + SQLExpr right; + + public IsExpr(boolean isNot) { + this.isNot = isNot; + } + + @Override + public EvalResult evaluate(EvalResult left, EvalResult right) throws SQLException { + //todo + /* Boolean cmp = false; + SQLExpr expr = right.getExpr(); + if (expr instanceof SQLNullExpr) { + cmp = left == null; + } + if (expr instanceof SQLNotNullConstraint) { + cmp = left != null; + } + + if (expr instanceof SQLBooleanExpr) { + long leftEvalResult = left.value().toLong(); + if (this.isNot && ((SQLBooleanExpr) expr).getBooleanValue()) { // not true + cmp = leftEvalResult != TRUE_FLAG; + } else if (this.isNot) { // not false + cmp = leftEvalResult == TRUE_FLAG; + } else if (((SQLBooleanExpr) expr).getBooleanValue()) { // true + cmp = leftEvalResult == TRUE_FLAG; + } else { // false + cmp = leftEvalResult != TRUE_FLAG; + } + } + + long value = 0; + if (cmp) { + value = TRUE_FLAG; + } + return new EvalResult(value, Query.Type.INT16);*/ + return null; + } + + @Override + public Query.Type type(Query.Type left) { + return Query.Type.INT64; + } + + @Override + public String string() { + return "is " + (isNot ? "not " : "") + right.toString(); + } + } + +} diff --git a/src/main/java/com/jd/jdbc/evalengine/Simplify.java b/src/main/java/com/jd/jdbc/evalengine/Simplify.java new file mode 100644 index 0000000..a393f44 --- /dev/null +++ b/src/main/java/com/jd/jdbc/evalengine/Simplify.java @@ -0,0 +1,31 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.evalengine; + +import java.sql.SQLException; + +public class Simplify { + public static EvalEngine.Expr simplifyExpr(EvalEngine.ExpressionEnv env, EvalEngine.Expr e) throws SQLException { + if (e.constant()) { + EvalResult res = env.evaluate(e); + return new EvalEngine.Literal(res); + } + return e; + } +} diff --git a/src/main/java/com/jd/jdbc/evalengine/TranslationLookup.java b/src/main/java/com/jd/jdbc/evalengine/TranslationLookup.java new file mode 100644 index 0000000..900393b --- /dev/null +++ b/src/main/java/com/jd/jdbc/evalengine/TranslationLookup.java @@ -0,0 +1,33 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.evalengine; + +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLName; +import java.sql.SQLException; + +public interface TranslationLookup { + + int columnLookup(SQLName col) throws SQLException; + + int collationForExpr(SQLExpr expr); + + int defaultCollation(); + +} diff --git a/src/main/java/com/jd/jdbc/key/DestinationKeyspaceID.java b/src/main/java/com/jd/jdbc/key/DestinationKeyspaceID.java index c6e1b34..c8a6779 100644 --- a/src/main/java/com/jd/jdbc/key/DestinationKeyspaceID.java +++ b/src/main/java/com/jd/jdbc/key/DestinationKeyspaceID.java @@ -18,6 +18,7 @@ package com.jd.jdbc.key; +import com.jd.jdbc.common.Hex; import io.vitess.proto.Topodata; import java.sql.SQLException; import java.util.Arrays; @@ -72,6 +73,6 @@ public Boolean isUnique() { @Override public String toString() { - return "DestinationKeyspaceID(" + Arrays.toString(value) + ")"; + return "DestinationKeyspaceID(" + Hex.encodeHexString(value) + ")"; } } diff --git a/src/main/java/com/jd/jdbc/monitor/ThreadPoolCollector.java b/src/main/java/com/jd/jdbc/monitor/ThreadPoolCollector.java index 35a278a..72b5d72 100644 --- a/src/main/java/com/jd/jdbc/monitor/ThreadPoolCollector.java +++ b/src/main/java/com/jd/jdbc/monitor/ThreadPoolCollector.java @@ -18,7 +18,6 @@ import com.jd.jdbc.util.threadpool.VtRejectedExecutionHandler; - import io.prometheus.client.Collector; import io.prometheus.client.GaugeMetricFamily; import java.util.Arrays; diff --git a/src/main/java/com/jd/jdbc/planbuilder/InsertPlan.java b/src/main/java/com/jd/jdbc/planbuilder/InsertPlan.java index fb5e121..4be0f85 100644 --- a/src/main/java/com/jd/jdbc/planbuilder/InsertPlan.java +++ b/src/main/java/com/jd/jdbc/planbuilder/InsertPlan.java @@ -47,7 +47,7 @@ import com.jd.jdbc.sqltypes.VtPlanValue; import com.jd.jdbc.tindexes.LogicTable; import com.jd.jdbc.vindexes.VKeyspace; -import static com.jd.jdbc.vindexes.Vschema.TYPE_PINNED_TABLE; +import static com.jd.jdbc.vindexes.VschemaConstant.TYPE_PINNED_TABLE; import io.netty.util.internal.StringUtil; import java.sql.SQLException; import java.sql.SQLFeatureNotSupportedException; diff --git a/src/main/java/com/jd/jdbc/planbuilder/PlanBuilder.java b/src/main/java/com/jd/jdbc/planbuilder/PlanBuilder.java index 22f7a3c..50d29c2 100644 --- a/src/main/java/com/jd/jdbc/planbuilder/PlanBuilder.java +++ b/src/main/java/com/jd/jdbc/planbuilder/PlanBuilder.java @@ -20,6 +20,7 @@ import com.google.common.collect.Sets; import com.jd.jdbc.VSchemaManager; +import com.jd.jdbc.common.Constant; import com.jd.jdbc.engine.Engine; import com.jd.jdbc.engine.JoinEngine; import com.jd.jdbc.engine.Plan; @@ -30,6 +31,7 @@ import com.jd.jdbc.evalengine.EvalEngine; import com.jd.jdbc.key.Bytes; import com.jd.jdbc.key.Destination; +import com.jd.jdbc.planbuilder.gen4.Gen4Planner; import com.jd.jdbc.sqlparser.SQLUtils; import com.jd.jdbc.sqlparser.SqlParser; import com.jd.jdbc.sqlparser.ast.SQLExpr; @@ -68,6 +70,7 @@ import com.jd.jdbc.sqlparser.dialect.mysql.ast.statement.MySqlSelectQueryBlock; import com.jd.jdbc.sqlparser.dialect.mysql.visitor.CheckNodeTypesVisitor; import com.jd.jdbc.sqlparser.dialect.mysql.visitor.VtHasSubqueryVisitor; +import com.jd.jdbc.sqlparser.dialect.mysql.visitor.VtRemoveDbNameVisitor; import com.jd.jdbc.sqlparser.support.logging.Log; import com.jd.jdbc.sqlparser.support.logging.LogFactory; import com.jd.jdbc.sqlparser.utils.StringUtils; @@ -89,6 +92,8 @@ public class PlanBuilder { private static final Log LOGGER = LogFactory.getLog(PlanBuilder.class); + private static final boolean GEN4_PLAN_ENABLE = Boolean.getBoolean(Constant.GEN4_PLAN_ENABLE); + /** * BuildFromStmt builds a plan based on the AST provided. * @@ -100,7 +105,7 @@ public class PlanBuilder { * @throws SQLException */ public static Plan buildFromStmt(final SQLStatement stmt, final VSchemaManager vm, final String defaultKeyspace, final BindVarNeeds bindVarNeeds, Destination destination) throws SQLException { - PrimitiveEngine instruction = createInstructionFor(stmt, vm, defaultKeyspace, destination); + PrimitiveEngine instruction = createInstructionFor(stmt, "", vm, defaultKeyspace, destination); return new Plan(SqlParser.astToStatementType(stmt), instruction, bindVarNeeds); } @@ -111,12 +116,16 @@ public static Plan buildFromStmt(final SQLStatement stmt, final VSchemaManager v * @return * @throws SQLException */ - private static PrimitiveEngine createInstructionFor(final SQLStatement stmt, final VSchemaManager vm, final String defaultKeyspace, Destination destination) throws SQLException { + private static PrimitiveEngine createInstructionFor(final SQLStatement stmt, final String query, final VSchemaManager vm, final String defaultKeyspace, Destination destination) + throws SQLException { if (destination != null) { return buildPlanForBypass(stmt, vm, defaultKeyspace, destination); } if (stmt instanceof SQLSelectStatement) { + if (GEN4_PLAN_ENABLE) { + return Gen4Planner.gen4SelectStmtPlanner(query, defaultKeyspace, (SQLSelectStatement) stmt, null, vm); + } if (((SQLSelectStatement) stmt).getSelect().getQuery() instanceof SQLUnionQuery) { return buildUnionPlan((SQLSelectStatement) stmt, vm, defaultKeyspace); } @@ -233,8 +242,8 @@ public static boolean isDualTable(final Builder builder) { * @return * @throws SQLException */ - private static PrimitiveEngine handleDualSelects(SQLSelectQuery query, VSchemaManager vm, String defaultKeyspace) throws SQLException { - if (!(query instanceof SQLSelectQueryBlock)) { + public static PrimitiveEngine handleDualSelects(SQLSelectQuery query, VSchemaManager vm, String defaultKeyspace) throws SQLException { + if (!(query instanceof MySqlSelectQueryBlock)) { throw new SQLFeatureNotSupportedException("unsupported sql statement: " + SQLUtils.toMySqlString(query, SQLUtils.NOT_FORMAT_OPTION).trim()); } @@ -429,6 +438,13 @@ public static List splitAndExpression(List filters, SQLExpr no return filters; } + // removeKeyspaceFromColName removes the Qualifier.Qualifier on all ColNames in the expression tree + public static SQLExpr removeKeyspaceFromColName(SQLExpr expr) { + VtRemoveDbNameVisitor visitor = new VtRemoveDbNameVisitor(); + expr.accept(visitor); + return expr; + } + /** * reorderBySubquery reorders the filters by pushing subqueries * to the end. This allows the non-subquery filters to be @@ -654,6 +670,20 @@ private static boolean nameMatch(final SQLExpr node, final String col) { return false; } + public static MySqlSelectQueryBlock getFirstSelect(SQLSelectQuery selStmt) throws SQLException { + if (selStmt == null) { + return null; + } + if (selStmt instanceof MySqlSelectQueryBlock) { + return (MySqlSelectQueryBlock) selStmt; + } else if (selStmt instanceof SQLUnionQuery) { + SQLUnionQuery sqlUnionQuery = (SQLUnionQuery) selStmt; + return getFirstSelect(sqlUnionQuery.getLeft()); + } else { + throw new SQLException("[BUG]: unknown type for SelectStatement"); + } + } + @Getter @AllArgsConstructor public static class NewSubqueryPlanResponse { diff --git a/src/main/java/com/jd/jdbc/planbuilder/PrimitiveBuilder.java b/src/main/java/com/jd/jdbc/planbuilder/PrimitiveBuilder.java index 46a98b0..1ef33fb 100644 --- a/src/main/java/com/jd/jdbc/planbuilder/PrimitiveBuilder.java +++ b/src/main/java/com/jd/jdbc/planbuilder/PrimitiveBuilder.java @@ -89,10 +89,10 @@ import com.jd.jdbc.tindexes.LogicTable; import com.jd.jdbc.tindexes.TableIndex; import com.jd.jdbc.vindexes.VKeyspace; -import static com.jd.jdbc.vindexes.Vschema.CODE_PINNED_TABLE; -import static com.jd.jdbc.vindexes.Vschema.TYPE_PINNED_TABLE; -import static com.jd.jdbc.vindexes.Vschema.TYPE_REFERENCE; -import static com.jd.jdbc.vindexes.Vschema.TYPE_SEQUENCE; +import static com.jd.jdbc.vindexes.VschemaConstant.CODE_PINNED_TABLE; +import static com.jd.jdbc.vindexes.VschemaConstant.TYPE_PINNED_TABLE; +import static com.jd.jdbc.vindexes.VschemaConstant.TYPE_REFERENCE; +import static com.jd.jdbc.vindexes.VschemaConstant.TYPE_SEQUENCE; import com.jd.jdbc.vindexes.hash.Binary; import com.jd.jdbc.vindexes.hash.BinaryHash; import com.jd.jdbc.vindexes.hash.Hash; @@ -112,29 +112,6 @@ import lombok.Getter; import vschema.Vschema; -import static com.jd.jdbc.engine.Engine.PulloutOpcode.PulloutExists; -import static com.jd.jdbc.engine.Engine.PulloutOpcode.PulloutIn; -import static com.jd.jdbc.engine.Engine.PulloutOpcode.PulloutNotIn; -import static com.jd.jdbc.engine.Engine.PulloutOpcode.PulloutValue; -import static com.jd.jdbc.engine.Engine.RouteOpcode.SelectDBA; -import static com.jd.jdbc.engine.Engine.RouteOpcode.SelectReference; -import static com.jd.jdbc.sqlparser.SqlParser.GroupByExpr.ColName; -import static com.jd.jdbc.sqlparser.SqlParser.GroupByExpr.Literal; -import static com.jd.jdbc.sqlparser.SqlParser.HAVING_STR; -import static com.jd.jdbc.sqlparser.SqlParser.SelectExpr.AliasedExpr; -import static com.jd.jdbc.sqlparser.SqlParser.SelectExpr.Nextval; -import static com.jd.jdbc.sqlparser.SqlParser.SelectExpr.StarExpr; -import static com.jd.jdbc.sqlparser.SqlParser.WHERE_STR; -import static com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOperator.BooleanAnd; -import static com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOperator.BooleanOr; -import static com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOperator.Equality; -import static com.jd.jdbc.sqlparser.ast.statement.SQLJoinTableSource.JoinType; -import static com.jd.jdbc.sqlparser.utils.JdbcConstants.MYSQL; -import static com.jd.jdbc.vindexes.Vschema.CODE_PINNED_TABLE; -import static com.jd.jdbc.vindexes.Vschema.TYPE_PINNED_TABLE; -import static com.jd.jdbc.vindexes.Vschema.TYPE_REFERENCE; -import static com.jd.jdbc.vindexes.Vschema.TYPE_SEQUENCE; - @Data public class PrimitiveBuilder { private static final Log log = LogFactory.getLog(PrimitiveBuilder.class); @@ -439,6 +416,9 @@ private void pushLimit(SQLLimit limit) throws SQLException { return; } } + if (limit.getParent() instanceof SQLUnionQuery) { + throw new SQLFeatureNotSupportedException("Incorrect usage of UNION and LIMIT - add parens to disambiguate your query"); + } LimitPlan lb = new LimitPlan(this.builder); try { lb.setLimit(limit); diff --git a/src/main/java/com/jd/jdbc/planbuilder/Symtab.java b/src/main/java/com/jd/jdbc/planbuilder/Symtab.java index e03e2ff..f518ba5 100644 --- a/src/main/java/com/jd/jdbc/planbuilder/Symtab.java +++ b/src/main/java/com/jd/jdbc/planbuilder/Symtab.java @@ -537,7 +537,10 @@ private Column searchTables(SQLName col) throws SQLException { // Check for nil because there can be nil entries if there // are duplicate columns across multiple tables. if (this.uniqueColumns.containsKey(col.getSimpleName().toLowerCase())) { - return this.uniqueColumns.get(col.getSimpleName().toLowerCase()); + Column c = this.uniqueColumns.get(col.getSimpleName().toLowerCase()); + if (c != null) { + return c; + } } if (this.tables.size() == 1) { diff --git a/src/main/java/com/jd/jdbc/planbuilder/UpdatePlan.java b/src/main/java/com/jd/jdbc/planbuilder/UpdatePlan.java index 053cf81..ef7fdef 100644 --- a/src/main/java/com/jd/jdbc/planbuilder/UpdatePlan.java +++ b/src/main/java/com/jd/jdbc/planbuilder/UpdatePlan.java @@ -143,8 +143,6 @@ private static void buildChangedTindexesVlaues(SQLUpdateStatement update, LogicT * planbuilder/update.go --> buildChangedVindexesValues */ public static void buildChangedVindexesValues(SQLUpdateStatement update, Vschema.Table table, String ksidCol) throws SQLException { - /*changedVindexes := make(map[string]*engine.VindexValues) - buf, offset := initialQuery(ksidCol, table)*/ for (Vschema.ColumnVindex vindex : table.getColumnVindexesList()) { Map vindexValueMap = new HashMap<>(); boolean first = true; @@ -168,48 +166,15 @@ public static void buildChangedVindexesValues(SQLUpdateStatement update, Vschema VtPlanValue pv = extractValueFromUpdate(assignment); vindexValueMap.put(vcol, pv); if (first) { - /*buf.Myprintf(", %v", assignment)*/ first = false; - } /*else{ - buf.Myprintf(" and %v", assignment) - }*/ + } } } if (vindexValueMap.size() != 0) { // Vindex not changing, continue throw new SQLFeatureNotSupportedException("unsupported: You can't update primary vindex columns. Invalid update on vindex: " + columns.get(0)); } - - /*if update.Limit != nil && len(update.OrderBy) == 0 { - return nil, - "", vterrors.Errorf(vtrpcpb.Code_UNIMPLEMENTED, "unsupported: Need to provide order by clause when using limit. Invalid update on vindex: %v", vindex.Name) - } - if i == 0 { - return nil, - "", vterrors.Errorf(vtrpcpb.Code_UNIMPLEMENTED, "unsupported: You can't update primary vindex columns. Invalid update on vindex: %v", vindex.Name) - } - if _, ok :=vindex.Vindex. (vindexes.Lookup); - !ok { - return nil, - "", vterrors.Errorf(vtrpcpb.Code_UNIMPLEMENTED, "unsupported: You can only update lookup vindexes. Invalid update on vindex: %v", vindex.Name) - } - if !vindex.Owned { - return nil, - "", vterrors.Errorf(vtrpcpb.Code_UNIMPLEMENTED, "unsupported: You can only update owned vindexes. Invalid update on vindex: %v", vindex.Name) - } - changedVindexes[vindex.Name] = &engine.VindexValues { - PvMap: - vindexValueMap, - Offset:offset, - } - offset++*/ - } - /*if len(changedVindexes) == 0 { - return nil,"", nil } - // generate rest of the owned vindex query. - buf.Myprintf(" from %v%v%v%v for update", table.Name, update.Where, update.OrderBy, update.Limit) - return changedVindexes,buf.String(), nil*/ } @Override diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/AggregationPushing.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/AggregationPushing.java new file mode 100644 index 0000000..9cb6d3b --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/AggregationPushing.java @@ -0,0 +1,109 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4; + +import com.jd.jdbc.common.tuple.Pair; +import com.jd.jdbc.common.tuple.Triple; +import com.jd.jdbc.context.PlanningContext; +import com.jd.jdbc.engine.Engine; +import com.jd.jdbc.engine.gen4.AbstractAggregateGen4; +import com.jd.jdbc.planbuilder.gen4.logical.JoinGen4Plan; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.ast.expr.SQLAggregateExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLAllColumnExpr; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; + +public class AggregationPushing { + + @FunctionalInterface + interface Func { + Pair, List>> passThrough(List groupByOffsets, List> aggrOffsets); + } + + public static Pair, List> splitAggregationsToLeftAndRight( + PlanningContext ctx, List aggregations, JoinGen4Plan join) throws SQLException { + + List lhsAggrs = new ArrayList<>(); + List rhsAggrs = new ArrayList<>(); + + for (QueryProjection.Aggr aggr : aggregations) { + boolean foundCountStar = false; + if (aggr.getOriginal().getExpr() instanceof SQLAggregateExpr) { // CountStar + SQLAggregateExpr aggrExpr = (SQLAggregateExpr) aggr.getOriginal().getExpr(); + Engine.AggregateOpcodeG4 opcode = AbstractAggregateGen4.SUPPORTED_AGGREGATES.get(aggrExpr.getMethodName().toLowerCase()); + if (opcode == Engine.AggregateOpcodeG4.AggregateCountStar) { // countstar count(*) ?? + lhsAggrs.add(aggr); + rhsAggrs.add(aggr); + foundCountStar = true; + } + } + + if (!foundCountStar) { + TableSet deps = ctx.getSemTable().recursiveDeps(aggr.getOriginal().getExpr()); + // if we are sending down min/max, we don't have to multiply the results with anything + QueryProjection.Aggr other = null; + if (aggr.getOpCode() != Engine.AggregateOpcodeG4.AggregateMax || aggr.getOpCode() != Engine.AggregateOpcodeG4.AggregateMin) { + // create countstar expr + SQLAggregateExpr countStarExpr = new SQLAggregateExpr("count"); + countStarExpr.getArguments().add(new SQLAllColumnExpr()); + other = new QueryProjection.Aggr(new SQLSelectItem(countStarExpr), Engine.AggregateOpcodeG4.AggregateCountStar, "count(*)", 0); + } + + if (deps.isSolvedBy(join.getLeft().containsTables())) { + lhsAggrs.add(aggr); + rhsAggrs.add(other); + } else if (deps.isSolvedBy(join.getRight().containsTables())) { + rhsAggrs.add(aggr); + lhsAggrs.add(other); + } else { + throw new SQLException("aggregation on columns from different sources not supported yet"); + } + } + } + return Pair.of(lhsAggrs, rhsAggrs); + } + + public static Triple, List, List> splitGroupingsToLeftAndRight( + PlanningContext ctx, JoinGen4Plan join, List grouping, List lhsGrouping) throws SQLException { + + List rhsGrouping = new ArrayList<>(); + + TableSet lhsTS = join.getLeft().containsTables(); + TableSet rhsTS = join.getRight().containsTables(); + // here we store information about which side the grouping value is coming from. + // Negative values from the left operator and positive values are offsets into the RHS + List groupingOffsets = new ArrayList<>(); + for (QueryProjection.GroupBy groupBy : grouping) { + TableSet deps = ctx.getSemTable().recursiveDeps(groupBy.getInner()); + if (deps.isSolvedBy(lhsTS)) { + groupingOffsets.add(-(lhsGrouping.size() + 1)); + lhsGrouping.add(groupBy); + } else if (deps.isSolvedBy(rhsTS)) { + groupingOffsets.add(rhsGrouping.size() + 1); + rhsGrouping.add(groupBy); + } else { + throw new SQLException("grouping on columns from different sources not supported yet"); + } + } + return Triple.of(lhsGrouping, rhsGrouping, groupingOffsets); + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/Gen4Planner.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/Gen4Planner.java new file mode 100644 index 0000000..ba07178 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/Gen4Planner.java @@ -0,0 +1,145 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4; + +import com.jd.jdbc.VSchemaManager; +import com.jd.jdbc.context.PlanningContext; +import com.jd.jdbc.engine.PrimitiveEngine; +import com.jd.jdbc.planbuilder.PlanBuilder; +import com.jd.jdbc.planbuilder.gen4.logical.LogicalPlan; +import com.jd.jdbc.planbuilder.gen4.logical.RouteGen4Plan; +import com.jd.jdbc.planbuilder.gen4.operator.OperatorTransformers; +import com.jd.jdbc.planbuilder.gen4.operator.OperatorUtil; +import com.jd.jdbc.planbuilder.gen4.operator.logical.LogicalOperator; +import com.jd.jdbc.planbuilder.gen4.operator.physical.PhysicalOperator; +import com.jd.jdbc.planbuilder.gen4.operator.physical.RoutePlanning; +import com.jd.jdbc.planbuilder.semantics.Analyzer; +import com.jd.jdbc.planbuilder.semantics.SemTable; +import com.jd.jdbc.planbuilder.semantics.VSchema; +import com.jd.jdbc.sqlparser.ast.SQLLimit; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectQuery; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectStatement; +import com.jd.jdbc.sqlparser.ast.statement.SQLUnionQuery; +import com.jd.jdbc.sqlparser.dialect.mysql.ast.statement.MySqlSelectQueryBlock; +import java.sql.SQLException; + +public class Gen4Planner { + + public static PrimitiveEngine gen4SelectStmtPlanner(String query, String defaultKeyspace, SQLSelectStatement stmt, Object reservedVars, VSchemaManager vschema) throws SQLException { + if (stmt.getSelect().getQuery() instanceof MySqlSelectQueryBlock) { + // handle dual table for processing at vtgate. + PrimitiveEngine p = PlanBuilder.handleDualSelects(stmt.getSelect().getQuery(), vschema, defaultKeyspace); + boolean calcFoundRows = ((MySqlSelectQueryBlock) stmt.getSelect().getQuery()).isCalcFoundRows(); + if (p != null) { + return p; + } + } + + VSchema vSchema = new VSchema(defaultKeyspace, vschema); + LogicalPlan plan = newBuildSelectPlan(stmt, reservedVars, vSchema, null); + + return plan.getPrimitiveEngine(); + } + + private static LogicalPlan newBuildSelectPlan(SQLSelectStatement selStmt, Object reservedVars, VSchema vschema, Object version) throws SQLException { + String ksName = vschema.getDefaultKeyspace(); + SemTable semTable = Analyzer.analyze(selStmt, ksName, vschema); + + // record any warning as planner warning. + vschema.plannerWarning(semTable.getWarning()); + + PlanningContext ctx = new PlanningContext(reservedVars, semTable, vschema, vschema.getVschemaKeyspace()); + + if (!vschema.getVschemaKeyspace().getSharded() && semTable.singleUnshardedKeyspace() != null) { + return SingleShardedShortcut.unshardedShortcut(ctx, selStmt.getSelect().getQuery(), vschema.getVschemaKeyspace()); + } + + Rewriter.queryRewrite(semTable, reservedVars, selStmt); + + + LogicalOperator logical = OperatorUtil.createLogicalOperatorFromAST(selStmt, semTable); + logical.checkValid(); + + PhysicalOperator physOp = RoutePlanning.createPhysicalOperator(ctx, logical); + + LogicalPlan plan = OperatorTransformers.transformToLogicalPlan(ctx, physOp, true); + + plan = planHorizon(ctx, plan, selStmt.getSelect().getQuery(), true); + + plan.wireupGen4(ctx); + + plan = pushCommentDirectivesOnPlan(plan, selStmt); + + return plan; + } + + public static LogicalPlan planHorizon(PlanningContext ctx, LogicalPlan plan, SQLSelectQuery in, boolean truncateColumns) throws SQLException { + if (in instanceof MySqlSelectQueryBlock) { + MySqlSelectQueryBlock selectQueryBlock = (MySqlSelectQueryBlock) in; + HorizonPlanning hp = new HorizonPlanning(in); +// replaceSubQuery(ctx, node) + LogicalPlan horizonPlan = hp.planHorizon(ctx, plan, truncateColumns); + return planLimit(selectQueryBlock.getLimit(), horizonPlan); + } + if (in instanceof SQLUnionQuery) { + if (!(plan instanceof RouteGen4Plan) && ctx.getSemTable().getNotSingleRouteErr() != null) { + throw new SQLException(ctx.getSemTable().getNotSingleRouteErr()); + } + if (plan instanceof RouteGen4Plan && ((RouteGen4Plan) plan).isSingleShard()) { + HorizonPlanning.planSingleShardRoutePlan(in, (RouteGen4Plan) plan); + } else { + plan = planOrderByOnUnion(ctx, plan, in); + } + + return planLimit(((SQLUnionQuery) in).getLimit(), plan); + } + return plan; + } + + private static LogicalPlan planOrderByOnUnion(PlanningContext ctx, LogicalPlan plan, SQLSelectQuery union) throws SQLException { + QueryProjection qp = QueryProjection.createQPFromUnion(union); + HorizonPlanning hp = new HorizonPlanning(qp); + if (qp.getOrderExprs().size() > 0) { + plan = hp.planOrderBy(ctx, qp.getOrderExprs(), plan); + } + return plan; + } + + private static LogicalPlan pushCommentDirectivesOnPlan(LogicalPlan plan, SQLSelectStatement stmt) { + return plan; + } + + private static LogicalPlan planLimit(SQLLimit limit, LogicalPlan plan) throws SQLException { + if (limit == null) { + return plan; + } + + if (plan instanceof RouteGen4Plan) { + RouteGen4Plan planGen4 = (RouteGen4Plan) plan; + if (planGen4.isSingleShard()) { + planGen4.setLimit(limit); + return plan; + } + } + LogicalPlan lPlan = PostProcess.createLimit(plan, limit); + // visit does not modify the plan. + LogicalPlan.visit(lPlan, new PostProcess.SetUpperLimit()); + return lPlan; + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/HorizonPlanning.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/HorizonPlanning.java new file mode 100644 index 0000000..899c846 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/HorizonPlanning.java @@ -0,0 +1,1414 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4; + +import com.google.common.collect.Sets; +import com.jd.jdbc.common.tuple.ImmutablePair; +import com.jd.jdbc.common.tuple.ImmutableTriple; +import com.jd.jdbc.common.tuple.MutableTriple; +import com.jd.jdbc.common.tuple.Pair; +import com.jd.jdbc.common.tuple.Triple; +import com.jd.jdbc.common.util.CollectionUtils; +import com.jd.jdbc.context.PlanningContext; +import com.jd.jdbc.engine.Engine; +import com.jd.jdbc.engine.gen4.AbstractAggregateGen4; +import com.jd.jdbc.engine.gen4.GroupByParams; +import com.jd.jdbc.engine.gen4.MemorySortGen4Engine; +import com.jd.jdbc.engine.gen4.OrderByParamsGen4; +import com.jd.jdbc.planbuilder.MemorySortPlan; +import com.jd.jdbc.planbuilder.PlanBuilder; +import com.jd.jdbc.planbuilder.gen4.logical.DistinctGen4Plan; +import com.jd.jdbc.planbuilder.gen4.logical.FilterGen4Plan; +import com.jd.jdbc.planbuilder.gen4.logical.HashJoinPlan; +import com.jd.jdbc.planbuilder.gen4.logical.JoinGen4Plan; +import com.jd.jdbc.planbuilder.gen4.logical.LimitGen4Plan; +import com.jd.jdbc.planbuilder.gen4.logical.LogicalPlan; +import com.jd.jdbc.planbuilder.gen4.logical.MemorySortGen4Plan; +import com.jd.jdbc.planbuilder.gen4.logical.OrderedAggregateGen4Plan; +import com.jd.jdbc.planbuilder.gen4.logical.ProjectionGen4Plan; +import com.jd.jdbc.planbuilder.gen4.logical.RouteGen4Plan; +import com.jd.jdbc.planbuilder.gen4.logical.SimpleProjectionGen4Plan; +import com.jd.jdbc.planbuilder.semantics.SemTable; +import com.jd.jdbc.planbuilder.semantics.TableInfo; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.planbuilder.semantics.Type; +import com.jd.jdbc.sqlparser.SqlParser; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLName; +import com.jd.jdbc.sqlparser.ast.SQLOrderBy; +import com.jd.jdbc.sqlparser.ast.SQLOrderingSpecification; +import com.jd.jdbc.sqlparser.ast.SQLSetQuantifier; +import com.jd.jdbc.sqlparser.ast.expr.SQLAggregateExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLExprUtils; +import com.jd.jdbc.sqlparser.ast.expr.SQLIdentifierExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLIntegerExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLMethodInvokeExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLNullExpr; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectGroupByClause; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectOrderByItem; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectQuery; +import com.jd.jdbc.sqlparser.ast.statement.SQLUnionQuery; +import com.jd.jdbc.sqlparser.dialect.mysql.ast.statement.MySqlSelectQueryBlock; +import com.jd.jdbc.sqlparser.dialect.mysql.visitor.CheckNodeTypesVisitor; +import com.jd.jdbc.sqlparser.dialect.mysql.visitor.RewriteHavingAggrWithOffsetVisitor; +import com.jd.jdbc.sqlparser.dialect.mysql.visitor.VtRemoveDbNameExpectSystemDbVisitor; +import com.jd.jdbc.sqlparser.dialect.mysql.visitor.VtRemoveDbNameInColumnVisitor; +import com.jd.jdbc.sqlparser.utils.StringUtils; +import com.jd.jdbc.sqltypes.VtType; +import java.sql.SQLException; +import java.sql.SQLFeatureNotSupportedException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import lombok.Getter; +import lombok.NoArgsConstructor; +import lombok.Setter; +import vschema.Vschema; + +@Getter +@Setter +public class HorizonPlanning { + + private SQLSelectQuery sel; + + private QueryProjection qp; + + private boolean hasHaving; + + public HorizonPlanning(QueryProjection qp) { + this.qp = qp; + } + + public HorizonPlanning(SQLSelectQuery sel) { + this.sel = sel; + } + + public LogicalPlan planHorizon(PlanningContext ctx, LogicalPlan plan, boolean truncateColumns) throws SQLException { + boolean isRoute = plan instanceof RouteGen4Plan; + if (!isRoute && ctx.getSemTable().getNotSingleRouteErr() != null) { + // If we got here, we don't have a single shard plan + throw new SQLException(ctx.getSemTable().getNotSingleRouteErr()); + } + + if (isRoute && ((RouteGen4Plan) plan).isSingleShard()) { + this.planSingleShardRoutePlan(this.sel, (RouteGen4Plan) plan); + return plan; + } + + // If the current plan is a simpleProjection, we want to rewrite derived expression. + // In transformDerivedPlan (operator_transformers.go), derived tables that are not + // a simple route are put behind a simpleProjection. In this simple projection, + // every Route will represent the original derived table. Thus, pushing new expressions + // to those Routes require us to rewrite them. + // On the other hand, when a derived table is a simple Route, we do not put it under + // a simpleProjection. We create a new Route that contains the derived table in the + // FROM clause. Meaning that, when we push expressions to the select list of this + // new Route, we do not want them to rewrite them. + + this.qp = new QueryProjection(); + this.qp.createQPFromSelect(this.sel); + + boolean needsOrdering = !this.qp.getOrderExprs().isEmpty(); + if (this.sel instanceof MySqlSelectQueryBlock) { + SQLSelectGroupByClause groupBy = ((MySqlSelectQueryBlock) this.sel).getGroupBy(); + if (groupBy != null) { + this.hasHaving = groupBy.getHaving() != null; + } + } + + boolean canShortcut = isRoute && !hasHaving && !needsOrdering; + + // If we still have a HAVING clause, it's because it could not be pushed to the WHERE, + // so it probably has aggregations + if (qp.needsAggregating() || hasHaving) { + plan = this.planAggregations(ctx, plan); + // if we already did sorting, we don't need to do it again + needsOrdering = needsOrdering && !this.qp.isCanPushDownSorting(); + } else if (canShortcut) { + planSingleShardRoutePlan(this.sel, (RouteGen4Plan) plan); + } else { + this.pushProjections(ctx, plan, this.qp.selectExprs); + } + + // If we didn't already take care of ORDER BY during aggregation planning, we need to handle it now + if (needsOrdering) { + // this.planOr + plan = this.planOrderBy(ctx, this.qp.orderExprs, plan); + } + + plan = this.planDistinct(ctx, plan); + + if (!truncateColumns) { + return plan; + } + plan = this.truncateColumnsIfNeeded(ctx, plan); + + return plan; + } + + public static void planSingleShardRoutePlan(SQLSelectQuery sel, RouteGen4Plan rb) { + stripDownQuery(sel, rb.select); + VtRemoveDbNameExpectSystemDbVisitor visitor = new VtRemoveDbNameExpectSystemDbVisitor(); + rb.select.accept(visitor); + } + + private static void stripDownQuery(SQLSelectQuery from, SQLSelectQuery to) { + VtRemoveDbNameInColumnVisitor visitor = new VtRemoveDbNameInColumnVisitor(); + if (from instanceof MySqlSelectQueryBlock) { + ((MySqlSelectQueryBlock) to).setGroupBy(((MySqlSelectQueryBlock) from).getGroupBy()); + ((MySqlSelectQueryBlock) to).setOrderBy(((MySqlSelectQueryBlock) from).getOrderBy()); + SQLOrderBy orderBy = ((MySqlSelectQueryBlock) to).getOrderBy(); + if (orderBy != null) { + for (SQLSelectOrderByItem item : orderBy.getItems()) { + boolean isSpecialOrderBy = isSpecialOrderBy(item.getExpr()); + if (item.getType() == null && !isSpecialOrderBy) { + item.setType(SQLOrderingSpecification.ASC); + } + } + } + ((MySqlSelectQueryBlock) to).setHints(((MySqlSelectQueryBlock) from).getHints()); + ((MySqlSelectQueryBlock) to).getSelectList().addAll(((MySqlSelectQueryBlock) from).getSelectList()); + for (SQLSelectItem expr : ((MySqlSelectQueryBlock) to).getSelectList()) { + expr.accept(visitor); + } + + } else if (from instanceof SQLUnionQuery) { + stripDownQuery(((SQLUnionQuery) from).getLeft(), ((SQLUnionQuery) to).getLeft()); + stripDownQuery(((SQLUnionQuery) from).getRight(), ((SQLUnionQuery) to).getRight()); + } else { + + } + } + + private LogicalPlan planAggregations(PlanningContext ctx, LogicalPlan plan) throws SQLException { + boolean isPushable = !this.isJoin(plan); + List grouping = this.qp.getGrouping(); + boolean vindexOverlapWithGrouping = this.hasUniqueVindex(ctx.getSemTable(), grouping); + if (isPushable && vindexOverlapWithGrouping) { + // If we have a plan that we can push the group by and aggregation through, we don't need to do aggregation + // at the vtgate level at all + this.planAggregationWithoutOA(ctx, plan); + LogicalPlan resultPlan = this.planOrderBy(ctx, this.qp.orderExprs, plan); + LogicalPlan newPlan = this.planHaving(ctx, resultPlan); + return newPlan; + } + + return this.planAggrUsingOA(ctx, plan, grouping); + } + + public static boolean isJoin(LogicalPlan plan) { + if (plan instanceof JoinGen4Plan) { + return true; + } + + if (plan instanceof HashJoinPlan) { + return true; + } + + /* + switch plan.(type) { + case *joinGen4, *hashJoin: + return true + default: + return false + } + */ + //TODO + return false; + } + + private boolean exprHasVindex(SemTable semTable, SQLExpr expr, boolean hasToBeUnique) { + boolean isCol = expr instanceof SQLName; + if (!isCol) { + return false; + } + TableSet ts = semTable.recursiveDeps(expr); + if (ts == null) { + return false; + } + TableInfo tableInfo = semTable.tableInfoFor(ts); + if (tableInfo == null) { + return false; + } + Vschema.Table vschemaTable = tableInfo.getVindexTable(); + for (Vschema.ColumnVindex vindex : vschemaTable.getColumnVindexesList()) { + // if len(vindex.Columns) > 1 || hasToBeUnique && !vindex.IsUnique() { + if (vschemaTable.getColumnVindexesList().size() > 1 || hasToBeUnique && false) { + return false; + } + if (Objects.equals(((SQLName) expr).getSimpleName(), vindex.getColumn())) { + return true; + } + } + return false; + } + + private boolean exprHasUniqueVindex(SemTable semTable, SQLExpr expr) { + return exprHasVindex(semTable, expr, true); + } + + private boolean hasUniqueVindex(SemTable semTable, List groupByExprs) { + for (QueryProjection.GroupBy groupByExpr : groupByExprs) { + if (this.exprHasUniqueVindex(semTable, groupByExpr.weightStrExpr)) { + return true; + } + } + return false; + } + + private LogicalPlan planAggrUsingOA(PlanningContext ctx, LogicalPlan plan, List grouping) throws SQLException { + OrderedAggregateGen4Plan oa = new OrderedAggregateGen4Plan(new ArrayList<>(grouping.size())); + List order; + + if (this.qp.isCanPushDownSorting()) { + this.qp.alignGroupByAndOrderBy(); + // the grouping order might have changed, so we reload the grouping expressions + grouping = this.qp.getGrouping(); + order = this.qp.orderExprs; + } else { + order = new ArrayList<>(grouping.size()); + for (QueryProjection.GroupBy expr : grouping) { + order.add(expr.asOrderBy()); + } + } + + // here we are building up the grouping keys for the OA, + // but they are lacking the input offsets because we have yet to push the columns down + for (QueryProjection.GroupBy expr : grouping) { + GroupByParams groupByParams = new GroupByParams(expr.inner, true); + oa.getGroupByKeys().add(groupByParams); + } + + rewriterHaving(); + + List aggregationExprs = this.qp.aggregationExpressions(); + + // If we have a distinct aggregating expression, + // we handle it by pushing it down to the underlying input as a grouping column + ImmutableTriple, List, List> distinctAggr = handleDistinctAggr(ctx, aggregationExprs); + List distinctGroupBy = distinctAggr.getLeft(); + List distinctOffsets = distinctAggr.getMiddle(); + List aggrs = distinctAggr.getRight(); + + if (distinctGroupBy.size() > 0) { + grouping.addAll(distinctGroupBy); + // all the distinct grouping aggregates use the same expression, so it should be OK to just add it once + order.add(distinctGroupBy.get(0).asOrderBy()); + oa.setPreProcess(true); + } + + PushAggregationResult pushAggregationResult = this.pushAggregation(ctx, plan, grouping, aggrs, false); + if (!pushAggregationResult.isPushed()) { + oa.setPreProcess(true); + oa.setAggrOnEngine(true); + } + plan = pushAggregationResult.getOutput(); + + LogicalPlan aggPlan = plan; + ProjectionGen4Plan proj = null; + boolean isRoute = plan instanceof RouteGen4Plan; + boolean needsProj = !isRoute; + if (needsProj) { + aggPlan = proj; + } + List aggrParams = generateAggregateParams(aggrs, pushAggregationResult.getOutputAggrsOffset(), proj, pushAggregationResult.isPushed()); + if (proj != null) { + + } + + // Next we add the aggregation expressions and grouping offsets to the OA + addColumnsToOA(ctx, oa, distinctGroupBy, aggrParams, distinctOffsets, pushAggregationResult.getGroupingOffsets(), aggregationExprs); + + aggPlan = planOrderBy(ctx, order, aggPlan); + oa.setInput(aggPlan); + + return this.planHaving(ctx, oa); + } + + /** + * @param ctx + * @param oa + * @param distinctGroupBy these are the group by expressions that where added because we have unique aggregations + * @param aggrParams these are the aggregate params we already have for non-distinct aggregations + * @param distinctOffsets distinctOffsets mark out where we need to use the distinctGroupBy offsets to create *engine.AggregateParams for the distinct aggregations + * @param groupings these are the offsets for the group by params + * @param aggregationExprs aggregationExprs are all the original aggregation expressions the query requested + */ + private void addColumnsToOA(PlanningContext ctx, OrderedAggregateGen4Plan oa, List distinctGroupBy, List aggrParams, + List distinctOffsets, + List groupings, List aggregationExprs) { + if (distinctGroupBy.size() == 0) { + oa.setAggregates(aggrParams); + } else { + int count = groupings.size() - distinctOffsets.size(); + int lastOffset = distinctOffsets.get(distinctOffsets.size() - 1); + int distinctIdx = 0; + for (int i = 0; i <= lastOffset || i <= aggrParams.size(); i++) { + while (distinctIdx < distinctOffsets.size() && i == distinctOffsets.get(distinctIdx)) { + // we loop here since we could be dealing with multiple distinct aggregations after each other + Offsets groupOffset = groupings.get(count); + count++; + QueryProjection.Aggr aggrExpr = aggregationExprs.get(i); + List arguments = aggrExpr.getFunc().getArguments(); + SQLExpr sqlExpr = CollectionUtils.isEmpty(arguments) ? null : arguments.get(0); + int colId = ctx.getSemTable().collationForExpr(sqlExpr); + AbstractAggregateGen4.AggregateParams addAggregate = new AbstractAggregateGen4.AggregateParams(); + addAggregate.setCol(groupOffset.getCol()); + addAggregate.setKeyCol(groupOffset.getCol()); + addAggregate.setWAssigned(groupOffset.getWsCol() >= 0); + addAggregate.setWCol(groupOffset.getWsCol()); + addAggregate.setOpcode(aggrExpr.getOpCode()); + addAggregate.setAlias(aggrExpr.getAlias()); + addAggregate.setOriginal(aggrExpr.getOriginal()); + addAggregate.setCollationId(colId); + + oa.getAggregates().add(addAggregate); + + distinctIdx++; + } + if (i < aggrParams.size()) { + oa.getAggregates().add(aggrParams.get(i)); + } + } + // we have to remove the tail of the grouping offsets, so we only have the offsets for the GROUP BY in the query + groupings = groupings.subList(0, groupings.size() - distinctOffsets.size()); + } + for (int i = 0; i < groupings.size(); i++) { + Offsets offsets = groupings.get(i); + List groupByKeys = oa.getGroupByKeys(); + GroupByParams groupByParams = groupByKeys.get(i); + groupByParams.setKeyCol(offsets.getCol()); + groupByParams.setWeightStringCol(offsets.getWsCol()); + } + } + + private ImmutableTriple, List, List> handleDistinctAggr(PlanningContext ctx, List exprs) throws SQLException { + SQLExpr distinctExpr = null; + List distincts = new ArrayList<>(); + List offsets = new ArrayList<>(); + List aggrs = new ArrayList<>(); + + for (int i = 0; i < exprs.size(); i++) { + QueryProjection.Aggr expr = exprs.get(i); + if (!expr.getDistinct()) { + aggrs.add(expr); + continue; + } + List arguments = expr.getFunc().getArguments(); + SQLExpr sqlExpr = CollectionUtils.isEmpty(arguments) ? null : arguments.get(0); + SQLExpr[] innerAndInnerWS = this.getQp().getSimplifiedExpr(sqlExpr); + if (exprHasVindex(ctx.getSemTable(), innerAndInnerWS[1], false)) { + aggrs.add(expr); + continue; + } + if (distinctExpr == null) { + distinctExpr = innerAndInnerWS[1]; + } else { + if (!SQLExprUtils.equals(distinctExpr, innerAndInnerWS[1])) { + throw new SQLException("unsupported: only one distinct aggregation allowed in a select: " + expr.getOriginal()); + } + } + distincts.add(new QueryProjection.GroupBy(innerAndInnerWS[0], innerAndInnerWS[1], expr.getIndex(), null)); + offsets.add(i); + } + return new ImmutableTriple<>(distincts, offsets, aggrs); + } + + private List generateAggregateParams(List aggrs, List> aggrParamOffsets, ProjectionGen4Plan proj, boolean pushed) { + List aggrParams = new ArrayList<>(aggrs.size()); + for (int idx = 0; idx < aggrParamOffsets.size(); idx++) { + List paramOffset = aggrParamOffsets.get(idx); + QueryProjection.Aggr aggr = aggrs.get(idx); + Integer incomingOffset = paramOffset.get(0).getCol(); + Integer offset = null; + if (proj != null) { + + } else { + offset = incomingOffset; + } + Engine.AggregateOpcodeG4 opCode = Engine.AggregateOpcodeG4.AggregateSum; + switch (aggr.getOpCode()) { + case AggregateMin: + case AggregateMax: + case AggregateRandom: + opCode = aggr.getOpCode(); + break; + case AggregateCount: + case AggregateCountDistinct: + case AggregateSumDistinct: + case AggregateSum: +// case engine.AggregateCountStar + if (!pushed) { + opCode = aggr.getOpCode(); + } + break; + default: + } + AbstractAggregateGen4.AggregateParams aggregateParams = new AbstractAggregateGen4.AggregateParams(opCode, offset, aggr.getAlias(), aggr.getOriginal().getExpr(), aggr.getOriginal()); + aggregateParams.setOrigOpcode(aggr.getOpCode()); + aggrParams.add(idx, aggregateParams); + } + return aggrParams; + } + + /** + * pushAggregation pushes grouping and aggregation as far down in the tree as possible + * the output `outputAggrsOffset` needs a little explaining: this is the offsets for aggregation - remember + * that aggregation can be broken down into multiple expressions that are later combined. + * this is why this output is a slice of slices + * + * @param ctx + * @param plan + * @param grouping + * @param aggregations + * @param ignoreOutputOrder + * @return + * @throws SQLException + */ + private PushAggregationResult pushAggregation(PlanningContext ctx, LogicalPlan plan, List grouping, List aggregations, + boolean ignoreOutputOrder) throws SQLException { + if (plan instanceof RouteGen4Plan) { + PushAggregationResult pushAggregationResult = new PushAggregationResult(); + pushAggregationResult.setPushed(true); + pushAggregationResult.setOutput(plan); + Pair, List>> aReturn = pushAggrOnRoute(ctx, (RouteGen4Plan) plan, aggregations, grouping, ignoreOutputOrder); + pushAggregationResult.setGroupingOffsets(aReturn.getLeft()); + pushAggregationResult.setOutputAggrsOffset(aReturn.getRight()); + return pushAggregationResult; + } + if (plan instanceof JoinGen4Plan) { + PushAggregationResult pushAggregationResult = new PushAggregationResult(); + pushAggregationResult.setPushed(true); + pushAggregationResult.setOutput(plan); + Pair, List>> aReturn = pushAggrOnJoin(ctx, (JoinGen4Plan) plan, grouping, aggregations); + pushAggregationResult.setGroupingOffsets(aReturn.getLeft()); + pushAggregationResult.setOutputAggrsOffset(aReturn.getRight()); + return pushAggregationResult; + } + // TODO Semi join + + // TODO Simple Projection + + if (plan instanceof LimitGen4Plan) { + // if we are seeing a limit, it's because we are building on top of a derived table. + PushAggregationResult pushAggregationResult = new PushAggregationResult(); + pushAggregationResult.setPushed(true); + pushAggregationResult.setOutput(plan); + + List groupingOffsets = new ArrayList<>(grouping.size()); + List> outputAggrsOffset = new ArrayList<>(); + + for (QueryProjection.GroupBy grp : grouping) { + Pair result = this.wrapAndPushExpr(ctx, grp.getInner(), grp.getWeightStrExpr(), ((LimitGen4Plan) plan).getInput()); + int offset = result.getLeft(); + int wOffset = result.getRight(); + groupingOffsets.add(new Offsets(offset, wOffset)); + } + + for (QueryProjection.Aggr aggr : aggregations) { + int offset = 0; + if (aggr.getOriginal().getExpr() instanceof SQLAggregateExpr) { + SQLAggregateExpr aggrExpr = (SQLAggregateExpr) aggr.getOriginal().getExpr(); + Engine.AggregateOpcodeG4 opcode = AbstractAggregateGen4.SUPPORTED_AGGREGATES.get(aggrExpr.getMethodName().toLowerCase()); + if (opcode == Engine.AggregateOpcodeG4.AggregateCountStar) { // countstar count(*) ?? + //TODO + } else { + if (aggrExpr.getArguments().size() != 1) { + throw new SQLException("[BUG]: unexpected expression: " + aggrExpr.toString()); + } + + Pair ret = ProjectionPushing.pushProjection(ctx, new SQLSelectItem(aggrExpr.getArguments().get(0)), ((LimitGen4Plan) plan).getInput(), true, true, false); + offset = ret.getLeft(); + } + Offsets[] offsets = {new Offsets(offset, -1)}; + outputAggrsOffset.add(Arrays.asList(offsets)); + } else { + throw new SQLException("[BUG] unexpected expression:" + aggr.getOriginal().toString()); + } + } + + pushAggregationResult.setGroupingOffsets(groupingOffsets); + pushAggregationResult.setOutputAggrsOffset(outputAggrsOffset); + return pushAggregationResult; + } + throw new SQLFeatureNotSupportedException("using aggregation on top of a plan is not yet supported"); + } + + private Pair, List>> pushAggrOnRoute(PlanningContext ctx, RouteGen4Plan plan, List aggregations, List grouping, + boolean ignoreOutputOrder) throws SQLException { + boolean columnOrderMatters = !ignoreOutputOrder; + if (!(plan.getSelect() instanceof MySqlSelectQueryBlock)) { + throw new SQLFeatureNotSupportedException("can't plan aggregation on union"); + } + MySqlSelectQueryBlock sel = (MySqlSelectQueryBlock) plan.getSelect(); + + List groupingCols = new ArrayList<>(); + List> aggregation = new ArrayList<>(); + AggregationPushing.Func reorg = null; + if (columnOrderMatters) { + // During this first run, we push the projections for the normal columns (not the weigh_string ones, that is) + // in the order that the user asked for it + // sortOffsets also returns a reorgFunc, + // that can be used to rearrange the produced outputs to the original order + Triple, AggregationPushing.Func, SortedIterator> sortOffsetsResult = sortOffsets(grouping, aggregations); + reorg = sortOffsetsResult.getMiddle(); + grouping = sortOffsetsResult.getLeft(); + Pair>, List> orderReturn = pushAggrsAndGroupingInOrder(ctx, plan, sortOffsetsResult.getRight(), sel, aggregation, groupingCols); + aggregation = orderReturn.getLeft(); + groupingCols = orderReturn.getRight(); + } else { + // if we haven't already pushed the aggregations, now is the time + for (QueryProjection.Aggr aggr : aggregations) { + Offsets param = addAggregationToSelect(sel, aggr); + aggregation.add(Collections.singletonList(param)); + } + } + + List groupingOffsets = new ArrayList<>(grouping.size()); + for (int idx = 0; idx < grouping.size(); idx++) { + QueryProjection.GroupBy expr = grouping.get(idx); + addGroupBy(sel, expr.getInner()); + Offsets pos; + if (ignoreOutputOrder) { + // we have not yet pushed anything, so we need to push the expression first + Integer col = addExpressionToRoute(ctx, plan, new SQLSelectItem(expr.getInner()), true); + pos = newOffset(col); + } else { + pos = newOffset(groupingCols.get(idx)); + } + + if (expr.getWeightStrExpr() != null && ctx.getSemTable().needsWeightString(expr.getInner())) { + SQLExpr wsExpr = this.weightStringFor(expr.getWeightStrExpr()); + + Integer wsCol = addExpressionToRoute(ctx, plan, new SQLSelectItem(wsExpr), true); + pos.setWsCol(wsCol); + addGroupBy(sel, wsExpr); + } + groupingOffsets.add(pos); + } + Pair, List>> pushAggrsAndGroupingResult = reorg.passThrough(groupingOffsets, aggregation); + return pushAggrsAndGroupingResult; + } + + /** + * We push down aggregations using the logic from the paper Orthogonal Optimization of Subqueries and Aggregation, by + * Cesar A. Galindo-Legaria and Milind M. Joshi from Microsoft Corp. + *

+ * It explains how one can split an aggregation into local aggregates that depend on only one side of the join. + * The local aggregates can then be gathered together to produce the global + * group by/aggregate query that the user asked for. + *

+ * In Vitess, this is particularly useful because it allows us to push aggregation down to the routes, even when + * we have to join the results at the vtgate level. Instead of doing all the grouping and aggregation at the + * vtgate level, we can offload most of the work to MySQL, and at the vtgate just summarize the results. + * + * @param ctx + * @param join + * @param grouping + * @param aggregations + * @return + */ + private Pair, List>> pushAggrOnJoin(PlanningContext ctx, JoinGen4Plan join, List grouping, List aggregations) + throws SQLException { + Pair, List> splitRes = AggregationPushing.splitAggregationsToLeftAndRight(ctx, aggregations, join); + + List lhsAggr = splitRes.getLeft(); + List rhsAggr = splitRes.getRight(); + + + // We need to group by the columns used in the join condition. + // If we don't, the LHS will not be able to return the column, and it can't be used to send down to the RHS + List lhsCols = this.createGroupingsForColumns(join.getLHSColumns()); + + // Here we split the grouping depending on if they should with the LHS or RHS of the query + // This is done by using the semantic table and checking dependencies + Triple, List, List> tripRet = AggregationPushing.splitGroupingsToLeftAndRight(ctx, join, grouping, lhsCols); + + List lhsGrouping = tripRet.getLeft(); + List rhsGrouping = tripRet.getMiddle(); + List groupingOffsets = tripRet.getRight(); + + + // If the rhs has no grouping column then a count(*) will return 0 from the query and will get mapped to the record from left hand side. + // This is an incorrect behaviour as the join condition has not matched, so we add a literal 1 to the select query and also group by on it. + // So that only if join condition matches the records will be mapped and returned. + if (rhsGrouping.size() == 0 && rhsAggr.size() != 0) { + SQLIntegerExpr l = new SQLIntegerExpr(1); + SQLSelectItem aExpr = new SQLSelectItem(l); + Pair ret = ProjectionPushing.pushProjection(ctx, aExpr, join.getRight(), true, true, false); + int offset = ret.getLeft(); + l.setNumber(offset + 1); + rhsGrouping.add(new QueryProjection.GroupBy(l)); + } + + // Next we push the aggregations to both sides + + PushAggregationResult leftResult = this.filteredPushAggregation(ctx, join.getLeft(), lhsGrouping, lhsAggr, true); + PushAggregationResult rightResult = this.filteredPushAggregation(ctx, join.getRight(), rhsGrouping, rhsAggr, true); + + join.setLeft(leftResult.getOutput()); + join.setRight(rightResult.getOutput()); + + // Next, we have to pass through the grouping values through the join and the projection we add on top + // We added new groupings to the LHS because of the join condition, so we don't want to pass through everything, + // just the groupings that are used by operators on top of this current one + + int wsOutputGrpOffset = groupingOffsets.size() + join.getCols().size(); + List outputGroupings = new ArrayList<>(groupingOffsets.size()); + + List wsOffsets = new ArrayList<>(); + + for (int groupBy : groupingOffsets) { + Offsets offset = null; + int fac = 1; + if (groupBy < 0) { + offset = leftResult.getGroupingOffsets().get(-groupBy - 1); + fac = -1; + } else { + offset = rightResult.getGroupingOffsets().get(groupBy - 1); + } + Offsets outputGrouping = newOffset(join.getCols().size()); + join.getCols().add(offset.getCol() * fac); + + if (offset.getWsCol() > -1) { + // we add the weight_string calls at the end of the join columns + outputGrouping.setWsCol(wsOutputGrpOffset + wsOffsets.size()); + wsOffsets.add(offset.getWsCol() * fac); + } + outputGroupings.add(outputGrouping); + } + join.getCols().addAll(wsOffsets); + + + List> outputAggrOffsets = new ArrayList<>(aggregations.size()); + for (int idx = 0; idx < aggregations.size(); idx++) { + List l = leftResult.getOutputAggrsOffset().get(idx); + List r = rightResult.getOutputAggrsOffset().get(idx); + + List offSlice = new ArrayList<>(); + + for (Offsets off : l) { + offSlice.add(newOffset(join.getCols().size())); + join.getCols().add(-(off.getCol() + 1)); + } + + for (Offsets off : r) { + offSlice.add(newOffset(join.getCols().size())); + join.getCols().add(off.getCol() + 1); + } + + outputAggrOffsets.add(offSlice); + } + return Pair.of(outputGroupings, outputAggrOffsets); + } + + /** + * this method takes a slice of aggregations that can have missing spots in the form of `nil`, + * and pushes the non-empty values down. + * during aggregation planning, it's important to know which of + * the incoming aggregations correspond to what is sent to the LHS and RHS. + * Some aggregations only need to be sent to one of the sides of the join, and in that case, + * the other side will have a nil in this offset of the aggregations + * + * @param ctx + * @param plan + * @param grouping + * @param aggregations + * @param ignoreOutputOrder + * @return + */ + private PushAggregationResult filteredPushAggregation( + PlanningContext ctx, LogicalPlan plan, List grouping, List aggregations, Boolean ignoreOutputOrder + ) throws SQLException { + List used = new ArrayList<>(aggregations.size()); + + List aggrs = new ArrayList<>(aggregations.size()); + + for (int idx = 0; idx < aggregations.size(); idx++) { + if (aggregations.get(idx) != null) { + used.add(true); + aggrs.add(aggregations.get(idx)); + } else { + used.add(false); + } + } + + PushAggregationResult result = this.pushAggregation(ctx, plan, grouping, aggrs, ignoreOutputOrder); + + PushAggregationResult ret = new PushAggregationResult(); + ret.setOutput(result.getOutput()); + ret.setPushed(result.isPushed()); + ret.setGroupingOffsets(result.getGroupingOffsets()); + ret.setOutputAggrsOffset(new ArrayList<>()); + + int idx = 0; + for (Boolean b : used) { + if (!b) { + ret.getOutputAggrsOffset().add(null); + continue; + } + ret.getOutputAggrsOffset().add(result.getOutputAggrsOffset().get(idx)); + idx++; + } + return ret; + } + + private void addGroupBy(MySqlSelectQueryBlock sel, SQLExpr inner) { + if (sel.getGroupBy() == null) { + SQLSelectGroupByClause newGroupBy = new SQLSelectGroupByClause(); + sel.setGroupBy(newGroupBy); + } + for (SQLExpr item : sel.getGroupBy().getItems()) { + if (Objects.equals(item, inner)) { + return; + } + } + sel.getGroupBy().addItem(inner); + } + + private Pair>, List> pushAggrsAndGroupingInOrder(PlanningContext ctx, RouteGen4Plan plan, SortedIterator sortedIterator, MySqlSelectQueryBlock sel, + List> vtgateAggregation, + List groupingCols) throws SQLException { + while (sortedIterator.hasNext()) { + SortedIterator current = sortedIterator.next(); + QueryProjection.Aggr aggregation = current.getValueA(); + QueryProjection.GroupBy groupBy = current.getValueGB(); + if (aggregation != null) { + Offsets param = addAggregationToSelect(sel, aggregation); + vtgateAggregation.add(Collections.singletonList(param)); + continue; + } + if (groupBy != null) { + boolean reuseCol = groupBy.getInnerIndex() == null; + Integer col = addExpressionToRoute(ctx, plan, groupBy.asAliasedExpr(), reuseCol); + groupingCols.add(col); + } + } + return new ImmutablePair<>(vtgateAggregation, groupingCols); + } + + private Integer addExpressionToRoute(PlanningContext ctx, RouteGen4Plan rb, SQLSelectItem asAliasedExpr, boolean reuseCol) throws SQLException { + if (reuseCol) { + int offset = checkIfAlreadyExists(asAliasedExpr, rb.getSelect(), ctx.getSemTable()); + if (offset != -1) { + return offset; + } + } +// expr.Expr = sqlparser.RemoveKeyspaceFromColName(expr.Expr) +// PlanBuilder.removeKeyspaceFromColName(asAliasedExpr.getExpr()); + if (!(rb.getSelect() instanceof MySqlSelectQueryBlock)) { + throw new SQLFeatureNotSupportedException("unsupported: pushing projection"); + } + if (ctx.isRewriteDerivedExpr()) { + // if we are trying to push a projection that belongs to a DerivedTable + // we rewrite that expression, so it matches the column name used inside + // that derived table. + } + MySqlSelectQueryBlock sel = (MySqlSelectQueryBlock) rb.getSelect(); + Integer offset = sel.getSelectList().size(); + sel.getSelectList().add(asAliasedExpr); + return offset; + } + + /** + * addAggregationToSelect adds the aggregation to the SELECT statement and returns the AggregateParams to be used outside + * + * @param sel + * @param aggregation + * @return + */ + private Offsets addAggregationToSelect(MySqlSelectQueryBlock sel, QueryProjection.Aggr aggregation) { + // TODO: removing duplicated aggregation expression should also be done at the join level + for (int i = 0; i < sel.getSelectList().size(); i++) { + SQLSelectItem selectItem = sel.getSelectList().get(i); + SqlParser.SelectExpr selectExpr = SqlParser.SelectExpr.type(selectItem); + if (!Objects.equals(SqlParser.SelectExpr.AliasedExpr, selectExpr)) { + continue; + } + + if (Objects.equals(selectItem.getExpr(), aggregation.getOriginal().getExpr())) { + return newOffset(i); + } + } + sel.addSelectItem(aggregation.getOriginal()); + return newOffset(sel.getSelectList().size() - 1); + } + + private Offsets newOffset(int col) { + return new Offsets(col, -1); + } + + private List createGroupingsForColumns(List columns) { + List lhsGrouping = new ArrayList<>(); + for (SQLName lhsColumn : columns) { + SQLExpr[] ret = this.qp.getSimplifiedExpr(lhsColumn); + lhsGrouping.add(new QueryProjection.GroupBy( + ret[0], + ret[1], + 0, + null + )); + } + return lhsGrouping; + } + + private Triple, AggregationPushing.Func, SortedIterator> sortOffsets(List grouping, List aggregations) { + List originalGrouping = new ArrayList<>(grouping); + List originalAggr = new ArrayList<>(aggregations); + qp.sortAggregations(aggregations); + qp.sortGrouping(grouping); + + AggregationPushing.Func reorg = (groupByOffsets, aggrOffsets) -> { + List orderedGroupingOffsets = new ArrayList<>(originalGrouping.size()); + for (QueryProjection.GroupBy og : originalGrouping) { + for (int i = 0; i < grouping.size(); i++) { + QueryProjection.GroupBy g = grouping.get(i); + if (og.getInner() == g.getInner()) { + orderedGroupingOffsets.add(groupByOffsets.get(i)); + } + } + } + List> orderedAggrs = new ArrayList<>(originalGrouping.size()); + for (QueryProjection.Aggr og : originalAggr) { + for (int i = 0; i < aggregations.size(); i++) { + QueryProjection.Aggr g = aggregations.get(i); + if (og.getOriginal() == g.getOriginal()) { + orderedAggrs.add(aggrOffsets.get(i)); + } + } + } + return new ImmutablePair(orderedGroupingOffsets, orderedAggrs); + }; + + SortedIterator sortedIterator = new SortedIterator(grouping, aggregations); + return new ImmutableTriple<>(grouping, reorg, sortedIterator); + } + + private void planAggregationWithoutOA(PlanningContext ctx, LogicalPlan plan) throws SQLException { + for (QueryProjection.SelectExpr expr : this.qp.getSelectExprs()) { + SQLSelectItem selectItem = expr.getAliasedExpr(); + ProjectionPushing.pushProjection(ctx, selectItem, plan, true, false, false); + } + for (QueryProjection.GroupBy expr : this.qp.getGrouping()) { + // since all the grouping will be done at the mysql level, + // we know that we won't need any weight_string() calls + this.planGroupByGen4(ctx, expr, plan /*weighString*/, false); + } + } + + private void pushProjections(PlanningContext ctx, LogicalPlan plan, List selectExprs) throws SQLException { + for (QueryProjection.SelectExpr e : selectExprs) { + ProjectionPushing.pushProjection(ctx, e.getAliasedExpr(), plan, true, false, false); + } + } + + public LogicalPlan planOrderBy(PlanningContext ctx, List orderExprs, LogicalPlan plan) throws SQLException { + if (plan instanceof RouteGen4Plan) { + LogicalPlan newPlan = this.planOrderByForRoute(ctx, orderExprs, (RouteGen4Plan) plan, this.qp.hasStar); + return newPlan; + } else if (plan instanceof OrderedAggregateGen4Plan) { + // remove ORDER BY NULL from the list of order by expressions since we will be doing the ordering on vtgate level so NULL is not useful + List orderExprsWithoutNils = new ArrayList<>(orderExprs.size()); + for (QueryProjection.OrderBy expr : orderExprs) { + if (expr.inner.getExpr() == null) { + continue; + } + orderExprsWithoutNils.add(expr); + } + orderExprs = orderExprsWithoutNils; + CheckNodeTypesVisitor visitor = new CheckNodeTypesVisitor(Sets.newHashSet(CheckNodeTypesVisitor.CheckNodeType.AGGREGATE)); + for (QueryProjection.OrderBy expr : orderExprs) { + expr.weightStrExpr.accept(visitor); + boolean hasAggr = visitor.getCheckResult(); + if (hasAggr) { + return createMemorySortPlanOnAggregation(ctx, (OrderedAggregateGen4Plan) plan, orderExprs); + } + } + LogicalPlan newInput = planOrderBy(ctx, orderExprs, ((OrderedAggregateGen4Plan) plan).getInput()); + ((OrderedAggregateGen4Plan) plan).setInput(newInput); + return plan; + + } else if (plan instanceof MemorySortPlan) { + return plan; + } else if (plan instanceof JoinGen4Plan) { + return this.planOrderByForJoin(ctx, orderExprs, (JoinGen4Plan) plan); + } else { + throw new SQLException("ordering on complex query " + plan.toString()); + } + } + + private LogicalPlan planHaving(PlanningContext ctx, LogicalPlan plan) throws SQLException { + if (this.sel instanceof MySqlSelectQueryBlock) { + MySqlSelectQueryBlock mySqlSelectQueryBlock = (MySqlSelectQueryBlock) this.sel; + if (mySqlSelectQueryBlock.getGroupBy() != null && mySqlSelectQueryBlock.getGroupBy().getHaving() != null) { + return pushHaving(ctx, mySqlSelectQueryBlock.getGroupBy().getHaving(), plan); + } + } + return plan; + } + + private LogicalPlan pushHaving(PlanningContext ctx, SQLExpr expr, LogicalPlan plan) throws SQLException { + if (plan instanceof RouteGen4Plan) { + SQLSelectQuery selectQuery = ((RouteGen4Plan) plan).getSelect(); + MySqlSelectQueryBlock planSelect = PlanBuilder.getFirstSelect(selectQuery); + planSelect.getGroupBy().addHaving(expr); + return plan; + } else if (plan instanceof OrderedAggregateGen4Plan) { + return new FilterGen4Plan(ctx, plan, expr); + } else { + throw new SQLException("[BUG] unreachable filtering: " + plan.toString()); + } + } + + private void rewriterHaving() { + if (!this.hasHaving) { + return; + } + if (this.sel instanceof MySqlSelectQueryBlock) { + SQLExpr havingExpr = ((MySqlSelectQueryBlock) this.sel).getGroupBy().getHaving(); + RewriteHavingAggrWithOffsetVisitor visitor = new RewriteHavingAggrWithOffsetVisitor(this.getQp()); + havingExpr.accept(visitor); + } + } + + private LogicalPlan planOrderByForRoute(PlanningContext ctx, List orderExprs, RouteGen4Plan plan, boolean hasStar) throws SQLException { + for (QueryProjection.OrderBy order : orderExprs) { + boolean isSpecialOrderBy = isSpecialOrderBy(order.inner.getExpr()); + if (order.inner.getType() == null && !isSpecialOrderBy) { + order.inner.setType(SQLOrderingSpecification.ASC); + } + + // AddOrder + if (plan.select instanceof MySqlSelectQueryBlock) { + MySqlSelectQueryBlock selectQueryBlock = (MySqlSelectQueryBlock) plan.select; + if (selectQueryBlock.getOrderBy() == null) { + selectQueryBlock.setOrderBy(new SQLOrderBy()); + } + selectQueryBlock.getOrderBy().addItem(order.inner); + } else if (plan.select instanceof SQLUnionQuery) { + SQLUnionQuery selectUnionQuery = (SQLUnionQuery) plan.select; + if (selectUnionQuery.getOrderBy() == null) { + SQLOrderBy sqlOrderBy = new SQLOrderBy(); + selectUnionQuery.setOrderBy(sqlOrderBy); + } + selectUnionQuery.getOrderBy().addItem(order.inner); + } + + if (isSpecialOrderBy) { + continue; + } + SQLExpr wsExpr = null; + if (ctx.getSemTable().needsWeightString(order.inner.getExpr())) { + wsExpr = order.weightStrExpr; + } + Pair wsResult = this.wrapAndPushExpr(ctx, order.inner.getExpr(), wsExpr, plan); + plan.eroute.getOrderBy().add(new OrderByParamsGen4(wsResult.getLeft(), order.inner.getType() == SQLOrderingSpecification.DESC, wsResult.getRight(), null)); + } + return plan; + } + + private static boolean isSpecialOrderBy(SQLExpr expr) { + if (expr instanceof SQLNullExpr) { + return true; + } + if (expr instanceof SQLMethodInvokeExpr) { + String methodName = ((SQLMethodInvokeExpr) expr).getMethodName(); + return "rand".equalsIgnoreCase(methodName); + } + return false; + } + + /** + * wrapAndPushExpr pushes the expression and weighted_string function to the plan using semantics.SemTable + * It returns (expr offset, weight_string offset, error) + * + * @param ctx + * @param expr + * @param weightStrExpr + * @param plan + * @return + */ + private Pair wrapAndPushExpr(PlanningContext ctx, SQLExpr expr, SQLExpr weightStrExpr, LogicalPlan plan) throws SQLException { + Pair result = ProjectionPushing.pushProjection(ctx, new SQLSelectItem(expr), plan, true, true, false); + if (weightStrExpr == null) { + //result.setValue(-1); + return Pair.of(result.getLeft(), -1); + } + + if (!(expr instanceof SQLName)) { + if (expr instanceof SQLMethodInvokeExpr) { + String methodName = ((SQLMethodInvokeExpr) expr).getMethodName(); + if ("cast".equalsIgnoreCase(methodName) || "convert".equalsIgnoreCase(methodName)) { + expr = ((SQLMethodInvokeExpr) expr).getParameters().get(0); + } + } + if (!(expr instanceof SQLName)) { + throw new SQLException("unsupported: in scatter query: complex order by expression: " + expr.toString()); + } + } + + Type qt = ctx.getSemTable().typeFor(expr); + boolean wsNeeded = true; + if (qt != null && VtType.isNumber(qt.getType())) { + wsNeeded = false; + } + // int weightStringOffset = -1; + if (wsNeeded) { + SQLExpr aliasedExpr = this.weightStringFor(weightStrExpr); + Pair wsret = ProjectionPushing.pushProjection(ctx, new SQLSelectItem(aliasedExpr), plan, true, true, false); + //result.setValue(wsret.getLeft()); + return Pair.of(result.getLeft(), wsret.getLeft()); + } + return result; + } + + private LogicalPlan planOrderByForJoin(PlanningContext ctx, List orderExprs, JoinGen4Plan plan) throws SQLException { + if (orderExprs.size() == 1 && isSpecialOrderBy(orderExprs.get(0).inner.getExpr())) { + LogicalPlan lhs = this.planOrderBy(ctx, orderExprs, plan.getLeft()); + LogicalPlan rhs = this.planOrderBy(ctx, orderExprs, plan.getRight()); + + plan.setLeft(lhs); + plan.setRight(rhs); + return plan; + } + // We can only push down sorting on the LHS of the join. + // If the order is on the RHS, we need to do the sorting on the vtgate + if (orderExprsDependsOnTableSet(orderExprs, ctx.getSemTable(), plan.getLeft().containsTables())) { + LogicalPlan newLeft = this.planOrderBy(ctx, orderExprs, plan.getLeft()); + plan.setLeft(newLeft); + return plan; + } + LogicalPlan sortPlan = this.createMemorySortPlan(ctx, plan, orderExprs, true); + return sortPlan; + } + + private static Boolean orderExprsDependsOnTableSet(List orderExprs, SemTable semTable, TableSet ts) { + for (QueryProjection.OrderBy expr : orderExprs) { + TableSet exprDependencies = semTable.recursiveDeps(expr.inner.getExpr()); + if (!exprDependencies.isSolvedBy(ts)) { + return false; + } + } + return true; + } + + private LogicalPlan planDistinct(PlanningContext ctx, LogicalPlan plan) throws SQLException { + if (!this.qp.needsDistinct()) { + return plan; + } + if (plan instanceof RouteGen4Plan) { + // we always make the underlying query distinct, + // and then we might also add a distinct operator on top if it is needed + SQLSelectQuery sel = ((RouteGen4Plan) plan).select; + ((MySqlSelectQueryBlock) sel).setDistionOption(SQLSetQuantifier.DISTINCT); + + if (((RouteGen4Plan) plan).isSingleShard() || this.selectHasUniqueVindex(ctx.getSemTable(), this.qp.getSelectExprs())) { + return plan; + } + + return this.addDistinct(ctx, plan); + } else if (plan instanceof OrderedAggregateGen4Plan) { + return this.planDistinctOA(ctx.getSemTable(), (OrderedAggregateGen4Plan) plan); + } else if (plan instanceof DistinctGen4Plan) { + return plan; + } else { + throw new SQLException("unknown plan type for DISTINCT ", plan.toString()); + } + } + + private LogicalPlan planDistinctOA(SemTable semTable, OrderedAggregateGen4Plan currPlan) throws SQLException { + OrderedAggregateGen4Plan oa = new OrderedAggregateGen4Plan(); + oa.setInput(currPlan); + for (QueryProjection.SelectExpr sExpr : this.qp.selectExprs) { + SQLExpr expr = sExpr.getExpr(); + boolean found = false; + for (GroupByParams grpParam : currPlan.getGroupByKeys()) { + if (SQLExprUtils.equals(expr, grpParam.getExpr())) { + found = true; + oa.getGroupByKeys().add(grpParam); + } + } + if (found) { + continue; + } + for (AbstractAggregateGen4.AggregateParams aggrParam : currPlan.getAggregates()) { + if (SQLExprUtils.equals(expr, aggrParam.getExpr())) { + found = true; + oa.getGroupByKeys().add(new GroupByParams(aggrParam.getCol(), -1)); + break; + } + } + if (!found) { + throw new SQLException("[BUG] unable to plan distinct query as the column is not projected: %s" + sExpr.getCol()); + } + } + return oa; + } + + private LogicalPlan createMemorySortPlan(PlanningContext ctx, LogicalPlan plan, List orderExprs, boolean useWeightStr) throws SQLException { + MemorySortGen4Engine primitive = new MemorySortGen4Engine(); + MemorySortGen4Plan memorySortPlan = new MemorySortGen4Plan(); + memorySortPlan.setInput(plan); + memorySortPlan.setTruncater(primitive); + memorySortPlan.setEMemorySort(primitive); + for (QueryProjection.OrderBy orderBy : orderExprs) { + SQLExpr wsExpr = orderBy.weightStrExpr; + if (!useWeightStr) { + wsExpr = null; + } + Pair offset = wrapAndPushExpr(ctx, orderBy.inner.getExpr(), wsExpr, plan); + + boolean isDesc = orderBy.inner.getType() == SQLOrderingSpecification.DESC; + memorySortPlan.getEMemorySort().getOrderByParams().add(new OrderByParamsGen4(offset.getLeft(), isDesc, offset.getRight(), offset.getLeft(), null)); + } + return memorySortPlan; + } + + private LogicalPlan createMemorySortPlanOnAggregation(PlanningContext ctx, OrderedAggregateGen4Plan plan, List orderExprs) throws SQLException { + MemorySortGen4Engine primitive = new MemorySortGen4Engine(); + MemorySortGen4Plan memorySortPlan = new MemorySortGen4Plan(); + memorySortPlan.setInput(plan); + memorySortPlan.setTruncater(primitive); + memorySortPlan.setEMemorySort(primitive); + + for (QueryProjection.OrderBy orderBy : orderExprs) { + MutableTriple offset = findExprInOrderedAggr(plan, orderBy); + if (!offset.getRight()) { + throw new SQLException("expected to find the order by expression in orderedAggregate. Expression: " + orderBy.toString()); + } + boolean isDesc = orderBy.inner.getType() == SQLOrderingSpecification.DESC; + memorySortPlan.getEMemorySort().getOrderByParams().add(new OrderByParamsGen4(offset.getLeft(), isDesc, offset.getMiddle(), offset.getLeft(), null)); + } + return memorySortPlan; + } + + private MutableTriple findExprInOrderedAggr(OrderedAggregateGen4Plan plan, QueryProjection.OrderBy orderBy) { + int keyCol = 0; + int weightStringCol = 0; + boolean found = false; + + for (GroupByParams key : plan.getGroupByKeys()) { + if (SQLExprUtils.equals(orderBy.getWeightStrExpr(), key.getExpr()) || SQLExprUtils.equals(orderBy.getInner().getExpr(), key.getExpr())) { + keyCol = key.getKeyCol(); + weightStringCol = key.getWeightStringCol(); + found = true; + return new MutableTriple<>(keyCol, weightStringCol, found); + } + } + + for (AbstractAggregateGen4.AggregateParams aggregate : plan.getAggregates()) { + if (SQLExprUtils.equals(orderBy.getWeightStrExpr(), aggregate.getOriginal().getExpr()) || SQLExprUtils.equals(orderBy.getInner().getExpr(), aggregate.getOriginal().getExpr())) { + keyCol = aggregate.getCol(); + weightStringCol = -1; + found = true; + return new MutableTriple<>(keyCol, weightStringCol, found); + } + } + return new MutableTriple<>(keyCol, weightStringCol, found); + } + + private void planGroupByGen4(PlanningContext ctx, QueryProjection.GroupBy groupExpr, LogicalPlan plan, boolean wsAdded) throws SQLException { + if (plan instanceof RouteGen4Plan) { + SQLSelectGroupByClause groupByClause = ((MySqlSelectQueryBlock) ((RouteGen4Plan) plan).select).getGroupBy(); + if (groupByClause == null) { + groupByClause = new SQLSelectGroupByClause(); + ((MySqlSelectQueryBlock) ((RouteGen4Plan) plan).select).setGroupBy(groupByClause); + } + groupByClause.addItem(groupExpr.inner); + // If a weight_string function is added to the select list, + // then we need to add that to the group by clause otherwise the query will fail on mysql with full_group_by error + // as the weight_string function might not be functionally dependent on the group by. + if (wsAdded) { + groupByClause.addItem(weightStringFor(groupExpr.weightStrExpr)); + } + } else { + throw new SQLException("unsupported: group by on: " + plan.toString()); + } + } + + private boolean selectHasUniqueVindex(SemTable semTable, List selectExprs) throws SQLException { + for (QueryProjection.SelectExpr expr : selectExprs) { + SQLSelectItem exp = expr.getAliasedExpr(); + if (exp == null) { + // TODO: handle star expression error + return false; + } + if (this.exprHasUniqueVindex(semTable, exp.getExpr())) { + return true; + } + } + return false; + } + + private LogicalPlan addDistinct(PlanningContext ctx, LogicalPlan plan) throws SQLException { + ArrayList orderExprs = new ArrayList<>(); + ArrayList groupByKeys = new ArrayList<>(); + for (int index = 0; index < this.getQp().getSelectExprs().size(); index++) { + QueryProjection.SelectExpr sExpr = this.getQp().getSelectExprs().get(index); + if (isAmbiguousOrderBy(index, sExpr.getAliasedExpr().getExpr(), this.getQp().getSelectExprs())) { + throw new SQLException("generating order by clause: ambiguous symbol reference: " + sExpr.getAliasedExpr().toString()); + } + SQLExpr inner; + if (sExpr.col.getAlias() == null || sExpr.col.getAlias().isEmpty()) { + inner = sExpr.getAliasedExpr().getExpr(); + } else { + // If we have an alias, we need to use the alias and not the original expression + // to make sure dependencies work correctly, + // we simply copy the dependencies of the original expression here + inner = new SQLIdentifierExpr(sExpr.col.getAlias()); + ctx.getSemTable().copyDependencies(sExpr.col.getExpr(), inner); + } + + GroupByParams groupByParams = new GroupByParams(); + groupByParams.setKeyCol(index); + groupByParams.setCollationID(ctx.getSemTable().collationForExpr(inner)); + groupByParams.setExpr(inner); + Pair offset = wrapAndPushExpr(ctx, sExpr.getAliasedExpr().getExpr(), sExpr.getAliasedExpr().getExpr(), plan); + int weightOffset = offset.getRight(); + groupByParams.setWeightStringCol(weightOffset); + + groupByKeys.add(groupByParams); + + QueryProjection.OrderBy orderByExpr = new QueryProjection.OrderBy(new SQLSelectOrderByItem(inner), sExpr.getAliasedExpr().getExpr()); + orderExprs.add(orderByExpr); + + } + LogicalPlan innerPlan = planOrderBy(ctx, orderExprs, plan); + OrderedAggregateGen4Plan orderedAggregateGen4Plan = new OrderedAggregateGen4Plan(); + orderedAggregateGen4Plan.setInput(innerPlan); + orderedAggregateGen4Plan.setGroupByKeys(groupByKeys); + + return orderedAggregateGen4Plan; + } + + private LogicalPlan truncateColumnsIfNeeded(PlanningContext ctx, LogicalPlan plan) throws SQLException { + if (plan.outputColumns().size() == this.qp.getColumnCount()) { + return plan; + } + if (isJoin(plan)) { + // since this is a join, we can safely add extra columns and not need to truncate them + return plan; + } + if (plan instanceof RouteGen4Plan) { + ((RouteGen4Plan) plan).eroute.setTruncateColumnCount(this.qp.getColumnCount()); + } else if (plan instanceof OrderedAggregateGen4Plan) { + ((OrderedAggregateGen4Plan) plan).setTruncateColumnCount(this.qp.getColumnCount()); + } else if (plan instanceof MemorySortGen4Plan) { + ((MemorySortGen4Plan) plan).getTruncater().setTruncateColumnCount(this.getQp().getColumnCount()); + } else { + plan = new SimpleProjectionGen4Plan(plan); + List exprs = this.getQp().getSelectExprs().subList(0, this.getQp().getColumnCount()); + pushProjections(ctx, plan, exprs); + } + return plan; + } + + public static int checkIfAlreadyExists(SQLSelectItem expr, SQLSelectQuery node, SemTable semTable) { + TableSet exprDep = semTable.recursiveDeps(expr.getExpr()); + // Here to find if the expr already exists in the SelectStatement, we have 3 cases + // input is a Select -> In this case we want to search in the select + // input is a Union -> In this case we want to search in the First Select of the Union + // input is a Parenthesised Select -> In this case we want to search in the select + // all these three cases are handled by the call to GetFirstSelect. + boolean isExprCol = false; + SQLName exprCol = null; + if (expr.getExpr() instanceof SQLName) { + isExprCol = true; + exprCol = (SQLName) expr.getExpr(); + } + + if (node instanceof MySqlSelectQueryBlock) { + MySqlSelectQueryBlock selectQueryBlock = (MySqlSelectQueryBlock) node; + + // first pass - search for aliased expressions + for (int i = 0; i < selectQueryBlock.getSelectList().size(); i++) { + if (!isExprCol) { + break; + } + + SQLSelectItem selectItem = selectQueryBlock.getSelectList().get(i); + SqlParser.SelectExpr selectExpr = SqlParser.SelectExpr.type(selectItem); + if (SqlParser.SelectExpr.AliasedExpr.equals(selectExpr)) { + if (StringUtils.isNotEmpty(selectItem.getAlias()) && selectItem.getAlias().equals(((SQLName) expr.getExpr()).getSimpleName())) { + return i; + } + } + } + + // next pass - we are searching the actual expressions and not the aliases + for (int i = 0; i < selectQueryBlock.getSelectList().size(); i++) { + SQLSelectItem selectItem = selectQueryBlock.getSelectList().get(i); + SqlParser.SelectExpr selectExpr = SqlParser.SelectExpr.type(selectItem); + if (!SqlParser.SelectExpr.AliasedExpr.equals(selectExpr)) { + continue; + } + + boolean isSelectExprCol = false; + SQLName selectExprCol = null; + if (selectItem.getExpr() instanceof SQLName) { + isSelectExprCol = true; + selectExprCol = (SQLName) selectItem.getExpr(); + } + + TableSet selectExprDep = semTable.recursiveDeps(selectItem.getExpr()); + + // Check that the two expressions have the same dependencies + if (!selectExprDep.equals(exprDep)) { + continue; + } + + if (isSelectExprCol && isExprCol && exprCol.getSimpleName().equals(selectExprCol.getSimpleName())) { + // the expressions are ColName, we compare their name + return i; + } + + if (SQLExprUtils.equals(selectItem.getExpr(), expr.getExpr())) { + // the expressions are not ColName, so we just compare the expressions + return i; + } + } + } + return -1; + } + + public static boolean isAmbiguousOrderBy(int index, SQLExpr col, List exprs) { + return false; + } + + public static SQLExpr weightStringFor(SQLExpr params) { + SQLExpr expr = new SQLMethodInvokeExpr("weight_string", null, params); + return expr; + } + + @NoArgsConstructor + @Getter + @Setter + static class PushAggregationResult { + private LogicalPlan output; + + private List groupingOffsets; + + private List> outputAggrsOffset; + + private boolean pushed; + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/InnerJoin.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/InnerJoin.java new file mode 100644 index 0000000..9daf74e --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/InnerJoin.java @@ -0,0 +1,39 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4; + +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import java.util.ArrayList; +import java.util.List; +import lombok.Getter; + +public class InnerJoin { + + @Getter + private TableSet deps; + + @Getter + private List exprs = new ArrayList<>(); + + public InnerJoin(TableSet ts, List asList) { + this.deps = ts; + this.exprs = asList; + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/IntroducesTable.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/IntroducesTable.java new file mode 100644 index 0000000..631be83 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/IntroducesTable.java @@ -0,0 +1,29 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4; + + +import vschema.Vschema; + +// IntroducesTable is used to make it possible to gather information about the table an operator introduces +public interface IntroducesTable { + QueryTable getQTable(); + + Vschema.Table getVTable(); +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/Offsets.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/Offsets.java new file mode 100644 index 0000000..75df25f --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/Offsets.java @@ -0,0 +1,33 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4; + +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.Setter; + +@AllArgsConstructor +public class Offsets { + @Getter + private Integer col; + + @Setter + @Getter + private Integer wsCol; +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/PlanVisitor.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/PlanVisitor.java new file mode 100644 index 0000000..f6b3b79 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/PlanVisitor.java @@ -0,0 +1,28 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4; + +import com.jd.jdbc.common.tuple.Pair; +import com.jd.jdbc.planbuilder.gen4.logical.LogicalPlan; +import java.sql.SQLException; + +@FunctionalInterface +public interface PlanVisitor { + Pair func(LogicalPlan logicalPlan) throws SQLException; +} \ No newline at end of file diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/PostProcess.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/PostProcess.java new file mode 100644 index 0000000..a0fa42f --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/PostProcess.java @@ -0,0 +1,72 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4; + +import com.jd.jdbc.common.tuple.ImmutablePair; +import com.jd.jdbc.common.tuple.Pair; +import com.jd.jdbc.evalengine.EvalEngine; +import com.jd.jdbc.planbuilder.gen4.logical.LimitGen4Plan; +import com.jd.jdbc.planbuilder.gen4.logical.LogicalPlan; +import com.jd.jdbc.planbuilder.gen4.logical.MemorySortGen4Plan; +import com.jd.jdbc.planbuilder.gen4.logical.RouteGen4Plan; +import com.jd.jdbc.sqlparser.ast.SQLLimit; +import com.jd.jdbc.sqlparser.ast.expr.SQLVariantRefExpr; +import com.jd.jdbc.sqlparser.dialect.mysql.ast.statement.MySqlSelectQueryBlock; +import java.sql.SQLException; + +public class PostProcess { + + public static LogicalPlan createLimit(LogicalPlan plan, SQLLimit limit) throws SQLException { + LogicalPlan limitPlan = LimitGen4Plan.createLimitPlan(plan, limit); + return limitPlan; + } + + /** + * setUpperLimit is an optimization hint that tells that primitive + * that it does not need to return more than the specified number of rows. + * A primitive that cannot perform this can ignore the request. + */ + static class SetUpperLimit implements PlanVisitor { + + @Override + public Pair func(LogicalPlan plan) throws SQLException { + + if (HorizonPlanning.isJoin(plan)) { + return new ImmutablePair(false, plan); + } + + if (plan instanceof RouteGen4Plan) { + SQLVariantRefExpr upperLimitVar = new SQLVariantRefExpr(); + upperLimitVar.setName(":__upper_limit"); + upperLimitVar.setIndex(-2); + + // The route pushes the limit regardless of the plan. + // If it's a scatter query, the rows returned will be + // more than the upper limit, but enough for the limit + ((MySqlSelectQueryBlock) ((RouteGen4Plan) plan).getSelect()).setLimit(new SQLLimit(upperLimitVar)); + } else if (plan instanceof MemorySortGen4Plan) { + EvalEngine.BindVariable pv = new EvalEngine.BindVariable("__upper_limit"); + ((MemorySortGen4Plan) plan).getEMemorySort().setUpperLimit(pv); + // we don't want to go down to the rest of the tree + return new ImmutablePair(false, plan); + } + return new ImmutablePair(true, plan); + } + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/ProjectionPushing.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/ProjectionPushing.java new file mode 100644 index 0000000..3f3e77a --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/ProjectionPushing.java @@ -0,0 +1,195 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4; + +import com.jd.jdbc.common.tuple.Pair; +import com.jd.jdbc.common.tuple.Triple; +import com.jd.jdbc.context.PlanningContext; +import com.jd.jdbc.engine.Engine; +import com.jd.jdbc.engine.gen4.AbstractAggregateGen4; +import com.jd.jdbc.engine.gen4.GroupByParams; +import com.jd.jdbc.planbuilder.gen4.logical.DistinctGen4Plan; +import com.jd.jdbc.planbuilder.gen4.logical.FilterGen4Plan; +import com.jd.jdbc.planbuilder.gen4.logical.JoinGen4Plan; +import com.jd.jdbc.planbuilder.gen4.logical.LimitGen4Plan; +import com.jd.jdbc.planbuilder.gen4.logical.LogicalPlan; +import com.jd.jdbc.planbuilder.gen4.logical.OrderedAggregateGen4Plan; +import com.jd.jdbc.planbuilder.gen4.logical.RouteGen4Plan; +import com.jd.jdbc.planbuilder.gen4.logical.SimpleProjectionGen4Plan; +import com.jd.jdbc.planbuilder.gen4.operator.OperatorFuncs; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLName; +import com.jd.jdbc.sqlparser.ast.expr.SQLExprUtils; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import com.jd.jdbc.sqlparser.dialect.mysql.ast.statement.MySqlSelectQueryBlock; +import com.jd.jdbc.sqlparser.dialect.mysql.visitor.VtRemoveDbNameVisitor; +import com.jd.jdbc.util.SelectItemUtil; +import java.sql.SQLException; +import java.sql.SQLFeatureNotSupportedException; +import java.util.List; + +public class ProjectionPushing { + + public static Pair pushProjection(PlanningContext ctx, SQLSelectItem expr, LogicalPlan plan, boolean inner, boolean reuseCol, boolean hasAggregation) throws SQLException { + if (plan instanceof RouteGen4Plan) { + return addExpressionToRoute(ctx, (RouteGen4Plan) plan, expr, reuseCol); + } else if (plan instanceof SimpleProjectionGen4Plan) { + return pushProjectionIntoSimpleProj(ctx, expr, (SimpleProjectionGen4Plan) plan, inner, hasAggregation, reuseCol); + } else if (plan instanceof FilterGen4Plan || plan instanceof LimitGen4Plan || plan instanceof DistinctGen4Plan) { + LogicalPlan src = plan.inputs()[0]; + return pushProjection(ctx, expr, src, inner, reuseCol, hasAggregation); + } else if (plan instanceof OrderedAggregateGen4Plan) { + return pushProjectionIntoOA(ctx, expr, (OrderedAggregateGen4Plan) plan, inner, hasAggregation); + } else if (plan instanceof JoinGen4Plan) { + return pushProjectionIntoJoin(ctx, expr, (JoinGen4Plan) plan, reuseCol, inner, hasAggregation); + } else { + throw new SQLFeatureNotSupportedException(); + } + } + + private static Pair addExpressionToRoute(PlanningContext ctx, RouteGen4Plan rb, SQLSelectItem expr, boolean reuseCol) throws SQLException { + if (reuseCol) { + int getOffset = HorizonPlanning.checkIfAlreadyExists(expr, rb.getSelect(), ctx.getSemTable()); + if (getOffset != -1) { + return Pair.of(getOffset, 0); + } + } + // remove keyspace + VtRemoveDbNameVisitor visitor = new VtRemoveDbNameVisitor(); + expr.accept(visitor); + if (!(rb.select instanceof MySqlSelectQueryBlock)) { + // error + throw new SQLException("unsupported: pushing projection " + expr.toString()); + } + + if (ctx.isRewriteDerivedExpr()) { + + } + + int offset = ((MySqlSelectQueryBlock) rb.select).getSelectList().size(); + ((MySqlSelectQueryBlock) rb.select).addSelectItem(expr); + + return Pair.of(offset, 1); + } + + private static Pair pushProjectionIntoSimpleProj(PlanningContext ctx, SQLSelectItem expr, SimpleProjectionGen4Plan node, + boolean inner, boolean hasAggregation, boolean reuseCol) throws SQLException { + Pair ret = pushProjection(ctx, expr, node.getInput(), inner, true, hasAggregation); + int offset = ret.getLeft(); + for (int i = 0; i < node.getESimpleProjection().getCols().size(); i++) { + // we return early if we already have the column in the simple projection's + // output list so we do not add it again. + if (reuseCol && node.getESimpleProjection().getCols().get(i).equals(offset)) { + return Pair.of(i, 0); + } + } + node.getESimpleProjection().getCols().add(offset); + return Pair.of(node.getESimpleProjection().getCols().size() - 1, 1); + } + + private static Pair pushProjectionIntoOA(PlanningContext ctx, SQLSelectItem expr, OrderedAggregateGen4Plan node, + boolean inner, boolean hasAggregation) throws SQLException { + boolean isColName = false; + if (expr.getExpr() instanceof SQLName) { + isColName = true; + } + for (AbstractAggregateGen4.AggregateParams aggregate : node.getAggregates()) { + if (SQLExprUtils.equals(aggregate.getExpr(), expr.getExpr())) { + return Pair.of(aggregate.getCol(), 0); + } + if (isColName && ((SQLName) expr.getExpr()).getSimpleName().equals(aggregate.getAlias())) { + return Pair.of(aggregate.getCol(), 0); + } + } + for (GroupByParams key : node.getGroupByKeys()) { + if (SQLExprUtils.equals(key.getExpr(), expr.getExpr())) { + return Pair.of(key.getKeyCol(), 0); + } + } + Pair ret = pushProjection(ctx, expr, node.getInput(), inner, true, hasAggregation); + int offset = ret.getLeft(); + AbstractAggregateGen4.AggregateParams addAggregateParams = new AbstractAggregateGen4.AggregateParams(Engine.AggregateOpcodeG4.AggregateRandom, offset, SelectItemUtil.columnName(expr)); + addAggregateParams.setExpr(expr.getExpr()); + addAggregateParams.setOriginal(expr); + node.getAggregates().add(addAggregateParams); + return Pair.of(offset, 1); + } + + private static Pair pushProjectionIntoJoin(PlanningContext ctx, SQLSelectItem expr, JoinGen4Plan node, + boolean reuseCol, boolean inner, boolean hasAggregation) throws SQLException { + TableSet lhsSolves = node.getLeft().containsTables(); + TableSet rhsSolves = node.getRight().containsTables(); + TableSet deps = ctx.getSemTable().recursiveDeps(expr.getExpr()); + + int column; + int appended; + + boolean passDownReuseCol = reuseCol ? reuseCol : (expr.getAlias() == null || expr.getAlias().isEmpty()); + + if (deps.isSolvedBy(lhsSolves)) { + Pair result = pushProjection(ctx, expr, node.getLeft(), inner, passDownReuseCol, hasAggregation); + column = -(result.getLeft() + 1); + appended = result.getRight(); + } else if (deps.isSolvedBy(rhsSolves)) { + boolean newInner = inner && node.getOpcode() != Engine.JoinOpcode.LeftJoin; + Pair result = pushProjection(ctx, expr, node.getRight(), newInner, passDownReuseCol, hasAggregation); + column = result.getLeft() + 1; + appended = result.getRight(); + } else { + // if an expression has aggregation, then it should not be split up and pushed to both sides, + // for example an expression like count(*) will have dependencies on both sides, but we should not push it + // instead we should return an error + if (hasAggregation) { + throw new SQLException("unsupported: cross-shard query with aggregates"); + } + // now we break the expression into left and right side dependencies and rewrite the left ones to bind variables + Triple, List, SQLExpr> result = OperatorFuncs.breakExpressioninLHSandRHS(ctx, expr.getExpr(), lhsSolves); + List bvName = result.getLeft(); + List cols = result.getMiddle(); + SQLExpr rewrittenExpr = result.getRight(); + // go over all the columns coming from the left side of the tree and push them down. While at it, also update the bind variable map. + // It is okay to reuse the columns on the left side since + // the final expression which will be selected will be pushed into the right side. + for (int i = 0; i < cols.size(); i++) { + Pair tmpResult = pushProjection(ctx, new SQLSelectItem(cols.get(i)), node.getLeft(), inner, true, false); + node.getVars().put(bvName.get(i), tmpResult.getLeft()); + } + // push the rewritten expression on the right side of the tree. Here we should take care whether we want to reuse the expression or not. + expr.setExpr(rewrittenExpr); + boolean newInner = inner && node.getOpcode() != Engine.JoinOpcode.LeftJoin; + Pair tmpResult = pushProjection(ctx, expr, node.getRight(), newInner, passDownReuseCol, false); + column = tmpResult.getLeft() + 1; + appended = tmpResult.getRight(); + } + if (reuseCol && (appended == 0)) { + for (int idx = 0; idx < node.getCols().size(); idx++) { + int col = node.getCols().get(idx); + if (column == col) { + return Pair.of(idx, 0); + } + } + // the column was not appended to either child, but we could not find it in out cols list, + // so we'll still add it + } + node.getCols().add(column); + return Pair.of(node.getCols().size() - 1, 1); + + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/QueryProjection.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/QueryProjection.java new file mode 100644 index 0000000..724b480 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/QueryProjection.java @@ -0,0 +1,631 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4; + +import com.jd.jdbc.common.tuple.ImmutablePair; +import com.jd.jdbc.common.tuple.Pair; +import com.jd.jdbc.common.util.CollectionUtils; +import com.jd.jdbc.engine.Engine; +import com.jd.jdbc.engine.gen4.AbstractAggregateGen4; +import com.jd.jdbc.planbuilder.PlanBuilder; +import com.jd.jdbc.sqlparser.SqlParser; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLName; +import com.jd.jdbc.sqlparser.ast.SQLOrderBy; +import com.jd.jdbc.sqlparser.ast.SQLOrderingSpecification; +import com.jd.jdbc.sqlparser.ast.SQLSetQuantifier; +import com.jd.jdbc.sqlparser.ast.expr.SQLAggregateExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLAggregateOption; +import com.jd.jdbc.sqlparser.ast.expr.SQLAllColumnExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLExprUtils; +import com.jd.jdbc.sqlparser.ast.expr.SQLIdentifierExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLNullExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLPropertyExpr; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectOrderByItem; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectQuery; +import com.jd.jdbc.sqlparser.ast.statement.SQLUnionQuery; +import com.jd.jdbc.sqlparser.dialect.mysql.ast.statement.MySqlSelectQueryBlock; +import com.jd.jdbc.sqlparser.utils.StringUtils; +import com.jd.jdbc.sqlparser.visitor.CheckForInvalidGroupingExpressionsVisitor; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import lombok.Getter; +import lombok.Setter; + +@Getter +@Setter +public class QueryProjection { + /** + * SelectExpr provides whether the columns is aggregation expression or not. + */ + @Setter + @Getter + public static class SelectExpr { + SQLSelectItem col; + + boolean aggr; + + SelectExpr() { + + } + + SelectExpr(SQLSelectItem col) { + this.col = col; + } + + public SelectExpr(boolean aggr, SQLSelectItem col) { + this.aggr = aggr; + this.col = col; + } + + /** + * GetAliasedExpr returns the SelectExpr as a *sqlparser.AliasedExpr if its type allows it, + * otherwise an error is returned. + * + * @return + * @throws SQLException + */ + public SQLSelectItem getAliasedExpr() throws SQLException { + SqlParser.SelectExpr selectExpr = SqlParser.SelectExpr.type(col); + if (SqlParser.SelectExpr.AliasedExpr.equals(selectExpr)) { + return this.col; + } else if (SqlParser.SelectExpr.StarExpr.equals(selectExpr)) { + throw new SQLException("unsupported: '*' expression in cross-shard query"); + // return null; + } else { + throw new SQLException("not an aliased expression:" + selectExpr.toString()); + // return null; + } + } + + /** + * GetExpr returns the underlying sqlparser.Expr of our SelectExpr + * + * @return + */ + public SQLExpr getExpr() { + SqlParser.SelectExpr selectExpr = SqlParser.SelectExpr.type(col); + if (SqlParser.SelectExpr.AliasedExpr.equals(selectExpr)) { + return this.col.getExpr(); + } else { + return null; + } + } + } + + /** + * GroupBy contains the expression to used in group by and also if grouping is needed at VTGate level + * then what the weight_string function expression to be sent down for evaluation. + */ + @Setter + @Getter + static class GroupBy { + SQLExpr inner; + + SQLExpr weightStrExpr; + + /** + * The index at which the user expects to see this column. Set to nil, if the user does not ask for it + */ + Integer innerIndex; + + + /** + * The original aliased expression that this group by is referring + */ + SQLSelectItem aliasedExpr; + + public GroupBy(SQLExpr inner) { + this.inner = inner; + + } + + public GroupBy(SQLExpr inner, SQLExpr weightStrExpr, Integer innerIndex, SQLSelectItem aliasedExpr) { + this.inner = inner; + this.weightStrExpr = weightStrExpr; + this.innerIndex = innerIndex; + this.aliasedExpr = aliasedExpr; + } + + @Override + public GroupBy clone() { + return new GroupBy(this.inner.clone(), this.weightStrExpr.clone(), this.innerIndex, this.aliasedExpr.clone()); + } + + public OrderBy asOrderBy() { + SQLSelectOrderByItem inner = new SQLSelectOrderByItem(this.inner); + inner.setType(SQLOrderingSpecification.ASC); + return new OrderBy(inner, this.weightStrExpr); + } + + public SQLSelectItem asAliasedExpr() { + if (aliasedExpr != null) { + return aliasedExpr; + } + return new SQLSelectItem(this.getInner()); + } + } + + /** + * OrderBy contains the expression to used in order by and also if ordering is needed at VTGate level + * then what the weight_string function expression to be sent down for evaluation. + */ + static class OrderBy { + + @Getter + @Setter + SQLSelectOrderByItem inner; + + @Getter + @Setter + SQLExpr weightStrExpr; + + public OrderBy(SQLSelectOrderByItem inner, SQLExpr weightStrExpr) { + this.inner = inner; + this.weightStrExpr = weightStrExpr; + } + } + + /** + * Aggr encodes all information needed for aggregation functions + */ + @Getter + @Setter + static class Aggr { + + private SQLSelectItem original; + + private SQLAggregateExpr func; + + private Engine.AggregateOpcodeG4 opCode; + + private String alias; + + // The index at which the user expects to see this aggregated function. Set to nil, if the user does not ask for it + private Integer index; + + private Boolean distinct = false; + + public Aggr(SQLSelectItem original, Engine.AggregateOpcodeG4 opCode, String alias, Integer index) { + this.original = original; + this.opCode = opCode; + this.alias = alias; + this.index = index; + this.func = null; + } + } + + List selectExprs = new ArrayList<>(); + + boolean hasAggr; + + boolean distinct; + + List groupByExprs = new ArrayList<>(); + + List orderExprs = new ArrayList<>(); + + @Getter + private boolean canPushDownSorting; + + boolean hasStar; + + // AddedColumn keeps a counter for expressions added to solve HAVING expressions the user is not selecting + int addedColumn; + + + /** + * CreateQPFromSelect creates the QueryProjection for the input *sqlparser.Select + * + * @param sel + * @throws SQLException + */ + public void createQPFromSelect(SQLSelectQuery sel) throws SQLException { + this.distinct = ((MySqlSelectQueryBlock) sel).getDistionOption() == SQLSetQuantifier.DISTINCT; + this.addSelectExpressions((MySqlSelectQueryBlock) sel); + if (((MySqlSelectQueryBlock) sel).getGroupBy() != null) { + for (SQLExpr group : ((MySqlSelectQueryBlock) sel).getGroupBy().getItems()) { + Pair retObjs = this.findSelectExprIndexForExpr(group); + SQLExpr[] retExprs = this.getSimplifiedExpr(group); + checkForInvalidGroupingExpressions(retExprs[1]); + GroupBy groupBy = new GroupBy(retExprs[0], retExprs[1], retObjs.getLeft(), retObjs.getRight()); + this.groupByExprs.add(groupBy); + } + } + + this.addOrderBy(((MySqlSelectQueryBlock) sel).getOrderBy()); + + if (this.distinct && !this.hasAggr) { + this.groupByExprs.clear(); + } + } + + /** + * CreateQPFromUnion creates the QueryProjection for the input *sqlparser.Union + * + * @param union + * @return + */ + public static QueryProjection createQPFromUnion(SQLSelectQuery union) throws SQLException { + QueryProjection qp = new QueryProjection(); + MySqlSelectQueryBlock sel = PlanBuilder.getFirstSelect(union); + qp.addSelectExpressions(sel); + if (union instanceof SQLUnionQuery) { + qp.addOrderBy(((SQLUnionQuery) union).getOrderBy()); + } + return qp; + } + + /** + * returns true if we either have aggregate functions or grouping defined + * + * @return + */ + public boolean needsAggregating() { + return this.hasAggr || !this.groupByExprs.isEmpty(); + } + + public boolean needsDistinct() { + if (!distinct) { + return false; + } + + if (onlyAggr() && groupByExprs.size() == 0) { + return false; + } + return true; + } + + public boolean onlyAggr() { + if (!hasAggr) { + return false; + } + for (SelectExpr expr : selectExprs) { + if (!expr.aggr) { + return false; + } + } + return true; + } + + /** + * GetGrouping returns a copy of the grouping parameters of the QP + * + * @return + */ + public List getGrouping() { + return new ArrayList<>(this.groupByExprs); + } + + public int getColumnCount() { + return this.selectExprs.size() - this.addedColumn; + } + + /** + * AlignGroupByAndOrderBy aligns the group by and order by columns, so they are in the same order + * The GROUP BY clause is a set - the order between the elements does not make any difference, + * so we can simply re-arrange the column order + * We are also free to add more ORDER BY columns than the user asked for which we leverage, + * so the input is already ordered according to the GROUP BY columns used + */ + public void alignGroupByAndOrderBy() { + // The ORDER BY can be performed before the OA + + List newGrouping = null; + if (CollectionUtils.isEmpty(this.orderExprs)) { + // The query didn't ask for any particular order, so we are free to add arbitrary ordering. + // We'll align the grouping and ordering by the output columns + newGrouping = this.getGrouping(); + sortGrouping(newGrouping); + for (GroupBy groupBy : newGrouping) { + this.orderExprs.add(groupBy.asOrderBy()); + } + } else { + // Here we align the GROUP BY and ORDER BY. + // First step is to make sure that the GROUP BY is in the same order as the ORDER BY + boolean[] used = new boolean[this.groupByExprs.size()]; + newGrouping = new ArrayList<>(); + for (OrderBy orderExpr : this.orderExprs) { + for (int i = 0; i < this.groupByExprs.size(); i++) { + GroupBy groupingExpr = groupByExprs.get(i); + if (!used[i] && SQLExprUtils.equals(groupingExpr.getWeightStrExpr(), orderExpr.getWeightStrExpr())) { + newGrouping.add(groupingExpr); + used[i] = true; + } + } + } + + if (newGrouping.size() != this.groupByExprs.size()) { + // we are missing some groupings. We need to add them both to the new groupings list, but also to the ORDER BY + for (int i = 0; i < used.length; i++) { + boolean added = used[i]; + if (!added) { + GroupBy groupBy = this.groupByExprs.get(i); + newGrouping.add(groupBy); + this.orderExprs.add(groupBy.asOrderBy()); + } + } + } + } + this.groupByExprs = newGrouping; + } + + public List aggregationExpressions() throws SQLException { + for (OrderBy orderExpr : this.orderExprs) { + if (this.isOrderByExprInGroupBy(orderExpr)) { + continue; + } + + SQLExpr orderSQLExpr = orderExpr.inner.getExpr(); + + boolean found = false; + + for (QueryProjection.SelectExpr selectExpr : this.getSelectExprs()) { + // if(selectExpr.getCol() instanceof Aliased) + if (Objects.equals(selectExpr.getCol().getExpr(), orderSQLExpr)) { + found = true; + break; + } + } + if (found) { + continue; + } + + SQLSelectItem newItem = new SQLSelectItem(orderSQLExpr); + boolean hasAggr = PlanBuilder.selectItemsHasAggregates(Collections.singletonList(newItem)); + this.selectExprs.add(new SelectExpr(hasAggr, newItem)); + this.addedColumn++; + } + List out = new ArrayList<>(); + for (int idx = 0; idx < this.selectExprs.size(); idx++) { + SelectExpr expr = this.selectExprs.get(idx); + SQLSelectItem aliasedExpr = expr.getAliasedExpr(); + int idxCopy = idx; + + if (!PlanBuilder.selectItemsHasAggregates(Collections.singletonList(expr.getCol()))) { + if (!this.isExprInGroupByExprs(expr)) { + out.add(new Aggr(aliasedExpr, Engine.AggregateOpcodeG4.AggregateRandom, aliasedExpr.toString(), idxCopy)); + } + continue; + } + + if (!(expr.getExpr() instanceof SQLAggregateExpr)) { + throw new SQLException("unsupported: in scatter query: complex aggregate expression"); + } + SQLAggregateExpr aggrExpr = (SQLAggregateExpr) expr.getExpr(); + String aggrFunName = aggrExpr.getMethodName().toLowerCase(); + + Engine.AggregateOpcodeG4 opcode = AbstractAggregateGen4.SUPPORTED_AGGREGATES.get(aggrFunName); + if (opcode == null) { + throw new SQLException("unsupported: in scatter query: aggregation function " + aggrFunName); + } + + if (opcode == Engine.AggregateOpcodeG4.AggregateCount) { + // count star + if (aggrExpr.getArguments().size() == 1 && aggrExpr.getArguments().get(0) instanceof SQLAllColumnExpr) { + opcode = Engine.AggregateOpcodeG4.AggregateCountStar; + } + } + + if (aggrExpr.getOption() == SQLAggregateOption.DISTINCT) { + if (opcode == Engine.AggregateOpcodeG4.AggregateCount) { + opcode = Engine.AggregateOpcodeG4.AggregateCountDistinct; + } else if (opcode == Engine.AggregateOpcodeG4.AggregateSum) { + opcode = Engine.AggregateOpcodeG4.AggregateSumDistinct; + } + } + + String alias = aliasedExpr.getAlias(); + if (alias == null) { + if (aliasedExpr.getExpr() instanceof SQLIdentifierExpr) { + alias = ((SQLIdentifierExpr) aliasedExpr.getExpr()).getName(); + } else if (aliasedExpr.getExpr() instanceof SQLPropertyExpr) { + alias = ((SQLPropertyExpr) aliasedExpr.getExpr()).getName(); + } else { + alias = aliasedExpr.getExpr().toString(); + } + } + + Aggr addAggr = new Aggr(aliasedExpr, opcode, alias, idxCopy); + if (aliasedExpr.getExpr() instanceof SQLAggregateExpr) { + SQLAggregateExpr sqlAggregateExpr = (SQLAggregateExpr) aliasedExpr.getExpr(); + addAggr.setFunc(sqlAggregateExpr); + + if (sqlAggregateExpr.getOption() == SQLAggregateOption.DISTINCT) { + addAggr.setDistinct(true); + } + } + out.add(addAggr); + } + return out; + } + + private boolean isExprInGroupByExprs(SelectExpr expr) { + for (GroupBy groupByExpr : this.groupByExprs) { + SQLExpr exp = expr.getExpr(); + if (exp == null) { + return false; + } + if (SQLExprUtils.equals(groupByExpr.weightStrExpr, exp)) { + return true; + } + } + return false; + } + + private boolean isOrderByExprInGroupBy(OrderBy order) { + // ORDER BY NULL or Aggregation functions need not be present in group by + boolean isAggregate = order.weightStrExpr instanceof SQLAggregateExpr; + if (order.inner.getExpr() instanceof SQLNullExpr || isAggregate) { + return true; + } + for (GroupBy groupByExpr : this.groupByExprs) { + if (SQLExprUtils.equals(groupByExpr.weightStrExpr, order.weightStrExpr)) { + return true; + } + } + return false; + } + + public void sortGrouping(List newGrouping) { + newGrouping.sort((o1, o2) -> { + if (o1.getInnerIndex() == null) { + return 1; + } else if (o2.getInnerIndex() == null) { + return -1; + } + return o1.getInnerIndex() - o2.getInnerIndex(); + }); + } + + public void sortAggregations(List aggregations) { + aggregations.sort((o1, o2) -> { + if (o1.getIndex() == null) { + return 1; + } else if (o2.getIndex() == null) { + return -1; + } + return o1.getIndex() - o2.getIndex(); + }); + } + + private void addSelectExpressions(MySqlSelectQueryBlock sel) throws SQLException { + for (SQLSelectItem item : sel.getSelectList()) { // same as PrimitiveBuilder.pushSelectRoutes + SqlParser.SelectExpr selectExpr = SqlParser.SelectExpr.type(item); + if (SqlParser.SelectExpr.AliasedExpr.equals(selectExpr)) { + // checkForInvalidAggregations + SelectExpr col = new SelectExpr(item); + if (item.getExpr() instanceof SQLAggregateExpr) { + this.hasAggr = true; + col.setAggr(true); + } + this.selectExprs.add(col); + } else if (SqlParser.SelectExpr.StarExpr.equals(selectExpr)) { + this.hasStar = true; + SelectExpr col = new SelectExpr(item); + this.selectExprs.add(col); + } else { + throw new SQLException("BUG: unexpected select expression type: " + item); + } + } + } + + /** + * FindSelectExprIndexForExpr returns the index of the given expression in the select expressions, if it is part of it + * returns null otherwise. + * + * @param expr + * @return Pair + */ + private Pair findSelectExprIndexForExpr(SQLExpr expr) { + boolean isCol = expr instanceof SQLName; + for (int idx = 0; idx < this.selectExprs.size(); idx++) { + SQLSelectItem col = selectExprs.get(idx).getCol(); + SqlParser.SelectExpr selectExpr = SqlParser.SelectExpr.type(col); + boolean isAliasedExpr = SqlParser.SelectExpr.AliasedExpr.equals(selectExpr); + if (!isAliasedExpr) { + continue; + } + if (isCol) { + boolean isAliasExpr = StringUtils.isNotEmpty(col.getAlias()); + if (isAliasExpr && ((SQLName) expr).getSimpleName().equals(col.getAlias())) { + return new ImmutablePair<>(idx, col); + } + } + + if (SQLExprUtils.equals(col.getExpr(), expr)) { + return new ImmutablePair<>(idx, col); + } + } + return ImmutablePair.nullPair(); + } + + /** + * takes an expression used in ORDER BY or GROUP BY, and returns an expression that is simpler to evaluate + *

+ * If the ORDER BY is against a column alias, we need to remember the expression + * behind the alias. The weightstring(.) calls needs to be done against that expression and not the alias. + * Eg - select music.foo as bar, weightstring(music.foo) from music order by bar + * + * @param e + * @return + */ + public SQLExpr[] getSimplifiedExpr(SQLExpr e) { + SQLExpr[] ret = new SQLExpr[2]; + ret[0] = e; + ret[1] = e; + boolean isColName = e instanceof SQLName; + if (!isColName) { + return ret; + } + if (e instanceof SQLNullExpr) { + ret[0] = e; + ret[1] = null; + return ret; + } + + if (e instanceof SQLIdentifierExpr) { + for (SelectExpr expr : this.selectExprs) { + SqlParser.SelectExpr selectExpr = SqlParser.SelectExpr.type(expr.getCol()); + boolean isAliasedExpr = SqlParser.SelectExpr.AliasedExpr.equals(selectExpr); + if (!isAliasedExpr) { + continue; + } + boolean isAliasExpr = StringUtils.isNotEmpty(expr.getCol().getAlias()); + if (isAliasExpr && ((SQLName) e).getSimpleName().equals(expr.getCol().getAlias())) { + ret[1] = expr.getCol().getExpr(); + return ret; + } + } + } + + return ret; + } + + private void addOrderBy(SQLOrderBy orderBy) { + boolean canPushDownSorting = true; + if (orderBy == null) { + this.canPushDownSorting = canPushDownSorting; + return; + } + for (SQLSelectOrderByItem item : orderBy.getItems()) { + SQLExpr[] sqlExprs = this.getSimplifiedExpr(item.getExpr()); + if (sqlExprs[1] instanceof SQLNullExpr) { + continue; + } + SQLSelectOrderByItem newItem = new SQLSelectOrderByItem(sqlExprs[0]); + newItem.setType(item.getType()); + this.orderExprs.add(new OrderBy(newItem, sqlExprs[1])); + canPushDownSorting = canPushDownSorting && (!(sqlExprs[1] instanceof SQLAggregateExpr)); + } + this.canPushDownSorting = canPushDownSorting; + } + + private void checkForInvalidGroupingExpressions(SQLExpr expr) throws SQLException { + CheckForInvalidGroupingExpressionsVisitor visitor = new CheckForInvalidGroupingExpressionsVisitor(); + expr.accept(visitor); + if (visitor.getException() != null) { + throw visitor.getException(); + } + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/QueryTable.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/QueryTable.java new file mode 100644 index 0000000..852ff96 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/QueryTable.java @@ -0,0 +1,50 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4; + +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.statement.SQLExprTableSource; +import java.util.ArrayList; +import java.util.List; +import lombok.Getter; +import lombok.Setter; + +@Getter +@Setter +public class QueryTable { + private TableSet id; + + // sqlparser.AliasedTableExpr + private SQLExprTableSource alias; + + private TableName table; + + private List predicates; + + private boolean isInfSchema; + + public QueryTable(TableSet id, SQLExprTableSource alias, TableName table, Boolean isInfSchema) { + this.id = id; + this.alias = alias; + this.table = table; + this.isInfSchema = isInfSchema; + this.predicates = new ArrayList<>(); + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/Rewriter.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/Rewriter.java new file mode 100644 index 0000000..932a18c --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/Rewriter.java @@ -0,0 +1,92 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4; + +import com.jd.jdbc.planbuilder.PlanBuilder; +import com.jd.jdbc.planbuilder.semantics.SemTable; +import com.jd.jdbc.sqlparser.SqlParser; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectStatement; +import com.jd.jdbc.sqlparser.dialect.mysql.ast.statement.MySqlSelectQueryBlock; +import com.jd.jdbc.sqlparser.utils.StringUtils; +import com.jd.jdbc.sqlparser.visitor.RewriteHavingVisitor; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +public class Rewriter { + private SemTable semTable; + + private Object reservedVars; + + private int inSubquery; + + private String err; + + public Rewriter(SemTable semTable, Object reservedVars) { + this.semTable = semTable; + this.reservedVars = reservedVars; + } + + public static void queryRewrite(SemTable semTable, Object reservedVars, SQLSelectStatement statement) throws SQLException { + Rewriter rewriter = new Rewriter(semTable, reservedVars); + if (statement.getSelect().getQuery() instanceof MySqlSelectQueryBlock) { + rewriter.rewriteHavingClause((MySqlSelectQueryBlock) statement.getSelect().getQuery()); + } + } + + private void rewriteHavingClause(MySqlSelectQueryBlock query) { + if (query.getGroupBy() == null) { + return; + } + if (query.getGroupBy() != null && query.getGroupBy().getHaving() == null) { + return; + } + + Map selectExprMap = new HashMap<>(query.getSelectList().size()); + for (SQLSelectItem selectItem : query.getSelectList()) { + SqlParser.SelectExpr selectExpr = SqlParser.SelectExpr.type(selectItem); + if (!Objects.equals(SqlParser.SelectExpr.AliasedExpr, selectExpr) || StringUtils.isEmpty(selectItem.getAlias())) { + continue; + } + selectExprMap.put(selectItem.getAlias().toLowerCase(), selectItem.getExpr()); + } + + // for each expression in the having clause, we check if it contains aggregation. + // if it does, we keep the expression in the having clause ; and if it does not + // and the expression is in the select list, we replace the expression by the one + // used in the select list and add it to the where clause instead of the having clause. + List filters = PlanBuilder.splitAndExpression(new ArrayList<>(), query.getGroupBy().getHaving()); + query.getGroupBy().setHaving(null); + for (SQLExpr expr : filters) { + RewriteHavingVisitor rewriteHavingVisitor = new RewriteHavingVisitor(selectExprMap); + expr.accept(rewriteHavingVisitor); + + if (rewriteHavingVisitor.isHasAggr()) { + query.getGroupBy().addHaving(expr); + } else { + query.addWhere(expr); + } + } + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/SingleShardedShortcut.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/SingleShardedShortcut.java new file mode 100644 index 0000000..9128bbe --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/SingleShardedShortcut.java @@ -0,0 +1,95 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4; + +import com.jd.jdbc.context.PlanningContext; +import com.jd.jdbc.engine.Engine; +import com.jd.jdbc.engine.gen4.RouteGen4Engine; +import com.jd.jdbc.planbuilder.gen4.logical.LogicalPlan; +import com.jd.jdbc.planbuilder.gen4.logical.RouteGen4Plan; +import com.jd.jdbc.planbuilder.semantics.SemTable; +import com.jd.jdbc.planbuilder.semantics.TableInfo; +import com.jd.jdbc.sqlparser.ast.statement.SQLExprTableSource; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectQuery; +import com.jd.jdbc.sqlparser.ast.statement.SQLTableSource; +import com.jd.jdbc.sqlparser.dialect.mysql.visitor.VtRemoveDbNameExpectSystemDbVisitor; +import com.jd.jdbc.sqlparser.dialect.mysql.visitor.VtRemoveDbNameInColumnVisitor; +import com.jd.jdbc.vindexes.VKeyspace; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import vschema.Vschema; + +public class SingleShardedShortcut { + + public static LogicalPlan unshardedShortcut(PlanningContext ctx, SQLSelectQuery stmt, VKeyspace keyspace) throws SQLException { + // remove keyspace name + VtRemoveDbNameExpectSystemDbVisitor visitor = new VtRemoveDbNameExpectSystemDbVisitor(); + stmt.accept(visitor); + + VtRemoveDbNameInColumnVisitor visitor1 = new VtRemoveDbNameInColumnVisitor(); + stmt.accept(visitor1); + + List tableNames = getTableNames(ctx.getSemTable()); + + RouteGen4Engine engine = new RouteGen4Engine(Engine.RouteOpcode.SelectUnsharded, keyspace); + engine.setTableName(String.join(",", tableNames)); + + RouteGen4Plan plan = new RouteGen4Plan(); + plan.eroute = engine; + plan.select = stmt; + + plan.wireupGen4(ctx); + return plan; + } + + public static List getTableNames(SemTable semTable) { + Set tableNameSet = new HashSet<>(); + for (TableInfo tableInfo : semTable.getTables()) { + Vschema.Table tblObj = tableInfo.getVindexTable(); + if (tblObj == null) { + // probably a derived table + continue; + } + + String name = ""; + if (tableInfo.isInfSchema()) { + name = "tableName"; + } else { + SQLTableSource tbl = tableInfo.getExpr(); + if (tbl instanceof SQLExprTableSource) { + name = ((SQLExprTableSource) tbl).getName().getSimpleName(); + } else { + name = tbl.getAlias(); + } + } + tableNameSet.add(name); + } + List tableNames = new ArrayList<>(tableNameSet.size()); + for (String name : tableNameSet) { + tableNames.add(name); + } + Collections.sort(tableNames); + return tableNames; + } + +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/SortedIterator.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/SortedIterator.java new file mode 100644 index 0000000..fffddda --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/SortedIterator.java @@ -0,0 +1,91 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4; + +import java.util.Iterator; +import java.util.List; +import lombok.Getter; + +public class SortedIterator implements Iterator { + private List grouping; + + private List aggregations; + + @Getter + private QueryProjection.GroupBy valueGB; + + @Getter + private QueryProjection.Aggr valueA; + + private int groupbyIdx; + + private int aggrIdx; + + public SortedIterator(List grouping, List aggregations) { + this.grouping = grouping; + this.aggregations = aggregations; + } + + @Override + public boolean hasNext() { + if (this.aggrIdx < aggregations.size() && this.groupbyIdx < grouping.size()) { + QueryProjection.Aggr aggregation = aggregations.get(aggrIdx); + QueryProjection.GroupBy groupBy = grouping.get(groupbyIdx); + if (compareRefInt(aggregation.getIndex(), groupBy.getInnerIndex())) { + aggrIdx++; + valueA = aggregation; + valueGB = null; + return true; + } + groupbyIdx++; + valueA = null; + valueGB = groupBy; + return true; + } + if (this.groupbyIdx < grouping.size()) { + QueryProjection.GroupBy groupBy = grouping.get(groupbyIdx); + groupbyIdx++; + valueA = null; + valueGB = groupBy; + return true; + } + if (this.aggrIdx < aggregations.size()) { + QueryProjection.Aggr aggregation = aggregations.get(aggrIdx); + aggrIdx++; + valueA = aggregation; + valueGB = null; + return true; + } + return false; + } + + @Override + public SortedIterator next() { + return this; + } + + private boolean compareRefInt(Integer a, Integer b) { + if (a == null) { + return false; + } else if (b == null) { + return true; + } + return (b - a) > 0; + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/TableName.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/TableName.java new file mode 100644 index 0000000..13de6dd --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/TableName.java @@ -0,0 +1,43 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4; + + +import lombok.Getter; + +// TableName represents a table name. +// Qualifier, if specified, represents a database or keyspace. +// TableName is a value struct whose fields are case sensitive. +// This means two TableName vars can be compared for equality +// and a TableName can also be used as key in a map. +@Getter +public class TableName { + private String name; + + private String qualifier; + + public TableName(String qualifier, String tableName) { + this.qualifier = qualifier; + this.name = tableName; + } + + public String getDbAndTableName() { + return qualifier + ".`" + name + "`"; + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/AbstractGen4Plan.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/AbstractGen4Plan.java new file mode 100644 index 0000000..ea6878f --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/AbstractGen4Plan.java @@ -0,0 +1,87 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.logical; + +import com.jd.jdbc.context.PlanningContext; +import com.jd.jdbc.engine.PrimitiveEngine; +import com.jd.jdbc.planbuilder.Builder; +import com.jd.jdbc.planbuilder.Jointab; +import com.jd.jdbc.planbuilder.ResultColumn; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLName; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import java.sql.SQLException; +import java.util.List; + +public abstract class AbstractGen4Plan implements LogicalPlan { + + public AbstractGen4Plan() { + } + + @Override + public Integer order() { + return null; + } + + @Override + public List resultColumns() { + return null; + } + + @Override + public void reorder(Integer order) { + } + + @Override + public void wireup(Builder bldr, Jointab jt) throws SQLException { + } + + @Override + public void supplyVar(Integer from, Integer to, SQLName colName, String varName) throws SQLException { + } + + @Override + public Builder.SupplyColResponse supplyCol(SQLName col) throws SQLException { + return null; + } + + @Override + public Integer supplyWeightString(Integer colNumber) throws SQLException { + throw new SQLException("[BUG]: should not be called. This is a Gen4 primitive"); + //return null; + } + + // abstract + + public abstract void wireupGen4(PlanningContext ctx) throws SQLException; + + public abstract PrimitiveEngine getPrimitiveEngine() throws SQLException; + + public abstract LogicalPlan[] inputs() throws SQLException; + + public abstract LogicalPlan[] rewrite(LogicalPlan... inputs) throws SQLException; + + public abstract TableSet containsTables(); + + public abstract List outputColumns() throws SQLException; + + public void setUpperLimit(SQLExpr count) { + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/ConcatenateGen4Plan.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/ConcatenateGen4Plan.java new file mode 100644 index 0000000..9b78866 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/ConcatenateGen4Plan.java @@ -0,0 +1,59 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.logical; + +import com.jd.jdbc.context.PlanningContext; +import com.jd.jdbc.engine.PrimitiveEngine; +import com.jd.jdbc.engine.gen4.ConcatenateGen4Engine; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import lombok.Getter; + +public class ConcatenateGen4Plan extends LogicalPlanCommon implements LogicalPlan { + @Getter + private List sources; + + // These column offsets do not need to be typed checked - they usually contain weight_string() + // columns that are not going to be returned to the user + @Getter + private List noNeedToTypeCheck; + + public ConcatenateGen4Plan(List sources) { + this.sources = sources; + this.noNeedToTypeCheck = new ArrayList<>(); + } + + @Override + public void wireupGen4(PlanningContext ctx) throws SQLException { + for (LogicalPlan source : sources) { + source.wireupGen4(ctx); + } + } + + @Override + public PrimitiveEngine getPrimitiveEngine() throws SQLException { + List engines = new ArrayList<>(); + for (LogicalPlan source : sources) { + engines.add(source.getPrimitiveEngine()); + } + return new ConcatenateGen4Engine(engines, new HashMap<>()); + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/DistinctGen4Plan.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/DistinctGen4Plan.java new file mode 100644 index 0000000..e22da81 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/DistinctGen4Plan.java @@ -0,0 +1,56 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.logical; + +import com.jd.jdbc.engine.PrimitiveEngine; +import com.jd.jdbc.engine.gen4.CheckCol; +import com.jd.jdbc.engine.gen4.DistinctGen4Engine; +import java.sql.SQLException; +import java.sql.SQLFeatureNotSupportedException; +import java.util.List; + +public class DistinctGen4Plan extends LogicalPlanCommon implements LogicalPlan { + private List checkCols; + + private boolean needToTruncate; + + public DistinctGen4Plan(LogicalPlan source, List checkCols, boolean needToTruncate) { + this.checkCols = checkCols; + this.needToTruncate = needToTruncate; + this.setInput(source); + } + + @Override + public PrimitiveEngine getPrimitiveEngine() throws SQLException { + if (this.checkCols == null) { + // If we are missing the checkCols information, we are on the V3 planner and should produce a V3 Distinct + throw new SQLFeatureNotSupportedException(); + } + boolean truncate = false; + if (this.needToTruncate) { + for (CheckCol checkCol : this.checkCols) { + if (checkCol.getWsCol() != null) { + truncate = true; + break; + } + } + } + return new DistinctGen4Engine(this.getInput().getPrimitiveEngine(), this.checkCols, truncate); + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/FilterGen4Plan.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/FilterGen4Plan.java new file mode 100644 index 0000000..89539df --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/FilterGen4Plan.java @@ -0,0 +1,112 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.logical; + +import com.jd.jdbc.context.PlanningContext; +import com.jd.jdbc.engine.PrimitiveEngine; +import com.jd.jdbc.engine.gen4.FilterGen4Engine; +import com.jd.jdbc.engine.gen4.SimpleConverterLookup; +import com.jd.jdbc.evalengine.EvalEngine; +import com.jd.jdbc.planbuilder.Builder; +import com.jd.jdbc.planbuilder.Jointab; +import com.jd.jdbc.planbuilder.ResultColumn; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLName; +import java.sql.SQLException; +import java.util.List; +import lombok.Getter; +import lombok.Setter; + +@Getter +@Setter +public class FilterGen4Plan extends LogicalPlanCommon implements LogicalPlan { + + private FilterGen4Engine efilter; + + public FilterGen4Plan(PlanningContext ctx, LogicalPlan plan, SQLExpr expr) throws SQLException { + SimpleConverterLookup scl = new SimpleConverterLookup(ctx, plan); + EvalEngine.Expr predicate = EvalEngine.translate(expr, scl); + this.setInput(plan); + this.setEfilter(new FilterGen4Engine(expr, predicate)); + } + + public FilterGen4Plan(PlanningContext ctx, LogicalPlan plan, SQLExpr expr, boolean canPushProjection) throws SQLException { + SimpleConverterLookup scl = new SimpleConverterLookup(ctx, plan, canPushProjection); + EvalEngine.Expr predicate = EvalEngine.translate(expr, scl); + this.setInput(plan); + this.setEfilter(new FilterGen4Engine(expr, predicate)); + } + + @Override + public Integer order() { + return null; + } + + @Override + public List resultColumns() { + return null; + } + + @Override + public void reorder(Integer order) { + + } + + @Override + public void wireup(Builder bldr, Jointab jt) throws SQLException { + + } + + @Override + public void supplyVar(Integer from, Integer to, SQLName colName, String varName) throws SQLException { + + } + + @Override + public Builder.SupplyColResponse supplyCol(SQLName col) throws SQLException { + return null; + } + + @Override + public Integer supplyWeightString(Integer colNumber) throws SQLException { + return null; + } + + @Override + public PrimitiveEngine getPrimitiveEngine() throws SQLException { + this.efilter.setInput(this.getInput().getPrimitiveEngine()); + return this.getEfilter(); + } + + @Override + public LogicalPlan[] rewrite(LogicalPlan... inputs) throws SQLException { + return new LogicalPlan[0]; + } + + @Override + public TableSet containsTables() { + return null; + } + + @Override + public void setUpperLimit(SQLExpr count) throws SQLException { + + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/HashJoinPlan.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/HashJoinPlan.java new file mode 100644 index 0000000..3f582d8 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/HashJoinPlan.java @@ -0,0 +1,61 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.logical; + +import com.jd.jdbc.context.PlanningContext; +import com.jd.jdbc.engine.PrimitiveEngine; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import java.sql.SQLException; +import java.util.List; + +/** + * hashJoin is used to build a HashJoin primitive. + */ +public class HashJoinPlan extends AbstractGen4Plan{ + @Override + public void wireupGen4(PlanningContext ctx) throws SQLException { + + } + + @Override + public PrimitiveEngine getPrimitiveEngine() throws SQLException { + return null; + } + + @Override + public LogicalPlan[] inputs() throws SQLException { + return new LogicalPlan[0]; + } + + @Override + public LogicalPlan[] rewrite(LogicalPlan... inputs) throws SQLException { + return new LogicalPlan[0]; + } + + @Override + public TableSet containsTables() { + return null; + } + + @Override + public List outputColumns() throws SQLException { + return null; + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/JoinGen4Plan.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/JoinGen4Plan.java new file mode 100644 index 0000000..2e15492 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/JoinGen4Plan.java @@ -0,0 +1,146 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.logical; + +import com.jd.jdbc.context.PlanningContext; +import com.jd.jdbc.engine.Engine; +import com.jd.jdbc.engine.PrimitiveEngine; +import com.jd.jdbc.engine.gen4.JoinGen4Engine; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.ast.SQLName; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import lombok.Getter; +import lombok.Setter; + +/** + * joinGen4 is used to build a Join primitive. + * It's used to build an inner join and only used by the Gen4 planner + */ +public class JoinGen4Plan extends AbstractGen4Plan { + + /** + * Left and Right are the nodes for the join. + */ + @Getter + @Setter + LogicalPlan left; + + /** + * Left and Right are the nodes for the join. + */ + @Getter + @Setter + LogicalPlan right; + + /** + * The Opcode tells us if this is an inner or outer join + */ + @Getter + Engine.JoinOpcode opcode; + + /** + * These are the columns that will be produced by this plan. + * Negative offsets come from the LHS, and positive from the RHS + */ + + @Getter + List cols; + + /** + * Vars are the columns that will be sent from the LHS to the RHS + * the number is the offset on the LHS result, and the string is the bind variable name used in the RHS + */ + @Getter + Map vars; + + /** + * LHSColumns are the columns from the LHS used for the join. + * These are the same columns pushed on the LHS that are now used in the Vars field + */ + + @Getter + List lHSColumns; + + + public JoinGen4Plan(LogicalPlan left, LogicalPlan right, Engine.JoinOpcode opcode, List cols, Map vars, List lHSColumns) { + this.left = left; + this.right = right; + this.opcode = opcode; + this.cols = cols; + this.vars = vars; + this.lHSColumns = lHSColumns; + } + + @Override + public void wireupGen4(PlanningContext ctx) throws SQLException { + this.left.wireupGen4(ctx); + this.right.wireupGen4(ctx); + } + + @Override + public PrimitiveEngine getPrimitiveEngine() throws SQLException { + JoinGen4Engine engine = new JoinGen4Engine(this.opcode, this.vars); + engine.setLeft(this.left.getPrimitiveEngine()); + engine.setRight(this.right.getPrimitiveEngine()); + engine.setCols(this.cols); + return engine; + } + + @Override + public LogicalPlan[] inputs() throws SQLException { + return new LogicalPlan[] {this.left, this.right}; + } + + @Override + public LogicalPlan[] rewrite(LogicalPlan... inputs) throws SQLException { + if (inputs.length != 2) { + throw new SQLException("[JoinGen4]wrong number of children"); + } + this.left = inputs[0]; + this.right = inputs[1]; + return null; + } + + @Override + public TableSet containsTables() { + return this.left.containsTables().merge(this.right.containsTables()); + } + + @Override + public List outputColumns() throws SQLException { + return JoinGen4Plan.getOutputColumnsFromJoin(this.cols, this.left.outputColumns(), this.right.outputColumns()); + } + + public static List getOutputColumnsFromJoin(List ints, List lhs, List rhs) { + List cols = new ArrayList<>(ints.size()); + for (Integer col : ints) { + if (col < 0) { + col *= -1; + cols.add(lhs.get(col - 1)); + } else { + cols.add(rhs.get(col - 1)); + } + } + return cols; + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/LimitGen4Plan.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/LimitGen4Plan.java new file mode 100644 index 0000000..96157c3 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/LimitGen4Plan.java @@ -0,0 +1,132 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.logical; + +import com.jd.jdbc.engine.PrimitiveEngine; +import com.jd.jdbc.engine.gen4.LimitGen4Engine; +import com.jd.jdbc.evalengine.EvalEngine; +import com.jd.jdbc.planbuilder.Builder; +import com.jd.jdbc.planbuilder.Jointab; +import com.jd.jdbc.planbuilder.ResultColumn; +import com.jd.jdbc.planbuilder.semantics.SemTable; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLLimit; +import com.jd.jdbc.sqlparser.ast.SQLName; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import java.sql.SQLException; +import java.util.List; + +public class LimitGen4Plan extends LogicalPlanCommon implements LogicalPlan { + private LimitGen4Engine elimit; + + // newLimit builds a new limit. + public LimitGen4Plan(LogicalPlan plan) { + super.input = plan; + this.elimit = new LimitGen4Engine(); + } + + @Override + public Integer order() { + return null; + } + + @Override + public List resultColumns() { + return null; + } + + @Override + public void reorder(Integer order) { + + } + + @Override + public void wireup(Builder bldr, Jointab jt) throws SQLException { + + } + + @Override + public void supplyVar(Integer from, Integer to, SQLName colName, String varName) throws SQLException { + + } + + @Override + public Builder.SupplyColResponse supplyCol(SQLName col) throws SQLException { + return null; + } + + @Override + public Integer supplyWeightString(Integer colNumber) throws SQLException { + return null; + } + + @Override + public PrimitiveEngine getPrimitiveEngine() throws SQLException { + this.elimit.setInput(this.getInput().getPrimitiveEngine()); + return this.elimit; + } + + @Override + public LogicalPlan[] rewrite(LogicalPlan... inputs) throws SQLException { + return new LogicalPlan[0]; + } + + @Override + public TableSet containsTables() { + return null; + } + + @Override + public List outputColumns() throws SQLException { + return null; + } + + public static LogicalPlan createLimitPlan(LogicalPlan plan, SQLLimit limit) throws SQLException { + LimitGen4Plan limitPlan = new LimitGen4Plan(plan); + limitPlan.setLimit(limit); + return limitPlan; + } + + private void setLimit(SQLLimit limit) throws SQLException { + EvalEngine.Expr pv; + SemTable emptySemTable = new SemTable(); + try { + pv = EvalEngine.translate(limit.getRowCount(), emptySemTable); + } catch (SQLException e) { + throw new SQLException("unexpected expression in LIMIT: " + e.getMessage(), e); + } + this.elimit.setCount(pv); + + if (limit.getOffset() != null) { + EvalEngine.Expr pvOffset; + try { + pvOffset = EvalEngine.translate(limit.getRowCount(), emptySemTable); + } catch (SQLException e) { + throw new SQLException("unexpected expression in OFFSET: " + e.getMessage(), e); + } + this.elimit.setOffset(pvOffset); + } + } + + @Override + public void setUpperLimit(SQLExpr count) { + } + +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/LogicalPlan.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/LogicalPlan.java new file mode 100644 index 0000000..54a260b --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/LogicalPlan.java @@ -0,0 +1,201 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.logical; + +import com.jd.jdbc.common.tuple.Pair; +import com.jd.jdbc.context.PlanningContext; +import com.jd.jdbc.engine.PrimitiveEngine; +import com.jd.jdbc.planbuilder.Builder; +import com.jd.jdbc.planbuilder.Jointab; +import com.jd.jdbc.planbuilder.ResultColumn; +import com.jd.jdbc.planbuilder.gen4.PlanVisitor; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLName; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import java.sql.SQLException; +import java.util.List; + +/** + * logicalPlan defines the interface that a primitive must satisfy. + */ +public interface LogicalPlan { + /** + * Order is the execution order of the primitive. If there are subprimitives, + * the order is one above the order of the subprimitives. + * This is because the primitive executes its subprimitives first and + * processes their results to generate its own values. + * Please copy code from an existing primitive to define this function. + * + * @return + */ + Integer order(); + + /** + * ResultColumns returns the list of result columns the + * primitive returns. + * Please copy code from an existing primitive to define this function. + * + * @return + */ + List resultColumns(); + + /** + * Reorder reassigns order for the primitive and its sub-primitives. + * The input is the order of the previous primitive that should + * execute before this one. + * + * @param order + */ + void reorder(Integer order); + + + /** + * Wireup performs the wire-up work. Nodes should be traversed + * from right to left because the rhs nodes can request vars from + * the lhs nodes. + * + * @param bldr + * @param jt + * @throws SQLException + */ + void wireup(Builder bldr, Jointab jt) throws SQLException; + + /** + * WireupGen4 does the wire up work for the Gen4 planner + * + * @param ctx + * @throws SQLException + */ + void wireupGen4(PlanningContext ctx) throws SQLException; + + /** + * 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 + * just supply it to the 'to' node. Otherwise, it should request + * for it by calling SupplyCol on the 'from' sub-tree to request the + * column, and then supply it to the 'to' node. + * + * @param from + * @param to + * @param colName + * @param varName + * @throws SQLException + */ + void supplyVar(Integer from, Integer to, SQLName colName, String varName) throws SQLException; + + /** + * SupplyCol is meant to be used for the wire-up process. This function + * changes the primitive to supply the requested column and returns + * the resultColumn and column number of the result. SupplyCol + * is different from PushSelect because it may reuse an existing + * resultColumn, whereas PushSelect guarantees the addition of a new + * result column and returns a distinct symbol for it. + * + * @param col + * @return + * @throws SQLException + */ + Builder.SupplyColResponse supplyCol(SQLName col) throws SQLException; + + /** + * SupplyWeightString must supply a weight_string expression of the + * specified column. It returns an error if we cannot supply a weight column for it. + * + * @param colNumber + * @return + * @throws SQLException + */ + Integer supplyWeightString(Integer colNumber) throws SQLException; + + /** + * Primitive returns the underlying primitive. + * This function should only be called after Wireup is finished. + * + * @return {@link PrimitiveEngine} + */ + PrimitiveEngine getPrimitiveEngine() throws SQLException; + + /** + * Inputs are the children of this plan + * + * @return + * @throws SQLException + */ + LogicalPlan[] inputs() throws SQLException; + + /** + * Rewrite replaces the inputs of this plan with the ones provided + * + * @return + * @throws SQLException + */ + LogicalPlan[] rewrite(LogicalPlan... inputs) throws SQLException; + + /** + * 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 Gen4 planner + * + * @return + */ + TableSet containsTables(); + + /** + * OutputColumns shows the columns that this plan will produce + * + * @return + */ + List outputColumns() throws SQLException; + + /** + * SetUpperLimit is an optimization hint that tells that primitive + * that it does not need to return more than the specified number of rows. + * A primitive that cannot perform this can ignore the request. + * + * @param count + * @throws SQLException + */ + void setUpperLimit(SQLExpr count) throws SQLException; + + static LogicalPlan visit(LogicalPlan node, PlanVisitor visitor) throws SQLException { + if (visitor != null) { + Pair pair = visitor.func(node); + Boolean kontinue = pair.getLeft(); + LogicalPlan newNode = pair.getRight(); + if (!kontinue) { + return newNode; + } + } + LogicalPlan[] inputs = node.inputs(); + boolean rewrite = false; + for (int i = 0; i < inputs.length; i++) { + LogicalPlan input = inputs[i]; + LogicalPlan newInput = visit(input, visitor); + if (newInput != input) { + rewrite = true; + } + inputs[i] = newInput; + } + if (rewrite) { + node.rewrite(inputs); + } + return node; + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/LogicalPlanCommon.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/LogicalPlanCommon.java new file mode 100644 index 0000000..2b70940 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/LogicalPlanCommon.java @@ -0,0 +1,117 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.logical; + +import com.jd.jdbc.context.PlanningContext; +import com.jd.jdbc.engine.PrimitiveEngine; +import com.jd.jdbc.planbuilder.Builder; +import com.jd.jdbc.planbuilder.Jointab; +import com.jd.jdbc.planbuilder.ResultColumn; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLName; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import java.sql.SQLException; +import java.util.List; +import lombok.Getter; +import lombok.Setter; + +@Getter +@Setter +public class LogicalPlanCommon implements LogicalPlan { + + private int order; + + protected LogicalPlan input; + + @Override + public Integer order() { + return input.order(); + } + + @Override + public List resultColumns() { + return null; + } + + @Override + public void reorder(Integer order) { + this.getInput().reorder(order); + this.order = this.getInput().order() + 1; + } + + @Override + public void wireup(Builder bldr, Jointab jt) throws SQLException { + this.getInput().wireup(bldr, jt); + } + + @Override + public void wireupGen4(PlanningContext ctx) throws SQLException { + this.getInput().wireupGen4(ctx); + } + + @Override + public void supplyVar(Integer from, Integer to, SQLName colName, String varName) throws SQLException { + this.getInput().supplyVar(from, to, colName, varName); + } + + @Override + public Builder.SupplyColResponse supplyCol(SQLName col) throws SQLException { + return this.getInput().supplyCol(col); + } + + @Override + public Integer supplyWeightString(Integer colNumber) throws SQLException { + return this.getInput().supplyWeightString(colNumber); + } + + @Override + public PrimitiveEngine getPrimitiveEngine() throws SQLException { + return this.getInput().getPrimitiveEngine(); + } + + @Override + public LogicalPlan[] inputs() throws SQLException { + return new LogicalPlan[] {this.getInput()}; + } + + @Override + public LogicalPlan[] rewrite(LogicalPlan... inputs) throws SQLException { + if (inputs.length != 1) { + throw new SQLException("builderCommon: wrong number of inputs"); + } + this.input = inputs[0]; + return null; + } + + @Override + public TableSet containsTables() { + return this.getInput().containsTables(); + } + + @Override + public List outputColumns() throws SQLException { + return this.getInput().outputColumns(); + } + + @Override + public void setUpperLimit(SQLExpr count) throws SQLException { + this.getInput().setUpperLimit(count); + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/MemorySortGen4Plan.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/MemorySortGen4Plan.java new file mode 100644 index 0000000..2c32cd5 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/MemorySortGen4Plan.java @@ -0,0 +1,54 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.logical; + +import com.jd.jdbc.engine.PrimitiveEngine; +import com.jd.jdbc.engine.gen4.MemorySortGen4Engine; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import java.sql.SQLException; +import lombok.Getter; +import lombok.Setter; + +@Getter +@Setter +public class MemorySortGen4Plan extends ResultsBuilder implements LogicalPlan { + + private MemorySortGen4Engine eMemorySort; + + public MemorySortGen4Plan() { + + } + + private int findColNumber(MemorySortGen4Plan ms, SQLSelectItem expr) { + return -1; + } + + @Override + public PrimitiveEngine getPrimitiveEngine() throws SQLException { + PrimitiveEngine primitiveEngine = this.getInput().getPrimitiveEngine(); + this.getEMemorySort().setInput(primitiveEngine); + return this.eMemorySort; + } + + @Override + public void setUpperLimit(SQLExpr count) throws SQLException { + throw new SQLException("memorySort.Limit: unreachable"); + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/OrderedAggregateGen4Plan.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/OrderedAggregateGen4Plan.java new file mode 100644 index 0000000..94aaa40 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/OrderedAggregateGen4Plan.java @@ -0,0 +1,114 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.logical; + +import com.jd.jdbc.common.util.CollectionUtils; +import com.jd.jdbc.context.PlanningContext; +import com.jd.jdbc.engine.PrimitiveEngine; +import com.jd.jdbc.engine.gen4.AbstractAggregateGen4; +import com.jd.jdbc.engine.gen4.GroupByParams; +import com.jd.jdbc.engine.gen4.OrderedAggregateGen4Engine; +import com.jd.jdbc.engine.gen4.ScalarAggregateGen4Engine; +import com.jd.jdbc.planbuilder.Truncater; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLName; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import lombok.Getter; +import lombok.Setter; + +@Getter +@Setter +public class OrderedAggregateGen4Plan extends ResultsBuilder implements LogicalPlan, Truncater { + + private SQLName extraDistinct; + + /** + * preProcess is true if one of the aggregates needs preprocessing. + */ + private boolean preProcess; + + private boolean aggrOnEngine; + + /** + * aggregates specifies the aggregation parameters for each + * aggregation function: function opcode and input column number. + */ + private List aggregates = new ArrayList<>(); + + /** + * groupByKeys specifies the input values that must be used for + * the aggregation key. + */ + private List groupByKeys = new ArrayList<>(); + + private int truncateColumnCount; + + public OrderedAggregateGen4Plan() { + } + + public OrderedAggregateGen4Plan(List groupByKeys) { + this.groupByKeys = groupByKeys; + } + + @Override + public PrimitiveEngine getPrimitiveEngine() throws SQLException { + PrimitiveEngine primitiveEngine = this.getInput().getPrimitiveEngine(); + if (CollectionUtils.isEmpty(this.groupByKeys)) { + return new ScalarAggregateGen4Engine(this.preProcess, aggregates, this.aggrOnEngine, this.truncateColumnCount, null, primitiveEngine); + } + + return new OrderedAggregateGen4Engine(this.preProcess, this.aggregates, this.aggrOnEngine, this.truncateColumnCount, this.groupByKeys, null, primitiveEngine); + } + + + @Override + public void wireupGen4(PlanningContext ctx) throws SQLException { + this.getInput().wireupGen4(ctx); + } + + @Override + public List outputColumns() throws SQLException { + List outputCols = new ArrayList<>(); + // creates a deep clone of the outputColumns. + for (SQLSelectItem selectItem : super.input.outputColumns()) { + outputCols.add(selectItem.clone()); + } + + for (AbstractAggregateGen4.AggregateParams aggr : this.aggregates) { + outputCols.set(aggr.getCol(), new SQLSelectItem(aggr.getExpr(), aggr.getAlias())); + } + if (this.truncateColumnCount > 0) { + return outputCols.subList(0, this.truncateColumnCount); + } + return outputCols; + } + + @Override + public void setUpperLimit(SQLExpr count) throws SQLException { + + } + + @Override + public void setTruncateColumnCount(Integer count) { + this.truncateColumnCount = count; + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/ProjectionGen4Plan.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/ProjectionGen4Plan.java new file mode 100644 index 0000000..3f4726e --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/ProjectionGen4Plan.java @@ -0,0 +1,103 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.logical; + +import com.jd.jdbc.context.PlanningContext; +import com.jd.jdbc.engine.PrimitiveEngine; +import com.jd.jdbc.planbuilder.Builder; +import com.jd.jdbc.planbuilder.Jointab; +import com.jd.jdbc.planbuilder.ResultColumn; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLName; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import java.sql.SQLException; +import java.util.List; + +public class ProjectionGen4Plan implements LogicalPlan { + @Override + public Integer order() { + return null; + } + + @Override + public List resultColumns() { + return null; + } + + @Override + public void reorder(Integer order) { + + } + + @Override + public void wireup(Builder bldr, Jointab jt) throws SQLException { + + } + + @Override + public void wireupGen4(PlanningContext ctx) throws SQLException { + + } + + @Override + public void supplyVar(Integer from, Integer to, SQLName colName, String varName) throws SQLException { + + } + + @Override + public Builder.SupplyColResponse supplyCol(SQLName col) throws SQLException { + return null; + } + + @Override + public Integer supplyWeightString(Integer colNumber) throws SQLException { + return null; + } + + @Override + public PrimitiveEngine getPrimitiveEngine() throws SQLException { + return null; + } + + @Override + public LogicalPlan[] inputs() throws SQLException { + return new LogicalPlan[0]; + } + + @Override + public LogicalPlan[] rewrite(LogicalPlan... inputs) throws SQLException { + return new LogicalPlan[0]; + } + + @Override + public TableSet containsTables() { + return null; + } + + @Override + public List outputColumns() throws SQLException { + return null; + } + + @Override + public void setUpperLimit(SQLExpr count) throws SQLException { + + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/ResultsBuilder.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/ResultsBuilder.java new file mode 100644 index 0000000..ae9512a --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/ResultsBuilder.java @@ -0,0 +1,47 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.logical; + +import com.jd.jdbc.planbuilder.ResultColumn; +import com.jd.jdbc.planbuilder.Truncater; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import lombok.Getter; +import lombok.Setter; + +/** + * resultsBuilder is a superset of logicalPlanCommon. It also handles resultsColumn functionality. + */ +@Getter +@Setter +public class ResultsBuilder extends LogicalPlanCommon { + + private List resultColumns; + + private Map weightStrings; + + private Truncater truncater; + + public ResultsBuilder() { + weightStrings = new HashMap<>(); + resultColumns = new ArrayList<>(); + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/RouteGen4Plan.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/RouteGen4Plan.java new file mode 100644 index 0000000..af0a1d3 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/RouteGen4Plan.java @@ -0,0 +1,198 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.logical; + +import com.jd.jdbc.context.PlanningContext; +import com.jd.jdbc.engine.Engine; +import com.jd.jdbc.engine.PrimitiveEngine; +import com.jd.jdbc.engine.gen4.RouteGen4Engine; +import com.jd.jdbc.engine.gen4.VindexLookup; +import com.jd.jdbc.planbuilder.gen4.Gen4Planner; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.SQLUtils; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLLimit; +import com.jd.jdbc.sqlparser.ast.SQLStatement; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectQuery; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectStatement; +import com.jd.jdbc.sqlparser.ast.statement.SQLUnionQuery; +import com.jd.jdbc.sqlparser.dialect.mysql.ast.statement.MySqlSelectQueryBlock; +import com.jd.jdbc.sqlparser.dialect.mysql.visitor.VtFormatImpossibleQueryVisitor; +import com.jd.jdbc.sqlparser.dialect.mysql.visitor.VtRouteWireupFixUpAstVisitor; +import com.jd.jdbc.vindexes.LookupPlanable; +import java.sql.SQLException; +import java.sql.SQLFeatureNotSupportedException; +import java.util.Arrays; +import java.util.List; +import lombok.Getter; +import lombok.Setter; + +public class RouteGen4Plan extends AbstractGen4Plan { + + // Select is the AST for the query fragment that will be + // executed by this route. + @Getter + @Setter + public SQLSelectQuery select; + + // condition stores the AST condition that will be used + // to resolve the ERoute Values field. + @Getter + public SQLExpr condition; + + // eroute is the primitive being built. + @Getter + public RouteGen4Engine eroute; + + // is the engine primitive we will return from the Primitive() method. Note that it could be different than eroute + public PrimitiveEngine enginePrimitive; + + // tables keeps track of which tables this route is covering + public TableSet tables; + + public RouteGen4Plan() { + } + + public RouteGen4Plan(RouteGen4Engine eroute, SQLSelectQuery sel, TableSet tables, SQLExpr condition) { + this.eroute = eroute; + this.select = sel; + this.tables = tables; + this.condition = condition; + } + + @Override + public void wireupGen4(PlanningContext ctx) throws SQLException { + // this.eroute.setQuery(SQLUtils.toMySqlString(this.select, SQLUtils.NOT_FORMAT_OPTION).trim()); + // this.enginePrimitive = this.eroute; + + // todo + + this.prepareTheAST(); + + this.eroute.setQuery(SQLUtils.toMySqlString(this.select, SQLUtils.NOT_FORMAT_OPTION).trim()); + + SQLSelectQuery selectClone = this.select.clone(); + VtFormatImpossibleQueryVisitor formatImpossibleQueryVisitor = new VtFormatImpossibleQueryVisitor(); + selectClone.accept(formatImpossibleQueryVisitor); + this.eroute.setFieldQuery(SQLUtils.toMySqlString(selectClone, SQLUtils.NOT_FORMAT_OPTION).trim()); + this.eroute.setSelectFieldQuery(selectClone); + + if (!(this.eroute.getRoutingParameters().getVindex() instanceof LookupPlanable)) { + this.enginePrimitive = this.eroute; + return; + } + + LookupPlanable planableVindex = (LookupPlanable) this.eroute.getRoutingParameters().getVindex(); + String[] querys = planableVindex.Query(); + String query = querys[0]; + String[] queryArgs = Arrays.copyOfRange(querys, 1, querys.length); + + SQLStatement stmt = SQLUtils.parseSingleMysqlStatement(query); + + PrimitiveEngine lookupPrimitive = Gen4Planner.gen4SelectStmtPlanner(query, ctx.getVschema().getDefaultKeyspace(), (SQLSelectStatement) stmt, null, ctx.getVschema().getVSchemaManager()); + + this.enginePrimitive = new VindexLookup( + this.eroute.getRoutingParameters().getRouteOpcode(), + planableVindex, + this.eroute.getRoutingParameters().getKeyspace(), + this.eroute.getRoutingParameters().getValues(), + this.eroute, + Arrays.asList(queryArgs), + lookupPrimitive + ); + + this.eroute.getRoutingParameters().setRouteOpcode(Engine.RouteOpcode.ByDestination); + this.eroute.getRoutingParameters().setValues(null); + this.eroute.getRoutingParameters().setVindex(null); + + //throw new SQLFeatureNotSupportedException("unsupported vindexloopup"); + + } + + @Override + public PrimitiveEngine getPrimitiveEngine() throws SQLException { + return enginePrimitive; + } + + @Override + public LogicalPlan[] inputs() throws SQLException { + return new LogicalPlan[0]; + } + + @Override + public LogicalPlan[] rewrite(LogicalPlan... inputs) throws SQLException { + if (inputs.length != 0) { + throw new SQLException("[RouteGen4]route: wrong number of inputs"); + } + return null; + } + + @Override + public TableSet containsTables() { + return tables; + } + + @Override + public List outputColumns() throws SQLException { + if (this.select instanceof MySqlSelectQueryBlock) { + MySqlSelectQueryBlock mySqlSelectQueryBlock = (MySqlSelectQueryBlock) this.select; + return mySqlSelectQueryBlock.getSelectList(); + } else { + throw new SQLFeatureNotSupportedException(); + } + } + + + // ========== ================ + + public void setLimit(SQLLimit limit) { + if (this.select instanceof MySqlSelectQueryBlock) { + ((MySqlSelectQueryBlock) this.select).setLimit(limit); + } else if (this.select instanceof SQLUnionQuery) { + ((SQLUnionQuery) this.select).setLimit(limit); + } + } + + @Override + public void setUpperLimit(SQLExpr count) { + ((MySqlSelectQueryBlock) this.select).setLimit(new SQLLimit(count)); + } + + // public Boolean isLocal() + + public boolean isSingleShard() { + switch (this.eroute.getRoutingParameters().getRouteOpcode()) { + case SelectUnsharded: + case SelectDBA: + case SelectNext: + case SelectEqualUnique: + case SelectReference: + return true; + default: + return false; + } + } + + // prepareTheAST does minor fixups of the SELECT struct before producing the query string + public void prepareTheAST() { + VtRouteWireupFixUpAstVisitor fixUpAstVisitor = new VtRouteWireupFixUpAstVisitor(null); + this.select.accept(fixUpAstVisitor); + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/SimpleProjectionGen4Plan.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/SimpleProjectionGen4Plan.java new file mode 100644 index 0000000..f8bcb17 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/logical/SimpleProjectionGen4Plan.java @@ -0,0 +1,130 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.logical; + +import com.jd.jdbc.engine.PrimitiveEngine; +import com.jd.jdbc.engine.gen4.SimpleProjectionGen4Engine; +import com.jd.jdbc.planbuilder.Builder; +import com.jd.jdbc.planbuilder.Jointab; +import com.jd.jdbc.planbuilder.ResultColumn; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLName; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import lombok.Getter; +import lombok.Setter; + + +// simpleProjection is used for wrapping a derived table. +// This primitive wraps any derived table that results +// in something that's not a route. It builds a +// 'table' for the derived table allowing higher level +// constructs to reference its columns. If a derived table +// results in a route primitive, we instead build +// a new route that keeps the subquery in the FROM +// clause, because a route is more versatile than +// a simpleProjection. +// this should not be used by the gen4 planner +@Getter +@Setter +public class SimpleProjectionGen4Plan extends LogicalPlanCommon implements LogicalPlan { + + private List resultColumns; + + private SimpleProjectionGen4Engine eSimpleProjection; + + public SimpleProjectionGen4Plan(LogicalPlan plan) { + this.setInput(plan); + this.setESimpleProjection(new SimpleProjectionGen4Engine()); + } + + @Override + public Integer order() { + return null; + } + + @Override + public List resultColumns() { + return this.getResultColumns(); + } + + @Override + public void reorder(Integer order) { + + } + + @Override + public void wireup(Builder bldr, Jointab jt) throws SQLException { + + } + + @Override + public void supplyVar(Integer from, Integer to, SQLName colName, String varName) throws SQLException { + + } + + @Override + public Builder.SupplyColResponse supplyCol(SQLName col) throws SQLException { + return null; + } + + @Override + public Integer supplyWeightString(Integer colNumber) throws SQLException { + return null; + } + + @Override + public PrimitiveEngine getPrimitiveEngine() throws SQLException { + this.getESimpleProjection().setInput(this.getInput().getPrimitiveEngine()); + return this.getESimpleProjection(); + } + + @Override + public LogicalPlan[] inputs() throws SQLException { + return new LogicalPlan[0]; + } + + @Override + public LogicalPlan[] rewrite(LogicalPlan... inputs) throws SQLException { + return new LogicalPlan[0]; + } + + @Override + public TableSet containsTables() { + return null; + } + + @Override + public void setUpperLimit(SQLExpr count) throws SQLException { + + } + + @Override + public List outputColumns() throws SQLException { + List exprs = new ArrayList<>(this.getESimpleProjection().getCols().size()); + List outputCols = this.getInput().outputColumns(); + for (Integer colID : this.getESimpleProjection().getCols()) { + exprs.add(outputCols.get(colID)); + } + return exprs; + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/Operator.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/Operator.java new file mode 100644 index 0000000..14dd120 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/Operator.java @@ -0,0 +1,52 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.operator; + +import com.jd.jdbc.planbuilder.semantics.SemTable; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import java.sql.SQLException; + +/** + * Operator forms the tree of operators, representing the declarative query provided. + */ +public interface Operator { + /** + * TableID returns a TableSet of the tables contained within + * + * @return + */ + TableSet tableID(); + + /** + * UnsolvedPredicates returns any predicates that have dependencies on the given Operator and + * on the outside of it (a parent Select expression, any other table not used by Operator, etc). + * + * @param semTable + * @return + */ + SQLSelectItem unsolvedPredicates(SemTable semTable); + + /** + * CheckValid checks if we have a valid operator tree, and returns an error if something is wrong + * + * @throws SQLException + */ + void checkValid() throws SQLException; +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/OperatorFuncs.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/OperatorFuncs.java new file mode 100644 index 0000000..1e4406d --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/OperatorFuncs.java @@ -0,0 +1,252 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.operator; + +import com.jd.jdbc.common.tuple.Pair; +import com.jd.jdbc.common.tuple.Triple; +import com.jd.jdbc.context.PlanningContext; +import com.jd.jdbc.planbuilder.gen4.operator.physical.ApplyJoin; +import com.jd.jdbc.planbuilder.gen4.operator.physical.Filter; +import com.jd.jdbc.planbuilder.gen4.operator.physical.PhysicalOperator; +import com.jd.jdbc.planbuilder.gen4.operator.physical.Route; +import com.jd.jdbc.planbuilder.gen4.operator.physical.Table; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.SQLUtils; +import com.jd.jdbc.sqlparser.SqlParser; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLName; +import com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOpExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOperator; +import com.jd.jdbc.sqlparser.visitor.VtBreakExpressionInLHSandRHSVisitor; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +public class OperatorFuncs { + + /** + * PushPredicate is used to push predicates. It pushed it as far down as is possible in the tree. + * If we encounter a join and the predicate depends on both sides of the join, the predicate will be split into two parts, + * where data is fetched from the LHS of the join to be used in the evaluation on the RHS + * + * @param ctx + * @param expr + * @param op + * @return + */ + public static PhysicalOperator pushPredicate(PlanningContext ctx, SQLExpr expr, PhysicalOperator op) throws SQLException { + if (op instanceof Route) { + ((Route) op).updateRoutingLogic(ctx, expr); + PhysicalOperator newSrc = pushPredicate(ctx, expr, ((Route) op).getSource()); + ((Route) op).setSource(newSrc); + return op; + } + + if (op instanceof ApplyJoin) { + TableSet deps = ctx.getSemTable().recursiveDeps(expr); + if (deps.isSolvedBy(((ApplyJoin) op).getLHS().tableID())) { + PhysicalOperator newSrc = pushPredicate(ctx, expr, ((ApplyJoin) op).getLHS()); + ((ApplyJoin) op).setLHS(newSrc); + return op; + } + + if (deps.isSolvedBy(((ApplyJoin) op).getRHS().tableID())) { + if (!((ApplyJoin) op).getLeftJoin()) { + PhysicalOperator newSrc = pushPredicate(ctx, expr, ((ApplyJoin) op).getRHS()); + ((ApplyJoin) op).setRHS(newSrc); + return op; + } + + // we are looking for predicates like `tbl.col = <>` or `<> = tbl.col`, + // where tbl is on the rhs of the left outer join + + if (SqlParser.isComparison(expr) + && ((SQLBinaryOpExpr) expr).getOperator() != SQLBinaryOperator.LessThanOrEqualOrGreaterThan) { + SQLExpr left = ((SQLBinaryOpExpr) expr).getLeft(); + SQLExpr right = ((SQLBinaryOpExpr) expr).getRight(); + if ( + (left instanceof SQLName) + && ctx.getSemTable().recursiveDeps(left).isSolvedBy(((ApplyJoin) op).getRHS().tableID()) + || (right instanceof SQLName) + && ctx.getSemTable().recursiveDeps(right).isSolvedBy(((ApplyJoin) op).getRHS().tableID()) + ) { + // When the predicate we are pushing is using information from an outer table, we can + // check whether the predicate is "null-intolerant" or not. Null-intolerant in this context means that + // the predicate will not return true if the table columns are null. + // Since an outer join is an inner join with the addition of all the rows from the left-hand side that + // matched no rows on the right-hand, if we are later going to remove all the rows where the right-hand + // side did not match, we might as well turn the join into an inner join. + + // This is based on the paper "Canonical Abstraction for Outerjoin Optimization" by J Rao et al + + ((ApplyJoin) op).setLeftJoin(false); + PhysicalOperator newSrc = pushPredicate(ctx, expr, ((ApplyJoin) op).getRHS()); + ((ApplyJoin) op).setRHS(newSrc); + return op; + } + } + // finally, if we can't turn the outer join into an inner, + // we need to filter after the join has been evaluated + return new Filter(op, new ArrayList<>(Arrays.asList(expr))); + } + + if (deps.isSolvedBy(op.tableID())) { + // breakExpressionInLHSandRHS + Triple, List, SQLExpr> tripleRet = breakExpressioninLHSandRHS(ctx, expr, ((ApplyJoin) op).getLHS().tableID()); + + Pair> pairRet = pushOutputColumns(ctx, ((ApplyJoin) op).getLHS(), tripleRet.getMiddle()); + + ((ApplyJoin) op).setLHS(pairRet.getLeft()); + + for (int i = 0; i < pairRet.getRight().size(); i++) { + String bvName = tripleRet.getLeft().get(i); + int idx = pairRet.getRight().get(i); + ((ApplyJoin) op).getVars().put(bvName, idx); + } + + PhysicalOperator newSrc = pushPredicate(ctx, tripleRet.getRight(), ((ApplyJoin) op).getRHS()); + ((ApplyJoin) op).setRHS(newSrc); + + ((ApplyJoin) op).setPredicate(SQLBinaryOpExpr.and(((ApplyJoin) op).getPredicate(), expr)); + return op; + } + + throw new SQLException("Cannot push predicate: " + expr.toString()); + } + + if (op instanceof Table) { + return new Filter(op, new ArrayList<>(Arrays.asList(expr))); + } + + if (op instanceof Filter) { + ((Filter) op).getPredicates().add(expr); + return op; + } + // TODO derived + + throw new SQLException("we cannot push predicates into " + op.getClass().getName()); + } + + /** + * BreakExpressionInLHSandRHS takes an expression and + * extracts the parts that are coming from one of the sides into `ColName`s that are needed + * + * @param ctx + * @param expr + * @param lhs + * @return + */ + public static Triple, List, SQLExpr> breakExpressioninLHSandRHS( + PlanningContext ctx, SQLExpr expr, TableSet lhs) throws SQLException { + SQLExpr rewrittenExpr = expr.clone(); + + VtBreakExpressionInLHSandRHSVisitor visitor = new VtBreakExpressionInLHSandRHSVisitor(lhs, ctx.getSemTable()); + rewrittenExpr.accept(visitor); + if (visitor.getBErr()) { + throw new SQLException(visitor.getErrMsg()); + } + List joinPredicates = ctx.getJoinPredicates().get(expr); + if (joinPredicates != null) { + joinPredicates.add(rewrittenExpr); + } + return Triple.of(visitor.getBvNames(), visitor.getColumus(), rewrittenExpr); + } + + /** + * PushOutputColumns will push the columns to the table they originate from, + * making sure that intermediate operators pass the data through + * + * @param ctx + * @param op + * @param columns + * @return + */ + public static Pair> pushOutputColumns(PlanningContext ctx, PhysicalOperator op, List columns) throws SQLException { + if (op instanceof Route) { + Pair> ret = pushOutputColumns(ctx, ((Route) op).getSource(), columns); + ((Route) op).setSource(ret.getLeft()); + return Pair.of(op, ret.getRight()); + } + + if (op instanceof ApplyJoin) { + List toTheLeft = new ArrayList<>(); + List lhs = new ArrayList<>(); + List rhs = new ArrayList<>(); + + for (SQLName col : columns) { + if (ctx.getSemTable().recursiveDeps(col).isSolvedBy(((ApplyJoin) op).getLHS().tableID())) { + lhs.add(col); + toTheLeft.add(true); + } else { + rhs.add(col); + toTheLeft.add(false); + } + } + Pair> retLeft = pushOutputColumns(ctx, ((ApplyJoin) op).getLHS(), lhs); + ((ApplyJoin) op).setLHS(retLeft.getLeft()); + + Pair> retRight = pushOutputColumns(ctx, ((ApplyJoin) op).getRHS(), rhs); + ((ApplyJoin) op).setRHS(retRight.getLeft()); + + List outputColumns = new ArrayList<>(toTheLeft.size()); + + int l = 0, r = 0; + for (int i = 0; i < toTheLeft.size(); i++) { + outputColumns.add(((ApplyJoin) op).getColumns().size()); + if (toTheLeft.get(i)) { + ((ApplyJoin) op).getColumns().add(0 - retLeft.getRight().get(l) - 1); + l++; + } else { + ((ApplyJoin) op).getColumns().add(retRight.getRight().get(r) + 1); + r++; + } + } + return Pair.of(op, outputColumns); + } + + if (op instanceof Table) { + List offsets = new ArrayList<>(); + for (SQLName col : columns) { + boolean exists = false; + for (int idx = 0; idx < ((Table) op).getColumns().size(); idx++) { + if (SQLUtils.nameEquals(col, ((Table) op).getColumns().get(idx))) { + exists = true; + offsets.add(idx); + } + } + if (!exists) { + offsets.add(((Table) op).getColumns().size()); + ((Table) op).getColumns().add(col); + } + } + return Pair.of(op, offsets); + } + + if (op instanceof Filter) { + Pair> ret = pushOutputColumns(ctx, ((Filter) op).getSource(), columns); + ((Filter) op).setSource(ret.getLeft()); + return Pair.of(op, ret.getRight()); + } + + //TODO vindex derived + + throw new SQLException("we cannot push output columns into" + op.getClass().getName()); + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/OperatorToQuery.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/OperatorToQuery.java new file mode 100644 index 0000000..321e598 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/OperatorToQuery.java @@ -0,0 +1,200 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.operator; + +import com.jd.jdbc.context.PlanningContext; +import com.jd.jdbc.planbuilder.gen4.operator.physical.ApplyJoin; +import com.jd.jdbc.planbuilder.gen4.operator.physical.Filter; +import com.jd.jdbc.planbuilder.gen4.operator.physical.PhysicalOperator; +import com.jd.jdbc.planbuilder.gen4.operator.physical.Table; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLName; +import com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOpExpr; +import com.jd.jdbc.sqlparser.ast.statement.SQLExprTableSource; +import com.jd.jdbc.sqlparser.ast.statement.SQLJoinTableSource; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectQuery; +import com.jd.jdbc.sqlparser.ast.statement.SQLTableSource; +import com.jd.jdbc.sqlparser.dialect.mysql.ast.statement.MySqlSelectQueryBlock; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; + +public class OperatorToQuery { + + PlanningContext ctx; + + MySqlSelectQueryBlock sel; + + List tableNames = new ArrayList<>(); + + public OperatorToQuery(PlanningContext ctx) { + this.ctx = ctx; + } + + public static SQLSelectQuery toSQL(PlanningContext ctx, PhysicalOperator op) throws SQLException { + OperatorToQuery q = new OperatorToQuery(ctx); + q.buildQuery(op); + q.sortTables(); + return q.sel; + } + + private void buildQuery(PhysicalOperator op) throws SQLException { + if (op instanceof Table) { + this.buildQueryForTable((Table) op); + } else if (op instanceof Filter) { + this.buildQuery(((Filter) op).getSource()); + for (SQLExpr pred : ((Filter) op).getPredicates()) { + this.addPredicate(pred); + } + } else if (op instanceof ApplyJoin) { + this.buildQuery(((ApplyJoin) op).getLHS()); + // If we are going to add the predicate used in join here + // We should not add the predicate's copy of when it was split into + // two parts. To avoid this, we use the SkipPredicates map. + List joinPredicates = this.ctx.getJoinPredicates().get(((ApplyJoin) op).getPredicate()); + if (joinPredicates != null) { + for (SQLExpr expr : joinPredicates) { + this.ctx.getSkipPredicates().remove(expr); + } + } + + OperatorToQuery qbR = new OperatorToQuery(this.ctx); + qbR.buildQuery(((ApplyJoin) op).getRHS()); + if (((ApplyJoin) op).getLeftJoin()) { + this.joinOuterWith(qbR, ((ApplyJoin) op).getPredicate()); + } else { + this.joinInnerWith(qbR, ((ApplyJoin) op).getPredicate()); + } + } else { + throw new SQLException("gen4 planner unsupported !! " + op.getClass().toString()); + } + } + + private void buildQueryForTable(Table op) { + String dbName = ""; + this.addTable(op.getQTable().getTable().getName(), op.getQTable().getAlias().computeAlias(), op.tableID(), op.getQTable().getAlias()); + for (SQLExpr expr : op.getQTable().getPredicates()) { + this.addPredicate(expr); // add where + } + + // Add Projection + for (SQLName columnExpr : op.getColumns()) { + this.sel.addSelectItem(columnExpr); + } + + } + + + private void addPredicate(SQLExpr expr) { + if (this.ctx.getSkipPredicates().get(expr) != null) { + // This is a predicate that was added to the RHS of an ApplyJoin. + // The original predicate will be added, so we don't have to add this here + return; + } + + /* + * addWhere 已经处理过多个条件的情况,不需要再拆分 + * + if (this.sel.getWhere() == null){ + this.sel.addWhere(expr); + return; + } + List filters = PlanBuilder.splitAndExpression(null, expr); + for(SQLExpr filter: filters){ + this.sel.addWhere(filter); + } + */ + + this.sel.addWhere(expr); + + } + + private void sortTables() { + + } + + private void addTable(String tableName, String alias, TableSet tableID, SQLExprTableSource source) { + this.addTableExpr(tableName, source, tableID); + } + + private void addTableExpr(String tableName, SQLExprTableSource tableSource, TableSet tableID) { + if (this.sel == null) { + this.sel = new MySqlSelectQueryBlock(); + } + + // SQLExprTableSource tableSource = new SQLExprTableSource(); + try { + this.ctx.getSemTable().replaceTableSetFor(tableID, tableSource); + } catch (SQLException ex) { + + } + + this.sel.setFrom(tableSource); + this.tableNames.add(tableName); + } + + private void joinInnerWith(OperatorToQuery other, SQLExpr onCondition) { + SQLTableSource lhs = this.sel.getFrom(); + SQLTableSource rhs = other.sel.getFrom(); + SQLTableSource newTableSource = new SQLJoinTableSource(lhs, SQLJoinTableSource.JoinType.COMMA, rhs, null); + this.sel.setFrom(newTableSource); + + this.sel.getSelectList().addAll(other.sel.getSelectList()); + + SQLExpr predicate = null; + + if (this.sel.getWhere() != null) { + predicate = this.sel.getWhere(); + } + + if (other.sel.getWhere() != null) { + predicate = SQLBinaryOpExpr.and(predicate, other.sel.getWhere()); + } + + if (predicate != null) { + this.sel.setWhere(predicate); + } + this.addPredicate(onCondition); + } + + private void joinOuterWith(OperatorToQuery other, SQLExpr onCondition) { + SQLTableSource lhs = this.sel.getFrom(); + SQLTableSource rhs = other.sel.getFrom(); + + SQLTableSource newTableSource = new SQLJoinTableSource(lhs, SQLJoinTableSource.JoinType.LEFT_OUTER_JOIN, rhs, onCondition); + this.sel.setFrom(newTableSource); + + this.sel.getSelectList().addAll(other.sel.getSelectList()); + + SQLExpr predicate = null; + + if (this.sel.getWhere() != null) { + predicate = this.sel.getWhere(); + } + + if (other.sel.getWhere() != null) { + predicate = SQLBinaryOpExpr.and(predicate, other.sel.getWhere()); + } + + if (predicate != null) { + this.sel.setWhere(predicate); + } + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/OperatorTransformers.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/OperatorTransformers.java new file mode 100644 index 0000000..8219b1b --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/OperatorTransformers.java @@ -0,0 +1,465 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.operator; + +import com.jd.jdbc.context.PlanningContext; +import com.jd.jdbc.engine.Engine; +import com.jd.jdbc.engine.gen4.CheckCol; +import com.jd.jdbc.engine.gen4.RouteGen4Engine; +import com.jd.jdbc.evalengine.EvalEngine; +import com.jd.jdbc.planbuilder.gen4.Gen4Planner; +import com.jd.jdbc.planbuilder.gen4.HorizonPlanning; +import com.jd.jdbc.planbuilder.gen4.logical.ConcatenateGen4Plan; +import com.jd.jdbc.planbuilder.gen4.logical.DistinctGen4Plan; +import com.jd.jdbc.planbuilder.gen4.logical.FilterGen4Plan; +import com.jd.jdbc.planbuilder.gen4.logical.JoinGen4Plan; +import com.jd.jdbc.planbuilder.gen4.logical.LogicalPlan; +import com.jd.jdbc.planbuilder.gen4.logical.RouteGen4Plan; +import com.jd.jdbc.planbuilder.gen4.operator.physical.ApplyJoin; +import com.jd.jdbc.planbuilder.gen4.operator.physical.Filter; +import com.jd.jdbc.planbuilder.gen4.operator.physical.PhysicalOperator; +import com.jd.jdbc.planbuilder.gen4.operator.physical.Route; +import com.jd.jdbc.planbuilder.gen4.operator.physical.RoutePlanning; +import com.jd.jdbc.planbuilder.gen4.operator.physical.Table; +import com.jd.jdbc.planbuilder.gen4.operator.physical.Union; +import com.jd.jdbc.planbuilder.semantics.Scoper; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLName; +import com.jd.jdbc.sqlparser.ast.SQLSetQuantifier; +import com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOpExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOperator; +import com.jd.jdbc.sqlparser.ast.expr.SQLInListExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLInSubQueryExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLVariantRefListExpr; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectQuery; +import com.jd.jdbc.sqlparser.ast.statement.SQLUnionOperator; +import com.jd.jdbc.sqlparser.ast.statement.SQLUnionQuery; +import com.jd.jdbc.sqlparser.dialect.mysql.ast.statement.MySqlSelectQueryBlock; +import com.jd.jdbc.vindexes.SingleColumn; +import com.jd.jdbc.vindexes.VKeyspace; +import com.jd.jdbc.vindexes.Vindex; +import java.sql.SQLException; +import java.sql.SQLFeatureNotSupportedException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.TreeSet; + +public class OperatorTransformers { + public static LogicalPlan transformToLogicalPlan(PlanningContext ctx, PhysicalOperator op, boolean isRoot) throws SQLException { + if (op instanceof Route) { + return transformRoutePlan(ctx, (Route) op); + } + if (op instanceof ApplyJoin) { + return transformApplyJoinPlan(ctx, (ApplyJoin) op); + } + if (op instanceof Union) { + return transformUnionPlan(ctx, (Union) op, isRoot); + } + if (op instanceof Filter) { + LogicalPlan plan = transformToLogicalPlan(ctx, ((Filter) op).getSource(), false); + SQLExpr ast = SQLBinaryOpExpr.combine(((Filter) op).getPredicates(), SQLBinaryOperator.BooleanAnd); + return new FilterGen4Plan(ctx, plan, ast, true); + } + return null; + } + + private static LogicalPlan transformUnionPlan(PlanningContext ctx, Union op, boolean isRoot) throws SQLException { + List sources; + if (op.isDistinct()) { + sources = transformAndMerge(ctx, op); + for (LogicalPlan source : sources) { + pushDistinct(source); + } + } else { + sources = transformAndMergeInOrder(ctx, op); + } + + LogicalPlan result; + if (sources.size() == 1) { + LogicalPlan src = sources.get(0); + if (src instanceof RouteGen4Plan && ((RouteGen4Plan) src).isSingleShard()) { + // if we have a single shard route, we don't need to do anything to make it distinct + // TODO + // rb.Select.SetLimit(op.limit) + // rb.Select.SetOrderBy(op.ordering) + return src; + } + result = src; + } else { + if (op.getOrderBy() != null) { + throw new SQLFeatureNotSupportedException("can't do ORDER BY on top of UNION"); + } + result = new ConcatenateGen4Plan(sources); + } + + if (op.isDistinct()) { + List colls = getCollationsFor(ctx, op); + List checkCols = getCheckColsForUnion(ctx, result, colls); + return new DistinctGen4Plan(result, checkCols, isRoot); + } + return result; + } + + private static List getCollationsFor(PlanningContext ctx, Union n) throws SQLException { + // TODO: coerce selects' select expressions' collations + List colls = new ArrayList<>(); + SQLSelectQuery select1 = n.getSelectStmts().get(0); + if (select1 instanceof MySqlSelectQueryBlock) { + for (SQLSelectItem item : ((MySqlSelectQueryBlock) select1).getSelectList()) { + colls.add(0); + } + } else { + throw new SQLException("Unknown expectations"); + } + return colls; + } + + private static List getCheckColsForUnion(PlanningContext ctx, LogicalPlan result, List colls) throws SQLException { + List checkCols = new ArrayList<>(colls.size()); + for (int i = 0; i < colls.size(); i++) { + CheckCol checkCol = new CheckCol(i, colls.get(i)); + if (colls.get(i) != 0) { + checkCols.add(checkCol); + continue; + } + // We might need a weight string - let's push one + // `might` because we just don't know what type we are dealing with. + // If we encounter a numerical value, we don't need any weight_string values + Integer newOffset = pushWeightStringForDistinct(ctx, result, i); + checkCol.setWsCol(newOffset); + checkCols.add(checkCol); + } + return checkCols; + } + + private static List transformAndMerge(PlanningContext ctx, Union op) throws SQLException { + List sources = new ArrayList<>(); + for (int i = 0; i < op.getSources().size(); i++) { + // first we go over all the operator inputs and turn them into logical plans, + // including horizon planning + PhysicalOperator source = op.getSources().get(i); + LogicalPlan plan = createLogicalPlan(ctx, source, op.getSelectStmts().get(i)); + sources.add(plan); + } + + // next we'll go over all the plans from and check if any two can be merged. if they can, they are merged, + // and we continue checking for pairs of plans that can be merged into a single route + int idx = 0; + while (idx < sources.size()) { + Set keep = new HashSet<>(); + LogicalPlan srcA = sources.get(idx); + boolean merged = false; + for (int j = 0; j < sources.size(); j++) { + if (j <= idx) { + continue; + } + LogicalPlan srcB = sources.get(j); + LogicalPlan newPlan = mergeUnionLogicalPlans(ctx, srcA, srcB); + if (newPlan != null) { + sources.set(idx, newPlan); + srcA = newPlan; + merged = true; + } else { + keep.add(j); + } + } + if (!merged) { + return sources; + } + List phase = new ArrayList<>(); + for (int i = 0; i < sources.size(); i++) { + if (keep.contains(i) || i <= idx) { + phase.add(sources.get(i)); + } + } + idx++; + sources = phase; + } + return sources; + } + + private static List transformAndMergeInOrder(PlanningContext ctx, Union op) throws SQLException { + List sources = new ArrayList<>(); + // We go over all the input operators and turn them into logical plans + for (int i = 0; i < op.getSources().size(); i++) { + PhysicalOperator source = op.getSources().get(i); + LogicalPlan plan = createLogicalPlan(ctx, source, op.getSelectStmts().get(i)); + if (i == 0) { + sources.add(plan); + continue; + } + // next we check if the last plan we produced can be merged with this new plan + LogicalPlan last = sources.get(sources.size() - 1); + LogicalPlan newPlan = mergeUnionLogicalPlans(ctx, last, plan); + if (newPlan != null) { + // if we could merge them, let's replace the last plan with this new merged one + sources.set(sources.size() - 1, newPlan); + continue; + } + // else we just add the new plan to the end of list + sources.add(plan); + } + return sources; + } + + private static void pushDistinct(LogicalPlan plan) { + if (plan instanceof RouteGen4Plan) { + SQLSelectQuery selectQuery = ((RouteGen4Plan) plan).getSelect(); + if (selectQuery instanceof MySqlSelectQueryBlock) { + ((MySqlSelectQueryBlock) selectQuery).setDistionOption(SQLSetQuantifier.DISTINCT); + } else if (selectQuery instanceof SQLUnionQuery) { + ((SQLUnionQuery) selectQuery).setOperator(SQLUnionOperator.UNION); + } + } else if (plan instanceof ConcatenateGen4Plan) { + List sources = ((ConcatenateGen4Plan) plan).getSources(); + for (LogicalPlan source : sources) { + pushDistinct(source); + } + } + } + + private static Integer pushWeightStringForDistinct(PlanningContext ctx, LogicalPlan plan, int offset) throws SQLException { + int newOffset = 0; + if (plan instanceof RouteGen4Plan) { + RouteGen4Plan routePlan = (RouteGen4Plan) plan; + List allSelects = Scoper.getAllSelects(routePlan.getSelect()); + for (MySqlSelectQueryBlock sel : allSelects) { + SQLSelectItem item = sel.getSelectList().get(offset); + SQLSelectItem expr = getWeightStringForSelectExpr(item); + int getWsIdx = HorizonPlanning.checkIfAlreadyExists(expr, sel, ctx.getSemTable()); + if (getWsIdx != -1) { + return getWsIdx; + } + sel.getSelectList().add(expr); + newOffset = sel.getSelectList().size() - 1; + } + // we leave the responsibility of truncating to distinct + routePlan.getEroute().setTruncateColumnCount(0); + } else if (plan instanceof ConcatenateGen4Plan) { + ConcatenateGen4Plan node = (ConcatenateGen4Plan) plan; + for (LogicalPlan source : node.getSources()) { + newOffset = pushWeightStringForDistinct(ctx, source, offset); + } + node.getNoNeedToTypeCheck().add(newOffset); + } else if (plan instanceof JoinGen4Plan) { + throw new SQLException("todo: pushWeightStringForDistinct on JoinGen4Plan"); + } else { + throw new SQLException("bug: not supported pushWeightStringForDistinct on" + plan.getClass()); + } + return newOffset; + } + + private static SQLSelectItem getWeightStringForSelectExpr(SQLSelectItem item) throws SQLException { + return new SQLSelectItem(HorizonPlanning.weightStringFor(item.getExpr())); + + } + + private static LogicalPlan mergeUnionLogicalPlans(PlanningContext ctx, LogicalPlan left, LogicalPlan right) { + if (!(left instanceof RouteGen4Plan)) { + return null; + } + if (!(right instanceof RouteGen4Plan)) { + return null; + } + RouteGen4Plan lroute = (RouteGen4Plan) left; + RouteGen4Plan rroute = (RouteGen4Plan) right; + if (canMergeUnionPlans(ctx, lroute, rroute)) { + SQLUnionQuery sqlUnionQuery = new SQLUnionQuery(lroute.getSelect(), SQLUnionOperator.UNION, rroute.getSelect()); + lroute.setSelect(sqlUnionQuery); + return mergeSystemTableInformation(lroute, rroute); + } + return null; + } + + /** + * mergeSystemTableInformation copies over information from the second route to the first and appends to it + * @param a + * @param b + * @return + */ + private static LogicalPlan mergeSystemTableInformation(RouteGen4Plan a, RouteGen4Plan b) { + // safe to append system table schema and system table names, since either the routing will match or either side would be throwing an error + // during run-time which we want to preserve. For example outer side has User in sys table schema and inner side has User and Main in sys table schema + // Inner might end up throwing an error at runtime, but if it doesn't then it is safe to merge. + + return a; + } + + private static boolean canMergeUnionPlans(PlanningContext ctx, RouteGen4Plan a, RouteGen4Plan b) { + // this method should be close to tryMerge below. it does the same thing, but on logicalPlans instead of queryTrees + switch (a.getEroute().getRoutingParameters().getRouteOpcode()) { + case SelectUnsharded: + case SelectReference: + return a.getEroute().getRoutingParameters().getRouteOpcode() == b.getEroute().getRoutingParameters().getRouteOpcode(); + case SelectDBA: + return canSelectDBAMerge(a, b); + case SelectEqualUnique: + // Check if they target the same shard. + if (b.getEroute().getRoutingParameters().getRouteOpcode() == Engine.RouteOpcode.SelectEqualUnique + && a.getEroute().getRoutingParameters().getVindex() == b.getEroute().getRoutingParameters().getVindex() + && a.getCondition() != null && b.getCondition() != null + && gen4ValuesEqual(ctx, a.getCondition(), b.getCondition()) + ) { + return true; + } + return false; + case SelectScatter: + return b.getEroute().getRoutingParameters().getRouteOpcode() == Engine.RouteOpcode.SelectScatter; + case SelectNext: + return false; + default: + return false; + } + } + + private static boolean canSelectDBAMerge(RouteGen4Plan a, RouteGen4Plan b) { + return false; + } + + private static boolean gen4ValuesEqual(PlanningContext ctx, SQLExpr condition, SQLExpr condition1) { + return false; + } + + private static LogicalPlan createLogicalPlan(PlanningContext ctx, PhysicalOperator source, SQLSelectQuery selStmt) throws SQLException { + LogicalPlan plan = transformToLogicalPlan(ctx, source, false); + if (selStmt != null) { + plan = Gen4Planner.planHorizon(ctx, plan, selStmt, true); + } + return plan; + } + + private static LogicalPlan transformRoutePlan(PlanningContext ctx, Route op) throws SQLException { + + OperatorTransformers ot = new OperatorTransformers(); + String[] tableNames = ot.getAllTableNames(op); + + Vindex vindex = null; + List values = new ArrayList<>(); + if (op.selectedVindex() != null) { + vindex = op.getSelected().getFoundVindex(); + values = op.getSelected().getValues(); + } + + SQLExpr condition = getVindexPredicate(ctx, op); + + SQLSelectQuery sel = OperatorToQuery.toSQL(ctx, op.getSource()); + + replaceSubQuery(ctx, sel); + + VKeyspace keyspace = new VKeyspace(ctx.getVschema().getDefaultKeyspace(), op.getKeyspace().getSharded()); + + RouteGen4Engine eroute = new RouteGen4Engine(op.getRouterOpCode(), keyspace); + eroute.setTableName(String.join(",", tableNames)); + eroute.getRoutingParameters().setVindex((SingleColumn) vindex); + eroute.getRoutingParameters().setValues(values); + eroute.setSelectQuery(sel); + eroute.getRoutingParameters().getSystableTableSchema().addAll(op.getSysTableTableSchema()); + + return new RouteGen4Plan(eroute, sel, op.tableID(), condition); + } + + private static LogicalPlan transformApplyJoinPlan(PlanningContext ctx, ApplyJoin n) throws SQLException { + LogicalPlan lhs = transformToLogicalPlan(ctx, n.getLHS(), false); + LogicalPlan rhs = transformToLogicalPlan(ctx, n.getRHS(), false); + + Engine.JoinOpcode opCode = Engine.JoinOpcode.NormalJoin; + if (n.getLeftJoin()) { + opCode = Engine.JoinOpcode.LeftJoin; + } + + return new JoinGen4Plan( + lhs, + rhs, + opCode, + n.getColumns(), + n.getVars(), + n.getLhsColumns() + ); + } + + class GetOpAllTableNameFunc implements RoutePlanning.VisitOperatorFunc { + public Set tableNameSet; + + public GetOpAllTableNameFunc() { + tableNameSet = new TreeSet(); + } + + public String[] getAllTableNames() { + String ret[] = new String[tableNameSet.size()]; + ret = tableNameSet.toArray(ret); + Arrays.sort(ret); + return ret; + } + + @Override + public boolean doFunc(PhysicalOperator op) { + if (op instanceof Table) { + if (((Table) op).getQTable().isInfSchema()) { + this.tableNameSet.add(((Table) op).getQTable().getTable().getDbAndTableName()); + } else { + this.tableNameSet.add(((Table) op).getQTable().getTable().getName()); + } + } + return true; + } + } + + private String[] getAllTableNames(Route op) throws SQLException { + GetOpAllTableNameFunc func = new GetOpAllTableNameFunc(); + RoutePlanning.visitOperator(op, func); + return func.getAllTableNames(); + } + + private static SQLExpr getVindexPredicate(PlanningContext ctx, Route op) { + SQLExpr condition = null; + if (op.getSelected() != null) { + if (op.getSelected().getValueExprs().size() > 0) { + condition = op.getSelected().getValueExprs().get(0); + } + // TODO SQLInSubQueryExpr SQLInListExpr + // 不支持多列vindex + for (SQLExpr predicate : op.getSelected().getPredicates()) { + if (predicate instanceof SQLInListExpr) { // col in (xx,yy,zz) + SQLExpr leftExpr = ((SQLInListExpr) predicate).getExpr(); + if (leftExpr instanceof SQLName) { + ((SQLInListExpr) predicate).setTargetList(new ArrayList() {{ + add(new SQLVariantRefListExpr("::" + Engine.LIST_VAR_NAME)); + }}); + } + } else if (predicate instanceof SQLInSubQueryExpr) { // col in (select xxx from xxx) + SQLExpr leftExpr = ((SQLInSubQueryExpr) predicate).getExpr(); + if (leftExpr instanceof SQLName) { + // TODO + // SQLSelect subq = ((SQLInSubQueryExpr)predicate).getSubQuery(); + } + } + } + } + return condition; + } + + private static void replaceSubQuery(PlanningContext ctx, SQLSelectQuery sql) { + + } + +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/OperatorUtil.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/OperatorUtil.java new file mode 100644 index 0000000..704a025 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/OperatorUtil.java @@ -0,0 +1,265 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.operator; + +import com.google.common.collect.Lists; +import com.jd.jdbc.planbuilder.PlanBuilder; +import com.jd.jdbc.planbuilder.gen4.InnerJoin; +import com.jd.jdbc.planbuilder.gen4.QueryTable; +import com.jd.jdbc.planbuilder.gen4.TableName; +import com.jd.jdbc.planbuilder.gen4.operator.logical.Concatenate; +import com.jd.jdbc.planbuilder.gen4.operator.logical.Join; +import com.jd.jdbc.planbuilder.gen4.operator.logical.LogicalOperator; +import com.jd.jdbc.planbuilder.gen4.operator.logical.QueryGraph; +import com.jd.jdbc.planbuilder.gen4.operator.logical.SubQuery; +import com.jd.jdbc.planbuilder.semantics.SemTable; +import com.jd.jdbc.planbuilder.semantics.TableInfo; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.planbuilder.semantics.VindexTable; +import com.jd.jdbc.sqlparser.SQLUtils; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLLimit; +import com.jd.jdbc.sqlparser.ast.SQLName; +import com.jd.jdbc.sqlparser.ast.SQLOrderBy; +import com.jd.jdbc.sqlparser.ast.SQLStatement; +import com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOpExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOperator; +import com.jd.jdbc.sqlparser.ast.expr.SQLIdentifierExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLPropertyExpr; +import com.jd.jdbc.sqlparser.ast.statement.SQLExprTableSource; +import com.jd.jdbc.sqlparser.ast.statement.SQLJoinTableSource; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectQuery; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectStatement; +import com.jd.jdbc.sqlparser.ast.statement.SQLSubqueryTableSource; +import com.jd.jdbc.sqlparser.ast.statement.SQLTableSource; +import com.jd.jdbc.sqlparser.ast.statement.SQLUnionOperator; +import com.jd.jdbc.sqlparser.ast.statement.SQLUnionQuery; +import com.jd.jdbc.sqlparser.ast.statement.SQLUpdateStatement; +import com.jd.jdbc.sqlparser.dialect.mysql.ast.statement.MySqlSelectQueryBlock; +import com.jd.jdbc.sqlparser.dialect.mysql.visitor.VtRemoveDbNameVisitor; +import java.sql.SQLException; +import java.sql.SQLFeatureNotSupportedException; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; + +public class OperatorUtil { + /** + * CreateLogicalOperatorFromAST creates an operator tree that represents the input SELECT or UNION query + * + * @return + * @throws SQLException + */ + public static LogicalOperator createLogicalOperatorFromAST(SQLStatement stmt, SemTable semTable) throws SQLException { + LogicalOperator op; + if (stmt instanceof SQLSelectStatement) { + SQLSelectQuery query = ((SQLSelectStatement) stmt).getSelect().getQuery(); + op = createLogicalOperatorFromAST(query, semTable); + } else if (stmt instanceof SQLUpdateStatement) { + throw new SQLFeatureNotSupportedException(); + } else { + throw new SQLException("BUG: unexpected statement type: " + SQLUtils.toMySqlString(stmt, SQLUtils.NOT_FORMAT_OPTION)); + } + return op.compact(semTable); + } + + private static LogicalOperator createLogicalOperatorFromAST(SQLSelectQuery query, SemTable semTable) throws SQLException { + LogicalOperator op; + if (query instanceof MySqlSelectQueryBlock) { + op = createOperatorFromSelect((MySqlSelectQueryBlock) query, semTable); + } else if (query instanceof SQLUnionQuery) { + op = createOperatorFromUnion((SQLUnionQuery) query, semTable); + } else { + throw new SQLFeatureNotSupportedException(); + } + return op.compact(semTable); + } + + /** + * createOperatorFromSelect creates an operator tree that represents the input SELECT query + * + * @param query + * @param semTable + * @return + * @throws SQLException + */ + private static LogicalOperator createOperatorFromSelect(MySqlSelectQueryBlock query, SemTable semTable) throws SQLException { + SubQuery subQuery = createSubqueryFromStatement(query, semTable); + LogicalOperator op = crossJoin(query.getFrom(), semTable); + + if (query.getWhere() != null) { + List exprs = PlanBuilder.splitAndExpression(null, query.getWhere()); + for (SQLExpr expr : exprs) { + op = op.pushPredicate(PlanBuilder.removeKeyspaceFromColName(expr), semTable); + addColumnEquality(semTable, expr); + } + } + + if (subQuery == null) { + return op; + } + subQuery.setOuter(op); + return subQuery; + } + + private static LogicalOperator createOperatorFromUnion(SQLUnionQuery node, SemTable semTable) throws SQLException { + LogicalOperator opLHS = createLogicalOperatorFromAST(node.getLeft(), semTable); + if (node.getRight() instanceof SQLUnionQuery) { + throw new SQLException("nesting of unions at the right-hand side is not yet supported"); + } + LogicalOperator opRHS = createLogicalOperatorFromAST(node.getRight(), semTable); + + SQLOrderBy orderBy = node.getOrderBy(); + SQLLimit limit = node.getLimit(); + boolean distinct; + if (Objects.equals(node.getOperator(), SQLUnionOperator.UNION_ALL)) { + distinct = false; + } else if (Objects.equals(node.getOperator(), SQLUnionOperator.UNION)) { + distinct = true; + } else { + throw new SQLFeatureNotSupportedException(node.getOperator().toString()); + } + return new Concatenate(distinct, Lists.newArrayList(node.getLeft(), node.getRight()), Lists.newArrayList(opLHS, opRHS), orderBy, limit); + } + + private static SubQuery createSubqueryFromStatement(MySqlSelectQueryBlock selectStatement, SemTable semTable) throws SQLException { + if (semTable.getSubqueryMap().size() == 0) { + return null; + } + SubQuery subQuery = new SubQuery(); + return subQuery; + } + + private static LogicalOperator crossJoin(SQLTableSource exprs, SemTable semTable) throws SQLException { + LogicalOperator output; + if (exprs instanceof SQLExprTableSource) { + output = getOperatorFromTableExpr((SQLExprTableSource) exprs, semTable); + } else if (exprs instanceof SQLJoinTableSource) { + LogicalOperator leftOutput = crossJoin(((SQLJoinTableSource) exprs).getLeft(), semTable); + LogicalOperator rightOutput = crossJoin(((SQLJoinTableSource) exprs).getRight(), semTable); + + + SQLJoinTableSource.JoinType joinType = ((SQLJoinTableSource) exprs).getJoinType(); + if (joinType == SQLJoinTableSource.JoinType.COMMA + || joinType == SQLJoinTableSource.JoinType.INNER_JOIN + || joinType == SQLJoinTableSource.JoinType.NATURAL_INNER_JOIN + || joinType == SQLJoinTableSource.JoinType.JOIN) { //normal join + + output = createJoin(leftOutput, rightOutput); + if (((SQLJoinTableSource) exprs).getCondition() != null) { + output.pushPredicate(((SQLJoinTableSource) exprs).getCondition(), semTable); + } + } else if (joinType == SQLJoinTableSource.JoinType.LEFT_OUTER_JOIN + || joinType == SQLJoinTableSource.JoinType.RIGHT_OUTER_JOIN) { + + if (joinType == SQLJoinTableSource.JoinType.RIGHT_OUTER_JOIN) { + LogicalOperator tmp = leftOutput; + leftOutput = rightOutput; + rightOutput = tmp; + } + output = new Join(leftOutput, rightOutput); + ((Join) output).setLeftJoin(true); + VtRemoveDbNameVisitor visitor = new VtRemoveDbNameVisitor(); + SQLExpr predicate = ((SQLJoinTableSource) exprs).getCondition(); + predicate.accept(visitor); + ((Join) output).setPredicate(predicate); + } else { + throw new SQLException("unsupported: " + joinType.toString()); + } + } else if (exprs instanceof SQLSubqueryTableSource) { + throw new SQLException("BUG: unexpected statement type: " + SQLUtils.toMySqlString(exprs, SQLUtils.NOT_FORMAT_OPTION)); + } else { + throw new SQLException("BUG: unexpected statement type: " + SQLUtils.toMySqlString(exprs, SQLUtils.NOT_FORMAT_OPTION)); + } + return output; + } + + private static LogicalOperator getOperatorFromTableExpr(SQLExprTableSource tableSource, SemTable semTable) throws SQLException { + if (tableSource.getExpr() instanceof SQLName) { + String qualifier = null; + String tableName = null; + if (tableSource.getExpr() instanceof SQLIdentifierExpr) { + SQLIdentifierExpr identifierExpr = (SQLIdentifierExpr) tableSource.getExpr(); + qualifier = ""; + tableName = identifierExpr.getName(); + } else if (tableSource.getExpr() instanceof SQLPropertyExpr) { + SQLPropertyExpr propertyExpr = (SQLPropertyExpr) tableSource.getExpr(); + qualifier = propertyExpr.getOwnernName(); + tableName = propertyExpr.getName(); + } + TableSet tableID = semTable.tableSetFor(tableSource); + TableInfo tableInfo = semTable.tableInfoFor(tableID); + if (tableInfo instanceof VindexTable) { + throw new SQLFeatureNotSupportedException("unsupported TableInfo.VindexTable"); + } + + boolean isInfSchema = tableInfo.isInfSchema(); + QueryGraph qg = new QueryGraph(); + QueryTable qt = new QueryTable(tableID, tableSource, new TableName(qualifier, tableName), isInfSchema); + qg.getTables().add(qt); + return qg; + } else { + // todo: case tableSource type + throw new SQLException("BUG: unexpected statement type: " + SQLUtils.toMySqlString(tableSource)); + } + } + + private static LogicalOperator createJoin(LogicalOperator lhs, LogicalOperator rhs) { + Boolean lok = lhs instanceof QueryGraph; + Boolean rok = rhs instanceof QueryGraph; + if (lok && rok) { + List tables = new ArrayList<>(); + tables.addAll(((QueryGraph) lhs).getTables()); + tables.addAll(((QueryGraph) rhs).getTables()); + + List innerJoins = new ArrayList<>(); + innerJoins.addAll(((QueryGraph) lhs).getInnerJoins()); + innerJoins.addAll(((QueryGraph) rhs).getInnerJoins()); + + SQLExpr noDeps = SQLBinaryOpExpr.and(((QueryGraph) lhs).getNoDeps(), ((QueryGraph) rhs).getNoDeps()); + + return new QueryGraph(tables, innerJoins, noDeps); + } + + return new Join(lhs, rhs); + } + + private static void addColumnEquality(SemTable semTable, SQLExpr expr) { + if (expr instanceof SQLBinaryOpExpr) { + SQLBinaryOpExpr sqlBinaryOpExpr = (SQLBinaryOpExpr) expr; + if (sqlBinaryOpExpr.getOperator() != SQLBinaryOperator.Equality) { + return; + } + // todo 需要考虑a.b情况,判断是否为列应该有方法 + if (sqlBinaryOpExpr.getLeft() instanceof SQLName) { + semTable.addColumnEquality((SQLName) sqlBinaryOpExpr.getLeft(), sqlBinaryOpExpr.getRight()); + } + if (sqlBinaryOpExpr.getRight() instanceof SQLName) { + semTable.addColumnEquality((SQLName) sqlBinaryOpExpr.getRight(), sqlBinaryOpExpr.getLeft()); + } + if (sqlBinaryOpExpr.getLeft() instanceof SQLPropertyExpr) { + + } + if (sqlBinaryOpExpr.getRight() instanceof SQLPropertyExpr) { + + } + } + } +} + diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/logical/Concatenate.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/logical/Concatenate.java new file mode 100644 index 0000000..812b7e9 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/logical/Concatenate.java @@ -0,0 +1,108 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.operator.logical; + +import com.jd.jdbc.planbuilder.semantics.SemTable; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLLimit; +import com.jd.jdbc.sqlparser.ast.SQLOrderBy; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectQuery; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import lombok.Getter; + +/** + * Concatenate represents a UNION ALL/DISTINCT. + */ +public class Concatenate implements LogicalOperator { + + @Getter + private boolean distinct; + + @Getter + private List selectStmts; + + @Getter + private List sources; + + @Getter + private SQLOrderBy orderBy; + + @Getter + private SQLLimit limit; + + public Concatenate(boolean distinct, List selectStmts, List sources, SQLOrderBy orderBy, SQLLimit limit) { + this.distinct = distinct; + this.selectStmts = selectStmts; + this.sources = sources; + this.orderBy = orderBy; + this.limit = limit; + } + + @Override + public TableSet tableID() { +// TableSet tableSet + return null; + } + + @Override + public SQLSelectItem unsolvedPredicates(SemTable semTable) { + return null; + } + + @Override + public void checkValid() throws SQLException { + + } + + @Override + public LogicalOperator pushPredicate(SQLExpr expr, SemTable semTable) throws SQLException { + return null; + } + + @Override + public LogicalOperator compact(SemTable semTable) throws SQLException { + List newSources = new ArrayList<>(); + List newSels = new ArrayList<>(); + + for (int i = 0; i < this.sources.size(); i++) { + LogicalOperator source = this.sources.get(i); + if (!(source instanceof Concatenate)) { + newSources.add(source); + newSels.add(this.selectStmts.get(i)); + continue; + } + Concatenate other = (Concatenate) source; + if (other.limit == null && other.orderBy == null && !other.distinct || this.distinct && other.limit == null) { + // if the current UNION is a DISTINCT, we can safely ignore everything from children UNIONs, except LIMIT + newSources.addAll(other.sources); + newSels.addAll(other.selectStmts); + } else { + newSources.add(other); + newSels.add(null); + } + } + this.sources = newSources; + this.selectStmts = newSels; + return this; + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/logical/Derived.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/logical/Derived.java new file mode 100644 index 0000000..bd45b30 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/logical/Derived.java @@ -0,0 +1,83 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.operator.logical; + +import com.jd.jdbc.context.PlanningContext; +import com.jd.jdbc.planbuilder.semantics.SemTable; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectStatement; +import java.sql.SQLException; +import lombok.Getter; + +/** + * Derived represents a derived table in the query + */ +@Getter +public class Derived implements LogicalOperator { + private SQLSelectStatement sel; + + private LogicalOperator inner; + + private String alias; + + private String ColumnAliases; + + @Override + public TableSet tableID() { + return this.inner.tableID(); + } + + @Override + public SQLSelectItem unsolvedPredicates(SemTable semTable) { + return this.inner.unsolvedPredicates(semTable); + } + + @Override + public void checkValid() throws SQLException { + this.inner.checkValid(); + } + + @Override + public LogicalOperator pushPredicate(SQLExpr expr, SemTable semTable) throws SQLException { + //todo + return null; + } + + @Override + public LogicalOperator compact(SemTable semTable) throws SQLException { + return this; + } + + /** + * IsMergeable is not a great name for this function. Suggestions for a better one are welcome! + * This function will return false if the derived table inside it has to run on the vtgate side, and so can't be merged with subqueries + * This logic can also be used to check if this is a derived table that can be had on the left hand side of a vtgate join. + * Since vtgate joins are always nested loop joins, we can't execute them on the RHS + * if they do some things, like LIMIT or GROUP BY on wrong columns + * + * @param ctx + * @return + */ + public Boolean isMergeable(PlanningContext ctx) { + // TODO + return false; + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/logical/Filter.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/logical/Filter.java new file mode 100644 index 0000000..38decb3 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/logical/Filter.java @@ -0,0 +1,80 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.operator.logical; + +import com.jd.jdbc.planbuilder.semantics.SemTable; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import java.sql.SQLException; +import java.util.List; +import lombok.Getter; +import lombok.Setter; + +@Getter +@Setter +public class Filter implements LogicalOperator { + + private LogicalOperator source; + + private List predicates; + + public Filter() { + + } + + public Filter(LogicalOperator source, List predicates) { + this.source = source; + this.predicates = predicates; + } + + @Override + public TableSet tableID() { + return this.source.tableID(); + } + + @Override + public SQLSelectItem unsolvedPredicates(SemTable semTable) { + return this.source.unsolvedPredicates(semTable); + } + + @Override + public void checkValid() throws SQLException { + this.source.checkValid(); + } + + @Override + public LogicalOperator pushPredicate(SQLExpr expr, SemTable semTable) throws SQLException { + LogicalOperator op = this.source.pushPredicate(expr, semTable); + + if (op instanceof Filter) { + ((Filter) op).predicates.addAll(this.predicates); + return op; + } + return new Filter(op, this.predicates); + } + + @Override + public LogicalOperator compact(SemTable semTable) throws SQLException { + if (this.predicates.size() == 0) { + return this.source; + } + return this; + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/logical/Join.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/logical/Join.java new file mode 100644 index 0000000..e60cbaa --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/logical/Join.java @@ -0,0 +1,200 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.operator.logical; + +import com.jd.jdbc.planbuilder.gen4.InnerJoin; +import com.jd.jdbc.planbuilder.gen4.QueryTable; +import com.jd.jdbc.planbuilder.semantics.SemTable; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLName; +import com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOpExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOperator; +import com.jd.jdbc.sqlparser.ast.statement.SQLNotNullConstraint; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import lombok.Getter; +import lombok.Setter; + +/** + * Join represents a join. If we have a predicate, this is an inner join. If no predicate exists, it is a cross join + */ +@Getter +@Setter +public class Join implements LogicalOperator { + + private LogicalOperator lHS; + + private LogicalOperator rHS; + + private SQLExpr predicate; + + private Boolean leftJoin = false; + + + public Join() { + + } + + public Join(LogicalOperator lHS, LogicalOperator rHS) { + this.lHS = lHS; + this.rHS = rHS; + } + + @Override + public TableSet tableID() { + return this.rHS.tableID().merge(this.lHS.tableID()); + } + + @Override + public SQLSelectItem unsolvedPredicates(SemTable semTable) { + + //TODO + + return null; + } + + @Override + public void checkValid() throws SQLException { + this.lHS.checkValid(); + this.rHS.checkValid(); + } + + @Override + public LogicalOperator pushPredicate(SQLExpr expr, SemTable semTable) throws SQLException { + TableSet deps = semTable.recursiveDeps(expr); + if (deps.isSolvedBy(this.lHS.tableID())) { + this.lHS = this.lHS.pushPredicate(expr, semTable); + return this; + } else if (deps.isSolvedBy(this.rHS.tableID())) { + this.tryConvertToInnerJoin(expr, semTable); + if (!this.leftJoin) { + this.rHS = this.rHS.pushPredicate(expr, semTable); + return this; + } + + LogicalOperator op = new Filter(this, Collections.singletonList(expr)); + return op; + } else if (deps.isSolvedBy(this.lHS.tableID().merge(this.rHS.tableID()))) { + this.tryConvertToInnerJoin(expr, semTable); + if (!this.leftJoin) { + this.predicate = SQLBinaryOpExpr.and(this.predicate, expr); + return this; + } + LogicalOperator op = new Filter(this, Collections.singletonList(expr)); + return op; + } + + throw new SQLException("Cannot push predicate: " + expr.toString()); + } + + @Override + public LogicalOperator compact(SemTable semTable) throws SQLException { + if (this.leftJoin) { + return this; + } + + Boolean lok = this.lHS instanceof QueryGraph; + Boolean rok = this.rHS instanceof QueryGraph; + + if (!lok || !rok) { + return this; + } + + List tables = new ArrayList<>(); + tables.addAll(((QueryGraph) this.lHS).getTables()); + tables.addAll(((QueryGraph) this.rHS).getTables()); + + List innerJoins = new ArrayList<>(); + innerJoins.addAll(((QueryGraph) this.lHS).getInnerJoins()); + innerJoins.addAll(((QueryGraph) this.rHS).getInnerJoins()); + + SQLExpr noDeps = SQLBinaryOpExpr.and(((QueryGraph) this.lHS).getNoDeps(), ((QueryGraph) this.rHS).getNoDeps()); + + QueryGraph op = new QueryGraph(tables, innerJoins, noDeps); + if (this.predicate != null) { + op.collectPredicate(this.predicate, semTable); + } + + return op; + } + + /** + * When a predicate uses information from an outer table, we can convert from an outer join to an inner join + * if the predicate is "null-intolerant". + *

+ * Null-intolerant in this context means that the predicate will not be true if the table columns are null. + *

+ * Since an outer join is an inner join with the addition of all the rows from the left-hand side that + * matched no rows on the right-hand, if we are later going to remove all the rows where the right-hand + * side did not match, we might as well turn the join into an inner join. + *

+ * This is based on the paper "Canonical Abstraction for Outerjoin Optimization" by J Rao et al + * + * @param expr + * @param semTable + */ + private void tryConvertToInnerJoin(SQLExpr expr, SemTable semTable) { + if (!this.leftJoin) { + return; + } + + if (expr instanceof SQLBinaryOpExpr) { + SQLBinaryOperator op = ((SQLBinaryOpExpr) expr).getOperator(); + if (op == SQLBinaryOperator.LessThanOrEqualOrGreaterThan) { + return; + } + + if (op == SQLBinaryOperator.Is || op == SQLBinaryOperator.IsNot) { + if (!(((SQLBinaryOpExpr) expr).getRight() instanceof SQLNotNullConstraint)) { + return; + } + Boolean checkLeft = ((SQLBinaryOpExpr) expr).getLeft() instanceof SQLName; + checkLeft = checkLeft && semTable.recursiveDeps(((SQLBinaryOpExpr) expr).getLeft()).isSolvedBy(this.rHS.tableID()); + + if (checkLeft) { + this.leftJoin = false; + return; + } + } + + Boolean checkLeft = ((SQLBinaryOpExpr) expr).getLeft() instanceof SQLName; + checkLeft = checkLeft && semTable.recursiveDeps(((SQLBinaryOpExpr) expr).getLeft()).isSolvedBy(this.rHS.tableID()); + + if (checkLeft) { + this.leftJoin = false; + return; + } + + Boolean checkRight = ((SQLBinaryOpExpr) expr).getRight() instanceof SQLName; + checkRight = checkRight && semTable.recursiveDeps(((SQLBinaryOpExpr) expr).getRight()).isSolvedBy(this.rHS.tableID()); + + if (checkRight) { + this.leftJoin = false; + return; + } + + } + + + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/logical/LogicalOperator.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/logical/LogicalOperator.java new file mode 100644 index 0000000..07789e7 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/logical/LogicalOperator.java @@ -0,0 +1,48 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.operator.logical; + +import com.jd.jdbc.planbuilder.gen4.operator.Operator; +import com.jd.jdbc.planbuilder.semantics.SemTable; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import java.sql.SQLException; + +public interface LogicalOperator extends Operator { + default void isLogical() { + + } + + /** + * PushPredicate pushes a predicate to the closest possible operator + * + * @param expr + * @param semTable + * @return + */ + LogicalOperator pushPredicate(SQLExpr expr, SemTable semTable) throws SQLException; + + /** + * Compact will optimise the operator tree into a smaller but equivalent version + * + * @param semTable + * @return + * @throws SQLException + */ + LogicalOperator compact(SemTable semTable) throws SQLException; +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/logical/QueryGraph.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/logical/QueryGraph.java new file mode 100644 index 0000000..9fef16c --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/logical/QueryGraph.java @@ -0,0 +1,163 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.operator.logical; + +import com.jd.jdbc.planbuilder.PlanBuilder; +import com.jd.jdbc.planbuilder.gen4.InnerJoin; +import com.jd.jdbc.planbuilder.gen4.QueryTable; +import com.jd.jdbc.planbuilder.semantics.SemTable; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOpExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOperator; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import lombok.Getter; +import lombok.Setter; + +/** + * 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. + */ +@Getter +@Setter +public class QueryGraph implements LogicalOperator { + private List tables; + + private List innerJoins; + + private SQLExpr noDeps; + + public QueryGraph() { + tables = new ArrayList<>(); + innerJoins = new ArrayList<>(); + } + + public QueryGraph(List tables, List innerJoins, SQLExpr noDeps) { + this.tables = tables; + this.innerJoins = innerJoins; + this.noDeps = noDeps; + } + + @Override + public TableSet tableID() { + TableSet ts = new TableSet(); + for (QueryTable table : this.tables) { + ts = ts.merge(table.getId()); + } + return ts; + } + + /** + * GetPredicates returns the predicates that are applicable for the two given TableSets + * + * @param lhs + * @param rhs + * @return + */ + public List getPredicates(TableSet lhs, TableSet rhs) { + List allExprs = new ArrayList<>(); + for (InnerJoin join : this.innerJoins) { + if (join.getDeps().isSolvedBy(lhs.merge(rhs)) && + join.getDeps().isOverlapping(rhs) && + join.getDeps().isOverlapping(lhs)) { + allExprs.addAll(join.getExprs()); + } + } + return allExprs; + } + + @Override + public SQLSelectItem unsolvedPredicates(SemTable semTable) { + return null; + } + + @Override + public void checkValid() throws SQLException { + } + + @Override + public LogicalOperator pushPredicate(SQLExpr expr, SemTable semTable) throws SQLException { + List subExpr = PlanBuilder.splitAndExpression(null, expr); + for (SQLExpr sqlExpr : subExpr) { + collectPredicate(sqlExpr, semTable); + } + return this; + } + + @Override + public LogicalOperator compact(SemTable semTable) throws SQLException { + return this; + } + + public void collectPredicate(SQLExpr predicate, SemTable semTable) throws SQLException { + TableSet deps = semTable.recursiveDeps(predicate); + switch (deps.numberOfTables()) { + case 0: + addNoDepsPredicate(predicate); + break; + case 1: + boolean found = addToSingleTable(deps, predicate); + if (!found) { + // this could be a predicate that only has dependencies from outside this QG + addJoinPredicates(deps, predicate); + } + break; + default: + addJoinPredicates(deps, predicate); + } + } + + public void addNoDepsPredicate(SQLExpr predicate) { + if (this.noDeps == null) { + this.setNoDeps(predicate); + } else { + SQLBinaryOpExpr andexpr = new SQLBinaryOpExpr(this.noDeps, SQLBinaryOperator.BooleanAnd, predicate); + this.setNoDeps(andexpr); + } + } + + public boolean addToSingleTable(TableSet tableSet, SQLExpr predicate) { + for (int i = 0; i < this.getTables().size(); i++) { + if (this.getTables().get(i).getId().equals(tableSet)) { + this.getTables().get(i).getPredicates().add(predicate); + return true; + } + } + return false; + } + + public void addJoinPredicates(TableSet ts, SQLExpr predicate) { + for (InnerJoin join : this.innerJoins) { + if (join.getDeps() == ts) { + join.getExprs().add(predicate); + return; + } + } + this.innerJoins.add(new InnerJoin(ts, Arrays.asList(new SQLExpr[] {predicate}))); + } + +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/logical/SubQuery.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/logical/SubQuery.java new file mode 100644 index 0000000..d20c82d --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/logical/SubQuery.java @@ -0,0 +1,76 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.operator.logical; + +import com.jd.jdbc.planbuilder.semantics.SemTable; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import java.sql.SQLException; +import java.util.List; +import lombok.Getter; +import lombok.Setter; + +@Getter +@Setter +// SubQuery stores the information about subquery +public class SubQuery implements LogicalOperator { + private List inner; + + private LogicalOperator outer; + + @Override + public LogicalOperator pushPredicate(SQLExpr expr, SemTable semTable) throws SQLException { + return null; + } + + @Override + public LogicalOperator compact(SemTable semTable) throws SQLException { + return null; + } + + @Override + public TableSet tableID() { + return null; + } + + @Override + public SQLSelectItem unsolvedPredicates(SemTable semTable) { + return null; + } + + @Override + public void checkValid() throws SQLException { + + } + +// SubQueryInner stores the subquery information for a select statement + + private static class SubQueryInner { + // Inner is the Operator inside the parenthesis of the subquery. + // i.e: select (select 1 union select 1), the Inner here would be + // of type Concatenate since we have a Union. + private LogicalOperator inner; + + // ExtractedSubquery contains all information we need about this subquery + private Object ExtractedSubquery; + } +} + + diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/logical/Update.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/logical/Update.java new file mode 100644 index 0000000..88fbe21 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/logical/Update.java @@ -0,0 +1,64 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.operator.logical; + +import com.jd.jdbc.planbuilder.gen4.QueryTable; +import com.jd.jdbc.planbuilder.semantics.SemTable; +import com.jd.jdbc.planbuilder.semantics.TableInfo; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import com.jd.jdbc.sqlparser.ast.statement.SQLUpdateStatement; +import java.sql.SQLException; +import java.util.Map; + +public class Update implements LogicalOperator { + private static QueryTable table; + + private static TableInfo tableInfo; + + private static Map assignments; + + private static SQLUpdateStatement ast; + + @Override + public LogicalOperator pushPredicate(SQLExpr expr, SemTable semTable) throws SQLException { + throw new SQLException("can't accept predicates"); + } + + @Override + public LogicalOperator compact(SemTable semTable) throws SQLException { + return this; + } + + @Override + public TableSet tableID() { + return table.getId(); + } + + @Override + public SQLSelectItem unsolvedPredicates(SemTable semTable) { + return null; + } + + @Override + public void checkValid() throws SQLException { + + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/ApplyJoin.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/ApplyJoin.java new file mode 100644 index 0000000..9c4f229 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/ApplyJoin.java @@ -0,0 +1,141 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.operator.physical; + +import com.jd.jdbc.planbuilder.semantics.SemTable; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLName; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.Setter; + +/** + * ApplyJoin is a nested loop join - for each row on the LHS, + * we'll execute the plan on the RHS, feeding data from left to righ + */ + +@Getter +@Setter +@AllArgsConstructor +public class ApplyJoin implements PhysicalOperator { + + private PhysicalOperator lHS, rHS; + + /** + * Columns stores the column indexes of the columns coming from the left and right side + * negative value comes from LHS and positive from RHS + */ + private List columns; + + /** + * Vars are the arguments that need to be copied from the LHS to the RHS + */ + private Map vars; + + /** + * LeftJoin will be true in the case of an outer join + */ + private Boolean leftJoin; + + /** + * JoinCols are the columns from the LHS used for the join. + * These are the same columns pushed on the LHS that are now used in the Vars field + */ + private List lhsColumns; + + private SQLExpr predicate; + + public ApplyJoin(PhysicalOperator lHS, PhysicalOperator rHS, Boolean leftJoin) { + this.lHS = lHS; + this.rHS = rHS; + this.columns = new ArrayList<>(); + this.vars = new HashMap<>(); + this.leftJoin = leftJoin; + this.lhsColumns = new ArrayList<>(); + this.predicate = null; + } + + public ApplyJoin(PhysicalOperator lHS, PhysicalOperator rHS, Boolean leftJoin, SQLExpr predicate) { + this.lHS = lHS; + this.rHS = rHS; + this.columns = new ArrayList<>(); + this.vars = new HashMap<>(); + this.leftJoin = leftJoin; + this.lhsColumns = new ArrayList<>(); + this.predicate = predicate; + } + + @Override + public TableSet tableID() { + return this.lHS.tableID().merge(this.rHS.tableID()); + } + + @Override + public SQLSelectItem unsolvedPredicates(SemTable semTable) { + return null; + } + + @Override + public void checkValid() throws SQLException { + this.lHS.checkValid(); + this.rHS.checkValid(); + } + + @Override + public Integer cost() { + return this.lHS.cost() + this.rHS.cost(); + } + + @Override + public PhysicalOperator clone() { + // vars clone + Map varsClone = new HashMap<>(); + for (Map.Entry entry : this.vars.entrySet()) { + varsClone.put(entry.getKey(), entry.getValue()); + } + + // columns clone + List columnsClone = new ArrayList<>(this.columns.size()); + for (Integer col : this.columns) { + columnsClone.add(col); + } + // lhs columns clone + List lhsColNameClone = new ArrayList<>(this.lhsColumns.size()); + for (SQLName colName : this.lhsColumns) { + lhsColNameClone.add(colName.clone()); + } + + return new ApplyJoin( + this.lHS.clone(), + this.rHS.clone(), + columnsClone, + varsClone, + this.leftJoin, + lhsColNameClone, + this.predicate != null ? this.predicate.clone() : null + ); + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/Cost.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/Cost.java new file mode 100644 index 0000000..d167649 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/Cost.java @@ -0,0 +1,35 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.operator.physical; + +import com.jd.jdbc.engine.Engine; +import lombok.Getter; +import lombok.Setter; + +@Getter +@Setter +// Cost is used to make it easy to compare the Cost of two plans with each other +public class Cost { + + private int vindexCost; + + private boolean isUnique; + + private Engine.RouteOpcode opCode; +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/Filter.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/Filter.java new file mode 100644 index 0000000..cbdd3d8 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/Filter.java @@ -0,0 +1,77 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.operator.physical; + +import com.jd.jdbc.planbuilder.semantics.SemTable; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import lombok.Getter; +import lombok.Setter; + +@Getter +@Setter +public class Filter implements PhysicalOperator { + + private PhysicalOperator source; + + private List predicates; + + public Filter() { + + } + + public Filter(PhysicalOperator source, List predicates) { + this.source = source; + this.predicates = predicates; + } + + @Override + public TableSet tableID() { + return this.source.tableID(); + } + + @Override + public SQLSelectItem unsolvedPredicates(SemTable semTable) { + return null; + } + + @Override + public void checkValid() throws SQLException { + + } + + @Override + public Integer cost() { + return this.getSource().cost(); + } + + @Override + public PhysicalOperator clone() { + + List predicatesClone = new ArrayList<>(this.predicates.size()); + for (SQLExpr pred : this.predicates) { + predicatesClone.add(pred.clone()); + } + return new Filter(source.clone(), predicatesClone); + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/OpCache.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/OpCache.java new file mode 100644 index 0000000..3c6e8ae --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/OpCache.java @@ -0,0 +1,34 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.operator.physical; + +import java.util.HashMap; +import java.util.Map; +import lombok.Getter; +import lombok.Setter; + +@Getter +@Setter +public class OpCache { + private Map opCacheMap; + + public OpCache() { + opCacheMap = new HashMap<>(); + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/PhysicalOperator.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/PhysicalOperator.java new file mode 100644 index 0000000..af83dfc --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/PhysicalOperator.java @@ -0,0 +1,43 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.operator.physical; + +import com.jd.jdbc.planbuilder.gen4.operator.Operator; + +public interface PhysicalOperator extends Operator { + + default void iPhysical() { + + } + + /** + * Cost is simply the number of routes in the operator tree + * + * @return + */ + Integer cost(); + + /** + * Clone creates a copy of the operator that can be updated without changing the original + * + * @return + */ + PhysicalOperator clone(); + +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/Route.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/Route.java new file mode 100644 index 0000000..c622f87 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/Route.java @@ -0,0 +1,497 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.operator.physical; + +import com.google.common.collect.Lists; +import com.jd.jdbc.context.PlanningContext; +import com.jd.jdbc.engine.Engine; +import com.jd.jdbc.evalengine.EvalEngine; +import com.jd.jdbc.key.Destination; +import com.jd.jdbc.planbuilder.semantics.SemTable; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.SQLUtils; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLName; +import com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOpExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLInListExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLListExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLNullExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLVariantRefListExpr; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import com.jd.jdbc.vindexes.VKeyspace; +import com.jd.jdbc.vindexes.Vindex; +import com.jd.jdbc.vindexes.hash.BinaryHash; +import java.sql.SQLException; +import java.sql.SQLFeatureNotSupportedException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.Setter; +import vschema.Vschema; + +@Getter +@Setter +public class Route implements PhysicalOperator { + private PhysicalOperator source; + + private Engine.RouteOpcode routerOpCode; + + private VKeyspace keyspace; + + // here we store the possible vindexes we can use so that when we add predicates to the plan, + // we can quickly check if the new predicates enables any new vindex Options + private List vindexPreds; + + // the best option available is stored here + private VindexOption selected; + + // The following two fields are used when routing information_schema queries + //SysTableTableSchema []evalengine.Expr + // todo: SysTableTableName map[string]evalengine.Expr + + private List sysTableTableSchema; + + // seenPredicates contains all the predicates that have had a chance to influence routing. + // If we need to replan routing, we'll use this list + private List seenPredicates; + + // targetDestination specifies an explicit target destination tablet type + private Destination targetDestination; + + public Route() { + this.vindexPreds = new ArrayList<>(10); + this.seenPredicates = new ArrayList<>(8); + this.sysTableTableSchema = new ArrayList<>(); + } + + public Route(Route r) { + this.source = r.getSource(); + this.routerOpCode = r.getRouterOpCode(); + this.keyspace = r.getKeyspace(); + this.vindexPreds = r.vindexPreds; + this.selected = r.getSelected(); + this.seenPredicates = r.seenPredicates; + this.targetDestination = r.targetDestination; + // this.sysTableTableSchema = new ArrayList<>(r.getSysTableTableSchema().size()); + this.sysTableTableSchema = r.getSysTableTableSchema(); + } + + @Override + public TableSet tableID() { + return this.getSource().tableID(); + } + + @Override + public SQLSelectItem unsolvedPredicates(SemTable semTable) { + return null; + } + + @Override + public void checkValid() throws SQLException { + + } + + @Override + public Integer cost() { + switch (routerOpCode) { + case SelectDBA: + case SelectNext: + case SelectNone: + case SelectReference: + case SelectUnsharded: + return 0; + case SelectEqualUnique: + return 1; + case SelectEqual: + return 5; + case SelectIN: + return 10; + case SelectScatter: + return 20; + default: + return 1; + } + } + + @Override + public PhysicalOperator clone() { + + Route clone = new Route(this); + + clone.setSource(this.getSource().clone()); + List vindexPreds = new ArrayList<>(this.getVindexPreds().size()); + for (VindexPlusPredicates pred : this.getVindexPreds()) { + vindexPreds.add(pred); + } + clone.setVindexPreds(vindexPreds); + return clone; + } + + public Vindex selectedVindex() { + if (this.selected == null) { + return null; + } + return this.selected.getFoundVindex(); + } + + public List vindexExpressions() { + if (this.selected == null) { + return null; + } + return this.selected.getValueExprs(); + } + + private boolean canImprove() { + if (this.getRouterOpCode().equals(Engine.RouteOpcode.SelectNone)) { + return false; + } + return true; + } + + public void updateRoutingLogic(PlanningContext ctx, SQLExpr expr) throws SQLException { + this.getSeenPredicates().add(expr); + tryImprovingVindex(ctx, expr); + } + + private void tryImprovingVindex(PlanningContext ctx, SQLExpr expr) throws SQLException { + if (canImprove()) { + boolean newVindexFound = searchForNewVindexes(ctx, expr); + if (newVindexFound) { + pickBestAvailableVindex(); + } + } + } + + private boolean searchForNewVindexes(PlanningContext ctx, SQLExpr predicate) throws SQLException { + boolean newVindexFound = false; + planComparisonRetrun planComparisonRetrun; + if (predicate instanceof SQLBinaryOpExpr) { + planComparisonRetrun = planComparison(ctx, (SQLBinaryOpExpr) predicate); + } else if (predicate instanceof SQLInListExpr) { + planComparisonRetrun = planComparison(ctx, (SQLInListExpr) predicate); + } else { + //throw new SQLFeatureNotSupportedException(); + return false; + } + + if (planComparisonRetrun.exitEarly) { + return false; + } + if (planComparisonRetrun.found) { + newVindexFound = true; + } + return newVindexFound; + } + + private planComparisonRetrun planComparison(PlanningContext ctx, SQLBinaryOpExpr binaryOpExpr) throws SQLException { + switch (binaryOpExpr.getOperator()) { + case Equality: + boolean found = planEqualOp(ctx, binaryOpExpr); + return new planComparisonRetrun(found, false); + case Is: + boolean foundis = planIsExpr(ctx, binaryOpExpr); + return new planComparisonRetrun(foundis, false); + default: + return new planComparisonRetrun(false, false); + } + } + + private planComparisonRetrun planComparison(PlanningContext ctx, SQLInListExpr inListExpr) throws SQLException { + if (inListExpr.isNot()) { + // NOT IN is always a scatter, except when we can be sure it would return nothing + if (isImpossibleNotIN(inListExpr)) { + return new planComparisonRetrun(false, true); + } + } else { + if (isImpossibleIN(inListExpr)) { + return new planComparisonRetrun(false, true); + } + boolean found = planInOp(ctx, inListExpr); + return new planComparisonRetrun(found, false); + } + return new planComparisonRetrun(false, false); + } + + private boolean planEqualOp(PlanningContext ctx, SQLBinaryOpExpr binaryOpExpr) throws SQLException { + if (binaryOpExpr.getLeft() instanceof SQLNullExpr || binaryOpExpr.getRight() instanceof SQLNullExpr) { + // 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 + this.setSelectNoneOpcode(); + return false; + } + String columnName; + + SQLExpr vdValue = binaryOpExpr.getRight(); + if (!(binaryOpExpr.getLeft() instanceof SQLName)) { + if (binaryOpExpr.getRight() instanceof SQLName) { + // either the LHS or RHS have to be a column to be useful for the vindex + vdValue = binaryOpExpr.getLeft(); + columnName = ((SQLName) binaryOpExpr.getRight()).getSimpleName(); + } else { + return false; + } + } else { + columnName = ((SQLName) binaryOpExpr.getLeft()).getSimpleName(); + } + + EvalEngine.Expr val = makeEvalEngineExpr(ctx, vdValue); + + return haveMatchingVindex(ctx, binaryOpExpr, vdValue, columnName, Collections.singletonList(val), Engine.RouteOpcode.SelectEqualUnique); + } + + /** + * makePlanValue transforms the given sqlparser.Expr into a sqltypes.PlanValue. + * If the given sqlparser.Expr is an argument and can be found in the r.argToReplaceBySelect then the + * method will stops and return nil values. + * Otherwise, the method will try to apply makePlanValue for any equality the sqlparser.Expr n has. + * The first PlanValue that is successfully produced will be returned. + * + * @param ctx + * @param n + * @return + * @throws SQLException + */ + private EvalEngine.Expr makeEvalEngineExpr(PlanningContext ctx, SQLExpr n) throws SQLException { + List exprEqualList = ctx.getSemTable().getExprAndEqualities(n); + for (SQLExpr expr : exprEqualList) { + EvalEngine.Expr pv = EvalEngine.translate(expr, ctx.getSemTable()); + if (pv != null) { + return pv; + } + } + return null; + } + + private EvalEngine.Expr makeInListEvalEngineExpr(PlanningContext ctx, List targetList) throws SQLException { + List exprEqualList = ctx.getSemTable().getExprAndEqualities(targetList); + EvalEngine.Expr pv = EvalEngine.translateEx(exprEqualList, ctx.getSemTable(),true); + return pv; + } + + private boolean isImpossibleIN(SQLInListExpr inListExpr) { + List right = inListExpr.getTargetList(); + if ((right.size() == 1) && (right.get(0) instanceof SQLNullExpr)) { + // WHERE col IN (null) + this.setSelectNoneOpcode(); + return true; + } + return false; + } + + private boolean planInOp(PlanningContext ctx, SQLInListExpr inListExpr) throws SQLException { + SQLExpr left = inListExpr.getExpr(); + if (left instanceof SQLName) { + for (VindexPlusPredicates v : this.getVindexPreds()) { + if (!SQLUtils.nameEquals(v.getColVindex().getColumn(), ((SQLName) left).getSimpleName())) { + // 如果不是Vindex,提前返回 + return false; + } + } + + EvalEngine.Expr pv = this.makeInListEvalEngineExpr(ctx, inListExpr.getTargetList()); + if (pv == null) { + return false; + } + inListExpr.setTargetList(Collections.singletonList(new SQLVariantRefListExpr("::" + Engine.LIST_VAR_NAME))); + return haveMatchingVindex(ctx, inListExpr, null, ((SQLName) left).getSimpleName(), Collections.singletonList(pv), Engine.RouteOpcode.SelectIN); + } + if (left instanceof SQLListExpr) { + List right = inListExpr.getTargetList(); + return planCompositeInOpRecursive(ctx, inListExpr, (SQLListExpr) left, right, new ArrayList<>()); + } + throw new SQLFeatureNotSupportedException(); + } + + private boolean planCompositeInOpRecursive(PlanningContext ctx, SQLInListExpr cmp, SQLListExpr left, List right, List coordinates) throws SQLException { + boolean foundVindex = false; + int cindex = coordinates.size(); + coordinates.add(0); + for (int i = 0; i < left.getItems().size(); i++) { + SQLExpr expr = left.getItems().get(i); + coordinates.set(cindex, i); + if (expr instanceof SQLListExpr) { + boolean ok = this.planCompositeInOpRecursive(ctx, cmp, (SQLListExpr) expr, right, coordinates); + return ok || foundVindex; + } else if (expr instanceof SQLName) { + // check if left col is a vindex + if (!this.hasVindex((SQLName) expr)) { + continue; + } + + + List rightVals = new ArrayList<>(); + + for (SQLExpr currRight : right) { + if (currRight instanceof SQLListExpr) { + SQLExpr val = tupleAccess(currRight, coordinates); + if (val == null) { + return false; + } + rightVals.add(val); + } else { + return false; + } + } + + EvalEngine.Expr evalEngineExpr = makeInListEvalEngineExpr(ctx, rightVals); + if (evalEngineExpr == null) { + return false; + } + boolean newVindex = this.haveMatchingVindex(ctx, cmp, null, ((SQLName) expr).getSimpleName(), Collections.singletonList(evalEngineExpr), Engine.RouteOpcode.SelectIN); + foundVindex = newVindex || foundVindex; + } + } + return foundVindex; + } + + private static SQLExpr tupleAccess(SQLExpr expr, List coordinates) { + SQLListExpr tuple = (SQLListExpr) expr; + for (int idx : coordinates) { + if (tuple == null || idx >= tuple.getItems().size()) { + return null; + } + + expr = tuple.getItems().get(idx); + + if (expr instanceof SQLListExpr) { + tuple = (SQLListExpr) expr; + } else { + tuple = null; + } + } + return expr; + } + + + private boolean isImpossibleNotIN(SQLInListExpr inListExpr) { + List right = inListExpr.getTargetList(); + for (SQLExpr sqlExpr : right) { + if (sqlExpr instanceof SQLNullExpr) { + this.setSelectNoneOpcode(); + return true; + } + } + return false; + } + + private boolean planIsExpr(PlanningContext ctx, SQLBinaryOpExpr binaryOpExpr) throws SQLException { + // we only handle IS NULL correct. IsExpr can contain other expressions as well + if (!(binaryOpExpr.getRight() instanceof SQLNullExpr)) { + return false; + } + if (!(binaryOpExpr.getLeft() instanceof SQLName)) { + return false; + } + SQLNullExpr sqlNullExpr = new SQLNullExpr(); + EvalEngine.Expr val = makeEvalEngineExpr(ctx, sqlNullExpr); + String columnName = ((SQLName) binaryOpExpr.getLeft()).getSimpleName(); + Engine.RouteOpcode routeOpcode = Engine.RouteOpcode.SelectScatter; + for (VindexPlusPredicates v : this.getVindexPreds()) { + if (SQLUtils.nameEquals(v.getColVindex().getColumn(), columnName)) { + routeOpcode = Engine.RouteOpcode.SelectEqualUnique; + break; + } + } + return haveMatchingVindex(ctx, binaryOpExpr, sqlNullExpr, columnName, Collections.singletonList(val), routeOpcode); + } + + private boolean hasVindex(SQLName column) { + for (VindexPlusPredicates v : this.getVindexPreds()) { + if (SQLUtils.nameEquals(v.getColVindex().getColumn(), column.getSimpleName())) { + return true; + } + } + return false; + } + + private boolean haveMatchingVindex(PlanningContext ctx, SQLExpr node, SQLExpr valueExpr, String columnName, List vtPlanValues, Engine.RouteOpcode routeOpcode) { + boolean newVindexFound = false; + for (VindexPlusPredicates v : this.getVindexPreds()) { +// if !ctx.SemTable.DirectDeps(column).IsSolvedBy(v.TableID) { +// continue +// } + if (SQLUtils.nameEquals(v.getColVindex().getColumn(), columnName)) { + // single column vindex - just add the option + VindexOption vOption = new VindexOption(); + vOption.setValues(vtPlanValues); + vOption.setValueExprs(Lists.newArrayList(valueExpr)); + vOption.setPredicates(Lists.newArrayList(node)); + vOption.setOpCode(routeOpcode); + vOption.setFoundVindex(new BinaryHash()); + vOption.setCost(costFor(v.getColVindex(), routeOpcode)); + vOption.setReady(true); + + v.getOptions().add(vOption); + newVindexFound = true; + } + } + return newVindexFound; + } + + private void setSelectNoneOpcode() { + this.setRouterOpCode(Engine.RouteOpcode.SelectNone); + // clear any chosen vindex as this query does not need to be sent down. + this.selected = null; + } + + // PickBestAvailableVindex goes over the available vindexes for this route and picks the best one available. + public void pickBestAvailableVindex() { + for (VindexPlusPredicates vindexPredicates : this.getVindexPreds()) { + VindexOption option = vindexPredicates.bestOption(); + if (option != null) { + this.selected = option; + this.routerOpCode = option.getOpCode(); + } + } + } + + // costFor returns a cost struct to make route choices easier to compare + private Cost costFor(Vschema.ColumnVindex colVindex, Engine.RouteOpcode opcode) { + Cost cost = new Cost(); + switch (opcode) { + case SelectUnsharded: + case SelectNext: + case SelectDBA: + case SelectReference: + case SelectNone: + case SelectScatter: + cost.setOpCode(opcode); + return cost; + } + + // foundVindex.Cost() always 1; + // foundVindex.IsUnique() always true; + cost.setVindexCost(1); + cost.setUnique(true); + + cost.setOpCode(opcode); + return cost; + } + + @Getter + @AllArgsConstructor + static class planComparisonRetrun { + private boolean found; + + private boolean exitEarly; + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/RoutePlanning.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/RoutePlanning.java new file mode 100644 index 0000000..726d976 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/RoutePlanning.java @@ -0,0 +1,932 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.operator.physical; + +import com.jd.jdbc.common.tuple.Pair; +import com.jd.jdbc.common.tuple.Triple; +import com.jd.jdbc.common.util.CollectionUtils; +import com.jd.jdbc.context.PlanningContext; +import com.jd.jdbc.engine.Engine; +import com.jd.jdbc.evalengine.EvalEngine; +import com.jd.jdbc.planbuilder.PlanBuilder; +import com.jd.jdbc.planbuilder.gen4.IntroducesTable; +import com.jd.jdbc.planbuilder.gen4.QueryTable; +import com.jd.jdbc.planbuilder.gen4.operator.Operator; +import com.jd.jdbc.planbuilder.gen4.operator.OperatorFuncs; +import com.jd.jdbc.planbuilder.gen4.operator.logical.Concatenate; +import com.jd.jdbc.planbuilder.gen4.operator.logical.Derived; +import com.jd.jdbc.planbuilder.gen4.operator.logical.Filter; +import com.jd.jdbc.planbuilder.gen4.operator.logical.Join; +import com.jd.jdbc.planbuilder.gen4.operator.logical.LogicalOperator; +import com.jd.jdbc.planbuilder.gen4.operator.logical.QueryGraph; +import com.jd.jdbc.planbuilder.gen4.operator.logical.Update; +import com.jd.jdbc.planbuilder.semantics.SchemaInformation; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLName; +import com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOpExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOperator; +import com.jd.jdbc.sqlparser.ast.expr.SQLExprUtils; +import com.jd.jdbc.sqlparser.ast.expr.SQLInListExpr; +import com.jd.jdbc.sqlparser.dialect.mysql.visitor.VtRewriteTableSchemaVisitor; +import com.jd.jdbc.sqlparser.utils.StringUtils; +import com.jd.jdbc.vindexes.VKeyspace; +import com.jd.jdbc.vindexes.VschemaConstant; +import static com.jd.jdbc.vindexes.VschemaConstant.TYPE_REFERENCE; +import com.jd.jdbc.vindexes.hash.Binary; +import java.sql.SQLException; +import java.sql.SQLFeatureNotSupportedException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import lombok.Getter; +import vschema.Vschema; + +public class RoutePlanning { + + public static PhysicalOperator createPhysicalOperator(PlanningContext ctx, LogicalOperator logicalOperator) throws SQLException { + if (logicalOperator instanceof QueryGraph) { + // case ctx.PlannerVersion == querypb.ExecuteOptions_Gen4Left2Right: + // return leftToRightSolve(ctx, op) + return greedySolve(ctx, (QueryGraph) logicalOperator); + } else if (logicalOperator instanceof Join) { + return optimizeJoin(ctx, (Join) logicalOperator); + } else if (logicalOperator instanceof Concatenate) { + return optimizeUnion(ctx, (Concatenate) logicalOperator); + } else if (logicalOperator instanceof Filter) { + return optimizeFilter(ctx, (Filter) logicalOperator); + } else if (logicalOperator instanceof Update) { + // todo + throw new SQLFeatureNotSupportedException(); + } else { + throw new SQLException("BUG: unexpected logicalOperator type: ", logicalOperator.toString()); + } + } + + private static PhysicalOperator optimizeUnion(PlanningContext ctx, Concatenate op) throws SQLException { + List sources = new ArrayList<>(); + for (LogicalOperator source : op.getSources()) { + PhysicalOperator qt = createPhysicalOperator(ctx, source); + sources.add(qt); + } + return new Union(sources, op.getSelectStmts(), op.isDistinct(), op.getOrderBy()); + } + + private static PhysicalOperator optimizeFilter(PlanningContext ctx, Filter op) throws SQLException { + PhysicalOperator src = createPhysicalOperator(ctx, op.getSource()); + + com.jd.jdbc.planbuilder.gen4.operator.physical.Filter filter = new com.jd.jdbc.planbuilder.gen4.operator.physical.Filter(); + + filter.setPredicates(op.getPredicates()); + + if (src instanceof Route) { + filter.setSource(((Route) src).getSource()); + ((Route) src).setSource(filter); + return src; + } + + filter.setSource(src); + return filter; + + } + + private static PhysicalOperator optimizeJoin(PlanningContext ctx, Join op) throws SQLException { + PhysicalOperator lhs = createPhysicalOperator(ctx, op.getLHS()); + PhysicalOperator rhs = createPhysicalOperator(ctx, op.getRHS()); + + List joinPredicates = PlanBuilder.splitAndExpression(new ArrayList<>(), op.getPredicate()); + return mergeOrJoin(ctx, lhs, rhs, joinPredicates, !op.getLeftJoin()); + } + + private static PhysicalOperator greedySolve(PlanningContext ctx, QueryGraph logicalOperator) throws SQLException { + List routeOps = seedOperatorList(ctx, logicalOperator); + OpCache planCache = new OpCache(); + + return mergeRoutes(ctx, logicalOperator, routeOps, planCache, false); + } + + /** + * seedOperatorList returns a route for each table in the qg + * + * @param ctx + * @param qg + * @return + * @throws SQLException + */ + private static List seedOperatorList(PlanningContext ctx, QueryGraph qg) throws SQLException { + int tablesSize = qg.getTables().size(); + List plans = new ArrayList<>(tablesSize); + // we start by seeding the table with the single routes + for (int i = 0; i < tablesSize; i++) { + QueryTable table = qg.getTables().get(i); + TableSet solves = ctx.getSemTable().tableSetFor(table.getAlias()); + Route plan = createRoute(ctx, table, solves); + if (qg.getNoDeps() != null) { + plan.setSource(new com.jd.jdbc.planbuilder.gen4.operator.physical.Filter(plan.getSource(), Collections.singletonList(qg.getNoDeps()))); + } + plans.add(i, plan); + } + return plans; + } + + private static Route createRoute(PlanningContext ctx, QueryTable table, TableSet solves) throws SQLException { + if (table.isInfSchema()) { + return createInfSchemaRoute(ctx, table); + } + SchemaInformation.SchemaInformationContext vschemaTable = ctx.getVschema().findTableOrVindex(table.getAlias()); + if (vschemaTable.getTable() == null) { + throw new SQLException("table " + table.getTable().getName() + " not found"); + } + Table tableSource = new Table(); + tableSource.setQTable(table); + tableSource.setVTable(vschemaTable.getTable()); + Route plan = new Route(); + plan.setSource(tableSource); + plan.setKeyspace(ctx.getKeyspace()); + + for (Vschema.ColumnVindex colVindex : vschemaTable.getTable().getColumnVindexesList()) { + VindexPlusPredicates vindexPlusPredicates = new VindexPlusPredicates(); + vindexPlusPredicates.setColVindex(colVindex); + vindexPlusPredicates.setTableId(solves); + plan.getVindexPreds().add(vindexPlusPredicates); + } + + if (Objects.equals(VschemaConstant.TYPE_SEQUENCE, vschemaTable.getTable().getType())) { + plan.setRouterOpCode(Engine.RouteOpcode.SelectNext); + } else if (Objects.equals(VschemaConstant.TYPE_REFERENCE, vschemaTable.getTable().getType())) { + plan.setRouterOpCode(Engine.RouteOpcode.SelectReference); + } else if (!ctx.getKeyspace().getSharded()) { + plan.setRouterOpCode(Engine.RouteOpcode.SelectUnsharded); + } else if (StringUtils.isNotEmpty(vschemaTable.getTable().getPinned()) && Objects.equals(VschemaConstant.CODE_PINNED_TABLE, vschemaTable.getTable().getPinned())) { + // Pinned tables have their keyspace ids already assigned. + // Use the Binary vindex, which is the identity function + // for keyspace id. + plan.setRouterOpCode(Engine.RouteOpcode.SelectEqualUnique); + + Cost selectCost = new Cost(); + selectCost.setOpCode(Engine.RouteOpcode.SelectEqualUnique); + + VindexOption selectOption = new VindexOption(); + selectOption.setReady(true); + // todo + //selectOption.setValues(Lists.newArrayList(EvalEngine.newLiteralString(vschemaTable.getTable().getPinned().getBytes()))); + selectOption.setOpCode(Engine.RouteOpcode.SelectEqualUnique); + selectOption.setFoundVindex(new Binary()); + selectOption.setCost(selectCost); + + plan.setSelected(selectOption); + } else { + plan.setRouterOpCode(Engine.RouteOpcode.SelectScatter); + } + + for (SQLExpr predicate : table.getPredicates()) { + plan.updateRoutingLogic(ctx, predicate); + } + + if (plan.getRouterOpCode().equals(Engine.RouteOpcode.SelectScatter) && table.getPredicates().size() > 0) { + // If we have a scatter query, it's worth spending a little extra time seeing if we can't improve it + for (SQLExpr pred : table.getPredicates()) { + List rewrittenList = tryRewriteOrToIn(pred); + if (CollectionUtils.isEmpty(rewrittenList)) { + break; + } + for (SQLExpr rewritten : rewrittenList) { + plan.updateRoutingLogic(ctx, rewritten); + } + } + } + + return plan; + } + + private static Route createInfSchemaRoute(PlanningContext ctx, QueryTable table) throws SQLException { + VKeyspace vKeyspace = ctx.getVschema().anyKeyspace(); + if (vKeyspace == null) { + throw new SQLFeatureNotSupportedException(); + } + + Table src = new Table(); + src.setQTable(table); + // src.setVTable(new Vschema.Table()); + + Route route = new Route(); + route.setRouterOpCode(Engine.RouteOpcode.SelectDBA); + route.setSource(src); + route.setKeyspace(vKeyspace); + + for (SQLExpr sqlExpr : table.getPredicates()) { + // todo + VtRewriteTableSchemaVisitor visitor = new VtRewriteTableSchemaVisitor(); + sqlExpr.accept(visitor); + SQLException exception = visitor.getException(); + if (exception != null) { + throw exception; + } + List tableNameExpressionList = visitor.getTableNameExpressionList(); + if (tableNameExpressionList != null && !tableNameExpressionList.isEmpty()) { + route.getSysTableTableSchema().addAll(tableNameExpressionList); + } + } + + return route; + } + + public static List tryRewriteOrToIn(SQLExpr expr) { + List sqlExprs = splitOrExpression(null, expr); + if (CollectionUtils.isEmpty(sqlExprs)) { + return null; + } + Map> map = new HashMap<>(); + for (SQLExpr sqlExpr : sqlExprs) { + if (sqlExpr instanceof SQLBinaryOpExpr) { + SQLBinaryOpExpr binaryOpExpr = (SQLBinaryOpExpr) sqlExpr; + if (binaryOpExpr.getLeft() instanceof SQLName && Objects.equals(SQLBinaryOperator.Equality, binaryOpExpr.getOperator())) { + map.computeIfAbsent(binaryOpExpr.getLeft(), key -> new ArrayList<>()).add(sqlExpr); + } else if (binaryOpExpr.getRight() instanceof SQLName && Objects.equals(SQLBinaryOperator.Equality, binaryOpExpr.getOperator())) { + map.computeIfAbsent(binaryOpExpr.getRight(), key -> new ArrayList<>()).add(sqlExpr); + } + } + if (sqlExpr instanceof SQLInListExpr) { + SQLInListExpr inListExpr = (SQLInListExpr) sqlExpr; + if (inListExpr.getExpr() instanceof SQLName) { + map.computeIfAbsent(inListExpr.getExpr(), key -> new ArrayList<>()).add(sqlExpr); + } + } + } + List sqlExprList = new ArrayList<>(); + for (Map.Entry> entry : map.entrySet()) { + if (entry.getValue().size() <= 1) { + continue; + } + List tuple = new ArrayList<>(); + for (SQLExpr sqlExpr : entry.getValue()) { + if (sqlExpr instanceof SQLBinaryOpExpr) { + SQLBinaryOpExpr binaryOpExpr = (SQLBinaryOpExpr) sqlExpr; + if (binaryOpExpr.getLeft() instanceof SQLName) { + tuple.add(binaryOpExpr.getRight()); + } else if (binaryOpExpr.getRight() instanceof SQLName) { + tuple.add(binaryOpExpr.getLeft()); + } + } + if (sqlExpr instanceof SQLInListExpr) { + SQLInListExpr inListExpr = (SQLInListExpr) sqlExpr; + if (inListExpr.getExpr() instanceof SQLName) { + tuple.addAll(inListExpr.getTargetList()); + } + } + } + SQLInListExpr inListExpr = new SQLInListExpr(entry.getKey()); + inListExpr.setTargetList(tuple); + sqlExprList.add(inListExpr); + } + return sqlExprList; + } + + private static List splitOrExpression(List filters, SQLExpr node) { + if (node == null) { + return filters; + } + if (filters == null) { + filters = new ArrayList<>(); + } + if (node instanceof SQLBinaryOpExpr) { + if (((SQLBinaryOpExpr) node).getOperator().equals(SQLBinaryOperator.BooleanOr)) { + filters = splitOrExpression(filters, ((SQLBinaryOpExpr) node).getLeft()); + return splitOrExpression(filters, ((SQLBinaryOpExpr) node).getRight()); + } + } + filters.add(node); + return filters; + } + + private static PhysicalOperator mergeRoutes(PlanningContext ctx, QueryGraph qg, List physicalOps, OpCache planCache, Boolean crossJoinsOK) throws SQLException { + if (physicalOps.size() == 0) { + return null; + } + + while (physicalOps.size() > 1) { + //throw new SQLException("BUG: logicalOperator length > 1"); + Triple ret = findBestJoin(ctx, qg, physicalOps, planCache, crossJoinsOK); + // if we found a plan, we'll replace the two plans that were joined with the join plan created + if (ret.getLeft() != null) { + // we remove one plan, and replace the other + int lIdx = ret.getMiddle(); + int rIdx = ret.getRight(); + + if (rIdx > lIdx) { + physicalOps.remove(rIdx); + physicalOps.remove(lIdx); + } else { + physicalOps.remove(lIdx); + physicalOps.remove(rIdx); + } + physicalOps.add(ret.getLeft()); + } else { + if (crossJoinsOK) { + throw new SQLException("BUG: should not happen"); + } + // 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 physicalOps with predicates first + crossJoinsOK = true; + } + } + return physicalOps.get(0); + } + + + private static Triple findBestJoin(PlanningContext ctx, QueryGraph qg, List plans, OpCache planCache, Boolean crossJoinsOK) + throws SQLException { + PhysicalOperator bestPlan = null; + int lIdx = 0; + int rIdx = 0; + for (int i = 0; i < plans.size(); i++) { + PhysicalOperator lhs = plans.get(i); + for (int j = 0; j < plans.size(); j++) { + if (i == j) { + continue; + } + PhysicalOperator rhs = plans.get(j); + List joinPredicates = qg.getPredicates(lhs.tableID(), rhs.tableID()); + + if (joinPredicates.size() == 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; + } + PhysicalOperator plan = getJoinFor(ctx, planCache, lhs, rhs, joinPredicates); + if (bestPlan == null || plan.cost() < bestPlan.cost()) { + bestPlan = plan; + // remember which plans we based on, so we can remove them later + lIdx = i; + rIdx = j; + } + + } + } + return Triple.of(bestPlan, lIdx, rIdx); + } + + private static PhysicalOperator getJoinFor(PlanningContext ctx, OpCache cm, PhysicalOperator lhs, PhysicalOperator rhs, List joinPredicates) throws SQLException { + TableSetPair solves = new TableSetPair(lhs.tableID(), rhs.tableID()); + PhysicalOperator cachedPlan = cm.getOpCacheMap().get(solves); + if (cachedPlan != null) { + return cachedPlan; + } + + PhysicalOperator join = mergeOrJoin(ctx, lhs, rhs, joinPredicates, true); + cm.getOpCacheMap().put(solves, join); + return join; + } + + /** + * requiresSwitchingSides will return true if any of the operators with the root from the given operator tree + * is of the type that should not be on the RHS of a join + * + * @param ctx + * @param op + * @return + */ + private static Boolean requiresSwitchingSides(PlanningContext ctx, PhysicalOperator op) throws SQLException { + VisitDerived vd = new VisitDerived(ctx); + visitOperator(op, vd); + return vd.isRequired(); + } + + private static PhysicalOperator mergeOrJoin(PlanningContext ctx, PhysicalOperator lhs, PhysicalOperator rhs, List joinPredicates, boolean inner) throws SQLException { + JoinMergerFunc merger = new JoinMergerFunc(joinPredicates, inner); + + PhysicalOperator newPlan = tryMerge(ctx, lhs, rhs, joinPredicates, merger); + if (newPlan != null) { + return newPlan; + } + + if (joinPredicates.size() > 0) { + if (requiresSwitchingSides(ctx, rhs)) { + if (!inner) { + throw new SQLException("unsupported: LEFT JOIN not supported for derived tables"); + } + if (requiresSwitchingSides(ctx, lhs)) { + throw new SQLException("unsupported: JOIN not supported between derived tables"); + } + + ApplyJoin join = new ApplyJoin(lhs.clone(), rhs.clone(), !inner); + return pushJoinPredicates(ctx, joinPredicates, join); + } + } + ApplyJoin join = new ApplyJoin(lhs.clone(), rhs.clone(), !inner); + return pushJoinPredicates(ctx, joinPredicates, join); + + } + + public static Route createRouteOperatorForJoin(Route aRoute, Route bRoute, List joinPredicates, boolean inner) { + + // append system table names from both the routes. + + Route newRoute = new Route(); + newRoute.setRouterOpCode(aRoute.getRouterOpCode()); + newRoute.setKeyspace(aRoute.getKeyspace()); + + List predicates = new ArrayList<>(); + predicates.addAll(aRoute.getVindexPreds()); + predicates.addAll(bRoute.getVindexPreds()); + newRoute.setVindexPreds(predicates); + + List seenPredicates = new ArrayList<>(); + seenPredicates.addAll(aRoute.getSeenPredicates()); + seenPredicates.addAll(bRoute.getSeenPredicates()); + newRoute.setSeenPredicates(seenPredicates); + + newRoute.getSysTableTableSchema().addAll(aRoute.getSysTableTableSchema()); + newRoute.getSysTableTableSchema().addAll(bRoute.getSysTableTableSchema()); + + + ApplyJoin source = new ApplyJoin(aRoute.getSource(), bRoute.getSource(), !inner, SQLBinaryOpExpr.combine(joinPredicates, SQLBinaryOperator.BooleanAnd)); + newRoute.setSource(source); + + // TODO == ? + if (aRoute.selectedVindex() == bRoute.selectedVindex()) { + newRoute.setSelected(aRoute.getSelected()); + } + + return newRoute; + } + + private static PhysicalOperator tryMerge(PlanningContext ctx, PhysicalOperator a, PhysicalOperator b, List joinPredicates, MergerFunc mergerFunc) throws SQLException { + // operatorsToRoutes + Route aRoute = null; + if (a instanceof Route) { + aRoute = (Route) a.clone(); + } else { + return null; + } + + Route bRoute = null; + if (b instanceof Route) { + bRoute = (Route) b.clone(); + } else { + return null; + } + + Boolean sameKeyspace = aRoute.getKeyspace().equals(bRoute.getKeyspace()); + if (sameKeyspace || (isDualTable(aRoute) || isDualTable(bRoute))) { + // tryMergeReferenceTable + Route tree = tryMergeReferenceTable(aRoute, bRoute, mergerFunc); + if (tree != null) { + return tree; + } + } + + switch (aRoute.getRouterOpCode()) { + case SelectUnsharded: + case SelectDBA: + if (aRoute.getRouterOpCode() == bRoute.getRouterOpCode() && sameKeyspace) { + return mergerFunc.merger(aRoute, bRoute); + } + break; + case SelectEqualUnique: + // If the two routes fully match, they can be merged together. + if (bRoute.getRouterOpCode() == Engine.RouteOpcode.SelectEqualUnique) { + boolean checkVindex = false; + if (aRoute.selectedVindex() != null && bRoute.selectedVindex() != null) { + // 我们仅仅比较vindex的类型是否一致 + checkVindex = aRoute.selectedVindex().getClass() == bRoute.selectedVindex().getClass(); + } + if (checkVindex && gen4ValuesEqual(ctx, aRoute.vindexExpressions(), bRoute.vindexExpressions())) { + return mergerFunc.merger(aRoute, bRoute); + } + + } + // If the two routes don't match, fall through to the next case and see if we + // can merge via join predicates instead. + // fallthrough + case SelectScatter: + case SelectIN: + case SelectNone: + if (joinPredicates.isEmpty()) { + // 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 null; + } + if (!sameKeyspace) { + throw new SQLException("unsupported: cross-shard correlated subquery"); + } + + boolean canMerge = canMergeOnFilters(ctx, aRoute, bRoute, joinPredicates); + if (!canMerge) { + return null; + } + + Route r = mergerFunc.merger(aRoute, bRoute); + + // If we have a `None` route opcode, we want to keep it - + // we only try to find a better Vindex for other route opcodes + if (aRoute.getRouterOpCode() != Engine.RouteOpcode.SelectNone) { + r.pickBestAvailableVindex(); + } + return r; + } + return null; + } + + private static Boolean isDualTable(Route route) throws SQLException { + List sources = leaves(route); + if (sources.size() > 1) { + return false; + } + + Operator src = sources.get(0); + + if (!(src instanceof Table)) { + return false; + } + + if (((Table) src).getVTable() == null) { + return false; + } + + if (((Table) src).getQTable() == null) { + return false; + } + + + return TYPE_REFERENCE.equalsIgnoreCase(((Table) src).getVTable().getType()) && ((Table) src).getQTable().getTable().getName().equalsIgnoreCase("dual") && + ((Table) src).getQTable().getTable().getQualifier().isEmpty(); + + } + + private static List leaves(Operator op) throws SQLException { + if (op instanceof QueryGraph || op instanceof Table) { + return new ArrayList<>(); + } + // logical + if (op instanceof Derived) { + return new ArrayList<>(Arrays.asList(((Derived) op).getInner())); + } + + if (op instanceof Join) { + return new ArrayList<>(Arrays.asList(((Join) op).getLHS(), ((Join) op).getRHS())); + } + + // physical + + if (op instanceof ApplyJoin) { + return new ArrayList<>(Arrays.asList(((ApplyJoin) op).getLHS(), ((ApplyJoin) op).getRHS())); + } + + if (op instanceof com.jd.jdbc.planbuilder.gen4.operator.physical.Filter) { + return new ArrayList<>(Arrays.asList(((com.jd.jdbc.planbuilder.gen4.operator.physical.Filter) op).getSource())); + } + + if (op instanceof Route) { + return new ArrayList<>(Arrays.asList(((Route) op).getSource())); + } + + throw new SQLException("leaves unknown type " + op.getClass().toString()); + } + + private static Route tryMergeReferenceTable(Route aRoute, Route bRoute, MergerFunc mergerFunc) throws SQLException { + Engine.RouteOpcode opCode; + VindexOption vindex; + VKeyspace ks; + + if (aRoute.getRouterOpCode() == Engine.RouteOpcode.SelectReference) { + vindex = bRoute.getSelected(); + opCode = bRoute.getRouterOpCode(); + ks = bRoute.getKeyspace(); + } else if (bRoute.getRouterOpCode() == Engine.RouteOpcode.SelectReference) { + vindex = aRoute.getSelected(); + opCode = aRoute.getRouterOpCode(); + ks = aRoute.getKeyspace(); + } else { + return null; + } + + Route r = mergerFunc.merger(aRoute, bRoute); + + r.setRouterOpCode(opCode); + r.setSelected(vindex); + r.setKeyspace(ks); + return r; + } + + private static Boolean gen4ValuesEqual(PlanningContext ctx, List a, List b) { + if (a == null && b == null) { + return true; + } + if (a == null || b == null) { + return false; + } + + if (a.size() != b.size()) { + return false; + } + + for (int i = 0; i < a.size(); i++) { + SQLExpr aExpr = a.get(i); + SQLExpr bExpr = b.get(i); + if (!SQLExprUtils.equals(aExpr, bExpr)) { + return false; + } + // ColName + if (aExpr instanceof SQLName) { + if (ctx.getSemTable().directDeps(aExpr) != ctx.getSemTable().directDeps(bExpr)) { + return false; + } + } + } + return true; + } + + private static PhysicalOperator pushJoinPredicates(PlanningContext ctx, List exprs, PhysicalOperator op) throws SQLException { + if (exprs.isEmpty()) { + return op; + } + + if (op instanceof ApplyJoin) { + return pushJoinPredicateOnJoin(ctx, exprs, (ApplyJoin) op); + } else if (op instanceof Route) { + return pushJoinPredicateOnRoute(ctx, exprs, (Route) op); + } else if (op instanceof Table) { + return OperatorFuncs.pushPredicate(ctx, SQLBinaryOpExpr.combine(exprs, SQLBinaryOperator.BooleanAnd), op); + } else if (op instanceof Derived) { + return pushJoinPredicateOnDerived(ctx, exprs, (Derived) op); + } else if (op instanceof com.jd.jdbc.planbuilder.gen4.operator.physical.Filter) { + ((com.jd.jdbc.planbuilder.gen4.operator.physical.Filter) op).getPredicates().addAll(exprs); + return op; + } else { + throw new SQLException("unknown type pushJoinPredicates :" + op.getClass().getName()); + } + + } + + private static PhysicalOperator pushJoinPredicateOnJoin(PlanningContext ctx, List exprs, ApplyJoin inNode) throws SQLException { + ApplyJoin node = (ApplyJoin) inNode.clone(); + List lhsPreds = new ArrayList<>(); + List rhsPreds = new ArrayList<>(); + List lhsVarsName = new ArrayList<>(); + + for (SQLExpr expr : exprs) { + // We find the dependencies for the given expression and if they are solved entirely by one + // side of the join tree, then we push the predicate there and do not break it into parts. + // In case a predicate has no dependencies, then it is pushed to both sides so that we can filter + // rows as early as possible making join cheaper on the vtgate level. + + TableSet depsForExpr = ctx.getSemTable().recursiveDeps(expr); + boolean singleSodeDeps = false; + TableSet lhsTables = node.getLHS().tableID(); + + if (depsForExpr.isSolvedBy(lhsTables)) { + lhsPreds.add(expr); + singleSodeDeps = true; + } + if (depsForExpr.isSolvedBy(node.getRHS().tableID())) { + rhsPreds.add(expr); + singleSodeDeps = true; + } + + if (singleSodeDeps) { + continue; + } + + Triple, List, SQLExpr> ret = OperatorFuncs.breakExpressioninLHSandRHS(ctx, expr, lhsTables); + node.getLhsColumns().addAll(ret.getMiddle()); + lhsVarsName.addAll(ret.getLeft()); + rhsPreds.add(ret.getRight()); + } + + if (!node.getLhsColumns().isEmpty() && !lhsVarsName.isEmpty()) { + Pair> ret = OperatorFuncs.pushOutputColumns(ctx, node.getLHS(), node.getLhsColumns()); + node.setLHS(ret.getLeft()); + for (int i = 0; i < ret.getRight().size(); i++) { + node.getVars().put(lhsVarsName.get(i), ret.getRight().get(i)); + } + } + + PhysicalOperator lhsPlan = pushJoinPredicates(ctx, lhsPreds, node.getLHS()); + PhysicalOperator rhsPlan = pushJoinPredicates(ctx, rhsPreds, node.getRHS()); + + node.setLHS(lhsPlan); + node.setRHS(rhsPlan); + + if (node.getPredicate() != null) { + exprs.add(node.getPredicate()); + } + node.setPredicate(SQLBinaryOpExpr.combine(exprs, SQLBinaryOperator.BooleanAnd)); + return node; + } + + private static PhysicalOperator pushJoinPredicateOnRoute(PlanningContext ctx, List exprs, Route op) throws SQLException { + for (SQLExpr expr : exprs) { + op.updateRoutingLogic(ctx, expr); + } + + PhysicalOperator newSrc = pushJoinPredicates(ctx, exprs, op.getSource()); + op.setSource(newSrc); + return op; + } + + private static PhysicalOperator pushJoinPredicateOnDerived(PlanningContext ctx, List exprs, Derived node) { + //TODO + return (PhysicalOperator) node; + } + + private static Boolean canMergeOnFilter(PlanningContext ctx, Route a, Route b, SQLExpr predicate) throws SQLException { + if (!(predicate instanceof SQLBinaryOpExpr)) { + return false; + } + if (((SQLBinaryOpExpr) predicate).getOperator() != SQLBinaryOperator.Equality) { + return false; + } + + SQLExpr left = ((SQLBinaryOpExpr) predicate).getLeft(); + SQLExpr right = ((SQLBinaryOpExpr) predicate).getRight(); + + Vschema.ColumnVindex lVindex = findColumnVindex(ctx, a, left); + if (lVindex == null) { + // left right 交换 + SQLExpr tmp = left; + left = right; + right = tmp; + lVindex = findColumnVindex(ctx, a, left); + } + Vschema.ColumnVindex rVindex = findColumnVindex(ctx, b, right); + + if (lVindex == null || rVindex == null) { + return false; + } + return lVindex.getName().equalsIgnoreCase(rVindex.getName()); + } + + public static Vschema.ColumnVindex findColumnVindex(PlanningContext ctx, PhysicalOperator a, SQLExpr exp) throws SQLException { + if (!(exp instanceof SQLName)) { + return null; + } + exp = unwrapDerivedTables(ctx, exp); + + // for each equality expression that exp has with other column name, we check if it + // can be solved by any table in our routeTree. If an equality expression can be solved, + // we check if the equality expression and our table share the same vindex, if they do: + // the method will return the associated vindexes.SingleColumn. + + List exprs = ctx.getSemTable().getExprAndEqualities(exp); + for (SQLExpr expr : exprs) { + if (!(expr instanceof SQLName)) { + continue; + } + + TableSet deps = ctx.getSemTable().recursiveDeps(expr); + FindSingleColumn fsc = new FindSingleColumn(deps, ((SQLName) expr)); + visitOperator(a, fsc); + if (fsc.getFindVindex() != null) { + return fsc.getFindVindex(); + } + } + return null; + } + + /** + * unwrapDerivedTables we want to find the bottom layer of derived tables + * + * @return + */ + private static SQLExpr unwrapDerivedTables(PlanningContext ctx, SQLExpr exp) { + return exp; + } + + private static Boolean canMergeOnFilters(PlanningContext ctx, Route a, Route b, List joinPredicates) throws SQLException { + for (SQLExpr predicate : joinPredicates) { + List filters = PlanBuilder.splitAndExpression(new ArrayList<>(), predicate); + for (SQLExpr expr : filters) { + if (canMergeOnFilter(ctx, a, b, expr)) { + return true; + } + } + } + return false; + } + + public static void visitOperator(PhysicalOperator op, VisitOperatorFunc func) throws SQLException { + if (!func.doFunc(op)) { + return; + } + if (op instanceof Table) { + return; + } else if (op instanceof Route) { + visitOperator(((Route) op).getSource(), func); + } else if (op instanceof com.jd.jdbc.planbuilder.gen4.operator.physical.Filter) { + visitOperator(((com.jd.jdbc.planbuilder.gen4.operator.physical.Filter) op).getSource(), func); + } else if (op instanceof ApplyJoin) { + visitOperator(((ApplyJoin) op).getLHS(), func); + visitOperator(((ApplyJoin) op).getRHS(), func); + } else { + throw new SQLException("unknown operator type while visiting " + op.getClass().getName()); + } + } + + public interface VisitOperatorFunc { + boolean doFunc(PhysicalOperator op); + } + + public interface MergerFunc { + Route merger(Route a, Route b) throws SQLException; + } + + static class JoinMergerFunc implements MergerFunc { + + private List joinPredicates; + + private boolean inner; + + public JoinMergerFunc(List joinPredicates, boolean inner) { + this.inner = inner; + this.joinPredicates = joinPredicates; + } + + @Override + public Route merger(Route a, Route b) throws SQLException { + return RoutePlanning.createRouteOperatorForJoin(a, b, this.joinPredicates, this.inner); + } + } + + @Getter + static class FindSingleColumn implements VisitOperatorFunc { + + private TableSet deps; + + private SQLName col; + + private Vschema.ColumnVindex findVindex; + + public FindSingleColumn(TableSet deps, SQLName col) { + this.deps = deps; + this.col = col; + } + + @Override + public boolean doFunc(PhysicalOperator op) { + if (!(op instanceof IntroducesTable)) { + return true; + } + if (this.deps.isSolvedBy(((IntroducesTable) op).getQTable().getId())) { + List vindexList = ((IntroducesTable) op).getVTable().getColumnVindexesList(); + for (Vschema.ColumnVindex vindex : vindexList) { + // Boolean isSingle = SingleColumn; + + if (vindex.getColumn().equalsIgnoreCase(this.col.getSimpleName())) { + this.findVindex = vindex; + return false; + } + } + } + return false; + } + } + + @Getter + static class VisitDerived implements VisitOperatorFunc { + + private boolean required = false; + + private PlanningContext ctx; + + public VisitDerived(PlanningContext ctx) { + this.ctx = ctx; + } + + @Override + public boolean doFunc(PhysicalOperator op) { + + if (op instanceof Derived) { + if (((Derived) op).isMergeable(this.ctx)) { + this.required = true; + return false; + } + } + return true; + } + } + +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/Table.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/Table.java new file mode 100644 index 0000000..afc71b7 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/Table.java @@ -0,0 +1,81 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.operator.physical; + +import com.jd.jdbc.planbuilder.gen4.IntroducesTable; +import com.jd.jdbc.planbuilder.gen4.QueryTable; +import com.jd.jdbc.planbuilder.semantics.SemTable; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.ast.SQLName; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import lombok.Getter; +import lombok.Setter; +import vschema.Vschema; + +@Getter +@Setter +public class Table implements PhysicalOperator, IntroducesTable { + private QueryTable qTable; + + private Vschema.Table vTable; + + private List columns; + + public Table() { + this.columns = new ArrayList<>(); + } + + public Table(QueryTable qTable, Vschema.Table vTable, List columns) { + this.qTable = qTable; + this.vTable = vTable; + this.columns = columns; + } + + @Override + public TableSet tableID() { + return qTable.getId(); + } + + @Override + public SQLSelectItem unsolvedPredicates(SemTable semTable) { + return null; + } + + @Override + public void checkValid() throws SQLException { + + } + + @Override + public Integer cost() { + return 0; + } + + @Override + public PhysicalOperator clone() { + List columns = new ArrayList<>(this.columns.size()); + for (SQLName name : this.columns) { + columns.add(name.clone()); + } + return new Table(this.qTable, this.vTable, columns); + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/TableSetPair.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/TableSetPair.java new file mode 100644 index 0000000..3252bca --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/TableSetPair.java @@ -0,0 +1,50 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.operator.physical; + +import com.jd.jdbc.planbuilder.semantics.TableSet; +import java.util.Objects; +import lombok.AllArgsConstructor; +import lombok.Getter; + +@Getter +@AllArgsConstructor +public class TableSetPair { + private TableSet left; + + private TableSet right; + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + TableSetPair that = (TableSetPair) o; + return left.equals(that.left) && right.equals(that.right); + } + + @Override + public int hashCode() { + return Objects.hash(left.hashCode(), right.hashCode()); + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/Union.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/Union.java new file mode 100644 index 0000000..6ee13ae --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/Union.java @@ -0,0 +1,76 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.operator.physical; + +import com.jd.jdbc.planbuilder.semantics.SemTable; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.ast.SQLOrderBy; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectQuery; +import java.sql.SQLException; +import java.util.List; +import lombok.Getter; + +public class Union implements PhysicalOperator { + + @Getter + private List sources; + + @Getter + private List selectStmts; + + @Getter + private boolean distinct; + + // TODO this should be removed. For now it's used to fail queries + @Getter + private SQLOrderBy orderBy; + + public Union(List sources, List selectStmts, boolean distinct, SQLOrderBy orderBy) { + this.sources = sources; + this.selectStmts = selectStmts; + this.distinct = distinct; + this.orderBy = orderBy; + } + + @Override + public TableSet tableID() { + return null; + } + + @Override + public SQLSelectItem unsolvedPredicates(SemTable semTable) { + return null; + } + + @Override + public void checkValid() throws SQLException { + + } + + @Override + public Integer cost() { + return null; + } + + @Override + public PhysicalOperator clone() { + return null; + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/VindexOption.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/VindexOption.java new file mode 100644 index 0000000..9188404 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/VindexOption.java @@ -0,0 +1,54 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.operator.physical; + +import com.jd.jdbc.engine.Engine; +import com.jd.jdbc.evalengine.EvalEngine; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.vindexes.Vindex; +import java.util.List; +import java.util.Map; +import lombok.Getter; +import lombok.Setter; + +/** + * VindexOption stores the information needed to know if we have all the information needed to use a vindex + */ +@Getter +@Setter +public class VindexOption { + private boolean ready; + + private List values; + + /** + * columns that we have seen so far. Used only for multi-column vindexes so that we can track how many columns part of the vindex we have seen + */ + private Map colsSeen; + + private List valueExprs; + + private List predicates; + + private Engine.RouteOpcode opCode; + + private Vindex foundVindex; + + private Cost cost; +} \ No newline at end of file diff --git a/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/VindexPlusPredicates.java b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/VindexPlusPredicates.java new file mode 100644 index 0000000..e7bab97 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/gen4/operator/physical/VindexPlusPredicates.java @@ -0,0 +1,63 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.operator.physical; + +import com.jd.jdbc.planbuilder.semantics.TableSet; +import java.util.ArrayList; +import java.util.List; +import lombok.Getter; +import lombok.Setter; +import vschema.Vschema; + +@Getter +@Setter +// VindexPlusPredicates is a struct used to store all the predicates that the vindex can be used to query +public class VindexPlusPredicates { + private TableSet tableId; + + private Vschema.ColumnVindex colVindex; + + // during planning, we store the alternatives found for this route in this slice + private List options; + + public VindexPlusPredicates() { + this.options = new ArrayList<>(); + } + + public VindexPlusPredicates(TableSet tableId, Vschema.ColumnVindex colVindex, List options) { + this.tableId = tableId; + this.colVindex = colVindex; + this.options = options; + } + + public VindexOption bestOption() { + VindexOption bestVindexOption = null; + if (this.getOptions().size() > 0) { + bestVindexOption = this.getOptions().get(0); + + } + return bestVindexOption; + } + + @Override + public VindexPlusPredicates clone() { + return new VindexPlusPredicates(this.tableId, this.colVindex, this.options); + } + +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/semantics/Analyzer.java b/src/main/java/com/jd/jdbc/planbuilder/semantics/Analyzer.java new file mode 100644 index 0000000..0300109 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/semantics/Analyzer.java @@ -0,0 +1,286 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import com.jd.jdbc.common.tuple.ImmutableTriple; +import com.jd.jdbc.common.tuple.Triple; +import com.jd.jdbc.planbuilder.PlanBuilder; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLObject; +import com.jd.jdbc.sqlparser.ast.statement.SQLExprTableSource; +import com.jd.jdbc.sqlparser.ast.statement.SQLJoinTableSource; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelect; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectStatement; +import com.jd.jdbc.sqlparser.ast.statement.SQLTableSource; +import com.jd.jdbc.sqlparser.ast.statement.SQLUnionQuery; +import com.jd.jdbc.sqlparser.ast.statement.SQLUpdateStatement; +import com.jd.jdbc.sqlparser.dialect.mysql.ast.statement.MySqlSelectQueryBlock; +import com.jd.jdbc.sqlparser.visitor.AnalyzerVisitor; +import com.jd.jdbc.sqlparser.visitor.CheckUnionVisitor; +import java.sql.SQLException; +import java.util.HashMap; +import lombok.Getter; +import lombok.Setter; + +/** + * analyzer controls the flow of the analysis. + * It starts the tree walking and controls which part of the analysis sees which parts of the tree + */ +public class Analyzer implements Originable { + @Getter + private Scoper scoper; + + @Getter + private TableCollector tables; + + @Getter + @Setter + private Binder binder; + + @Getter + @Setter + private Typer typer; + + @Getter + @Setter + private EarlyRewriter rewriter; + + private SQLException err; + + private int inProjection; + + private SQLException projErr; + //private String projErr; + + private SQLException unshardedErr; + //private String unshardedErr; + + private String warning; + + public Analyzer(Scoper scoper, TableCollector tables, Typer typer) { + this.scoper = scoper; + this.tables = tables; + this.typer = typer; + } + + /** + * newAnalyzer create the semantic analyzer + * + * @return + */ + public static Analyzer newAnalyzer(String dbName, SchemaInformation si) { + // TODO dependencies between these components are a little tangled. We should try to clean up + Scoper s = new Scoper(); + TableCollector tables = new TableCollector(s, si, dbName); + Typer typer = new Typer(); + Analyzer a = new Analyzer(s, tables, typer); + s.setOrg(a); + a.getTables().setOrg(a); + Binder b = new Binder(s, a, a.getTables(), a.getTyper()); + + a.setBinder(b); + EarlyRewriter earlyRewriter = new EarlyRewriter(s, b); + a.setRewriter(earlyRewriter); + s.setBinder(b); + return a; + } + + /** + * Analyze analyzes the parsed query. + * + * @param statement + * @param currentDb + * @param si + * @return + */ + public static SemTable analyze(SQLSelectStatement statement, String currentDb, SchemaInformation si) throws SQLException { + Analyzer analyzer = newAnalyzer(currentDb, si); + + // Analysis for initial scope + analyzer.analyze(statement); + + // Creation of the semantic table + return analyzer.newSemTable(statement, null); + } + + private void analyze(SQLSelectStatement statement) throws SQLException { + AnalyzerVisitor analyzerVisitor = new AnalyzerVisitor(this); + statement.accept(analyzerVisitor); + if (this.err != null) { + throw this.err; + } + } + + private SemTable newSemTable(SQLSelectStatement statement, Object coll) { + String comments = ""; + String projErr = this.projErr == null ? null : this.projErr.getMessage(); + String unshardedErr = this.unshardedErr == null ? null : this.unshardedErr.getMessage(); + + return new SemTable(this.binder.getRecursive(), this.binder.getDirect(), this.typer.getExprTypes(), this.tables.getTables(), this.getScoper().getRScope(), + projErr, unshardedErr, this.warning, comments, this.binder.getSubqueryMap(), this.binder.getSubqueryRef(), new HashMap<>(16), coll); + } + + @Override + public TableSet tableSetFor(SQLTableSource t) throws SQLException { + return this.tables.tableSetFor(t); + } + + @Override + public Triple depsForExpr(SQLExpr expr) { + TableSet direct = this.binder.getDirect().dependencies(expr); + TableSet recursive = this.binder.getRecursive().dependencies(expr); + Type qt = this.typer.getExprTypes().get(expr); + return new ImmutableTriple<>(direct, recursive, qt); + } + + private void setError(SQLException error) { + if (error instanceof Exception.ProjErrorException) { + this.projErr = error; + } else if (error instanceof Exception.UnshardedErrorException) { + this.unshardedErr = error; + } else { + this.err = error; + } + } + + public boolean analyzeDown(SQLObject x) { + // If we have an error we keep on going down the tree without checking for anything else + // this way we can abort when we come back up. + if (!this.shouldContinue()) { + return true; + } + + try { + this.scoper.down(x); + this.checkForInvalidConstruct(x); + this.rewriter.down(x); + } catch (SQLException e) { + this.setError(e); + return true; + } + this.warning = this.rewriter.getWarning(); + this.enterProjection(x); + // this is the visitor going down the tree. Returning false here would just not visit the children + // to the current node, but that is not what we want if we have encountered an error. + // In order to abort the whole visitation, we have to return true here and then return false in the `analyzeUp` method + return true; + } + + public boolean analyzeUp(SQLObject x) { + if (!this.shouldContinue()) { + return true; + } + try { + this.binder.up(x); + } catch (SQLException e) { + this.setError(e); + return true; + } + try { + this.scoper.up(x); + this.tables.up(x); + this.typer.up(x); + } catch (SQLException e) { + this.setError(e); + return false; + } + this.leaveProjection(x); + return this.shouldContinue(); + } + + private boolean shouldContinue() { + return this.err == null; + } + + private void checkForInvalidConstruct(SQLObject x) throws SQLException { + if (x instanceof SQLUpdateStatement) { + if (!(((SQLUpdateStatement) x).getFrom() instanceof SQLExprTableSource)) { + throw new SQLException("unsupported: only support single simple table in update"); + } + } else if (x instanceof MySqlSelectQueryBlock) { + SQLObject parent = x.getParent(); + if (parent instanceof SQLUnionQuery && ((MySqlSelectQueryBlock) x).isCalcFoundRows()) { + throw new SQLException("SQL_CALC_FOUND_ROWS not supported with union"); + } + if (!(parent instanceof SQLSelect) && ((MySqlSelectQueryBlock) x).isCalcFoundRows()) { + throw new SQLException("Incorrect usage/placement of 'SQL_CALC_FOUND_ROWS'"); + } + if (((MySqlSelectQueryBlock) x).getInto() == null) { + return; + } + throw new SQLException("unsupported: doesn't support into expr in select"); + } else if (x instanceof SQLJoinTableSource) { + // vitess其实支持排除了natural join类型,现在先不支持所有JOIN + SQLJoinTableSource.JoinType joinType = ((SQLJoinTableSource) x).getJoinType(); + if (joinType == SQLJoinTableSource.JoinType.NATURAL_JOIN || joinType == SQLJoinTableSource.JoinType.NATURAL_INNER_JOIN) { + throw new SQLException("unsupported: " + ((SQLJoinTableSource) x).getJoinType()); + } + } else if (x instanceof SQLUnionQuery) { + CheckUnionVisitor visitor = new CheckUnionVisitor(); + if (((SQLUnionQuery) x).getOrderBy() != null) { + ((SQLUnionQuery) x).getOrderBy().accept(visitor); + } + if (visitor.getErr() != null) { + throw new SQLException(visitor.getErr()); + } + checkUnionColumns((SQLUnionQuery) x); + } + } + + private void checkUnionColumns(SQLUnionQuery union) throws SQLException { + MySqlSelectQueryBlock firstProj = PlanBuilder.getFirstSelect(union); + if (firstProj.selectItemHasAllColumn()) { + // if we still have *, we can't figure out if the query is invalid or not + // we'll fail it at run time instead + return; + } + + MySqlSelectQueryBlock secondProj = PlanBuilder.getFirstSelect(union.getRight()); + if (secondProj.selectItemHasAllColumn()) { + return; + } + + if (firstProj.getSelectList().size() != secondProj.getSelectList().size()) { + throw new SQLException("The used SELECT statements have a different number of columns"); + } + } + + /** + * errors that happen when we are evaluating SELECT expressions are saved until we know + * if we can merge everything into a single route or not + * + * @param cursor + */ + private void enterProjection(SQLObject cursor) { + if (cursor instanceof SQLSelectItem) { + if (cursor.getParent() instanceof MySqlSelectQueryBlock) { + this.inProjection++; + } + } + } + + private void leaveProjection(SQLObject cursor) { + if (cursor instanceof SQLSelectItem) { + if (cursor.getParent() instanceof MySqlSelectQueryBlock) { + this.inProjection--; + } + } + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/semantics/Binder.java b/src/main/java/com/jd/jdbc/planbuilder/semantics/Binder.java new file mode 100644 index 0000000..bb70fe9 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/semantics/Binder.java @@ -0,0 +1,221 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLName; +import com.jd.jdbc.sqlparser.ast.SQLObject; +import com.jd.jdbc.sqlparser.ast.expr.SQLAggregateExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLAllColumnExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLExprUtils; +import com.jd.jdbc.sqlparser.ast.expr.SQLPropertyExpr; +import com.jd.jdbc.sqlparser.ast.statement.SQLJoinTableSource; +import com.jd.jdbc.sqlparser.ast.statement.SQLTableSource; +import com.jd.jdbc.sqlparser.dialect.mysql.ast.statement.MySqlSelectQueryBlock; +import java.sql.SQLException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import lombok.Getter; + +/** + * binder is responsible for finding all the column references in + * the query and bind them to the table that they belong to. + * While doing this, it will also find the types for columns and + * store these in the typer:s expression map + */ +public class Binder { + private static final SQLAggregateExpr COUNT_STAR; + + static { + COUNT_STAR = new SQLAggregateExpr("COUNT"); + COUNT_STAR.addArgument(new SQLAllColumnExpr()); + } + + @Getter + private ExprDependencies recursive; + + @Getter + private ExprDependencies direct; + + private Scoper scoper; + + private TableCollector tc; + + private Originable org; + + private Typer typer; + + @Getter + private Map subqueryMap; + + @Getter + private Map subqueryRef; + + /** + * every table will have an entry in the outer map. it will point to a map with all the columns + * that this map is joined with using USING. + * This information is used to expand `*` correctly, and is not available post-analysis + */ + private Map> usingJoinInfo; + + public Binder(Scoper scoper, Originable org, TableCollector tc, Typer typer) { + this.scoper = scoper; + this.tc = tc; + this.org = org; + this.typer = typer; + this.recursive = new ExprDependencies(); + this.direct = new ExprDependencies(); + this.subqueryMap = new HashMap<>(16); + this.subqueryRef = new HashMap<>(16); + this.usingJoinInfo = new HashMap<>(16); + } + + public void up(SQLObject cursor) throws SQLException { + if (cursor instanceof SQLJoinTableSource) { + Scope currentScope = this.scoper.currentScope(); + // using + List usings = ((SQLJoinTableSource) cursor).getUsing(); + for (SQLExpr expr : usings) { + Dependency deps = this.resolveColumn((SQLName) expr, currentScope, true); + currentScope.getJoinUsing().put(((SQLName) expr).getSimpleName().toLowerCase(), deps.getDirect()); + } + if (usings.size() > 0) { + EarlyRewriter.rewriteJoinUsing(currentScope, usings, this.org); + // TODO reset using .. + } + + // throw new SQLException(); + } + if (cursor instanceof SQLName) { + Scope currentScope = this.scoper.currentScope(); + Dependency deps = new Dependency(); + try { + deps = this.resolveColumn((SQLName) cursor, currentScope, false); + } catch (SQLException err) { + String errMsg = err.getMessage(); + boolean checkNumberOfTables = deps.getDirect().numberOfTables() == 0; + if (checkNumberOfTables + || !errMsg.endsWith("is ambiguous") + || !this.canRewriteUsingJoin(deps, (SQLName) cursor)) { + throw err; + } + + // if we got here it means we are dealing with a ColName that is involved in a JOIN USING. + // we do the rewriting of these ColName structs here because it would be difficult to copy all the + // needed state over to the earlyRewriter + deps = rewriteJoinUsingColName(deps, (SQLName) cursor, currentScope); + } + this.recursive.put(cursor, deps.getRecursive()); + this.direct.put(cursor, deps.getDirect()); + if (deps.getTyp() != null) { + this.typer.setTypeFor(cursor, deps.getTyp()); + } + } + // count(*) + if (cursor instanceof SQLAggregateExpr && SQLExprUtils.equals(COUNT_STAR, (SQLAggregateExpr) cursor)) { + TableSet ts = new TableSet(); + Scope scope = this.scoper.currentScope(); + for (TableInfo table : scope.getTables()) { + SQLTableSource expr = table.getExpr(); + if (expr != null) { + ts.mergeInPlace(this.tc.tableSetFor(expr)); + } + } + this.recursive.put(cursor, ts); + this.direct.put(cursor, ts); + } + } + + private Dependency resolveColumn(SQLName colName, Scope current, boolean allowMulti) throws SQLException { + Dependencies thisDeps = null; + while (current != null) { + SQLException err = null; + try { + thisDeps = this.resolveColumnInScope(colName, current, allowMulti); + } catch (SQLException e) { + err = makeAmbiguousError(colName, e); + if (thisDeps == null) { + throw err; + } + } + if (!thisDeps.empty()) { + try { + Dependency deps = thisDeps.get(); + return deps; + } catch (SQLException e) { + err = makeAmbiguousError(colName, e); + throw err; + } + } + if (err != null) { + throw err; + } + current = current.getParent(); + } + throw new SQLException("symbol " + colName + " not found"); + } + + private Dependencies resolveColumnInScope(SQLName expr, Scope current, boolean allowMulti) throws SQLException { + Dependencies deps = new Nothing(); + for (TableInfo table : current.getTables()) { + if (expr instanceof SQLPropertyExpr && !table.matches(expr)) { + continue; + } + Dependencies thisDeps = table.dependencies(expr.getSimpleName(), this.org); + deps = thisDeps.merge(deps, allowMulti); + } + if (deps instanceof Uncertain && ((Uncertain) deps).isFail()) { + // if we have a failure from uncertain, we matched the column to multiple non-authoritative tables + throw new Exception.ProjErrorException("Column '" + expr + "' in field list is ambiguous"); + } + return deps; + } + + private SQLException makeAmbiguousError(SQLName colName, SQLException err) { + if (err instanceof Exception.AmbiguousException) { + return new SQLException("Column '" + colName + "' in field list is ambiguous"); + } + return err; + } + + /** + * canRewriteUsingJoin will return true when this ColName is safe to rewrite since it can only belong to a USING JOIN + * + * @param deps + * @param node + * @return + */ + private boolean canRewriteUsingJoin(Dependency deps, SQLName node) { + List tbls = deps.getDirect().constituents(); + String colName = node.getSimpleName().toLowerCase(); + for (TableSet tbl : tbls) { + Map m = this.usingJoinInfo.get(tbl); + if (m == null || m.get(colName) == null) { + return false; + } + } + return true; + } + + private Dependency rewriteJoinUsingColName(Dependency deps, SQLName node, Scope currentScope) throws SQLException { + //TODO; + return deps; + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/semantics/Certain.java b/src/main/java/com/jd/jdbc/planbuilder/semantics/Certain.java new file mode 100644 index 0000000..ab64f0d --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/semantics/Certain.java @@ -0,0 +1,59 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import java.sql.SQLException; +import java.util.Objects; + +public class Certain extends Dependency implements Dependencies { + private SQLException err; + + public Certain(TableSet direct, TableSet recursive) { + super(direct, recursive); + } + + @Override + public boolean empty() { + return false; + } + + @Override + public Dependency get() throws SQLException { + if (err != null) { + throw err; + } + return this; + } + + @Override + public Dependencies merge(Dependencies d, boolean allowMulti) { + if (d instanceof Certain) { + if (Objects.equals(((Certain) d).getRecursive(), super.getRecursive())) { + return this; + } + this.getDirect().mergeInPlace(((Certain) d).getDirect()); + this.getRecursive().mergeInPlace(((Certain) d).getRecursive()); + if (!allowMulti) { + err = new Exception.AmbiguousException("ambiguous"); + } + return this; + } + return this; + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/semantics/ColumnInfo.java b/src/main/java/com/jd/jdbc/planbuilder/semantics/ColumnInfo.java new file mode 100644 index 0000000..49eecff --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/semantics/ColumnInfo.java @@ -0,0 +1,41 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import lombok.Data; + +/** + * ColumnInfo contains information about columns + */ +@Data +public class ColumnInfo { + + private String name; + + private Type type; + + public ColumnInfo(String name, Type type) { + this.name = name; + this.type = type; + } + + public ColumnInfo(String name) { + this.name = name; + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/semantics/ColumnName.java b/src/main/java/com/jd/jdbc/planbuilder/semantics/ColumnName.java new file mode 100644 index 0000000..9704f80 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/semantics/ColumnName.java @@ -0,0 +1,46 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import java.util.Objects; +import lombok.AllArgsConstructor; + +@AllArgsConstructor +public class ColumnName { + private TableSet table; + + private String columnName; + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ColumnName that = (ColumnName) o; + return Objects.equals(table, that.table) && Objects.equals(columnName, that.columnName); + } + + @Override + public int hashCode() { + return Objects.hash(table, columnName); + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/semantics/Dependencies.java b/src/main/java/com/jd/jdbc/planbuilder/semantics/Dependencies.java new file mode 100644 index 0000000..dcf9e37 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/semantics/Dependencies.java @@ -0,0 +1,47 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import io.vitess.proto.Query; +import java.sql.SQLException; + +/** + * these types are used to go over dependencies provided by multiple + * tables and figure out bindings and/or errors by merging dependencies together + */ +public interface Dependencies { + + boolean empty(); + + Dependency get() throws SQLException; + + Dependencies merge(Dependencies other, boolean allowMulti); + + static Certain createCertain(TableSet direct, TableSet recursive, Type qt) { + Certain c = new Certain(direct, recursive); + if (qt != null && qt.getType() != Query.Type.NULL_TYPE) { + c.setTyp(qt); + } + return c; + } + + static Uncertain createUncertain(TableSet direct, TableSet recursive) { + return new Uncertain(direct, recursive); + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/semantics/Dependency.java b/src/main/java/com/jd/jdbc/planbuilder/semantics/Dependency.java new file mode 100644 index 0000000..554abeb --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/semantics/Dependency.java @@ -0,0 +1,44 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import lombok.Getter; +import lombok.Setter; + +public class Dependency { + @Getter + private TableSet direct; + + @Getter + private TableSet recursive; + + @Setter + @Getter + private Type typ; + + public Dependency(TableSet direct, TableSet recursive) { + this.direct = direct; + this.recursive = recursive; + } + + public Dependency() { + this.direct = new TableSet(0, null); + this.recursive = new TableSet(0, null); + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/semantics/DerivedTable.java b/src/main/java/com/jd/jdbc/planbuilder/semantics/DerivedTable.java new file mode 100644 index 0000000..3ff01b7 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/semantics/DerivedTable.java @@ -0,0 +1,172 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import com.jd.jdbc.common.tuple.Triple; +import com.jd.jdbc.sqlparser.SQLUtils; +import com.jd.jdbc.sqlparser.SqlParser; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLObject; +import com.jd.jdbc.sqlparser.ast.expr.SQLIdentifierExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLPropertyExpr; +import com.jd.jdbc.sqlparser.ast.statement.SQLExprTableSource; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import com.jd.jdbc.sqlparser.ast.statement.SQLTableSource; +import com.jd.jdbc.sqlparser.utils.TableNameUtils; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import lombok.Getter; +import lombok.Setter; +import vschema.Vschema; + +@Setter +@Getter +public class DerivedTable implements TableInfo { + + String tableName; + + SQLTableSource astnode; + + List columnNames; + + List cols; + + TableSet tables; + + public DerivedTable() { + this.tables = new TableSet(); + this.cols = new ArrayList<>(); + this.columnNames = new ArrayList<>(); + } + + @Override + public SQLExprTableSource name() throws SQLException { + if (astnode.getAlias() != null) { + return new SQLExprTableSource(new SQLIdentifierExpr(astnode.getAlias()), null); + } + throw new SQLException("Every derived table must have its own alias"); + } + + @Override + public Vschema.Table getVindexTable() { + return null; + } + + @Override + public boolean isInfSchema() { + return false; + } + + @Override + public boolean matches(SQLObject expr) throws SQLException { + String tableName; + SQLPropertyExpr leftExpr = (SQLPropertyExpr) expr; + tableName = leftExpr.getOwnernName(); + String qualifier = TableNameUtils.getQualifier(astnode); + return SQLUtils.nameEquals(tableName, qualifier) && SQLUtils.nameEquals(tableName, this.tableName); + } + + @Override + public boolean authoritative() { + return false; + } + + @Override + public SQLTableSource getExpr() { + return this.astnode; + } + + @Override + public List getColumns() { + return null; + } + + @Override + public Dependencies dependencies(String colNames, Originable org) throws SQLException { + TableSet directDeps = org.tableSetFor(this.astnode); + for (int i = 0; i < this.columnNames.size(); i++) { + if (!this.columnNames.get(i).equals(colNames)) { + continue; + } + Triple recursiveDeps = org.depsForExpr(this.cols.get(i)); + return Dependencies.createCertain(directDeps, recursiveDeps.getMiddle(), recursiveDeps.getRight()); + } + if (!this.hasStar()) { + return new Nothing(); + } + + return Dependencies.createUncertain(directDeps, this.tables); + + } + + public Boolean hasStar() { + if (this.tables == null) { + return false; + } + return this.tables.numberOfTables() > 0; + } + + @Override + public TableSet getTableSet(Originable org) { + return this.tables; + } + + //todo sqlparser.Columns不清楚什么类型节点用Object[]先代替 + public static DerivedTable createDerivedTableForExpressions(List expressions, Object[] cols, List tables, Originable org) throws SQLException { + DerivedTable vTbl = new DerivedTable(); + for (int i = 0; i < expressions.size(); i++) { + SQLSelectItem expr = expressions.get(i); + if (SqlParser.SelectExpr.StarExpr.equals(SqlParser.SelectExpr.type(expr))) { + for (TableInfo table : tables) { + vTbl.tables.mergeInPlace(table.getTableSet(org)); + } + } else { + vTbl.cols.add(expr.getExpr()); + if (cols != null) { + vTbl.columnNames.add(cols[i].toString()); + } else if (expr.getAlias() == null) { + // for projections, we strip out the qualifier and keep only the column name + if (expr.getExpr() instanceof SQLPropertyExpr) { + vTbl.columnNames.add(((SQLPropertyExpr) expr.getExpr()).getName()); + } else { + vTbl.columnNames.add(expr.getExpr().toString()); + } + } else { + vTbl.columnNames.add(expr.getAlias()); + } + } + } + return vTbl; + } + + public void checkForDuplicates() throws SQLException { + for (int i = 0; i < this.columnNames.size(); i++) { + String name = this.columnNames.get(i); + for (int j = 0; j < this.columnNames.size(); j++) { + if (i == j) { + continue; + } + if (this.columnNames.get(j).equals(name)) { + throw new SQLException("Duplicate column name '" + name + "'"); + } + } + } + } +} \ No newline at end of file diff --git a/src/main/java/com/jd/jdbc/planbuilder/semantics/EarlyRewriter.java b/src/main/java/com/jd/jdbc/planbuilder/semantics/EarlyRewriter.java new file mode 100644 index 0000000..145a5c4 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/semantics/EarlyRewriter.java @@ -0,0 +1,215 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import com.jd.jdbc.sqlparser.ASTUtils; +import com.jd.jdbc.sqlparser.SQLUtils; +import com.jd.jdbc.sqlparser.SqlParser; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLName; +import com.jd.jdbc.sqlparser.ast.SQLObject; +import com.jd.jdbc.sqlparser.ast.SQLOrderBy; +import com.jd.jdbc.sqlparser.ast.SQLStatement; +import com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOpExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOperator; +import com.jd.jdbc.sqlparser.ast.expr.SQLIdentifierExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLIntegerExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLLiteralExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLPropertyExpr; +import com.jd.jdbc.sqlparser.ast.statement.SQLExprTableSource; +import com.jd.jdbc.sqlparser.ast.statement.SQLJoinTableSource; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectGroupByClause; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import com.jd.jdbc.sqlparser.dialect.mysql.ast.statement.MySqlSelectQueryBlock; +import com.jd.jdbc.sqlparser.utils.StringUtils; +import com.jd.jdbc.sqlparser.visitor.OrderByGroupByNumberRewriteVisitor; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import lombok.Getter; + +public class EarlyRewriter { + private Binder binder; + + private Scoper scoper; + + private String clause; + + @Getter + private String warning; + + public EarlyRewriter(Scoper scoper, Binder binder) { + this.binder = binder; + this.scoper = scoper; + } + + public void down(SQLObject cursor) throws SQLException { + if (cursor instanceof SQLSelectGroupByClause) { + this.clause = "group statement"; + return; + } + if (cursor instanceof SQLOrderBy) { + this.clause = "order clause"; + return; + } + if (cursor instanceof SQLLiteralExpr) { + SQLExpr newNode = rewriteOrderByExpr((SQLLiteralExpr) cursor); + if (newNode != null) { + replace(cursor, newNode); + } + return; + } + if (cursor instanceof SQLSelectItem) { + SqlParser.SelectExpr selectExpr = SqlParser.SelectExpr.type((SQLSelectItem) cursor); + if (SqlParser.SelectExpr.StarExpr.equals(selectExpr)) { + // expandStar + this.expandStar((SQLSelectItem) cursor); + } + return; + } + if (cursor instanceof SQLJoinTableSource) { + // straight join is converted to normal join + if (((SQLJoinTableSource) cursor).getJoinType() == SQLJoinTableSource.JoinType.STRAIGHT_JOIN) { + ((SQLJoinTableSource) cursor).setJoinType(SQLJoinTableSource.JoinType.JOIN); + } + } + } + + private void replace(SQLObject cursor, SQLExpr newNode) { + OrderByGroupByNumberRewriteVisitor rewriteVisitor = new OrderByGroupByNumberRewriteVisitor(cursor, newNode); + SQLObject parent = ASTUtils.getOrderByGroupByParent(cursor); + if (parent != null) { + parent.accept(rewriteVisitor); + } + } + + private SQLExpr rewriteOrderByExpr(SQLLiteralExpr node) throws SQLException { + Scope currScope = this.scoper.getSpecialExprScopes().get(node); + if (currScope == null) { + return null; + } + int num; + if (node instanceof SQLIntegerExpr) { + if (!(((SQLIntegerExpr) node).getNumber() instanceof Integer)) { + throw new SQLException("error parsing column number: " + node); + } + num = ((SQLIntegerExpr) node).getNumber().intValue(); + } else { + throw new SQLException("error parsing column number: " + node); + } + MySqlSelectQueryBlock stmt; + if (currScope.getStmt() instanceof MySqlSelectQueryBlock) { + stmt = (MySqlSelectQueryBlock) currScope.getStmt(); + } else { + throw new SQLException("error invalid statement type, expect Select, got:" + currScope.getStmt().getClass().getSimpleName()); + } + if (num < 1 || num > stmt.getSelectList().size()) { + throw new SQLException("Unknown column '" + num + "' in '" + this.clause + "'"); + } + for (int i = 0; i < num; i++) { + SQLSelectItem selectItem = stmt.getSelectList().get(i); + SqlParser.SelectExpr selectExpr = SqlParser.SelectExpr.type(selectItem); + if (!SqlParser.SelectExpr.AliasedExpr.equals(selectExpr)) { + throw new SQLException("cannot use column offsets in " + this.clause + " when using `" + selectItem + "`"); + } + } + SQLSelectItem aliasedExpr = stmt.getSelectList().get(num - 1); + if (StringUtils.isNotEmpty(aliasedExpr.getAlias())) { + return new SQLIdentifierExpr(aliasedExpr.getAlias()); + } + SQLExpr expr = realCloneOfColNames(aliasedExpr.getExpr(), currScope.isUnion()); + return expr; + } + + /** + * realCloneOfColNames clones all the expressions including ColName. + * Since sqlparser.CloneRefOfColName does not clone col names, this method is needed. + * + * @param expr + * @param union + * @return + */ + private SQLExpr realCloneOfColNames(SQLExpr expr, boolean union) { + if (expr instanceof SQLName) { + if (union) { + if (expr instanceof SQLPropertyExpr) { + String name = ((SQLPropertyExpr) expr).getName(); + return new SQLIdentifierExpr(name); + } + } + return expr.clone(); + } + return expr.clone(); + } + + public static void rewriteJoinUsing(Scope current, List usings, Originable org) throws SQLException { + Map> joinUsing = current.prepareUsingMap(); + List predicates = new ArrayList<>(); + for (SQLExpr column : usings) { + List foundTables = new ArrayList<>(); + String colName = ((SQLName) column).getSimpleName(); + for (TableInfo tbl : current.getTables()) { + if (!tbl.authoritative()) { + throw new SQLException("can't handle JOIN USING without authoritative tables"); + } + SQLExprTableSource tblName = tbl.name(); + TableSet currTable = tbl.getTableSet(org); + Map usingCols = joinUsing.get(currTable); + if (usingCols == null) { + usingCols = new HashMap<>(); + } + for (ColumnInfo col : tbl.getColumns()) { + TableSet found = usingCols.get(col.getName()); + if (found != null) { + foundTables.add(tblName); + } + } + } + for (int i = 0; i < foundTables.size(); i++) { + SQLExprTableSource lft = foundTables.get(i); + for (int j = i + 1; j < foundTables.size(); j++) { + SQLExprTableSource rgt = foundTables.get(j); + SQLExpr left = new SQLPropertyExpr((SQLExpr) lft, colName); + SQLExpr right = new SQLPropertyExpr((SQLExpr) rgt, colName); + predicates.add(new SQLBinaryOpExpr(left, SQLBinaryOperator.Equality, right)); + } + } + } + + // now, we go up the scope until we find a SELECT with a where clause we can add this predicate to + + while (current != null) { + SQLObject sel = current.getStmt(); + if (sel instanceof MySqlSelectQueryBlock) { + SQLExpr where = SQLBinaryOpExpr.combine(predicates, SQLBinaryOperator.BooleanAnd); + SQLUtils.addCondition((SQLStatement) sel, SQLBinaryOperator.BooleanAnd, where, false); + return; + } + current = current.getParent(); + } + throw new SQLException("did not find WHERE clause"); + + } + + private void expandStar(SQLSelectItem node) { + // TOOD + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/semantics/Exception.java b/src/main/java/com/jd/jdbc/planbuilder/semantics/Exception.java new file mode 100644 index 0000000..05c8e4d --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/semantics/Exception.java @@ -0,0 +1,49 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import java.sql.SQLException; + +public class Exception { + public static class ProjErrorException extends SQLException { + public ProjErrorException(String reason) { + super(reason); + } + } + + public static class UnshardedErrorException extends SQLException { + public UnshardedErrorException(String reason) { + super(reason); + } + } + + public static class AmbiguousException extends SQLException { + public AmbiguousException(String reason) { + super(reason); + } + } + + public static class ErrMultipleTablesException extends SQLException { + public ErrMultipleTablesException(String reason) { + super(reason); + } + } + +} + diff --git a/src/main/java/com/jd/jdbc/planbuilder/semantics/ExprDependencies.java b/src/main/java/com/jd/jdbc/planbuilder/semantics/ExprDependencies.java new file mode 100644 index 0000000..47f41e9 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/semantics/ExprDependencies.java @@ -0,0 +1,69 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLObject; +import com.jd.jdbc.sqlparser.visitor.ExprDependenciesVisitor; +import java.util.HashMap; +import java.util.Map; + +/** + * ExprDependencies stores the tables that an expression depends on as a map + */ +public class ExprDependencies { + private Map tableSetMap; + + public ExprDependencies() { + this.tableSetMap = new HashMap<>(16); + } + + public void put(SQLObject sqlObject, TableSet tableSet) { + tableSetMap.put(new SQLObjectExpr(sqlObject), tableSet); + } + + public TableSet get(SQLObject sqlObject) { + return tableSetMap.get(new SQLObjectExpr(sqlObject)); + } + + /** + * dependencies return the table dependencies of the expression. This method finds table dependencies recursively + * + * @param expr + * @return + */ + public TableSet dependencies(SQLExpr expr) { + //todo BUG:tableSetMap.get可能取不到对应的key值,对应operator测试用例 # we should remove the keyspace from predicates + if (Scoper.validAsMapKey(expr)) { + // we have something that could live in the cache + TableSet deps = this.get(expr); + if (deps != null) { + return deps; + } + } + + // 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. + ExprDependenciesVisitor visitor = new ExprDependenciesVisitor(this, expr); + expr.accept(visitor); + TableSet deps = visitor.getDeps(); + return deps; + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/semantics/LargeTableSet.java b/src/main/java/com/jd/jdbc/planbuilder/semantics/LargeTableSet.java new file mode 100644 index 0000000..a9bdbc0 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/semantics/LargeTableSet.java @@ -0,0 +1,199 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import java.util.Arrays; +import java.util.function.Consumer; +import lombok.Data; +@Data +class LargeTableSet { + + + private long[] tables; + + public LargeTableSet(long[] tables) { + this.tables = tables; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + LargeTableSet that = (LargeTableSet) o; + return Arrays.equals(tables, that.tables); + } + + @Override + public int hashCode() { + return tables.hashCode(); + } + + @Override + public LargeTableSet clone() { + return new LargeTableSet(this.tables.clone()); + } + + public static LargeTableSet newLargeTableSet(Long small, int tableidx) { + int chunk = tableidx / 64; + int offset = tableidx % 64; + long[] tables = new long[chunk + 1]; + tables[0] = small; + tables[chunk] |= 1L << offset; + return new LargeTableSet(tables); + } + + public int popcount() { + int count = 0; + for (Long t : tables) { + count += Long.bitCount(t); + } + return count; + } + + public LargeTableSet merge(LargeTableSet other) { + long[] smallTables = null; + long[] largeTables = null; + int m = 0; + if (this.tables.length >= other.tables.length) { + smallTables = other.tables; + largeTables = this.tables; + } else { + smallTables = this.tables; + largeTables = other.tables; + } + + long[] merged = new long[largeTables.length]; + + while (m < smallTables.length) { + merged[m] = smallTables[m] | largeTables[m]; + m++; + } + while (m < largeTables.length) { + merged[m] = largeTables[m]; + m++; + } + return new LargeTableSet(merged); + } + + public LargeTableSet mergeSmall(long small) { + long[] merged = new long[this.tables.length]; + System.arraycopy(tables, 0, merged, 0, this.tables.length); + merged[0] |= small; + return new LargeTableSet(merged); + } + + public void mergeSmallInPlace(Long small) { + this.tables[0] |= small; + } + + public void mergeInPlace(LargeTableSet other) { + if (other.tables.length > this.tables.length) { + long[] merged = new long[other.tables.length]; + System.arraycopy(tables, 0, merged, 0, this.tables.length); + this.tables = merged; + } + + for (int i = 0; i < other.tables.length; i++) { + this.tables[i] |= other.tables[i]; + } + } + + public int tableOffset() { + int offset = 0; + boolean found = false; + + for (int chunk = 0; chunk < tables.length; chunk++) { + long t = tables[chunk]; + if (t != 0) { + if (found || Long.bitCount(t) != 1) { + return -1; + } + offset = chunk * 64 + Long.numberOfTrailingZeros(t); + found = true; + } + } + return offset; + } + + public void forEachTable(Consumer tableSetCallback) { + for (int i = 0; i < tables.length; i++) { + long bitset = tables[i]; + while (bitset != 0) { + long t = bitset & -bitset; + int r = Long.numberOfTrailingZeros(bitset); + tableSetCallback.accept(i * 64 + r); + bitset ^= t; + } + } + } + + public Boolean containsSmall(long small) { + return (small & this.tables[0]) == small; + } + + public Boolean isContainedBy(LargeTableSet b) { + if (this.tables.length > b.tables.length) { + return false; + } + + for (int i = 0; i < this.tables.length; i++) { + long t = this.tables[i]; + if ((t & b.tables[i]) != t) { + return false; + } + } + return true; + } + + public Boolean overlapsSamll(long small) { + return (tables[0] & small) != 0; + } + + public Boolean overlaps(LargeTableSet b) { + int min = Math.min(this.tables.length, b.tables.length); + for (int t = 0; t < min; t++) { + if ((this.tables[t] & b.tables[t]) != 0) { + return true; + } + } + return false; + } + + /** + * Adds the given table index to this largeTableSet. + * + * @param tableidx the index of the table to add + */ + public void add(int tableidx) { + int chunk = tableidx / 64; + int offset = tableidx % 64; + + if (tables.length <= chunk) { + long[] newTables = Arrays.copyOf(tables, chunk + 1); + tables = newTables; + } + + tables[chunk] |= 1L << offset; + } + +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/semantics/Nothing.java b/src/main/java/com/jd/jdbc/planbuilder/semantics/Nothing.java new file mode 100644 index 0000000..85970c9 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/semantics/Nothing.java @@ -0,0 +1,42 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import java.sql.SQLException; + +public class Nothing implements Dependencies { + + public Nothing() { + } + + @Override + public boolean empty() { + return true; + } + + @Override + public Dependency get() throws SQLException { + return new Dependency(); + } + + @Override + public Dependencies merge(Dependencies d, boolean allowMulti) { + return d; + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/semantics/Originable.java b/src/main/java/com/jd/jdbc/planbuilder/semantics/Originable.java new file mode 100644 index 0000000..d4a5470 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/semantics/Originable.java @@ -0,0 +1,34 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import com.jd.jdbc.common.tuple.Triple; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.statement.SQLTableSource; +import java.sql.SQLException; + +public interface Originable { + TableSet tableSetFor(SQLTableSource sqlTableSource) throws SQLException; + + Triple depsForExpr(SQLExpr sqlExpr); + /* + tableSetFor(t *sqlparser.AliasedTableExpr) TableSet + depsForExpr(expr sqlparser.Expr) (direct, recursive TableSet, typ *Type) + */ +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/semantics/RealTable.java b/src/main/java/com/jd/jdbc/planbuilder/semantics/RealTable.java new file mode 100644 index 0000000..62def94 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/semantics/RealTable.java @@ -0,0 +1,188 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import com.jd.jdbc.sqlparser.SQLUtils; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLObject; +import com.jd.jdbc.sqlparser.ast.expr.SQLPropertyExpr; +import com.jd.jdbc.sqlparser.ast.statement.SQLExprTableSource; +import com.jd.jdbc.sqlparser.ast.statement.SQLTableSource; +import com.jd.jdbc.sqlparser.utils.TableNameUtils; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import lombok.Setter; +import vschema.Vschema; + +/** + * RealTable contains the alias table expr and vindex table + */ +@Setter +public class RealTable implements TableInfo { + @Setter + private String dbName; + + @Setter + private String tableName; + + private SQLTableSource astnode; + + private Vschema.Table table; + + private boolean isInfSchema; + + public RealTable(String tableName, SQLTableSource astnode, Vschema.Table table, boolean isInfSchema) { + this.tableName = tableName; + this.astnode = astnode; + this.table = table; + this.isInfSchema = isInfSchema; + } + + /** + * GetColumns implements the TableInfo interface + * + * @return + */ + @Override + public List getColumns() { + return vindexTableToColumnInfo(this.table); + } + + @Override + public SQLExprTableSource name() throws SQLException { + if (astnode instanceof SQLExprTableSource) { + return (SQLExprTableSource) astnode; + } + throw new SQLException(); + } + + @Override + public Vschema.Table getVindexTable() { + return this.table; + } + + @Override + public boolean isInfSchema() { + return isInfSchema; + } + + /** + * Matches implements the TableInfo interface + * + * @param expr + * @return + */ + @Override + public boolean matches(SQLObject expr) throws SQLException { + if (expr instanceof SQLPropertyExpr) { + boolean flag; + String tableName; + String databaseName; + SQLPropertyExpr leftExpr = (SQLPropertyExpr) expr; + SQLExpr owner = leftExpr.getOwner(); + if (owner instanceof SQLPropertyExpr) { + tableName = ((SQLPropertyExpr) owner).getName(); + databaseName = ((SQLPropertyExpr) owner).getOwnernName(); + flag = SQLUtils.nameEquals(databaseName, this.dbName); + String qualifier = TableNameUtils.getQualifier(astnode); + flag = flag && (SQLUtils.nameEquals(tableName, qualifier) || SQLUtils.nameEquals(tableName, this.tableName)); + return flag; + } else { + tableName = leftExpr.getOwnernName(); + String qualifier = TableNameUtils.getQualifier(astnode); + flag = SQLUtils.nameEquals(tableName, qualifier) || SQLUtils.nameEquals(tableName, this.tableName); + return flag; + } + } else { + return false; + } + } + + @Override + public boolean authoritative() { + return this.table != null && this.table.getColumnListAuthoritative(); + } + + @Override + public SQLTableSource getExpr() { + return this.astnode; + } + + /** + * dependencies implements the TableInfo interface + * + * @param colName + * @param org + * @return + */ + @Override + public Dependencies dependencies(String colName, Originable org) throws SQLException { + TableSet ts = org.tableSetFor(this.astnode); + + for (ColumnInfo info : this.getColumns()) { + if (SQLUtils.nameEquals(info.getName(), colName)) { + return Dependencies.createCertain(ts, ts, info.getType()); + } + } + if (this.authoritative()) { + return new Nothing(); + } + return Dependencies.createUncertain(ts, ts); + } + + @Override + public TableSet getTableSet(Originable org) throws SQLException { + return org.tableSetFor(this.astnode); + } + + private List vindexTableToColumnInfo(Vschema.Table tbl) { + if (tbl == null) { + return Collections.EMPTY_LIST; + } + Set nameMap = new HashSet<>(16); + List cols = new ArrayList<>(tbl.getColumnsList().size() + tbl.getColumnVindexesList().size()); + + for (Vschema.Column col : tbl.getColumnsList()) { + ColumnInfo columnInfo = new ColumnInfo(col.getName(), new Type(col.getType())); + cols.add(columnInfo); + nameMap.add(col.getName()); + } + + // If table is authoritative, we do not need ColumnVindexes to help in resolving the unqualified columns. + if (tbl.getColumnListAuthoritative()) { + return cols; + } + // 把vindex的列放入cols后返回 + for (Vschema.ColumnVindex vindex : tbl.getColumnVindexesList()) { + // List columns = new ArrayList<>(vindex.getColumnsList()); + // vindex 都是单列的 + String column = vindex.getColumn(); + if (nameMap.contains(column)) { + continue; + } + cols.add(new ColumnInfo(column)); + nameMap.add(column); + } + return cols; + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/semantics/SQLObjectExpr.java b/src/main/java/com/jd/jdbc/planbuilder/semantics/SQLObjectExpr.java new file mode 100644 index 0000000..e40d84e --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/semantics/SQLObjectExpr.java @@ -0,0 +1,59 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import com.jd.jdbc.sqlparser.ast.SQLObject; +import java.util.Objects; +import java.util.StringJoiner; + +public class SQLObjectExpr { + private SQLObject sqlObject; + + private final int identityHashCode; + + public SQLObjectExpr(SQLObject sqlObject) { + this.sqlObject = sqlObject; + this.identityHashCode = System.identityHashCode(sqlObject); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SQLObjectExpr that = (SQLObjectExpr) o; + return identityHashCode == that.identityHashCode && Objects.equals(sqlObject, that.sqlObject); + } + + @Override + public int hashCode() { + return Objects.hash(sqlObject, identityHashCode); + } + + @Override + public String toString() { + return new StringJoiner(", ", SQLObjectExpr.class.getSimpleName() + "[", "]") + .add("sqlObject=" + sqlObject) + .add("identityHashCode=" + identityHashCode) + .toString(); + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/semantics/SchemaInformation.java b/src/main/java/com/jd/jdbc/planbuilder/semantics/SchemaInformation.java new file mode 100644 index 0000000..6e515ce --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/semantics/SchemaInformation.java @@ -0,0 +1,43 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import com.jd.jdbc.sqlparser.ast.statement.SQLExprTableSource; +import java.sql.SQLException; +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.Setter; +import vschema.Vschema; + +/** + * SchemaInformation is used tp provide table information from Vschema. + */ +public interface SchemaInformation { + + SchemaInformationContext findTableOrVindex(SQLExprTableSource tableSource) throws SQLException; + + @Getter + @AllArgsConstructor + class SchemaInformationContext { + @Setter + private Vschema.Table table; + + private Vschema.ColumnVindex vindex; + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/semantics/Scope.java b/src/main/java/com/jd/jdbc/planbuilder/semantics/Scope.java new file mode 100644 index 0000000..7758d5a --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/semantics/Scope.java @@ -0,0 +1,86 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import com.jd.jdbc.sqlparser.ast.SQLObject; +import com.jd.jdbc.sqlparser.ast.statement.SQLExprTableSource; +import com.jd.jdbc.sqlparser.utils.StringUtils; +import com.jd.jdbc.sqlparser.utils.TableNameUtils; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import lombok.Data; + +@Data +public class Scope { + private Scope parent; + + private SQLObject stmt; + + private List tables; + + private boolean isUnion; + + private Map joinUsing; + + public Scope(Scope parent) { + this.parent = parent; + this.joinUsing = new HashMap<>(16); + this.tables = new ArrayList<>(10); + } + + public void addTable(TableInfo info) throws SQLException { + SQLExprTableSource name = info.name(); + String tblName = name.getAlias(); + if (StringUtils.isEmpty(tblName)) { + tblName = TableNameUtils.getTableSimpleName(name); + } + for (TableInfo table : this.tables) { + SQLExprTableSource tableName = table.name(); + + String simpleName = tableName.getAlias(); + if (StringUtils.isEmpty(simpleName)) { + simpleName = TableNameUtils.getTableSimpleName(tableName); + } + if (Objects.equals(simpleName, tblName)) { + throw new SQLException("Not unique table/alias: '" + tblName + "'" + "==='" + simpleName + "'"); + } + } + this.tables.add(info); + } + + public Map> prepareUsingMap() { + Map> result = new HashMap<>(this.getJoinUsing().size()); + for (Map.Entry entry : this.getJoinUsing().entrySet()) { + TableSet tss = entry.getValue(); + for (TableSet ts : tss.constituents()) { + Map m = result.get(ts); + if (m == null) { + m = new HashMap<>(this.getJoinUsing().size()); + } + m.put(entry.getKey(), tss); + result.put(ts, m); + } + } + return result; + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/semantics/Scoper.java b/src/main/java/com/jd/jdbc/planbuilder/semantics/Scoper.java new file mode 100644 index 0000000..7546cc1 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/semantics/Scoper.java @@ -0,0 +1,293 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import com.google.common.collect.Lists; +import com.jd.jdbc.common.util.CollectionUtils; +import com.jd.jdbc.sqlparser.ASTUtils; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLObject; +import com.jd.jdbc.sqlparser.ast.SQLOrderBy; +import com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOpExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLIntegerExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLLiteralExpr; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectGroupByClause; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectOrderByItem; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectQuery; +import com.jd.jdbc.sqlparser.ast.statement.SQLTableSource; +import com.jd.jdbc.sqlparser.ast.statement.SQLUnionQuery; +import com.jd.jdbc.sqlparser.ast.statement.SQLUpdateStatement; +import com.jd.jdbc.sqlparser.dialect.mysql.ast.statement.MySqlSelectQueryBlock; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import lombok.Getter; +import lombok.Setter; + +/** + * scoper is responsible for figuring out the scoping for the query, + * and keeps the current scope when walking the tree + */ +public class Scoper { + @Getter + private Map rScope; + + @Getter + private Map wScope; + + private List scopes; + + @Setter + private Originable org; + + @Setter + private Binder binder; + + /** + * These scopes are only used for rewriting ORDER BY 1 and GROUP BY 1 + */ + @Getter + private Map specialExprScopes; + + public Scoper() { + this.rScope = new HashMap<>(16); + this.wScope = new HashMap<>(16); + this.specialExprScopes = new HashMap<>(16); + this.scopes = new ArrayList<>(10); + } + + public static boolean validAsMapKey(SQLExpr sqlExpr) { + // todo + return true; + } + + public void down(SQLObject cursor) throws SQLException { + if (cursor instanceof SQLUpdateStatement) { + throw new SQLException(); + } + if (cursor instanceof MySqlSelectQueryBlock) { + Scope currScope = new Scope(this.currentScope()); + this.push(currScope); + + // Needed for order by with Literal to find the Expression. + currScope.setStmt(cursor); + this.rScope.put(cursor, currScope); + this.wScope.put(cursor, new Scope(null)); + return; + } + if (cursor instanceof SQLTableSource) { + if (cursor.getParent() instanceof MySqlSelectQueryBlock) { + // when checking the expressions used in JOIN conditions, special rules apply where the ON expression + // can only see the two tables involved in the JOIN, and no other tables. + // To create this special context, we create a special scope here that is then merged with + // the surrounding scope when we come back out from the JOIN + Scope nScope = new Scope(null); + nScope.setStmt(cursor.getParent()); + this.push(nScope); + } + return; + } + if (cursor instanceof SQLSelectItem) { + if (!(cursor.getParent() instanceof MySqlSelectQueryBlock)) { + return; + } + if (cursor != ((MySqlSelectQueryBlock) cursor.getParent()).getSelectList().get(0)) { + return; + } + List selectItemList = ((MySqlSelectQueryBlock) cursor.getParent()).getSelectList(); + // adding a vTableInfo for each SELECT, so it can be used by GROUP BY, HAVING, ORDER BY + // the vTableInfo we are creating here should not be confused with derived tables' vTableInfo + Scope wScope = this.wScope.get(cursor.getParent()); + if (wScope == null) { + return; + } + wScope.setTables(Lists.newArrayList(Vtable.createVTableInfoForExpressions(selectItemList, this.currentScope().getTables(), this.org))); + return; + } + if (cursor instanceof SQLSelectGroupByClause) { + SQLSelectGroupByClause groupBy = (SQLSelectGroupByClause) cursor; + if (CollectionUtils.isNotEmpty(groupBy.getItems())) { + // groupBy + createSpecialScopePostProjection(cursor.getParent()); + for (SQLExpr expr : ((SQLSelectGroupByClause) cursor).getItems()) { + SQLIntegerExpr lit = keepIntLiteral(expr); + if (lit != null) { + this.specialExprScopes.put(lit, this.currentScope()); + } + } + } + return; + } + if (cursor instanceof SQLBinaryOpExpr) { + // having + if (cursor.getParent() instanceof SQLSelectGroupByClause) { + SQLSelectGroupByClause groupBy = (SQLSelectGroupByClause) cursor.getParent(); + if (groupBy.getHaving() == cursor) { + createSpecialScopePostProjection(cursor.getParent().getParent()); + } + } + return; + } + if (cursor instanceof SQLOrderBy) { + if (!(cursor.getParent() instanceof MySqlSelectQueryBlock || cursor.getParent() instanceof SQLUnionQuery)) { + return; + } + createSpecialScopePostProjection(cursor.getParent()); + for (SQLSelectOrderByItem expr : ((SQLOrderBy) cursor).getItems()) { + SQLIntegerExpr lit = keepIntLiteral(expr.getExpr()); + if (lit != null) { + this.specialExprScopes.put(lit, this.currentScope()); + } + } + } + } + + public void up(SQLObject cursor) throws SQLException { + if (cursor instanceof SQLOrderBy) { + if (cursor.getParent() instanceof MySqlSelectQueryBlock || cursor.getParent() instanceof SQLUnionQuery) { + this.popScope(); + } + return; + } + if (cursor instanceof SQLSelectGroupByClause) { + SQLSelectGroupByClause groupBy = (SQLSelectGroupByClause) cursor; + if (CollectionUtils.isNotEmpty(groupBy.getItems())) { + this.popScope(); + } + return; + } + if (cursor instanceof SQLBinaryOpExpr) { + // having + if (cursor.getParent() instanceof SQLSelectGroupByClause) { + SQLSelectGroupByClause groupBy = (SQLSelectGroupByClause) cursor.getParent(); + if (groupBy.getHaving() == cursor) { + this.popScope(); + } + } + return; + } + if (cursor instanceof MySqlSelectQueryBlock) { + this.popScope(); + return; + } + if (cursor instanceof SQLTableSource) { + if (cursor.getParent() instanceof MySqlSelectQueryBlock) { + Scope curScope = this.currentScope(); + this.popScope(); + Scope earlierScope = this.currentScope(); + if (CollectionUtils.isEmpty(curScope.getTables())) { + return; + } + // copy curScope into the earlierScope + for (TableInfo table : curScope.getTables()) { + earlierScope.addTable(table); + } + } + } + + } + + public Scope currentScope() { + if (scopes.isEmpty()) { + return null; + } + return scopes.get(scopes.size() - 1); + } + + private void push(Scope scope) { + scopes.add(scope); + } + + private void popScope() { + scopes.remove(scopes.size() - 1); + } + + /** + * createSpecialScopePostProjection is used for the special projection in ORDER BY, GROUP BY and HAVING + * + * @param parent + */ + private void createSpecialScopePostProjection(SQLObject parent) throws SQLException { + if (parent instanceof MySqlSelectQueryBlock) { + // In ORDER BY, GROUP BY and HAVING, we can see both the scope in the FROM part of the query, and the SELECT columns created + // so before walking the rest of the tree, we change the scope to match this behaviour + Scope incomingScope = this.currentScope(); + Scope nScope = new Scope(incomingScope); + nScope.setTables(this.wScope.get(parent).getTables()); + nScope.setStmt(incomingScope.getStmt()); + this.push(nScope); + if (this.rScope.get(parent) != incomingScope) { + throw new SQLException("BUG: scope counts did not match"); + } + } + if (parent instanceof SQLUnionQuery) { + Scope nScope = new Scope(null); + nScope.setUnion(true); + VTableInfo tableInfo = null; + List allSelects = this.getAllSelects((SQLUnionQuery) parent); + for (int i = 0; i < allSelects.size(); i++) { + MySqlSelectQueryBlock sel = allSelects.get(i); + if (i == 0) { + nScope.setStmt(sel); + tableInfo = Vtable.createVTableInfoForExpressions(sel.getSelectList(), null /*needed for star expressions*/, this.org); + nScope.getTables().add(tableInfo); + } + VTableInfo thisTableInfo = Vtable.createVTableInfoForExpressions(sel.getSelectList(), null /*needed for star expressions*/, this.org); + if (tableInfo.getCols().size() != thisTableInfo.getCols().size()) { + throw new SQLException("The used SELECT statements have a different number of columns"); + } + for (int j = 0; j < tableInfo.getCols().size(); j++) { + // at this stage, we don't store the actual dependencies, we only store the expressions. + // only later will we walk the expression tree and figure out the deps. so, we need to create a + // composite expression that contains all the expressions in the SELECTs that this UNION consists of + SQLExpr col = tableInfo.getCols().get(j); + tableInfo.getCols().set(j, ASTUtils.andExpressions(col, thisTableInfo.getCols().get(j))); + } + } + this.push(nScope); + } + } + + private SQLIntegerExpr keepIntLiteral(SQLExpr e) { + if (e instanceof SQLBinaryOpExpr) { + if (((SQLBinaryOpExpr) e).getLeft() instanceof SQLIntegerExpr) { + return (SQLIntegerExpr) ((SQLBinaryOpExpr) e).getLeft(); + } + } + if (e instanceof SQLIntegerExpr) { + return (SQLIntegerExpr) e; + } + return null; + } + + public static List getAllSelects(SQLSelectQuery selStmt) { + List sqlSelectItems = new ArrayList<>(); + if (selStmt instanceof MySqlSelectQueryBlock) { + sqlSelectItems.add((MySqlSelectQueryBlock) selStmt); + } + if (selStmt instanceof SQLUnionQuery) { + sqlSelectItems.addAll(getAllSelects(((SQLUnionQuery) selStmt).getLeft())); + sqlSelectItems.addAll(getAllSelects(((SQLUnionQuery) selStmt).getRight())); + } + return sqlSelectItems; + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/semantics/SemTable.java b/src/main/java/com/jd/jdbc/planbuilder/semantics/SemTable.java new file mode 100644 index 0000000..b14397c --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/semantics/SemTable.java @@ -0,0 +1,310 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import com.jd.jdbc.evalengine.TranslationLookup; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLName; +import com.jd.jdbc.sqlparser.ast.SQLObject; +import com.jd.jdbc.sqlparser.ast.statement.SQLExprTableSource; +import com.jd.jdbc.sqlparser.ast.statement.SQLTableSource; +import com.jd.jdbc.sqlparser.dialect.mysql.ast.statement.MySqlSelectQueryBlock; +import com.jd.jdbc.sqltypes.VtType; +import static com.jd.jdbc.vindexes.VschemaConstant.TYPE_REFERENCE; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import lombok.Getter; +import vschema.Vschema; + +@Getter +public class SemTable implements TranslationLookup { + + private List tables; + + /** + * NotSingleRouteErr stores any errors that have to be generated if the query cannot be planned as a single route. + */ + private String notSingleRouteErr; + + /** + * NotUnshardedErr stores any errors that have to be generated if the query is not unsharded. + */ + private String notUnshardedErr; + + /** + * Recursive contains the dependencies from the expression to the actual tables + * in the query (i.e. not including derived tables). If an expression is a column on a derived table, + * this map will contain the accumulated dependencies for the column expression inside the derived table + */ + private ExprDependencies recursive; + + /** + * Direct keeps information about the closest dependency for an expression. + * It does not recurse inside derived tables and the like to find the original dependencies + */ + private ExprDependencies direct; + + private Map exprTypes; + + private Map selectScope; + + private String comments; + + private Map subqueryMap; + + private Map subqueryRef; + + /** + * ColumnEqualities is used to enable transitive closures + * if a == b and b == c then a == c + */ + private Map> columnEqualities; + + // // DefaultCollation is the default collation for this query, which is usually +// // inherited from the connection's default collation. +// Collation collations.ID + private int collation; + + private String warning; + + public SemTable() { + this.recursive = new ExprDependencies(); + this.direct = new ExprDependencies(); + this.columnEqualities = new HashMap<>(16); + } + + public SemTable(ExprDependencies recursive, ExprDependencies direct, Map exprTypes, List tables, + Map selectScope, String notSingleRouteErr, String notUnshardedErr, String warning, + String comments, Map subqueryMap, Map subqueryRef, + Map> columnEqualities, Object collation) { + this.tables = tables; + this.notSingleRouteErr = notSingleRouteErr; + this.notUnshardedErr = notUnshardedErr; + this.recursive = recursive; + this.direct = direct; + this.exprTypes = exprTypes; + this.selectScope = selectScope; + this.comments = comments; + this.subqueryMap = subqueryMap; + this.subqueryRef = subqueryRef; + this.columnEqualities = columnEqualities; + this.warning = warning; + } + + /** + * TableSetFor returns the bitmask for this particular table + * + * @param t + * @return + */ + public TableSet tableSetFor(SQLTableSource t) { + for (int i = 0; i < this.tables.size(); i++) { + if (t == this.tables.get(i).getExpr()) { + return TableSet.singleTableSet(i); + } + } + return new TableSet(); + } + + /** + * RecursiveDeps return the table dependencies of the expression. + * + * @param expr + * @return + */ + public TableSet recursiveDeps(SQLExpr expr) { + return this.recursive.dependencies(expr); + } + + /** + * DirectDeps return the table dependencies of the expression. + * + * @param expr + * @return + */ + public TableSet directDeps(SQLExpr expr) { + return this.direct.dependencies(expr); + } + + // ReplaceTableSetFor replaces the given single TabletSet with the new *sqlparser.AliasedTableExpr + public void replaceTableSetFor(TableSet id, SQLTableSource t) throws SQLException { + if (id.numberOfTables() != 1) { + throw new SQLException("BUG: tablet identifier should represent single table: table number = " + id.numberOfTables()); + } + + int tblOffset = id.tableOffset(); + + if (tblOffset < 0 || tblOffset >= this.tables.size()) { + throw new SQLException("BUG: tablet identifier greater than number of tables: table size = " + this.tables.size()); + } + + TableInfo ti = this.tables.get(tblOffset); + + if (ti instanceof RealTable) { + ((RealTable) ti).setAstnode(t); + } else if (ti instanceof DerivedTable) { + ((DerivedTable) ti).setAstnode(t); + } else { + throw new SQLException("BUG: replacement not expected for " + ti.toString()); + } + } + + public TableInfo tableInfoFor(TableSet id) { + int offset = id.tableOffset(); + if (offset < 0 || offset >= this.tables.size()) { + // Log + return null; + } + return this.tables.get(offset); + + } + + /** + * TableInfoForExpr returns the table info of the table that this expression depends on. + * Careful: this only works for expressions that have a single table dependency + * + * @param expr + * @return + */ + public TableInfo tableInfoForExpr(SQLExpr expr) { + TableSet ts = this.direct.dependencies(expr); + return this.tableInfoFor(ts); + } + + /** + * NeedsWeightString returns true if the given expression needs weight_string to do safe comparisons + * + * @param expr + * @return + */ + public boolean needsWeightString(SQLExpr expr) { + Type typ = this.exprTypes.get(expr); + if (typ == null) { + return true; + } + // typ.Collation == collations.Unknown && !sqltypes.IsNumber(typ.Type) + return false && !VtType.isNumber(typ.getType()); + } + + // AddColumnEquality adds a relation of the given colName to the ColumnEqualities map + public void addColumnEquality(SQLName colName, SQLExpr expr) { + TableSet tableSet = this.direct.dependencies(colName); + ColumnName col = new ColumnName(tableSet, colName.getSimpleName()); + if (!this.columnEqualities.containsKey(col)) { + this.columnEqualities.put(col, new ArrayList<>()); + } + this.columnEqualities.get(col).add(expr); + } + + // GetExprAndEqualities returns a slice containing the given expression, and it's known equalities if any + public List getExprAndEqualities(SQLExpr expr) { + List result = new ArrayList<>(); + result.add(expr); + if (expr instanceof SQLName) { + TableSet table = this.getDirect().dependencies(expr); + ColumnName columnName = new ColumnName(table, ((SQLName) expr).getSimpleName()); + List exprs = this.getColumnEqualities().get(columnName); + if (exprs != null) { + result.addAll(exprs); + } + } + return result; + } + + public List getExprAndEqualities(List exprList) { + List result = new ArrayList<>(exprList); + return result; + } + + @Override + public int columnLookup(SQLName col) throws SQLException { + throw new SQLException("column access not supported here"); + } + + + // CollationForExpr returns the collation name of expressions in the query + @Override + public int collationForExpr(SQLExpr expr) { + Type typ = this.getExprTypes().get(expr); + if (typ == null) { + return 0; + } + return typ.getCollations(); + } + + @Override + public int defaultCollation() { + return this.getCollation(); + } + + public Type typeFor(SQLExpr expr) { + return this.exprTypes.get(expr); + } + + public void copyExprInfo(SQLExpr src, SQLExpr dest) { + Type srcType = this.exprTypes.get(src); + if (srcType != null) { + this.exprTypes.put(dest, srcType); + } + } + + /** + * returns the single keyspace if all tables in the query are in the same, unsharded keyspace + * + * @return + */ + public List singleUnshardedKeyspace() { + List tables = new ArrayList<>(); + for (TableInfo table : this.tables) { + Vschema.Table vindexTable = table.getVindexTable(); + if (vindexTable == null) { + if (table instanceof DerivedTable) { + // derived tables are ok, as long as all real tables are from the same unsharded keyspace + // we check the real tables inside the derived table as well for same unsharded keyspace. + continue; + } + return null; + } + + if (vindexTable.getType() != "") { + // A reference table is not an issue when seeing if a query is going to an unsharded keyspace + if (vindexTable.getType() == TYPE_REFERENCE) { + continue; + } + return null; + } + SQLTableSource tbl = table.getExpr(); + if (!(tbl instanceof SQLExprTableSource)) { + return null; + } + tables.add(vindexTable); + } + return tables; + } + + // CopyDependencies copies the dependencies from one expression into the other + public void copyDependencies(SQLExpr from, SQLExpr to) { + recursive.put(to,recursiveDeps(from)); + direct.put(to,directDeps(from)); + } + +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/semantics/TableCollector.java b/src/main/java/com/jd/jdbc/planbuilder/semantics/TableCollector.java new file mode 100644 index 0000000..2ef0981 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/semantics/TableCollector.java @@ -0,0 +1,153 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import com.jd.jdbc.planbuilder.RoutePlan; +import static com.jd.jdbc.planbuilder.semantics.DerivedTable.createDerivedTableForExpressions; +import com.jd.jdbc.sqlparser.ast.SQLObject; +import com.jd.jdbc.sqlparser.ast.statement.SQLExprTableSource; +import com.jd.jdbc.sqlparser.ast.statement.SQLSubqueryTableSource; +import com.jd.jdbc.sqlparser.ast.statement.SQLTableSource; +import com.jd.jdbc.sqlparser.ast.statement.SQLUnionQuery; +import com.jd.jdbc.sqlparser.dialect.mysql.ast.statement.MySqlSelectQueryBlock; +import com.jd.jdbc.sqlparser.utils.StringUtils; +import com.jd.jdbc.sqlparser.utils.TableNameUtils; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import lombok.Getter; +import lombok.Setter; +import vschema.Vschema; + +/** + * tableCollector is responsible for gathering information about the tables listed in the FROM clause, + * and adding them to the current scope, plus keeping the global list of tables used in the query + */ +public class TableCollector { + @Getter + private List tables; + + private Scoper scoper; + + private SchemaInformation si; + + private String currentDb; + + @Setter + private Originable org; + + public TableCollector(Scoper scoper, SchemaInformation si, String currentDb) { + this.scoper = scoper; + this.si = si; + this.currentDb = currentDb; + this.tables = new ArrayList<>(10); + } + + public void up(SQLObject cursor) throws SQLException { + if (cursor instanceof SQLSubqueryTableSource) { + if (((SQLSubqueryTableSource) cursor).getSelect().getQuery() instanceof SQLUnionQuery) { + throw new SQLException(" unsupport union derived table "); + } else if (((SQLSubqueryTableSource) cursor).getSelect().getQuery() instanceof MySqlSelectQueryBlock) { + MySqlSelectQueryBlock queryBlock = (MySqlSelectQueryBlock) ((SQLSubqueryTableSource) cursor).getSelect().getQueryBlock(); + Scope tables = this.scoper.getWScope().get(queryBlock); + DerivedTable tableInfo = createDerivedTableForExpressions(queryBlock.getSelectList(), null, tables.getTables(), this.org); + tableInfo.checkForDuplicates(); + tableInfo.setAstnode((SQLSubqueryTableSource) cursor); + tableInfo.setTableName(((SQLSubqueryTableSource) cursor).getAlias()); + this.tables.add(tableInfo); + this.scoper.currentScope().addTable(tableInfo); + + } else { + throw new SQLException("[BUG] " + cursor.toString() + " in a derived table"); + } + return; + } + if (cursor instanceof SQLExprTableSource) { + boolean isInfSchema = false; + Vschema.Table tbl = null; + Vschema.ColumnVindex vindex = null; + if (RoutePlan.systemTable(TableNameUtils.getQualifier((SQLExprTableSource) cursor))) { + isInfSchema = true; + } else { + SchemaInformation.SchemaInformationContext context = this.si.findTableOrVindex((SQLExprTableSource) cursor); + if (context != null) { + tbl = context.getTable(); + vindex = context.getVindex(); + } + /* 生成一个VindexTable可以用sql查找用的hash算法计算对应值到哪个分片上 + https://vitess.io/docs/15.0/reference/features/vindexes/#query-vindex-functions*/ + + } + Scope scope = this.scoper.currentScope(); + TableInfo tableInfo = this.createTable((SQLExprTableSource) cursor, tbl, isInfSchema, vindex); + tables.add(tableInfo); + scope.addTable(tableInfo); + return; + } + } + + private TableInfo createTable(SQLExprTableSource tableSource, Vschema.Table tbl, boolean isInfSchema, Vschema.ColumnVindex vindex) throws SQLException { + String alias = TableNameUtils.getAlias(tableSource); + RealTable table = new RealTable(alias, tableSource, tbl, isInfSchema); + if (StringUtils.isEmpty(alias)) { + String dbName = TableNameUtils.getDatabaseName(tableSource); + if (StringUtils.isEmpty(dbName)) { + dbName = currentDb; + } + table.setDbName(dbName); + table.setTableName(TableNameUtils.getTableSimpleName(tableSource)); + } + if (vindex != null) { + return new VindexTable(table, vindex); + } + return table; + } + + /** + * tabletSetFor implements the originable interface, and that is why it lives on the analyser struct. + * The code lives in this file since it is only touching tableCollector data + * + * @param t + * @return + */ + public TableSet tableSetFor(SQLTableSource t) throws SQLException { + for (int i = 0; i < tables.size(); i++) { + TableInfo t2 = tables.get(i); + if (t == t2.getExpr()) { + return TableSet.singleTableSet(i); + } + } + throw new SQLException("unknown table"); + } + + /** + * tableInfoFor returns the table info for the table set. It should contains only single table. + * + * @param id + * @return + */ + public TableInfo tableInfoFor(TableSet id) throws SQLException { + int offset = id.tableOffset(); + if (offset < 0) { + throw new Exception.ErrMultipleTablesException("[BUG] should only be used for single tables"); + } + return tables.get(offset); + } + +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/semantics/TableInfo.java b/src/main/java/com/jd/jdbc/planbuilder/semantics/TableInfo.java new file mode 100644 index 0000000..faf1a77 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/semantics/TableInfo.java @@ -0,0 +1,90 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import com.jd.jdbc.sqlparser.ast.SQLObject; +import com.jd.jdbc.sqlparser.ast.statement.SQLExprTableSource; +import com.jd.jdbc.sqlparser.ast.statement.SQLTableSource; +import java.sql.SQLException; +import java.util.List; +import vschema.Vschema; + +/** + * TableInfo contains information about tables + */ +public interface TableInfo { + + /** + * Name returns the table name + * + * @return + * @throws SQLException + */ + SQLExprTableSource name() throws SQLException; + + /** + * GetVindexTable returns the vschema version of this TableInfo + * + * @return + */ + Vschema.Table getVindexTable(); + + /** + * IsInfSchema returns true if this table is information_schema + * + * @return + */ + boolean isInfSchema(); + + /** + * matches returns true if the provided table name matches this TableInfo + * + * @param name + * @return + */ + boolean matches(SQLObject name) throws SQLException; + + + /** + * authoritative is true if we have exhaustive column information + * + * @return + */ + boolean authoritative(); + + /** + * getExpr returns the AST struct behind this table + * + * @return + */ + SQLTableSource getExpr(); + + /** + * getColumns returns the known column information for this table + * + * @return + */ + List getColumns(); + + Dependencies dependencies(String colNames, Originable org) throws SQLException; + + TableSet getTableSet(Originable org) throws SQLException; +// getExprFor(s string) (sqlparser.Expr, error) +// getTableSet(org originable) TableSet +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/semantics/TableSet.java b/src/main/java/com/jd/jdbc/planbuilder/semantics/TableSet.java new file mode 100644 index 0000000..016a8c8 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/semantics/TableSet.java @@ -0,0 +1,312 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; +import java.util.StringJoiner; +import java.util.function.Consumer; +import lombok.Getter; +import lombok.Setter; + +/** + * TableSet is how a set of tables is expressed. + * Tables get unique bits assigned in the order that they are encountered during semantic analysis. + * This TableSet implementation is optimized for sets of less than 64 tables, but can grow to support an arbitrary + * large amount of tables. + */ +@Getter +@Setter +public class TableSet { + private long small; + + private LargeTableSet large; + + public TableSet(long small, LargeTableSet large) { + this.small = small; + this.large = large; + } + + public TableSet() { + } + + @Override + public TableSet clone() { + return new TableSet(this.small, this.large.clone()); + } + + /** + * SingleTableSet creates a TableSet that contains only the given table + * + * @param tableidx + * @return + */ + public static TableSet singleTableSet(int tableidx) { + if (tableidx < 64) { + long smll = 1L << tableidx; + return new TableSet(smll, null); + } + return new TableSet(0, newLargeTableSet(0L, tableidx)); + } + + /** + * EmptyTableSet creates an empty TableSet + * + * @return + */ + public static TableSet emptyTableSet() { + return new TableSet(0, null); + } + + + /** + * NumberOfTables returns the number of bits set + * + * @return + */ + public int numberOfTables() { + if (this.large == null) { + return Long.bitCount(this.small); + } + return this.large.popcount(); + } + + // Merge creates a TableSet that contains both inputs + public TableSet merge(TableSet other) { + TableSet tableSet = new TableSet(); + if (this.large == null && other.large == null) { + tableSet.setSmall(this.small | other.small); + } else if (this.large == null) { + tableSet.setLarge(other.large.mergeSmall(this.small)); + } else if (other.large == null) { + tableSet.setLarge(this.large.mergeSmall(other.small)); + } else { + tableSet.setLarge(this.large.merge(other.large)); + } + return tableSet; + } + + + /** + * MergeInPlace merges all the tables in `other` into this TableSet + * + * @param other + */ + public void mergeInPlace(TableSet other) { + if (this.large == null && other.large == null) { + this.small |= other.small; + } else if (this.large == null) { + this.large = other.large.mergeSmall(this.small); + } else if (other.large == null) { + this.large.mergeSmallInPlace(other.small); + } else { + this.large.mergeInPlace(other.large); + } + } + + // MergeTableSets merges all the given TableSet into a single one + public static TableSet mergeTableSets(TableSet... tss) { + TableSet res = new TableSet(0, null); + for (TableSet ts : tss) { + res.mergeInPlace(ts); + } + return res; + } + + // ForEachTable calls the given callback with the indices for all tables in this TableSet + private void forEachTable(Consumer tableSetCallback) { + if (this.large == null) { + long bitset = this.small; + while (bitset != 0) { + long t = bitset & -bitset; + tableSetCallback.accept(Long.numberOfTrailingZeros(bitset)); + bitset ^= t; + } + } else { + this.large.forEachTable(tableSetCallback); + } + } + + // Constituents returns a slice with the indices for all tables in this TableSet + public List constituents() { + List result = new ArrayList<>(); + forEachTable(t -> result.add(TableSet.singleTableSet((Integer) t))); + return result; + } + + /** + * returns true if `ts` and `other` contain the same tables + */ + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TableSet other = (TableSet) o; + return this.isSolvedBy(other) && other.isSolvedBy(this); + } + + @Override + public int hashCode() { + if (large != null) { + return Objects.hash(small, large.hashCode()); + } else { + return Objects.hash(small); + } + } + + @Override + public String toString() { + StringJoiner res = new StringJoiner(",", TableSet.class.getSimpleName() + "{", "}"); + List list = new ArrayList<>(); + forEachTable(t -> list.add((Integer) t)); + for (Integer s : list) { + res.add(s.toString()); + } + return res.toString(); + } + + /** + * TableOffset returns the offset in the Tables array from TableSet + * + * @return + */ + public int tableOffset() { + if (this.large == null) { + if (Long.bitCount(this.small) != 1) { + return -1; + } + return Long.numberOfTrailingZeros(this.small); + } + return this.large.tableOffset(); + } + + /** + * IsSolvedBy returns true if all of `ts` is contained in `other` + * + * @return + */ + public Boolean isSolvedBy(TableSet other) { + if (this.large == null && other.large == null) { + return (this.small & other.small) == this.small; + } else if (this.large == null) { + return other.large.containsSmall(this.small); + } else if (other.large == null) { + return false; + } else { + return this.large.isContainedBy(other.large); + } + } + + /** + * IsOverlapping returns true if at least one table exists in both sets + * + * @param other + * @return + */ + public Boolean isOverlapping(TableSet other) { + if (this.large == null && other.large == null) { + return (this.small & other.small) != 0; + } else if (this.large == null) { + return other.large.overlapsSamll(this.small); + } else if (other.large == null) { + return this.large.overlapsSamll(other.small); + } else { + return this.large.overlaps(other.large); + } + } + + /** + * AddTable adds the given table to this set + * + * @param tableidx + */ + public void addTable(int tableidx) { + if (large == null && tableidx < 64) { + small |= 1L << tableidx; + } else if (large == null) { + large = newLargeTableSet(small, tableidx); + } else { + large.add(tableidx); + } + } + + /** + * Creates a new largeTableSet with the given small table set and table index. + * + * @param small the small table set + * @param tableidx the index of the table to add + * @return the new largeTableSet + */ + private static LargeTableSet newLargeTableSet(long small, int tableidx) { + int chunk = tableidx / 64; + int offset = tableidx % 64; + + long[] tables = new long[chunk + 1]; + tables[0] = small; + tables[chunk] |= 1L << offset; + + return new LargeTableSet(tables); + } + + /** + * // KeepOnly removes all the tables not in `other` from this TableSet + * @param other + */ + public void keepOnly(TableSet other) { + if (this.large == null && other.large == null) { + this.small &= other.small; + } else if (this.large == null) { + this.small &= other.large.getTables()[0]; + } else if (other.large == null) { + this.small = this.large.getTables()[0] & other.small; + this.large = null; + } else { + for (int i = 0; i < this.large.getTables().length; i++) { + if (i >= other.large.getTables().length) { + this.large.setTables(Arrays.copyOf(this.large.getTables(), i)); + break; + } + this.large.getTables()[i] &= other.large.getTables()[i]; + } + } + } + + // RemoveInPlace removes all the tables in `other` from this TableSet + public void removeInPlace(TableSet other) { + if (large == null && other.large == null) { + small &= ~other.small; + } else if (large == null) { + for (int i = 0; i < other.large.getTables().length; i++) { + small &= ~other.large.getTables()[i]; + } + } else if (other.large == null) { + large.getTables()[0] &= ~other.small; + } else { + for (int i = 0; i < Math.min(large.getTables().length, other.large.getTables().length); i++) { + large.getTables()[i] &= ~other.large.getTables()[i]; + } + } + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/semantics/Type.java b/src/main/java/com/jd/jdbc/planbuilder/semantics/Type.java new file mode 100644 index 0000000..3c803da --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/semantics/Type.java @@ -0,0 +1,37 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import io.vitess.proto.Query; +import lombok.Getter; + +/** + * Type is the normal querypb.Type with collation + */ +@Getter +public class Type { + + private Query.Type type; + + private int collations; + + public Type(Query.Type type) { + this.type = type; + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/semantics/Typer.java b/src/main/java/com/jd/jdbc/planbuilder/semantics/Typer.java new file mode 100644 index 0000000..20d41ca --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/semantics/Typer.java @@ -0,0 +1,82 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import com.jd.jdbc.engine.Engine; +import com.jd.jdbc.engine.gen4.AbstractAggregateGen4; +import com.jd.jdbc.sqlparser.ast.SQLObject; +import com.jd.jdbc.sqlparser.ast.expr.SQLAggregateExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLCharExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLIntegerExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLNumberExpr; +import io.vitess.proto.Query; +import java.math.BigDecimal; +import java.util.HashMap; +import java.util.Map; +import lombok.Getter; + +/** + * typer is responsible for setting the type for expressions + * it does it's work after visiting the children (up), since the children types is often needed to type a node. + */ +public class Typer { + private static final Type TYPE_INT_32 = new Type(Query.Type.INT32); + + private static final Type DECIMAL = new Type(Query.Type.DECIMAL); + + private static final Type FLOAT_VAL = new Type(Query.Type.FLOAT64); + + @Getter + private Map exprTypes; + + public Typer() { + this.exprTypes = new HashMap<>(16); + } + + public void up(SQLObject cursor) { + if (cursor instanceof SQLIntegerExpr) { + this.exprTypes.put(cursor, TYPE_INT_32); + } + if (cursor instanceof SQLCharExpr) { + // TODO - add system default collation name + this.exprTypes.put(cursor, new Type(Query.Type.VARCHAR)); + } + if (cursor instanceof SQLNumberExpr) { + SQLNumberExpr numberExpr = (SQLNumberExpr) cursor; + if (numberExpr.getNumber() instanceof BigDecimal) { + this.exprTypes.put(cursor, DECIMAL); + } + } +// case sqlparser.FloatVal: +// t.exprTypes[node] = floatval + if (cursor instanceof SQLAggregateExpr) { + Engine.AggregateOpcodeG4 aggregateOpcode = AbstractAggregateGen4.SUPPORTED_AGGREGATES.get(((SQLAggregateExpr) cursor).getMethodName().toLowerCase()); + if (aggregateOpcode != null) { + Query.Type type = AbstractAggregateGen4.OPCODE_TYPE.get(aggregateOpcode); + if (type != null) { + this.exprTypes.put(cursor, new Type(type)); + } + } + } + } + + public void setTypeFor(SQLObject node, Type typ) { + exprTypes.put(node, typ); + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/semantics/Uncertain.java b/src/main/java/com/jd/jdbc/planbuilder/semantics/Uncertain.java new file mode 100644 index 0000000..ed6105e --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/semantics/Uncertain.java @@ -0,0 +1,60 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import java.sql.SQLException; +import lombok.Getter; + +public class Uncertain extends Dependency implements Dependencies { + @Getter + private boolean fail; + + public Uncertain(TableSet direct, TableSet recursive) { + super(direct, recursive); + this.fail = false; + } + + @Override + public boolean empty() { + return false; + } + + @Override + public Dependency get() throws SQLException { + if (fail) { + throw new Exception.AmbiguousException("ambiguous"); + } + return this; + } + + @Override + public Dependencies merge(Dependencies d, boolean allowMulti) { + if (d instanceof Uncertain) { + // 这里是用!=还是equals + if (((Uncertain) d).getRecursive() != super.getRecursive()) { + this.fail = true; + } + return this; + } + if (d instanceof Certain) { + return d; + } + return this; + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/semantics/VSchema.java b/src/main/java/com/jd/jdbc/planbuilder/semantics/VSchema.java new file mode 100644 index 0000000..7836ebe --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/semantics/VSchema.java @@ -0,0 +1,69 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import com.jd.jdbc.VSchemaManager; +import com.jd.jdbc.sqlparser.ast.statement.SQLExprTableSource; +import com.jd.jdbc.sqlparser.utils.TableNameUtils; +import com.jd.jdbc.vindexes.VKeyspace; +import java.sql.SQLException; +import lombok.Getter; +import vschema.Vschema; + +/** + * VSchema defines the interface for this package to fetch + * info about tables. + */ +@Getter +public class VSchema implements SchemaInformation { + + private String defaultKeyspace; + + private VSchemaManager vSchemaManager; + + public VSchema(String defaultKeyspace, VSchemaManager vSchemaManager) { + this.defaultKeyspace = defaultKeyspace; + this.vSchemaManager = vSchemaManager; + } + + /** + * PlannerWarning records warning created during planning. + * + * @return + */ + public void plannerWarning(String message) { + return; + } + + @Override + public SchemaInformationContext findTableOrVindex(SQLExprTableSource tableSource) throws SQLException { + Vschema.Keyspace keyspace = vSchemaManager.getKeyspace(defaultKeyspace); + Vschema.Table table = vSchemaManager.getTable(defaultKeyspace, TableNameUtils.getTableSimpleName(tableSource)); + return new SchemaInformationContext(table, null); + } + + public VKeyspace anyKeyspace() throws SQLException { + return getVschemaKeyspace(); + } + + public VKeyspace getVschemaKeyspace() throws SQLException { + Vschema.Keyspace ks = vSchemaManager.getKeyspace(defaultKeyspace); + return new VKeyspace(this.defaultKeyspace, ks.getSharded()); + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/semantics/VTableInfo.java b/src/main/java/com/jd/jdbc/planbuilder/semantics/VTableInfo.java new file mode 100644 index 0000000..39315be --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/semantics/VTableInfo.java @@ -0,0 +1,122 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import com.jd.jdbc.common.tuple.Triple; +import com.jd.jdbc.sqlparser.SQLUtils; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLObject; +import com.jd.jdbc.sqlparser.ast.statement.SQLExprTableSource; +import com.jd.jdbc.sqlparser.ast.statement.SQLTableSource; +import java.sql.SQLException; +import java.util.List; +import lombok.Getter; +import vschema.Vschema; + +/** + * vTableInfo is used to represent projected results, not real tables. It is used for + * ORDER BY, GROUP BY and HAVING that need to access result columns + */ +public class VTableInfo implements TableInfo { + + private String tableName; + + private List columnNames; + + @Getter + private List cols; + + private TableSet tables; + + public VTableInfo(List columnNames, List cols, TableSet tables) { + this.columnNames = columnNames; + this.cols = cols; + this.tables = tables; + } + + @Override + public SQLExprTableSource name() throws SQLException { + throw new SQLException("oh noes"); + } + + @Override + public Vschema.Table getVindexTable() { + return null; + } + + @Override + public boolean isInfSchema() { + return false; + } + + @Override + public boolean matches(SQLObject name) { + return false; + } + + @Override + public boolean authoritative() { + return false; + } + + @Override + public SQLTableSource getExpr() { + return null; + } + + @Override + public List getColumns() { + return null; + } + + /** + * dependencies implements the TableInfo interface + * + * @param colName + * @param org + * @return + */ + @Override + public Dependencies dependencies(String colName, Originable org) { + Dependencies deps = new Nothing(); + for (int i = 0; i < this.columnNames.size(); i++) { + String name = this.columnNames.get(i); + if (!SQLUtils.nameEquals(colName, name)) { + continue; + } + Triple triple = org.depsForExpr(this.cols.get(i)); + + Certain newDeps = Dependencies.createCertain(triple.getLeft(), triple.getMiddle(), triple.getRight()); + deps = deps.merge(newDeps, false); + } + if (deps.empty() && this.hasStar()) { + return Dependencies.createUncertain(this.tables, this.tables); + } + return deps; + } + + @Override + public TableSet getTableSet(Originable org) { + return null; + } + + private boolean hasStar() { + return this.tables.numberOfTables() > 0; + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/semantics/VindexTable.java b/src/main/java/com/jd/jdbc/planbuilder/semantics/VindexTable.java new file mode 100644 index 0000000..a82b24e --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/semantics/VindexTable.java @@ -0,0 +1,90 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import com.jd.jdbc.sqlparser.ast.SQLObject; +import com.jd.jdbc.sqlparser.ast.statement.SQLExprTableSource; +import com.jd.jdbc.sqlparser.ast.statement.SQLTableSource; +import java.sql.SQLException; +import java.util.List; +import lombok.Getter; +import vschema.Vschema; + +/** + * VindexTable contains a vindexes.Vindex and a TableInfo. The former represents the vindex + * we are keeping information about, and the latter represents the additional table information + * (usually a RealTable or an AliasedTable) of our vindex. + */ + +@Getter +public class VindexTable implements TableInfo { + private TableInfo table; + + private Vschema.ColumnVindex vindex; + + public VindexTable(TableInfo table, Vschema.ColumnVindex vindex) { + this.table = table; + this.vindex = vindex; + } + + @Override + public SQLExprTableSource name() throws SQLException { + return table.name(); + } + + @Override + public Vschema.Table getVindexTable() { + return null; + } + + @Override + public boolean isInfSchema() { + return false; + } + + @Override + public boolean matches(SQLObject name) { + return false; + } + + @Override + public boolean authoritative() { + return false; + } + + @Override + public SQLTableSource getExpr() { + return null; + } + + @Override + public List getColumns() { + return null; + } + + @Override + public Dependencies dependencies(String colNames, Originable org) throws SQLException { + return null; + } + + @Override + public TableSet getTableSet(Originable org) { + return null; + } +} diff --git a/src/main/java/com/jd/jdbc/planbuilder/semantics/Vtable.java b/src/main/java/com/jd/jdbc/planbuilder/semantics/Vtable.java new file mode 100644 index 0000000..5692e14 --- /dev/null +++ b/src/main/java/com/jd/jdbc/planbuilder/semantics/Vtable.java @@ -0,0 +1,66 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import com.jd.jdbc.sqlparser.SqlParser; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLName; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import com.jd.jdbc.sqlparser.utils.StringUtils; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; + +public class Vtable { + + public static VTableInfo createVTableInfoForExpressions(List expressions, List tables, Originable org) throws SQLException { + return selectExprsToInfos(expressions, tables, org); + } + + private static VTableInfo selectExprsToInfos(List expressions, List tables, Originable org) throws SQLException { + List cols = new ArrayList<>(expressions.size()); + List columnNames = new ArrayList<>(expressions.size()); + TableSet ts = new TableSet(); + + for (SQLSelectItem selectItem : expressions) { + SQLExpr expr = selectItem.getExpr(); + cols.add(expr); + if (StringUtils.isNotEmpty(selectItem.getAlias())) { + columnNames.add(selectItem.getAlias()); + continue; + } + + SqlParser.SelectExpr selectExpr = SqlParser.SelectExpr.type(selectItem); + if (SqlParser.SelectExpr.AliasedExpr.equals(selectExpr)) { + if (expr instanceof SQLName) { + columnNames.add(((SQLName) expr).getSimpleName()); + } else { + columnNames.add(expr.toString()); + } + } else if (SqlParser.SelectExpr.StarExpr.equals(selectExpr)) { + for (TableInfo table : tables) { + ts.mergeInPlace(table.getTableSet(org)); + } + } else { + throw new RuntimeException(); + } + } + return new VTableInfo(columnNames, cols, ts); + } +} diff --git a/src/main/java/com/jd/jdbc/sqlparser/ASTUtils.java b/src/main/java/com/jd/jdbc/sqlparser/ASTUtils.java new file mode 100644 index 0000000..3f5f3bd --- /dev/null +++ b/src/main/java/com/jd/jdbc/sqlparser/ASTUtils.java @@ -0,0 +1,104 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.sqlparser; + +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLObject; +import com.jd.jdbc.sqlparser.ast.SQLOrderBy; +import com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOpExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOperator; +import com.jd.jdbc.sqlparser.ast.expr.SQLExprUtils; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectGroupByClause; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectQuery; + +public class ASTUtils { + private ASTUtils() { + } + + public static SQLObject getOrderByGroupByParent(SQLObject cursor) { + SQLObject result = cursor; + if (result instanceof SQLSelectQuery) { + return null; + } + if (result != null && !(result instanceof SQLOrderBy || result instanceof SQLSelectGroupByClause)) { + result = getOrderByGroupByParent(result.getParent()); + } + return result; + } + + public static SQLOrderBy getOrderByParent(SQLObject cursor) { + SQLObject result = cursor; + if (result instanceof SQLSelectQuery || result instanceof SQLSelectGroupByClause) { + return null; + } + if (result != null && !(result instanceof SQLOrderBy)) { + result = getOrderByParent(result.getParent()); + } + return result != null ? (SQLOrderBy) result : null; + } + + public static SQLSelectGroupByClause getGroupByParent(SQLObject cursor) { + SQLObject result = cursor; + if (result instanceof SQLSelectQuery || result instanceof SQLOrderBy) { + return null; + } + if (result != null && !(result instanceof SQLSelectGroupByClause)) { + result = getGroupByParent(result.getParent()); + } + return result != null ? (SQLSelectGroupByClause) result : null; + } + + + /** + * AndExpressions ands together two or more expressions, minimising the expr when possible + */ + public static SQLExpr andExpressions(SQLExpr... exprs) { + switch (exprs.length) { + case 0: + return null; + case 1: + return exprs[0]; + default: + SQLExpr result = null; + // we'll loop and remove any duplicates + for (int i = 0; i < exprs.length; i++) { + SQLExpr expr = exprs[i]; + if (expr == null) { + continue; + } + if (result == null) { + result = expr; + continue; + } + boolean continueFlag = false; + for (int j = 0; j < i; j++) { + if (SQLExprUtils.equals(expr, exprs[j])) { + continueFlag = true; + continue; + } + } + if (continueFlag) { + continue; + } + result = new SQLBinaryOpExpr(result, SQLBinaryOperator.BooleanAnd, expr); + } + return result; + } + } +} diff --git a/src/main/java/com/jd/jdbc/sqlparser/ast/expr/SQLAggregateExpr.java b/src/main/java/com/jd/jdbc/sqlparser/ast/expr/SQLAggregateExpr.java index db4ce12..5982318 100644 --- a/src/main/java/com/jd/jdbc/sqlparser/ast/expr/SQLAggregateExpr.java +++ b/src/main/java/com/jd/jdbc/sqlparser/ast/expr/SQLAggregateExpr.java @@ -20,10 +20,10 @@ import com.jd.jdbc.sqlparser.ast.*; import com.jd.jdbc.sqlparser.utils.FnvHash; import com.jd.jdbc.sqlparser.visitor.SQLASTVisitor; - import java.io.Serializable; import java.util.ArrayList; import java.util.List; +import java.util.Map; public class SQLAggregateExpr extends SQLExprImpl implements Serializable, SQLReplaceable { @@ -242,6 +242,16 @@ public SQLAggregateExpr clone() { x.ignoreNulls = ignoreNulls; + if (attributes != null) { + for (Map.Entry entry : attributes.entrySet()) { + String key = entry.getKey(); + Object value = entry.getValue(); + if (value instanceof SQLObject) { + value = ((SQLObject) value).clone(); + } + x.putAttribute(key, value); + } + } return x; } diff --git a/src/main/java/com/jd/jdbc/sqlparser/ast/expr/VtOffset.java b/src/main/java/com/jd/jdbc/sqlparser/ast/expr/VtOffset.java new file mode 100644 index 0000000..4cedf81 --- /dev/null +++ b/src/main/java/com/jd/jdbc/sqlparser/ast/expr/VtOffset.java @@ -0,0 +1,87 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.sqlparser.ast.expr; + +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLExprImpl; +import com.jd.jdbc.sqlparser.dialect.mysql.visitor.MySqlOutputVisitor; +import com.jd.jdbc.sqlparser.utils.StringUtils; +import com.jd.jdbc.sqlparser.visitor.SQLASTVisitor; +import lombok.Getter; +import lombok.Setter; + +@Getter +@Setter +public class VtOffset extends SQLExprImpl { + + private int value; + + private String original = ""; + + public VtOffset() { + } + + public VtOffset(int value, String origin) { + this.value = value; + this.original = origin; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof VtOffset)) { + return false; + } + + VtOffset that = (VtOffset) o; + + return (this.value == that.value) && (this.original.equals(that.original)); + } + + @Override + public int hashCode() { + return Integer.hashCode(value) + original.hashCode(); + } + + @Override + protected void accept0(SQLASTVisitor visitor) { + if (visitor instanceof MySqlOutputVisitor) { + MySqlOutputVisitor mySqlOutputVisitor = (MySqlOutputVisitor) visitor; + mySqlOutputVisitor.print(this.toString()); + } + } + + @Override + public SQLExpr clone() { + VtOffset x = new VtOffset(value, original); + x.setParent(this.parent); + return x; + } + + @Override + public String toString() { + if (StringUtils.isEmpty(this.original)) { + return "OFFSET(" + value + ")"; + } else { + return "OFFSET(" + value + ", '" + original + "')"; + } + } +} diff --git a/src/main/java/com/jd/jdbc/sqlparser/dialect/mysql/visitor/RewriteHavingAggrWithOffsetVisitor.java b/src/main/java/com/jd/jdbc/sqlparser/dialect/mysql/visitor/RewriteHavingAggrWithOffsetVisitor.java new file mode 100644 index 0000000..7ac8ce5 --- /dev/null +++ b/src/main/java/com/jd/jdbc/sqlparser/dialect/mysql/visitor/RewriteHavingAggrWithOffsetVisitor.java @@ -0,0 +1,69 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.sqlparser.dialect.mysql.visitor; + +import com.jd.jdbc.planbuilder.gen4.QueryProjection; +import com.jd.jdbc.sqlparser.SQLUtils; +import com.jd.jdbc.sqlparser.ast.expr.SQLAggregateExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLExprUtils; +import com.jd.jdbc.sqlparser.ast.expr.VtOffset; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import java.sql.SQLException; + +public class RewriteHavingAggrWithOffsetVisitor extends MySqlASTVisitorAdapter { + + private final QueryProjection queryProjection; + + private SQLException err; + + public RewriteHavingAggrWithOffsetVisitor(QueryProjection qp) { + this.queryProjection = qp; + } + + @Override + public boolean visit(final SQLAggregateExpr x) { + if (err != null) { + return false; + } + for (int offset = 0; offset < queryProjection.getSelectExprs().size(); offset++) { + QueryProjection.SelectExpr expr = queryProjection.getSelectExprs().get(offset); + SQLSelectItem ae; + try { + ae = expr.getAliasedExpr(); + } catch (SQLException exception) { + this.err = exception; + return false; + } + if (SQLExprUtils.equals(ae.getExpr(), x)) { + VtOffset vtOffset = new VtOffset(offset, x.toString()); + SQLUtils.replaceInParent(x, vtOffset); + return false; + } + } + + QueryProjection.SelectExpr col = new QueryProjection.SelectExpr(true, new SQLSelectItem(x.clone())); + queryProjection.setHasAggr(true); + + VtOffset vtOffset = new VtOffset(queryProjection.getSelectExprs().size(), x.toString()); + SQLUtils.replaceInParent(x, vtOffset); + queryProjection.getSelectExprs().add(col); + queryProjection.setAddedColumn(queryProjection.getAddedColumn() + 1); + return true; + } +} diff --git a/src/main/java/com/jd/jdbc/sqlparser/dialect/mysql/visitor/VtFormatImpossibleQueryVisitor.java b/src/main/java/com/jd/jdbc/sqlparser/dialect/mysql/visitor/VtFormatImpossibleQueryVisitor.java new file mode 100644 index 0000000..51c6c9e --- /dev/null +++ b/src/main/java/com/jd/jdbc/sqlparser/dialect/mysql/visitor/VtFormatImpossibleQueryVisitor.java @@ -0,0 +1,66 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.sqlparser.dialect.mysql.visitor; + +import com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOpExpr; +import static com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOperator.NotEqual; +import com.jd.jdbc.sqlparser.ast.expr.SQLIntegerExpr; +import com.jd.jdbc.sqlparser.dialect.mysql.ast.statement.MySqlSelectQueryBlock; + +/** +* FormatImpossibleQuery creates an impossible query in a TrackedBuffer. +* An impossible query is a modified version of a query where all selects have where clauses that are +* impossible for mysql to resolve. This is used in the vtgate and vttablet: +* +* - In the vtgate it's used for joins: if the first query returns no result, then vtgate uses the impossible +* query just to fetch field info from vttablet +* - In the vttablet, it's just an optimization: the field info is fetched once form MySQL, cached and reused +* for subsequent queries + */ +public class VtFormatImpossibleQueryVisitor extends MySqlASTVisitorAdapter { + + public VtFormatImpossibleQueryVisitor() { + } + + @Override + public boolean visit(final MySqlSelectQueryBlock x) { + // where 1 != 1 + x.setWhere(new SQLBinaryOpExpr(new SQLIntegerExpr(1), + NotEqual, + new SQLIntegerExpr(1))); + // remove limit + x.setLimit(null); + + // remove distinct + x.setDistionOption(0); + + // remove order by + x.setOrderBy(null); + + // remove hint + x.setHints(null); + + // remove having + if (x.getGroupBy() != null) { + x.getGroupBy().setHaving(null); + } + + return true; + } +} diff --git a/src/main/java/com/jd/jdbc/sqlparser/dialect/mysql/visitor/VtHasSubqueryVisitor.java b/src/main/java/com/jd/jdbc/sqlparser/dialect/mysql/visitor/VtHasSubqueryVisitor.java index 0b8d322..cd2dacd 100644 --- a/src/main/java/com/jd/jdbc/sqlparser/dialect/mysql/visitor/VtHasSubqueryVisitor.java +++ b/src/main/java/com/jd/jdbc/sqlparser/dialect/mysql/visitor/VtHasSubqueryVisitor.java @@ -18,6 +18,7 @@ import com.jd.jdbc.sqlparser.ast.expr.SQLInSubQueryExpr; import com.jd.jdbc.sqlparser.ast.expr.SQLQueryExpr; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelect; import com.jd.jdbc.sqlparser.ast.statement.SQLSubqueryTableSource; import lombok.Getter; @@ -38,6 +39,12 @@ public boolean visit(final SQLQueryExpr x) { return false; } + @Override + public boolean visit(final SQLSelect x) { + this.hasSubquery = Boolean.TRUE; + return false; + } + @Override public boolean visit(final SQLInSubQueryExpr x) { this.hasSubquery = Boolean.TRUE; diff --git a/src/main/java/com/jd/jdbc/sqlparser/dialect/mysql/visitor/VtRemoveDbNameExpectSystemDbVisitor.java b/src/main/java/com/jd/jdbc/sqlparser/dialect/mysql/visitor/VtRemoveDbNameExpectSystemDbVisitor.java new file mode 100644 index 0000000..7e260d0 --- /dev/null +++ b/src/main/java/com/jd/jdbc/sqlparser/dialect/mysql/visitor/VtRemoveDbNameExpectSystemDbVisitor.java @@ -0,0 +1,41 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.sqlparser.dialect.mysql.visitor; + +import com.jd.jdbc.planbuilder.RoutePlan; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLPropertyExpr; +import com.jd.jdbc.sqlparser.ast.statement.SQLExprTableSource; + +public class VtRemoveDbNameExpectSystemDbVisitor extends MySqlASTVisitorAdapter { + + @Override + public boolean visit(final SQLExprTableSource x) { + SQLExpr tableExpr = x.getExpr(); + if (tableExpr instanceof SQLPropertyExpr) { + String tableName = ((SQLPropertyExpr) tableExpr).getName(); + String keySpace = ((SQLPropertyExpr) tableExpr).getOwnernName(); + if (RoutePlan.systemTable(keySpace)) { + return false; + } + x.setExpr(tableName); + } + return false; + } +} diff --git a/src/main/java/com/jd/jdbc/sqlparser/dialect/mysql/visitor/VtRemoveDbNameInColumnVisitor.java b/src/main/java/com/jd/jdbc/sqlparser/dialect/mysql/visitor/VtRemoveDbNameInColumnVisitor.java new file mode 100644 index 0000000..2171e00 --- /dev/null +++ b/src/main/java/com/jd/jdbc/sqlparser/dialect/mysql/visitor/VtRemoveDbNameInColumnVisitor.java @@ -0,0 +1,42 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.sqlparser.dialect.mysql.visitor; + +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLIdentifierExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLPropertyExpr; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; + +public class VtRemoveDbNameInColumnVisitor extends MySqlASTVisitorAdapter { + + @Override + public boolean visit(final SQLSelectItem x) { + SQLExpr selectExpr = x.getExpr(); + if (selectExpr instanceof SQLPropertyExpr) { + SQLExpr owner = ((SQLPropertyExpr) selectExpr).getOwner(); + if (owner instanceof SQLPropertyExpr) { + SQLPropertyExpr newSelectExpr = new SQLPropertyExpr( + new SQLIdentifierExpr(((SQLPropertyExpr) owner).getName()), + ((SQLPropertyExpr) selectExpr).getName()); + x.setExpr(newSelectExpr); + } + } + return false; + } +} diff --git a/src/main/java/com/jd/jdbc/sqlparser/dialect/mysql/visitor/VtRemoveDbNameVisitor.java b/src/main/java/com/jd/jdbc/sqlparser/dialect/mysql/visitor/VtRemoveDbNameVisitor.java index c963d04..b529e19 100644 --- a/src/main/java/com/jd/jdbc/sqlparser/dialect/mysql/visitor/VtRemoveDbNameVisitor.java +++ b/src/main/java/com/jd/jdbc/sqlparser/dialect/mysql/visitor/VtRemoveDbNameVisitor.java @@ -17,6 +17,7 @@ package com.jd.jdbc.sqlparser.dialect.mysql.visitor; import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLIdentifierExpr; import com.jd.jdbc.sqlparser.ast.expr.SQLPropertyExpr; import com.jd.jdbc.sqlparser.ast.statement.SQLExprTableSource; import com.jd.jdbc.sqlparser.ast.statement.SQLTableSource; @@ -50,4 +51,14 @@ public boolean visit(final SQLExprTableSource x) { } return false; } + + @Override + public boolean visit(final SQLPropertyExpr x) { + SQLExpr owner = x.getOwner(); + if (owner instanceof SQLPropertyExpr) { + x.setOwner(new SQLIdentifierExpr(((SQLPropertyExpr) owner).getName())); + } + return false; + } + } diff --git a/src/main/java/com/jd/jdbc/sqlparser/dialect/mysql/visitor/VtRouteWireupFixUpAstVisitor.java b/src/main/java/com/jd/jdbc/sqlparser/dialect/mysql/visitor/VtRouteWireupFixUpAstVisitor.java index 894a4a7..c0311b4 100644 --- a/src/main/java/com/jd/jdbc/sqlparser/dialect/mysql/visitor/VtRouteWireupFixUpAstVisitor.java +++ b/src/main/java/com/jd/jdbc/sqlparser/dialect/mysql/visitor/VtRouteWireupFixUpAstVisitor.java @@ -17,6 +17,7 @@ package com.jd.jdbc.sqlparser.dialect.mysql.visitor; import com.jd.jdbc.planbuilder.RoutePlan; +import com.jd.jdbc.sqlparser.SqlParser; import com.jd.jdbc.sqlparser.ast.SQLExpr; import com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOpExpr; import com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOperator; @@ -29,6 +30,13 @@ public class VtRouteWireupFixUpAstVisitor extends MySqlASTVisitorAdapter { private final RoutePlan rb; + private boolean exprIsValue(SQLExpr expr) { + if (this.rb != null) { + return this.rb.exprIsValue(expr); + } + return SqlParser.isValue(expr); + } + public VtRouteWireupFixUpAstVisitor(final RoutePlan rb) { this.rb = rb; } @@ -47,7 +55,7 @@ public boolean visit(final SQLBinaryOpExpr x) { SQLExpr leftValue = x.getLeft(); SQLExpr rightValue = x.getRight(); if (SQLBinaryOperator.Equality.equals(x.getOperator())) { - if (this.rb.exprIsValue(leftValue) && !this.rb.exprIsValue(rightValue)) { + if (exprIsValue(leftValue) && !exprIsValue(rightValue)) { x.setRight(leftValue); x.setLeft(rightValue); } diff --git a/src/main/java/com/jd/jdbc/sqlparser/parser/SQLSelectParser.java b/src/main/java/com/jd/jdbc/sqlparser/parser/SQLSelectParser.java index 5d56c08..ba47ab3 100644 --- a/src/main/java/com/jd/jdbc/sqlparser/parser/SQLSelectParser.java +++ b/src/main/java/com/jd/jdbc/sqlparser/parser/SQLSelectParser.java @@ -134,17 +134,29 @@ public SQLSelectQuery queryRest(SQLSelectQuery selectQuery, boolean acceptUnion) if (right instanceof SQLSelectQueryBlock) { SQLSelectQueryBlock rightQuery = (SQLSelectQueryBlock) right; SQLOrderBy orderBy = rightQuery.getOrderBy(); - SQLLimit limit = rightQuery.getLimit(); - if (orderBy != null && limit == null) { + if (orderBy != null) { union.setOrderBy(orderBy); rightQuery.setOrderBy(null); } + + SQLLimit limit = rightQuery.getLimit(); + if (limit != null) { + union.setLimit(limit); + rightQuery.setLimit(null); + } } else if (right instanceof SQLUnionQuery) { SQLUnionQuery rightUnion = (SQLUnionQuery) right; - if (rightUnion.getOrderBy() != null) { - union.setOrderBy(rightUnion.getOrderBy()); + final SQLOrderBy orderBy = rightUnion.getOrderBy(); + if (orderBy != null) { + union.setOrderBy(orderBy); rightUnion.setOrderBy(null); } + + SQLLimit limit = rightUnion.getLimit(); + if (limit != null) { + union.setLimit(limit); + rightUnion.setLimit(null); + } } } diff --git a/src/main/java/com/jd/jdbc/sqlparser/utils/TableNameUtils.java b/src/main/java/com/jd/jdbc/sqlparser/utils/TableNameUtils.java index ba970e2..2e33cef 100644 --- a/src/main/java/com/jd/jdbc/sqlparser/utils/TableNameUtils.java +++ b/src/main/java/com/jd/jdbc/sqlparser/utils/TableNameUtils.java @@ -16,8 +16,13 @@ package com.jd.jdbc.sqlparser.utils; +import com.jd.jdbc.sqlparser.ast.SQLExpr; import com.jd.jdbc.sqlparser.ast.SQLName; +import com.jd.jdbc.sqlparser.ast.expr.SQLIdentifierExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLPropertyExpr; import com.jd.jdbc.sqlparser.ast.statement.SQLExprTableSource; +import com.jd.jdbc.sqlparser.ast.statement.SQLSubqueryTableSource; +import com.jd.jdbc.sqlparser.ast.statement.SQLTableSource; import io.netty.util.internal.StringUtil; import java.sql.SQLException; @@ -30,4 +35,42 @@ public static String getTableSimpleName(SQLExprTableSource tableSource) throws S } return tableName.getSimpleName(); } + + public static String getQualifier(SQLTableSource tableSource) { + if (tableSource instanceof SQLExprTableSource) { + SQLExpr tableSourceExpr = ((SQLExprTableSource) tableSource).getExpr(); + if (tableSourceExpr instanceof SQLPropertyExpr) { + return ((SQLPropertyExpr) tableSourceExpr).getOwnernName(); + } + return tableSource.getAlias(); + } + if (tableSource instanceof SQLSubqueryTableSource) { + return tableSource.getAlias(); + } + return ""; + } + + public static String getAlias(SQLTableSource tableSource) { + if (tableSource instanceof SQLExprTableSource) { + return tableSource.getAlias(); + } + return tableSource.getAlias(); + } + + public static String getDatabaseName(SQLExprTableSource tableSource) { + SQLExpr tableExpr = tableSource.getExpr(); + if (!(tableExpr instanceof SQLPropertyExpr)) { + return null; + } + + SQLExpr schemaExpr = ((SQLPropertyExpr) tableExpr).getOwner(); + if (schemaExpr instanceof SQLIdentifierExpr) { + String schemaName = ((SQLIdentifierExpr) schemaExpr).getSimpleName(); +// if (RoutePlan.systemTable(schemaName)) { +// return null; +// } + return schemaName; + } + return null; + } } \ No newline at end of file diff --git a/src/main/java/com/jd/jdbc/sqlparser/utils/Utils.java b/src/main/java/com/jd/jdbc/sqlparser/utils/Utils.java index 6ca3be8..577db24 100644 --- a/src/main/java/com/jd/jdbc/sqlparser/utils/Utils.java +++ b/src/main/java/com/jd/jdbc/sqlparser/utils/Utils.java @@ -25,7 +25,6 @@ import java.io.PrintWriter; import java.io.Reader; import java.io.StringWriter; -import java.io.UnsupportedEncodingException; import java.lang.management.ManagementFactory; import java.nio.charset.StandardCharsets; import java.security.MessageDigest; diff --git a/src/main/java/com/jd/jdbc/sqlparser/visitor/AnalyzerVisitor.java b/src/main/java/com/jd/jdbc/sqlparser/visitor/AnalyzerVisitor.java new file mode 100644 index 0000000..83764a3 --- /dev/null +++ b/src/main/java/com/jd/jdbc/sqlparser/visitor/AnalyzerVisitor.java @@ -0,0 +1,287 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.sqlparser.visitor; + +import com.jd.jdbc.planbuilder.semantics.Analyzer; +import com.jd.jdbc.sqlparser.ASTUtils; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLOrderBy; +import com.jd.jdbc.sqlparser.ast.expr.SQLAggregateExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOpExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLCharExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLIdentifierExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLIntegerExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLNumberExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLPropertyExpr; +import com.jd.jdbc.sqlparser.ast.statement.SQLExprTableSource; +import com.jd.jdbc.sqlparser.ast.statement.SQLJoinTableSource; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectGroupByClause; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectOrderByItem; +import com.jd.jdbc.sqlparser.ast.statement.SQLSubqueryTableSource; +import com.jd.jdbc.sqlparser.ast.statement.SQLTableSource; +import com.jd.jdbc.sqlparser.ast.statement.SQLUnionQuery; +import com.jd.jdbc.sqlparser.dialect.mysql.ast.statement.MySqlSelectQueryBlock; +import com.jd.jdbc.sqlparser.dialect.mysql.visitor.MySqlASTVisitorAdapter; + +public class AnalyzerVisitor extends MySqlASTVisitorAdapter { + + private final Analyzer analyzer; + + public AnalyzerVisitor(Analyzer analyzer) { + this.analyzer = analyzer; + } + + @Override + public boolean visit(MySqlSelectQueryBlock x) { + analyzer.analyzeDown(x); + SQLTableSource from = x.getFrom(); + if (from == null) { + from = new SQLExprTableSource(new SQLIdentifierExpr("dual")); + } + from.accept(this); + x.getAfterCommentsDirect(); + for (SQLSelectItem sqlSelectItem : x.getSelectList()) { + sqlSelectItem.accept(this); + } + if (x.getWhere() != null) { + x.getWhere().accept(this); + } + if (x.getGroupBy() != null) { + x.getGroupBy().accept(this); + if (x.getGroupBy().getHaving() != null) { + x.getGroupBy().getHaving().accept(this); + } + } + if (x.getOrderBy() != null) { + x.getOrderBy().accept(this); + } + return false; + } + + @Override + public void endVisit(SQLUnionQuery x) { + analyzer.analyzeUp(x); + } + + @Override + public boolean visit(SQLUnionQuery x) { + analyzer.analyzeDown(x); + return true; + } + + @Override + public void endVisit(MySqlSelectQueryBlock x) { + analyzer.analyzeUp(x); + } + + @Override + public boolean visit(SQLExprTableSource x) { + analyzer.analyzeDown(x); + return false; + } + + @Override + public void endVisit(SQLExprTableSource x) { + analyzer.analyzeUp(x); + } + + @Override + public boolean visit(SQLJoinTableSource x) { + analyzer.analyzeDown(x); + return true; + } + + @Override + public void endVisit(SQLJoinTableSource x) { + analyzer.analyzeUp(x); + } + + @Override + public boolean visit(SQLSubqueryTableSource x) { + analyzer.analyzeDown(x); + x.getSelect().accept(this); + return false; + } + + @Override + public void endVisit(SQLSubqueryTableSource x) { + analyzer.analyzeUp(x); + } + + @Override + public boolean visit(SQLSelectItem x) { + analyzer.analyzeDown(x); + return true; + } + + @Override + public void endVisit(SQLSelectItem x) { + analyzer.analyzeUp(x); + } + + @Override + public boolean visit(SQLSelectGroupByClause x) { + analyzer.analyzeDown(x); + for (SQLExpr expr : x.getItems()) { + expr.accept(this); + } + return false; + } + + @Override + public void endVisit(SQLSelectGroupByClause x) { + analyzer.analyzeUp(x); + } + + @Override + public boolean visit(SQLOrderBy x) { + analyzer.analyzeDown(x); + return true; + } + + @Override + public void endVisit(SQLOrderBy x) { + analyzer.analyzeUp(x); + } + + @Override + public boolean visit(SQLIdentifierExpr x) { + analyzer.analyzeDown(x); + return false; + } + + @Override + public void endVisit(SQLIdentifierExpr x) { + analyzer.analyzeUp(x); + } + + @Override + public boolean visit(SQLPropertyExpr x) { + analyzer.analyzeDown(x); + return false; + } + + @Override + public void endVisit(SQLPropertyExpr x) { + analyzer.analyzeUp(x); + } + + @Override + public boolean visit(SQLBinaryOpExpr x) { + if (x.getParent() instanceof SQLSelectGroupByClause) { + SQLSelectGroupByClause groupBy = (SQLSelectGroupByClause) x.getParent(); + if (groupBy.getHaving() == x) { + analyzer.analyzeDown(x); + return true; + } + } + return true; + } + + @Override + public void endVisit(SQLBinaryOpExpr x) { + if (x.getParent() instanceof SQLSelectGroupByClause) { + SQLSelectGroupByClause groupBy = (SQLSelectGroupByClause) x.getParent(); + if (groupBy.getHaving() == x) { + analyzer.analyzeUp(x); + } + } + } + + @Override + public boolean visit(SQLIntegerExpr x) { + analyzer.analyzeDown(x); + SQLOrderBy orderByParent = ASTUtils.getOrderByParent(x); + if (orderByParent != null) { + for (SQLSelectOrderByItem item : orderByParent.getItems()) { + if (x == item.getExpr()) { + continue; + } + analyzer.analyzeDown(item.getExpr()); + } + } + SQLSelectGroupByClause groupByClause = ASTUtils.getGroupByParent(x); + if (groupByClause != null) { + for (SQLExpr item : groupByClause.getItems()) { + if (x == item) { + continue; + } + analyzer.analyzeDown(item); + } + } + return false; + } + + @Override + public void endVisit(SQLIntegerExpr x) { + analyzer.analyzeUp(x); + SQLOrderBy orderByParent = ASTUtils.getOrderByParent(x); + if (orderByParent != null) { + for (SQLSelectOrderByItem item : orderByParent.getItems()) { + if (x == item.getExpr()) { + continue; + } + analyzer.analyzeUp(item.getExpr()); + } + } + SQLSelectGroupByClause groupByClause = ASTUtils.getGroupByParent(x); + if (groupByClause != null) { + for (SQLExpr item : groupByClause.getItems()) { + if (x == item) { + continue; + } + analyzer.analyzeUp(item); + } + } + } + + @Override + public boolean visit(SQLCharExpr x) { + analyzer.analyzeDown(x); + return false; + } + + @Override + public void endVisit(SQLCharExpr x) { + analyzer.analyzeUp(x); + } + + @Override + public boolean visit(SQLNumberExpr x) { + analyzer.analyzeDown(x); + return false; + } + + @Override + public void endVisit(SQLNumberExpr x) { + analyzer.analyzeUp(x); + } + + @Override + public boolean visit(SQLAggregateExpr x) { + analyzer.analyzeDown(x); + return true; + } + + @Override + public void endVisit(SQLAggregateExpr x) { + analyzer.analyzeUp(x); + } +} diff --git a/src/main/java/com/jd/jdbc/sqlparser/visitor/CheckForInvalidGroupingExpressionsVisitor.java b/src/main/java/com/jd/jdbc/sqlparser/visitor/CheckForInvalidGroupingExpressionsVisitor.java new file mode 100644 index 0000000..1e246ce --- /dev/null +++ b/src/main/java/com/jd/jdbc/sqlparser/visitor/CheckForInvalidGroupingExpressionsVisitor.java @@ -0,0 +1,35 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.sqlparser.visitor; + +import com.jd.jdbc.sqlparser.ast.expr.SQLAggregateExpr; +import com.jd.jdbc.sqlparser.dialect.mysql.visitor.MySqlASTVisitorAdapter; +import java.sql.SQLException; +import lombok.Getter; + +public class CheckForInvalidGroupingExpressionsVisitor extends MySqlASTVisitorAdapter { + @Getter + private SQLException exception = null; + + @Override + public boolean visit(SQLAggregateExpr x) { + exception = new SQLException("Can't group on '" + x.toString() + "'"); + return true; + } +} diff --git a/src/main/java/com/jd/jdbc/sqlparser/visitor/CheckUnionVisitor.java b/src/main/java/com/jd/jdbc/sqlparser/visitor/CheckUnionVisitor.java new file mode 100644 index 0000000..a7fef6c --- /dev/null +++ b/src/main/java/com/jd/jdbc/sqlparser/visitor/CheckUnionVisitor.java @@ -0,0 +1,34 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.sqlparser.visitor; + +import com.jd.jdbc.sqlparser.ast.expr.SQLPropertyExpr; +import com.jd.jdbc.sqlparser.dialect.mysql.visitor.MySqlASTVisitorAdapter; +import lombok.Getter; + +public class CheckUnionVisitor extends MySqlASTVisitorAdapter { + @Getter + private String err; + + @Override + public boolean visit(SQLPropertyExpr x) { + err = "Table '" + x.getOwnernName() + "' from one of the SELECTs cannot be used in global ORDER clause"; + return false; + } +} diff --git a/src/main/java/com/jd/jdbc/sqlparser/visitor/ExprDependenciesVisitor.java b/src/main/java/com/jd/jdbc/sqlparser/visitor/ExprDependenciesVisitor.java new file mode 100644 index 0000000..bdf9c4b --- /dev/null +++ b/src/main/java/com/jd/jdbc/sqlparser/visitor/ExprDependenciesVisitor.java @@ -0,0 +1,64 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.sqlparser.visitor; + +import com.jd.jdbc.planbuilder.semantics.ExprDependencies; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLIdentifierExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLPropertyExpr; +import com.jd.jdbc.sqlparser.dialect.mysql.visitor.MySqlASTVisitorAdapter; +import lombok.Getter; + +public class ExprDependenciesVisitor extends MySqlASTVisitorAdapter { + private ExprDependencies exprDependencies; + + @Getter + private TableSet deps; + + private final SQLExpr expr; + + public ExprDependenciesVisitor(ExprDependencies exprDependencies, SQLExpr expr) { + this.exprDependencies = exprDependencies; + this.deps = new TableSet(0L, null); + this.expr = expr; + } + + @Override + public boolean visit(SQLIdentifierExpr x) { + TableSet set = exprDependencies.get(x); + if (set != null) { + deps.mergeInPlace(set); + exprDependencies.put(expr, deps); + return false; + } + return true; + } + + @Override + public boolean visit(SQLPropertyExpr x) { + TableSet set = exprDependencies.get(x); + if (set != null) { + deps.mergeInPlace(set); + exprDependencies.put(expr, deps); + return false; + } + return true; + } +} diff --git a/src/main/java/com/jd/jdbc/sqlparser/visitor/OrderByGroupByNumberRewriteVisitor.java b/src/main/java/com/jd/jdbc/sqlparser/visitor/OrderByGroupByNumberRewriteVisitor.java new file mode 100644 index 0000000..06250eb --- /dev/null +++ b/src/main/java/com/jd/jdbc/sqlparser/visitor/OrderByGroupByNumberRewriteVisitor.java @@ -0,0 +1,89 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.sqlparser.visitor; + +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLObject; +import com.jd.jdbc.sqlparser.ast.SQLOrderBy; +import com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOpExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLBinaryOperator; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectGroupByClause; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectOrderByItem; +import com.jd.jdbc.sqlparser.dialect.mysql.visitor.MySqlASTVisitorAdapter; + +public class OrderByGroupByNumberRewriteVisitor extends MySqlASTVisitorAdapter { + private final SQLObject cursor; + + private final SQLExpr newNode; + + public OrderByGroupByNumberRewriteVisitor(SQLObject cursor, SQLExpr newNode) { + this.cursor = cursor; + this.newNode = newNode; + } + + @Override + public boolean visit(SQLSelectGroupByClause groupByClause) { + int num = -1; + for (int i = 0; i < groupByClause.getItems().size(); i++) { + if (groupByClause.getItems().get(i) == cursor) { + num = i; + break; + } + } + if (num != -1) { + groupByClause.getItems().set(num, newNode); + return false; + } + return true; + } + + @Override + public boolean visit(SQLOrderBy orderBy) { + int num = -1; + boolean collateFlag = false; + for (int i = 0; i < orderBy.getItems().size(); i++) { + if (orderBy.getItems().get(i).getExpr() == cursor) { + num = i; + break; + } + // support order by id collate utf8_general_ci rewrite + if (orderBy.getItems().get(i).getExpr() instanceof SQLBinaryOpExpr) { + SQLBinaryOpExpr binaryOpExpr = (SQLBinaryOpExpr) orderBy.getItems().get(i).getExpr(); + if (binaryOpExpr.getOperator() == SQLBinaryOperator.COLLATE) { + if (binaryOpExpr.getLeft() == cursor) { + num = i; + collateFlag = true; + break; + } + } + } + } + if (num != -1) { + SQLSelectOrderByItem sqlSelectOrderByItem = orderBy.getItems().get(num); + if (!collateFlag) { + sqlSelectOrderByItem.setExpr(newNode); + } else { + SQLBinaryOpExpr binaryOpExpr = (SQLBinaryOpExpr) sqlSelectOrderByItem.getExpr(); + binaryOpExpr.setLeft(newNode); + } + return false; + } + return true; + } +} diff --git a/src/main/java/com/jd/jdbc/sqlparser/visitor/RewriteHavingVisitor.java b/src/main/java/com/jd/jdbc/sqlparser/visitor/RewriteHavingVisitor.java new file mode 100644 index 0000000..9e10d08 --- /dev/null +++ b/src/main/java/com/jd/jdbc/sqlparser/visitor/RewriteHavingVisitor.java @@ -0,0 +1,63 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.sqlparser.visitor; + + +import com.google.common.collect.Sets; +import com.jd.jdbc.sqlparser.SQLUtils; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLAggregateExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLIdentifierExpr; +import com.jd.jdbc.sqlparser.dialect.mysql.visitor.CheckNodeTypesVisitor; +import com.jd.jdbc.sqlparser.dialect.mysql.visitor.MySqlASTVisitorAdapter; +import java.util.Map; +import lombok.Getter; + +public class RewriteHavingVisitor extends MySqlASTVisitorAdapter { + private Map selectExprMap; + + @Getter + private boolean hasAggr; + + public RewriteHavingVisitor(Map selectExprMap) { + this.selectExprMap = selectExprMap; + hasAggr = false; + } + + @Override + public boolean visit(SQLIdentifierExpr x) { + SQLExpr originalExpr = selectExprMap.get(x.getLowerName()); + if (originalExpr != null) { + CheckNodeTypesVisitor visitor = new CheckNodeTypesVisitor(Sets.newHashSet(CheckNodeTypesVisitor.CheckNodeType.AGGREGATE)); + originalExpr.accept(visitor); + if (visitor.getCheckResult()) { + hasAggr = true; + } else { + SQLUtils.replaceInParent(x, originalExpr); + } + } + return false; + } + + @Override + public boolean visit(final SQLAggregateExpr x) { + hasAggr = true; + return true; + } +} diff --git a/src/main/java/com/jd/jdbc/sqlparser/visitor/SQLASTOutputVisitor.java b/src/main/java/com/jd/jdbc/sqlparser/visitor/SQLASTOutputVisitor.java index 54174ac..f9ab989 100644 --- a/src/main/java/com/jd/jdbc/sqlparser/visitor/SQLASTOutputVisitor.java +++ b/src/main/java/com/jd/jdbc/sqlparser/visitor/SQLASTOutputVisitor.java @@ -2828,6 +2828,7 @@ public boolean visit(SQLUnionQuery x) { && left instanceof SQLUnionQuery && ((SQLUnionQuery) left).getOperator() == operator && !right.isBracket() + && x.getLimit() == null && orderBy == null) { SQLUnionQuery leftUnion = (SQLUnionQuery) left; diff --git a/src/main/java/com/jd/jdbc/sqlparser/visitor/VtBreakExpressionInLHSandRHSVisitor.java b/src/main/java/com/jd/jdbc/sqlparser/visitor/VtBreakExpressionInLHSandRHSVisitor.java new file mode 100644 index 0000000..b9a24d8 --- /dev/null +++ b/src/main/java/com/jd/jdbc/sqlparser/visitor/VtBreakExpressionInLHSandRHSVisitor.java @@ -0,0 +1,90 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.sqlparser.visitor; + +import com.jd.jdbc.planbuilder.semantics.SemTable; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.SQLUtils; +import com.jd.jdbc.sqlparser.ast.SQLName; +import com.jd.jdbc.sqlparser.ast.expr.SQLIdentifierExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLPropertyExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLVariantRefExpr; +import com.jd.jdbc.sqlparser.dialect.mysql.visitor.MySqlASTVisitorAdapter; +import java.util.ArrayList; +import java.util.List; +import lombok.Getter; + +@Getter +public class VtBreakExpressionInLHSandRHSVisitor extends MySqlASTVisitorAdapter { + + private TableSet lhs; + + private SemTable semTable; + + private String errMsg; + + private Boolean bErr; + + private List bvNames; + + private List columus; + + public VtBreakExpressionInLHSandRHSVisitor(TableSet lhs, SemTable semTable) { + this.lhs = lhs; + this.semTable = semTable; + this.errMsg = ""; + this.bErr = false; + this.bvNames = new ArrayList<>(); + this.columus = new ArrayList<>(); + } + + public boolean visit(SQLIdentifierExpr node) { + return this.visitColName(node); + } + + public boolean visit(SQLPropertyExpr node) { + return this.visitColName(node); + } + + private boolean visitColName(SQLName node) { + TableSet deps = this.semTable.recursiveDeps(node); + if (deps.numberOfTables() == 0) { + this.errMsg = "unknown column. has the AST been copied? :" + node.getSimpleName(); + this.bErr = true; + return false; + } + + if (deps.isSolvedBy(this.lhs)) { + this.columus.add(node); + String bvName = node.getSimpleName(); + if (node instanceof SQLIdentifierExpr) { + bvName = ((SQLIdentifierExpr) node).getName(); + } else if (node instanceof SQLPropertyExpr) { + bvName = ((SQLPropertyExpr) node).getOwnernName() + "_" + ((SQLPropertyExpr) node).getName(); + } + this.bvNames.add(bvName); + SQLVariantRefExpr arg = new SQLVariantRefExpr(":" + bvName); + // we are replacing one of the sides of the comparison with an argument, + // but we don't want to lose the type information we have, so we copy it over + this.semTable.copyExprInfo(node, arg); + SQLUtils.replaceInParent(node, arg); + } + return false; + } +} diff --git a/src/main/java/com/jd/jdbc/sqltypes/Constants.java b/src/main/java/com/jd/jdbc/sqltypes/Constants.java index 3e7d37d..e6be8ba 100644 --- a/src/main/java/com/jd/jdbc/sqltypes/Constants.java +++ b/src/main/java/com/jd/jdbc/sqltypes/Constants.java @@ -37,6 +37,14 @@ public class Constants { public static final BigDecimal BIG_DECIMAL_MAX_LONG_VALUE = BigDecimal.valueOf(9223372036854775807L); + public static final BigDecimal BIG_DECIMAL_MAX_DOUBLE_VALUE = BigDecimal.valueOf(1.7976931348623157E308D); + + public static final BigDecimal BIG_DECIMAL_MAX_NEGATIVE_DOUBLE_VALUE = BigDecimal.valueOf(-1.7976931348623157E308D); + + public static final BigDecimal BIG_DECIMAL_MAX_FLOAT_VALUE = BigDecimal.valueOf(3.4028234663852886E38D); + + public static final BigDecimal BIG_DECIMAL_MAX_NEGATIVE_FLOAT_VALUE = BigDecimal.valueOf(-3.4028234663852886E38D); + private Constants() { } } diff --git a/src/main/java/com/jd/jdbc/sqltypes/SqlTypes.java b/src/main/java/com/jd/jdbc/sqltypes/SqlTypes.java index 7c722e4..8c750be 100644 --- a/src/main/java/com/jd/jdbc/sqltypes/SqlTypes.java +++ b/src/main/java/com/jd/jdbc/sqltypes/SqlTypes.java @@ -113,4 +113,8 @@ public static BindVariable stringBindVariable(String v) throws SQLException { public static Query.Value vtValueToProto(VtResultValue vtValue) { return Query.Value.newBuilder().setType(vtValue.getVtType()).setValue(ByteString.copyFrom(vtValue.toBytes())).build(); } + + public static Query.Value vtValueToProto(VtValue vtValue) { + return Query.Value.newBuilder().setType(vtValue.getVtType()).setValue(ByteString.copyFrom(vtValue.toBytes())).build(); + } } diff --git a/src/main/java/com/jd/jdbc/sqltypes/VtResultSet.java b/src/main/java/com/jd/jdbc/sqltypes/VtResultSet.java index c2e735e..7d688be 100644 --- a/src/main/java/com/jd/jdbc/sqltypes/VtResultSet.java +++ b/src/main/java/com/jd/jdbc/sqltypes/VtResultSet.java @@ -18,13 +18,14 @@ package com.jd.jdbc.sqltypes; -import com.jd.jdbc.vitess.VitessStatement; import io.vitess.proto.Query; import java.math.BigInteger; import java.sql.SQLException; import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Objects; +import java.util.StringJoiner; import lombok.Getter; import lombok.Setter; @@ -45,8 +46,6 @@ public class VtResultSet implements VtRowList { private boolean closed = false; - private VitessStatement owningStatement = null; - public VtResultSet() { this.rows = new ArrayList<>(); this.insertID = BigInteger.valueOf(0); @@ -201,10 +200,6 @@ public void close() throws SQLException { return; } - if (owningStatement != null) { - owningStatement.removeOpenResultSet(this); - } - this.closed = true; } @@ -232,6 +227,7 @@ public VtRowList clone() { return vtRowList; } + @Override public BigInteger getInsertID() { return this.insertID; } @@ -244,4 +240,36 @@ public void setInsertID(BigInteger setId) { this.insertID = setId; } + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + VtResultSet that = (VtResultSet) o; + return rowsAffected == that.rowsAffected && isDML == that.isDML && curRow == that.curRow && closed == that.closed && Arrays.equals(fields, that.fields) && + Objects.equals(insertID, that.insertID) && Objects.equals(rows, that.rows); + } + + @Override + public int hashCode() { + int result = Objects.hash(rowsAffected, insertID, rows, isDML, curRow, closed); + result = 31 * result + Arrays.hashCode(fields); + return result; + } + + @Override + public String toString() { + return new StringJoiner(", ", VtResultSet.class.getSimpleName() + "[", "]") + .add("fields=" + Arrays.toString(fields)) + .add("rowsAffected=" + rowsAffected) + .add("insertID=" + insertID) + .add("rows=" + rows) + .add("isDML=" + isDML) + .add("curRow=" + curRow) + .add("closed=" + closed) + .toString(); + } } diff --git a/src/main/java/com/jd/jdbc/sqltypes/VtResultValue.java b/src/main/java/com/jd/jdbc/sqltypes/VtResultValue.java index 2ad921e..90508bb 100644 --- a/src/main/java/com/jd/jdbc/sqltypes/VtResultValue.java +++ b/src/main/java/com/jd/jdbc/sqltypes/VtResultValue.java @@ -23,6 +23,7 @@ import java.math.BigDecimal; import java.math.BigInteger; import java.sql.SQLException; +import java.util.Objects; import lombok.Getter; import lombok.Setter; @@ -168,6 +169,27 @@ public boolean isNull() { return vtType == null || vtType.equals(Query.Type.NULL_TYPE) || value == null; } + /** + * IsComparable returns true if the Value is null safe comparable without collation information. + */ + public boolean isComparable() { + + if (vtType == null || VtType.isNumber(vtType) || VtType.isBinary(vtType)) { + return true; + } + switch (vtType){ + case TIMESTAMP: + case DATE: + case TIME: + case DATETIME: + case ENUM: + case SET: + case BIT: + return true; + } + return false; + } + public byte[] toBytes() { if (vtType == Query.Type.EXPRESSION) { return null; @@ -189,4 +211,21 @@ public String toString() { return String.valueOf(value); } } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + VtResultValue value1 = (VtResultValue) o; + return Objects.equals(value, value1.value) && vtType == value1.vtType; + } + + @Override + public int hashCode() { + return Objects.hash(value, vtType); + } } diff --git a/src/main/java/com/jd/jdbc/sqltypes/VtStreamResultSet.java b/src/main/java/com/jd/jdbc/sqltypes/VtStreamResultSet.java index 9db87a9..1d87b07 100644 --- a/src/main/java/com/jd/jdbc/sqltypes/VtStreamResultSet.java +++ b/src/main/java/com/jd/jdbc/sqltypes/VtStreamResultSet.java @@ -136,10 +136,6 @@ public void close() throws SQLException { this.vtStream.close(); this.vtStream = null; - if (owningStatement != null) { - owningStatement.removeOpenResultSet(this); - } - this.closed = true; } diff --git a/src/main/java/com/jd/jdbc/sqltypes/VtType.java b/src/main/java/com/jd/jdbc/sqltypes/VtType.java index 8c8c5c5..2848055 100644 --- a/src/main/java/com/jd/jdbc/sqltypes/VtType.java +++ b/src/main/java/com/jd/jdbc/sqltypes/VtType.java @@ -25,11 +25,10 @@ import java.sql.SQLException; import java.sql.Timestamp; import java.sql.Types; +import static java.sql.Types.LONGVARBINARY; import java.util.HashMap; import java.util.Map; -import static java.sql.Types.LONGVARBINARY; - public class VtType { private static final int flagIsIntegral = Query.Flag.ISINTEGRAL_VALUE; @@ -120,6 +119,10 @@ public static boolean isNumber(Query.Type typ) { return isIntegral(typ) || isFloat(typ) || typ.getNumber() == Query.Type.DECIMAL_VALUE; } + public static boolean isDate(Query.Type t) { + return t == Query.Type.DATETIME || t == Query.Type.DATE || t == Query.Type.TIMESTAMP || t == Query.Type.TIME; + } + public static Query.Type getQueryType(String columnTypeName) throws SQLException { Query.Type type = TYPE_MAP.get(columnTypeName); if (type == null) { diff --git a/src/main/java/com/jd/jdbc/sqltypes/VtValue.java b/src/main/java/com/jd/jdbc/sqltypes/VtValue.java index 77bc9cf..5ec7d7d 100644 --- a/src/main/java/com/jd/jdbc/sqltypes/VtValue.java +++ b/src/main/java/com/jd/jdbc/sqltypes/VtValue.java @@ -323,6 +323,25 @@ public String toString() { return StringUtils.toString(vtValue, charEncoding); } + /** + * String returns a printable version of the value. + * + * @return + */ + public String string() { + if (isNull()) { + return "null"; + } + String string = this.toString(); + if (this.vtType == Query.Type.VARBINARY || this.vtType == Query.Type.VARCHAR) { + string = "\"" + string + "\""; + } + if (this.isQuoted() || this.vtType == Query.Type.BIT) { + return this.vtType + "(" + string + ")"; + } + return this.vtType + "(" + string + ")"; + } + public boolean toBoolean() { if (isNull()) { return false; @@ -478,6 +497,48 @@ public byte[] toBytes() { return vtValue; } + public float toFloat() throws SQLException { + if (isNull()) { + return 0; + } + + if (isNumber()) { + BigDecimal d = new BigDecimal(new String(vtValue, 0, vtValue.length)); + if ((d.compareTo(Constants.BIG_DECIMAL_MAX_NEGATIVE_FLOAT_VALUE) < 0 || d.compareTo(Constants.BIG_DECIMAL_MAX_FLOAT_VALUE) > 0)) { + throw new SQLException("number out of range: " + d.toString()); + } + return (float) d.doubleValue(); + } + + if (getVtType() == Query.Type.BIT) { + return new BigInteger(ByteBuffer.allocate(vtValue.length + 1).put((byte) 0).put(vtValue, 0, vtValue.length).array()).floatValue(); + } + + String s = new String(vtValue, 0, vtValue.length); + return Float.parseFloat(s); + } + + public double toDouble() throws SQLException { + if (isNull()) { + return 0; + } + + if (isNumber()) { + BigDecimal d = new BigDecimal(new String(vtValue, 0, vtValue.length)); + if ((d.compareTo(Constants.BIG_DECIMAL_MAX_NEGATIVE_DOUBLE_VALUE) < 0 || d.compareTo(Constants.BIG_DECIMAL_MAX_DOUBLE_VALUE) > 0)) { + throw new SQLException("number out of range: " + d.toString()); + } + return d.doubleValue(); + } + + if (getVtType() == Query.Type.BIT) { + return new BigInteger(ByteBuffer.allocate(vtValue.length + 1).put((byte) 0).put(vtValue, 0, vtValue.length).array()).doubleValue(); + } + + String s = new String(vtValue, 0, vtValue.length); + return Double.parseDouble(s); + } + public Query.Value toQueryValue() { if (this.isNull()) { return Query.Value.newBuilder().build(); diff --git a/src/main/java/com/jd/jdbc/srvtopo/ScatterConn.java b/src/main/java/com/jd/jdbc/srvtopo/ScatterConn.java index ba5b7e5..481cde9 100644 --- a/src/main/java/com/jd/jdbc/srvtopo/ScatterConn.java +++ b/src/main/java/com/jd/jdbc/srvtopo/ScatterConn.java @@ -403,10 +403,6 @@ private IQueryService getQueryService(ResolvedShard rs, ShardActionInfo info) { */ private Exception oneShard(ResolvedShard rs, Integer i, SafeSession safeSession, Boolean autocommit, ShardActionTransactionFunc action) { Exception exception = null; - /* TODO - startTime, statsKey := stc.startAction(name, rs.Target) - defer stc.endAction(startTime, allErrors, statsKey, &err, session) - */ ShardActionInfo shardActionInfo = actionInfo(rs.getTarget(), safeSession, autocommit); ShardActionTransactionFuncResponse response = action.action(rs, i, shardActionInfo); ShardActionInfo updated = response.getShardActionInfo(); diff --git a/src/main/java/com/jd/jdbc/topo/Topo.java b/src/main/java/com/jd/jdbc/topo/Topo.java index b3057e5..bf56533 100644 --- a/src/main/java/com/jd/jdbc/topo/Topo.java +++ b/src/main/java/com/jd/jdbc/topo/Topo.java @@ -18,13 +18,6 @@ package com.jd.jdbc.topo; -import com.jd.jdbc.topo.etcd2topo.Etcd2TopoFactory; -import io.vitess.proto.Vtrpc; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - import static com.jd.jdbc.topo.TopoExceptionCode.NO_IMPLEMENTATION; import static com.jd.jdbc.topo.TopoServer.CELLS_PATH; import static com.jd.jdbc.topo.TopoServer.CELL_INFO_FILE; @@ -35,6 +28,12 @@ import static com.jd.jdbc.topo.TopoServer.TABLETS_PATH; import static com.jd.jdbc.topo.TopoServer.TABLET_FILE; import static com.jd.jdbc.topo.TopoServer.VSCHEMA_FILE; +import com.jd.jdbc.topo.etcd2topo.Etcd2TopoFactory; +import io.vitess.proto.Vtrpc; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; public class Topo { diff --git a/src/main/java/com/jd/jdbc/topo/TopoServer.java b/src/main/java/com/jd/jdbc/topo/TopoServer.java index 05c607d..3e147c4 100644 --- a/src/main/java/com/jd/jdbc/topo/TopoServer.java +++ b/src/main/java/com/jd/jdbc/topo/TopoServer.java @@ -23,7 +23,13 @@ import com.jd.jdbc.key.CurrentShard; import com.jd.jdbc.sqlparser.support.logging.Log; import com.jd.jdbc.sqlparser.support.logging.LogFactory; +import static com.jd.jdbc.topo.Topo.pathForCellInfo; +import static com.jd.jdbc.topo.Topo.pathForSrvKeyspaceFile; +import static com.jd.jdbc.topo.Topo.pathForTabletAlias; +import static com.jd.jdbc.topo.Topo.pathForVschemaFile; +import static com.jd.jdbc.topo.TopoConnection.ConnGetResponse; import com.jd.jdbc.topo.TopoConnection.DirEntry; +import static com.jd.jdbc.topo.TopoExceptionCode.NO_NODE; import com.jd.jdbc.topo.topoproto.TopoProto; import io.vitess.proto.Topodata; import java.util.ArrayList; @@ -37,13 +43,6 @@ import java.util.concurrent.locks.ReentrantLock; import vschema.Vschema; -import static com.jd.jdbc.topo.Topo.pathForCellInfo; -import static com.jd.jdbc.topo.Topo.pathForSrvKeyspaceFile; -import static com.jd.jdbc.topo.Topo.pathForTabletAlias; -import static com.jd.jdbc.topo.Topo.pathForVschemaFile; -import static com.jd.jdbc.topo.TopoConnection.ConnGetResponse; -import static com.jd.jdbc.topo.TopoExceptionCode.NO_NODE; - public class TopoServer implements Resource, TopoCellInfo, TopoSrvKeyspace, TopoTablet, TopoVschema { private static final Log log = LogFactory.getLog(TopoServer.class); diff --git a/src/main/java/com/jd/jdbc/util/SelectItemUtil.java b/src/main/java/com/jd/jdbc/util/SelectItemUtil.java new file mode 100644 index 0000000..af39e01 --- /dev/null +++ b/src/main/java/com/jd/jdbc/util/SelectItemUtil.java @@ -0,0 +1,41 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.util; + +import com.jd.jdbc.sqlparser.ast.expr.SQLIdentifierExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLPropertyExpr; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; + +public class SelectItemUtil { + + public static String columnName(SQLSelectItem aliasedExpr) { + String alias = aliasedExpr.getAlias(); + if (alias == null) { + if (aliasedExpr.getExpr() instanceof SQLIdentifierExpr) { + alias = ((SQLIdentifierExpr) aliasedExpr.getExpr()).getName(); + } else if (aliasedExpr.getExpr() instanceof SQLPropertyExpr) { + alias = ((SQLPropertyExpr) aliasedExpr.getExpr()).getName(); + } else { + alias = aliasedExpr.getExpr().toString(); + } + } + return alias; + } + +} diff --git a/src/main/java/com/jd/jdbc/vindexes/LookupPlanable.java b/src/main/java/com/jd/jdbc/vindexes/LookupPlanable.java new file mode 100644 index 0000000..4aabc53 --- /dev/null +++ b/src/main/java/com/jd/jdbc/vindexes/LookupPlanable.java @@ -0,0 +1,36 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.vindexes; + +import com.jd.jdbc.key.Destination; +import com.jd.jdbc.sqltypes.VtResultSet; +import com.jd.jdbc.sqltypes.VtValue; + +// LookupPlanable are for lookup vindexes where we can extract the lookup query at plan time +public interface LookupPlanable { + String String(); + + String[] Query(); // (selQuery string, arguments []string) + + Destination[] MapResult(VtValue[] ids, VtResultSet[] results); + + Boolean AllowBatch(); + + Object GetCommitOrder(); +} diff --git a/src/main/java/com/jd/jdbc/vindexes/MultiColumn.java b/src/main/java/com/jd/jdbc/vindexes/MultiColumn.java new file mode 100644 index 0000000..a05b0e0 --- /dev/null +++ b/src/main/java/com/jd/jdbc/vindexes/MultiColumn.java @@ -0,0 +1,52 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.vindexes; + +import com.jd.jdbc.key.Destination; +import com.jd.jdbc.sqltypes.VtValue; +import java.sql.SQLException; + +public interface MultiColumn extends Vindex{ + /* + Map(ctx context.Context, vcursor VCursor, rowsColValues [][]sqltypes.Value) ([]key.Destination, error) + Verify(ctx context.Context, vcursor VCursor, rowsColValues [][]sqltypes.Value, ksids [][]byte) ([]bool, error) + */ + /** + * + * @param rowsColValues array of {@link VtValue} + * @return array of {@link Destination} + */ + Destination[] map(VtValue[][] rowsColValues); + + /** + * + * @param rowsColValues array of {@link VtValue} + * @param ksids two dimensional array or byte + * @return array of {@link Boolean} + * @throws SQLException when an error occurs + */ + Boolean[] verify(VtValue[][] rowsColValues, byte[][] ksids) throws SQLException; + + /** + * PartialVindex returns true if subset of columns can be passed in to the vindex Map and Verify function. + * @return + */ + // + Boolean PartialVindex(); +} diff --git a/src/main/java/com/jd/jdbc/vindexes/VKeyspace.java b/src/main/java/com/jd/jdbc/vindexes/VKeyspace.java index 2d5c0a4..4c159c8 100644 --- a/src/main/java/com/jd/jdbc/vindexes/VKeyspace.java +++ b/src/main/java/com/jd/jdbc/vindexes/VKeyspace.java @@ -35,4 +35,15 @@ public VKeyspace(String name, Boolean sharded) { this.name = name; this.sharded = sharded; } + + @Override + public boolean equals(final Object obj) { + if (obj == this) { + return true; + } + if (obj instanceof VKeyspace) { + return this.name == ((VKeyspace) obj).name && this.sharded == ((VKeyspace) obj).sharded; + } + return false; + } } diff --git a/src/main/java/com/jd/jdbc/vindexes/Vschema.java b/src/main/java/com/jd/jdbc/vindexes/VschemaConstant.java similarity index 96% rename from src/main/java/com/jd/jdbc/vindexes/Vschema.java rename to src/main/java/com/jd/jdbc/vindexes/VschemaConstant.java index 2d59351..d9b31c3 100644 --- a/src/main/java/com/jd/jdbc/vindexes/Vschema.java +++ b/src/main/java/com/jd/jdbc/vindexes/VschemaConstant.java @@ -18,7 +18,7 @@ package com.jd.jdbc.vindexes; -public class Vschema { +public class VschemaConstant { public static final String TYPE_PINNED_TABLE = ""; public static final String CODE_PINNED_TABLE = "00"; diff --git a/src/main/java/com/jd/jdbc/vitess/VitessStatement.java b/src/main/java/com/jd/jdbc/vitess/VitessStatement.java index 6842973..7b801f9 100755 --- a/src/main/java/com/jd/jdbc/vitess/VitessStatement.java +++ b/src/main/java/com/jd/jdbc/vitess/VitessStatement.java @@ -73,10 +73,8 @@ import java.sql.SQLWarning; import java.util.ArrayList; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import lombok.AllArgsConstructor; @@ -115,7 +113,7 @@ public class VitessStatement extends AbstractVitessStatement { protected List resultSets = new ArrayList<>(); - protected Set openedResultSets = new HashSet<>(); + protected List openedResultSets = new ArrayList<>(); protected volatile Boolean isClosed; @@ -464,10 +462,6 @@ protected boolean getExecuteInternalResult() { return resultSets.get(0).isQuery(); } - public void removeOpenResultSet(VtRowList result) { - openedResultSets.remove(result); - } - protected int executeUpdateInternal(IContext ctx, String sql, Map bindVariableMap, boolean clearBatchedGeneratedKeys, boolean returnGeneratedKeys) throws SQLException { this.retrieveGeneratedKeys = returnGeneratedKeys; if (clearBatchedGeneratedKeys) { diff --git a/src/test/java/com/jd/jdbc/engine/FakePrimitive.java b/src/test/java/com/jd/jdbc/engine/FakePrimitive.java new file mode 100644 index 0000000..c0b9b8f --- /dev/null +++ b/src/test/java/com/jd/jdbc/engine/FakePrimitive.java @@ -0,0 +1,123 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.engine; + +import com.jd.jdbc.IExecute; +import com.jd.jdbc.context.IContext; +import static com.jd.jdbc.engine.vcursor.FakeVcursorUtil.printBindVars; +import com.jd.jdbc.sqltypes.VtResultSet; +import com.jd.jdbc.srvtopo.BindVariable; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import lombok.AllArgsConstructor; +import org.junit.Assert; + +/** + * fakePrimitive fakes a primitive. For every call, it sends the + * next result from the results. If the next result is nil, it + * returns sendErr. For streaming calls, it sends the field info + * first and two rows at a time till all rows are sent. + */ +@AllArgsConstructor +public class FakePrimitive implements PrimitiveEngine { + + private List resultSet; + + private int curResult; + + // sendErr is sent at the end of the stream if it's set. + private SQLException sendErr; + + private List log = new ArrayList<>(); + + private boolean allResultsInOneCall; + + public FakePrimitive(List resultSet) { + this.resultSet = resultSet; + log = new ArrayList<>(); + } + + public FakePrimitive(List resultSet, SQLException err) { + this.resultSet = resultSet; + this.sendErr = err; + log = new ArrayList<>(); + } + + public FakePrimitive(SQLException sendErr) { + this.sendErr = sendErr; + } + + @Override + public String getKeyspaceName() { + return "fakeKs"; + } + + @Override + public String getTableName() { + return "fakeTable"; + } + + @Override + public IExecute.ExecuteMultiShardResponse execute(IContext ctx, Vcursor vcursor, Map bindVariableMap, boolean wantFields) throws SQLException { + log.add(String.format("Execute %s %s", printBindVars(bindVariableMap), wantFields)); + if (this.resultSet == null) { + if (this.sendErr != null) { + throw this.sendErr; + } + return null; + } + VtResultSet r = resultSet.get(curResult); + this.curResult++; + if (r == null) { + if (this.sendErr != null) { + throw this.sendErr; + } + return null; + } + return new IExecute.ExecuteMultiShardResponse(r); + + } + + @Override + public VtResultSet getFields(Vcursor vcursor, Map bindVariableMap) throws SQLException { + log.add(String.format("GetFields %s", printBindVars(bindVariableMap))); + return (VtResultSet) this.execute(null, vcursor, bindVariableMap, true).getVtRowList(); + } + + @Override + public Boolean needsTransaction() { + return false; + } + + public void rewind() { + this.curResult = 0; + this.log = new ArrayList<>(); + } + + public void expectLog(List want) { + if (want.size() != log.size()) { + Assert.fail("wants size " + want.size() + " actual size" + log.size()); + } + for (int i = 0; i < want.size(); i++) { + Assert.assertEquals(want.get(i), log.get(i)); + } + } +} diff --git a/src/test/java/com/jd/jdbc/engine/gen4/ConcatenateGen4EngineTest.java b/src/test/java/com/jd/jdbc/engine/gen4/ConcatenateGen4EngineTest.java new file mode 100644 index 0000000..9d69e83 --- /dev/null +++ b/src/test/java/com/jd/jdbc/engine/gen4/ConcatenateGen4EngineTest.java @@ -0,0 +1,210 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.engine.gen4; + +import com.google.common.collect.Lists; +import com.jd.BaseTest; +import com.jd.jdbc.IExecute; +import com.jd.jdbc.context.VtContext; +import com.jd.jdbc.engine.FakePrimitive; +import com.jd.jdbc.engine.PrimitiveEngine; +import com.jd.jdbc.engine.util.TestResult; +import com.jd.jdbc.engine.vcursor.NoopVCursor; +import com.jd.jdbc.sqltypes.VtResultSet; +import com.jd.jdbc.util.threadpool.impl.VtQueryExecutorService; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import lombok.AllArgsConstructor; +import lombok.Data; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class ConcatenateGen4EngineTest extends BaseTest { + + public VtResultSet r(String names, String types, String... rows) { + return TestResult.makeTestResult(TestResult.makeTestFields(names, types), rows); + } + + @Before + public void init() { + VtQueryExecutorService.initialize(null, null, null, null); + } + + @Test + public void testConcatenateNoErrors() throws InterruptedException { + VtResultSet r1 = r("id|col1|col2", "int64|varbinary|varbinary", "1|a1|b1", "2|a2|b2"); + VtResultSet r2 = r("id|col1|col2", "int32|varbinary|varbinary", "1|a1|b1", "2|a2|b2"); + VtResultSet combinedResult = r1; + r1.getRows().addAll(r2.getRows()); + + List testCases = new ArrayList<>(); + + // 测试用例1 + List inputs1 = new ArrayList<>(); + inputs1.add(r("id1|col11|col12", "int64|varbinary|varbinary")); + inputs1.add(r("id2|col21|col22", "int64|varbinary|varbinary")); + inputs1.add(r("id3|col31|col32", "int64|varbinary|varbinary")); + testCases.add(new TestCase("empty results", inputs1, r("id1|col11|col12", "int64|varbinary|varbinary"))); + + // 测试用例2 + List inputs2 = new ArrayList<>(); + inputs2.add(r("myid|mycol1|mycol2", "int64|varchar|varbinary", "11|m1|n1", "22|m2|n2")); + inputs2.add(r("id|col1|col2", "int64|varchar|varbinary", "1|a1|b1", "2|a2|b2")); + inputs2.add(r("id2|col2|col3", "int64|varchar|varbinary", "3|a3|b3")); + inputs2.add(r("id2|col2|col3", "int64|varchar|varbinary", "4|a4|b4")); + testCases.add(new TestCase("2 non empty result", inputs2, r("myid|mycol1|mycol2", "int64|varchar|varbinary", "11|m1|n1", "22|m2|n2", "1|a1|b1", "2|a2|b2", "3|a3|b3", "4|a4|b4"))); + +// 测试用例3 + List inputs3 = new ArrayList<>(); + inputs3.add(r("id|col1|col2", "int64|varbinary|varbinary", "1|a1|b1", "2|a2|b2")); + inputs3.add(r("id|col1|col2", "int64|varbinary|varbinary", "1|a1|b1", "2|a2|b2")); + inputs3.add(r("id|col3|col4", "int64|varchar|varbinary", "1|a1|b1", "2|a2|b2")); + testCases.add(new TestCase("column field type does not match for name", inputs3, "merging field of different types is not supported")); + + // 测试用例4 + List inputs4 = new ArrayList<>(); + inputs4.add(r("id|col1|col2", "int64|varbinary|varbinary", "1|a1|b1", "2|a2|b2")); + inputs4.add(r("id|col1|col2", "int32|varbinary|varbinary", "1|a1|b1", "2|a2|b2")); + testCases.add(new TestCase("ignored field types - ignored", inputs4, combinedResult, null, Lists.newArrayList(0))); + + // 测试用例5 + List inputs5 = new ArrayList<>(); + inputs5.add(r("id|col1|col2", "int64|varchar|varchar", "1|a1|b1", "2|a2|b2")); + inputs5.add(r("id|col1|col2", "int64|varchar|varchar", "1|a1|b1", "2|a2|b2")); + inputs5.add(r("id|col3|col4|col5", "int64|varchar|varchar|int32", "1|a1|b1|5", "2|a2|b2|6")); + testCases.add(new TestCase("input source has different column count", inputs5, "The used SELECT statements have a different number of columns")); + + // 测试用例6 + List inputs6 = new ArrayList<>(); + inputs6.add(r("myid|mycol1|mycol2", "int64|varchar|varbinary")); + inputs6.add(r("id|col1|col2", "int64|varchar|varbinary", "1|a1|b1", "2|a2|b2")); + testCases.add(new TestCase("1 empty result and 1 non empty result", inputs6, r("myid|mycol1|mycol2", "int64|varchar|varbinary", "1|a1|b1", "2|a2|b2"))); + CountDownLatch latch = new CountDownLatch(testCases.size()); + ExecutorService executorService = new ThreadPoolExecutor( + 10, 10, 0L, TimeUnit.MILLISECONDS, + new LinkedBlockingQueue<>()); + AtomicBoolean errorFlag = new AtomicBoolean(true); + for (TestCase tc : testCases) { + List sources = new ArrayList<>(); + for (VtResultSet input : tc.inputs) { + // input is added twice, since the first one is used by execute and the next by stream execute + sources.add(new FakePrimitive(Lists.newArrayList(input, input))); + } + ConcatenateGen4Engine concatenate = new ConcatenateGen4Engine(sources, tc.ignoreTypes); + executorService.execute(() -> { + try { + IExecute.ExecuteMultiShardResponse qr = concatenate.execute(VtContext.background(), new NoopVCursor(), null, true); + try { + Assert.assertEquals(printFail(tc.testName + " is [FAIL]"), tc.expectedResult, qr.getVtRowList()); + printOk(tc.testName + " is[OK]"); + } catch (AssertionError e) { + e.printStackTrace(); + errorFlag.set(false); + } + + } catch (Exception e) { + if (tc.expectedError != null && !e.getMessage().contains(tc.expectedError) || tc.expectedError == null) { + errorFlag.set(false); + System.out.println(printFail(tc.testName + " is [FAIL]" + e.getMessage())); + e.printStackTrace(); + } else { + printOk(tc.testName + " is[OK]"); + } + } finally { + latch.countDown(); + } + }); + } + if (!latch.await(10, TimeUnit.SECONDS)) { + Assert.fail("testConcatenateNoErrors is [FAIL] timeout"); + } + Assert.assertTrue(printFail("testConcatenateNoErrors is [FAIL]"), errorFlag.get()); + + } + + @Test + public void testConcatenateWithErrors() { + SQLException strFailed = new SQLException("failed"); + VtResultSet fake = r("id|col1|col2", "int64|varchar|varbinary", "1|a1|b1", "2|a2|b2"); + FakePrimitive fake1 = new FakePrimitive( + Lists.newArrayList(fake, fake) + ); + FakePrimitive fake2 = new FakePrimitive(null, strFailed); + FakePrimitive fake3 = new FakePrimitive( + Lists.newArrayList(fake, fake) + ); + List primitives = Lists.newArrayList(fake1, fake2, fake3); + ConcatenateGen4Engine concatenate = new ConcatenateGen4Engine(primitives, new ArrayList<>()); + try { + concatenate.execute(VtContext.background(), new NoopVCursor(), null, true); + } catch (SQLException e) { + Assert.assertEquals(strFailed, e); + } + + concatenate = new ConcatenateGen4Engine(primitives, new ArrayList<>()); + try { + concatenate.execute(VtContext.background(), new NoopVCursor(), null, true); + } catch (SQLException e) { + Assert.assertEquals(strFailed, e); + } + } + + @AllArgsConstructor + @Data + class TestCase { + private String testName; + + private List inputs; + + private VtResultSet expectedResult; + + private String expectedError; + + private List ignoreTypes; + + TestCase(String testName, List inputs, VtResultSet expectedResult) { + this.testName = testName; + this.inputs = inputs; + this.expectedResult = expectedResult; + + } + + TestCase(String testName, List inputs, String expectedError) { + this.testName = testName; + this.inputs = inputs; + this.expectedError = expectedError; + + } + + TestCase(String testName, List inputs, VtResultSet expectedResult, List ignoreTypes) { + this.testName = testName; + this.inputs = inputs; + this.expectedResult = expectedResult; + this.ignoreTypes = ignoreTypes; + } + } +} diff --git a/src/test/java/com/jd/jdbc/engine/gen4/DistinctGen4EngineTest.java b/src/test/java/com/jd/jdbc/engine/gen4/DistinctGen4EngineTest.java new file mode 100644 index 0000000..5ebec52 --- /dev/null +++ b/src/test/java/com/jd/jdbc/engine/gen4/DistinctGen4EngineTest.java @@ -0,0 +1,114 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.engine.gen4; + +import com.jd.BaseTest; +import com.jd.jdbc.context.VtContext; +import com.jd.jdbc.engine.FakePrimitive; +import com.jd.jdbc.engine.util.TestResult; +import com.jd.jdbc.engine.vcursor.NoopVCursor; +import com.jd.jdbc.sqltypes.VtResultSet; +import com.jd.jdbc.sqltypes.VtResultValue; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import org.junit.Assert; +import org.junit.Test; + +public class DistinctGen4EngineTest extends BaseTest { + + @Test + public void testDistinct() { + List testCases = Arrays.asList( + new TestCase("empty", r("id1|col11|col12", "int64|varbinary|varbinary"), r("id1|col11|col12", "int64|varbinary|varbinary")), + new TestCase("int64 numbers", r("myid", "int64", "0", "1", "1", "null", "null"), r("myid", "int64", "0", "1", "null")), + new TestCase("int64 numbers, two columns", r("a|b", "int64|int64", "0|0", "1|1", "1|1", "null|null", "null|null", "1|2"), r("a|b", "int64|int64", "0|0", "1|1", "null|null", "1|2")), + new TestCase("int64 numbers, two columns", r("a|b", "int64|int64", "3|3", "3|3", "3|4", "5|1", "5|1"), r("a|b", "int64|int64", "3|3", "3|4", "5|1")), + new TestCase("float64 columns designed to produce the same hashcode but not be equal", r("a|b", "float64|float64", "0.1|0.2", "0.1|0.3", "0.1|0.4", "0.1|0.5"), r("a|b", "float64|float64", "0.1|0.2", "0.1|0.3", "0.1|0.4", "0.1|0.5")), + new TestCase("varchar columns without collations", r("myid", "varchar", "monkey", "horse"), "text type with an unknown/unsupported collation cannot be hashed") +// new TestCase("varchar columns with collations", null, r("myid", "varchar", "monkey", "horse", "Horse", "Monkey", "horses", "MONKEY"), r("myid", "varchar", "monkey", "horse", "horses")), +// new TestCase("mixed columns", null, r("myid|id", "varchar|int64", "monkey|1", "horse|1", "Horse|1", "Monkey|1", "horses|1", "MONKEY|2"), r("myid|id", "varchar|int64", "monkey|1", "horse|1", "horses|1", "MONKEY|2")) + ); + + for (TestCase tc : testCases) { + List checkCols = new ArrayList<>(); + if (tc.inputs.getRows().size() > 0) { + for (int i = 0; i < tc.inputs.getRows().get(0).size(); i++) { + checkCols.add(new CheckCol(i, -1)); + } + } + + // Execute + try { + FakePrimitive source = new FakePrimitive(Collections.singletonList(tc.inputs)); + DistinctGen4Engine distinct = new DistinctGen4Engine(source, checkCols, false); + VtResultSet qr = (VtResultSet) distinct.execute(VtContext.background(), new NoopVCursor(), null, true).getVtRowList(); + if (tc.expectedError == null) { + List> got = qr.getRows(); + List> expected = tc.expectedResult.getRows(); + Assert.assertEquals("result not what correct", expected, got); + } else { + Assert.fail(); + } + } catch (SQLException exception) { + Assert.assertEquals(tc.expectedError, exception.getMessage()); + } + printOk("testDistinct is [OK],case name = " + tc.testName); + + // StreamExecute + } + } + + private static class TestCase { + private String testName; + + private VtResultSet inputs; + + private List collations; + + private VtResultSet expectedResult; + + private String expectedError; + + TestCase(String testName, VtResultSet inputs, VtResultSet expectedResult) { + this.testName = testName; + this.inputs = inputs; + this.expectedResult = expectedResult; + } + + TestCase(String testName, VtResultSet inputs, String expectedError) { + this.testName = testName; + this.inputs = inputs; + this.expectedError = expectedError; + } + + TestCase(String testName, List collations, VtResultSet inputs, VtResultSet expectedResult) { + this.testName = testName; + this.collations = collations; + this.inputs = inputs; + this.expectedResult = expectedResult; + } + } + + private VtResultSet r(String names, String types, String... rows) { + return TestResult.makeTestResult(TestResult.makeTestFields(names, types), rows); + } +} \ No newline at end of file diff --git a/src/test/java/com/jd/jdbc/engine/gen4/FilterGen4EngineTest.java b/src/test/java/com/jd/jdbc/engine/gen4/FilterGen4EngineTest.java new file mode 100644 index 0000000..68f1ad1 --- /dev/null +++ b/src/test/java/com/jd/jdbc/engine/gen4/FilterGen4EngineTest.java @@ -0,0 +1,108 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.engine.gen4; + +import com.jd.jdbc.IExecute; +import com.jd.jdbc.engine.FakePrimitive; +import com.jd.jdbc.engine.util.TestResult; +import com.jd.jdbc.evalengine.Comparisons; +import com.jd.jdbc.evalengine.EvalEngine; +import com.jd.jdbc.sqltypes.VtResultSet; +import com.jd.jdbc.sqltypes.VtResultValue; +import io.vitess.proto.Query; +import java.math.BigInteger; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.Setter; +import org.junit.Assert; +import org.junit.Ignore; +import org.junit.Test; + +public class FilterGen4EngineTest { + + @Test + public void testFilterPass() throws SQLException { + @Getter + @Setter + @AllArgsConstructor + class TestCase { + String name; + + VtResultSet res; + + List> expectResult; + } + + List testCases = new ArrayList<>(2); + testCases.add( + new TestCase("int32", TestResult.makeTestResult(TestResult.makeTestFields("a|b", "int32|int32"), "0|1", "1|0", "2|3"), + Collections.singletonList(Arrays.asList(new VtResultValue(1, Query.Type.INT32), new VtResultValue(0, Query.Type.INT32))))); + testCases.add( + new TestCase("uint64_int64", TestResult.makeTestResult(TestResult.makeTestFields("a|b", "uint64|int64"), "0|1", "1|0", "2|3"), + Collections.singletonList(Arrays.asList(new VtResultValue(new BigInteger("1"), Query.Type.UINT64), new VtResultValue(0L, Query.Type.INT64))))); + + Comparisons.ComparisonExpr predicate = new Comparisons.ComparisonExpr(new Comparisons.CompareGT(), new EvalEngine.Column(0), new EvalEngine.Column(1)); + for (TestCase tc : testCases) { + FilterGen4Engine filterGen4Engine = new FilterGen4Engine(); + filterGen4Engine.setPredicate(predicate); + filterGen4Engine.setInput(new FakePrimitive(Collections.singletonList(tc.getRes()))); + VtResultSet rowList = (VtResultSet) filterGen4Engine.execute(null, null, null, false).getVtRowList(); + + Assert.assertEquals(tc.getExpectResult(), rowList.getRows()); + } + } + + @Test + @Ignore + public void testFilterMixedFail() throws SQLException { + @Getter + @Setter + @AllArgsConstructor + class TestCase { + String name; + + VtResultSet res; + + String expErr; + } + + Comparisons.ComparisonExpr predicate = new Comparisons.ComparisonExpr(new Comparisons.CompareGT(), new EvalEngine.Column(0), new EvalEngine.Column(1)); + + List testCases = new ArrayList<>(); + testCases.add( + new TestCase("uint64_int32", TestResult.makeTestResult(TestResult.makeTestFields("a|b", "uint64|int32"), "0|1", "1|0", "2|3"), "unsupported: cannot compare UINT64 and INT32")); + + for (TestCase tc : testCases) { + FilterGen4Engine filterGen4Engine = new FilterGen4Engine(); + filterGen4Engine.setPredicate(predicate); + filterGen4Engine.setInput(new FakePrimitive(Collections.singletonList(tc.getRes()))); + + try { + IExecute.ExecuteMultiShardResponse response = filterGen4Engine.execute(null, null, null, false); + } catch (SQLException e) { + Assert.assertEquals(tc.expErr, e.getMessage()); + } + } + } +} diff --git a/src/test/java/com/jd/jdbc/engine/gen4/JoinGen4EngineTest.java b/src/test/java/com/jd/jdbc/engine/gen4/JoinGen4EngineTest.java new file mode 100644 index 0000000..7eaf294 --- /dev/null +++ b/src/test/java/com/jd/jdbc/engine/gen4/JoinGen4EngineTest.java @@ -0,0 +1,281 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.engine.gen4; + +import com.google.common.collect.Lists; +import com.jd.BaseTest; +import com.jd.jdbc.IExecute; +import com.jd.jdbc.context.VtContext; +import com.jd.jdbc.engine.Engine; +import com.jd.jdbc.engine.FakePrimitive; +import com.jd.jdbc.engine.util.TestResult; +import com.jd.jdbc.engine.vcursor.FakeVcursorUtil; +import com.jd.jdbc.engine.vcursor.NoopVCursor; +import com.jd.jdbc.sqltypes.SqlTypes; +import com.jd.jdbc.sqltypes.VtResultSet; +import com.jd.jdbc.srvtopo.BindVariable; +import io.vitess.proto.Query; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import lombok.AllArgsConstructor; +import lombok.Getter; +import org.junit.Assert; +import org.junit.Test; + +public class JoinGen4EngineTest extends BaseTest { + + @Test + public void testJoinExecute() throws SQLException { + Query.Field[] leftField = TestResult.makeTestFields("col1|col2|col3", "int64|varchar|varchar"); + VtResultSet leftResultSet = TestResult.makeTestResult(leftField, "1|a|aa", "2|b|bb", "3|c|cc"); + FakePrimitive leftPrim = new FakePrimitive(Lists.newArrayList(leftResultSet)); + Query.Field[] rightField = TestResult.makeTestFields("col4|col5|col6", "int64|varchar|varchar"); + List rightResultSet = new ArrayList<>(); + rightResultSet.add(TestResult.makeTestResult(rightField, "4|d|dd")); + rightResultSet.add(TestResult.makeTestResult(rightField)); + rightResultSet.add(TestResult.makeTestResult(rightField, "5|e|ee", "6|f|ff", "7|g|gg")); + FakePrimitive rightPrim = new FakePrimitive(rightResultSet); + Map bv = new HashMap<>(); + bv.put("a", SqlTypes.int64BindVariable(10L)); + // Normal join + Map vars = new HashMap<>(); + vars.put("bv", 1); + JoinGen4Engine jn = new JoinGen4Engine(Engine.JoinOpcode.NormalJoin, vars); + jn.setLeft(leftPrim); + jn.setRight(rightPrim); + jn.setCols(Lists.newArrayList(-1, -2, 1, 2)); + IExecute.ExecuteMultiShardResponse result = jn.execute(VtContext.background(), new NoopVCursor(), bv, true); + leftPrim.expectLog(Lists.newArrayList("Execute a: type:INT64 value:\"10\" true")); + rightPrim.expectLog(Lists.newArrayList("Execute a: type:INT64 value:\"10\" bv: type:VARCHAR value:\"a\" true", + "Execute a: type:INT64 value:\"10\" bv: type:VARCHAR value:\"b\" false", + "Execute a: type:INT64 value:\"10\" bv: type:VARCHAR value:\"c\" false")); + + Query.Field[] expectResultField = TestResult.makeTestFields("col1|col2|col4|col5", "int64|varchar|int64|varchar"); + VtResultSet expectResultSet = TestResult.makeTestResult(expectResultField, "1|a|4|d", + "3|c|5|e", + "3|c|6|f", + "3|c|7|g"); + Assert.assertEquals(printFail(" testJoinExecute Normaljoin is FAIL"), expectResultSet, result.getVtRowList()); + + // Left Join + leftPrim.rewind(); + rightPrim.rewind(); + jn.setOpcode(Engine.JoinOpcode.LeftJoin); + result = jn.execute(VtContext.background(), new NoopVCursor(), bv, true); + leftPrim.expectLog(Lists.newArrayList("Execute a: type:INT64 value:\"10\" true")); + rightPrim.expectLog(Lists.newArrayList("Execute a: type:INT64 value:\"10\" bv: type:VARCHAR value:\"a\" true", + "Execute a: type:INT64 value:\"10\" bv: type:VARCHAR value:\"b\" false", + "Execute a: type:INT64 value:\"10\" bv: type:VARCHAR value:\"c\" false")); + expectResultField = TestResult.makeTestFields("col1|col2|col4|col5", "int64|varchar|int64|varchar"); + expectResultSet = TestResult.makeTestResult(expectResultField, "1|a|4|d", + "2|b|null|null", + "3|c|5|e", + "3|c|6|f", + "3|c|7|g"); + Assert.assertEquals(printFail(" testJoinExecute leftjoin is FAIL"), expectResultSet, result.getVtRowList()); + } + + + @Test + public void testJoinExecuteMaxMemoryRows() throws SQLException { + int saveMax = FakeVcursorUtil.testMaxMemoryRows; + + boolean saveIgnore = FakeVcursorUtil.testIgnoreMaxMemoryRows; + + FakeVcursorUtil.testMaxMemoryRows = 3; + @Getter + @AllArgsConstructor + class TestCase { + boolean ignoreMaxMemoryRows; + + String err; + } + List testCases = new ArrayList<>(); + testCases.add(new TestCase(true, null)); + testCases.add(new TestCase(false, "in-memory row count exceeded allowed limit of 3")); + + for (TestCase tc : testCases) { + Query.Field[] leftField = TestResult.makeTestFields("col1|col2|col3", "int64|varchar|varchar"); + VtResultSet leftResultSet = TestResult.makeTestResult(leftField, "1|a|aa", "2|b|bb", "3|c|cc"); + FakePrimitive leftPrim = new FakePrimitive(Lists.newArrayList(leftResultSet)); + Query.Field[] rightField = TestResult.makeTestFields("col4|col5|col6", "int64|varchar|varchar"); + List rightResultSet = new ArrayList<>(); + rightResultSet.add(TestResult.makeTestResult(rightField, "4|d|dd")); + rightResultSet.add(TestResult.makeTestResult(rightField)); + rightResultSet.add(TestResult.makeTestResult(rightField, "5|e|ee", "6|f|ff", "7|g|gg")); + FakePrimitive rightPrim = new FakePrimitive(rightResultSet); + Map bv = new HashMap<>(); + bv.put("a", SqlTypes.int64BindVariable(10L)); + // Normal join + Map vars = new HashMap<>(); + vars.put("bv", 1); + JoinGen4Engine jn = new JoinGen4Engine(Engine.JoinOpcode.NormalJoin, vars); + jn.setLeft(leftPrim); + jn.setRight(rightPrim); + jn.setCols(Lists.newArrayList(-1, -2, 1, 2)); + + FakeVcursorUtil.testIgnoreMaxMemoryRows = tc.ignoreMaxMemoryRows; + try { + jn.execute(VtContext.background(), new NoopVCursor(), bv, true); + if (!FakeVcursorUtil.testIgnoreMaxMemoryRows) { + Assert.fail(" testJoinExecuteMaxMemoryRows is FAIL"); + } + } catch (SQLException e) { + if (!FakeVcursorUtil.testIgnoreMaxMemoryRows) { + Assert.assertEquals(printFail(" testJoinExecuteMaxMemoryRows is FAIL"), tc.err, e.getMessage()); + } else { + throw e; + } + } + } + FakeVcursorUtil.testMaxMemoryRows = saveMax; + FakeVcursorUtil.testIgnoreMaxMemoryRows = saveIgnore; + + } + + @Test + public void testJoinExecuteNoResult() throws SQLException { + Query.Field[] leftField = TestResult.makeTestFields("col1|col2|col3", "int64|varchar|varchar"); + VtResultSet leftResultSet = TestResult.makeTestResult(leftField); + FakePrimitive leftPrim = new FakePrimitive(Lists.newArrayList(leftResultSet)); + Query.Field[] rightField = TestResult.makeTestFields("col4|col5|col6", "int64|varchar|varchar"); + VtResultSet rightResultSet = TestResult.makeTestResult(rightField); + FakePrimitive rightPrim = new FakePrimitive(Lists.newArrayList(rightResultSet)); + Map vars = new HashMap<>(); + vars.put("bv", 1); + JoinGen4Engine jn = new JoinGen4Engine(Engine.JoinOpcode.NormalJoin, vars); + jn.setLeft(leftPrim); + jn.setRight(rightPrim); + jn.setCols(Lists.newArrayList(-1, -2, 1, 2)); + IExecute.ExecuteMultiShardResponse result = jn.execute(VtContext.background(), new NoopVCursor(), null, true); + leftPrim.expectLog(Lists.newArrayList("Execute true")); + rightPrim.expectLog(Lists.newArrayList("GetFields bv: ", + "Execute bv: true")); + + Query.Field[] expectResultField = TestResult.makeTestFields("col1|col2|col4|col5", "int64|varchar|int64|varchar"); + VtResultSet expectResultSet = TestResult.makeTestResult(expectResultField); + Assert.assertEquals(printFail(" testJoinExecuteNoResult is FAIL"), expectResultSet, result.getVtRowList()); + } + + @Test + public void testJoinExecuteErrors() { + // Error on left query + FakePrimitive leftPrim = new FakePrimitive(new SQLException("left err")); + JoinGen4Engine jn = new JoinGen4Engine(Engine.JoinOpcode.NormalJoin, null); + jn.setLeft(leftPrim); + try { + jn.execute(VtContext.background(), new NoopVCursor(), null, true); + Assert.fail(" testJoinExecuteErrors is FAIL"); + } catch (SQLException e) { + Assert.assertEquals(printFail(" testJoinExecuteErrors is FAIL"), "left err", e.getMessage()); + } + + // Error on right query + Query.Field[] leftField = TestResult.makeTestFields("col1|col2|col3", "int64|varchar|varchar"); + VtResultSet leftResultSet = TestResult.makeTestResult(leftField, "1|a|aa", + "2|b|bb", + "3|c|cc"); + leftPrim = new FakePrimitive(Lists.newArrayList(leftResultSet)); + FakePrimitive rightPrim = new FakePrimitive(new SQLException("right err")); + Map vars = new HashMap<>(); + vars.put("bv", 1); + jn = new JoinGen4Engine(Engine.JoinOpcode.NormalJoin, vars); + jn.setLeft(leftPrim); + jn.setRight(rightPrim); + jn.setCols(Lists.newArrayList(-1, -2, 1, 2)); + try { + jn.execute(VtContext.background(), new NoopVCursor(), null, true); + Assert.fail(" testJoinExecuteErrors is FAIL"); + } catch (SQLException e) { + Assert.assertEquals(printFail(" testJoinExecuteErrors is FAIL"), "right err", e.getMessage()); + } + + // Error on right getfields + leftResultSet = TestResult.makeTestResult(leftField); + leftPrim = new FakePrimitive(Lists.newArrayList(leftResultSet)); + jn = new JoinGen4Engine(Engine.JoinOpcode.NormalJoin, vars); + jn.setLeft(leftPrim); + jn.setRight(rightPrim); + try { + jn.execute(VtContext.background(), new NoopVCursor(), null, true); + Assert.fail(" testJoinExecuteErrors is FAIL"); + } catch (SQLException e) { + Assert.assertEquals(printFail(" testJoinExecuteErrors is FAIL"), "right err", e.getMessage()); + } + } + + /* @Test + public void testJoinStreamExecute() {}*/ + + @Test + public void testGetFields() throws SQLException { + Query.Field[] leftField = TestResult.makeTestFields("col1|col2|col3", "int64|varchar|varchar"); + VtResultSet leftResultSet = TestResult.makeTestResult(leftField); + FakePrimitive leftPrim = new FakePrimitive(Lists.newArrayList(leftResultSet)); + Query.Field[] rightField = TestResult.makeTestFields("col4|col5|col6", "int64|varchar|varchar"); + VtResultSet rightResultSet = TestResult.makeTestResult(rightField); + FakePrimitive rightPrim = new FakePrimitive(Lists.newArrayList(rightResultSet)); + Map vars = new HashMap<>(); + vars.put("bv", 1); + JoinGen4Engine jn = new JoinGen4Engine(Engine.JoinOpcode.NormalJoin, vars); + jn.setLeft(leftPrim); + jn.setRight(rightPrim); + jn.setCols(Lists.newArrayList(-1, -2, 1, 2)); + VtResultSet fields = jn.getFields(null, null); + leftPrim.expectLog(Lists.newArrayList("GetFields ", "Execute true")); + rightPrim.expectLog(Lists.newArrayList("GetFields bv: ", + "Execute bv: true")); + Query.Field[] expectResultField = TestResult.makeTestFields("col1|col2|col4|col5", "int64|varchar|int64|varchar"); + VtResultSet expectResultSet = TestResult.makeTestResult(expectResultField); + Assert.assertEquals(printFail(" testGetFields is FAIL"), expectResultSet, fields); + } + + @Test + public void testGetFieldsErrors() { + FakePrimitive leftPrim = new FakePrimitive(new SQLException("left err")); + FakePrimitive rightPrim = new FakePrimitive(new SQLException("right err")); + Map vars = new HashMap<>(); + vars.put("bv", 1); + JoinGen4Engine jn = new JoinGen4Engine(Engine.JoinOpcode.NormalJoin, vars); + jn.setLeft(leftPrim); + jn.setRight(rightPrim); + jn.setCols(Lists.newArrayList(-1, -2, 1, 2)); + try { + jn.execute(VtContext.background(), new NoopVCursor(), null, true); + Assert.fail(" testGetFieldsErrors is FAIL"); + } catch (SQLException e) { + Assert.assertEquals(printFail(" testGetFieldsErrors is FAIL"), "left err", e.getMessage()); + } + + Query.Field[] leftField = TestResult.makeTestFields("col1|col2|col3", "int64|varchar|varchar"); + VtResultSet leftResultSet = TestResult.makeTestResult(leftField); + leftPrim = new FakePrimitive(Lists.newArrayList(leftResultSet)); + jn.setLeft(leftPrim); + try { + jn.execute(VtContext.background(), new NoopVCursor(), null, true); + Assert.fail(" testGetFieldsErrors is FAIL"); + } catch (SQLException e) { + Assert.assertEquals(printFail(" testGetFieldsErrors is FAIL"), "right err", e.getMessage()); + } + } + +} diff --git a/src/test/java/com/jd/jdbc/engine/gen4/LimitGen4EngineTest.java b/src/test/java/com/jd/jdbc/engine/gen4/LimitGen4EngineTest.java new file mode 100644 index 0000000..53f3d70 --- /dev/null +++ b/src/test/java/com/jd/jdbc/engine/gen4/LimitGen4EngineTest.java @@ -0,0 +1,251 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.engine.gen4; + +import com.jd.jdbc.context.VtContext; +import com.jd.jdbc.engine.FakePrimitive; +import com.jd.jdbc.engine.util.TestResult; +import com.jd.jdbc.engine.vcursor.NoopVCursor; +import com.jd.jdbc.evalengine.EvalEngine; +import com.jd.jdbc.evalengine.EvalResult; +import com.jd.jdbc.sqltypes.VtResultSet; +import com.jd.jdbc.sqltypes.VtResultValue; +import com.jd.jdbc.srvtopo.BindVariable; +import io.vitess.proto.Query; +import java.math.BigInteger; +import java.sql.SQLException; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +public class LimitGen4EngineTest { + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Test + public void testLimitExecute() throws SQLException { + Map bindVariableMap = new LinkedHashMap(); + Query.Field[] fields = TestResult.makeTestFields("col1|col2", "varchar|int64"); + VtResultSet inputResult = TestResult.makeTestResult(fields, "a|1", "b|2", "c|3"); + + FakePrimitive fp = new FakePrimitive(Collections.singletonList(inputResult)); + LimitGen4Engine l = new LimitGen4Engine(fp, new EvalEngine.Literal(new EvalResult(BigInteger.valueOf(2), Query.Type.UINT64)), null); + + // Test with limit smaller than input. + VtResultSet resultSet = (VtResultSet) l.execute(VtContext.background(), null, bindVariableMap, false).getVtRowList(); + List> expectResult = Arrays.asList( + Arrays.asList(new VtResultValue("a", Query.Type.VARCHAR), new VtResultValue(1, Query.Type.INT64)), + Arrays.asList(new VtResultValue("b", Query.Type.VARCHAR), new VtResultValue(2, Query.Type.INT64)) + ); + Assert.assertEquals(expectResult.toString(), resultSet.getRows().toString()); + + // Test with limit equal to input. + VtResultSet inputResult2 = TestResult.makeTestResult(fields, "a|1", "b|2", "c|3"); + FakePrimitive fp2 = new FakePrimitive(Collections.singletonList(inputResult2)); + LimitGen4Engine l2 = new LimitGen4Engine(fp2, new EvalEngine.Literal(new EvalResult(BigInteger.valueOf(3), Query.Type.UINT64)), null); + List> expectResult2 = Arrays.asList( + Arrays.asList(new VtResultValue("a", Query.Type.VARCHAR), new VtResultValue(1, Query.Type.INT64)), + Arrays.asList(new VtResultValue("b", Query.Type.VARCHAR), new VtResultValue(2, Query.Type.INT64)), + Arrays.asList(new VtResultValue("c", Query.Type.VARCHAR), new VtResultValue(3, Query.Type.INT64)) + ); + VtResultSet resultSet2 = (VtResultSet) l2.execute(VtContext.background(), null, bindVariableMap, false).getVtRowList(); + Assert.assertEquals(expectResult2.toString(), resultSet2.getRows().toString()); + + // Test with limit higher than input. + VtResultSet inputResult3 = TestResult.makeTestResult(fields, "a|1", "b|2", "c|3"); + FakePrimitive fp3 = new FakePrimitive(Collections.singletonList(inputResult3)); + LimitGen4Engine l3 = new LimitGen4Engine(fp3, new EvalEngine.Literal(new EvalResult(BigInteger.valueOf(4), Query.Type.UINT64)), null); + List> expectResult3 = Arrays.asList( + Arrays.asList(new VtResultValue("a", Query.Type.VARCHAR), new VtResultValue(1, Query.Type.INT64)), + Arrays.asList(new VtResultValue("b", Query.Type.VARCHAR), new VtResultValue(2, Query.Type.INT64)), + Arrays.asList(new VtResultValue("c", Query.Type.VARCHAR), new VtResultValue(3, Query.Type.INT64)) + ); + VtResultSet resultSet3 = (VtResultSet) l3.execute(VtContext.background(), null, bindVariableMap, false).getVtRowList(); + Assert.assertEquals(expectResult3.toString(), resultSet3.getRows().toString()); + } + + @Test + public void testLimitOffsetExecute() throws SQLException { + Query.Field[] fields = TestResult.makeTestFields("col1|col2", "varchar|int64"); + Map bindVariableMap = new LinkedHashMap(); + + // Test with offset 0 + // limit 2,0 + VtResultSet inputResult1 = TestResult.makeTestResult(fields, "a|1", "b|2", "c|3", "c|4", "c|5", "c|6"); + LimitGen4Engine l1 = new LimitGen4Engine(); + l1.setInput(new FakePrimitive(Collections.singletonList(inputResult1))); + l1.setOffset(new EvalEngine.Literal(new EvalResult(BigInteger.valueOf(2), Query.Type.UINT64))); + l1.setCount(new EvalEngine.Literal(new EvalResult(BigInteger.valueOf(0), Query.Type.UINT64))); + + VtResultSet resultSet1 = (VtResultSet) l1.execute(VtContext.background(), null, bindVariableMap, false).getVtRowList(); + + List> expectResult1 = Arrays.asList(); + Assert.assertEquals(expectResult1.toString(), resultSet1.getRows().toString()); + + // Test with offset set + // limit 1,2 + VtResultSet inputResult2 = TestResult.makeTestResult(fields, "a|1", "b|2", "c|3", "c|4", "c|5", "c|6"); + LimitGen4Engine l2 = new LimitGen4Engine(); + l2.setInput(new FakePrimitive(Collections.singletonList(inputResult2))); + l2.setOffset(new EvalEngine.Literal(new EvalResult(BigInteger.valueOf(1), Query.Type.UINT64))); + l2.setCount(new EvalEngine.Literal(new EvalResult(BigInteger.valueOf(2), Query.Type.UINT64))); + + VtResultSet resultSet2 = (VtResultSet) l2.execute(VtContext.background(), null, bindVariableMap, false).getVtRowList(); + + List> expectResult2 = Arrays.asList( + Arrays.asList(new VtResultValue("b", Query.Type.VARCHAR), new VtResultValue(2, Query.Type.INT64)), + Arrays.asList(new VtResultValue("c", Query.Type.VARCHAR), new VtResultValue(3, Query.Type.INT64)) + ); + Assert.assertEquals(expectResult2.toString(), resultSet2.getRows().toString()); + + // Works on boundary condition (elements == limit + offset) + VtResultSet inputResult3 = TestResult.makeTestResult(fields, "a|1", "b|2", "c|3", "c|4", "c|5", "c|6"); + LimitGen4Engine l3 = new LimitGen4Engine(); + l3.setInput(new FakePrimitive(Collections.singletonList(inputResult3))); + l3.setOffset(new EvalEngine.Literal(new EvalResult(BigInteger.valueOf(1), Query.Type.UINT64))); + l3.setCount(new EvalEngine.Literal(new EvalResult(BigInteger.valueOf(2), Query.Type.UINT64))); + + VtResultSet resultSet3 = (VtResultSet) l3.execute(VtContext.background(), null, bindVariableMap, false).getVtRowList(); + + List> expectResult3 = Arrays.asList( + Arrays.asList(new VtResultValue("b", Query.Type.VARCHAR), new VtResultValue(2, Query.Type.INT64)), + Arrays.asList(new VtResultValue("c", Query.Type.VARCHAR), new VtResultValue(3, Query.Type.INT64)) + ); + Assert.assertEquals(expectResult3.toString(), resultSet3.getRows().toString()); + + // Works on boundary condition (elements == limit + offset) + // limit 2,4 + VtResultSet inputResult4 = TestResult.makeTestResult(fields, "a|1", "b|2", "c|3", "c|4", "c|5", "c|6"); + LimitGen4Engine l4 = new LimitGen4Engine(); + l4.setInput(new FakePrimitive(Collections.singletonList(inputResult4))); + l4.setOffset(new EvalEngine.Literal(new EvalResult(BigInteger.valueOf(2), Query.Type.UINT64))); + l4.setCount(new EvalEngine.Literal(new EvalResult(BigInteger.valueOf(4), Query.Type.UINT64))); + + VtResultSet resultSet4 = (VtResultSet) l4.execute(VtContext.background(), null, bindVariableMap, false).getVtRowList(); + + List> expectResult4 = Arrays.asList( + Arrays.asList(new VtResultValue("c", Query.Type.VARCHAR), new VtResultValue(3, Query.Type.INT64)), + Arrays.asList(new VtResultValue("c", Query.Type.VARCHAR), new VtResultValue(4, Query.Type.INT64)), + Arrays.asList(new VtResultValue("c", Query.Type.VARCHAR), new VtResultValue(5, Query.Type.INT64)), + Arrays.asList(new VtResultValue("c", Query.Type.VARCHAR), new VtResultValue(6, Query.Type.INT64)) + ); + Assert.assertEquals(expectResult4.toString(), resultSet4.getRows().toString()); + + // test when limit is beyond the number of available elements + // limit 5,2 + VtResultSet inputResult5 = TestResult.makeTestResult(fields, "a|1", "b|2", "c|3", "c|4", "c|5", "c|6"); + LimitGen4Engine l5 = new LimitGen4Engine(); + l5.setInput(new FakePrimitive(Collections.singletonList(inputResult5))); + l5.setOffset(new EvalEngine.Literal(new EvalResult(BigInteger.valueOf(5), Query.Type.UINT64))); + l5.setCount(new EvalEngine.Literal(new EvalResult(BigInteger.valueOf(2), Query.Type.UINT64))); + + VtResultSet resultSet5 = (VtResultSet) l5.execute(VtContext.background(), null, bindVariableMap, false).getVtRowList(); + + List> expectResult5 = Arrays.asList( + Arrays.asList(new VtResultValue("c", Query.Type.VARCHAR), new VtResultValue(6, Query.Type.INT64)) + ); + Assert.assertEquals(expectResult5.toString(), resultSet5.getRows().toString()); + + // Works when offset is beyond the response + // limit 7,2 + VtResultSet inputResult6 = TestResult.makeTestResult(fields, "a|1", "b|2", "c|3", "c|4", "c|5", "c|6"); + LimitGen4Engine l6 = new LimitGen4Engine(); + l6.setInput(new FakePrimitive(Collections.singletonList(inputResult6))); + l6.setOffset(new EvalEngine.Literal(new EvalResult(BigInteger.valueOf(7), Query.Type.UINT64))); + l6.setCount(new EvalEngine.Literal(new EvalResult(BigInteger.valueOf(2), Query.Type.UINT64))); + + VtResultSet resultSet6 = (VtResultSet) l6.execute(VtContext.background(), null, bindVariableMap, false).getVtRowList(); + + Assert.assertFalse(resultSet6.hasNext()); + } + + @Test + public void testLimitStreamExecute(){ + // todo + } + + @Test + public void testOffsetStreamExecute(){ + // todo + } + + @Test + public void testLimitGetFields() throws SQLException { + Query.Field[] fields = TestResult.makeTestFields("col1|col2", "varchar|int64"); + VtResultSet inputResult = TestResult.makeTestResult(fields); + LimitGen4Engine l1 = new LimitGen4Engine(); + l1.setInput(new FakePrimitive(Collections.singletonList(inputResult))); + + VtResultSet resultSet = l1.getFields(null, new HashMap<>()); + Assert.assertEquals(inputResult, resultSet); + } + + @Test + public void testLimitInputFail() throws SQLException { + FakePrimitive fp = new FakePrimitive(new SQLException("input fail")); + LimitGen4Engine l = new LimitGen4Engine(fp, new EvalEngine.Literal(new EvalResult(BigInteger.valueOf(2), Query.Type.UINT64)), null); + + String want = "input fail"; + thrown.expect(SQLException.class); + thrown.expectMessage(want); + + l.execute(VtContext.background(), null, new LinkedHashMap(), false); + +// fp.rewind(); +// l.streamExecute(VtContext.background(), null, new LinkedHashMap() {{ +// put("0", new BindVariable("2".getBytes(), Query.Type.INT32)); +// }}, false); + + fp.rewind(); + l.getFields(null, null); + } + + @Test + public void testLimitInvalidCount() throws SQLException { + Query.Field[] fields = TestResult.makeTestFields("col1|col2", "varchar|int64"); + FakePrimitive fp = new FakePrimitive(Collections.singletonList(TestResult.makeTestResult(fields))); + + String want = "could not parse value: '1.2'"; + thrown.expect(SQLException.class); + thrown.expectMessage(want); + + LimitGen4Engine l = new LimitGen4Engine(fp, new EvalEngine.Literal(new EvalResult(1.2, Query.Type.FLOAT64)), null); + l.execute(VtContext.background(), null, new LinkedHashMap(), false); + } + + @Test + public void testLimitOutOfRange() throws SQLException { + Query.Field[] fields = TestResult.makeTestFields("col1|col2", "varchar|int64"); + FakePrimitive fp = new FakePrimitive(Collections.singletonList(TestResult.makeTestResult(fields))); + + String want = "requested limit is out of range: 18446744073709551615"; + thrown.expect(SQLException.class); + thrown.expectMessage(want); + + LimitGen4Engine l = new LimitGen4Engine(fp, new EvalEngine.Literal(new EvalResult(new BigInteger("18446744073709551615"), Query.Type.UINT64)), null); + l.getCount(new NoopVCursor(), null); + } +} diff --git a/src/test/java/com/jd/jdbc/engine/gen4/MemorySortGen4EngineTest.java b/src/test/java/com/jd/jdbc/engine/gen4/MemorySortGen4EngineTest.java new file mode 100644 index 0000000..ff46c86 --- /dev/null +++ b/src/test/java/com/jd/jdbc/engine/gen4/MemorySortGen4EngineTest.java @@ -0,0 +1,185 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.engine.gen4; + +import com.google.common.collect.Lists; +import com.jd.BaseTest; +import com.jd.jdbc.context.VtContext; +import com.jd.jdbc.engine.FakePrimitive; +import com.jd.jdbc.engine.util.TestResult; +import com.jd.jdbc.engine.vcursor.NoopVCursor; +import com.jd.jdbc.evalengine.EvalEngine; +import com.jd.jdbc.sqlparser.ast.expr.SQLVariantRefExpr; +import com.jd.jdbc.sqltypes.SqlTypes; +import com.jd.jdbc.sqltypes.VtResultSet; +import com.jd.jdbc.srvtopo.BindVariable; +import io.vitess.proto.Query; +import java.sql.SQLException; +import java.util.HashMap; +import java.util.Map; +import org.junit.Assert; +import org.junit.Ignore; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +public class MemorySortGen4EngineTest extends BaseTest { + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Test + public void testMemorySortExecute() throws SQLException { + Query.Field[] fields = TestResult.makeTestFields("c1|c2", "varbinary|decimal"); + VtResultSet resultSet = TestResult.makeTestResult(fields, "a|1", "g|2", "a|1", "c|4", "c|3"); + FakePrimitive fp = new FakePrimitive(Lists.newArrayList(resultSet)); + + OrderByParamsGen4 orderByParamsGen4 = new OrderByParamsGen4(1, false, -1, null); + MemorySortGen4Engine ms = new MemorySortGen4Engine(Lists.newArrayList(orderByParamsGen4), fp); + + VtResultSet result = (VtResultSet) ms.execute(VtContext.background(), new NoopVCursor(), null, false).getVtRowList(); + + VtResultSet wantResult = TestResult.makeTestResult(fields, "a|1", "a|1", "g|2", "c|3", "c|4"); + Assert.assertEquals(wantResult, result); + + fp.rewind(); + EvalEngine.BindVariable pv = new EvalEngine.BindVariable("__upper_limit"); + ms.setUpperLimit(pv); + + Map bindVariableMap = new HashMap<>(); + bindVariableMap.put("__upper_limit", SqlTypes.int64BindVariable(3L)); + result = (VtResultSet) ms.execute(VtContext.background(), new NoopVCursor(), bindVariableMap, false).getVtRowList(); + wantResult = TestResult.makeTestResult(fields, "a|1", "a|1", "g|2"); + Assert.assertEquals(wantResult, result); + } + + public void testMemorySortStreamExecuteWeightString() throws SQLException { + + } + + @Test + public void testMemorySortExecuteWeightString() throws SQLException { + Query.Field[] fields = TestResult.makeTestFields("c1|c2", "varchar|varbinary"); + VtResultSet resultSet = TestResult.makeTestResult(fields, "a|1", "g|2", "a|1", "c|4", "c|3"); + FakePrimitive fp = new FakePrimitive(Lists.newArrayList(resultSet)); + + OrderByParamsGen4 orderByParamsGen4 = new OrderByParamsGen4(0, false, 1, null); + MemorySortGen4Engine ms = new MemorySortGen4Engine(Lists.newArrayList(orderByParamsGen4), fp); + + VtResultSet result = (VtResultSet) ms.execute(VtContext.background(), new NoopVCursor(), null, false).getVtRowList(); + VtResultSet wantResult = TestResult.makeTestResult(fields, "a|1", "a|1", "g|2", "c|3", "c|4"); + Assert.assertEquals(wantResult, result); + + fp.rewind(); + SQLVariantRefExpr upperLimitVar = new SQLVariantRefExpr(); + EvalEngine.BindVariable pv = new EvalEngine.BindVariable("__upper_limit"); + ms.setUpperLimit(pv); + + Map bindVariableMap = new HashMap<>(); + bindVariableMap.put("__upper_limit", SqlTypes.int64BindVariable(3L)); + result = (VtResultSet) ms.execute(VtContext.background(), new NoopVCursor(), bindVariableMap, false).getVtRowList(); + wantResult = TestResult.makeTestResult(fields, "a|1", "a|1", "g|2"); + Assert.assertEquals(wantResult, result); + } + + public void testMemorySortStreamExecuteCollation() throws SQLException { + + } + + public void testMemorySortExecuteCollation() throws SQLException { + + } + + public void testMemorySortStreamExecute() throws SQLException { + + } + + @Test + public void testMemorySortGetFields() throws SQLException { + Query.Field[] fields = TestResult.makeTestFields("col1|col2", "int64|varchar"); + VtResultSet resultSet = TestResult.makeTestResult(fields); + FakePrimitive fp = new FakePrimitive(Lists.newArrayList(resultSet)); + + MemorySortGen4Engine ms = new MemorySortGen4Engine(null, fp); + VtResultSet got = ms.getFields(null, null); + Assert.assertEquals(resultSet, got); + } + + @Test + public void testMemorySortExecuteTruncate() throws SQLException { + Query.Field[] fields = TestResult.makeTestFields("c1|c2|c3", "varbinary|decimal|int64"); + VtResultSet resultSet = TestResult.makeTestResult(fields, "a|1|1", "g|2|1", "a|1|1", "c|4|1", "c|3|1"); + FakePrimitive fp = new FakePrimitive(Lists.newArrayList(resultSet)); + + OrderByParamsGen4 orderByParamsGen4 = new OrderByParamsGen4(1, false, -1, null); + MemorySortGen4Engine ms = new MemorySortGen4Engine(Lists.newArrayList(orderByParamsGen4), fp); + ms.setTruncateColumnCount(2); + + VtResultSet result = (VtResultSet) ms.execute(VtContext.background(), new NoopVCursor(), null, false).getVtRowList(); + VtResultSet wantResult = TestResult.makeTestResult(new Query.Field[] {fields[0], fields[1]}, "a|1", "a|1", "g|2", "c|3", "c|4"); + Assert.assertEquals(wantResult, result); + } + + public void testMemorySortStreamExecuteTruncate() throws SQLException { + + } + + @Test + public void testMemorySortMultiColumn() throws SQLException { + Query.Field[] fields = TestResult.makeTestFields("c1|c2", "varbinary|decimal"); + VtResultSet resultSet = TestResult.makeTestResult(fields, "a|1", "b|2", "b|1", "c|4", "c|3"); + FakePrimitive fp = new FakePrimitive(Lists.newArrayList(resultSet)); + + OrderByParamsGen4 orderByParams1 = new OrderByParamsGen4(1, false, -1, null); + OrderByParamsGen4 orderByParams2 = new OrderByParamsGen4(0, true, -1, null); + MemorySortGen4Engine ms = new MemorySortGen4Engine(Lists.newArrayList(orderByParams1, orderByParams2), fp); + + VtResultSet result = (VtResultSet) ms.execute(VtContext.background(), new NoopVCursor(), null, false).getVtRowList(); + VtResultSet wantResult = TestResult.makeTestResult(fields, "b|1", "a|1", "b|2", "c|3", "c|4"); + Assert.assertEquals(wantResult, result); + + fp.rewind(); + EvalEngine.BindVariable pv = new EvalEngine.BindVariable("__upper_limit"); + ms.setUpperLimit(pv); + Map bindVariableMap = new HashMap<>(); + bindVariableMap.put("__upper_limit", SqlTypes.int64BindVariable(3L)); + result = (VtResultSet) ms.execute(VtContext.background(), new NoopVCursor(), bindVariableMap, false).getVtRowList(); + wantResult = TestResult.makeTestResult(fields, "b|1", "a|1", "b|2"); + Assert.assertEquals(wantResult, result); + } + + public void testMemorySortMaxMemoryRows() throws SQLException { + + } + + @Test + @Ignore + public void TestMemorySortExecuteNoVarChar() throws SQLException { + Query.Field[] fields = TestResult.makeTestFields("c1|c2", "varchar|decimal"); + VtResultSet resultSet = TestResult.makeTestResult(fields, "a|1", "b|2", "a|1", "c|4", "c|3"); + FakePrimitive fp = new FakePrimitive(Lists.newArrayList(resultSet)); + + OrderByParamsGen4 orderByParams = new OrderByParamsGen4(1, false, -1, null); + MemorySortGen4Engine ms = new MemorySortGen4Engine(Lists.newArrayList(orderByParams), fp); + + VtResultSet result = (VtResultSet) ms.execute(VtContext.background(), new NoopVCursor(), null, false).getVtRowList(); + String want = "cannot compare strings, collation is unknown or unsupported (collation ID: 0)"; + thrown.expect(SQLException.class); + thrown.expectMessage(want); + } +} \ No newline at end of file diff --git a/src/test/java/com/jd/jdbc/engine/gen4/OrderedAggregateGen4EngineTest.java b/src/test/java/com/jd/jdbc/engine/gen4/OrderedAggregateGen4EngineTest.java new file mode 100644 index 0000000..5438524 --- /dev/null +++ b/src/test/java/com/jd/jdbc/engine/gen4/OrderedAggregateGen4EngineTest.java @@ -0,0 +1,461 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.engine.gen4; + +import com.google.common.collect.Lists; +import com.jd.BaseTest; +import com.jd.jdbc.common.tuple.Pair; +import com.jd.jdbc.context.VtContext; +import com.jd.jdbc.engine.Engine; +import com.jd.jdbc.engine.FakePrimitive; +import com.jd.jdbc.engine.util.TestResult; +import com.jd.jdbc.sqltypes.VtResultSet; +import com.jd.jdbc.sqltypes.VtResultValue; +import io.vitess.proto.Query; +import java.math.BigDecimal; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import org.junit.Assert; +import org.junit.Ignore; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +public class OrderedAggregateGen4EngineTest extends BaseTest { + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Test + public void testOrderedAggregateExecute() throws SQLException { + Query.Field[] fields = TestResult.makeTestFields("col|count(*)", "varbinary|decimal"); + VtResultSet resultSet = TestResult.makeTestResult(fields, "a|1", "a|1", "b|2", "c|3", "c|4"); + FakePrimitive fp = new FakePrimitive(Lists.newArrayList(resultSet)); + + AbstractAggregateGen4.AggregateParams aggr = new AbstractAggregateGen4.AggregateParams(Engine.AggregateOpcodeG4.AggregateSum, 1); + GroupByParams groupByParams = new GroupByParams(); + groupByParams.setKeyCol(0); + OrderedAggregateGen4Engine oa = new OrderedAggregateGen4Engine(false, Lists.newArrayList(aggr), false, 0, Lists.newArrayList(groupByParams), null, fp); + + VtResultSet result = (VtResultSet) oa.execute(VtContext.background(), null, null, false).getVtRowList(); + + VtResultSet wantResult = TestResult.makeTestResult(fields, "a|2", "b|2", "c|7"); + Assert.assertEquals(wantResult, result); + } + + @Test + public void testOrderedAggregateExecuteTruncate() throws SQLException { + Query.Field[] fields = TestResult.makeTestFields("col|count(*)|weight_string(col)", "varchar|decimal|varbinary"); + VtResultSet resultSet = TestResult.makeTestResult(fields, "a|1|A", "A|1|A", "b|2|B", "C|3|C", "c|4|C"); + FakePrimitive fp = new FakePrimitive(Lists.newArrayList(resultSet)); + + AbstractAggregateGen4.AggregateParams aggr = new AbstractAggregateGen4.AggregateParams(Engine.AggregateOpcodeG4.AggregateSum, 1); + GroupByParams groupByParams = new GroupByParams(); + groupByParams.setKeyCol(2); + OrderedAggregateGen4Engine oa = new OrderedAggregateGen4Engine(false, Lists.newArrayList(aggr), false, 2, Lists.newArrayList(groupByParams), null, fp); + + VtResultSet result = (VtResultSet) oa.execute(VtContext.background(), null, null, false).getVtRowList(); + + Query.Field[] wantFields = TestResult.makeTestFields("col|count(*)", "varchar|decimal"); + VtResultSet wantResult = TestResult.makeTestResult(wantFields, "a|2", "b|2", "C|7"); + Assert.assertEquals(wantResult, result); + } + + public void testOrderedAggregateStreamExecute() { + // todo + } + + public void testOrderedAggregateStreamExecuteTruncate() { + // todo + } + + @Test + public void testOrderedAggregateGetFields() throws SQLException { + VtResultSet input = TestResult.makeTestResult(TestResult.makeTestFields("col|count(*)", "varbinary|decimal")); + FakePrimitive fp = new FakePrimitive(Lists.newArrayList(input)); + + OrderedAggregateGen4Engine oa = new OrderedAggregateGen4Engine(false, null, false, 0, null, null, fp); + VtResultSet got = oa.getFields(null, null); + + Assert.assertEquals(got, input); + } + + @Test + public void testOrderedAggregateGetFieldsTruncate() throws SQLException { + VtResultSet input = TestResult.makeTestResult(TestResult.makeTestFields("col|count(*)|weight_string(col)", "varchar|decimal|varbinary")); + FakePrimitive fp = new FakePrimitive(Lists.newArrayList(input)); + + OrderedAggregateGen4Engine oa = new OrderedAggregateGen4Engine(false, null, false, 2, null, null, fp); + VtResultSet got = oa.getFields(null, null); + VtResultSet wantResult = TestResult.makeTestResult(TestResult.makeTestFields("col|count(*)", "varchar|decimal")); + Assert.assertEquals(got, wantResult); + } + + @Test + public void testOrderedAggregateInputFail() throws SQLException { + FakePrimitive fp = new FakePrimitive(new SQLException("input fail")); + OrderedAggregateGen4Engine oa = new OrderedAggregateGen4Engine(false, null, false, 0, null, null, fp); + + String want = "input fail"; + thrown.expect(SQLException.class); + thrown.expectMessage(want); + oa.execute(VtContext.background(), null, null, false); + + fp.rewind(); + thrown.expect(SQLException.class); + thrown.expectMessage(want); + oa.getFields(null, null); + } + + @Test + public void testOrderedAggregateExecuteCountDistinct() throws SQLException { + VtResultSet input = TestResult.makeTestResult(TestResult.makeTestFields("col1|col2|count(*)", "varbinary|decimal|int64"), + // Two identical values + "a|1|1", + "a|1|2", + // Single value + "b|1|1", + // Two different values + "c|3|1", + "c|4|1", + // Single null + "d|null|1", + // Start with null + "e|null|1", + "e|1|1", + // Null comes after first + "f|1|1", + "f|null|1", + // Identical to non-identical transition + "g|1|1", + "g|1|1", + "g|2|1", + "g|3|1", + // Non-identical to identical transition + "h|1|1", + "h|2|1", + "h|2|1", + "h|3|1", + // Key transition, should still count 3 + "i|3|1", + "i|4|1"); + FakePrimitive fp = new FakePrimitive(Lists.newArrayList(input)); + + AbstractAggregateGen4.AggregateParams aggr1 = new AbstractAggregateGen4.AggregateParams(Engine.AggregateOpcodeG4.AggregateCountDistinct, 1, "count(distinct col2)"); + AbstractAggregateGen4.AggregateParams aggr2 = new AbstractAggregateGen4.AggregateParams(Engine.AggregateOpcodeG4.AggregateSum, 2); + GroupByParams groupByParams = new GroupByParams(); + groupByParams.setKeyCol(0); + OrderedAggregateGen4Engine oa = new OrderedAggregateGen4Engine(true, Lists.newArrayList(aggr1, aggr2), false, 0, Lists.newArrayList(groupByParams), null, fp); + VtResultSet result = (VtResultSet) oa.execute(VtContext.background(), null, null, false).getVtRowList(); + + VtResultSet wantResult = TestResult.makeTestResult(TestResult.makeTestFields("col1|count(distinct col2)|count(*)", "varbinary|int64|int64"), "a|1|3", + "b|1|1", + "c|2|2", + "d|0|1", + "e|1|2", + "f|1|2", + "g|3|4", + "h|3|4", + "i|2|2"); + + Assert.assertEquals(wantResult, result); + } + + public void testOrderedAggregateStreamCountDistinct() { + // todo + } + + @Test + public void testOrderedAggregateSumDistinctGood() throws SQLException { + VtResultSet input = TestResult.makeTestResult(TestResult.makeTestFields("col1|col2|sum(col3)", "varbinary|int64|decimal"), + // Two identical values + "a|1|1", + "a|1|2", + // Single value + "b|1|1", + // Two different values + "c|3|1", + "c|4|1", + // Single null + "d|null|1", + "d|1|1", + // Start with null + "e|null|1", + "e|1|1", + // Null comes after first + "f|1|1", + "f|null|1", + // Identical to non-identical transition + "g|1|1", + "g|1|1", + "g|2|1", + "g|3|1", + // Non-identical to identical transition + "h|1|1", + "h|2|1", + "h|2|1", + "h|3|1", + // Key transition, should still count 3 + "i|3|1", + "i|4|1"); + FakePrimitive fp = new FakePrimitive(Lists.newArrayList(input)); + + AbstractAggregateGen4.AggregateParams aggr1 = new AbstractAggregateGen4.AggregateParams(Engine.AggregateOpcodeG4.AggregateSumDistinct, 1, "sum(distinct col2)"); + AbstractAggregateGen4.AggregateParams aggr2 = new AbstractAggregateGen4.AggregateParams(Engine.AggregateOpcodeG4.AggregateSum, 2); + GroupByParams groupByParams = new GroupByParams(); + groupByParams.setKeyCol(0); + OrderedAggregateGen4Engine oa = new OrderedAggregateGen4Engine(true, Lists.newArrayList(aggr1, aggr2), false, 0, Lists.newArrayList(groupByParams), null, fp); + VtResultSet result = (VtResultSet) oa.execute(VtContext.background(), null, null, false).getVtRowList(); + + VtResultSet wantResult = TestResult.makeTestResult(TestResult.makeTestFields("col1|sum(distinct col2)|sum(col3)", "varbinary|decimal|decimal"), "a|1|3", + "b|1|1", + "c|7|2", + "d|1|2", + "e|1|2", + "f|1|2", + "g|6|4", + "h|6|4", + "i|7|2"); + Assert.assertEquals(wantResult, result); + } + + @Test + public void testOrderedAggregateSumDistinctTolerateError() throws SQLException { + VtResultSet input = TestResult.makeTestResult(TestResult.makeTestFields("col1|col2", "varbinary|varbinary"), + "a|aaa", + "a|0", + "a|1"); + FakePrimitive fp = new FakePrimitive(Lists.newArrayList(input)); + + AbstractAggregateGen4.AggregateParams aggr1 = new AbstractAggregateGen4.AggregateParams(Engine.AggregateOpcodeG4.AggregateSumDistinct, 1, "sum(distinct col2)"); + GroupByParams groupByParams = new GroupByParams(); + groupByParams.setKeyCol(0); + OrderedAggregateGen4Engine oa = new OrderedAggregateGen4Engine(true, Lists.newArrayList(aggr1), false, 0, Lists.newArrayList(groupByParams), null, fp); + VtResultSet result = (VtResultSet) oa.execute(VtContext.background(), null, null, false).getVtRowList(); + + VtResultSet wantResult = TestResult.makeTestResult(TestResult.makeTestFields("col1|sum(distinct col2)", "varbinary|decimal"), "a|1"); + Assert.assertEquals(wantResult, result); + } + + @Test + @Ignore + public void testOrderedAggregateKeysFail() throws SQLException { + VtResultSet input = TestResult.makeTestResult(TestResult.makeTestFields("col|count(*)", "varchar|decimal"), "a|1", "a|1"); + FakePrimitive fp = new FakePrimitive(Lists.newArrayList(input)); + + AbstractAggregateGen4.AggregateParams aggr1 = new AbstractAggregateGen4.AggregateParams(Engine.AggregateOpcodeG4.AggregateSum, 1); + GroupByParams groupByParams = new GroupByParams(); + groupByParams.setKeyCol(0); + OrderedAggregateGen4Engine oa = new OrderedAggregateGen4Engine(false, Lists.newArrayList(aggr1), false, 0, Lists.newArrayList(groupByParams), null, fp); + + String want = "cannot compare strings, collation is unknown or unsupported (collation ID: 0)"; + thrown.expect(SQLException.class); + thrown.expectMessage(want); + + oa.execute(VtContext.background(), null, null, false); + + fp.rewind(); + } + + @Test + public void testOrderedAggregateMergeFail() throws SQLException { + VtResultSet input = TestResult.makeTestResult(TestResult.makeTestFields("col|count(*)", "varbinary|decimal"), "a|1", "a|0"); + FakePrimitive fp = new FakePrimitive(Lists.newArrayList(input)); + + AbstractAggregateGen4.AggregateParams aggr1 = new AbstractAggregateGen4.AggregateParams(Engine.AggregateOpcodeG4.AggregateSum, 1); + GroupByParams groupByParams = new GroupByParams(); + groupByParams.setKeyCol(0); + OrderedAggregateGen4Engine oa = new OrderedAggregateGen4Engine(false, Lists.newArrayList(aggr1), false, 0, Lists.newArrayList(groupByParams), null, fp); + + Query.Field field1 = Query.Field.newBuilder().setName("col").setType(Query.Type.VARBINARY).build(); + Query.Field field2 = Query.Field.newBuilder().setName("count(*)").setType(Query.Type.DECIMAL).build(); + + VtResultValue value1 = new VtResultValue("a", Query.Type.VARBINARY); + VtResultValue value2 = new VtResultValue(BigDecimal.valueOf(1), Query.Type.DECIMAL); + List vtResultValues = Lists.newArrayList(value1, value2); + List> rows = new ArrayList<>(); + rows.add(vtResultValues); + VtResultSet result = new VtResultSet(new Query.Field[] {field1, field2}, rows); + + VtResultSet res = (VtResultSet) oa.execute(VtContext.background(), null, null, false).getVtRowList(); + + Assert.assertEquals(result, res); + + fp.rewind(); + } + + @Test + public void testMerge() throws SQLException { + AbstractAggregateGen4.AggregateParams aggr1 = new AbstractAggregateGen4.AggregateParams(Engine.AggregateOpcodeG4.AggregateSum, 1); + AbstractAggregateGen4.AggregateParams aggr2 = new AbstractAggregateGen4.AggregateParams(Engine.AggregateOpcodeG4.AggregateSum, 2); + AbstractAggregateGen4.AggregateParams aggr3 = new AbstractAggregateGen4.AggregateParams(Engine.AggregateOpcodeG4.AggregateMin, 3); + AbstractAggregateGen4.AggregateParams aggr4 = new AbstractAggregateGen4.AggregateParams(Engine.AggregateOpcodeG4.AggregateMax, 4); + OrderedAggregateGen4Engine oa = new OrderedAggregateGen4Engine(false, Lists.newArrayList(aggr1, aggr2, aggr3, aggr4), false, 0, null, null, null); + + Query.Field[] fields = TestResult.makeTestFields("a|b|c|d|e", "int64|int64|decimal|int32|varbinary"); + VtResultSet r = TestResult.makeTestResult(fields, "1|2|3.2|3|ab", "1|3|2.8|2|bc"); + + Pair, List> pair = oa.merge(fields, r.getRows().get(0), r.getRows().get(1), null, null, oa.getAggregates()); + List merged = pair.getLeft(); + List want = TestResult.makeTestResult(fields, "1|5|6.0|2|bc").getRows().get(0); + Assert.assertEquals(want, merged); + + // swap and retry + pair = oa.merge(fields, r.getRows().get(1), r.getRows().get(0), null, null, oa.getAggregates()); + merged = pair.getLeft(); + Assert.assertEquals(want, merged); + } + + public void testOrderedAggregateExecuteGtid() { + // todo + } + + @Test + public void testCountDistinctOnVarchar() throws SQLException { + Query.Field[] fields = TestResult.makeTestFields("c1|c2|weight_string(c2)", "int64|varchar|varbinary"); + VtResultSet r = TestResult.makeTestResult(fields, "10|a|0x41", "10|a|0x41", "10|b|0x42", "20|b|0x42"); + FakePrimitive fp = new FakePrimitive(Lists.newArrayList(r)); + + AbstractAggregateGen4.AggregateParams aggr1 = new AbstractAggregateGen4.AggregateParams(Engine.AggregateOpcodeG4.AggregateCountDistinct, 1, "count(distinct c2)"); + aggr1.setWCol(2); + aggr1.setWAssigned(true); + GroupByParams groupByParams = new GroupByParams(); + groupByParams.setKeyCol(0); + OrderedAggregateGen4Engine oa = new OrderedAggregateGen4Engine(true, Lists.newArrayList(aggr1), false, 2, Lists.newArrayList(groupByParams), null, fp); + + VtResultSet want = TestResult.makeTestResult(TestResult.makeTestFields("c1|count(distinct c2)", "int64|int64"), "10|2", "20|1"); + + VtResultSet qr = (VtResultSet) oa.execute(VtContext.background(), null, null, false).getVtRowList(); + Assert.assertEquals(want, qr); + } + + @Test + public void testCountDistinctOnVarcharWithNulls() throws SQLException { + Query.Field[] fields = TestResult.makeTestFields("c1|c2|weight_string(c2)", "int64|varchar|varbinary"); + VtResultSet r = TestResult.makeTestResult(fields, "null|null|null", + "null|a|0x41", + "null|b|0x42", + "10|null|null", + "10|null|null", + "10|a|0x41", + "10|a|0x41", + "10|b|0x42", + "20|null|null", + "20|b|0x42", + "30|null|null", + "30|null|null", + "30|null|null", + "30|null|null"); + FakePrimitive fp = new FakePrimitive(Lists.newArrayList(r)); + + AbstractAggregateGen4.AggregateParams aggr1 = new AbstractAggregateGen4.AggregateParams(Engine.AggregateOpcodeG4.AggregateCountDistinct, 1, "count(distinct c2)"); + aggr1.setWCol(2); + aggr1.setWAssigned(true); + GroupByParams groupByParams = new GroupByParams(); + groupByParams.setKeyCol(0); + OrderedAggregateGen4Engine oa = new OrderedAggregateGen4Engine(true, Lists.newArrayList(aggr1), false, 2, Lists.newArrayList(groupByParams), null, fp); + + VtResultSet want = TestResult.makeTestResult(TestResult.makeTestFields("c1|count(distinct c2)", "int64|int64"), "null|2", "10|2", "20|1", "30|0"); + + VtResultSet qr = (VtResultSet) oa.execute(VtContext.background(), null, null, false).getVtRowList(); + Assert.assertEquals(want, qr); + } + + /** + * 涉及到对字符串与null调用com.jd.jdbc.evalengine.EvalEngine#nullSafeAdd方法,暂不支持 + * + * @throws SQLException + */ + @Test + @Ignore + public void testSumDistinctOnVarcharWithNulls() throws SQLException { + Query.Field[] fields = TestResult.makeTestFields("c1|c2|weight_string(c2)", "int64|varchar|varbinary"); + VtResultSet r = TestResult.makeTestResult(fields, "null|null|null", + "null|a|0x41", + "null|b|0x42", + "10|null|null", + "10|null|null", + "10|a|0x41", + "10|a|0x41", + "10|b|0x42", + "20|null|null", + "20|b|0x42", + "30|null|null", + "30|null|null", + "30|null|null", + "30|null|null"); + FakePrimitive fp = new FakePrimitive(Lists.newArrayList(r)); + + AbstractAggregateGen4.AggregateParams aggr1 = new AbstractAggregateGen4.AggregateParams(Engine.AggregateOpcodeG4.AggregateSumDistinct, 1, "sum(distinct c2)"); + aggr1.setWCol(2); + aggr1.setWAssigned(true); + GroupByParams groupByParams = new GroupByParams(); + groupByParams.setKeyCol(0); + OrderedAggregateGen4Engine oa = new OrderedAggregateGen4Engine(true, Lists.newArrayList(aggr1), false, 2, Lists.newArrayList(groupByParams), null, fp); + + VtResultSet want = TestResult.makeTestResult(TestResult.makeTestFields("c1|sum(distinct c2)", "int64|decimal"), "null|0", "10|0", "20|0", "30|null"); + + VtResultSet qr = (VtResultSet) oa.execute(VtContext.background(), null, null, false).getVtRowList(); + Assert.assertEquals(want, qr); + } + + @Test + public void testMultiDistinct() throws SQLException { + Query.Field[] fields = TestResult.makeTestFields("c1|c2|c3", "int64|int64|int64"); + VtResultSet r = TestResult.makeTestResult(fields, "null|null|null", + "null|1|2", + "null|2|2", + "10|null|null", + "10|2|null", + "10|2|1", + "10|2|3", + "10|3|3", + "20|null|null", + "20|null|null", + "30|1|1", + "30|1|2", + "30|1|3", + "40|1|1", + "40|2|1", + "40|3|1"); + FakePrimitive fp = new FakePrimitive(Lists.newArrayList(r)); + + AbstractAggregateGen4.AggregateParams aggr1 = new AbstractAggregateGen4.AggregateParams(Engine.AggregateOpcodeG4.AggregateCountDistinct, 1, "count(distinct c2)"); + AbstractAggregateGen4.AggregateParams aggr2 = new AbstractAggregateGen4.AggregateParams(Engine.AggregateOpcodeG4.AggregateSumDistinct, 2, "sum(distinct c3)"); + GroupByParams groupByParams = new GroupByParams(); + groupByParams.setKeyCol(0); + OrderedAggregateGen4Engine oa = new OrderedAggregateGen4Engine(true, Lists.newArrayList(aggr1, aggr2), false, 0, Lists.newArrayList(groupByParams), null, fp); + + VtResultSet want = TestResult.makeTestResult(TestResult.makeTestFields("c1|count(distinct c2)|sum(distinct c3)", "int64|int64|decimal"), "null|2|2", "10|2|4", "20|0|null", "30|1|6", "40|3|1"); + VtResultSet qr = (VtResultSet) oa.execute(VtContext.background(), null, null, false).getVtRowList(); + Assert.assertEquals(want, qr); + } + + public void testOrderedAggregateCollate() { + + } + + public void testOrderedAggregateCollateAS() { + + } + + public void testOrderedAggregateCollateKS() { + + } +} diff --git a/src/test/java/com/jd/jdbc/engine/gen4/RouteGen4EngineTest.java b/src/test/java/com/jd/jdbc/engine/gen4/RouteGen4EngineTest.java new file mode 100644 index 0000000..91b06bc --- /dev/null +++ b/src/test/java/com/jd/jdbc/engine/gen4/RouteGen4EngineTest.java @@ -0,0 +1,471 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.engine.gen4; + +import com.google.common.collect.Lists; +import com.jd.BaseTest; +import com.jd.jdbc.IExecute; +import com.jd.jdbc.context.VtContext; +import com.jd.jdbc.engine.Engine; +import com.jd.jdbc.engine.util.TestResult; +import com.jd.jdbc.engine.vcursor.LoggingVCursor; +import com.jd.jdbc.evalengine.EvalEngine; +import com.jd.jdbc.evalengine.EvalResult; +import com.jd.jdbc.sqlparser.dialect.mysql.ast.statement.MySqlSelectQueryBlock; +import com.jd.jdbc.sqltypes.VtResultSet; +import com.jd.jdbc.sqltypes.VtRowList; +import com.jd.jdbc.vindexes.VKeyspace; +import com.jd.jdbc.vindexes.hash.Binary; +import com.jd.jdbc.vindexes.hash.BinaryHash; +import io.vitess.proto.Query; +import java.nio.charset.StandardCharsets; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import lombok.AllArgsConstructor; +import org.junit.Assert; +import org.junit.Ignore; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import vschema.Vschema; + +public class RouteGen4EngineTest extends BaseTest { + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + private final VtResultSet defaultSelectResult = TestResult.makeTestResult(TestResult.makeTestFields("id", "int64"), "1"); + + @Test + public void testSelectUnsharded() throws SQLException { + RouteGen4Engine sel = new RouteGen4Engine(Engine.RouteOpcode.SelectUnsharded, new VKeyspace("ks", false), "select", "select_field", new MySqlSelectQueryBlock()); + LoggingVCursor vc = new LoggingVCursor(Lists.newArrayList("0"), Lists.newArrayList(defaultSelectResult)); + IExecute.ExecuteMultiShardResponse result = sel.execute(VtContext.background(), vc, new HashMap<>(), false); + List wants = Lists.newArrayList("ResolveDestinations ks [] Destinations:DestinationAllShard()", "ExecuteMultiShard ks.0: select {} false false"); + vc.expectLog(wants); + Assert.assertEquals(printFail("testSelectUnsharded is FAIL"), result.getVtRowList(), defaultSelectResult); + +// vc.Rewind() +// result, err = wrapStreamExecute(sel, vc, map[string]*querypb.BindVariable{}, false) +// require.NoError(t, err) +// vc.ExpectLog(t, []string{ +// `ResolveDestinations ks [] Destinations:DestinationAllShards()`, +// `StreamExecuteMulti dummy_select ks.0: {} `, +// }) +// expectResult(t, "sel.StreamExecute", result, defaultSelectResult) + } + + @Ignore + @Test + public void testInformationSchemaWithTableAndSchemaWithRoutedTables() throws SQLException { + @AllArgsConstructor + class TestCase { + String testName; + + List tableSchema; + + Map tableName; + + boolean routed; + + List expectedLog; + } + + List testCases = new ArrayList<>(); + testCases.add(new TestCase("both schema and table predicates - routed table", Lists.newArrayList("schema"), newTableName("table_name", "table"), true, + Lists.newArrayList("FindTable(`schema`.`table`)", "ResolveDestinations routedKeyspace [] Destinations:DestinationAnyShard()", + "ExecuteMultiShard routedKeyspace.1: select {__replacevtschemaname: type:INT64 value:\"1\" table_name: type:VARCHAR value:\"routedTable\"} false false"))); + for (TestCase tc : testCases) { + RouteGen4Engine sel = new RouteGen4Engine(Engine.RouteOpcode.SelectDBA, new VKeyspace("ks", false), "select", "select_field", new MySqlSelectQueryBlock()); + sel.getRoutingParameters().setSystableTableSchema(stringListToExprList(tc.tableSchema)); + sel.getRoutingParameters().setSystableTableName(tc.tableName); + LoggingVCursor vc = new LoggingVCursor(Lists.newArrayList("1"), Lists.newArrayList(defaultSelectResult)); + if (tc.routed) { + Vschema.Table tbl = Vschema.Table.newBuilder().build(); + vc.setTableRoutes(tbl); + } + sel.execute(VtContext.background(), vc, null, false); + vc.expectLog(tc.expectedLog); + } + } + + private List stringListToExprList(List in) { + List schema = new ArrayList<>(); + for (String s : in) { + schema.add(newLiteralString(s)); + } + return schema; + } + + public Map newTableName(String table, String val) { + Map res = new HashMap<>(); + res.put(table, newLiteralString(val)); + return res; + } + + public EvalEngine.Literal newLiteralString(String t) { + byte[] val = t.getBytes(StandardCharsets.UTF_8); + return new EvalEngine.Literal(new EvalResult(val, Query.Type.VARBINARY)); + } + + @Test + public void testSelectScatter() throws SQLException { + RouteGen4Engine sel = new RouteGen4Engine(Engine.RouteOpcode.SelectScatter, new VKeyspace("ks", true), "select", "select_field", new MySqlSelectQueryBlock()); + LoggingVCursor vc = new LoggingVCursor(Lists.newArrayList("-20", "20-"), Lists.newArrayList(defaultSelectResult)); + IExecute.ExecuteMultiShardResponse result = sel.execute(VtContext.background(), vc, null, false); + List wants = Lists.newArrayList("ResolveDestinations ks [] Destinations:DestinationAllShard()", "ExecuteMultiShard ks.-20: select {} ks.20-: select {} false false"); + vc.expectLog(wants); + Assert.assertEquals(printFail(" testSelectScatter is FAIL"), result.getVtRowList(), defaultSelectResult); + + //流式查询 +// vc.Rewind() +// result, err = wrapStreamExecute(sel, vc, map[string]*querypb.BindVariable{}, false) +// require.NoError(t, err) +// vc.ExpectLog(t, []string{ +// `ResolveDestinations ks [] Destinations:DestinationAllShards()`, +// `StreamExecuteMulti dummy_select ks.-20: {} ks.20-: {} `, +// }) +// expectResult(t, "sel.StreamExecute", result, defaultSelectResult) + } + + @Test + public void testSelectEqualUnique() throws SQLException { + RouteGen4Engine sel = new RouteGen4Engine(Engine.RouteOpcode.SelectEqualUnique, new VKeyspace("ks", true), "select", "select_field", new MySqlSelectQueryBlock()); + sel.getRoutingParameters().setVindex(new BinaryHash()); + sel.getRoutingParameters().setValues(Lists.newArrayList(EvalEngine.newLiteralInt(1L))); + + LoggingVCursor vc = new LoggingVCursor(Lists.newArrayList("-20", "20-"), Lists.newArrayList(defaultSelectResult)); + IExecute.ExecuteMultiShardResponse result = sel.execute(VtContext.background(), vc, null, false); + List wants = + Lists.newArrayList("ResolveDestinations ks [type:INT64 value:\"1\"] Destinations:DestinationKeyspaceID(166b40b44aba4bd6)", "ExecuteMultiShard ks.-20: select {} false false"); + vc.expectLog(wants); + Assert.assertEquals(printFail(" testSelectEqualUniqueScatter is FAIL"), defaultSelectResult, result.getVtRowList()); + //流式查询 + /* + vc.Rewind() + result, err = wrapStreamExecute(sel, vc, map[string]*querypb.BindVariable{}, false) + require.NoError(t, err) + vc.ExpectLog(t, []string{ + `ResolveDestinations ks [type:INT64 value:"1"] Destinations:DestinationKeyspaceID(166b40b44aba4bd6)`, + `StreamExecuteMulti dummy_select ks.-20: {} `, + }) + expectResult(t, "sel.StreamExecute", result, defaultSelectResult) + + */ + } + + @Test + public void testSelectNone() throws SQLException { + RouteGen4Engine sel = new RouteGen4Engine(Engine.RouteOpcode.SelectNone, new VKeyspace("ks", true), "select", "select_field", new MySqlSelectQueryBlock()); + sel.getRoutingParameters().setVindex(new Binary()); + LoggingVCursor vc = new LoggingVCursor(Lists.newArrayList("-20", "20-"), new ArrayList<>()); + IExecute.ExecuteMultiShardResponse result = sel.execute(VtContext.background(), vc, null, false); + Assert.assertEquals(printFail(" testSelectNone is FAIL"), result.getVtRowList(), new VtResultSet()); + + vc.rewind(); + // test with special no-routes handling + sel.setNoRoutesSpecialHandling(true); + result = sel.execute(VtContext.background(), vc, null, false); + vc.expectLog( + Lists.newArrayList("ResolveDestinations ks [] Destinations:DestinationAnyShard()", "ExecuteMultiShard ks.-20: select {} false false") + ); + Assert.assertEquals(printFail(" testSelectNone is FAIL"), result.getVtRowList(), new VtResultSet()); + +// vc.Rewind() +// result, err = wrapStreamExecute(sel, vc, map[string]*querypb.BindVariable{}, false) +// require.NoError(t, err) +// vc.ExpectLog(t, []string{ +// `ResolveDestinations ks [] Destinations:DestinationAnyShard()`, +// `StreamExecuteMulti dummy_select ks.-20: {} `, +// }) +// expectResult(t, "sel.StreamExecute", result, &sqltypes.Result{}) + } + + @Test + public void testSelectEqualUniqueScatter() throws SQLException { + RouteGen4Engine sel = new RouteGen4Engine(Engine.RouteOpcode.SelectEqualUnique, new VKeyspace("ks", true), "select", "select_field", new MySqlSelectQueryBlock()); + sel.getRoutingParameters().setVindex(new BinaryHash()); + sel.getRoutingParameters().setValues(Lists.newArrayList(EvalEngine.newLiteralInt(1L))); + LoggingVCursor vc = new LoggingVCursor(Lists.newArrayList("-20", "20-"), Lists.newArrayList(defaultSelectResult)); + vc.setShardForKsid(Lists.newArrayList("-20", "20-")); + IExecute.ExecuteMultiShardResponse result = sel.execute(VtContext.background(), vc, null, false); + vc.expectLog( + Lists.newArrayList("ResolveDestinations ks [type:INT64 value:\"1\"] Destinations:DestinationKeyspaceID(166b40b44aba4bd6)", "ExecuteMultiShard ks.-20: select {} false false") + ); + Assert.assertEquals(printFail(" testSelectEqualUniqueScatter is FAIL"), result.getVtRowList(), defaultSelectResult); + +// vc.Rewind() +// result, err = wrapStreamExecute(sel, vc, map[string]*querypb.BindVariable{}, false) +// require.NoError(t, err) +// vc.ExpectLog(t, []string{ +// `ResolveDestinations ks [type:INT64 value:"1"] Destinations:DestinationKeyRange(-)`, +// `StreamExecuteMulti dummy_select ks.-20: {} ks.20-: {} `, +// }) +// expectResult(t, "sel.StreamExecute", result, defaultSelectResult) + } + + @Test + public void testSelectEqual() throws SQLException { + RouteGen4Engine sel = new RouteGen4Engine(Engine.RouteOpcode.SelectEqual, new VKeyspace("ks", true), "select", "select_field", new MySqlSelectQueryBlock()); + sel.getRoutingParameters().setVindex(new BinaryHash()); + sel.getRoutingParameters().setValues(Lists.newArrayList(EvalEngine.newLiteralInt(1L))); + LoggingVCursor vc = new LoggingVCursor(Lists.newArrayList("-20", "20-"), Lists.newArrayList(defaultSelectResult)); + IExecute.ExecuteMultiShardResponse result = sel.execute(VtContext.background(), vc, null, false); + vc.expectLog( + Lists.newArrayList("ResolveDestinations ks [type:INT64 value:\"1\"] Destinations:DestinationKeyspaceID(166b40b44aba4bd6)", "ExecuteMultiShard ks.-20: select {} false false") + ); + Assert.assertEquals(printFail(" TestSelectEqual is FAIL"), result.getVtRowList(), defaultSelectResult); + +// vc.Rewind() +// result, err = wrapStreamExecute(sel, vc, map[string]*querypb.BindVariable{}, false) +// require.NoError(t, err) +// vc.ExpectLog(t, []string{ +// `ResolveDestinations ks [type:INT64 value:"1"] Destinations:DestinationKeyRange(-)`, +// `StreamExecuteMulti dummy_select ks.-20: {} ks.20-: {} `, +// }) +// expectResult(t, "sel.StreamExecute", result, defaultSelectResult) + } + + @Test + public void testSelectEqualNoRoute() throws SQLException { + RouteGen4Engine sel = new RouteGen4Engine(Engine.RouteOpcode.SelectEqual, new VKeyspace("ks", true), "select", "select_field", new MySqlSelectQueryBlock()); + sel.getRoutingParameters().setVindex(new BinaryHash()); + sel.getRoutingParameters().setValues(Lists.newArrayList(EvalEngine.newLiteralInt(1L))); + LoggingVCursor vc = new LoggingVCursor(Lists.newArrayList("-20", "20-"), null); + IExecute.ExecuteMultiShardResponse result = sel.execute(VtContext.background(), vc, null, false); + vc.expectLog( + Lists.newArrayList("ResolveDestinations ks [type:INT64 value:\"1\"] Destinations:DestinationKeyspaceID(166b40b44aba4bd6)", "ExecuteMultiShard ks.-20: select {} false false") + ); + Assert.assertEquals(printFail(" testSelectEqualNoRoute is FAIL"), result.getVtRowList(), new VtResultSet()); + } + + @Test + public void testINUnique() throws SQLException { + RouteGen4Engine sel = new RouteGen4Engine(Engine.RouteOpcode.SelectIN, new VKeyspace("ks", true), "select", "select_field", new MySqlSelectQueryBlock()); + sel.getRoutingParameters().setVindex(new BinaryHash()); + sel.getRoutingParameters() + .setValues(Lists.newArrayList(new EvalEngine.TupleExpr(Lists.newArrayList(EvalEngine.newLiteralInt(1L), EvalEngine.newLiteralInt(2L), EvalEngine.newLiteralInt(4L))))); + LoggingVCursor vc = new LoggingVCursor(Lists.newArrayList("-20", "20-"), Lists.newArrayList(defaultSelectResult)); + vc.setShardForKsid(Lists.newArrayList("-20", "-20", "20-")); + IExecute.ExecuteMultiShardResponse result = sel.execute(VtContext.background(), vc, new HashMap<>(), false); + vc.expectLog(Lists.newArrayList( + "ResolveDestinations ks [type:INT64 value:\"1\" type:INT64 value:\"2\" type:INT64 value:\"4\"] Destinations:DestinationKeyspaceID(166b40b44aba4bd6),DestinationKeyspaceID(06e7ea22ce92708f),DestinationKeyspaceID(d2fd8867d50d2dfe)", + "ExecuteMultiShard ks.-20: select {__vals: type:TUPLE, values:[type:INT64 value:\"1\" type:INT64 value:\"2\"]} ks.20-: select {__vals: type:TUPLE, values:[type:INT64 value:\"4\"]} false false")); + Assert.assertEquals(printFail(" testINUnique is FAIL"), result.getVtRowList(), defaultSelectResult); + +// vc.Rewind() +// result, err = wrapStreamExecute(sel, vc, map[string]*querypb.BindVariable{}, false) +// require.NoError(t, err) +// vc.ExpectLog(t, []string{ +// `ResolveDestinations ks [type:INT64 value:"1"] Destinations:DestinationKeyRange(-)`, +// `StreamExecuteMulti dummy_select ks.-20: {} ks.20-: {} `, +// }) +// expectResult(t, "sel.StreamExecute", result, defaultSelectResult) + } + + @Test + public void testINNonUnique() throws SQLException { + RouteGen4Engine sel = new RouteGen4Engine(Engine.RouteOpcode.SelectIN, new VKeyspace("ks", true), "select", "select_field", new MySqlSelectQueryBlock()); + sel.getRoutingParameters().setVindex(new BinaryHash()); + sel.getRoutingParameters() + .setValues(Lists.newArrayList(new EvalEngine.TupleExpr(Lists.newArrayList(EvalEngine.newLiteralInt(1L), EvalEngine.newLiteralInt(2L), EvalEngine.newLiteralInt(4L))))); + Query.Field[] fields = TestResult.makeTestFields("fromc|toc", "int64|varbinary"); + VtResultSet vtResultSet = TestResult.makeTestResult(fields, "1|\\x00", "1|\\x80", "2|\\x00", "4|\\x80"); + LoggingVCursor vc = new LoggingVCursor(Lists.newArrayList("-20", "20-"), Lists.newArrayList(vtResultSet, defaultSelectResult)); + IExecute.ExecuteMultiShardResponse result = sel.execute(VtContext.background(), vc, new HashMap<>(), false); + vc.expectLog(Lists.newArrayList( + "ResolveDestinations ks [type:INT64 value:\"1\" type:INT64 value:\"2\" type:INT64 value:\"4\"] Destinations:DestinationKeyspaceID(166b40b44aba4bd6),DestinationKeyspaceID(06e7ea22ce92708f),DestinationKeyspaceID(d2fd8867d50d2dfe)", + "ExecuteMultiShard ks.-20: select {__vals: type:TUPLE, values:[type:INT64 value:\"1\" type:INT64 value:\"2\" type:INT64 value:\"4\"]} false false")); + Assert.assertEquals(printFail(" testINNonUnique is FAIL"), result.getVtRowList(), vtResultSet); + } + + public void testMultiEqual() throws SQLException { + + } + + @Test + public void testSelectDBA() throws SQLException { + RouteGen4Engine sel = new RouteGen4Engine(Engine.RouteOpcode.SelectDBA, new VKeyspace("ks", true), "select", "select_field", new MySqlSelectQueryBlock()); + LoggingVCursor vc = new LoggingVCursor(Lists.newArrayList("-20", "20-"), Lists.newArrayList(defaultSelectResult)); + IExecute.ExecuteMultiShardResponse result = sel.execute(VtContext.background(), vc, new HashMap<>(), false); + vc.expectLog(Lists.newArrayList("ResolveDestinations ks [] Destinations:DestinationAnyShard()", "ExecuteMultiShard ks.-20: select {} false false")); + Assert.assertEquals(printFail(" testSelectDBA is FAIL"), result.getVtRowList(), defaultSelectResult); + +// vc.Rewind() +// result, _ = wrapStreamExecute(sel, vc, map[string]*querypb.BindVariable{}, false) +// vc.ExpectLog(t, []string{ +// `ResolveDestinations ks [] Destinations:DestinationAnyShard()`, +// `StreamExecuteMulti dummy_select ks.-20: {} `, +// }) +// expectResult(t, "sel.StreamExecute", result, defaultSelectResult) + } + + @Test + public void testSelectReference() throws SQLException { + RouteGen4Engine sel = new RouteGen4Engine(Engine.RouteOpcode.SelectReference, new VKeyspace("ks", true), "select", "select_field", new MySqlSelectQueryBlock()); + LoggingVCursor vc = new LoggingVCursor(Lists.newArrayList("-20", "20-"), Lists.newArrayList(defaultSelectResult)); + IExecute.ExecuteMultiShardResponse result = sel.execute(VtContext.background(), vc, new HashMap<>(), false); + vc.expectLog(Lists.newArrayList("ResolveDestinations ks [] Destinations:DestinationAnyShard()", "ExecuteMultiShard ks.-20: select {} false false")); + Assert.assertEquals(printFail(" testSelectReference is FAIL"), result.getVtRowList(), defaultSelectResult); + +// vc.Rewind() +// result, _ = wrapStreamExecute(sel, vc, map[string]*querypb.BindVariable{}, false) +// vc.ExpectLog(t, []string{ +// `ResolveDestinations ks [] Destinations:DestinationAnyShard()`, +// `StreamExecuteMulti dummy_select ks.-20: {} `, +// }) +// expectResult(t, "sel.StreamExecute", result, defaultSelectResult) + } + + @Test + public void testRouteGetFields() throws SQLException { + RouteGen4Engine sel = new RouteGen4Engine(Engine.RouteOpcode.SelectEqual, new VKeyspace("ks", true), "select", "select_field", new MySqlSelectQueryBlock()); + sel.getRoutingParameters().setVindex(new BinaryHash()); + sel.getRoutingParameters().setValues(Lists.newArrayList(EvalEngine.newLiteralInt(1L))); + LoggingVCursor vc = new LoggingVCursor(Lists.newArrayList("-20", "20-"), null); + + VtRowList result = sel.execute(VtContext.background(), vc, new HashMap<>(), false).getVtRowList(); + List wants = + Lists.newArrayList("ResolveDestinations ks [type:INT64 value:\"1\"] Destinations:DestinationKeyspaceID(166b40b44aba4bd6)", "ExecuteMultiShard ks.-20: select {} false false"); + vc.expectLog(wants); + Assert.assertEquals(printFail("testRouteGetFields is FAIL"), new VtResultSet(), result); + } + + @Test + public void testRouteSort() throws SQLException { + RouteGen4Engine sel = new RouteGen4Engine(Engine.RouteOpcode.SelectUnsharded, new VKeyspace("ks", false), "select", "select_field", new MySqlSelectQueryBlock()); + OrderByParamsGen4 order = new OrderByParamsGen4(0, false, -1, null); + sel.getOrderBy().add(order); + LoggingVCursor vc = new LoggingVCursor(Lists.newArrayList("0"), Lists.newArrayList(TestResult.makeTestResult(TestResult.makeTestFields("id", "int64"), "1", "1", "3", "2"))); + VtRowList result = sel.execute(VtContext.background(), vc, new HashMap<>(), false).getVtRowList(); + List wants = Lists.newArrayList("ResolveDestinations ks [] Destinations:DestinationAllShard()", "ExecuteMultiShard ks.0: select {} false false"); + vc.expectLog(wants); + VtResultSet wantResult = TestResult.makeTestResult(TestResult.makeTestFields("id", "int64"), "1", "1", "2", "3"); + Assert.assertEquals(printFail("testRouteSort is FAIL"), wantResult, result); + + sel.getOrderBy().get(0).setDesc(true); + vc.rewind(); + result = sel.execute(VtContext.background(), vc, new HashMap<>(), false).getVtRowList(); + wantResult = TestResult.makeTestResult(TestResult.makeTestFields("id", "int64"), "3", "2", "1", "1"); + Assert.assertEquals(printFail("testRouteSort is FAIL"), wantResult, result); + + vc = new LoggingVCursor(Lists.newArrayList("0"), Lists.newArrayList(TestResult.makeTestResult(TestResult.makeTestFields("id", "varchar"), "1", "2", "3"))); + sel.execute(VtContext.background(), vc, new HashMap<>(), false); +// require.EqualError(t, err, `cannot compare strings, collation is unknown or unsupported (collation ID: 0)`) + } + + @Test + public void testRouteSortWeightStrings() throws SQLException { + RouteGen4Engine sel = new RouteGen4Engine(Engine.RouteOpcode.SelectUnsharded, new VKeyspace("ks", false), "select", "select_field", new MySqlSelectQueryBlock()); + OrderByParamsGen4 order = new OrderByParamsGen4(1, false, 0, null); + sel.getOrderBy().add(order); + LoggingVCursor vc = new LoggingVCursor(Lists.newArrayList("0"), + Lists.newArrayList(TestResult.makeTestResult(TestResult.makeTestFields("weightString|normal", "varbinary|varchar"), "v|x", "g|d", "a|a", "c|t", "f|p"))); + + // 后续调整为并发测试 + VtRowList result = sel.execute(VtContext.background(), vc, new HashMap<>(), false).getVtRowList(); + List wants = Lists.newArrayList("ResolveDestinations ks [] Destinations:DestinationAllShard()", "ExecuteMultiShard ks.0: select {} false false"); + vc.expectLog(wants); + VtResultSet wantResult = TestResult.makeTestResult(TestResult.makeTestFields("weightString|normal", "varbinary|varchar"), "a|a", "c|t", "f|p", "g|d", "v|x"); + Assert.assertEquals(printFail("Sort using Weight Strings is FAIL"), wantResult, result); + + sel.getOrderBy().get(0).setDesc(true); + vc.rewind(); + result = sel.execute(VtContext.background(), vc, new HashMap<>(), false).getVtRowList(); + wantResult = TestResult.makeTestResult(TestResult.makeTestFields("weightString|normal", "varbinary|varchar"), "v|x", "g|d", "f|p", "c|t", "a|a"); + Assert.assertEquals(printFail("Descending ordering using weighted strings is FAIL"), wantResult, result); + + sel.getOrderBy().clear(); + order = new OrderByParamsGen4(1, false, -1, null); + sel.getOrderBy().add(order); + vc = new LoggingVCursor(Lists.newArrayList("0"), + Lists.newArrayList(TestResult.makeTestResult(TestResult.makeTestFields("weightString|normal", "varbinary|varchar"), "v|x", "g|d", "a|a", "c|t", "f|p"))); + sel.execute(VtContext.background(), vc, new HashMap<>(), false); +// require.EqualError(t, err, `cannot compare strings, collation is unknown or unsupported (collation ID: 0)`) + } + + public void testRouteSortCollation() { + + } + + @Test + public void testRouteSortTruncate() throws SQLException { + RouteGen4Engine sel = new RouteGen4Engine(Engine.RouteOpcode.SelectUnsharded, new VKeyspace("ks", false), "select", "select_field", new MySqlSelectQueryBlock()); + OrderByParamsGen4 order = new OrderByParamsGen4(0, false, -1, null); + sel.getOrderBy().add(order); + sel.setTruncateColumnCount(1); + LoggingVCursor vc = new LoggingVCursor(Lists.newArrayList("0"), + Lists.newArrayList(TestResult.makeTestResult(TestResult.makeTestFields("id|col", "int64|int64"), "1|1", "1|1", "3|1", "2|1"))); + + VtRowList result = sel.execute(VtContext.background(), vc, new HashMap<>(), false).getVtRowList(); + List wants = Lists.newArrayList("ResolveDestinations ks [] Destinations:DestinationAllShard()", "ExecuteMultiShard ks.0: select {} false false"); + vc.expectLog(wants); + VtResultSet wantResult = TestResult.makeTestResult(TestResult.makeTestFields("id", "int64"), "1", "1", "2", "3"); + Assert.assertEquals(printFail("testRouteSortTruncate"), wantResult, result); + } + + @Test + public void testRouteStreamTruncate() throws SQLException { + RouteGen4Engine sel = new RouteGen4Engine(Engine.RouteOpcode.SelectUnsharded, new VKeyspace("ks", false), "select", "select_field", new MySqlSelectQueryBlock()); + sel.setTruncateColumnCount(1); + LoggingVCursor vc = new LoggingVCursor(Lists.newArrayList("0"), + Lists.newArrayList(TestResult.makeTestResult(TestResult.makeTestFields("id|col", "int64|int64"), "1|1", "2|1"))); + + VtRowList result = sel.execute(VtContext.background(), vc, new HashMap<>(), false).getVtRowList(); + List wants = Lists.newArrayList("ResolveDestinations ks [] Destinations:DestinationAllShard()", "ExecuteMultiShard ks.0: select {} false false"); + vc.expectLog(wants); + VtResultSet wantResult = TestResult.makeTestResult(TestResult.makeTestFields("id", "int64"), "1", "2"); + Assert.assertEquals(printFail("testRouteSortTruncate"), wantResult, result); + } + + public void testRouteStreamSortTruncate() { + + } + + @Test + public void testParamsFail() throws SQLException { + RouteGen4Engine sel = new RouteGen4Engine(Engine.RouteOpcode.SelectUnsharded, new VKeyspace("ks", false), "select", "select_field", new MySqlSelectQueryBlock()); + LoggingVCursor vc = new LoggingVCursor(new SQLException("shard error")); + + thrown.expect(SQLException.class); + thrown.expectMessage("shard error"); + sel.execute(VtContext.background(), vc, new HashMap<>(), false); + +// vc.Rewind() +// _, err = wrapStreamExecute(sel, vc, map[string]*querypb.BindVariable{}, false) +// require.EqualError(t, err, `shard error`) + } + + @Test + public void testExecFail() throws SQLException { + // normal route with no scatter errors as warnings + RouteGen4Engine sel = new RouteGen4Engine(Engine.RouteOpcode.SelectScatter, new VKeyspace("ks", true), "select", "select_field", new MySqlSelectQueryBlock()); + LoggingVCursor vc = new LoggingVCursor(Lists.newArrayList("-20", "20-"), Lists.newArrayList(defaultSelectResult)); + List sqlExceptions = new ArrayList<>(); + sqlExceptions.add(new SQLException("result error -20")); + vc.setMultiShardErrs(sqlExceptions); + + thrown.expect(SQLException.class); + thrown.expectMessage("result error -20"); + sel.execute(VtContext.background(), vc, new HashMap<>(), false); + } +} diff --git a/src/test/java/com/jd/jdbc/engine/gen4/ScalarAggregateGen4EngineTest.java b/src/test/java/com/jd/jdbc/engine/gen4/ScalarAggregateGen4EngineTest.java new file mode 100644 index 0000000..97e0d6a --- /dev/null +++ b/src/test/java/com/jd/jdbc/engine/gen4/ScalarAggregateGen4EngineTest.java @@ -0,0 +1,74 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.engine.gen4; + +import com.google.common.collect.Lists; +import com.jd.BaseTest; +import com.jd.jdbc.context.VtContext; +import com.jd.jdbc.engine.Engine.AggregateOpcodeG4; +import com.jd.jdbc.engine.FakePrimitive; +import com.jd.jdbc.engine.util.TestResult; +import com.jd.jdbc.engine.vcursor.NoopVCursor; +import com.jd.jdbc.sqltypes.VtResultSet; +import io.vitess.proto.Query; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import lombok.AllArgsConstructor; +import org.junit.Assert; +import org.junit.Test; + +public class ScalarAggregateGen4EngineTest extends BaseTest { + + @Test + public void testEmptyRows() throws SQLException { + @AllArgsConstructor + class TestCase { + AggregateOpcodeG4 opcode; + + AggregateOpcodeG4 origOpcode; + + String expectedVal; + + String expectedTyp; + + } + List testCases = new ArrayList<>(); + testCases.add(new TestCase(AggregateOpcodeG4.AggregateCountDistinct, null, "0", "INT64")); + testCases.add(new TestCase(AggregateOpcodeG4.AggregateCount, null, "0", "INT64")); + testCases.add(new TestCase(AggregateOpcodeG4.AggregateSumDistinct, null, "null", "DECIMAL")); + testCases.add(new TestCase(AggregateOpcodeG4.AggregateSum, null, "null", "INT64")); + testCases.add(new TestCase(AggregateOpcodeG4.AggregateSum, AggregateOpcodeG4.AggregateCount, "0", "INT64")); + testCases.add(new TestCase(AggregateOpcodeG4.AggregateMax, null, "null", "INT64")); + testCases.add(new TestCase(AggregateOpcodeG4.AggregateMin, null, "null", "INT64")); + for (TestCase test : testCases) { + Query.Field[] field = TestResult.makeTestFields(AbstractAggregateGen4.printOpcode(test.opcode), "int64"); + VtResultSet resultSet = TestResult.makeTestResult(field); + FakePrimitive fp = new FakePrimitive(Lists.newArrayList(resultSet)); + AbstractAggregateGen4.AggregateParams aggr = new AbstractAggregateGen4.AggregateParams(test.opcode, 0, AbstractAggregateGen4.printOpcode(test.opcode)); + aggr.setOrigOpcode(test.origOpcode); + ScalarAggregateGen4Engine oa = new ScalarAggregateGen4Engine(true, Lists.newArrayList(aggr), fp); + VtResultSet outResult = (VtResultSet) oa.execute(VtContext.background(), new NoopVCursor(), null, false).getVtRowList(); + Query.Field[] wantField = TestResult.makeTestFields(AbstractAggregateGen4.printOpcode(test.opcode), test.expectedTyp); + VtResultSet wantResult = TestResult.makeTestResult(wantField, test.expectedVal); + Assert.assertEquals(printFail(AbstractAggregateGen4.printOpcode(test.opcode) + " test is FAIL"), wantResult, outResult); + printOk(AbstractAggregateGen4.printOpcode(test.opcode) + " test is [OK] "); + } + } +} diff --git a/src/test/java/com/jd/jdbc/engine/gen4/VitessCompareTest.java b/src/test/java/com/jd/jdbc/engine/gen4/VitessCompareTest.java new file mode 100644 index 0000000..465fe15 --- /dev/null +++ b/src/test/java/com/jd/jdbc/engine/gen4/VitessCompareTest.java @@ -0,0 +1,90 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.engine.gen4; + +import com.google.common.collect.Lists; +import com.jd.BaseTest; +import com.jd.jdbc.sqltypes.VtResultValue; +import io.vitess.proto.Query; +import java.math.BigInteger; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import lombok.AllArgsConstructor; +import lombok.ToString; +import org.junit.Assert; +import org.junit.Test; + +public class VitessCompareTest extends BaseTest { + @Test + public void testComparer() throws SQLException { + @AllArgsConstructor + @ToString + class TestCase { + VitessCompare compare; + + List row1; + + List row2; + + int output; + } + + List testCases = new ArrayList<>(); + VitessCompare compare1 = new VitessCompare(0, -1, true, 0); + VtResultValue r11 = new VtResultValue(BigInteger.valueOf(23), Query.Type.INT64); + VtResultValue r12 = new VtResultValue(BigInteger.valueOf(34), Query.Type.INT64); + TestCase testCase1 = new TestCase(compare1, Lists.newArrayList(r11), Lists.newArrayList(r12), 1); + testCases.add(testCase1); + + VitessCompare compare2 = new VitessCompare(0, -1, false, 0); + VtResultValue r21 = new VtResultValue(BigInteger.valueOf(23), Query.Type.INT64); + VtResultValue r22 = new VtResultValue(BigInteger.valueOf(23), Query.Type.INT64); + TestCase testCase2 = new TestCase(compare2, Lists.newArrayList(r21), Lists.newArrayList(r22), 0); + testCases.add(testCase2); + + VitessCompare compare3 = new VitessCompare(0, -1, false, 0); + VtResultValue r31 = new VtResultValue(BigInteger.valueOf(23), Query.Type.INT64); + VtResultValue r32 = new VtResultValue(BigInteger.valueOf(12), Query.Type.INT64); + TestCase testCase3 = new TestCase(compare3, Lists.newArrayList(r31), Lists.newArrayList(r32), 1); + testCases.add(testCase3); + + VitessCompare compare4 = new VitessCompare(1, 0, false, 0); + VtResultValue r411 = new VtResultValue(BigInteger.valueOf(23), Query.Type.INT64); + VtResultValue r412 = new VtResultValue("b", Query.Type.VARCHAR); + VtResultValue r421 = new VtResultValue(BigInteger.valueOf(34), Query.Type.INT64); + VtResultValue r422 = new VtResultValue("a", Query.Type.VARCHAR); + TestCase testCase4 = new TestCase(compare4, Lists.newArrayList(r411, r412), Lists.newArrayList(r421, r422), -1); + testCases.add(testCase4); + + VitessCompare compare5 = new VitessCompare(1, 0, true, 0); + VtResultValue r511 = new VtResultValue(BigInteger.valueOf(23), Query.Type.INT64); + VtResultValue r512 = new VtResultValue("A", Query.Type.VARCHAR); + VtResultValue r521 = new VtResultValue(BigInteger.valueOf(23), Query.Type.INT64); + VtResultValue r522 = new VtResultValue("a", Query.Type.VARCHAR); + TestCase testCase5 = new TestCase(compare5, Lists.newArrayList(r511, r512), Lists.newArrayList(r521, r522), 0); + testCases.add(testCase5); + + for (TestCase testCase : testCases) { + int got = testCase.compare.compare(testCase.row1, testCase.row2); + Assert.assertEquals(testCase.output, got); + printOk(testCase + " test is [OK] "); + } + } +} \ No newline at end of file diff --git a/src/test/java/com/jd/jdbc/engine/util/TestResult.java b/src/test/java/com/jd/jdbc/engine/util/TestResult.java new file mode 100644 index 0000000..7c98d7a --- /dev/null +++ b/src/test/java/com/jd/jdbc/engine/util/TestResult.java @@ -0,0 +1,101 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.engine.util; + +import com.jd.jdbc.sqltypes.VtResultSet; +import com.jd.jdbc.sqltypes.VtResultValue; +import com.jd.jdbc.sqltypes.VtType; +import static com.jd.jdbc.vitess.resultset.ResultSetUtil.convertValue; +import io.vitess.proto.Query; +import java.util.ArrayList; +import java.util.List; + +public class TestResult { + + /** + * Functions in this file should only be used for testing. + * This is an experiment to see if test code bloat can be + * reduced and readability improved. + *

+ * MakeTestFields builds a []*querypb.Field for testing. + * fields := sqltypes.MakeTestFields( + * "a|b", + * "int64|varchar", + * ) + * The field types are as defined in querypb and are case + * insensitive. Column delimiters must be used only to sepearate + * strings and not at the beginning or the end. + */ + + public static Query.Field[] makeTestFields(String n, String t) { + String[] names = splitStr(n); + String[] types = splitStr(t); + if (names.length != types.length) { + throw new RuntimeException(); + } + Query.Field[] res = new Query.Field[names.length]; + for (int i = 0; i < names.length; i++) { + res[i] = Query.Field.newBuilder().setName(names[i]).setType(Query.Type.valueOf(types[i].toUpperCase())).build(); + } + return res; + } + + /** + * MakeTestResult builds a *sqltypes.Result object for testing. + * result := sqltypes.MakeTestResult( + * fields, + * " 1|a", + * "10|abcd", + * ) + * The field type values are set as the types for the rows built. + * Spaces are trimmed from row values. "null" is treated as NULL. + **/ + public static VtResultSet makeTestResult(Query.Field[] fields, String... rows) { + VtResultSet res = new VtResultSet(); + res.setFields(fields); + if (rows.length <= 0) { + return res; + } + List> resRows = new ArrayList<>(rows.length); + + for (int i = 0; i < rows.length; i++) { + List resRow = new ArrayList<>(fields.length); + String[] row = splitStr(rows[i]); + for (int j = 0; j < row.length; j++) { + VtResultValue item; + if (row[j].equals("null")) { + item = new VtResultValue(null, Query.Type.NULL_TYPE); + } else { + Class javaClass = VtType.DataTypeConverter.fromTypeString(fields[j].getType().toString()).getJavaClass(); + Object o = convertValue(row[j], javaClass); + item = new VtResultValue(o, fields[j].getType()); + } + resRow.add(item); + } + resRows.add(new ArrayList<>(resRow)); + } + res.setRows(resRows); + return res; + } + + public static String[] splitStr(String str) { + return str.split("\\|"); + } + +} diff --git a/src/test/java/com/jd/jdbc/engine/vcursor/FakeVcursorUtil.java b/src/test/java/com/jd/jdbc/engine/vcursor/FakeVcursorUtil.java new file mode 100644 index 0000000..4c761a9 --- /dev/null +++ b/src/test/java/com/jd/jdbc/engine/vcursor/FakeVcursorUtil.java @@ -0,0 +1,86 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.engine.vcursor; + +import com.jd.jdbc.srvtopo.BindVariable; +import io.vitess.proto.Query; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +public class FakeVcursorUtil { + public static int testMaxMemoryRows = 100; + + public static boolean testIgnoreMaxMemoryRows = false; + + public static String printBindVars(Map bindVariablesMap) { + if (bindVariablesMap == null) { + return ""; + } + List keys = new ArrayList<>(); + for (String key : bindVariablesMap.keySet()) { + keys.add(key); + } + Collections.sort(keys); + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < keys.size(); i++) { + if (i > 0) { + sb.append(" "); + } + String k = keys.get(i); + sb.append(k + ": " + printBindVariable(bindVariablesMap.get(k))); + } + + return sb.toString(); + } + + public static String printBindVariable(BindVariable bv) { + StringBuilder sb = new StringBuilder(); + if (bv.getValue() != null) { + sb.append("type:" + bv.getType() + " value:\"" + new String(bv.getValue()) + "\""); + return sb.toString(); + } + if (bv.getValuesList() != null) { + sb.append("type:" + bv.getType() + ", values:" + printValues(bv.getValuesList())); + return sb.toString(); + } + return ""; + } + + public static String printValues(List ids) { + if (ids == null) { + return "[]"; + } + StringBuilder sb = new StringBuilder("["); + for (Query.Value v : ids) { + sb.append("type:"); + sb.append(v.getType()); + sb.append(" value:\""); + String s = new String(v.getValue().toByteArray(), StandardCharsets.UTF_8); + sb.append(s + "\"" + " "); + } + sb.deleteCharAt(sb.length() - 1); + sb.append("]"); + return sb.toString(); + + } + +} diff --git a/src/test/java/com/jd/jdbc/engine/vcursor/LoggingVCursor.java b/src/test/java/com/jd/jdbc/engine/vcursor/LoggingVCursor.java new file mode 100644 index 0000000..41e20bf --- /dev/null +++ b/src/test/java/com/jd/jdbc/engine/vcursor/LoggingVCursor.java @@ -0,0 +1,292 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.engine.vcursor; + +import com.google.common.collect.Lists; +import com.jd.jdbc.IExecute; +import com.jd.jdbc.common.util.CollectionUtils; +import com.jd.jdbc.engine.Vcursor; +import static com.jd.jdbc.engine.vcursor.FakeVcursorUtil.printBindVars; +import static com.jd.jdbc.engine.vcursor.FakeVcursorUtil.printValues; +import com.jd.jdbc.key.Destination; +import com.jd.jdbc.key.DestinationAllShard; +import com.jd.jdbc.key.DestinationAnyShard; +import com.jd.jdbc.key.DestinationKeyspaceID; +import com.jd.jdbc.key.DestinationNone; +import com.jd.jdbc.key.DestinationShard; +import com.jd.jdbc.queryservice.StreamIterator; +import com.jd.jdbc.sqltypes.VtResultSet; +import com.jd.jdbc.sqltypes.VtRowList; +import com.jd.jdbc.srvtopo.BindVariable; +import com.jd.jdbc.srvtopo.BoundQuery; +import com.jd.jdbc.srvtopo.ResolvedShard; +import com.jd.jdbc.srvtopo.Resolver; +import io.vitess.proto.Query; +import io.vitess.proto.Topodata; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.locks.ReentrantLock; +import lombok.Data; +import org.junit.Assert; +import vschema.Vschema; + +/** + * loggingVCursor logs requests and allows you to verify + * that the correct requests were made. + */ +@Data +public class LoggingVCursor implements Vcursor { + + private NoopVCursor noopVCursor; + + private List shards; + + private List shardForKsid; + + private int curShardForKsid; + + private SQLException shardErr; + + private List results; + + private int curResult; + + private SQLException resultErr; + + private List warnings = new ArrayList<>(); + + // Optional errors that can be returned from nextResult() alongside the results for + // multi-shard queries + private List multiShardErrs; + + private List log = new ArrayList<>(); + + private final ReentrantLock lock = new ReentrantLock(); + + private Topodata.TabletType resolvedTargetTabletType = Topodata.TabletType.UNKNOWN; + + private Vschema.Table tableRoutes; + + private String dbDDLPlugin; + + private boolean ksAvailable; + + private boolean inReservedConn; + + private Map systemVariables; + + private boolean disableSetVar; + + // map different shards to keyspaces in the test. + private Map> ksShardMap; + + public LoggingVCursor(List shards, List results) { + this.shards = shards; + this.results = results; + } + + public LoggingVCursor(SQLException shardErr) { + this.shardErr = shardErr; + } + + @Override + public Integer maxMemoryRows() { + return null; + } + + @Override + public Boolean exceedsMaxMemoryRows(Integer numRows) { + return null; + } + + @Override + public Boolean autocommitApproval() { + return null; + } + + @Override + public IExecute.ExecuteMultiShardResponse executeMultiShard(List rss, List queries, Boolean rollbackOnError, Boolean canAutocommit) throws SQLException { + if (CollectionUtils.isNotEmpty(multiShardErrs)) { + throw multiShardErrs.get(0); + } + log.add(String.format("ExecuteMultiShard %s %s %s", printResolvedShardQueries(rss, queries), rollbackOnError, canAutocommit)); + + VtResultSet res = nextResult(); + return new IExecute.ExecuteMultiShardResponse(res); + } + + private String printResolvedShardQueries(List rss, List queries) { + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < rss.size(); i++) { + ResolvedShard rs = rss.get(i); + String format = String.format("%s.%s: %s{%s} ", rs.getTarget().getKeyspace(), rs.getTarget().getShard(), queries.get(i).getSql(), printBindVars(queries.get(i).getBindVariablesMap())); + sb.append(format); + } + sb.deleteCharAt(sb.length() - 1); + + return sb.toString(); + } + + + @Override + public IExecute.ExecuteBatchMultiShardResponse executeBatchMultiShard(List rss, List> queries, Boolean rollbackOnError, Boolean canAutocommit) throws SQLException { + return null; + } + + @Override + public List streamExecuteMultiShard(List rss, List queries) throws SQLException { + return null; + } + + @Override + public Resolver.ResolveDestinationResult resolveDestinations(String keyspace, List ids, List destinations) throws SQLException { + log.add(String.format("ResolveDestinations %s %s %s", keyspace, printValues(ids), destinationsString(destinations))); + if (shardErr != null) { + throw shardErr; + } + List rss = new ArrayList<>(); + List> values = new LinkedList<>(); + Map visited = new HashMap<>(); + for (int i = 0; i < destinations.size(); i++) { + List shards = new ArrayList<>(); + Destination destination = destinations.get(i); + if (destination instanceof DestinationAllShard) { + if (ksShardMap != null) { + if (ksShardMap.containsKey(keyspace)) { + shards = ksShardMap.get(keyspace); + } + } else { + shards = this.shards; + } + } else if (destination instanceof DestinationKeyspaceID) { + if (shardForKsid == null || curShardForKsid > shardForKsid.size()) { + shards = Lists.newArrayList("-20"); + } else { + shards = Lists.newArrayList(shardForKsid.get(curShardForKsid)); + curShardForKsid++; + } + } else if (destination instanceof DestinationAnyShard) { + // Take the first shard. + shards = Lists.newArrayList(this.shards.get(0)); + } else if (destination instanceof DestinationNone) { + // Nothing to do here. + } else if (destination instanceof DestinationShard) { + shards = Lists.newArrayList(destination.toString()); + } else { + throw new SQLException("unsupported destination: " + destination.toString()); + } + for (String shard : shards) { + Integer vi = visited.get(shard); + if (vi == null) { + vi = rss.size(); + visited.put(shard, vi); + Query.Target.Builder targetBuilder = Query.Target.newBuilder().setKeyspace(keyspace) + .setShard(shard) + .setTabletType(resolvedTargetTabletType); + ResolvedShard resolvedShard = new ResolvedShard(); + resolvedShard.setTarget(targetBuilder.build()); + rss.add(resolvedShard); + if (ids != null) { + values.add(new ArrayList<>()); + } + } + if (ids != null) { + values.get(vi).add(ids.get(i)); + } + } + } + return new Resolver.ResolveDestinationResult(rss, values); + } + + + + @Override + public Resolver.AllShardResult getAllShards(String keyspace, Topodata.TabletType tabletType) throws SQLException { + return null; + } + + @Override + public VtRowList executeStandalone(String sql, Map bindVars, ResolvedShard resolvedShard, boolean canAutocommit) throws SQLException { + return null; + } + + @Override + public Boolean getRollbackOnPartialExec() { + return null; + } + + @Override + public String getCharEncoding() { + return null; + } + + @Override + public int getMaxParallelNum() { + return 0; + } + + private VtResultSet nextResult() throws SQLException { + if (results == null || curResult >= results.size()) { + if (resultErr != null) { + throw resultErr; + } else { + return new VtResultSet(); + } + } + VtResultSet r = results.get(curResult); + curResult++; + if (r == null) { + throw resultErr; + } + return r; + } + + // DestinationsString returns a printed version of the destination array. + public String destinationsString(List destinations) { + StringBuilder sb = new StringBuilder("Destinations:"); + for (Destination d : destinations) { + sb.append(d.toString()); + sb.append(","); + } + sb.deleteCharAt(sb.length() - 1); + return sb.toString(); + } + + public void expectLog(List want) { + if (want.size() != log.size()) { + Assert.fail("wants size " + want.size() + " actual size" + log.size()); + } + for (int i = 0; i < want.size(); i++) { + Assert.assertEquals(want.get(i), log.get(i)); + } + } + + public void rewind() { + this.curShardForKsid = 0; + this.curResult = 0; + this.log.clear(); + this.warnings.clear(); + } + + +} diff --git a/src/test/java/com/jd/jdbc/engine/vcursor/NoopVCursor.java b/src/test/java/com/jd/jdbc/engine/vcursor/NoopVCursor.java new file mode 100644 index 0000000..36d75cf --- /dev/null +++ b/src/test/java/com/jd/jdbc/engine/vcursor/NoopVCursor.java @@ -0,0 +1,100 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.engine.vcursor; + +import com.jd.jdbc.IExecute; +import com.jd.jdbc.engine.Vcursor; +import com.jd.jdbc.key.Destination; +import com.jd.jdbc.queryservice.StreamIterator; +import com.jd.jdbc.sqltypes.VtRowList; +import com.jd.jdbc.srvtopo.BindVariable; +import com.jd.jdbc.srvtopo.BoundQuery; +import com.jd.jdbc.srvtopo.ResolvedShard; +import com.jd.jdbc.srvtopo.Resolver; +import io.vitess.proto.Query; +import io.vitess.proto.Topodata; +import java.sql.SQLException; +import java.util.List; +import java.util.Map; + +/** + * noopVCursor is used to build other vcursors. + */ +public class NoopVCursor implements Vcursor { + + @Override + public Integer maxMemoryRows() { + return FakeVcursorUtil.testMaxMemoryRows; + } + + @Override + public Boolean exceedsMaxMemoryRows(Integer numRows) { + return !FakeVcursorUtil.testIgnoreMaxMemoryRows && numRows > FakeVcursorUtil.testMaxMemoryRows; + } + + @Override + public Boolean autocommitApproval() { + return null; + } + + @Override + public IExecute.ExecuteMultiShardResponse executeMultiShard(List rss, List queries, Boolean rollbackOnError, Boolean canAutocommit) throws SQLException { + return null; + } + + @Override + public IExecute.ExecuteBatchMultiShardResponse executeBatchMultiShard(List rss, List> queries, Boolean rollbackOnError, Boolean canAutocommit) throws SQLException { + return null; + } + + @Override + public List streamExecuteMultiShard(List rss, List queries) throws SQLException { + return null; + } + + @Override + public Resolver.ResolveDestinationResult resolveDestinations(String keyspace, List ids, List destinations) throws SQLException { + return null; + } + + @Override + public Resolver.AllShardResult getAllShards(String keyspace, Topodata.TabletType tabletType) throws SQLException { + return null; + } + + @Override + public VtRowList executeStandalone(String sql, Map bindVars, ResolvedShard resolvedShard, boolean canAutocommit) throws SQLException { + return null; + } + + @Override + public Boolean getRollbackOnPartialExec() { + return null; + } + + @Override + public String getCharEncoding() { + return null; + } + + @Override + public int getMaxParallelNum() { + return 0; + } +} diff --git a/src/test/java/com/jd/jdbc/evalengine/ArithmeticTest.java b/src/test/java/com/jd/jdbc/evalengine/ArithmeticTest.java index 1724c9c..431b13d 100644 --- a/src/test/java/com/jd/jdbc/evalengine/ArithmeticTest.java +++ b/src/test/java/com/jd/jdbc/evalengine/ArithmeticTest.java @@ -16,6 +16,7 @@ package com.jd.jdbc.evalengine; +import com.jd.jdbc.sqltypes.VtNumberRange; import com.jd.jdbc.sqltypes.VtResultValue; import com.jd.jdbc.sqltypes.VtValue; import io.vitess.proto.Query; @@ -207,51 +208,51 @@ public void testCast() throws SQLException { @Test public void testAddNumeric() throws SQLException { List testCases = new ArrayList() {{ - add(new NumAddTestCase(new EvalEngine.EvalResult(Long.valueOf(1), Query.Type.INT64), - new EvalEngine.EvalResult(Long.valueOf(2), Query.Type.INT64), - new EvalEngine.EvalResult(Long.valueOf(3), Query.Type.INT64))); + add(new NumAddTestCase(new EvalResult(Long.valueOf(1), Query.Type.INT64), + new EvalResult(Long.valueOf(2), Query.Type.INT64), + new EvalResult(Long.valueOf(3), Query.Type.INT64))); - add(new NumAddTestCase(new EvalEngine.EvalResult(Long.valueOf(1), Query.Type.INT64), - new EvalEngine.EvalResult(BigInteger.valueOf(2), Query.Type.UINT64), - new EvalEngine.EvalResult(BigInteger.valueOf(3), Query.Type.UINT64))); + add(new NumAddTestCase(new EvalResult(Long.valueOf(1), Query.Type.INT64), + new EvalResult(BigInteger.valueOf(2), Query.Type.UINT64), + new EvalResult(BigInteger.valueOf(3), Query.Type.UINT64))); - add(new NumAddTestCase(new EvalEngine.EvalResult(Long.valueOf(1), Query.Type.INT64), - new EvalEngine.EvalResult(Double.valueOf(2), Query.Type.FLOAT64), - new EvalEngine.EvalResult(Double.valueOf(3), Query.Type.FLOAT64))); + add(new NumAddTestCase(new EvalResult(Long.valueOf(1), Query.Type.INT64), + new EvalResult(Double.valueOf(2), Query.Type.FLOAT64), + new EvalResult(Double.valueOf(3), Query.Type.FLOAT64))); - add(new NumAddTestCase(new EvalEngine.EvalResult(BigInteger.valueOf(1), Query.Type.UINT64), - new EvalEngine.EvalResult(BigInteger.valueOf(2), Query.Type.UINT64), - new EvalEngine.EvalResult(BigInteger.valueOf(3), Query.Type.UINT64))); + add(new NumAddTestCase(new EvalResult(BigInteger.valueOf(1), Query.Type.UINT64), + new EvalResult(BigInteger.valueOf(2), Query.Type.UINT64), + new EvalResult(BigInteger.valueOf(3), Query.Type.UINT64))); - add(new NumAddTestCase(new EvalEngine.EvalResult(BigInteger.valueOf(1), Query.Type.UINT64), - new EvalEngine.EvalResult(Double.valueOf(2), Query.Type.FLOAT64), - new EvalEngine.EvalResult(Double.valueOf(3), Query.Type.FLOAT64))); + add(new NumAddTestCase(new EvalResult(BigInteger.valueOf(1), Query.Type.UINT64), + new EvalResult(Double.valueOf(2), Query.Type.FLOAT64), + new EvalResult(Double.valueOf(3), Query.Type.FLOAT64))); - add(new NumAddTestCase(new EvalEngine.EvalResult(Double.valueOf(1), Query.Type.FLOAT64), - new EvalEngine.EvalResult(Double.valueOf(2), Query.Type.FLOAT64), - new EvalEngine.EvalResult(Double.valueOf(3), Query.Type.FLOAT64))); + add(new NumAddTestCase(new EvalResult(Double.valueOf(1), Query.Type.FLOAT64), + new EvalResult(Double.valueOf(2), Query.Type.FLOAT64), + new EvalResult(Double.valueOf(3), Query.Type.FLOAT64))); // Int64 overflow - add(new NumAddTestCase(new EvalEngine.EvalResult(9223372036854775807L, Query.Type.INT64), - new EvalEngine.EvalResult(2L, Query.Type.INT64), - new EvalEngine.EvalResult(9223372036854775809d, Query.Type.FLOAT64))); + add(new NumAddTestCase(new EvalResult(9223372036854775807L, Query.Type.INT64), + new EvalResult(2L, Query.Type.INT64), + new EvalResult(9223372036854775809d, Query.Type.FLOAT64))); // Int64 underflow - add(new NumAddTestCase(new EvalEngine.EvalResult(-9223372036854775807L, Query.Type.INT64), - new EvalEngine.EvalResult(-2L, Query.Type.INT64), - new EvalEngine.EvalResult(-9223372036854775809d, Query.Type.FLOAT64))); + add(new NumAddTestCase(new EvalResult(-9223372036854775807L, Query.Type.INT64), + new EvalResult(-2L, Query.Type.INT64), + new EvalResult(-9223372036854775809d, Query.Type.FLOAT64))); - add(new NumAddTestCase(new EvalEngine.EvalResult(-1L, Query.Type.INT64), - new EvalEngine.EvalResult(BigInteger.valueOf(2l), Query.Type.UINT64), - new EvalEngine.EvalResult(BigInteger.valueOf(1l), Query.Type.UINT64))); + add(new NumAddTestCase(new EvalResult(-1L, Query.Type.INT64), + new EvalResult(BigInteger.valueOf(2L), Query.Type.UINT64), + new EvalResult(BigInteger.valueOf(1L), Query.Type.UINT64))); // Uint64 overflow - add(new NumAddTestCase(new EvalEngine.EvalResult(new BigInteger("18446744073709551615"), Query.Type.UINT64), - new EvalEngine.EvalResult(BigInteger.valueOf(2l), Query.Type.UINT64), - new EvalEngine.EvalResult(new BigInteger("18446744073709551617"), Query.Type.UINT64))); + add(new NumAddTestCase(new EvalResult(new BigInteger("18446744073709551615"), Query.Type.UINT64), + new EvalResult(BigInteger.valueOf(2L), Query.Type.UINT64), + new EvalResult(new BigInteger("18446744073709551617"), Query.Type.UINT64))); }}; int i = 0; for (NumAddTestCase testCase : testCases) { i++; - EvalEngine.EvalResult result = Arithmetic.addNumeric(testCase.value1, testCase.value2); + EvalResult result = Arithmetic.addNumeric(testCase.value1, testCase.value2); printInfo("NO." + i + " Test case: [" + testCase + "]"); printInfo("NO." + i + " Test Result: [" + result.toString() + "]"); @@ -271,20 +272,20 @@ public void testAddNumeric() throws SQLException { @Test public void testCastFromNumeric() throws Exception { List testCases = new ArrayList() {{ - add(new CFNTestCase(Query.Type.INT64, new EvalEngine.EvalResult(1l, Query.Type.INT64), VtValue.newVtValue(Query.Type.INT64, "1".getBytes()))); - add(new CFNTestCase(Query.Type.INT64, new EvalEngine.EvalResult(BigInteger.valueOf(1l), Query.Type.UINT64), VtValue.newVtValue(Query.Type.INT64, "1".getBytes()))); - add(new CFNTestCase(Query.Type.INT64, new EvalEngine.EvalResult(1.2e-16, Query.Type.FLOAT64), VtValue.newVtValue(Query.Type.INT64, "0".getBytes()))); - add(new CFNTestCase(Query.Type.UINT64, new EvalEngine.EvalResult(1l, Query.Type.INT64), VtValue.newVtValue(Query.Type.UINT64, BigInteger.valueOf(1l).toString().getBytes()))); - add(new CFNTestCase(Query.Type.UINT64, new EvalEngine.EvalResult(BigInteger.valueOf(1l), Query.Type.UINT64), - VtValue.newVtValue(Query.Type.UINT64, BigInteger.valueOf(1l).toString().getBytes()))); - add(new CFNTestCase(Query.Type.UINT64, new EvalEngine.EvalResult(1.2e-16, Query.Type.FLOAT64), VtValue.newVtValue(Query.Type.UINT64, BigInteger.valueOf(0l).toString().getBytes()))); - add(new CFNTestCase(Query.Type.FLOAT64, new EvalEngine.EvalResult(1l, Query.Type.INT64), VtValue.newVtValue(Query.Type.FLOAT64, "1".getBytes()))); - add(new CFNTestCase(Query.Type.FLOAT64, new EvalEngine.EvalResult(BigInteger.valueOf(1l), Query.Type.UINT64), VtValue.newVtValue(Query.Type.FLOAT64, "1".getBytes()))); - add(new CFNTestCase(Query.Type.FLOAT64, new EvalEngine.EvalResult(1.2e-16, Query.Type.FLOAT64), VtValue.newVtValue(Query.Type.FLOAT64, "0.0000000000000001".getBytes()))); - add(new CFNTestCase(Query.Type.DECIMAL, new EvalEngine.EvalResult(1l, Query.Type.INT64), VtValue.newVtValue(Query.Type.DECIMAL, "1".getBytes()))); - add(new CFNTestCase(Query.Type.DECIMAL, new EvalEngine.EvalResult(BigInteger.valueOf(1l), Query.Type.UINT64), VtValue.newVtValue(Query.Type.DECIMAL, "1".getBytes()))); + add(new CFNTestCase(Query.Type.INT64, new EvalResult(1L, Query.Type.INT64), VtValue.newVtValue(Query.Type.INT64, "1".getBytes()))); + add(new CFNTestCase(Query.Type.INT64, new EvalResult(BigInteger.valueOf(1L), Query.Type.UINT64), VtValue.newVtValue(Query.Type.INT64, "1".getBytes()))); + add(new CFNTestCase(Query.Type.INT64, new EvalResult(1.2e-16, Query.Type.FLOAT64), VtValue.newVtValue(Query.Type.INT64, "0".getBytes()))); + add(new CFNTestCase(Query.Type.UINT64, new EvalResult(1L, Query.Type.INT64), VtValue.newVtValue(Query.Type.UINT64, BigInteger.valueOf(1L).toString().getBytes()))); + add(new CFNTestCase(Query.Type.UINT64, new EvalResult(BigInteger.valueOf(1L), Query.Type.UINT64), + VtValue.newVtValue(Query.Type.UINT64, BigInteger.valueOf(1L).toString().getBytes()))); + add(new CFNTestCase(Query.Type.UINT64, new EvalResult(1.2e-16, Query.Type.FLOAT64), VtValue.newVtValue(Query.Type.UINT64, BigInteger.valueOf(0L).toString().getBytes()))); + add(new CFNTestCase(Query.Type.FLOAT64, new EvalResult(1L, Query.Type.INT64), VtValue.newVtValue(Query.Type.FLOAT64, "1".getBytes()))); + add(new CFNTestCase(Query.Type.FLOAT64, new EvalResult(BigInteger.valueOf(1L), Query.Type.UINT64), VtValue.newVtValue(Query.Type.FLOAT64, "1".getBytes()))); + add(new CFNTestCase(Query.Type.FLOAT64, new EvalResult(1.2e-16, Query.Type.FLOAT64), VtValue.newVtValue(Query.Type.FLOAT64, "0.0000000000000001".getBytes()))); + add(new CFNTestCase(Query.Type.DECIMAL, new EvalResult(1L, Query.Type.INT64), VtValue.newVtValue(Query.Type.DECIMAL, "1".getBytes()))); + add(new CFNTestCase(Query.Type.DECIMAL, new EvalResult(BigInteger.valueOf(1L), Query.Type.UINT64), VtValue.newVtValue(Query.Type.DECIMAL, "1".getBytes()))); // For float, we should not use scientific notation. - add(new CFNTestCase(Query.Type.DECIMAL, new EvalEngine.EvalResult(0.00000000000000012d, Query.Type.FLOAT64), VtValue.newVtValue(Query.Type.DECIMAL, "0.0000000000000001".getBytes()))); + add(new CFNTestCase(Query.Type.DECIMAL, new EvalResult(0.00000000000000012d, Query.Type.FLOAT64), VtValue.newVtValue(Query.Type.DECIMAL, "0.0000000000000001".getBytes()))); }}; int i = 0; @@ -305,37 +306,37 @@ public void testCastFromNumeric() throws Exception { @Test public void testCompareNumeric() { List testCases = new ArrayList() {{ - add(new CompNumTestCase(new EvalEngine.EvalResult(1l, Query.Type.INT64), new EvalEngine.EvalResult(1l, Query.Type.INT64), 0)); - add(new CompNumTestCase(new EvalEngine.EvalResult(1l, Query.Type.INT64), new EvalEngine.EvalResult(2l, Query.Type.INT64), -1)); - add(new CompNumTestCase(new EvalEngine.EvalResult(2l, Query.Type.INT64), new EvalEngine.EvalResult(1l, Query.Type.INT64), 1)); + add(new CompNumTestCase(new EvalResult(1L, Query.Type.INT64), new EvalResult(1L, Query.Type.INT64), 0)); + add(new CompNumTestCase(new EvalResult(1L, Query.Type.INT64), new EvalResult(2L, Query.Type.INT64), -1)); + add(new CompNumTestCase(new EvalResult(2L, Query.Type.INT64), new EvalResult(1L, Query.Type.INT64), 1)); // Special case. - add(new CompNumTestCase(new EvalEngine.EvalResult(-1l, Query.Type.INT64), new EvalEngine.EvalResult(BigInteger.valueOf(1l), Query.Type.UINT64), -1)); - add(new CompNumTestCase(new EvalEngine.EvalResult(1l, Query.Type.INT64), new EvalEngine.EvalResult(BigInteger.valueOf(1l), Query.Type.UINT64), 0)); - add(new CompNumTestCase(new EvalEngine.EvalResult(1l, Query.Type.INT64), new EvalEngine.EvalResult(BigInteger.valueOf(2l), Query.Type.UINT64), -1)); - add(new CompNumTestCase(new EvalEngine.EvalResult(2l, Query.Type.INT64), new EvalEngine.EvalResult(BigInteger.valueOf(1l), Query.Type.UINT64), 1)); - add(new CompNumTestCase(new EvalEngine.EvalResult(1l, Query.Type.INT64), new EvalEngine.EvalResult(1d, Query.Type.FLOAT64), 0)); - add(new CompNumTestCase(new EvalEngine.EvalResult(1l, Query.Type.INT64), new EvalEngine.EvalResult(2d, Query.Type.FLOAT64), -1)); - add(new CompNumTestCase(new EvalEngine.EvalResult(2l, Query.Type.INT64), new EvalEngine.EvalResult(1d, Query.Type.FLOAT64), 1)); + add(new CompNumTestCase(new EvalResult(-1L, Query.Type.INT64), new EvalResult(BigInteger.valueOf(1L), Query.Type.UINT64), -1)); + add(new CompNumTestCase(new EvalResult(1L, Query.Type.INT64), new EvalResult(BigInteger.valueOf(1L), Query.Type.UINT64), 0)); + add(new CompNumTestCase(new EvalResult(1L, Query.Type.INT64), new EvalResult(BigInteger.valueOf(2L), Query.Type.UINT64), -1)); + add(new CompNumTestCase(new EvalResult(2L, Query.Type.INT64), new EvalResult(BigInteger.valueOf(1L), Query.Type.UINT64), 1)); + add(new CompNumTestCase(new EvalResult(1L, Query.Type.INT64), new EvalResult(1d, Query.Type.FLOAT64), 0)); + add(new CompNumTestCase(new EvalResult(1L, Query.Type.INT64), new EvalResult(2d, Query.Type.FLOAT64), -1)); + add(new CompNumTestCase(new EvalResult(2L, Query.Type.INT64), new EvalResult(1d, Query.Type.FLOAT64), 1)); // Special case. - add(new CompNumTestCase(new EvalEngine.EvalResult(BigInteger.valueOf(1l), Query.Type.UINT64), new EvalEngine.EvalResult(-1l, Query.Type.INT64), 1)); - add(new CompNumTestCase(new EvalEngine.EvalResult(BigInteger.valueOf(1l), Query.Type.UINT64), new EvalEngine.EvalResult(1l, Query.Type.INT64), 0)); - add(new CompNumTestCase(new EvalEngine.EvalResult(BigInteger.valueOf(1l), Query.Type.UINT64), new EvalEngine.EvalResult(2l, Query.Type.INT64), -1)); - add(new CompNumTestCase(new EvalEngine.EvalResult(BigInteger.valueOf(2l), Query.Type.UINT64), new EvalEngine.EvalResult(1l, Query.Type.INT64), 1)); - add(new CompNumTestCase(new EvalEngine.EvalResult(BigInteger.valueOf(1l), Query.Type.UINT64), new EvalEngine.EvalResult(BigInteger.valueOf(1l), Query.Type.UINT64), 0)); - add(new CompNumTestCase(new EvalEngine.EvalResult(BigInteger.valueOf(1l), Query.Type.UINT64), new EvalEngine.EvalResult(BigInteger.valueOf(2l), Query.Type.UINT64), -1)); - add(new CompNumTestCase(new EvalEngine.EvalResult(BigInteger.valueOf(2l), Query.Type.UINT64), new EvalEngine.EvalResult(BigInteger.valueOf(1l), Query.Type.UINT64), 1)); - add(new CompNumTestCase(new EvalEngine.EvalResult(BigInteger.valueOf(1l), Query.Type.UINT64), new EvalEngine.EvalResult(1d, Query.Type.FLOAT64), 0)); - add(new CompNumTestCase(new EvalEngine.EvalResult(BigInteger.valueOf(1l), Query.Type.UINT64), new EvalEngine.EvalResult(2d, Query.Type.FLOAT64), -1)); - add(new CompNumTestCase(new EvalEngine.EvalResult(BigInteger.valueOf(2l), Query.Type.UINT64), new EvalEngine.EvalResult(1d, Query.Type.FLOAT64), 1)); - add(new CompNumTestCase(new EvalEngine.EvalResult(1d, Query.Type.FLOAT64), new EvalEngine.EvalResult(1l, Query.Type.INT64), 0)); - add(new CompNumTestCase(new EvalEngine.EvalResult(1d, Query.Type.FLOAT64), new EvalEngine.EvalResult(2l, Query.Type.INT64), -1)); - add(new CompNumTestCase(new EvalEngine.EvalResult(2d, Query.Type.FLOAT64), new EvalEngine.EvalResult(1l, Query.Type.INT64), 1)); - add(new CompNumTestCase(new EvalEngine.EvalResult(1d, Query.Type.FLOAT64), new EvalEngine.EvalResult(BigInteger.valueOf(1l), Query.Type.UINT64), 0)); - add(new CompNumTestCase(new EvalEngine.EvalResult(1d, Query.Type.FLOAT64), new EvalEngine.EvalResult(BigInteger.valueOf(2l), Query.Type.UINT64), -1)); - add(new CompNumTestCase(new EvalEngine.EvalResult(2d, Query.Type.FLOAT64), new EvalEngine.EvalResult(BigInteger.valueOf(1l), Query.Type.UINT64), 1)); - add(new CompNumTestCase(new EvalEngine.EvalResult(1d, Query.Type.FLOAT64), new EvalEngine.EvalResult(1d, Query.Type.FLOAT64), 0)); - add(new CompNumTestCase(new EvalEngine.EvalResult(1d, Query.Type.FLOAT64), new EvalEngine.EvalResult(2d, Query.Type.FLOAT64), -1)); - add(new CompNumTestCase(new EvalEngine.EvalResult(2d, Query.Type.FLOAT64), new EvalEngine.EvalResult(1d, Query.Type.FLOAT64), 1)); + add(new CompNumTestCase(new EvalResult(BigInteger.valueOf(1L), Query.Type.UINT64), new EvalResult(-1L, Query.Type.INT64), 1)); + add(new CompNumTestCase(new EvalResult(BigInteger.valueOf(1L), Query.Type.UINT64), new EvalResult(1L, Query.Type.INT64), 0)); + add(new CompNumTestCase(new EvalResult(BigInteger.valueOf(1L), Query.Type.UINT64), new EvalResult(2L, Query.Type.INT64), -1)); + add(new CompNumTestCase(new EvalResult(BigInteger.valueOf(2L), Query.Type.UINT64), new EvalResult(1L, Query.Type.INT64), 1)); + add(new CompNumTestCase(new EvalResult(BigInteger.valueOf(1L), Query.Type.UINT64), new EvalResult(BigInteger.valueOf(1L), Query.Type.UINT64), 0)); + add(new CompNumTestCase(new EvalResult(BigInteger.valueOf(1L), Query.Type.UINT64), new EvalResult(BigInteger.valueOf(2L), Query.Type.UINT64), -1)); + add(new CompNumTestCase(new EvalResult(BigInteger.valueOf(2L), Query.Type.UINT64), new EvalResult(BigInteger.valueOf(1L), Query.Type.UINT64), 1)); + add(new CompNumTestCase(new EvalResult(BigInteger.valueOf(1L), Query.Type.UINT64), new EvalResult(1d, Query.Type.FLOAT64), 0)); + add(new CompNumTestCase(new EvalResult(BigInteger.valueOf(1L), Query.Type.UINT64), new EvalResult(2d, Query.Type.FLOAT64), -1)); + add(new CompNumTestCase(new EvalResult(BigInteger.valueOf(2L), Query.Type.UINT64), new EvalResult(1d, Query.Type.FLOAT64), 1)); + add(new CompNumTestCase(new EvalResult(1d, Query.Type.FLOAT64), new EvalResult(1L, Query.Type.INT64), 0)); + add(new CompNumTestCase(new EvalResult(1d, Query.Type.FLOAT64), new EvalResult(2L, Query.Type.INT64), -1)); + add(new CompNumTestCase(new EvalResult(2d, Query.Type.FLOAT64), new EvalResult(1L, Query.Type.INT64), 1)); + add(new CompNumTestCase(new EvalResult(1d, Query.Type.FLOAT64), new EvalResult(BigInteger.valueOf(1L), Query.Type.UINT64), 0)); + add(new CompNumTestCase(new EvalResult(1d, Query.Type.FLOAT64), new EvalResult(BigInteger.valueOf(2L), Query.Type.UINT64), -1)); + add(new CompNumTestCase(new EvalResult(2d, Query.Type.FLOAT64), new EvalResult(BigInteger.valueOf(1L), Query.Type.UINT64), 1)); + add(new CompNumTestCase(new EvalResult(1d, Query.Type.FLOAT64), new EvalResult(1d, Query.Type.FLOAT64), 0)); + add(new CompNumTestCase(new EvalResult(1d, Query.Type.FLOAT64), new EvalResult(2d, Query.Type.FLOAT64), -1)); + add(new CompNumTestCase(new EvalResult(2d, Query.Type.FLOAT64), new EvalResult(1d, Query.Type.FLOAT64), 1)); }}; int i = 0; @@ -353,6 +354,83 @@ public void testCompareNumeric() { } } + @Test + public void testToUint64() throws SQLException { + // max bigInteger with Query.Type.UINT64 + VtValue vtValue1 = VtValue.newVtValue(Query.Type.UINT64, VtNumberRange.BIGINTEGER_INT64_MAX.toString().getBytes()); + BigInteger ret = EvalEngine.toUint64(vtValue1); + Assert.assertEquals(VtNumberRange.BIGINTEGER_INT64_MAX, ret); + + // max unit64 with Query.Type.UINT64 + VtValue vtValue2 = VtValue.newVtValue(Query.Type.UINT64, VtNumberRange.UINT64_MAX.toString().getBytes()); + BigInteger ret2 = EvalEngine.toUint64(vtValue2); + Assert.assertEquals(VtNumberRange.UINT64_MAX, ret2); + + // max unit64 with Query.Type.VARCHAR + VtValue vtValue3 = VtValue.newVtValue(Query.Type.VARCHAR, VtNumberRange.UINT64_MAX.toString().getBytes()); + BigInteger ret3 = EvalEngine.toUint64(vtValue3); + Assert.assertEquals(VtNumberRange.UINT64_MAX, ret3); + + // switch VtValue INT64 to UINT64 when value out of range + VtValue vtValue4 = VtValue.newVtValue(Query.Type.INT64, VtNumberRange.BIGINTEGER_INT64_MAX.toString().getBytes()); + BigInteger ret4 = EvalEngine.toUint64(vtValue4); + Assert.assertEquals(VtNumberRange.BIGINTEGER_INT64_MAX, ret4); + + // switch VtValue INT64 to UINT64 when value out of range + VtValue vtValue5 = VtValue.newVtValue(Query.Type.INT64, VtNumberRange.UINT64_MAX.toString().getBytes()); + BigInteger ret5 = EvalEngine.toUint64(vtValue5); + Assert.assertEquals(VtNumberRange.UINT64_MAX, ret5); + + // For non-number type, Int64 is the default. + VtValue vtValue6 = VtValue.newVtValue(Query.Type.VARCHAR, Integer.toString(2).getBytes()); + BigInteger ret6 = EvalEngine.toUint64(vtValue6); + Assert.assertEquals(BigInteger.valueOf(2), ret6); + + // min unit64: 0 + VtValue vtValue7 = VtValue.newVtValue(Query.Type.UINT64, Long.toString(VtNumberRange.UINT64_MIN).getBytes()); + BigInteger ret7 = EvalEngine.toUint64(vtValue7); + Assert.assertEquals(BigInteger.ZERO, ret7); + + try { + String caseStr = "abcd"; + VtValue vtValue = VtValue.newVtValue(Query.Type.VARCHAR, caseStr.getBytes()); + EvalEngine.toUint64(vtValue); + } catch (SQLException e) { + Assert.assertEquals("could not parse value: 'abcd'", e.getMessage()); + } + + try { + VtValue vtValue = VtValue.newVtValue(Query.Type.UINT64, Integer.toString(-1).getBytes()); + EvalEngine.toUint64(vtValue); + } catch (SQLException e) { + Assert.assertEquals("wrong data type UINT64 for -1", e.getMessage()); + } + + // negative number with Query.Type.INT64 + try { + VtValue vtValue = VtValue.newVtValue(Query.Type.INT64, Long.toString(VtNumberRange.INT64_MIN).getBytes()); + EvalEngine.toUint64(vtValue); + } catch (SQLException e) { + Assert.assertEquals("negative number cannot be converted to unsigned: -9223372036854775808", e.getMessage()); + } + + try { + VtValue vtValue = VtValue.newVtValue(Query.Type.INT64, Double.toString(1.2).getBytes()); + EvalEngine.toUint64(vtValue); + } catch (NumberFormatException e) { + Assert.assertEquals("For input string: \"1.2\"", e.getMessage()); + } + + // bigger than maxBigInteger + try { + BigInteger biggerMaxBigInteger = VtNumberRange.BIGINTEGER_INT64_MAX.add(BigInteger.ONE); + VtValue vtValue = VtValue.newVtValue(Query.Type.UINT64, biggerMaxBigInteger.toString().getBytes()); + EvalEngine.toUint64(vtValue); + } catch (SQLException e) { + Assert.assertEquals("wrong data type UINT64 for 18446744073709551616", e.getMessage()); + } + } + private boolean VtResultValueCompare(VtResultValue v1, VtResultValue v2) { if (v1.isNull() && v2.isNull()) { return true; @@ -448,11 +526,11 @@ public String toString() { @AllArgsConstructor @Getter private class NumAddTestCase { - EvalEngine.EvalResult value1; + EvalResult value1; - EvalEngine.EvalResult value2; + EvalResult value2; - EvalEngine.EvalResult result; + EvalResult result; @Override public String toString() { @@ -466,7 +544,7 @@ public String toString() { private class CFNTestCase { Query.Type type; - EvalEngine.EvalResult value; + EvalResult value; VtValue out; @@ -480,9 +558,9 @@ public String toString() { @AllArgsConstructor @Getter private class CompNumTestCase { - EvalEngine.EvalResult value1; + EvalResult value1; - EvalEngine.EvalResult value2; + EvalResult value2; Integer out; diff --git a/src/test/java/com/jd/jdbc/planbuilder/AbstractPlanTest.java b/src/test/java/com/jd/jdbc/planbuilder/AbstractPlanTest.java index 375eeb1..ad09cc0 100644 --- a/src/test/java/com/jd/jdbc/planbuilder/AbstractPlanTest.java +++ b/src/test/java/com/jd/jdbc/planbuilder/AbstractPlanTest.java @@ -22,8 +22,10 @@ import com.jd.BaseTest; import com.jd.jdbc.VSchemaManager; import com.jd.jdbc.engine.Plan; +import com.jd.jdbc.engine.PrimitiveEngine; import com.jd.jdbc.key.Destination; import com.jd.jdbc.key.DestinationShard; +import com.jd.jdbc.planbuilder.gen4.Gen4Planner; import com.jd.jdbc.planbuilder.vschema.AbstractTable; import com.jd.jdbc.planbuilder.vschema.AutoIncrement; import com.jd.jdbc.planbuilder.vschema.ColumnVindexesItem; @@ -36,6 +38,8 @@ import com.jd.jdbc.planbuilder.vschema.User; import com.jd.jdbc.planbuilder.vschema.Vindexes; import com.jd.jdbc.sqlparser.SQLUtils; +import com.jd.jdbc.sqlparser.SqlParser; +import com.jd.jdbc.sqlparser.ast.SQLExpr; import com.jd.jdbc.sqlparser.ast.SQLStatement; import com.jd.jdbc.sqlparser.ast.expr.SQLIdentifierExpr; import com.jd.jdbc.sqlparser.ast.expr.SQLMethodInvokeExpr; @@ -53,12 +57,13 @@ import com.jd.jdbc.sqlparser.dialect.mysql.BindVarNeeds; import com.jd.jdbc.sqlparser.dialect.mysql.ast.statement.MySqlDeleteStatement; import com.jd.jdbc.sqlparser.dialect.mysql.ast.statement.MySqlInsertReplaceStatement; +import com.jd.jdbc.sqlparser.dialect.mysql.ast.statement.MySqlSelectQueryBlock; import com.jd.jdbc.sqlparser.dialect.mysql.ast.statement.MySqlUpdateStatement; import com.jd.jdbc.sqlparser.dialect.mysql.parser.MySqlLexer; import com.jd.jdbc.sqlparser.parser.Lexer; import com.jd.jdbc.sqlparser.utils.StringUtils; import com.jd.jdbc.sqlparser.utils.TableNameUtils; -import static com.jd.jdbc.vindexes.Vschema.TYPE_REFERENCE; +import com.jd.jdbc.vindexes.VschemaConstant; import io.vitess.proto.Query; import java.io.IOException; import java.nio.charset.StandardCharsets; @@ -71,8 +76,6 @@ import java.util.Map; import vschema.Vschema; -import static com.jd.jdbc.vindexes.Vschema.TYPE_REFERENCE; - public class AbstractPlanTest extends BaseTest { protected static final Map uniqueTables = new HashMap<>(); @@ -96,7 +99,7 @@ public class AbstractPlanTest extends BaseTest { Symtab.registSingleColumnVindex("hash_dup"); } - protected static Plan build(String query, VSchemaManager vm) throws Exception { + protected static Plan build(String query, VSchemaManager vm, boolean v3) throws Exception { Map prop = parseComment(query); SQLStatement stmt = SQLUtils.parseSingleMysqlStatement(query); for (Map.Entry entry : prop.entrySet()) { @@ -130,7 +133,7 @@ protected static Plan build(String query, VSchemaManager vm) throws Exception { defaultKeyspace = uniqueTables.get(tableName.toLowerCase()); } } else if (stmt instanceof MySqlDeleteStatement) { - MySqlDeleteStatement deleteStatement = ((MySqlDeleteStatement) stmt); + MySqlDeleteStatement deleteStatement = (MySqlDeleteStatement) stmt; if (deleteStatement.getFrom() != null) { defaultKeyspace = processSQLTableSource(deleteStatement.getFrom()); } else if (deleteStatement.getTableSource() != null) { @@ -152,6 +155,40 @@ protected static Plan build(String query, VSchemaManager vm) throws Exception { throw new SQLException("Not found defaultKeyspace"); } } + if (destination == null && stmt instanceof SQLSelectStatement && + (((SQLSelectStatement) stmt).getSelect().getQuery() instanceof SQLUnionQuery || ((SQLSelectStatement) stmt).getSelect().getQuery() instanceof MySqlSelectQueryBlock)) { + PrimitiveEngine instruction = null; + if (v3) { + if (((SQLSelectStatement) stmt).getSelect().getQuery() instanceof SQLUnionQuery) { + instruction = PlanBuilder.buildUnionPlan((SQLSelectStatement) stmt, vm, defaultKeyspace); + } + if (((SQLSelectStatement) stmt).getSelect().getQuery() instanceof MySqlSelectQueryBlock) { + instruction = PlanBuilder.buildSelectPlan((SQLSelectStatement) stmt, vm, defaultKeyspace); + } + } else { + if ((((SQLSelectStatement) stmt).getSelect().getQuery() instanceof MySqlSelectQueryBlock)) { + //todo 暂时跳过derived table + SQLTableSource from = ((SQLSelectStatement) stmt).getSelect().getQueryBlock().getFrom(); + if (PlanBuilder.hasSubquery(from)) { + return null; + } + //todo 暂时跳过join和where带subquery的测试用例 + SQLExpr where = ((SQLSelectStatement) stmt).getSelect().getQueryBlock().getWhere(); + List filters = PlanBuilder.splitAndExpression(new ArrayList<>(), where); + for (SQLExpr filter : filters) { + if (PlanBuilder.hasSubquery(filter)) { + return null; + } + } + + if (((SQLSelectStatement) stmt).getSelect().getQueryBlock().getFrom() instanceof SQLJoinTableSource) { + return null; + } + } + instruction = Gen4Planner.gen4SelectStmtPlanner(null, defaultKeyspace, (SQLSelectStatement) stmt, null, vm); + } + return new Plan(SqlParser.astToStatementType(stmt), instruction, new BindVarNeeds()); + } return PlanBuilder.buildFromStmt(stmt, vm, defaultKeyspace, new BindVarNeeds(), destination); } @@ -203,7 +240,11 @@ private static String processSQLTableSource(SQLTableSource tableSource) throws S defaultKeyspace = processExprTableSource((SQLExprTableSource) leftTableSource); } else if (leftTableSource instanceof SQLSubqueryTableSource) { SQLTableSource from = ((SQLSubqueryTableSource) leftTableSource).getSelect().getQueryBlock().getFrom(); - defaultKeyspace = processExprTableSource((SQLExprTableSource) from); + if (from instanceof SQLJoinTableSource) { + defaultKeyspace = processSQLTableSource(from); + } else { + defaultKeyspace = processExprTableSource((SQLExprTableSource) from); + } } if (StringUtils.isEmpty(defaultKeyspace)) { if (rightTableSource instanceof SQLExprTableSource) { @@ -237,6 +278,8 @@ private static String processSQLTableSource(SQLTableSource tableSource) throws S defaultKeyspace = processExprTableSource((SQLExprTableSource) from); } } + } else if (subqueryTableSource instanceof SQLSubqueryTableSource) { + defaultKeyspace = processSQLTableSource(subqueryTableSource); } } @@ -255,7 +298,8 @@ private static String processExprTableSource(SQLExprTableSource exprTableSource) if ("unsharded_no_metadata".equalsIgnoreCase(tableName) || "dual".equalsIgnoreCase(tableName)) { defaultKeyspace = "main"; } else { - defaultKeyspace = uniqueTables.get(tableName.toLowerCase()); + String tableNameOrg = tableName.toLowerCase().replaceAll("`", ""); + defaultKeyspace = uniqueTables.get(tableNameOrg); } } return defaultKeyspace; @@ -418,7 +462,7 @@ private Map buildVschemaTables(Object obj) { tableMap.put("unsharded_authoritative", this.buildVschemaTable(tables.getUnshardedAuthoritative())); tableMap.put("seq", this.buildVschemaTable(tables.getSeq())); } - tableMap.put("dual", Vschema.Table.newBuilder().setType(TYPE_REFERENCE).build()); + tableMap.put("dual", Vschema.Table.newBuilder().setType(VschemaConstant.TYPE_REFERENCE).build()); return tableMap; } diff --git a/src/test/java/com/jd/jdbc/planbuilder/AdvancedPlanTest.java b/src/test/java/com/jd/jdbc/planbuilder/AdvancedPlanTest.java index 891018f..b796eaa 100644 --- a/src/test/java/com/jd/jdbc/planbuilder/AdvancedPlanTest.java +++ b/src/test/java/com/jd/jdbc/planbuilder/AdvancedPlanTest.java @@ -86,7 +86,7 @@ private static SQLJoinTableSource generate(SQLJoinTableSource joinTableSource) { @Test public void case02() throws Exception { VSchemaManager vm = loadSchema("src/test/resources/plan/plan_schema.json"); - Plan plan = build("select * from unsharded_a left join unsharded_b on unsharded_a.id = unsharded_b.id", vm); + Plan plan = build("select * from unsharded_a left join unsharded_b on unsharded_a.id = unsharded_b.id", vm, true); System.out.println("plan = " + plan); } } diff --git a/src/test/java/com/jd/jdbc/planbuilder/Gen4PlanTest.java b/src/test/java/com/jd/jdbc/planbuilder/Gen4PlanTest.java new file mode 100644 index 0000000..a80859d --- /dev/null +++ b/src/test/java/com/jd/jdbc/planbuilder/Gen4PlanTest.java @@ -0,0 +1,39 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder; + +import java.io.IOException; +import org.junit.Test; + +public class Gen4PlanTest extends PlanTest { + @Test + public void testOne() throws IOException { + g4AssertTestFile("src/test/resources/plan/one_cases.txt", vm, 0); + testFile("src/test/resources/plan/one_cases.txt", vm, 0); + } + + @Test + public void g4TestPlan() throws IOException { + g4AssertTestFile("src/test/resources/plan/filter_cases.txt", vm, 0); + g4AssertTestFile("src/test/resources/plan/aggr_cases.txt", vm, 0); + g4AssertTestFile("src/test/resources/plan/postprocess_cases.txt", vm, 0); + g4AssertTestFile("src/test/resources/plan/from_cases.txt", vm, 0); + g4AssertTestFile("src/test/resources/plan/memory_sort_cases.txt", vm, 0); + } +} diff --git a/src/test/java/com/jd/jdbc/planbuilder/PlanPerformanceTest.java b/src/test/java/com/jd/jdbc/planbuilder/PlanPerformanceTest.java index 939aed5..bcb032d 100644 --- a/src/test/java/com/jd/jdbc/planbuilder/PlanPerformanceTest.java +++ b/src/test/java/com/jd/jdbc/planbuilder/PlanPerformanceTest.java @@ -36,7 +36,7 @@ public void init() throws IOException { public void testOnce() throws Exception { long sTime = System.currentTimeMillis(); for (int i = 0; i < 10000000; i++) { - PlanTest.build("select a, b, count(*) from user group by a order by b limit 10", vm); + PlanTest.build("select a, b, count(*) from user group by a order by b limit 10", vm, true); } long eTime = System.currentTimeMillis(); diff --git a/src/test/java/com/jd/jdbc/planbuilder/PlanTest.java b/src/test/java/com/jd/jdbc/planbuilder/PlanTest.java index 698a251..8ef6e00 100644 --- a/src/test/java/com/jd/jdbc/planbuilder/PlanTest.java +++ b/src/test/java/com/jd/jdbc/planbuilder/PlanTest.java @@ -15,6 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. */ + package com.jd.jdbc.planbuilder; import com.fasterxml.jackson.annotation.JsonProperty; @@ -41,7 +42,22 @@ import com.jd.jdbc.engine.SingleRowEngine; import com.jd.jdbc.engine.SubQueryEngine; import com.jd.jdbc.engine.UpdateEngine; +import com.jd.jdbc.engine.gen4.AbstractAggregateGen4; +import com.jd.jdbc.engine.gen4.ConcatenateGen4Engine; +import com.jd.jdbc.engine.gen4.DistinctGen4Engine; +import com.jd.jdbc.engine.gen4.FilterGen4Engine; +import com.jd.jdbc.engine.gen4.GroupByParams; +import com.jd.jdbc.engine.gen4.JoinGen4Engine; +import com.jd.jdbc.engine.gen4.LimitGen4Engine; +import com.jd.jdbc.engine.gen4.MemorySortGen4Engine; +import com.jd.jdbc.engine.gen4.OrderByParamsGen4; +import com.jd.jdbc.engine.gen4.OrderedAggregateGen4Engine; +import com.jd.jdbc.engine.gen4.RouteGen4Engine; +import com.jd.jdbc.engine.gen4.ScalarAggregateGen4Engine; +import com.jd.jdbc.engine.gen4.SimpleProjectionGen4Engine; +import com.jd.jdbc.engine.vcursor.NoopVCursor; import com.jd.jdbc.evalengine.EvalEngine; +import com.jd.jdbc.evalengine.EvalResult; import com.jd.jdbc.key.DestinationShard; import com.jd.jdbc.sqlparser.SQLUtils; import com.jd.jdbc.sqlparser.utils.StringUtils; @@ -50,7 +66,6 @@ import com.jd.jdbc.util.JsonUtil; import com.jd.jdbc.vindexes.VKeyspace; import io.netty.util.internal.StringUtil; -import static io.netty.util.internal.StringUtil.NEWLINE; import io.vitess.proto.Query; import java.io.BufferedReader; import java.io.IOException; @@ -61,51 +76,58 @@ import java.sql.SQLException; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.stream.Collectors; import lombok.AllArgsConstructor; import lombok.Data; import org.junit.Assert; import static org.junit.Assert.assertEquals; +import org.junit.Before; import org.junit.Test; public class PlanTest extends AbstractPlanTest { - @Test - public void testOne() throws IOException { - VSchemaManager vm = loadSchema("src/test/resources/plan/plan_schema.json"); - testFile("src/test/resources/plan/one_cases.txt", vm, 0); + private final String samePlanMarker = "Gen4 plan same as above"; + + private final String gen4ErrorPrefix = "Gen4 error: "; + + private final String gen3Skip = "Gen3 skip"; + + protected VSchemaManager vm; + + @Before + public void init() throws IOException { + vm = loadSchema("src/test/resources/plan/plan_schema.json"); } @Test public void testPlan() throws IOException { - VSchemaManager vm = loadSchema("src/test/resources/plan/plan_schema.json"); testFile("src/test/resources/plan/aggr_cases.txt", vm, 0); - testFile("src/test/resources/plan/dml_insert_cases.txt", vm, 0); testFile("src/test/resources/plan/filter_cases.txt", vm, 0); testFile("src/test/resources/plan/from_cases.txt", vm, 0); testFile("src/test/resources/plan/memory_sort_cases.txt", vm, 0); testFile("src/test/resources/plan/postprocess_cases.txt", vm, 0); testFile("src/test/resources/plan/select_cases.txt", vm, 0); testFile("src/test/resources/plan/union_cases.txt", vm, 0); + testFile("src/test/resources/plan/dml_insert_cases.txt", vm, 0); testFile("src/test/resources/plan/dml_delete_cases.txt", vm, 0); testFile("src/test/resources/plan/dml_update_cases.txt", vm, 0); } @Test public void testDestination() throws IOException { - VSchemaManager vm = loadSchema("src/test/resources/plan/plan_schema.json"); testFile("src/test/resources/plan/destination_case.txt", vm, 0); } - private void testFile(String filename, VSchemaManager vm, Integer startPos) throws IOException { + protected void testFile(String filename, VSchemaManager vm, Integer startPos) throws IOException { List testCaseList = iterateExecFile(filename); for (TestCase testCase : testCaseList) { - if (testCase.lineno <= startPos) { + if (testCase.lineno <= startPos || testCase.output.equals("")) { continue; } - printComment("Test Case: " + testCase.comments); + printComment("Gen3 Test Case: " + testCase.comments); printNormal("Input SQL: " + testCase.input); TestPlan fromFile; @@ -119,18 +141,138 @@ private void testFile(String filename, VSchemaManager vm, Integer startPos) thro } try { - Plan plan = build(testCase.input, vm); + Plan plan = build(testCase.input, vm, true); TestPlan fromCode = this.format(plan, testCase.input); printInfo("From Code: " + fromCode); - assertEquals(printFail("File: " + filename + ", Line: " + testCase.lineno + " is [FAIL]"), fromFile, fromCode); - printOk("File: " + filename + ", Line: " + testCase.lineno + " is [OK]"); } catch (Exception e) { String error = e.getMessage().replaceAll("\"", ""); printInfo("From Code: " + error); assertEquals(printFail("File: " + filename + ", Line: " + testCase.lineno + " is [FAIL]"), fromFile.errorMessage.toLowerCase(), error.toLowerCase()); - printOk("File: " + filename + ", Line: " + testCase.lineno + " is [OK]"); } + printOk("File: " + filename + ", Line: " + testCase.lineno + " is [OK]"); + System.out.println(); + } + } + + protected void g4TestFile(String filename, VSchemaManager vm, Integer startPos) throws IOException { + List testCaseList = iterateExecFile(filename); + for (TestCase testCase : testCaseList) { + + if (testCase.lineno <= startPos) { + continue; + } + if (testCase.output2ndPlanner.equals("")) { + continue; + + } + // 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 Gen4 planner + // produces the same plan as the V3 planner does + TestPlan g4fromFile; + try { + g4fromFile = new ObjectMapper().readValue(testCase.output2ndPlanner, TestPlan.class); + } catch (Exception e) { + g4fromFile = new TestPlan(); + g4fromFile.setErrorMessage(testCase.output2ndPlanner.replaceAll("\"", "")); + } + + try { + Plan plan = build(testCase.input, vm, false); + if (plan == null) { + continue; + } + TestPlan fromCode = this.format(plan, testCase.input); + if (!g4fromFile.equals(fromCode)) { + printComment("Gen4 Test Case: " + testCase.comments); + printNormal("Input SQL: " + testCase.input); + printInfo("From File: " + g4fromFile); + printInfo("From Code: " + fromCode); + System.out.println(printFail("File: " + filename + ", Line: " + testCase.lineno + " is [FAIL]")); + System.out.println(); + } + } catch (Exception e) { + if (e instanceof NullPointerException) { + printComment("Gen4 Test Case: " + testCase.comments); + printNormal("Input SQL: " + testCase.input); + System.out.println("空指针"); + System.out.println(printFail("File: " + filename + ", Line: " + testCase.lineno + " is [FAIL]")); + continue; + } + String error; + if (e.getMessage() != null) { + error = e.getMessage().replaceAll("\"", ""); + } else { + error = e.toString(); + } + if (g4fromFile.errorMessage == null || !g4fromFile.errorMessage.toLowerCase().equals(error.toLowerCase())) { + printComment("Gen4 Test Case: " + testCase.comments); + printNormal("Input SQL: " + testCase.input); + error = error.replaceAll("\"", ""); + if (g4fromFile.errorMessage != null) { + printInfo("From File: " + g4fromFile.errorMessage); + } else { + printInfo("From File: " + g4fromFile); + } + printInfo("From Code: " + error); + System.out.println(printFail("File: " + filename + ", Line: " + testCase.lineno + " is [FAIL],errorMessage " + e.getMessage())); + System.out.println(); + } + } + } + } + + protected void g4AssertTestFile(String filename, VSchemaManager vm, Integer startPos) throws IOException { + List testCaseList = iterateExecFile(filename); + for (TestCase testCase : testCaseList) { + + if (testCase.lineno <= startPos) { + continue; + } + if (testCase.output2ndPlanner.equals("")) { + continue; + } + // 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 Gen4 planner + // produces the same plan as the V3 planner does + printComment("Gen4 Test Case: " + testCase.comments); + printNormal("Input SQL: " + testCase.input); + TestPlan g4fromFile; + try { + g4fromFile = new ObjectMapper().readValue(testCase.output2ndPlanner, TestPlan.class); + } catch (Exception e) { + g4fromFile = new TestPlan(); + g4fromFile.setErrorMessage(testCase.output2ndPlanner.replaceAll("\"", "")); + } + try { + Plan plan = build(testCase.input, vm, false); + if (plan == null) { + printInfo("skip test"); + continue; + } + TestPlan fromCode = this.format(plan, testCase.input); + printInfo("From File: " + g4fromFile); + printInfo("From Code: " + fromCode); + assertEquals(printFail("File: " + filename + ", Line: " + testCase.lineno + " is [FAIL]"), g4fromFile, fromCode); + } catch (Exception e) { + String error = e.getMessage().replaceAll("\"", ""); + if (g4fromFile.errorMessage != null) { + printInfo("From File: " + g4fromFile.errorMessage); + } else { + printInfo("From File: " + g4fromFile); + } + printInfo("From Code: " + error); + assertEquals(printFail("File: " + filename + ", Line: " + testCase.lineno + " is [FAIL]"), g4fromFile.errorMessage.toLowerCase(), error.toLowerCase()); + } + printOk("File: " + filename + ", Line: " + testCase.lineno + " is [OK]"); System.out.println(); } } @@ -143,7 +285,7 @@ private List iterateExecFile(String filename) throws IOException { StringBuilder comments = new StringBuilder(); while ((line = br.readLine()) != null) { lineno++; - if (StringUtil.isNullOrEmpty(line) || NEWLINE.equalsIgnoreCase(line)) { + if (StringUtil.isNullOrEmpty(line) || StringUtil.NEWLINE.equalsIgnoreCase(line)) { continue; } if (line.startsWith("#")) { @@ -153,14 +295,38 @@ private List iterateExecFile(String filename) throws IOException { line = line.substring(1).substring(0, line.length() - 2); String l; StringBuilder output = new StringBuilder(); - while ((l = br.readLine()) != null) { + while (true) { + l = br.readLine(); + if (l.equals(gen3Skip)) { + output = new StringBuilder(); + break; + } lineno++; output.append(l); if (l.startsWith("}") || l.startsWith("\"")) { break; } } - testCaseList.add(new TestCase(filename, lineno, line, output.toString(), comments.toString())); + StringBuilder output2Planner = new StringBuilder(); + l = br.readLine(); + lineno++; + if (l != null && l.equals(samePlanMarker)) { + output2Planner = output; + } else if (l != null && l.startsWith(gen4ErrorPrefix)) { + output2Planner = new StringBuilder(l.substring(gen4ErrorPrefix.length())); + } else if (l != null && l.startsWith("{")) { + output2Planner.append(l); + while ((l = br.readLine()) != null) { + lineno++; + output2Planner.append(l); + if (l.startsWith("}") || l.startsWith("\"")) { + break; + } + } + } else if (l != null) { + output2Planner.append(l); + } + testCaseList.add(new TestCase(filename, lineno, line, output.toString(), output2Planner.toString(), comments.toString())); comments = new StringBuilder(); } return testCaseList; @@ -190,12 +356,32 @@ private TestPlan format(Plan plan, String query) { instructions = this.formatProjectionEngine((ProjectionEngine) primitive); } else if (primitive instanceof ConcatenateEngine) { instructions = this.formatConcatenateEngine((ConcatenateEngine) primitive); + } else if (primitive instanceof ConcatenateGen4Engine) { + instructions = this.formatConcatenateEngine((ConcatenateGen4Engine) primitive); + } else if (primitive instanceof DistinctGen4Engine) { + instructions = this.formatDistinctEngine((DistinctGen4Engine) primitive); } else if (primitive instanceof DeleteEngine) { instructions = this.formatDeleteEngine((DeleteEngine) primitive); } else if (primitive instanceof UpdateEngine) { instructions = this.formatUpdateEngine((UpdateEngine) primitive); } else if (primitive instanceof SendEngine) { instructions = this.formatSendEngine((SendEngine) primitive); + } else if (primitive instanceof ScalarAggregateGen4Engine) { + instructions = this.formatScalarAggregateGen4Engine((ScalarAggregateGen4Engine) primitive); + } else if (primitive instanceof MemorySortGen4Engine) { + instructions = this.formatMemorySortGen4Engine((MemorySortGen4Engine) primitive); + } else if (primitive instanceof OrderedAggregateGen4Engine) { + instructions = this.formatOrderedAggregateGen4Engine((OrderedAggregateGen4Engine) primitive); + } else if (primitive instanceof RouteGen4Engine) { + instructions = this.formatRouteEngine((RouteGen4Engine) primitive); + } else if (primitive instanceof LimitGen4Engine) { + instructions = this.formatLimitEngine((LimitGen4Engine) primitive); + } else if (primitive instanceof JoinGen4Engine) { + instructions = this.formatJoinGen4Engine((JoinGen4Engine) primitive); + } else if (primitive instanceof SimpleProjectionGen4Engine) { + instructions = this.formatSimpleProjection((SimpleProjectionGen4Engine) primitive); + } else if (primitive instanceof FilterGen4Engine) { + instructions = this.formatFilterEngine((FilterGen4Engine) primitive); } TestPlan testPlan = new TestPlan(); @@ -205,6 +391,217 @@ private TestPlan format(Plan plan, String query) { return testPlan; } + private Instructions formatDistinctEngine(DistinctGen4Engine engine) { + Instructions instructions = new Instructions(); + instructions.setOperatorType("Distinct"); + + instructions.setResultColumns(engine.getCheckCols().size()); + + List inputList = new ArrayList<>(); + PrimitiveEngine engineInput = engine.getSource(); + if (engineInput instanceof RouteGen4Engine) { + inputList.add(this.formatRouteEngine((RouteGen4Engine) engineInput)); + } else if (engineInput instanceof OrderedAggregateGen4Engine) { + inputList.add(this.formatOrderedAggregateGen4Engine((OrderedAggregateGen4Engine) engineInput)); + } else if (engineInput instanceof ScalarAggregateGen4Engine) { + inputList.add(this.formatScalarAggregateGen4Engine((ScalarAggregateGen4Engine) engineInput)); + } else if (engineInput instanceof MemorySortGen4Engine) { + inputList.add(this.formatMemorySortGen4Engine((MemorySortGen4Engine) engineInput)); + } else if (engineInput instanceof LimitGen4Engine) { + inputList.add(this.formatLimitEngine((LimitGen4Engine) engineInput)); + } else if (engineInput instanceof JoinGen4Engine) { + inputList.add(this.formatJoinGen4Engine((JoinGen4Engine) engineInput)); + } else if (engineInput instanceof SimpleProjectionGen4Engine) { + inputList.add(this.formatSimpleProjection((SimpleProjectionGen4Engine) engineInput)); + } else if (engineInput instanceof FilterGen4Engine) { + inputList.add(this.formatFilterEngine((FilterGen4Engine) engineInput)); + } + instructions.setInputs(inputList); + return instructions; + } + + private Instructions formatLimitEngine(LimitGen4Engine engine) { + Instructions instructions = new Instructions(); + instructions.setOperatorType("Limit"); + try { + if ((engine.getCount() != null) && (engine.getCount() instanceof EvalEngine.Literal)) { + instructions.setCount(engine.getCount(new NoopVCursor(), null)); + } + } catch (SQLException e) { + e.printStackTrace(); + } + + PrimitiveEngine engineInput = engine.getInput(); + if (engineInput != null) { + List inputList = new ArrayList<>(); + if (engineInput instanceof RouteGen4Engine) { + inputList.add(this.formatRouteEngine((RouteGen4Engine) engineInput)); + } else if (engineInput instanceof OrderedAggregateGen4Engine) { + inputList.add(this.formatOrderedAggregateGen4Engine((OrderedAggregateGen4Engine) engineInput)); + } else if (engineInput instanceof ScalarAggregateGen4Engine) { + inputList.add(this.formatScalarAggregateGen4Engine((ScalarAggregateGen4Engine) engineInput)); + } else if (engineInput instanceof MemorySortGen4Engine) { + inputList.add(this.formatMemorySortGen4Engine((MemorySortGen4Engine) engineInput)); + } else if (engineInput instanceof JoinGen4Engine) { + inputList.add(this.formatJoinGen4Engine((JoinGen4Engine) engineInput)); + } else if (engineInput instanceof SimpleProjectionGen4Engine) { + inputList.add(this.formatSimpleProjection((SimpleProjectionGen4Engine) engineInput)); + } else if (engineInput instanceof FilterGen4Engine) { + inputList.add(this.formatFilterEngine((FilterGen4Engine) engineInput)); + } + instructions.setInputs(inputList); + } + return instructions; + } + + private Instructions formatRouteEngine(RouteGen4Engine engine) { + VKeyspace vKeyspace = engine.getRoutingParameters().getKeyspace(); + Keyspace keyspace = new Keyspace(); + keyspace.setName(vKeyspace.getName()); + keyspace.setSharded(vKeyspace.getSharded()); + Instructions instructions = new Instructions(); + instructions.setOperatorType("Route"); + Engine.RouteOpcode routeOpcode = engine.getRoutingParameters().getRouteOpcode(); + instructions.setVariant(routeOpcode.name()); + instructions.setKeyspace(keyspace); + instructions.setFieldQuery(engine.getFieldQuery()); + instructions.setQuery(engine.getQuery()); + instructions.setTable(StringUtil.isNullOrEmpty(engine.getTableName()) ? null : engine.getTableName()); + List sysTableKeyspaceExpr = engine.getRoutingParameters().getSystableTableSchema(); + if (sysTableKeyspaceExpr != null && !sysTableKeyspaceExpr.isEmpty()) { + List exprList = new ArrayList<>(); + for (EvalEngine.Expr expr : sysTableKeyspaceExpr) { + if (expr instanceof EvalEngine.Literal) { + exprList.add(expr.string()); + } + } + instructions.setSysTableKeyspaceExpr(exprList); + } + if (engine.getTruncateColumnCount() > 0) { + instructions.setResultColumns(engine.getTruncateColumnCount()); + } + instructions.setOrderBy(buildOrderbyParamString(engine.getOrderBy())); + List values = new ArrayList<>(); + for (EvalEngine.Expr expr : engine.getRoutingParameters().getValues()) { + if (expr == null) { + continue; + } + String string = expr.string(); + values.add(string); + } + if (CollectionUtils.isNotEmpty(values)) { + instructions.setValueList(values); + } + return instructions; + } + + private String buildOrderbyParamString(List orderBys) { + if (CollectionUtils.isNotEmpty(orderBys)) { + List results = new ArrayList<>(); + for (OrderByParamsGen4 obp : orderBys) { + String val = String.valueOf(obp.getCol()); + if (obp.getStarColFixedIndex() > obp.getCol()) { + val = String.valueOf(obp.getStarColFixedIndex()); + } + if (obp.getWeightStrCol() != -1 && obp.getWeightStrCol() != obp.getCol()) { + val = String.format("(%s|%d)", val, obp.getWeightStrCol()); + } + if (obp.isDesc()) { + val += " DESC"; + } else { + val += " ASC"; + } + results.add(val); + } + return String.join(", ", results); + } + return null; + } + + private Instructions formatScalarAggregateGen4Engine(ScalarAggregateGen4Engine engine) { + Instructions instructions = new Instructions(); + instructions.setOperatorType("Aggregate"); + instructions.setVariant("Scalar"); + + instructions.setAggregates(buildAggregateParamsString(engine.getAggregates())); + if (engine.getTruncateColumnCount() > 0) { + instructions.setResultColumns(engine.getTruncateColumnCount()); + } + PrimitiveEngine engineInput = engine.getInput(); + + if (engineInput != null) { + List inputList = new ArrayList<>(); + if (engineInput instanceof RouteGen4Engine) { + inputList.add(this.formatRouteEngine((RouteGen4Engine) engineInput)); + } else if (engineInput instanceof OrderedAggregateGen4Engine) { + inputList.add(this.formatOrderedAggregateGen4Engine((OrderedAggregateGen4Engine) engineInput)); + } else if (engineInput instanceof ScalarAggregateGen4Engine) { + inputList.add(this.formatScalarAggregateGen4Engine((ScalarAggregateGen4Engine) engineInput)); + } else if (engineInput instanceof MemorySortGen4Engine) { + inputList.add(this.formatMemorySortGen4Engine((MemorySortGen4Engine) engineInput)); + } else if (engineInput instanceof LimitGen4Engine) { + inputList.add(this.formatLimitEngine((LimitGen4Engine) engineInput)); + } else if (engineInput instanceof JoinGen4Engine) { + inputList.add(this.formatJoinGen4Engine((JoinGen4Engine) engineInput)); + } else if (engineInput instanceof SimpleProjectionGen4Engine) { + inputList.add(this.formatSimpleProjection((SimpleProjectionGen4Engine) engineInput)); + } else if (engineInput instanceof FilterGen4Engine) { + inputList.add(this.formatFilterEngine((FilterGen4Engine) engineInput)); + } + instructions.setInputs(inputList); + } + return instructions; + } + + private String buildAggregateParamsString(List aggregates) { + if (CollectionUtils.isEmpty(aggregates)) { + return null; + } + List results = new ArrayList<>(); + for (AbstractAggregateGen4.AggregateParams ap : aggregates) { + String result; + String keyCol = String.valueOf(ap.getCol()); + if (ap.isWAssigned()) { + keyCol = String.format("%s|%d", keyCol, ap.getWCol()); + } + String dispOrigOp = ""; + if (ap.getOrigOpcode() != null && ap.getOrigOpcode() != ap.getOpcode()) { + dispOrigOp = "_" + getAggregateOpcodeG4String(ap.getOrigOpcode()); + } + if (!ap.getAlias().equals("")) { + String alias; + switch (ap.getOpcode()) { + case AggregateCountStar: + case AggregateCount: + case AggregateSum: + case AggregateMin: + case AggregateMax: + case AggregateCountDistinct: + case AggregateSumDistinct: + alias = ap.getAlias().toLowerCase(); + break; + default: + alias = ap.getAlias(); + } + + result = String.format("%s%s(%s) AS %s", getAggregateOpcodeG4String(ap.getOpcode()), dispOrigOp, keyCol, alias); + } else { + result = String.format("%s%s(%s)", getAggregateOpcodeG4String(ap.getOpcode()), dispOrigOp, keyCol); + } + results.add(result); + } + return String.join(", ", results); + } + + private String getAggregateOpcodeG4String(Engine.AggregateOpcodeG4 opcodeG4) { + for (Map.Entry entry : AbstractAggregateGen4.SUPPORTED_AGGREGATES.entrySet()) { + if (entry.getValue() == opcodeG4) { + return entry.getKey(); + } + } + return "ERROR"; + } + private Instructions formatSendEngine(SendEngine engine) { VKeyspace vKeyspace = engine.getKeyspace(); Keyspace keyspace = new Keyspace(); @@ -246,8 +643,7 @@ private Instructions formatRouteEngine(RouteEngine engine) { List exprList = new ArrayList<>(); for (EvalEngine.Expr expr : sysTableKeyspaceExpr) { if (expr instanceof EvalEngine.Literal) { - Query.Type type = ((EvalEngine.Literal) expr).getVal().getType(); - exprList.add(type.name() + "(\"" + expr.string() + "\")"); + exprList.add(expr.string()); } } instructions.setSysTableKeyspaceExpr(exprList); @@ -305,7 +701,6 @@ private Instructions formatOrderedAggregateEngine(OrderedAggregateEngine engine) if (keyList != null && !keyList.isEmpty()) { instructions.setGroupBy(keyList.stream().map(String::valueOf).collect(Collectors.joining(", "))); } - PrimitiveEngine engineInput = engine.getInput(); if (engineInput != null) { List inputList = new ArrayList<>(); @@ -323,6 +718,60 @@ private Instructions formatOrderedAggregateEngine(OrderedAggregateEngine engine) return instructions; } + private Instructions formatOrderedAggregateGen4Engine(OrderedAggregateGen4Engine engine) { + Instructions instructions = new Instructions(); + instructions.setOperatorType("Aggregate"); + instructions.setVariant("Ordered"); + + instructions.setGroupBy(buildGroupByParamsString(engine.getGroupByKeys())); + instructions.setAggregates(buildAggregateParamsString(engine.getAggregates())); + + if (engine.getTruncateColumnCount() > 0) { + instructions.setResultColumns(engine.getTruncateColumnCount()); + } + + PrimitiveEngine engineInput = engine.getInput(); + if (engineInput != null) { + List inputList = new ArrayList<>(); + if (engineInput instanceof RouteGen4Engine) { + inputList.add(this.formatRouteEngine((RouteGen4Engine) engineInput)); + } else if (engineInput instanceof OrderedAggregateGen4Engine) { + inputList.add(this.formatOrderedAggregateGen4Engine((OrderedAggregateGen4Engine) engineInput)); + } else if (engineInput instanceof ScalarAggregateGen4Engine) { + inputList.add(this.formatScalarAggregateGen4Engine((ScalarAggregateGen4Engine) engineInput)); + } else if (engineInput instanceof MemorySortGen4Engine) { + inputList.add(this.formatMemorySortGen4Engine((MemorySortGen4Engine) engineInput)); + } else if (engineInput instanceof LimitGen4Engine) { + inputList.add(this.formatLimitEngine((LimitGen4Engine) engineInput)); + } else if (engineInput instanceof JoinGen4Engine) { + inputList.add(this.formatJoinGen4Engine((JoinGen4Engine) engineInput)); + } else if (engineInput instanceof SimpleProjectionGen4Engine) { + inputList.add(this.formatSimpleProjection((SimpleProjectionGen4Engine) engineInput)); + } else if (engineInput instanceof FilterGen4Engine) { + inputList.add(this.formatFilterEngine((FilterGen4Engine) engineInput)); + } + instructions.setInputs(inputList); + } + return instructions; + } + + private String buildGroupByParamsString(List groupByKeys) { + if (CollectionUtils.isEmpty(groupByKeys)) { + return null; + } + List results = new ArrayList<>(); + for (GroupByParams gbp : groupByKeys) { + String result; + if (gbp.getWeightStringCol() == -1 || gbp.getKeyCol().equals(gbp.getWeightStringCol())) { + result = String.valueOf(gbp.getKeyCol()); + } else { + result = String.format("(%d|%d)", gbp.getKeyCol(), gbp.getWeightStringCol()); + } + results.add(result); + } + return String.join(", ", results); + } + private Instructions formatMemorySortEngine(MemorySortEngine engine) { Instructions instructions = new Instructions(); instructions.setOperatorType("Sort"); @@ -357,6 +806,42 @@ private Instructions formatMemorySortEngine(MemorySortEngine engine) { return instructions; } + private Instructions formatMemorySortGen4Engine(MemorySortGen4Engine engine) { + Instructions instructions = new Instructions(); + instructions.setOperatorType("Sort"); + instructions.setVariant("Memory"); + instructions.setOrderBy(buildOrderbyParamString(engine.getOrderByParams())); + + if (engine.getTruncateColumnCount() > 0) { + instructions.setResultColumns(engine.getTruncateColumnCount()); + } + + PrimitiveEngine engineInput = engine.getInput(); + if (engineInput != null) { + List inputList = new ArrayList<>(); + if (engineInput instanceof RouteGen4Engine) { + inputList.add(this.formatRouteEngine((RouteGen4Engine) engineInput)); + } else if (engineInput instanceof OrderedAggregateGen4Engine) { + inputList.add(this.formatOrderedAggregateGen4Engine((OrderedAggregateGen4Engine) engineInput)); + } else if (engineInput instanceof ScalarAggregateGen4Engine) { + inputList.add(this.formatScalarAggregateGen4Engine((ScalarAggregateGen4Engine) engineInput)); + } else if (engineInput instanceof MemorySortGen4Engine) { + inputList.add(this.formatMemorySortGen4Engine((MemorySortGen4Engine) engineInput)); + } else if (engineInput instanceof LimitGen4Engine) { + inputList.add(this.formatLimitEngine((LimitGen4Engine) engineInput)); + } else if (engineInput instanceof JoinGen4Engine) { + inputList.add(this.formatJoinGen4Engine((JoinGen4Engine) engineInput)); + } else if (engineInput instanceof SimpleProjectionGen4Engine) { + inputList.add(this.formatSimpleProjection((SimpleProjectionGen4Engine) engineInput)); + } else if (engineInput instanceof FilterGen4Engine) { + inputList.add(this.formatFilterEngine((FilterGen4Engine) engineInput)); + } + + instructions.setInputs(inputList); + } + return instructions; + } + private Instructions formatLimitEngine(LimitEngine engine) { Instructions instructions = new Instructions(); instructions.setOperatorType("Limit"); @@ -434,6 +919,97 @@ private Instructions formatJoinEngine(JoinEngine engine) { return instructions; } + private Instructions formatSimpleProjection(SimpleProjectionGen4Engine engine) { + Instructions instructions = new Instructions(); + instructions.setOperatorType("SimpleProjection"); + instructions.setColumns(engine.getCols()); + List inputs = engine.inputs(); + List inputList = new ArrayList<>(); + for (PrimitiveEngine engineInput : inputs) { + if (engineInput instanceof RouteGen4Engine) { + inputList.add(this.formatRouteEngine((RouteGen4Engine) engineInput)); + } else if (engineInput instanceof OrderedAggregateGen4Engine) { + inputList.add(this.formatOrderedAggregateGen4Engine((OrderedAggregateGen4Engine) engineInput)); + } else if (engineInput instanceof ScalarAggregateGen4Engine) { + inputList.add(this.formatScalarAggregateGen4Engine((ScalarAggregateGen4Engine) engineInput)); + } else if (engineInput instanceof MemorySortGen4Engine) { + inputList.add(this.formatMemorySortGen4Engine((MemorySortGen4Engine) engineInput)); + } else if (engineInput instanceof LimitGen4Engine) { + inputList.add(this.formatLimitEngine((LimitGen4Engine) engineInput)); + } else if (engineInput instanceof JoinGen4Engine) { + inputList.add(this.formatJoinGen4Engine((JoinGen4Engine) engineInput)); + } else if (engineInput instanceof SimpleProjectionGen4Engine) { + inputList.add(this.formatSimpleProjection((SimpleProjectionGen4Engine) engineInput)); + } else if (engineInput instanceof FilterGen4Engine) { + inputList.add(this.formatFilterEngine((FilterGen4Engine) engineInput)); + } + } + instructions.setInputs(inputList); + return instructions; + } + + private Instructions formatFilterEngine(FilterGen4Engine engine) { + Instructions instructions = new Instructions(); + instructions.setOperatorType("Filter"); + instructions.setPredicate(engine.getAstPredicate().toString()); + + List inputs = engine.inputs(); + List inputList = new ArrayList<>(); + for (PrimitiveEngine engineInput : inputs) { + if (engineInput instanceof RouteGen4Engine) { + inputList.add(this.formatRouteEngine((RouteGen4Engine) engineInput)); + } else if (engineInput instanceof OrderedAggregateGen4Engine) { + inputList.add(this.formatOrderedAggregateGen4Engine((OrderedAggregateGen4Engine) engineInput)); + } else if (engineInput instanceof ScalarAggregateGen4Engine) { + inputList.add(this.formatScalarAggregateGen4Engine((ScalarAggregateGen4Engine) engineInput)); + } else if (engineInput instanceof MemorySortGen4Engine) { + inputList.add(this.formatMemorySortGen4Engine((MemorySortGen4Engine) engineInput)); + } else if (engineInput instanceof LimitGen4Engine) { + inputList.add(this.formatLimitEngine((LimitGen4Engine) engineInput)); + } else if (engineInput instanceof JoinGen4Engine) { + inputList.add(this.formatJoinGen4Engine((JoinGen4Engine) engineInput)); + } else if (engineInput instanceof SimpleProjectionGen4Engine) { + inputList.add(this.formatSimpleProjection((SimpleProjectionGen4Engine) engineInput)); + } else if (engineInput instanceof FilterGen4Engine) { + inputList.add(this.formatFilterEngine((FilterGen4Engine) engineInput)); + } + } + instructions.setInputs(inputList); + return instructions; + } + + private Instructions formatJoinGen4Engine(JoinGen4Engine engine) { + Instructions instructions = new Instructions(); + instructions.setOperatorType("Join"); + instructions.setVariant(engine.getOpcode().name().replace("Normal", "")); + instructions.setJoinColumnIndexes(engine.getCols()); + instructions.setTableName(engine.getTableName()); + + List inputs = engine.inputs(); + List inputList = new ArrayList<>(); + for (PrimitiveEngine engineInput : inputs) { + if (engineInput instanceof RouteGen4Engine) { + inputList.add(this.formatRouteEngine((RouteGen4Engine) engineInput)); + } else if (engineInput instanceof OrderedAggregateGen4Engine) { + inputList.add(this.formatOrderedAggregateGen4Engine((OrderedAggregateGen4Engine) engineInput)); + } else if (engineInput instanceof ScalarAggregateGen4Engine) { + inputList.add(this.formatScalarAggregateGen4Engine((ScalarAggregateGen4Engine) engineInput)); + } else if (engineInput instanceof MemorySortGen4Engine) { + inputList.add(this.formatMemorySortGen4Engine((MemorySortGen4Engine) engineInput)); + } else if (engineInput instanceof LimitGen4Engine) { + inputList.add(this.formatLimitEngine((LimitGen4Engine) engineInput)); + } else if (engineInput instanceof JoinGen4Engine) { + inputList.add(this.formatJoinGen4Engine((JoinGen4Engine) engineInput)); + } else if (engineInput instanceof SimpleProjectionGen4Engine) { + inputList.add(this.formatSimpleProjection((SimpleProjectionGen4Engine) engineInput)); + } else if (engineInput instanceof FilterGen4Engine) { + inputList.add(this.formatFilterEngine((FilterGen4Engine) engineInput)); + } + } + instructions.setInputs(inputList); + return instructions; + } + private Instructions formatSubqueryEngine(SubQueryEngine engine) { Instructions instructions = new Instructions(); instructions.setOperatorType("Subquery"); @@ -544,6 +1120,34 @@ private Instructions formatConcatenateEngine(ConcatenateEngine engine) { return instructions; } + private Instructions formatConcatenateEngine(ConcatenateGen4Engine engine) { + Instructions instructions = new Instructions(); + instructions.setOperatorType("Concatenate"); + + List inputList = new ArrayList<>(); + for (PrimitiveEngine engineInput : engine.getSourceList()) { + if (engineInput instanceof RouteGen4Engine) { + inputList.add(this.formatRouteEngine((RouteGen4Engine) engineInput)); + } else if (engineInput instanceof OrderedAggregateGen4Engine) { + inputList.add(this.formatOrderedAggregateGen4Engine((OrderedAggregateGen4Engine) engineInput)); + } else if (engineInput instanceof ScalarAggregateGen4Engine) { + inputList.add(this.formatScalarAggregateGen4Engine((ScalarAggregateGen4Engine) engineInput)); + } else if (engineInput instanceof MemorySortGen4Engine) { + inputList.add(this.formatMemorySortGen4Engine((MemorySortGen4Engine) engineInput)); + } else if (engineInput instanceof LimitGen4Engine) { + inputList.add(this.formatLimitEngine((LimitGen4Engine) engineInput)); + } else if (engineInput instanceof JoinGen4Engine) { + inputList.add(this.formatJoinGen4Engine((JoinGen4Engine) engineInput)); + } else if (engineInput instanceof SimpleProjectionGen4Engine) { + inputList.add(this.formatSimpleProjection((SimpleProjectionGen4Engine) engineInput)); + } else if (engineInput instanceof FilterGen4Engine) { + inputList.add(this.formatFilterEngine((FilterGen4Engine) engineInput)); + } + } + instructions.setInputs(inputList); + return instructions; + } + private Instructions formatUpdateEngine(UpdateEngine engine) { VKeyspace vKeyspace = engine.getKeyspace(); Keyspace keyspace = new Keyspace(); @@ -586,7 +1190,7 @@ private Instructions formatDeleteEngine(DeleteEngine engine) { return instructions; } - private String formatEvalResult(EvalEngine.EvalResult evalResult) { + private String formatEvalResult(EvalResult evalResult) { String str = ""; Query.Type type = evalResult.getType(); switch (type) { @@ -710,6 +1314,8 @@ private static class TestCase { private String output; + private String output2ndPlanner; + private String comments; } @@ -724,7 +1330,8 @@ private static class TestPlan { @JsonProperty(value = "Instructions", index = 3) private Instructions instructions; - private String errorMessage; + @JsonProperty(value = "errorMessage", index = 4) + private String errorMessage = null; @Override public String toString() { @@ -800,21 +1407,26 @@ private static class Instructions { @JsonProperty(value = "Expressions", index = 22) private List expressions; - @JsonProperty(value = "Inputs", index = 23) + @JsonProperty(value = "ResultColumns", index = 23) + private Integer resultColumns; + + @JsonProperty(value = "Inputs", index = 24) private List inputs; - @JsonProperty(value = "TargetDestination", index = 24) + @JsonProperty(value = "TargetDestination", index = 25) private String targetDestination; - @JsonProperty(value = "IsDML", index = 25) + @JsonProperty(value = "IsDML", index = 26) private boolean isDML; - @JsonProperty(value = "SingleShardOnly", index = 26) + @JsonProperty(value = "SingleShardOnly", index = 27) private boolean singleShardOnly; - @JsonProperty(value = "ShardNameNeeded", index = 27) + @JsonProperty(value = "ShardNameNeeded", index = 28) private boolean shardNameNeeded; + @JsonProperty(value = "Predicate", index = 29) + private String predicate; @Override public boolean equals(Object o) { @@ -849,13 +1461,14 @@ public boolean equals(Object o) { Objects.equal(targetDestination, that.targetDestination) && Objects.equal(isDML, that.isDML) && Objects.equal(singleShardOnly, that.singleShardOnly) && - Objects.equal(shardNameNeeded, that.shardNameNeeded); + Objects.equal(shardNameNeeded, that.shardNameNeeded) && + Objects.equal(predicate, that.predicate); } @Override public int hashCode() { return Objects.hashCode(operatorType, variant, aggregates, distinct, groupBy, orderBy, count, keyspace, targetTabletType, multiShardAutocommit, fieldQuery, query, table, valueList, - joinColumnIndexes, tableName, columns, strColumns, sysTableKeyspaceExpr, expressions, inputs, targetDestination, isDML, singleShardOnly, shardNameNeeded); + joinColumnIndexes, tableName, columns, strColumns, sysTableKeyspaceExpr, expressions, inputs, targetDestination, isDML, singleShardOnly, shardNameNeeded, predicate); } @Override diff --git a/src/test/java/com/jd/jdbc/planbuilder/gen4/AnalyzerTest.java b/src/test/java/com/jd/jdbc/planbuilder/gen4/AnalyzerTest.java new file mode 100644 index 0000000..f6fbba2 --- /dev/null +++ b/src/test/java/com/jd/jdbc/planbuilder/gen4/AnalyzerTest.java @@ -0,0 +1,730 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4; + +import com.jd.BaseTest; +import com.jd.jdbc.planbuilder.semantics.Analyzer; +import com.jd.jdbc.planbuilder.semantics.FakeSI; +import com.jd.jdbc.planbuilder.semantics.SemTable; +import com.jd.jdbc.planbuilder.semantics.TableSet; +import com.jd.jdbc.sqlparser.SQLUtils; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLStatement; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectGroupByClause; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectQuery; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectStatement; +import com.jd.jdbc.sqlparser.ast.statement.SQLTableSource; +import com.jd.jdbc.sqlparser.ast.statement.SQLUnionQuery; +import com.jd.jdbc.sqlparser.dialect.mysql.ast.statement.MySqlSelectQueryBlock; +import com.jd.jdbc.sqlparser.utils.StringUtils; +import io.vitess.proto.Query; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import lombok.AllArgsConstructor; +import lombok.Data; +import org.junit.Assert; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; +import org.junit.Ignore; +import org.junit.Test; +import vschema.Vschema; + +public class AnalyzerTest extends BaseTest { + + private TableSet none = TableSet.emptyTableSet(); + + private TableSet t0 = new TableSet(0, null); + + private TableSet t1 = TableSet.singleTableSet(0); + + private TableSet t2 = TableSet.singleTableSet(1); + + private TableSet t3 = TableSet.singleTableSet(2); + + private TableSet t4 = TableSet.singleTableSet(3); + + private TableSet t5 = TableSet.singleTableSet(4); + + private Map ks1; + + private SQLExpr extract(SQLSelectStatement in, int idx) { + return in.getSelect().getFirstQueryBlock().getSelectList().get(idx).getExpr(); + } + + private SQLExpr extract(MySqlSelectQueryBlock in, int idx) { + return in.getSelectList().get(idx).getExpr(); + } + + @Test + public void testBindingSingleTablePositive() throws SQLException { + String[] queries = { + "select col from tabl", + "select uid from t2", + "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", + "select col+col from tabl", + "select max(col1+col2) from d.tabl", + "select max(id) from t1", + }; + for (String query : queries) { + ParseAndAnalyze stmAndsem = parseAndAnalyze(query, "d"); + SemTable semTable = stmAndsem.getSemTable(); + SQLSelectStatement sel = (SQLSelectStatement) stmAndsem.getStm(); + SQLTableSource t1 = sel.getSelect().getFirstQueryBlock().getFrom(); + TableSet ts = semTable.tableSetFor(t1); + assertEquals(printFail(query + ": get singleTableSet is [FAIL]"), TableSet.singleTableSet(0), ts); + TableSet recursiveDeps = semTable.recursiveDeps(extract(sel, 0)); + assertEquals(printFail(query + ": get recursiveDeps is [FAIL]"), this.t1, recursiveDeps); + assertEquals(printFail(query + ": get direct is [FAIL]"), this.t1, semTable.directDeps(extract(sel, 0))); + assertEquals(printFail(query + ": numberOfTables is [FAIL]"), 1, recursiveDeps.numberOfTables()); + printOk("TestBindingSingleTablePositive is [OK],current sql = " + query); + } + } + + @Test + public void testBindingSingleAliasedTablePositive() throws SQLException { + String[] queries = { + "select col from tabl as X", + "select tabl.col from d.X as tabl", + "select col from d.X as tabl", + "select tabl.col from X as tabl", + "select col+col from tabl as X", + "select max(tabl.col1 + tabl.col2) from d.X as tabl", + "select max(t.id) from t1 as t", + }; + for (String query : queries) { + ParseAndAnalyze stmAndsem = parseAndAnalyze(query, "d"); + SemTable semTable = stmAndsem.getSemTable(); + SQLSelectStatement sel = (SQLSelectStatement) stmAndsem.getStm(); + SQLTableSource t1 = sel.getSelect().getFirstQueryBlock().getFrom(); + TableSet ts = semTable.tableSetFor(t1); + assertEquals(printFail(query + ": get singleTableSet is [FAIL]"), TableSet.singleTableSet(0), ts); + TableSet recursiveDeps = semTable.recursiveDeps(extract(sel, 0)); + assertEquals(printFail(query + ": get recursiveDeps is [FAIL]"), this.t1, recursiveDeps); + assertEquals(printFail(query + ":number of tables is wrong"), 1, recursiveDeps.numberOfTables()); + printOk("testBindingSingleAliasedTablePositive is [OK],current sql = " + query); + } + } + + @Test + public void testBindingSingleTableNegative() { + String[] queries = { + "select foo.col from tabl", + "select ks.tabl.col from tabl", + "select ks.tabl.col from d.tabl", + "select d.tabl.col from ks.tabl", + "select foo.col from d.tabl", + "select tabl.col from d.tabl as t", + }; + for (String query : queries) { + SQLStatement parse = SQLUtils.parseSingleMysqlStatement(query); + try { + Analyzer.analyze((SQLSelectStatement) parse, "d", new FakeSI(new HashMap<>(), null)); + fail("expected an error"); + } catch (SQLException e) { + String message = e.getMessage(); + if (!(message.contains("symbol") && message.contains("not found"))) { + fail("unexpected error msg " + message); + } + printOk("testBindingSingleTableNegative is [OK],current sql = " + query); + } + } + } + + @Test + public void testBindingSingleAliasedTableNegative() { + String[] queries = { + "select tabl.col from d.tabl as t", + "select foo.col from tabl", + "select ks.tabl.col from tabl", + "select ks.tabl.col from d.tabl", + "select d.tabl.col from ks.tabl", + "select foo.col from d.tabl", + "select tabl.col from tabl as X", + "select d.X.col from d.X as tabl", + "select d.tabl.col from X as tabl", + "select d.tabl.col from ks.X as tabl", + "select d.tabl.col from d.X as tabl", + }; + + for (String query : queries) { + SQLStatement parse = SQLUtils.parseSingleMysqlStatement(query); + try { + Map map = new HashMap<>(); + map.put("t", Vschema.Table.newBuilder().build()); + Analyzer.analyze((SQLSelectStatement) parse, "d", new FakeSI(map, null)); + fail("expected an error"); + } catch (SQLException e) { + String message = e.getMessage(); + if (!(message.contains("symbol") && message.contains("not found"))) { + fail("unexpected error msg " + message); + } + printOk(query + " testBindingSingleAliasedTableNegative" + " is [OK]"); + } + } + } + + @Test + public void testBindingMultiTablePositive() throws SQLException { + List testCases = new ArrayList<>(); + testCases.add(new TestCase("select t.col from t,s", t1, 1)); + testCases.add(new TestCase("select s.col from t join s", t2, 1)); + testCases.add(new TestCase("select max(t.col+s.col) from t,s", TableSet.mergeTableSets(t1, t2), 2)); + testCases.add(new TestCase("select max(t.col+s.col) from t join s", TableSet.mergeTableSets(t1, t2), 2)); + testCases.add(new TestCase("select case t.col when s.col then r.col else u.col end from t,s,r,w,u", TableSet.mergeTableSets(t1, t2, t3, t5), 4)); + testCases.add(new TestCase("select u1.a + u2.a from u1,u2", TableSet.mergeTableSets(t1, t2), 2)); + + for (TestCase testCase : testCases) { + String query = testCase.getQuery(); + ParseAndAnalyze stmAndsem = parseAndAnalyze(query, "user"); + SemTable semTable = stmAndsem.getSemTable(); + SQLSelectStatement sel = (SQLSelectStatement) stmAndsem.getStm(); + TableSet d = semTable.recursiveDeps(extract(sel, 0)); + assertEquals(query, testCase.getDeps(), d); + assertEquals(query, testCase.getNumberOfTables(), d.numberOfTables()); + printOk("testBindingMultiTablePositive is [OK],current sql = " + query); + } + } + + @Test + public void testBindingMultiAliasedTablePositive() throws SQLException { + List testCases = new ArrayList<>(); + testCases.add(new TestCase("select X.col from t as X,s as S", t1, 1)); + testCases.add(new TestCase("select X.col + S.col from t as X, s as S", TableSet.mergeTableSets(t1, t2), 2)); + testCases.add(new TestCase("select max(X.col+S.col) from t as X,s as S", TableSet.mergeTableSets(t1, t2), 2)); + testCases.add(new TestCase("select max(X.col+s.col) from t as X,s", TableSet.mergeTableSets(t1, t2), 2)); + + for (TestCase testCase : testCases) { + String query = testCase.getQuery(); + ParseAndAnalyze stmAndsem = parseAndAnalyze(query, "user"); + SemTable semTable = stmAndsem.getSemTable(); + SQLSelectStatement sel = (SQLSelectStatement) stmAndsem.getStm(); + TableSet d = semTable.recursiveDeps(extract(sel, 0)); + assertEquals(query, testCase.getDeps(), d); + assertEquals(query, testCase.getNumberOfTables(), d.numberOfTables()); + printOk("testBindingMultiAliasedTablePositive is [OK],current sql = " + query); + } + } + + @Test + public void testBindingMultiTableNegative() { + } + + @Test + public void testBindingMultiAliasedTableNegative() { + + } + + @Test + public void testNotUniqueTableName() { + String[] queries = { + "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 (String query : queries) { + SQLStatement stmt = SQLUtils.parseSingleMysqlStatement(query); + try { + Analyzer.analyze((SQLSelectStatement) stmt, "test", new FakeSI(new HashMap<>(), null)); + fail("expected an error"); + } catch (SQLException e) { + String message = e.getMessage(); + if (!(message.contains("Not unique table/alias"))) { + fail("unexpected error msg " + message); + } + printOk("testNotUniqueTableName is [OK],current sql = " + query); + } + } + } + + @Test + public void testMissingTable() { + String[] queries = { + "select t.col from a", + }; + for (String query : queries) { + SQLStatement stmt = SQLUtils.parseSingleMysqlStatement(query); + try { + Analyzer.analyze((SQLSelectStatement) stmt, "", new FakeSI(new HashMap<>(), null)); + fail("expected an error"); + } catch (SQLException e) { + String message = e.getMessage(); + if (!(message.contains("symbol t.col not found"))) { + fail("unexpected error msg " + message); + } + printOk("testMissingTable is [OK],current sql = " + query); + } + } + } + + @Test + @Ignore + public void testScoping() { + List testCases = new ArrayList<>(); + testCases.add(new TestCase("select 1 from u1, u2 left join u3 on u1.a = u2.a", "symbol u1.a not found")); + + for (TestCase testCase : testCases) { + SQLStatement stmt = SQLUtils.parseSingleMysqlStatement(testCase.query); + try { + Map map = new HashMap<>(); + map.put("t", Vschema.Table.newBuilder().build()); + Analyzer.analyze((SQLSelectStatement) stmt, "user", new FakeSI(map, null)); + Assert.fail("expected an error"); + } catch (SQLException e) { + String message = e.getMessage(); + if (StringUtils.isEmpty(testCase.errorMessage)) { + Assert.fail(); + } else { + Assert.assertEquals(testCase.errorMessage, message); + } + } + } + } + + @Test + @Ignore + public void TestScopeForSubqueries() throws SQLException { + TestCase[] tcases = { + new TestCase("select t.col1, (select t.col2 from z as t) from x as t", t2), + new TestCase("select t.col1, (select t.col2 from z) from x as t", t1), + new TestCase("select t.col1, (select (select z.col2 from y) from z) from x as t", t2), + new TestCase("select t.col1, (select (select y.col2 from y) from z) from x as t", none), + new TestCase("select t.col1, (select (select (select (select w.col2 from w) from x) from y) from z) from x as t", none), + new TestCase("select t.col1, (select id from t) from x as t", t2) + }; + for (TestCase tc : tcases) { + ParseAndAnalyze stmAndsem = parseAndAnalyze(tc.query, "d"); + SemTable semTable = stmAndsem.getSemTable(); + SQLSelectStatement sel = (SQLSelectStatement) stmAndsem.getStm(); + // extract the first expression from the subquery (which should be the second expression in the outer query) + sel.getSelect().getFirstQueryBlock().getSelectList().get(1).getExpr(); + } + } + + @Test + public void testSubqueriesMappingWhereClause() { + + } + + @Test + public void testSubqueriesMappingSelectExprs() { + + } + + @Test + public void testSubqueryOrderByBinding() { + + } + + @Test + public void testOrderByBindingTable() throws SQLException { + List testCases = new ArrayList<>(); + testCases.add(new TestCase("select col from tabl order by col", t1)); + testCases.add(new TestCase("select tabl.col from d.tabl order by col", t1)); + testCases.add(new TestCase("select d.tabl.col from d.tabl order by col", t1)); + testCases.add(new TestCase("select col from tabl order by tabl.col", t1)); + testCases.add(new TestCase("select col from tabl order by d.tabl.col", t1)); + testCases.add(new TestCase("select col from tabl order by 1", t1)); + testCases.add(new TestCase("select col as c from tabl order by c", t1)); + testCases.add(new TestCase("select 1 as c from tabl order by c", none)); + testCases.add(new TestCase("select name, name from t1, t2 order by name", t2)); + testCases.add(new TestCase("(select id from t1) union (select uid from t2) order by id", TableSet.mergeTableSets(t1, t2))); + testCases.add(new TestCase("select id from t1 union (select uid from t2) order by 1", TableSet.mergeTableSets(t1, t2))); + testCases.add(new TestCase("select id from t1 union select uid from t2 union (select name from t) order by 1", TableSet.mergeTableSets(t1, t2, t3))); + testCases.add(new TestCase("select a.id from t1 as a union (select uid from t2) order by 1", TableSet.mergeTableSets(t1, t2))); + testCases.add(new TestCase("select b.id as a from t1 as b union (select uid as c from t2) order by 1", TableSet.mergeTableSets(t1, t2))); + testCases.add(new TestCase("select a.id from t1 as a union (select uid from t2, t union (select name from t) order by 1) order by 1", TableSet.mergeTableSets(t1, t2, t4))); + testCases.add(new TestCase("select a.id from t1 as a union (select uid from t2, t union (select name from t) order by 1) order by id", TableSet.mergeTableSets(t1, t2, t4))); + for (TestCase testCase : testCases) { + String query = testCase.getQuery(); + ParseAndAnalyze stmAndsem = parseAndAnalyze(query, "d"); + SemTable semTable = stmAndsem.getSemTable(); + SQLSelectStatement sel = (SQLSelectStatement) stmAndsem.getStm(); + SQLSelectQuery sqlSelectQuery = sel.getSelect().getQuery(); + SQLExpr order = null; + if (sqlSelectQuery instanceof MySqlSelectQueryBlock) { + order = ((MySqlSelectQueryBlock) sqlSelectQuery).getOrderBy().getItems().get(0).getExpr(); + } else if (sqlSelectQuery instanceof SQLUnionQuery) { + order = ((SQLUnionQuery) sqlSelectQuery).getOrderBy().getItems().get(0).getExpr(); + } else { + Assert.fail(); + } + TableSet d = semTable.recursiveDeps(order); + Assert.assertEquals(query, testCase.getDeps(), d); + printOk("testOrderByBindingTable is [OK],current sql = " + query); + } + } + + @Test + public void testGroupByBinding() throws SQLException { + List testCases = new ArrayList<>(); + testCases.add(new TestCase("select col from tabl group by col", t1)); + testCases.add(new TestCase("select col from tabl group by tabl.col", t1)); + testCases.add(new TestCase("select col from tabl group by d.tabl.col", t1)); + testCases.add(new TestCase("select tabl.col as x from tabl group by x", t1)); + testCases.add(new TestCase("select tabl.col as x from tabl group by col", t1)); + testCases.add(new TestCase("select d.tabl.col as x from tabl group by x", t1)); + testCases.add(new TestCase("select d.tabl.col as x from tabl group by col", t1)); + testCases.add(new TestCase("select col from tabl group by 1", t1)); + testCases.add(new TestCase("select col as c from tabl group by c", t1)); + testCases.add(new TestCase("select 1 as c from tabl group by c", none)); + testCases.add(new TestCase("select t1.id from t1, t2 group by id", t1)); + testCases.add(new TestCase("select id from t, t1 group by id", t2)); + testCases.add(new TestCase("select id from t, t1 group by id", t2)); + testCases.add(new TestCase("select a.id from t as a, t1 group by id", t1)); + testCases.add(new TestCase("select a.id from t, t1 as a group by id", t2)); + for (TestCase testCase : testCases) { + String query = testCase.getQuery(); + ParseAndAnalyze stmAndsem = parseAndAnalyze(query, "d"); + SemTable semTable = stmAndsem.getSemTable(); + SQLSelectStatement sel = (SQLSelectStatement) stmAndsem.getStm(); + SQLSelectGroupByClause grp = sel.getSelect().getFirstQueryBlock().getGroupBy(); + TableSet d = semTable.recursiveDeps(grp.getItems().get(0)); + assertEquals(query, testCase.getDeps(), d); + printOk("testGroupByBinding is [OK],current sql = " + query); + } + } + + @Test + public void testHavingBinding() throws SQLException { + List testCases = new ArrayList<>(); + testCases.add(new TestCase("select col from tabl having col = 1", t1)); + testCases.add(new TestCase("select col from tabl having tabl.col = 1", t1)); + testCases.add(new TestCase("select col from tabl having d.tabl.col = 1", t1)); + testCases.add(new TestCase("select tabl.col as x from tabl having x = 1", t1)); + testCases.add(new TestCase("select tabl.col as x from tabl having col", t1)); + testCases.add(new TestCase("select col from tabl having 1 = 1", none)); + testCases.add(new TestCase("select col as c from tabl having c = 1", t1)); + testCases.add(new TestCase("select 1 as c from tabl having c = 1", none)); + testCases.add(new TestCase("select t1.id from t1, t2 having id = 1", t1)); + testCases.add(new TestCase("select t.id from t, t1 having id = 1", t1)); + testCases.add(new TestCase("select t.id, count(*) as a from t, t1 group by t.id having a = 1", TableSet.mergeTableSets(t1, t2))); + testCases.add(new TestCase("select u2.a, u1.a from u1, u2 having u2.a = 2", t2)); + for (TestCase testCase : testCases) { + String query = testCase.getQuery(); + ParseAndAnalyze stmAndsem = parseAndAnalyze(query, "d"); + SemTable semTable = stmAndsem.getSemTable(); + SQLSelectStatement sel = (SQLSelectStatement) stmAndsem.getStm(); + SQLExpr having = sel.getSelect().getFirstQueryBlock().getGroupBy().getHaving(); + TableSet d = semTable.recursiveDeps(having); + assertEquals(query, testCase.getDeps(), d); + printOk("testHavingBinding is [OK],current sql = " + query); + } + } + + @Test + public void testGroupByHavingBinding() throws SQLException { + @AllArgsConstructor + class TmpCase { + String query; + + TableSet groupByDeps; + + TableSet havingDeps; + } + + List testCases = new ArrayList<>(); + testCases.add(new TmpCase("select col from tabl group by col having col > 10", t1, t1)); + testCases.add(new TmpCase("select col from tabl group by tabl.col having tabl.col > 10", t1, t1)); + testCases.add(new TmpCase("select col from tabl group by d.tabl.col having d.tabl.col > 10", t1, t1)); + testCases.add(new TmpCase("select tabl.col as x from tabl group by x having x = 1000", t1, t1)); + testCases.add(new TmpCase("select tabl.col as x from tabl group by col having x = 1000", t1, t1)); + testCases.add(new TmpCase("select d.tabl.col as x from tabl group by x having x = 1000", t1, t1)); + testCases.add(new TmpCase("select d.tabl.col as x from tabl group by col having x = 1000", t1, t1)); + testCases.add(new TmpCase("select col from tabl group by 1 having col < 10000", t1, t1)); + testCases.add(new TmpCase("select col as c from tabl group by c having c < 10000", t1, t1)); + testCases.add(new TmpCase("select 1 as c from tabl group by c having c < 10000", none, none)); + testCases.add(new TmpCase("select t1.id from t1, t2 group by id having id < 10000", t1, t1)); + testCases.add(new TmpCase("select id from t, t1 group by id having id < 10000", t2, t2)); + testCases.add(new TmpCase("select id from t, t1 group by id having id < 10000", t2, t2)); + testCases.add(new TmpCase("select a.id from t as a, t1 group by id having id < 10000", t1, t1)); + testCases.add(new TmpCase("select a.id from t, t1 as a group by id having id < 10000", t2, t2)); + testCases.add(new TmpCase("select col from tabl group by col having col = 1", t1, t1)); + testCases.add(new TmpCase("select col from tabl group by tabl.col having tabl.col = 1", t1, t1)); + testCases.add(new TmpCase("select col from tabl group by d.tabl.col having d.tabl.col = 1", t1, t1)); + testCases.add(new TmpCase("select tabl.col as x from tabl group by x having x = 1", t1, t1)); + testCases.add(new TmpCase("select tabl.col as x from tabl group by col having x = 1", t1, t1)); + testCases.add(new TmpCase("select tabl.col as x from tabl group by col having col", t1, t1)); + testCases.add(new TmpCase("select tabl.col as x from tabl group by x having col", t1, t1)); + testCases.add(new TmpCase("select col from tabl group by col having 1 = 1", t1, none)); + testCases.add(new TmpCase("select col as c from tabl group by col having c = 1", t1, t1)); + testCases.add(new TmpCase("select 1 as c from tabl group by c having c = 1", none, none)); + testCases.add(new TmpCase("select t1.id from t1, t2 group by id having id = 1", t1, t1)); + testCases.add(new TmpCase("select t.id from t, t1 group by id having id = 1", t1, t1)); + + for (TmpCase testCase : testCases) { + String query = testCase.query; + ParseAndAnalyze stmAndsem = parseAndAnalyze(query, "d"); + SemTable semTable = stmAndsem.getSemTable(); + SQLSelectStatement sel = (SQLSelectStatement) stmAndsem.getStm(); + SQLSelectGroupByClause groupBy = sel.getSelect().getFirstQueryBlock().getGroupBy(); + SQLExpr having = sel.getSelect().getFirstQueryBlock().getGroupBy().getHaving(); + TableSet groupByTableSet = semTable.recursiveDeps(groupBy.getItems().get(0)); + assertEquals(query, testCase.groupByDeps, groupByTableSet); + TableSet havingTableSet = semTable.recursiveDeps(having); + assertEquals(query, testCase.havingDeps, havingTableSet); + printOk("testGroupByHavingBinding is [OK],current sql = " + query); + } + } + + @Test + public void testUnionCheckFirstAndLastSelectsDeps() throws SQLException { + String query = "select col1 from tabl1 union select col2 from tabl2"; + ParseAndAnalyze stmAndsem = parseAndAnalyze(query, ""); + SQLUnionQuery union = (SQLUnionQuery) ((SQLSelectStatement) stmAndsem.getStm()).getSelect().getQuery(); + MySqlSelectQueryBlock sel1 = (MySqlSelectQueryBlock) union.getLeft(); + MySqlSelectQueryBlock sel2 = (MySqlSelectQueryBlock) union.getRight(); + + SemTable semTable = stmAndsem.getSemTable(); + TableSet ts1 = semTable.tableSetFor(sel1.getFrom()); + TableSet ts2 = semTable.tableSetFor(sel2.getFrom()); + Assert.assertEquals(t1, ts1); + Assert.assertEquals(t2, ts2); + + TableSet d1 = semTable.recursiveDeps(extract(sel1, 0)); + TableSet d2 = semTable.recursiveDeps(extract(sel2, 0)); + Assert.assertEquals(t1, d1); + Assert.assertEquals(t2, d2); + } + + @Test + public void testUnionOrderByRewrite() throws SQLException { + String query = "select tabl1.id from tabl1 union select 1 order by 1"; + ParseAndAnalyze stmAndsem = parseAndAnalyze(query, ""); + + SQLStatement sqlStatement = SQLUtils.parseSingleMysqlStatement("select tabl1.id from tabl1 union select 1 order by id"); + Assert.assertEquals(SQLUtils.toSQLString(sqlStatement), SQLUtils.toSQLString(stmAndsem.getStm())); + } + + @Test + public void testInvalidQueries() { + TestCase[] tcases = { + new TestCase("select t1.id, t1.col1 from t1 union select t2.uid from t2", "The used SELECT statements have a different number of columns"), + new TestCase("select t1.id from t1 union select t2.uid, t2.price from t2", "The used SELECT statements have a different number of columns"), + new TestCase("select t1.id from t1 union select t2.uid, t2.price from t2", "The used SELECT statements have a different number of columns"), + new TestCase("(select 1,2 union select 3,4) union (select 5,6 union select 7)", "The used SELECT statements have a different number of columns"), + new TestCase("select id from a union select 3 order by a.id", "Table 'a' from one of the SELECTs cannot be used in global ORDER clause"), + new TestCase("select a.id, b.id from a, b union select 1, 2 order by id", "Column 'id' in field list is ambiguous"), +// new TestCase("select sql_calc_found_rows id from a union select 1 limit 109", "SQL_CALC_FOUND_ROWS not supported with union"), +// new TestCase("select * from (select sql_calc_found_rows id from a) as t", "Incorrect usage/placement of 'SQL_CALC_FOUND_ROWS'"), +// new TestCase("select (select sql_calc_found_rows id from a) as t", "Incorrect usage/placement of 'SQL_CALC_FOUND_ROWS'"), + }; + for (TestCase tcase : tcases) { + SQLStatement parse = SQLUtils.parseSingleMysqlStatement(tcase.query); + try { + Analyzer.analyze((SQLSelectStatement) parse, "dbName", fakeSchemaInfo()); + Assert.fail("expected an error"); + } catch (SQLException e) { + String message = e.getMessage(); + Assert.assertEquals(tcase.errorMessage, message); + printOk("testInvalidQueries is [ok]current sql = " + tcase.query); + } + } + } + + @Test + public void testUnionWithOrderBy() throws SQLException { + String query = "select col1 from tabl1 union (select col2 from tabl2) order by 1"; + ParseAndAnalyze stmAndsem = parseAndAnalyze(query, ""); + + SQLUnionQuery union = (SQLUnionQuery) ((SQLSelectStatement) stmAndsem.getStm()).getSelect().getQuery(); + MySqlSelectQueryBlock sel1 = (MySqlSelectQueryBlock) union.getLeft(); + MySqlSelectQueryBlock sel2 = (MySqlSelectQueryBlock) union.getRight(); + + SemTable semTable = stmAndsem.getSemTable(); + TableSet ts1 = semTable.tableSetFor(sel1.getFrom()); + TableSet ts2 = semTable.tableSetFor(sel2.getFrom()); + Assert.assertEquals(t1, ts1); + Assert.assertEquals(t2, ts2); + + TableSet d1 = semTable.recursiveDeps(extract(sel1, 0)); + TableSet d2 = semTable.recursiveDeps(extract(sel2, 0)); + Assert.assertEquals(t1, d1); + Assert.assertEquals(t2, d2); + } + + @Test + public void testScopingWDerivedTables() { + TestCase[] tcases = { + new TestCase("select id from (select x as id from user) as t", null, t1, t2), + new TestCase("select id from (select foo as id from user) as t", null, t1, t2), + new TestCase("select id from (select foo as id from (select x as foo from user) as c) as t", null, t1, t3), + new TestCase("select t.id from (select foo as id from user) as t", null, t1, t2), + new TestCase("select t.id2 from (select foo as id from user) as t", "symbol t.id2 not found", null, null), + new TestCase("select id from (select 42 as id) as t", null, t0, t2), + new TestCase("select t.id from (select 42 as id) as t", null, t0, t2), + new TestCase("select ks.t.id from (select 42 as id) as t", "symbol ks.t.id not found", null, null), + new TestCase("select * from (select id, id from user) as t", "Duplicate column name 'id'", null, null), +// new TestCase("select t.baz = 1 from (select id as baz from user) as t", null, t2, t1), +// new TestCase("select t.id from (select * from user, music) as t", null, t3, TableSet.mergeTableSets(t1, t2)), +// new TestCase("select t.id from (select * from user, music) as t order by t.id", null, t3, TableSet.mergeTableSets(t1, t2)), +// new TestCase("select t.id from (select * from user) as t join user as u on t.id = u.id", null, t2, t1), +// new TestCase("select t.col1 from t3 ua join (select t1.id, t1.col1 from t1 join t2) as t", null, t4, t2), + new TestCase("select uu.test from (select id from t1) uu", "symbol uu.test not found", null, null), + new TestCase("select uu.id from (select id as col from t1) uu", "symbol uu.id not found", null, null), +// new TestCase("select uu.id from (select id from t1) as uu where exists (select * from t2 as uu where uu.id = uu.uid)", null, t2, t1), + new TestCase("select 1 from user uu where exists (select 1 from user where exists (select 1 from (select 1 from t1) uu where uu.user_id = uu.id))", null, t0, t0) + }; + Map tableMap = new HashMap<>(); + tableMap.put("t", null); + Map vindexTables = new HashMap<>(); + for (TestCase query : tcases) { + SQLSelectStatement sel = (SQLSelectStatement) SQLUtils.parseSingleMysqlStatement(query.query); + try { + SemTable st = Analyzer.analyze(sel, "user", new FakeSI(tableMap, vindexTables)); + st.recursiveDeps(extract(sel, 0)); + assertEquals("testScopingWDerivedTables is [FAIL]current sql = " + query.query, query.recursiveExpectation, st.recursiveDeps(extract(sel, 0))); + assertEquals("testScopingWDerivedTables is [FAIL]current sql = " + query.query, query.expectation, st.directDeps(extract(sel, 0))); + } catch (SQLException e) { + if (query.errorMessage == null) { + System.out.println(printFail("testScopingWDerivedTables is [FAIL]current sql = " + query.query + " , error: " + e.getMessage())); + fail(); + } + Assert.assertEquals(query.query + " is [FAIL]", query.errorMessage, e.getMessage()); + } + printOk("testScopingWDerivedTables is [ok]current sql = " + query.query); + } + } + + @Test + public void testDerivedTablesOrderClause() { + + } + + @Test + public void testScopingWComplexDerivedTables() { + + } + + /* @Test + public void testScopingWVindexTables() { + + @AllArgsConstructor + class TmpCase { + String query; + + String erroMsg; + + TableSet recursiveExpectation; + + TableSet expectation; + } + + TmpCase[] tcases = { + new TmpCase("select id from user_index where id = 1", null, t1, t1), + new TmpCase("select u.id + t.id from t as t join user_index as u where u.id = 1 and u.id = t.id", null, TableSet.mergeTableSets(t1, t2), TableSet.mergeTableSets(t1, t2)), + }; + for (TmpCase query : tcases) { + SQLSelectStatement sel = (SQLSelectStatement) SQLUtils.parseSingleMysqlStatement(query.query); + Map tableMap = new HashMap<>(); + tableMap.put("t", null); + Map vindexTables = new HashMap<>(); + vindexTables.put("user_index", Vschema.ColumnVindex.newBuilder().setName("hash").build()); + try { + SemTable st = Analyzer.analyze(sel, "user", new FakeSI(tableMap, vindexTables)); + assertEquals("testScopingWVindexTables is [[FAIL]current sql = " + query.query, query.recursiveExpectation, st.recursiveDeps(extract(sel, 0))); + assertEquals("testScopingWVindexTables is [[FAIL]current sql = " + query.query, query.expectation, st.directDeps(extract(sel, 0))); + } catch (SQLException e) { + if (!query.erroMsg.equals("") && !e.getMessage().equals(query.erroMsg)) { + fail("unexpect error : current sql =" + query.query); + } + } + } + }*/ + + private ParseAndAnalyze parseAndAnalyze(String query, String dbName) throws SQLException { + SQLStatement stmt = SQLUtils.parseSingleMysqlStatement(query); + SemTable semTable = Analyzer.analyze((SQLSelectStatement) stmt, dbName, fakeSchemaInfo()); + return new ParseAndAnalyze(stmt, semTable); + } + + private FakeSI fakeSchemaInfo() { +// Vschema.ColumnVindex cols1 = Vschema.ColumnVindex.newBuilder().setName("hash").setColumn("id").build(); +// Vschema.ColumnVindex cols2 = Vschema.ColumnVindex.newBuilder().setName("hash").setColumn("uid").build(); +// Vschema.ColumnVindex cols3 = Vschema.ColumnVindex.newBuilder().setName("hash").setColumn("name").build(); + + Map tables = new HashMap<>(); +// tables.put("t", Vschema.Table.newBuilder().setColumnListAuthoritative(false).build()); +// tables.put("t1", Vschema.Table.newBuilder().setColumnListAuthoritative(true).addColumnVindexes(cols1).build()); +// tables.put("t2", Vschema.Table.newBuilder().setColumnListAuthoritative(true).addColumnVindexes(cols2).addColumnVindexes(cols3).build()); + + Vschema.Column column1 = Vschema.Column.newBuilder().setName("id").setType(Query.Type.INT64).build(); + Vschema.Column column2 = Vschema.Column.newBuilder().setName("uid").setType(Query.Type.INT64).build(); + Vschema.Column column3 = Vschema.Column.newBuilder().setName("name").setType(Query.Type.VARCHAR).build(); + tables.put("t", Vschema.Table.newBuilder().build()); + tables.put("t1", Vschema.Table.newBuilder().setColumnListAuthoritative(true).addColumns(column1).build()); + tables.put("t2", Vschema.Table.newBuilder().setColumnListAuthoritative(true).addColumns(column2).addColumns(column3).build()); + + return new FakeSI(tables, null); + } + + @Data + @AllArgsConstructor + private static class ParseAndAnalyze { + private SQLStatement stm; + + private SemTable semTable; + } + + @Data + @AllArgsConstructor + private static class TestCase { + String query; + + String errorMessage; + + TableSet recursiveExpectation; + + TableSet expectation; + + TableSet deps; + + int numberOfTables; + + TestCase(String query, TableSet deps) { + this.query = query; + this.deps = deps; + } + + TestCase(String query, TableSet deps, int numberOfTables) { + this.query = query; + this.deps = deps; + this.numberOfTables = numberOfTables; + } + + TestCase(String query, String errorMessage, TableSet recursiveExpectation, TableSet expectation) { + this.query = query; + this.errorMessage = errorMessage; + this.recursiveExpectation = recursiveExpectation; + this.expectation = expectation; + } + + public TestCase(String query, String errorMessage) { + this.query = query; + this.errorMessage = errorMessage; + } + } +} diff --git a/src/test/java/com/jd/jdbc/planbuilder/gen4/HorizonPlanningTest.java b/src/test/java/com/jd/jdbc/planbuilder/gen4/HorizonPlanningTest.java new file mode 100644 index 0000000..8fc540c --- /dev/null +++ b/src/test/java/com/jd/jdbc/planbuilder/gen4/HorizonPlanningTest.java @@ -0,0 +1,84 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4; + +import com.jd.BaseTest; +import com.jd.jdbc.planbuilder.semantics.SemTable; +import com.jd.jdbc.sqlparser.SQLUtils; +import com.jd.jdbc.sqlparser.ast.SQLStatement; +import com.jd.jdbc.sqlparser.ast.expr.SQLIdentifierExpr; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectItem; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectQuery; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectStatement; +import java.util.Arrays; +import java.util.List; +import lombok.Getter; +import lombok.ToString; +import org.junit.Assert; +import org.junit.Test; + +public class HorizonPlanningTest extends BaseTest { + + @Test + public void testCheckIfAlreadyExists() { + List tests = Arrays.asList( + new CheckIfAlreadyExistsTest("No alias, both ColName", new SQLSelectItem(new SQLIdentifierExpr("id")), getSQLSelectQuery("select id"), 0), + new CheckIfAlreadyExistsTest("Aliased expression and ColName", new SQLSelectItem(new SQLIdentifierExpr("user_id")), getSQLSelectQuery("select user_id,id"), 0), + new CheckIfAlreadyExistsTest("Non-ColName expressions", new SQLSelectItem(new SQLIdentifierExpr("test")), getSQLSelectQuery("select test"), 0), + new CheckIfAlreadyExistsTest("No alias, multiple ColName in projection", new SQLSelectItem(new SQLIdentifierExpr("id")), getSQLSelectQuery("select foo,id"), 1), + new CheckIfAlreadyExistsTest("No matching entry", new SQLSelectItem(new SQLIdentifierExpr("id")), getSQLSelectQuery("select foo,name"), -1), + new CheckIfAlreadyExistsTest("No AliasedExpr in projection", new SQLSelectItem(new SQLIdentifierExpr("id")), getSQLSelectQuery("select user,people"), -1) + ); + SemTable semTable = new SemTable(); + for (CheckIfAlreadyExistsTest tt : tests) { + int got = HorizonPlanning.checkIfAlreadyExists(tt.getExpr(), tt.getSel(), semTable); + Assert.assertEquals(tt.getWant(), got); + printOk("testCheckIfAlreadyExists [OK] , case= " + tt); + } + } + + private SQLSelectQuery getSQLSelectQuery(String sql) { + SQLStatement stmt = SQLUtils.parseSingleMysqlStatement(sql); + if (stmt instanceof SQLSelectStatement) { + return ((SQLSelectStatement) stmt).getSelect().getQuery(); + } + return null; + } + + @ToString + private static class CheckIfAlreadyExistsTest { + private final String name; + + @Getter + private final SQLSelectItem expr; + + @Getter + private final SQLSelectQuery sel; + + @Getter + private final int want; + + CheckIfAlreadyExistsTest(String name, SQLSelectItem expr, SQLSelectQuery sel, int want) { + this.name = name; + this.expr = expr; + this.sel = sel; + this.want = want; + } + } +} diff --git a/src/test/java/com/jd/jdbc/planbuilder/gen4/OperatorTest.java b/src/test/java/com/jd/jdbc/planbuilder/gen4/OperatorTest.java new file mode 100644 index 0000000..6cddac6 --- /dev/null +++ b/src/test/java/com/jd/jdbc/planbuilder/gen4/OperatorTest.java @@ -0,0 +1,227 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4; + +import com.jd.jdbc.planbuilder.gen4.operator.OperatorUtil; +import com.jd.jdbc.planbuilder.gen4.operator.logical.Concatenate; +import com.jd.jdbc.planbuilder.gen4.operator.logical.LogicalOperator; +import com.jd.jdbc.planbuilder.gen4.operator.logical.QueryGraph; +import com.jd.jdbc.planbuilder.semantics.Analyzer; +import com.jd.jdbc.planbuilder.semantics.FakeSI; +import com.jd.jdbc.planbuilder.semantics.SemTable; +import com.jd.jdbc.sqlparser.SQLUtils; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import com.jd.jdbc.sqlparser.ast.SQLStatement; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectStatement; +import io.netty.util.internal.StringUtil; +import java.io.BufferedReader; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicBoolean; +import lombok.AllArgsConstructor; +import org.junit.Assert; +import org.junit.Ignore; +import org.junit.Test; +import testsuite.TestSuite; +import vschema.Vschema; + +public class OperatorTest extends TestSuite { + + @Test + public void testOperator() throws IOException, InterruptedException { + String fileName = "src/test/resources/plan/Gen4/operator_test_data.txt"; + List testCases = readTestCase(fileName); + testOperator(fileName, testCases); + } + + @Test + @Ignore + public void testOne() throws IOException, InterruptedException { + String fileName = "src/test/resources/plan/Gen4/operator_test_data_one.txt"; + List testCases = readTestCase(fileName); + testOperator(fileName, testCases); + } + + private void testOperator(String fileName, List testCases) throws InterruptedException { + Map vindexTables = new HashMap<>(); + vindexTables.put("user_index", Vschema.ColumnVindex.newBuilder().setName("hash").build()); + FakeSI si = new FakeSI(new HashMap<>(), vindexTables); + CountDownLatch latch = new CountDownLatch(testCases.size()); + ExecutorService executorService = getThreadPool(10,10); + AtomicBoolean errorFlag = new AtomicBoolean(true); + for (TestCase tc : testCases) { + executorService.execute(() -> { + try { + SQLStatement stmt = SQLUtils.parseSingleMysqlStatement(tc.query); + SemTable semTable = Analyzer.analyze((SQLSelectStatement) stmt, "", si); + LogicalOperator optree = OperatorUtil.createLogicalOperatorFromAST(stmt, semTable); + String output = testString(optree); + if (!tc.expected.equals(output)) { + System.out.println(printFail(fileName + "/" + tc.line + " is [FAIL], \nexecput:" + tc.expected + "\noutput:" + output)); + errorFlag.set(false); + } else { + printOk(fileName + "/" + tc.line + " SQL:" + tc.query + " is [OK]"); + } + } catch (Exception e) { + errorFlag.set(false); + e.printStackTrace(); + } finally { + latch.countDown(); + } + }); + } + latch.await(); + Assert.assertTrue(printFail("testOperator is [FAIL]"), errorFlag.get()); + } + + private String testString(LogicalOperator op) { + if (op instanceof QueryGraph) { + return "QueryGraph: " + qgTestString((QueryGraph) op); + } + if (op instanceof Concatenate) { + List inners = new ArrayList<>(); + Concatenate concatenate = (Concatenate) op; + for (LogicalOperator source : concatenate.getSources()) { + inners.add(indent(testString(source))); + } + if (concatenate.getOrderBy() != null) { + inners.add(indent(SQLUtils.toMySqlString(concatenate.getOrderBy()).toLowerCase())); + } + if (concatenate.getLimit() != null) { + inners.add(indent(SQLUtils.toMySqlString(concatenate.getLimit()).toLowerCase())); + } + String dist = ""; + if (concatenate.isDistinct()) { + dist = "(distinct)"; + } + return String.format("Concatenate%s {\n%s\n}", dist, String.join(",\n", inners)); + } + return null; + } + + private static String indent(String s) { + String[] lines = s.split("\\n"); + for (int i = 0; i < lines.length; i++) { + lines[i] = "\t" + lines[i]; + } + return String.join("\n", lines); + } + + private String qgTestString(QueryGraph op) { + return String.format("{\n" + "Tables:\n" + "%s%s%s\n" + "}", tableNames(op), crossPredicateString(op), noDepsString(op)); + } + + private String tableNames(QueryGraph qg) { + StringBuilder tables = new StringBuilder(); + for (int i = 0; i < qg.getTables().size(); i++) { + tables.append(tableName(qg.getTables().get(i))); + if (i + 1 != qg.getTables().size()) { + tables.append("\n"); + } + } + return tables.toString(); + } + + // the following code is only used by tests + private String tableName(QueryTable qt) { + String alias = ""; + if (qt.getAlias().getAlias() != null) { + alias = "AS" + qt.getAlias().getAlias(); + } + List preds = new ArrayList<>(); + for (SQLExpr predicate : qt.getPredicates()) { + preds.add(predicate.toString()); + } + String where = ""; + if (preds.size() > 0) { + where = " where " + String.join(" and ", preds); + } + return "\t" + qt.getId() + ":" + qt.getTable().getName() + alias + where; + } + + private String crossPredicateString(QueryGraph op) { + if (op.getInnerJoins().size() == 0) { + return ""; + } + //todo join + return null; + + } + + private String noDepsString(QueryGraph op) { + if (op.getNoDeps() == null) { + return ""; + } + + return String.format("\nForAll: %s", op.getNoDeps().toString().replaceAll("\n", " ").replaceAll("AND", "and")); + + } + + private List readTestCase(String filename) throws IOException { + List testCaseList = new ArrayList<>(); + BufferedReader br = Files.newBufferedReader(Paths.get(filename), StandardCharsets.UTF_8); + String line; + int lineno = 0; + int tmpLine; + while ((line = br.readLine()) != null) { + lineno++; + tmpLine = lineno; + if (StringUtil.isNullOrEmpty(line) || line.trim().isEmpty()) { + continue; + } + String query = line; + while (query.startsWith("#")) { + query = br.readLine(); + lineno++; + } + StringBuilder expected = new StringBuilder(); + while ((line = br.readLine()) != null) { + lineno++; + expected.append(line); + if (Objects.equals(line, "}")) { + break; + } + expected.append("\n"); + } + + testCaseList.add(new TestCase(tmpLine, query, expected.toString())); + } + + return testCaseList; + } + + @AllArgsConstructor + private static class TestCase { + private int line; + + private String query; + + private String expected; + } + +} diff --git a/src/test/java/com/jd/jdbc/planbuilder/gen4/QueryProjectionTest.java b/src/test/java/com/jd/jdbc/planbuilder/gen4/QueryProjectionTest.java new file mode 100644 index 0000000..f3d9abe --- /dev/null +++ b/src/test/java/com/jd/jdbc/planbuilder/gen4/QueryProjectionTest.java @@ -0,0 +1,120 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4; + +import com.google.common.collect.Lists; +import com.jd.BaseTest; +import com.jd.jdbc.planbuilder.semantics.Analyzer; +import com.jd.jdbc.planbuilder.semantics.FakeSI; +import com.jd.jdbc.sqlparser.SQLUtils; +import com.jd.jdbc.sqlparser.ast.SQLStatement; +import com.jd.jdbc.sqlparser.ast.expr.SQLCharExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLIdentifierExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLIntegerExpr; +import com.jd.jdbc.sqlparser.ast.expr.SQLMethodInvokeExpr; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectOrderByItem; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectStatement; +import com.jd.jdbc.sqlparser.dialect.mysql.ast.statement.MySqlSelectQueryBlock; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import lombok.ToString; +import org.junit.Assert; +import org.junit.Test; + +public class QueryProjectionTest extends BaseTest { + + @Test + public void testQP() { + SQLMethodInvokeExpr concatMethodExpr = new SQLMethodInvokeExpr("CONCAT"); + concatMethodExpr.addParameter(new SQLIdentifierExpr("last_name")); + concatMethodExpr.addParameter(new SQLCharExpr(", ")); + concatMethodExpr.addParameter(new SQLIdentifierExpr("first_name")); + + QP[] tcases = new QP[] { + new QP("select * from user"), + new QP("select 1, count(1) from user"), + new QP("select max(id) from user"), + new QP("select 1, count(1) from user order by 1", + Lists.newArrayList(new QueryProjection.OrderBy(new SQLSelectOrderByItem(new SQLIntegerExpr(1)), new SQLIntegerExpr(1)))), + new QP("select id from user order by col, id, 1", + Lists.newArrayList(new QueryProjection.OrderBy(new SQLSelectOrderByItem(new SQLIdentifierExpr("col")), new SQLIdentifierExpr("col")), + new QueryProjection.OrderBy(new SQLSelectOrderByItem(new SQLIdentifierExpr("id")), new SQLIdentifierExpr("id")), + new QueryProjection.OrderBy(new SQLSelectOrderByItem(new SQLIdentifierExpr("id")), new SQLIdentifierExpr("id"))) + ), + new QP("SELECT CONCAT(last_name,', ',first_name) AS full_name FROM mytable ORDER BY full_name", + Lists.newArrayList(new QueryProjection.OrderBy(new SQLSelectOrderByItem(new SQLIdentifierExpr("full_name")), concatMethodExpr))), + new QP("select count(*) b from user group by b", "Can't group on 'COUNT(*)'") + }; + + for (QP tcase : tcases) { + String sql = tcase.sql; + String expErr = tcase.expErr; + List expOrder = tcase.expOrder; + try { + SQLStatement parse = SQLUtils.parseSingleMysqlStatement(sql); + Analyzer.analyze((SQLSelectStatement) parse, "", new FakeSI(new HashMap<>(), null)); + QueryProjection qp = new QueryProjection(); + MySqlSelectQueryBlock query = (MySqlSelectQueryBlock) ((SQLSelectStatement) parse).getSelect().getQuery(); + qp.createQPFromSelect(query); + + if (expErr != null) { + Assert.fail(); + } else { + Assert.assertEquals(query.getSelectList().size(), qp.getSelectExprs().size()); + Assert.assertEquals(expOrder.size(), qp.getOrderExprs().size()); + for (int i = 0; i < expOrder.size(); i++) { + Assert.assertEquals(expOrder.get(i).getInner(), qp.getOrderExprs().get(i).getInner()); + Assert.assertEquals(expOrder.get(i).getWeightStrExpr(), qp.getOrderExprs().get(i).getWeightStrExpr()); + } + printOk("testQP [OK] , case= " + tcase); + } + } catch (Exception e) { + Assert.assertEquals(expErr, e.getMessage()); + } + } + } + + @ToString + class QP { + String sql; + + String expErr; + + List expOrder; + + public QP(String sql) { + this.sql = sql; + this.expOrder = new ArrayList<>(); + } + + public QP(String sql, List expOrder) { + this.sql = sql; + this.expOrder = expOrder; + } + + public QP(String sql, String expErr) { + this.sql = sql; + this.expErr = expErr; + } + } +} + + + diff --git a/src/test/java/com/jd/jdbc/planbuilder/gen4/RewriterTest.java b/src/test/java/com/jd/jdbc/planbuilder/gen4/RewriterTest.java new file mode 100644 index 0000000..fddffaa --- /dev/null +++ b/src/test/java/com/jd/jdbc/planbuilder/gen4/RewriterTest.java @@ -0,0 +1,105 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4; + +import com.jd.BaseTest; +import com.jd.jdbc.common.tuple.ImmutablePair; +import com.jd.jdbc.common.tuple.Pair; +import com.jd.jdbc.planbuilder.semantics.Analyzer; +import com.jd.jdbc.planbuilder.semantics.FakeSI; +import com.jd.jdbc.planbuilder.semantics.SemTable; +import com.jd.jdbc.sqlparser.SQLUtils; +import com.jd.jdbc.sqlparser.ast.SQLStatement; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectStatement; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.junit.Assert; +import org.junit.Test; + +public class RewriterTest extends BaseTest { + @Test + public void testHavingRewrite() throws SQLException { + class TmpCase { + String input; + + String output; + + Map sql; + + public TmpCase(String input, String output) { + this.input = input; + this.output = output; + } + + public TmpCase(String input, String output, Map sql) { + this.input = input; + this.output = output; + this.sql = sql; + } + } + + List tmpCases = new ArrayList<>(); + tmpCases.add(new TmpCase("select 1 from t1 having a = 1", "select 1 from t1 where a = 1")); + tmpCases.add(new TmpCase("select 1 from t1 where x = 1 and y = 2 having a = 1", "select 1 from t1 where x = 1 and y = 2 and a = 1")); + tmpCases.add(new TmpCase("select 1 from t1 where x = 1 or y = 2 having a = 1", "select 1 from t1 where (x = 1 or y = 2) and a = 1")); + tmpCases.add(new TmpCase("select 1 from t1 where x = 1 having a = 1 and b = 2", "select 1 from t1 where x = 1 and a = 1 and b = 2")); + tmpCases.add(new TmpCase("select 1 from t1 where x = 1 having a = 1 or b = 2", "select 1 from t1 where x = 1 and (a = 1 or b = 2)")); + tmpCases.add(new TmpCase("select 1 from t1 where x = 1 and y = 2 having a = 1 and b = 2", "select 1 from t1 where x = 1 and y = 2 and a = 1 and b = 2")); + tmpCases.add(new TmpCase("select 1 from t1 where x = 1 or y = 2 having a = 1 and b = 2", "select 1 from t1 where (x = 1 or y = 2) and a = 1 and b = 2")); + tmpCases.add(new TmpCase("select 1 from t1 where x = 1 and y = 2 having a = 1 or b = 2", "select 1 from t1 where x = 1 and y = 2 and (a = 1 or b = 2)")); + tmpCases.add(new TmpCase("select 1 from t1 where x = 1 or y = 2 having a = 1 or b = 2", "select 1 from t1 where (x = 1 or y = 2) and (a = 1 or b = 2)")); + tmpCases.add(new TmpCase("select 1 from t1 where x = 1 or y = 2 having a = 1 and count(*) = 1", "select 1 from t1 where (x = 1 or y = 2) and a = 1 having count(*) = 1")); + tmpCases.add(new TmpCase("select count(*) k from t1 where x = 1 or y = 2 having a = 1 and k = 1", "select count(*) as k from t1 where (x = 1 or y = 2) and a = 1 having k = 1")); + tmpCases.add(new TmpCase("select count(*) k from t1 having k = 10", "select count(*) as k from t1 having k = 10")); + tmpCases.add(new TmpCase("select 1 from t1 group by a having a = 1 and count(*) > 1", "select 1 from t1 where a = 1 group by a having count(*) > 1")); + // new + tmpCases.add(new TmpCase("select count(*) as k from t1 having k = 10", "select count(*) as k from t1 having k = 10")); + tmpCases.add(new TmpCase("select col1 as k from t1 group by t having k = 10", "select col1 as k from t1 where col1 = 10 group by t")); + tmpCases.add(new TmpCase("select t1.col1 as k from t1 group by t having k = 10", "select t1.col1 as k from t1 where t1.col1 = 10 group by t")); + tmpCases.add(new TmpCase("select t1.col1 from t1 group by t having t1.col1 = 10", "select t1.col1 from t1 where t1.col1 = 10 group by t")); + tmpCases.add(new TmpCase("select distinct col1 as k from t1 group by k having k = 10", "select distinct col1 as k from t1 where col1 = 10 group by k")); + tmpCases.add(new TmpCase("select distinct t1.col1 as k from t1 group by k having k = 10", "select distinct t1.col1 as k from t1 where t1.col1 = 10 group by k")); + tmpCases.add(new TmpCase("select distinct t1.col1 from t1 group by k having t1.col1 = 10", "select distinct t1.col1 from t1 where t1.col1 = 10 group by k")); + tmpCases.add(new TmpCase("select col1 as k,count(col2) from t1 group by t having k = 10", "select col1 as k, count(col2) from t1 where col1 = 10 group by t")); + tmpCases.add(new TmpCase("select col1 as k,count(col2) as c from t1 group by t having k = 10 and c > 1", "select col1 as k, count(col2) as c from t1 where col1 = 10 group by t having c > 1")); + tmpCases.add( + new TmpCase("select col1 as k,count(t1.col2) as c from t1 group by t having k = 10 and c > 1", "select col1 as k, count(t1.col2) as c from t1 where col1 = 10 group by t having c > 1")); + tmpCases.add(new TmpCase("select distinct col1 as k,count(t1.col2) as c from t1 group by t having k = 10 and c > 1", + "select distinct col1 as k, count(t1.col2) as c from t1 where col1 = 10 group by t having c > 1")); + tmpCases.add( + new TmpCase("select col1 as k,count(t1.col2) as c from t1 group by t having k = 10 and c > 1", "select col1 as k, count(t1.col2) as c from t1 where col1 = 10 group by t having c > 1")); + + for (TmpCase tmpCase : tmpCases) { + Pair pair = prepTest(tmpCase.input); + SQLSelectStatement sel = pair.getRight(); + Rewriter.queryRewrite(pair.getLeft(), null, sel); + Assert.assertEquals("", tmpCase.output, SQLUtils.toMySqlString(sel, SQLUtils.NOT_FORMAT_OPTION).trim()); + printOk("testHavingRewrite is [OK],current sql = " + tmpCase.input); + } + } + + private Pair prepTest(String sql) throws SQLException { + SQLStatement stmt = SQLUtils.parseSingleMysqlStatement(sql); + SemTable semTable = Analyzer.analyze((SQLSelectStatement) stmt, "", new FakeSI(new HashMap<>(), null)); + return new ImmutablePair(semTable, stmt); + } +} diff --git a/src/test/java/com/jd/jdbc/planbuilder/gen4/operator/physical/RoutePlanningTest.java b/src/test/java/com/jd/jdbc/planbuilder/gen4/operator/physical/RoutePlanningTest.java new file mode 100644 index 0000000..b572c8a --- /dev/null +++ b/src/test/java/com/jd/jdbc/planbuilder/gen4/operator/physical/RoutePlanningTest.java @@ -0,0 +1,72 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.gen4.operator.physical; + +import com.google.common.collect.Lists; +import com.jd.BaseTest; +import com.jd.jdbc.sqlparser.SQLUtils; +import com.jd.jdbc.sqlparser.ast.SQLExpr; +import java.util.ArrayList; +import java.util.List; +import lombok.AllArgsConstructor; +import org.junit.Assert; +import org.junit.Test; + +public class RoutePlanningTest extends BaseTest { + + @Test + public void tryRewriteOrToIn() { + @AllArgsConstructor + class TestCase { + SQLExpr orExpr; + + List targetExprs; + + @Override + public String toString() { + return "orExpr:" + orExpr; + } + } + List testCaseList = new ArrayList<>(); + testCaseList.add(new TestCase(SQLUtils.toMySqlExpr("a = 1 or a = 2"), Lists.newArrayList(SQLUtils.toMySqlExpr("a in (1,2)")))); + testCaseList.add(new TestCase(SQLUtils.toMySqlExpr("a = 1 or a = 2 or a = 3"), Lists.newArrayList(SQLUtils.toMySqlExpr("a in (1,2,3)")))); + testCaseList.add(new TestCase(SQLUtils.toMySqlExpr("1 = a or a = 2 or a = 3"), Lists.newArrayList(SQLUtils.toMySqlExpr("a in (1,2,3)")))); + testCaseList.add(new TestCase(SQLUtils.toMySqlExpr("1 = a or 2 = a or a = 3"), Lists.newArrayList(SQLUtils.toMySqlExpr("a in (1,2,3)")))); + testCaseList.add(new TestCase(SQLUtils.toMySqlExpr("a = 1 or a in (2,3)"), Lists.newArrayList(SQLUtils.toMySqlExpr("a in (1,2,3)")))); + testCaseList.add(new TestCase(SQLUtils.toMySqlExpr("1 = a or a in (2,3)"), Lists.newArrayList(SQLUtils.toMySqlExpr("a in (1,2,3)")))); + testCaseList.add(new TestCase(SQLUtils.toMySqlExpr("a in (1,4) or a in (2,3)"), Lists.newArrayList(SQLUtils.toMySqlExpr("a in (1,4,2,3)")))); + testCaseList.add(new TestCase(SQLUtils.toMySqlExpr("b = 0 or a = 1 or a = 2"), Lists.newArrayList(SQLUtils.toMySqlExpr("a in (1,2)")))); + testCaseList.add(new TestCase(SQLUtils.toMySqlExpr("b = 0 or a = 1 or a = 2 or a = 3"), Lists.newArrayList(SQLUtils.toMySqlExpr("a in (1,2,3)")))); + testCaseList.add(new TestCase(SQLUtils.toMySqlExpr("b = 0 or 1 = a or a = 2 or a = 3"), Lists.newArrayList(SQLUtils.toMySqlExpr("a in (1,2,3)")))); + testCaseList.add(new TestCase(SQLUtils.toMySqlExpr("b = 0 or 1 = a or 2 = a or a = 3"), Lists.newArrayList(SQLUtils.toMySqlExpr("a in (1,2,3)")))); + testCaseList.add(new TestCase(SQLUtils.toMySqlExpr("b = 0 or a = 1 or a in (2,3)"), Lists.newArrayList(SQLUtils.toMySqlExpr("a in (1,2,3)")))); + testCaseList.add(new TestCase(SQLUtils.toMySqlExpr("b = 0 or 1 = a or a in (2,3)"), Lists.newArrayList(SQLUtils.toMySqlExpr("a in (1,2,3)")))); + testCaseList.add(new TestCase(SQLUtils.toMySqlExpr("b = 0 or a in (1,4) or a in (2,3)"), Lists.newArrayList(SQLUtils.toMySqlExpr("a in (1,4,2,3)")))); + testCaseList.add(new TestCase(SQLUtils.toMySqlExpr("c = 100 or c = 99 or a = 1 or a = 2"), Lists.newArrayList(SQLUtils.toMySqlExpr("a in (1,2)"), SQLUtils.toMySqlExpr("c in (100,99)")))); + testCaseList.add(new TestCase(SQLUtils.toMySqlExpr("c = 100 or 99 = c or a = 1 or a = 2 or a = 3"), Lists.newArrayList(SQLUtils.toMySqlExpr("a in (1,2,3)"), SQLUtils.toMySqlExpr("c in (100,99)")))); + testCaseList.add(new TestCase(SQLUtils.toMySqlExpr("c = '100' or c = '99' or 1 = a or a = 2 or a = 3"), Lists.newArrayList(SQLUtils.toMySqlExpr("a in (1,2,3)"), SQLUtils.toMySqlExpr("c in ('100','99')")))); + testCaseList.add(new TestCase(SQLUtils.toMySqlExpr("c in (100,99) or 1 = a or 2 = a or a = 3"), Lists.newArrayList(SQLUtils.toMySqlExpr("a in (1,2,3)")))); + + for (TestCase testCase : testCaseList) { + List sqlExprList = RoutePlanning.tryRewriteOrToIn(testCase.orExpr); + Assert.assertEquals(testCase.targetExprs, sqlExprList); + printOk(testCase + "\n"); + } + } +} \ No newline at end of file diff --git a/src/test/java/com/jd/jdbc/planbuilder/semantics/EarlyRewriterTest.java b/src/test/java/com/jd/jdbc/planbuilder/semantics/EarlyRewriterTest.java new file mode 100644 index 0000000..010b056 --- /dev/null +++ b/src/test/java/com/jd/jdbc/planbuilder/semantics/EarlyRewriterTest.java @@ -0,0 +1,85 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import com.jd.BaseTest; +import com.jd.jdbc.sqlparser.SQLUtils; +import com.jd.jdbc.sqlparser.ast.SQLStatement; +import com.jd.jdbc.sqlparser.ast.statement.SQLSelectStatement; +import com.jd.jdbc.sqlparser.utils.StringUtils; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import lombok.AllArgsConstructor; +import org.junit.Assert; +import org.junit.Test; + +public class EarlyRewriterTest extends BaseTest { + @Test + public void testOrderByGroupByLiteral() { + FakeSI schemaInfo = new FakeSI(new HashMap<>(), null); + String cDB = "db"; + + @AllArgsConstructor + class TmpCase { + String sql; + + String expSQL; + + String expErr; + } + List tmpCases = new ArrayList<>(); + tmpCases.add(new TmpCase("select 1 as id from t1 order by 1 desc", "select 1 as id from t1 order by id desc", null)); + tmpCases.add(new TmpCase("select 1 as id from t1 order by 1", "select 1 as id from t1 order by id", null)); + tmpCases.add(new TmpCase("select 1 as id from t1 order by 1 asc", "select 1 as id from t1 order by id asc", null)); + tmpCases.add(new TmpCase("select t1.col from t1 order by 1 desc", "select t1.col from t1 order by t1.col desc", null)); + tmpCases.add(new TmpCase("select t1.col from t1 order by 1", "select t1.col from t1 order by t1.col", null)); + tmpCases.add(new TmpCase("select t1.col from t1 order by 1 asc", "select t1.col from t1 order by t1.col asc", null)); + tmpCases.add(new TmpCase("select t1.col from t1 group by 1", "select t1.col from t1 group by t1.col", null)); + tmpCases.add(new TmpCase("select t1.col as xyz from t1 group by 1", "select t1.col as xyz from t1 group by xyz", null)); + tmpCases.add(new TmpCase("select t1.col as xyz, count(*) from t1 group by 1 order by 2", "select t1.col as xyz, count(*) from t1 group by xyz order by count(*)", null)); + tmpCases.add(new TmpCase("select t1.col as xyz, count(*) from t1 group by 1 order by 2 asc", "select t1.col as xyz, count(*) from t1 group by xyz order by count(*) asc", null)); + tmpCases.add(new TmpCase("select t1.col as xyz, count(*) from t1 group by 1 order by 2 desc", "select t1.col as xyz, count(*) from t1 group by xyz order by count(*) desc", null)); + tmpCases.add(new TmpCase("select id from t1 group by 2", null, "Unknown column '2' in 'group statement'")); + tmpCases.add(new TmpCase("select id from t1 order by 2", null, "Unknown column '2' in 'order clause'")); + tmpCases.add(new TmpCase("select *, id from t1 order by 2", null, "cannot use column offsets in order clause when using `*`")); + tmpCases.add(new TmpCase("select *, id from t1 group by 2", null, "cannot use column offsets in group statement when using `*`")); + tmpCases.add(new TmpCase("select id from t1 order by 1 collate utf8_general_ci", "select id from t1 order by id collate utf8_general_ci", null)); + tmpCases.add(new TmpCase("select id from t1 order by 1 collate utf8_general_ci asc", "select id from t1 order by id collate utf8_general_ci asc", null)); + tmpCases.add(new TmpCase("select id from t1 order by 1 collate utf8_general_ci desc", "select id from t1 order by id collate utf8_general_ci desc", null)); + + for (TmpCase tmpCase : tmpCases) { + SQLStatement ast = SQLUtils.parseSingleMysqlStatement(tmpCase.sql); + Assert.assertNotNull(ast); + try { + SemTable semTable = Analyzer.analyze((SQLSelectStatement) ast, cDB, schemaInfo); + Assert.assertTrue(StringUtils.isEmpty(tmpCase.expErr)); + String trim = SQLUtils.toMySqlString(ast, SQLUtils.NOT_FORMAT_OPTION).trim(); + Assert.assertEquals(tmpCase.expSQL, trim); + printOk("testOrderByGroupByLiteral is [OK],current sql = " + tmpCase.sql); + } catch (SQLException e) { + Assert.assertFalse(StringUtils.isEmpty(tmpCase.expErr)); + Assert.assertEquals(tmpCase.expErr, e.getMessage()); + printOk("expect error [OK],current sql = " + tmpCase.sql); + } + + } + } +} \ No newline at end of file diff --git a/src/test/java/com/jd/jdbc/planbuilder/semantics/FakeSI.java b/src/test/java/com/jd/jdbc/planbuilder/semantics/FakeSI.java new file mode 100644 index 0000000..04efe1f --- /dev/null +++ b/src/test/java/com/jd/jdbc/planbuilder/semantics/FakeSI.java @@ -0,0 +1,51 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import com.jd.jdbc.sqlparser.ast.statement.SQLExprTableSource; +import com.jd.jdbc.sqlparser.utils.TableNameUtils; +import java.sql.SQLException; +import java.util.Map; +import vschema.Vschema; + +public class FakeSI implements SchemaInformation { + + private Map tableMap; + + private Map vindexTables; + + public FakeSI(Map tables, Map vindexTables) { + this.tableMap = tables; + this.vindexTables = vindexTables; + } + + @Override + public SchemaInformationContext findTableOrVindex(SQLExprTableSource tableSource) throws SQLException { + Vschema.Table table = tableMap.get(TableNameUtils.getTableSimpleName(tableSource)); + if (table != null) { + return new SchemaInformationContext(table, null); + } +// return null; + Vschema.ColumnVindex vindex = null; + if (vindexTables != null) { + vindex = vindexTables.get(TableNameUtils.getTableSimpleName(tableSource)); + } + return new SchemaInformationContext(null, vindex); + } +} diff --git a/src/test/java/com/jd/jdbc/planbuilder/semantics/TableSetTest.java b/src/test/java/com/jd/jdbc/planbuilder/semantics/TableSetTest.java new file mode 100644 index 0000000..c6d6777 --- /dev/null +++ b/src/test/java/com/jd/jdbc/planbuilder/semantics/TableSetTest.java @@ -0,0 +1,246 @@ +/* +Copyright 2023 JD Project Authors. Licensed under Apache-2.0. + +Copyright 2022 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 com.jd.jdbc.planbuilder.semantics; + +import static com.jd.jdbc.planbuilder.semantics.TableSet.singleTableSet; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Random; +import org.junit.Assert; +import org.junit.Test; + +public class TableSetTest { + private final TableSet f1 = singleTableSet(0); + + private final TableSet f2 = singleTableSet(1); + + private final TableSet f3 = singleTableSet(2); + + private final TableSet f12 = f1.merge(f2); + + private final TableSet f123 = f12.merge(f3); + + @Test + public void testTableOffset() { + Assert.assertEquals(0, f1.tableOffset()); + Assert.assertEquals(1, f2.tableOffset()); + Assert.assertEquals(2, f3.tableOffset()); + } + + @Test + public void testLargeOffset() { + for (int tid = 0; tid < 1024; tid++) { + TableSet ts = singleTableSet(tid); + Assert.assertEquals(tid, ts.tableOffset()); + } + } + + @Test + public void testTableSetConstituents() { + Assert.assertEquals(Arrays.asList(f1, f2, f3), f123.constituents()); + Assert.assertEquals(Arrays.asList(f1, f2), f12.constituents()); + Assert.assertEquals(Arrays.asList(f1, f3), f1.merge(f3).constituents()); + Assert.assertEquals(Arrays.asList(f2, f3), f2.merge(f3).constituents()); + Assert.assertEquals(new ArrayList<>(), new TableSet().constituents()); + } + + @Test + public void testIsOverlapping() { + Assert.assertTrue(f12.isOverlapping(f12)); + Assert.assertTrue(f1.isOverlapping(f12)); + Assert.assertTrue(f12.isOverlapping(f1)); + Assert.assertFalse(f3.isOverlapping(f12)); + Assert.assertFalse(f12.isOverlapping(f3)); + } + + @Test + public void testIsSolvedBy() { + Assert.assertTrue(f1.isSolvedBy(f12)); + Assert.assertFalse(f12.isSolvedBy(f1)); + Assert.assertFalse(f3.isSolvedBy(f12)); + Assert.assertFalse(f12.isSolvedBy(f3)); + } + + @Test + public void testConstituents() { + Assert.assertEquals(Arrays.asList(f1, f2, f3), f123.constituents()); + Assert.assertEquals(Arrays.asList(f1, f2), f12.constituents()); + Assert.assertEquals(Arrays.asList(f1, f3), f1.merge(f3).constituents()); + Assert.assertEquals(Arrays.asList(f2, f3), f2.merge(f3).constituents()); + Assert.assertTrue(new TableSet().constituents().isEmpty()); + } + + @Test + public void testLargeTablesConstituents() { + final int GapSize = 32; + + TableSet ts = new TableSet(); + List expected = new ArrayList<>(); + int table = 0; + + for (int t = 0; t < 256; t++) { + table += new Random().nextInt(GapSize) + 1; + expected.add(singleTableSet(table)); + ts.addTable(table); + } + Assert.assertEquals(expected, ts.constituents()); + } + + @Test + public void testTableSetLargeMergeInPlace() { + final int SetRange = 256; + final int Blocks = 64; + + TableSet[] tablesets = new TableSet[Blocks]; + for (int i = 0; i < Blocks; i++) { + tablesets[i] = new TableSet(); + int setrng = i * SetRange; + + for (int tid = 0; tid < SetRange; tid++) { + tablesets[i].addTable(setrng + tid); + } + } + TableSet result = new TableSet(); + for (TableSet ts : tablesets) { + result.mergeInPlace(ts); + } + List expected = new ArrayList<>(); + for (int tid = 0; tid < SetRange * Blocks; tid++) { + expected.add(singleTableSet(tid)); + } + Assert.assertEquals(expected, result.constituents()); + } + + @Test + public void testTabletSetLargeMerge() { + final int SetRange = 256; + final int Blocks = 64; + + TableSet[] tablesets = new TableSet[64]; + + for (int i = 0; i < tablesets.length; i++) { + TableSet ts = new TableSet(); + int setrng = i * SetRange; + + for (int tid = 0; tid < SetRange; tid++) { + ts.addTable(setrng + tid); + } + + tablesets[i] = ts; + } + + TableSet result = new TableSet(); + for (TableSet ts : tablesets) { + result = result.merge(ts); + } + + List expected = new ArrayList<>(); + for (int tid = 0; tid < SetRange * Blocks; tid++) { + expected.add(singleTableSet(tid)); + } + Assert.assertEquals(expected, result.constituents()); + } + + @Test + public void testTableSetKeepOnly() { + + TableSet ts1, ts2, result; + + // Test case 1 + ts1 = singleTableSet(1).merge(singleTableSet(2)).merge(singleTableSet(3)); + ts2 = singleTableSet(1).merge(singleTableSet(3)).merge(singleTableSet(4)); + result = singleTableSet(1).merge(singleTableSet(3)); + ts1.keepOnly(ts2); + Assert.assertEquals("both small test is fail", result, ts1); + + // Test case 2 + ts1 = singleTableSet(1428).merge(singleTableSet(2432)).merge(singleTableSet(3412)); + ts2 = singleTableSet(1428).merge(singleTableSet(3412)).merge(singleTableSet(4342)); + result = singleTableSet(1428).merge(singleTableSet(3412)); + ts1.keepOnly(ts2); + Assert.assertEquals("both large test is fail", result, ts1); + + // Test case 3 + ts1 = singleTableSet(1).merge(singleTableSet(2)).merge(singleTableSet(3)); + ts2 = singleTableSet(1).merge(singleTableSet(3)).merge(singleTableSet(4342)); + result = singleTableSet(1).merge(singleTableSet(3)); + ts1.keepOnly(ts2); + Assert.assertEquals("ts1 small ts2 large test is fail", result, ts1); + + // Test case 4 + ts1 = singleTableSet(1).merge(singleTableSet(2771)).merge(singleTableSet(3)); + ts2 = singleTableSet(1).merge(singleTableSet(3)).merge(singleTableSet(4)); + result = singleTableSet(1).merge(singleTableSet(3)); + ts1.keepOnly(ts2); + Assert.assertEquals("ts1 large ts2 small test is fail", result, ts1); + } + + @Test + public void testRemoveInPlace() { + class TableSetTestCase { + String name; + + TableSet ts1; + + TableSet ts2; + + TableSet result; + + public TableSetTestCase(String name, TableSet ts1, TableSet ts2, TableSet result) { + this.name = name; + this.ts1 = ts1; + this.ts2 = ts2; + this.result = result; + } + } + List testCases = Arrays.asList( + new TableSetTestCase( + "both small", + singleTableSet(1).merge(singleTableSet(2)).merge(singleTableSet(3)), + singleTableSet(1).merge(singleTableSet(5)).merge(singleTableSet(4)), + singleTableSet(2).merge(singleTableSet(3)) + ), + new TableSetTestCase( + "both large", + singleTableSet(1428).merge(singleTableSet(2432)).merge(singleTableSet(3412)), + singleTableSet(1424).merge(singleTableSet(2432)).merge(singleTableSet(4342)), + singleTableSet(1428).merge(singleTableSet(3412)) + ), + new TableSetTestCase( + "ts1 small ts2 large", + singleTableSet(1).merge(singleTableSet(2)).merge(singleTableSet(3)), + singleTableSet(14).merge(singleTableSet(2)).merge(singleTableSet(4342)), + singleTableSet(1).merge(singleTableSet(3)) + ), + new TableSetTestCase( + "ts1 large ts2 small", + singleTableSet(1).merge(singleTableSet(2771)).merge(singleTableSet(3)), + singleTableSet(1).merge(singleTableSet(3)).merge(singleTableSet(4)), + singleTableSet(2771) + ) + ); + + for (TableSetTestCase testCase : testCases) { + testCase.ts1.removeInPlace(testCase.ts2); + Assert.assertEquals(testCase.name + " test is fail", testCase.result, testCase.ts1); + } + } + +} \ No newline at end of file diff --git a/src/test/java/com/jd/jdbc/table/TableAutoGeneratedKeysTest.java b/src/test/java/com/jd/jdbc/table/TableAutoGeneratedKeysTest.java index 6345cf9..b9e042a 100644 --- a/src/test/java/com/jd/jdbc/table/TableAutoGeneratedKeysTest.java +++ b/src/test/java/com/jd/jdbc/table/TableAutoGeneratedKeysTest.java @@ -24,12 +24,11 @@ import java.sql.Statement; import org.junit.After; import org.junit.Assert; +import static org.junit.Assert.assertEquals; import org.junit.Before; import org.junit.Test; import testsuite.internal.TestSuiteShardSpec; -import static org.junit.Assert.assertEquals; - public class TableAutoGeneratedKeysTest extends VitessAutoGeneratedKeysTest { @Before diff --git a/src/test/java/com/jd/jdbc/topo/MemoryTopoFactory.java b/src/test/java/com/jd/jdbc/topo/MemoryTopoFactory.java index 35750a5..8a594e8 100644 --- a/src/test/java/com/jd/jdbc/topo/MemoryTopoFactory.java +++ b/src/test/java/com/jd/jdbc/topo/MemoryTopoFactory.java @@ -19,6 +19,7 @@ package com.jd.jdbc.topo; import com.jd.jdbc.context.VtBackgroundContext; +import static com.jd.jdbc.topo.TopoExceptionCode.NO_NODE; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import lombok.AllArgsConstructor; @@ -26,8 +27,6 @@ import lombok.Synchronized; import org.apache.commons.lang3.RandomUtils; -import static com.jd.jdbc.topo.TopoExceptionCode.NO_NODE; - @Getter public class MemoryTopoFactory implements TopoFactory { diff --git a/src/test/java/com/jd/jdbc/vitess/VitessAutoGeneratedKeysRewriteBatchTest.java b/src/test/java/com/jd/jdbc/vitess/VitessAutoGeneratedKeysRewriteBatchTest.java index 4571fef..406ad3b 100644 --- a/src/test/java/com/jd/jdbc/vitess/VitessAutoGeneratedKeysRewriteBatchTest.java +++ b/src/test/java/com/jd/jdbc/vitess/VitessAutoGeneratedKeysRewriteBatchTest.java @@ -20,9 +20,8 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; -import testsuite.internal.TestSuiteShardSpec; - import static org.junit.Assert.assertEquals; +import testsuite.internal.TestSuiteShardSpec; public class VitessAutoGeneratedKeysRewriteBatchTest extends VitessAutoGeneratedKeysTest { @Override diff --git a/src/test/java/testsuite/internal/environment/DriverEnv.java b/src/test/java/testsuite/internal/environment/DriverEnv.java index 3369cdc..c994482 100644 --- a/src/test/java/testsuite/internal/environment/DriverEnv.java +++ b/src/test/java/testsuite/internal/environment/DriverEnv.java @@ -20,9 +20,8 @@ import java.sql.SQLException; import testsuite.internal.TestSuiteShardSpec; import testsuite.internal.config.DriverJdbcCfg; -import testsuite.internal.config.TestSuiteCfgReader; - import static testsuite.internal.config.TestSuiteCfgPath.DEV; +import testsuite.internal.config.TestSuiteCfgReader; public class DriverEnv extends TestSuiteEnv { diff --git a/src/test/resources/plan/Gen4/operator_test_data.txt b/src/test/resources/plan/Gen4/operator_test_data.txt new file mode 100644 index 0000000..131ecf9 --- /dev/null +++ b/src/test/resources/plan/Gen4/operator_test_data.txt @@ -0,0 +1,169 @@ +select * from t +QueryGraph: { +Tables: + TableSet{0}:t +} + +select 1 from t where '1' = 1 and 12 = '12' +QueryGraph: { +Tables: + TableSet{0}:t +ForAll: '1' = 1 and 12 = '12' +} + +select id from unsharded union all select id from unsharded_auto +Concatenate { + QueryGraph: { + Tables: + TableSet{0}:unsharded + }, + QueryGraph: { + Tables: + TableSet{1}:unsharded_auto + } +} + +select id from unsharded union select id from unsharded_auto +Concatenate(distinct) { + QueryGraph: { + Tables: + TableSet{0}:unsharded + }, + QueryGraph: { + Tables: + TableSet{1}:unsharded_auto + } +} + +select 1 from a union select 2 from b +Concatenate(distinct) { + QueryGraph: { + Tables: + TableSet{0}:a + }, + QueryGraph: { + Tables: + TableSet{1}:b + } +} + +select id from unsharded union select id from unsharded_auto order by id +Concatenate(distinct) { + QueryGraph: { + Tables: + TableSet{0}:unsharded + }, + QueryGraph: { + Tables: + TableSet{1}:unsharded_auto + }, + order by id +} + +(select id from unsharded union all select id from unsharded_auto limit 10) union select id from x order by id +Concatenate(distinct) { + Concatenate { + QueryGraph: { + Tables: + TableSet{0}:unsharded + }, + QueryGraph: { + Tables: + TableSet{1}:unsharded_auto + }, + limit 10 + }, + QueryGraph: { + Tables: + TableSet{2}:x + }, + order by id +} + +(select id from unsharded union all select id from unsharded_auto) union all select id from x +Concatenate { + QueryGraph: { + Tables: + TableSet{0}:unsharded + }, + QueryGraph: { + Tables: + TableSet{1}:unsharded_auto + }, + QueryGraph: { + Tables: + TableSet{2}:x + } +} + +(select id from unsharded union select id from unsharded_auto) union select id from x +Concatenate(distinct) { + QueryGraph: { + Tables: + TableSet{0}:unsharded + }, + QueryGraph: { + Tables: + TableSet{1}:unsharded_auto + }, + QueryGraph: { + Tables: + TableSet{2}:x + } +} + +(select id from unsharded union select id from unsharded_auto) union all select id from x +Concatenate { + Concatenate(distinct) { + QueryGraph: { + Tables: + TableSet{0}:unsharded + }, + QueryGraph: { + Tables: + TableSet{1}:unsharded_auto + } + }, + QueryGraph: { + Tables: + TableSet{2}:x + } +} + +select 1 from a union select 2 from b union select 3 from c +Concatenate(distinct) { + QueryGraph: { + Tables: + TableSet{0}:a + }, + QueryGraph: { + Tables: + TableSet{1}:b + }, + QueryGraph: { + Tables: + TableSet{2}:c + } +} + +select 1 from a union select 2 from b union select 3 from c union all select 4 from d +Concatenate { + Concatenate(distinct) { + QueryGraph: { + Tables: + TableSet{0}:a + }, + QueryGraph: { + Tables: + TableSet{1}:b + }, + QueryGraph: { + Tables: + TableSet{2}:c + } + }, + QueryGraph: { + Tables: + TableSet{3}:d + } +} \ No newline at end of file diff --git a/src/test/resources/plan/Gen4/operator_test_data_bk.txt b/src/test/resources/plan/Gen4/operator_test_data_bk.txt new file mode 100644 index 0000000..8c45ca0 --- /dev/null +++ b/src/test/resources/plan/Gen4/operator_test_data_bk.txt @@ -0,0 +1,310 @@ +#todo from user_index的测试用例不用跑,ks.tab.col有问题没修复 +(select id from unsharded union all select id from unsharded_auto order by id) union select id from user +Concatenate(distinct) { + QueryGraph: { + Tables: + TableSet{0}:unsharded + }, + QueryGraph: { + Tables: + TableSet{1}:unsharded_auto + }, + QueryGraph: { + Tables: + TableSet{2}:`user` + } +} + +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 +QueryGraph: { +Tables: + TableSet{0}:t + TableSet{1}:y + TableSet{2}:z +JoinPredicates: + TableSet{0,1,2} - t.a = z.a or t.a = y.a + TableSet{0,1} - t.c = y.c +ForAll: 1 < 2 +} + +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 +QueryGraph: { +Tables: + TableSet{0}:t where t.`name` = 'foo' + TableSet{1}:y where y.col = 42 + TableSet{2}:z where z.baz = 101 +JoinPredicates: + TableSet{0,1} - t.id = y.t_id + TableSet{0,2} - t.id = z.t_id +} + +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 +QueryGraph: { +Tables: + TableSet{0}:t where t.`name` = 'foo' + TableSet{1}:y where y.col = 42 + TableSet{2}:z where z.baz = 101 +JoinPredicates: + TableSet{0,1} - t.id = y.t_id + TableSet{0,2} - t.id = z.t_id +} + +select 1 from t left join s on t.id = s.id +OuterJoin: { + Inner: QueryGraph: { + Tables: + TableSet{0}:t + } + Outer: QueryGraph: { + Tables: + TableSet{1}:s + } + Predicate: t.id = s.id +} + +select 1 from t join s on t.id = s.id and t.name = s.name +QueryGraph: { +Tables: + TableSet{0}:t + TableSet{1}:s +JoinPredicates: + TableSet{0,1} - t.id = s.id and t.`name` = s.`name` +} + +select 1 from t left join s on t.id = s.id where t.name = 'Mister' +OuterJoin: { + Inner: QueryGraph: { + Tables: + TableSet{0}:t where t.`name` = 'Mister' + } + Outer: QueryGraph: { + Tables: + TableSet{1}:s + } + Predicate: t.id = s.id +} + +select 1 from t right join s on t.id = s.id +OuterJoin: { + Inner: QueryGraph: { + Tables: + TableSet{1}:s + } + Outer: QueryGraph: { + Tables: + TableSet{0}:t + } + Predicate: t.id = s.id +} + +select 1 from (a left join b on a.id = b.id) join (c left join d on c.id = d.id) on a.id = c.id +Join: { + LHS: OuterJoin: { + Inner: QueryGraph: { + Tables: + TableSet{0}:a + } + Outer: QueryGraph: { + Tables: + TableSet{1}:b + } + Predicate: a.id = b.id + } + RHS: OuterJoin: { + Inner: QueryGraph: { + Tables: + TableSet{2}:c + } + Outer: QueryGraph: { + Tables: + TableSet{3}:d + } + Predicate: c.id = d.id + } + Predicate: a.id = c.id +} + +select 1 from (select 42 as id from tbl) as t +Derived t: { + Query: select 42 as id from tbl + Inner: QueryGraph: { + Tables: + TableSet{0}:tbl + } +} + +select 1 from (select id from tbl limit 10) as t join (select foo, count(*) from usr group by foo) as s on t.id = s.foo +Join: { + LHS: Derived t: { + Query: select id from tbl limit 10 + Inner: QueryGraph: { + Tables: + TableSet{0}:tbl + } + } + RHS: Derived s: { + Query: select foo, count(*) from usr group by foo + Inner: QueryGraph: { + Tables: + TableSet{2}:usr + } + } + Predicate: t.id = s.foo +} + +select (select 1) from t where exists (select 1) and id in (select 1) +SubQuery: { + SubQueries: [ + { + Type: PulloutValue + Query: QueryGraph: { + Tables: + TableSet{1}:dual + } + } + { + Type: PulloutExists + Query: QueryGraph: { + Tables: + TableSet{2}:dual + } + } + { + Type: PulloutIn + Query: QueryGraph: { + Tables: + TableSet{3}:dual + } + }] + Outer: QueryGraph: { + Tables: + TableSet{0}:t where id in (select 1 from dual) + ForAll: exists (select 1 from dual) + } +} + +select u.id from user u where u.id = (select id from user_extra where id = u.id) +SubQuery: { + SubQueries: [ + { + Type: PulloutValue + Query: QueryGraph: { + Tables: + TableSet{1}:user_extra + JoinPredicates: + TableSet{0,1} - id = u.id + } + }] + Outer: QueryGraph: { + Tables: + TableSet{0}:`user` AS u where u.id = (select id from user_extra where id = u.id) + } +} + +select id from user_index where id = :id +Vindex: { + Name: user_index + Value: :id +} + +select ui.id from user_index as ui join user as u where ui.id = 1 and ui.id = u.id +Join: { + LHS: Vindex: { + Name: user_index + Value: 1 + } + RHS: QueryGraph: { + Tables: + TableSet{1}:`user` AS u + } + Predicate: ui.id = u.id +} + +select u.id from (select id from user_index where id = 2) as u +Derived u: { + Query: select id from user_index where id = 2 + Inner: Vindex: { + Name: user_index + Value: 2 + } +} + +select id from user where exists(select user_id from user_extra where user_id = 3 and user_id < user.id) +SubQuery: { + SubQueries: [ + { + Type: PulloutExists + Query: QueryGraph: { + Tables: + TableSet{1}:user_extra where user_id = 3 + JoinPredicates: + TableSet{0,1} - user_id < `user`.id + } + }] + Outer: QueryGraph: { + Tables: + TableSet{0}:`user` where exists (select user_id from user_extra where user_id = 3 and user_id < `user`.id) + } +} + +#todo +# we should remove the keyspace from predicates +select ks.tbl.col from ks.tbl where ks.tbl.id = 1 +QueryGraph: { +Tables: + TableSet{0}:ks.tbl where tbl.id = 1 +} + +select 1 from ks.t join ks.y on ks.t.id = ks.y.t_id +QueryGraph: { +Tables: + TableSet{0}:ks.t + TableSet{1}:ks.y +JoinPredicates: + TableSet{0,1} - t.id = y.t_id +} + +select 1 from ks.t left join ks.y on ks.t.id = ks.y.t_id +OuterJoin: { + Inner: QueryGraph: { + Tables: + TableSet{0}:ks.t + } + Outer: QueryGraph: { + Tables: + TableSet{1}:ks.y + } + Predicate: t.id = y.t_id +} + +update tbl set col1 = apa +Update { + table: TableSet{0}:tbl +assignments: + col1 = apa +} + +update tbl set col1 = 'apa', col2 = 1337 where id = 12 and name = 'gangal' +Update { + table: TableSet{0}:tbl where id = 12 and `name` = 'gangal' +assignments: + col1 = 'apa' + col2 = 1337 +} + +update user set u = 1 where id = (select id from user_extra where id = 42) +SubQuery: { + SubQueries: [ + { + Type: PulloutValue + Query: QueryGraph: { + Tables: + TableSet{1}:user_extra where id = 42 + } + }] + Outer: Update { + table: TableSet{0}:`user` where id = (select id from user_extra where id = 42) + assignments: + u = 1 + } +} diff --git a/src/test/resources/plan/Gen4/operator_test_data_one.txt b/src/test/resources/plan/Gen4/operator_test_data_one.txt new file mode 100644 index 0000000..e69de29 diff --git a/src/test/resources/plan/aggr_cases.txt b/src/test/resources/plan/aggr_cases.txt index 5360cd7..5bb42b1 100644 --- a/src/test/resources/plan/aggr_cases.txt +++ b/src/test/resources/plan/aggr_cases.txt @@ -17,6 +17,7 @@ "Table": "unsharded" } } +Gen4 plan same as above # Aggregate on unique sharded "select count(*), col from user where name = 1" @@ -39,6 +40,7 @@ "Vindex": "user_index" } } +Gen4 plan same as above # Aggregate detection (non-aggregate function) "select fun(1), col from user" @@ -57,6 +59,7 @@ "Table": "user" } } +Gen4 plan same as above # select distinct with unique vindex for scatter route. "select distinct col1, name from user" @@ -75,6 +78,7 @@ "Table": "user" } } +Gen4 plan same as above # distinct and group by together for single route. "select distinct col1, name from user group by col1" @@ -93,6 +97,7 @@ "Table": "user" } } +Gen4 plan same as above # scatter group by a text column "select count(*), a, textcol1, b from user group by a, textcol1, b" @@ -120,6 +125,31 @@ ] } } +{ + "QueryType": "SELECT", + "Original": "select count(*), a, textcol1, b from user group by a, textcol1, b", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count_star(0) AS count(*)", + "GroupBy": "(1|4), 2, (3|5)", + "ResultColumns": 4, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select count(*), a, textcol1, b, weight_string(a), weight_string(b) from user where 1 != 1 group by a, weight_string(a), textcol1, b, weight_string(b)", + "OrderBy": "(1|4) ASC, 2 ASC, (3|5) ASC", + "Query": "select count(*), a, textcol1, b, weight_string(a), weight_string(b) from user group by a, weight_string(a), textcol1, b, weight_string(b) order by a asc, textcol1 asc, b asc", + "Table": "user" + } + ] + } +} # scatter group by a text column, reuse existing weight_string "select count(*) k, a, textcol1, b from user group by a, textcol1, b order by k, textcol1" @@ -154,6 +184,39 @@ ] } } +{ + "QueryType": "SELECT", + "Original": "select count(*) k, a, textcol1, b from user group by a, textcol1, b order by k, textcol1", + "Instructions": { + "OperatorType": "Sort", + "Variant": "Memory", + "OrderBy": "0 ASC, 2 ASC", + "ResultColumns": 4, + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count_star(0) AS k", + "GroupBy": "(1|4), 2, (3|5)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select count(*) as k, a, textcol1, b, weight_string(a), weight_string(b) from user where 1 != 1 group by a, weight_string(a), textcol1, b, weight_string(b)", + "OrderBy": "(1|4) ASC, 2 ASC, (3|5) ASC", + "Query": "select count(*) as k, a, textcol1, b, weight_string(a), weight_string(b) from user group by a, weight_string(a), textcol1, b, weight_string(b) order by a asc, textcol1 asc, b asc", + "Table": "user" + } + ] + } + ] + } +} + # count aggregate "select count(*) from user" @@ -180,6 +243,28 @@ ] } } +{ + "QueryType": "SELECT", + "Original": "select count(*) from user", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Scalar", + "Aggregates": "sum_count_star(0) AS count(*)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select count(*) from user where 1 != 1", + "Query": "select count(*) from user", + "Table": "user" + } + ] + } +} # sum aggregate "select sum(col) from user" @@ -206,6 +291,29 @@ ] } } +{ + "QueryType": "SELECT", + "Original": "select sum(col) from user", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Scalar", + "Aggregates": "sum(0) AS sum(col)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select sum(col) from user where 1 != 1", + "Query": "select sum(col) from user", + "Table": "user" + } + ] + } +} + # min aggregate "select min(col) from user" @@ -232,6 +340,28 @@ ] } } +{ + "QueryType": "SELECT", + "Original": "select min(col) from user", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Scalar", + "Aggregates": "min(0) AS min(col)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select min(col) from user where 1 != 1", + "Query": "select min(col) from user", + "Table": "user" + } + ] + } +} # max aggregate "select max(col) from user" @@ -258,6 +388,29 @@ ] } } +{ + "QueryType": "SELECT", + "Original": "select max(col) from user", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Scalar", + "Aggregates": "max(0) AS max(col)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select max(col) from user where 1 != 1", + "Query": "select max(col) from user", + "Table": "user" + } + ] + } +} + # distinct and group by together for scatter route "select distinct col1, col2 from user group by col1" @@ -284,6 +437,30 @@ ] } } +{ + "QueryType": "SELECT", + "Original": "select distinct col1, col2 from user group by col1", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "GroupBy": "(0|2), (1|3)", + "ResultColumns": 2, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col1, col2, weight_string(col1), weight_string(col2) from user where 1 != 1 group by col1", + "OrderBy": "(0|2) ASC, (1|3) ASC", + "Query": "select distinct col1, col2, weight_string(col1), weight_string(col2) from user group by col1 order by col1 asc, col2 asc", + "Table": "user" + } + ] + } +} # group by a unique vindex should use a simple route "select name, count(*) from user group by name" @@ -302,6 +479,7 @@ "Table": "user" } } +Gen4 plan same as above # group by a unique vindex and other column should use a simple route "select name, col, count(*) from user group by name, col" @@ -320,6 +498,7 @@ "Table": "user" } } +Gen4 plan same as above # group by a non-vindex column should use an OrderdAggregate primitive "select col, count(*) from user group by col" @@ -347,6 +526,30 @@ ] } } +{ + "QueryType": "SELECT", + "Original": "select col, count(*) from user group by col", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count_star(1) AS count(*)", + "GroupBy": "0", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col, count(*) from user where 1 != 1 group by col", + "OrderBy": "0 ASC", + "Query": "select col, count(*) from user group by col order by col asc", + "Table": "user" + } + ] + } +} # group by a non-unique vindex column should use an OrderdAggregate primitive "select id, count(*) from user group by id" @@ -374,6 +577,31 @@ ] } } +{ + "QueryType": "SELECT", + "Original": "select id, count(*) from user group by id", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count_star(1) AS count(*)", + "GroupBy": "(0|2)", + "ResultColumns": 2, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id, count(*), weight_string(id) from user where 1 != 1 group by id, weight_string(id)", + "OrderBy": "(0|2) ASC", + "Query": "select id, count(*), weight_string(id) from user group by id, weight_string(id) order by id asc", + "Table": "user" + } + ] + } +} # group by a unique vindex should use a simple route, even if aggr is complex "select name, 1+count(*) from user group by name" @@ -392,6 +620,7 @@ "Table": "user" } } +Gen4 plan same as above # group by a unique vindex where alias from select list is used "select name as val, 1+count(*) from user group by val" @@ -410,6 +639,7 @@ "Table": "user" } } +Gen4 plan same as above # group by a unique vindex where expression is qualified (alias should be ignored) "select val as name, 1+count(*) from user group by user.name" @@ -428,6 +658,7 @@ "Table": "user" } } +Gen4 plan same as above # group by a unique vindex where it should skip non-aliased expressions. "select *, name, 1+count(*) from user group by name" @@ -446,6 +677,7 @@ "Table": "user" } } +Gen4 error: unsupported: '*' expression in cross-shard query # group by a unique vindex should revert to simple route, and having clause should find the correct symbols. "select name, count(*) c from user group by name having name=1 and c=10" @@ -468,6 +700,25 @@ "Vindex": "user_index" } } +{ + "QueryType": "SELECT", + "Original": "select name, count(*) c from user group by name having name=1 and c=10", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select name, count(*) as c from user where 1 != 1 group by name", + "Query": "select name, count(*) as c from user where name = 1 group by name having c = 10", + "Table": "user", + "Values": [ + "INT64(1)" + ], + "Vindex": "user_index" + } +} # scatter aggregate with non-aggregate expressions. "select id, count(*) from user" @@ -494,17 +745,13 @@ ] } } - -# scatter aggregate using distinct -"select distinct col from user" { "QueryType": "SELECT", - "Original": "select distinct col from user", + "Original": "select id, count(*) from user", "Instructions": { "OperatorType": "Aggregate", - "Variant": "Ordered", - "Distinct": "false", - "GroupBy": "0", + "Variant": "Scalar", + "Aggregates": "random(0) AS id, sum_count_star(1) AS count(*)", "Inputs": [ { "OperatorType": "Route", @@ -513,19 +760,19 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select col from user where 1 != 1", - "Query": "select distinct col from user order by col asc", + "FieldQuery": "select id, count(*) from user where 1 != 1", + "Query": "select id, count(*) from user", "Table": "user" } ] } } -# scatter aggregate group by select col -"select col from user group by col" +# scatter aggregate using distinct +"select distinct col from user" { "QueryType": "SELECT", - "Original": "select col from user group by col", + "Original": "select distinct col from user", "Instructions": { "OperatorType": "Aggregate", "Variant": "Ordered", @@ -539,42 +786,19 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select col from user where 1 != 1 group by col", - "Query": "select col from user group by col order by col asc", + "FieldQuery": "select col from user where 1 != 1", + "Query": "select distinct col from user order by col asc", "Table": "user" } ] } } - -# count with distinct group by unique vindex -"select name, count(distinct col) from user group by name" -{ - "QueryType": "SELECT", - "Original": "select name, count(distinct col) from user group by name", - "Instructions": { - "OperatorType": "Route", - "Variant": "SelectScatter", - "Keyspace": { - "Name": "user", - "Sharded": true - }, - "FieldQuery": "select name, count(DISTINCT col) from user where 1 != 1 group by name", - "Query": "select name, count(DISTINCT col) from user group by name", - "Table": "user" - } -} - -# count with distinct unique vindex -"select col, count(distinct name) from user group by col" { "QueryType": "SELECT", - "Original": "select col, count(distinct name) from user group by col", + "Original": "select distinct col from user", "Instructions": { "OperatorType": "Aggregate", "Variant": "Ordered", - "Aggregates": "count(1)", - "Distinct": "false", "GroupBy": "0", "Inputs": [ { @@ -584,24 +808,24 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select col, count(DISTINCT name) from user where 1 != 1 group by col", - "Query": "select col, count(DISTINCT name) from user group by col order by col asc", + "FieldQuery": "select col from user where 1 != 1", + "OrderBy": "0 ASC", + "Query": "select distinct col from user order by col asc", "Table": "user" } ] } } -# count with distinct no unique vindex -"select col1, count(distinct col2) from user group by col1" +# scatter aggregate group by select col +"select col from user group by col" { "QueryType": "SELECT", - "Original": "select col1, count(distinct col2) from user group by col1", + "Original": "select col from user group by col", "Instructions": { "OperatorType": "Aggregate", "Variant": "Ordered", - "Aggregates": "count_distinct(1) AS count(distinct col2)", - "Distinct": "true", + "Distinct": "false", "GroupBy": "0", "Inputs": [ { @@ -611,24 +835,20 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select col1, col2 from user where 1 != 1 group by col1, col2", - "Query": "select col1, col2 from user group by col1, col2 order by col1 asc, col2 asc", + "FieldQuery": "select col from user where 1 != 1 group by col", + "Query": "select col from user group by col order by col asc", "Table": "user" } ] } } - -# count with distinct no unique vindex and no group by -"select count(distinct col2) from user" { "QueryType": "SELECT", - "Original": "select count(distinct col2) from user", + "Original": "select col from user group by col", "Instructions": { "OperatorType": "Aggregate", "Variant": "Ordered", - "Aggregates": "count_distinct(0) AS count(distinct col2)", - "Distinct": "true", + "GroupBy": "0", "Inputs": [ { "OperatorType": "Route", @@ -637,24 +857,44 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select col2 from user where 1 != 1 group by col2", - "Query": "select col2 from user group by col2 order by col2 asc", + "FieldQuery": "select col from user where 1 != 1 group by col", + "OrderBy": "0 ASC", + "Query": "select col from user group by col order by col asc", "Table": "user" } ] } } -# count with distinct no unique vindex, count expression aliased -"select col1, count(distinct col2) c2 from user group by col1" +# count with distinct group by unique vindex +"select name, count(distinct col) from user group by name" { "QueryType": "SELECT", - "Original": "select col1, count(distinct col2) c2 from user group by col1", + "Original": "select name, count(distinct col) from user group by name", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select name, count(DISTINCT col) from user where 1 != 1 group by name", + "Query": "select name, count(DISTINCT col) from user group by name", + "Table": "user" + } +} +Gen4 plan same as above + +# count with distinct unique vindex +"select col, count(distinct name) from user group by col" +{ + "QueryType": "SELECT", + "Original": "select col, count(distinct name) from user group by col", "Instructions": { "OperatorType": "Aggregate", "Variant": "Ordered", - "Aggregates": "count_distinct(1) AS c2", - "Distinct": "true", + "Aggregates": "count(1)", + "Distinct": "false", "GroupBy": "0", "Inputs": [ { @@ -664,24 +904,20 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select col1, col2 from user where 1 != 1 group by col1, col2", - "Query": "select col1, col2 from user group by col1, col2 order by col1 asc, col2 asc", + "FieldQuery": "select col, count(DISTINCT name) from user where 1 != 1 group by col", + "Query": "select col, count(DISTINCT name) from user group by col order by col asc", "Table": "user" } ] } } - -# sum with distinct no unique vindex -"select col1, sum(distinct col2) from user group by col1" { "QueryType": "SELECT", - "Original": "select col1, sum(distinct col2) from user group by col1", + "Original": "select col, count(distinct name) from user group by col", "Instructions": { "OperatorType": "Aggregate", "Variant": "Ordered", - "Aggregates": "sum_distinct(1) AS sum(distinct col2)", - "Distinct": "true", + "Aggregates": "sum_count_distinct(1) AS count(distinct name)", "GroupBy": "0", "Inputs": [ { @@ -691,24 +927,25 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select col1, col2 from user where 1 != 1 group by col1, col2", - "Query": "select col1, col2 from user group by col1, col2 order by col1 asc, col2 asc", + "FieldQuery": "select col, count(DISTINCT name) from user where 1 != 1 group by col", + "OrderBy": "0 ASC", + "Query": "select col, count(DISTINCT name) from user group by col order by col asc", "Table": "user" } ] } } -# min with distinct no unique vindex. distinct is ignored. -"select col1, min(distinct col2) from user group by col1" +# count with distinct no unique vindex +"select col1, count(distinct col2) from user group by col1" { "QueryType": "SELECT", - "Original": "select col1, min(distinct col2) from user group by col1", + "Original": "select col1, count(distinct col2) from user group by col1", "Instructions": { "OperatorType": "Aggregate", "Variant": "Ordered", - "Aggregates": "min(1)", - "Distinct": "false", + "Aggregates": "count_distinct(1) AS count(distinct col2)", + "Distinct": "true", "GroupBy": "0", "Inputs": [ { @@ -718,63 +955,22 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select col1, min(DISTINCT col2) from user where 1 != 1 group by col1", - "Query": "select col1, min(DISTINCT col2) from user group by col1 order by col1 asc", + "FieldQuery": "select col1, col2 from user where 1 != 1 group by col1, col2", + "Query": "select col1, col2 from user group by col1, col2 order by col1 asc, col2 asc", "Table": "user" } ] } } - -# order by count distinct -"select col1, count(distinct col2) k from user group by col1 order by k" -{ - "QueryType": "SELECT", - "Original": "select col1, count(distinct col2) k from user group by col1 order by k", - "Instructions": { - "OperatorType": "Sort", - "Variant": "Memory", - "OrderBy": "1 ASC", - "Inputs": [ - { - "OperatorType": "Aggregate", - "Variant": "Ordered", - "Aggregates": "count_distinct(1) AS k", - "Distinct": "true", - "GroupBy": "0", - "Inputs": [ - { - "OperatorType": "Route", - "Variant": "SelectScatter", - "Keyspace": { - "Name": "user", - "Sharded": true - }, - "FieldQuery": "select col1, col2 from user where 1 != 1 group by col1, col2", - "Query": "select col1, col2 from user group by col1, col2 order by col1 asc, col2 asc", - "Table": "user" - } - ] - } - ] - } -} - -# scatter aggregate group by aggregate function -" select count(*) b from user group by b" -"group by expression cannot reference an aggregate function: b" - -# scatter aggregate multiple group by (columns) -"select a, b, count(*) from user group by b, a" { "QueryType": "SELECT", - "Original": "select a, b, count(*) from user group by b, a", + "Original": "select col1, count(distinct col2) from user group by col1", "Instructions": { "OperatorType": "Aggregate", "Variant": "Ordered", - "Aggregates": "count(2)", - "Distinct": "false", - "GroupBy": "1, 0", + "Aggregates": "count_distinct(1|3) AS count(distinct col2)", + "GroupBy": "(0|2)", + "ResultColumns": 2, "Inputs": [ { "OperatorType": "Route", @@ -783,25 +979,25 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select a, b, count(*) from user where 1 != 1 group by b, a", - "Query": "select a, b, count(*) from user group by b, a order by b asc, a asc", + "FieldQuery": "select col1, col2, weight_string(col1), weight_string(col2) from user where 1 != 1 group by col1, weight_string(col1), col2, weight_string(col2)", + "OrderBy": "(0|2) ASC, (1|3) ASC", + "Query": "select col1, col2, weight_string(col1), weight_string(col2) from user group by col1, weight_string(col1), col2, weight_string(col2) order by col1 asc, col2 asc", "Table": "user" } ] } } -# scatter aggregate multiple group by (numbers) -"select a, b, count(*) from user group by 2, 1" +# count with distinct no unique vindex and no group by +"select count(distinct col2) from user" { "QueryType": "SELECT", - "Original": "select a, b, count(*) from user group by 2, 1", + "Original": "select count(distinct col2) from user", "Instructions": { "OperatorType": "Aggregate", "Variant": "Ordered", - "Aggregates": "count(2)", - "Distinct": "false", - "GroupBy": "1, 0", + "Aggregates": "count_distinct(0) AS count(distinct col2)", + "Distinct": "true", "Inputs": [ { "OperatorType": "Route", @@ -810,24 +1006,21 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select a, b, count(*) from user where 1 != 1 group by 2, 1", - "Query": "select a, b, count(*) from user group by 2, 1 order by b asc, a asc", + "FieldQuery": "select col2 from user where 1 != 1 group by col2", + "Query": "select col2 from user group by col2 order by col2 asc", "Table": "user" } ] } } - -# scatter aggregate group by column number -"select col from user group by 1" { "QueryType": "SELECT", - "Original": "select col from user group by 1", + "Original": "select count(distinct col2) from user", "Instructions": { "OperatorType": "Aggregate", - "Variant": "Ordered", - "Distinct": "false", - "GroupBy": "0", + "Variant": "Scalar", + "Aggregates": "count_distinct(0|1) AS count(distinct col2)", + "ResultColumns": 1, "Inputs": [ { "OperatorType": "Route", @@ -836,28 +1029,26 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select col from user where 1 != 1 group by 1", - "Query": "select col from user group by 1 order by col asc", + "FieldQuery": "select col2, weight_string(col2) from user where 1 != 1 group by col2, weight_string(col2)", + "OrderBy": "(0|1) ASC", + "Query": "select col2, weight_string(col2) from user group by col2, weight_string(col2) order by col2 asc", "Table": "user" } ] } } -# scatter aggregate group by invalid column number -"select col from user group by 2" -"column number out of range: 2" - -# scatter aggregate order by null -"select count(*) from user order by null" +# count with distinct no unique vindex, count expression aliased +"select col1, count(distinct col2) c2 from user group by col1" { "QueryType": "SELECT", - "Original": "select count(*) from user order by null", + "Original": "select col1, count(distinct col2) c2 from user group by col1", "Instructions": { "OperatorType": "Aggregate", "Variant": "Ordered", - "Aggregates": "count(0)", - "Distinct": "false", + "Aggregates": "count_distinct(1) AS c2", + "Distinct": "true", + "GroupBy": "0", "Inputs": [ { "OperatorType": "Route", @@ -866,29 +1057,22 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select count(*) from user where 1 != 1", - "Query": "select count(*) from user", + "FieldQuery": "select col1, col2 from user where 1 != 1 group by col1, col2", + "Query": "select col1, col2 from user group by col1, col2 order by col1 asc, col2 asc", "Table": "user" } ] } } - -# scatter aggregate with complex select list (can't build order by) -"select distinct a+1 from user" -"generating order by clause: cannot reference a complex expression" - -# scatter aggregate with numbered order by columns -"select a, b, c, d, count(*) from user group by 1, 2, 3 order by 1, 2, 3" { "QueryType": "SELECT", - "Original": "select a, b, c, d, count(*) from user group by 1, 2, 3 order by 1, 2, 3", + "Original": "select col1, count(distinct col2) c2 from user group by col1", "Instructions": { "OperatorType": "Aggregate", "Variant": "Ordered", - "Aggregates": "count(4)", - "Distinct": "false", - "GroupBy": "0, 1, 2", + "Aggregates": "count_distinct(1|3) AS c2", + "GroupBy": "(0|2)", + "ResultColumns": 2, "Inputs": [ { "OperatorType": "Route", @@ -897,25 +1081,26 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select a, b, c, d, count(*) from user where 1 != 1 group by 1, 2, 3", - "Query": "select a, b, c, d, count(*) from user group by 1, 2, 3 order by 1 asc, 2 asc, 3 asc", + "FieldQuery": "select col1, col2, weight_string(col1), weight_string(col2) from user where 1 != 1 group by col1, weight_string(col1), col2, weight_string(col2)", + "OrderBy": "(0|2) ASC, (1|3) ASC", + "Query": "select col1, col2, weight_string(col1), weight_string(col2) from user group by col1, weight_string(col1), col2, weight_string(col2) order by col1 asc, col2 asc", "Table": "user" } ] } } -# scatter aggregate with named order by columns -"select a, b, c, d, count(*) from user group by 1, 2, 3 order by a, b, c" +# sum with distinct no unique vindex +"select col1, sum(distinct col2) from user group by col1" { "QueryType": "SELECT", - "Original": "select a, b, c, d, count(*) from user group by 1, 2, 3 order by a, b, c", + "Original": "select col1, sum(distinct col2) from user group by col1", "Instructions": { "OperatorType": "Aggregate", "Variant": "Ordered", - "Aggregates": "count(4)", - "Distinct": "false", - "GroupBy": "0, 1, 2", + "Aggregates": "sum_distinct(1) AS sum(distinct col2)", + "Distinct": "true", + "GroupBy": "0", "Inputs": [ { "OperatorType": "Route", @@ -924,25 +1109,22 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select a, b, c, d, count(*) from user where 1 != 1 group by 1, 2, 3", - "Query": "select a, b, c, d, count(*) from user group by 1, 2, 3 order by a asc, b asc, c asc", + "FieldQuery": "select col1, col2 from user where 1 != 1 group by col1, col2", + "Query": "select col1, col2 from user group by col1, col2 order by col1 asc, col2 asc", "Table": "user" } ] } } - -# scatter aggregate with jumbled order by columns -"select a, b, c, d, count(*) from user group by 1, 2, 3, 4 order by d, b, a, c" { "QueryType": "SELECT", - "Original": "select a, b, c, d, count(*) from user group by 1, 2, 3, 4 order by d, b, a, c", + "Original": "select col1, sum(distinct col2) from user group by col1", "Instructions": { "OperatorType": "Aggregate", "Variant": "Ordered", - "Aggregates": "count(4)", - "Distinct": "false", - "GroupBy": "0, 1, 2, 3", + "Aggregates": "sum_distinct(1|3) AS sum(distinct col2)", + "GroupBy": "(0|2)", + "ResultColumns": 2, "Inputs": [ { "OperatorType": "Route", @@ -951,25 +1133,26 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select a, b, c, d, count(*) from user where 1 != 1 group by 1, 2, 3, 4", - "Query": "select a, b, c, d, count(*) from user group by 1, 2, 3, 4 order by d asc, b asc, a asc, c asc", + "FieldQuery": "select col1, col2, weight_string(col1), weight_string(col2) from user where 1 != 1 group by col1, weight_string(col1), col2, weight_string(col2)", + "OrderBy": "(0|2) ASC, (1|3) ASC", + "Query": "select col1, col2, weight_string(col1), weight_string(col2) from user group by col1, weight_string(col1), col2, weight_string(col2) order by col1 asc, col2 asc", "Table": "user" } ] } } -# scatter aggregate with jumbled group by and order by columns -"select a, b, c, d, count(*) from user group by 3, 2, 1, 4 order by d, b, a, c" +# min with distinct no unique vindex. distinct is ignored. +"select col1, min(distinct col2) from user group by col1" { "QueryType": "SELECT", - "Original": "select a, b, c, d, count(*) from user group by 3, 2, 1, 4 order by d, b, a, c", + "Original": "select col1, min(distinct col2) from user group by col1", "Instructions": { "OperatorType": "Aggregate", "Variant": "Ordered", - "Aggregates": "count(4)", + "Aggregates": "min(1)", "Distinct": "false", - "GroupBy": "2, 1, 0, 3", + "GroupBy": "0", "Inputs": [ { "OperatorType": "Route", @@ -978,25 +1161,22 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select a, b, c, d, count(*) from user where 1 != 1 group by 3, 2, 1, 4", - "Query": "select a, b, c, d, count(*) from user group by 3, 2, 1, 4 order by d asc, b asc, a asc, c asc", + "FieldQuery": "select col1, min(DISTINCT col2) from user where 1 != 1 group by col1", + "Query": "select col1, min(DISTINCT col2) from user group by col1 order by col1 asc", "Table": "user" } ] } } - -# scatter aggregate with some descending order by cols -"select a, b, c, count(*) from user group by 3, 2, 1 order by 1 desc, 3 desc, b" { "QueryType": "SELECT", - "Original": "select a, b, c, count(*) from user group by 3, 2, 1 order by 1 desc, 3 desc, b", + "Original": "select col1, min(distinct col2) from user group by col1", "Instructions": { "OperatorType": "Aggregate", "Variant": "Ordered", - "Aggregates": "count(3)", - "Distinct": "false", - "GroupBy": "2, 1, 0", + "Aggregates": "min(1|3) AS min(distinct col2)", + "GroupBy": "(0|2)", + "ResultColumns": 2, "Inputs": [ { "OperatorType": "Route", @@ -1005,28 +1185,30 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select a, b, c, count(*) from user where 1 != 1 group by 3, 2, 1", - "Query": "select a, b, c, count(*) from user group by 3, 2, 1 order by 1 desc, 3 desc, b asc", + "FieldQuery": "select col1, col2, weight_string(col1), weight_string(col2) from user where 1 != 1 group by col1, weight_string(col1), col2, weight_string(col2)", + "OrderBy": "(0|2) ASC, (1|3) ASC", + "Query": "select col1, col2, weight_string(col1), weight_string(col2) from user group by col1, weight_string(col1), col2, weight_string(col2) order by col1 asc, col2 asc", "Table": "user" } ] } } -# aggregate with limit -"select col, count(*) from user group by col limit 10" +# order by count distinct +"select col1, count(distinct col2) k from user group by col1 order by k" { "QueryType": "SELECT", - "Original": "select col, count(*) from user group by col limit 10", + "Original": "select col1, count(distinct col2) k from user group by col1 order by k", "Instructions": { - "OperatorType": "Limit", - "Count": 10, + "OperatorType": "Sort", + "Variant": "Memory", + "OrderBy": "1 ASC", "Inputs": [ { "OperatorType": "Aggregate", "Variant": "Ordered", - "Aggregates": "count(1)", - "Distinct": "false", + "Aggregates": "count_distinct(1) AS k", + "Distinct": "true", "GroupBy": "0", "Inputs": [ { @@ -1036,9 +1218,2926 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select col, count(*) from user where 1 != 1 group by col", - "Query": "select col, count(*) from user group by col order by col asc limit :__upper_limit", - "Table": "user" + "FieldQuery": "select col1, col2 from user where 1 != 1 group by col1, col2", + "Query": "select col1, col2 from user group by col1, col2 order by col1 asc, col2 asc", + "Table": "user" + } + ] + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select col1, count(distinct col2) k from user group by col1 order by k", + "Instructions": { + "OperatorType": "Sort", + "Variant": "Memory", + "OrderBy": "1 ASC", + "ResultColumns": 2, + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "count_distinct(1|3) AS k", + "GroupBy": "(0|2)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col1, col2, weight_string(col1), weight_string(col2) from user where 1 != 1 group by col1, weight_string(col1), col2, weight_string(col2)", + "OrderBy": "(0|2) ASC, (1|3) ASC", + "Query": "select col1, col2, weight_string(col1), weight_string(col2) from user group by col1, weight_string(col1), col2, weight_string(col2) order by col1 asc, col2 asc", + "Table": "user" + } + ] + } + ] + } +} + +# scatter aggregate group by aggregate function +" select count(*) b from user group by b" +"group by expression cannot reference an aggregate function: b" +Gen4 error: Can't group on 'count(*)' + +# scatter aggregate multiple group by (columns) +"select a, b, count(*) from user group by b, a" +{ + "QueryType": "SELECT", + "Original": "select a, b, count(*) from user group by b, a", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "count(2)", + "Distinct": "false", + "GroupBy": "1, 0", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select a, b, count(*) from user where 1 != 1 group by b, a", + "Query": "select a, b, count(*) from user group by b, a order by b asc, a asc", + "Table": "user" + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select a, b, count(*) from user group by b, a", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count_star(2) AS count(*)", + "GroupBy": "(0|3), (1|4)", + "ResultColumns": 3, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select a, b, count(*), weight_string(a), weight_string(b) from user where 1 != 1 group by a, weight_string(a), b, weight_string(b)", + "OrderBy": "(0|3) ASC, (1|4) ASC", + "Query": "select a, b, count(*), weight_string(a), weight_string(b) from user group by a, weight_string(a), b, weight_string(b) order by a asc, b asc", + "Table": "user" + } + ] + } +} + +# scatter aggregate multiple group by (numbers) +"select a, b, count(*) from user group by 2, 1" +{ + "QueryType": "SELECT", + "Original": "select a, b, count(*) from user group by 2, 1", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "count(2)", + "Distinct": "false", + "GroupBy": "1, 0", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select a, b, count(*) from user where 1 != 1 group by 2, 1", + "Query": "select a, b, count(*) from user group by 2, 1 order by b asc, a asc", + "Table": "user" + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select a, b, count(*) from user group by 2, 1", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count_star(2) AS count(*)", + "GroupBy": "(0|3), (1|4)", + "ResultColumns": 3, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select a, b, count(*), weight_string(a), weight_string(b) from user where 1 != 1 group by a, weight_string(a), b, weight_string(b)", + "OrderBy": "(0|3) ASC, (1|4) ASC", + "Query": "select a, b, count(*), weight_string(a), weight_string(b) from user group by a, weight_string(a), b, weight_string(b) order by a asc, b asc", + "Table": "user" + } + ] + } +} + +# scatter aggregate group by column number +"select col from user group by 1" +{ + "QueryType": "SELECT", + "Original": "select col from user group by 1", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Distinct": "false", + "GroupBy": "0", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col from user where 1 != 1 group by 1", + "Query": "select col from user group by 1 order by col asc", + "Table": "user" + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select col from user group by 1", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "GroupBy": "0", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col from user where 1 != 1 group by col", + "OrderBy": "0 ASC", + "Query": "select col from user group by col order by col asc", + "Table": "user" + } + ] + } +} + +# scatter aggregate group by invalid column number +"select col from user group by 2" +"column number out of range: 2" +Gen4 error: Unknown column '2' in 'group statement' + +# scatter aggregate order by null +"select count(*) from user order by null" +{ + "QueryType": "SELECT", + "Original": "select count(*) from user order by null", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "count(0)", + "Distinct": "false", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select count(*) from user where 1 != 1", + "Query": "select count(*) from user", + "Table": "user" + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select count(*) from user order by null", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Scalar", + "Aggregates": "sum_count_star(0) AS count(*)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select count(*) from user where 1 != 1", + "Query": "select count(*) from user", + "Table": "user" + } + ] + } +} + +# scatter aggregate with complex select list (can't build order by) +"select distinct a+1 from user" +"generating order by clause: cannot reference a complex expression" +Gen4 error: unsupported: in scatter query: complex order by expression: a + 1 + +# scatter aggregate with numbered order by columns +"select a, b, c, d, count(*) from user group by 1, 2, 3 order by 1, 2, 3" +{ + "QueryType": "SELECT", + "Original": "select a, b, c, d, count(*) from user group by 1, 2, 3 order by 1, 2, 3", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "count(4)", + "Distinct": "false", + "GroupBy": "0, 1, 2", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select a, b, c, d, count(*) from user where 1 != 1 group by 1, 2, 3", + "Query": "select a, b, c, d, count(*) from user group by 1, 2, 3 order by 1 asc, 2 asc, 3 asc", + "Table": "user" + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select a, b, c, d, count(*) from user group by 1, 2, 3 order by 1, 2, 3", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "random(3) AS d, sum_count_star(4) AS count(*)", + "GroupBy": "(0|5), (1|6), (2|7)", + "ResultColumns": 5, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select a, b, c, d, count(*), weight_string(a), weight_string(b), weight_string(c) from user where 1 != 1 group by a, weight_string(a), b, weight_string(b), c, weight_string(c)", + "OrderBy": "(0|5) ASC, (1|6) ASC, (2|7) ASC", + "Query": "select a, b, c, d, count(*), weight_string(a), weight_string(b), weight_string(c) from user group by a, weight_string(a), b, weight_string(b), c, weight_string(c) order by a asc, b asc, c asc", + "Table": "user" + } + ] + } +} + +# scatter aggregate with named order by columns +"select a, b, c, d, count(*) from user group by 1, 2, 3 order by a, b, c" +{ + "QueryType": "SELECT", + "Original": "select a, b, c, d, count(*) from user group by 1, 2, 3 order by a, b, c", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "count(4)", + "Distinct": "false", + "GroupBy": "0, 1, 2", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select a, b, c, d, count(*) from user where 1 != 1 group by 1, 2, 3", + "Query": "select a, b, c, d, count(*) from user group by 1, 2, 3 order by a asc, b asc, c asc", + "Table": "user" + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select a, b, c, d, count(*) from user group by 1, 2, 3 order by a, b, c", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "random(3) AS d, sum_count_star(4) AS count(*)", + "GroupBy": "(0|5), (1|6), (2|7)", + "ResultColumns": 5, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select a, b, c, d, count(*), weight_string(a), weight_string(b), weight_string(c) from user where 1 != 1 group by a, weight_string(a), b, weight_string(b), c, weight_string(c)", + "OrderBy": "(0|5) ASC, (1|6) ASC, (2|7) ASC", + "Query": "select a, b, c, d, count(*), weight_string(a), weight_string(b), weight_string(c) from user group by a, weight_string(a), b, weight_string(b), c, weight_string(c) order by a asc, b asc, c asc", + "Table": "user" + } + ] + } +} + +# scatter aggregate with jumbled order by columns +"select a, b, c, d, count(*) from user group by 1, 2, 3, 4 order by d, b, a, c" +{ + "QueryType": "SELECT", + "Original": "select a, b, c, d, count(*) from user group by 1, 2, 3, 4 order by d, b, a, c", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "count(4)", + "Distinct": "false", + "GroupBy": "0, 1, 2, 3", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select a, b, c, d, count(*) from user where 1 != 1 group by 1, 2, 3, 4", + "Query": "select a, b, c, d, count(*) from user group by 1, 2, 3, 4 order by d asc, b asc, a asc, c asc", + "Table": "user" + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select a, b, c, d, count(*) from user group by 1, 2, 3, 4 order by d, b, a, c", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count_star(4) AS count(*)", + "GroupBy": "(3|8), (1|6), (0|5), (2|7)", + "ResultColumns": 5, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select a, b, c, d, count(*), weight_string(a), weight_string(b), weight_string(c), weight_string(d) from user where 1 != 1 group by a, weight_string(a), b, weight_string(b), c, weight_string(c), d, weight_string(d)", + "OrderBy": "(3|8) ASC, (1|6) ASC, (0|5) ASC, (2|7) ASC", + "Query": "select a, b, c, d, count(*), weight_string(a), weight_string(b), weight_string(c), weight_string(d) from user group by a, weight_string(a), b, weight_string(b), c, weight_string(c), d, weight_string(d) order by d asc, b asc, a asc, c asc", + "Table": "user" + } + ] + } +} + +# scatter aggregate with jumbled group by and order by columns +"select a, b, c, d, count(*) from user group by 3, 2, 1, 4 order by d, b, a, c" +{ + "QueryType": "SELECT", + "Original": "select a, b, c, d, count(*) from user group by 3, 2, 1, 4 order by d, b, a, c", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "count(4)", + "Distinct": "false", + "GroupBy": "2, 1, 0, 3", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select a, b, c, d, count(*) from user where 1 != 1 group by 3, 2, 1, 4", + "Query": "select a, b, c, d, count(*) from user group by 3, 2, 1, 4 order by d asc, b asc, a asc, c asc", + "Table": "user" + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select a, b, c, d, count(*) from user group by 3, 2, 1, 4 order by d, b, a, c", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count_star(4) AS count(*)", + "GroupBy": "(3|8), (1|6), (0|5), (2|7)", + "ResultColumns": 5, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select a, b, c, d, count(*), weight_string(a), weight_string(b), weight_string(c), weight_string(d) from user where 1 != 1 group by a, weight_string(a), b, weight_string(b), c, weight_string(c), d, weight_string(d)", + "OrderBy": "(3|8) ASC, (1|6) ASC, (0|5) ASC, (2|7) ASC", + "Query": "select a, b, c, d, count(*), weight_string(a), weight_string(b), weight_string(c), weight_string(d) from user group by a, weight_string(a), b, weight_string(b), c, weight_string(c), d, weight_string(d) order by d asc, b asc, a asc, c asc", + "Table": "user" + } + ] + } +} + +# scatter aggregate with some descending order by cols +"select a, b, c, count(*) from user group by 3, 2, 1 order by 1 desc, 3 desc, b" +{ + "QueryType": "SELECT", + "Original": "select a, b, c, count(*) from user group by 3, 2, 1 order by 1 desc, 3 desc, b", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "count(3)", + "Distinct": "false", + "GroupBy": "2, 1, 0", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select a, b, c, count(*) from user where 1 != 1 group by 3, 2, 1", + "Query": "select a, b, c, count(*) from user group by 3, 2, 1 order by 1 desc, 3 desc, b asc", + "Table": "user" + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select a, b, c, count(*) from user group by 3, 2, 1 order by 1 desc, 3 desc, b", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count_star(3) AS count(*)", + "GroupBy": "(0|4), (2|6), (1|5)", + "ResultColumns": 4, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select a, b, c, count(*), weight_string(a), weight_string(b), weight_string(c) from user where 1 != 1 group by a, weight_string(a), b, weight_string(b), c, weight_string(c)", + "OrderBy": "(0|4) DESC, (2|6) DESC, (1|5) ASC", + "Query": "select a, b, c, count(*), weight_string(a), weight_string(b), weight_string(c) from user group by a, weight_string(a), b, weight_string(b), c, weight_string(c) order by a desc, c desc, b asc", + "Table": "user" + } + ] + } +} + +# aggregate with limit +"select col, count(*) from user group by col limit 10" +{ + "QueryType": "SELECT", + "Original": "select col, count(*) from user group by col limit 10", + "Instructions": { + "OperatorType": "Limit", + "Count": 10, + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "count(1)", + "Distinct": "false", + "GroupBy": "0", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col, count(*) from user where 1 != 1 group by col", + "Query": "select col, count(*) from user group by col order by col asc limit :__upper_limit", + "Table": "user" + } + ] + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select col, count(*) from user group by col limit 10", + "Instructions": { + "OperatorType": "Limit", + "Count": "10", + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count_star(1) AS count(*)", + "GroupBy": "0", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col, count(*) from user where 1 != 1 group by col", + "OrderBy": "0 ASC", + "Query": "select col, count(*) from user group by col order by col asc limit :__upper_limit", + "Table": "user" + } + ] + } + ] + } +} + +# invalid order by column numner for scatter +"select col, count(*) from user group by col order by 5 limit 10" +"column number out of range: 5" +Gen4 error: Unknown column '5' in 'order clause' + +# Group by with collate operator +"select user.col1 as a from user where user.name = 5 group by a collate utf8_general_ci" +{ + "QueryType": "SELECT", + "Original": "select user.col1 as a from user where user.name = 5 group by a collate utf8_general_ci", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.col1 as a from user where 1 != 1 group by a collate utf8_general_ci", + "Query": "select user.col1 as a from user where user.name = 5 group by a collate utf8_general_ci", + "Table": "user", + "Values": [ + "INT64(5)" + ], + "Vindex": "user_index" + } +} +Gen4 plan same as above + +# Group by invalid column number (code is duplicated from symab). +"select id from user group by 1.1" +"column number is not an int" +{ + "QueryType": "SELECT", + "Original": "select id from user group by 1.1", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "random(0) AS id", + "GroupBy": "1", + "ResultColumns": 1, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id, 1.1 from user where 1 != 1 group by 1.1", + "OrderBy": "1 ASC", + "Query": "select id, 1.1 from user group by 1.1 order by 1.1 asc", + "Table": "user" + } + ] + } +} + +# Group by out of range column number (code is duplicated from symab). +"select id from user group by 2" +"column number out of range: 2" +Gen4 error: Unknown column '2' in 'group statement' + +# syntax error detected by planbuilder +"select count(distinct *) from user" +"syntax error: count(distinct *)" +Gen4 error: unsupported: in scatter query: complex order by expression: * + +# scatter aggregate in a subquery +"select a from (select count(*) as a from user) t" +{ + "QueryType": "SELECT", + "Original": "select a from (select count(*) as a from user) t", + "Instructions": { + "OperatorType": "Subquery", + "Columns": [ + 0 + ], + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "count(0)", + "Distinct": "false", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select count(*) as a from user where 1 != 1", + "Query": "select count(*) as a from user", + "Table": "user" + } + ] + } + ] + } +} + +# scatter avg +"select avg(costly) from user" +{ + "QueryType": "SELECT", + "Original": "select avg(costly) from user", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "avg_sum(0) AS avg(costly);avg_count(1)", + "Distinct": "false", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select sum(costly), count(costly) from user where 1 != 1", + "Query": "select sum(costly), count(costly) from user", + "Table": "user" + } + ] + } +} + +# scatter avg group by non-vindex column +"select id, avg(costly) from user group by id" +{ + "QueryType": "SELECT", + "Original": "select id, avg(costly) from user group by id", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "avg_sum(1) AS avg(costly);avg_count(2)", + "Distinct": "false", + "GroupBy": "0", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id, sum(costly), count(costly) from user where 1 != 1 group by id", + "Query": "select id, sum(costly), count(costly) from user group by id order by id asc", + "Table": "user" + } + ] + } +} + +# scatter avg group by non-vindex column order by avg_cost +"select id, avg(costly) as avg_cost from user group by id order by avg_cost" +{ + "QueryType": "SELECT", + "Original": "select id, avg(costly) as avg_cost from user group by id order by avg_cost", + "Instructions": { + "OperatorType": "Sort", + "Variant": "Memory", + "OrderBy": "1 ASC", + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "avg_sum(1) AS avg_cost;avg_count(2)", + "Distinct": "false", + "GroupBy": "0", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id, sum(costly), count(costly) from user where 1 != 1 group by id", + "Query": "select id, sum(costly), count(costly) from user group by id order by id asc", + "Table": "user" + } + ] + } + ] + } +} + +# scatter avg group by vindex column +"select name, avg(costly) from user group by name" +{ + "QueryType": "SELECT", + "Original": "select name, avg(costly) from user group by name", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select name, avg(costly) from user where 1 != 1 group by name", + "Query": "select name, avg(costly) from user group by name", + "Table": "user" + } +} + +# Aggregate on join +"select user.a, count(*) from user join user_extra group by user.a" +"unsupported: cross-shard query with aggregates" +{ + "QueryType": "SELECT", + "Original": "select user.a, count(*) from user join user_extra group by user.a", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count_star(1) AS count(*)", + "GroupBy": "(0|2)", + "ResultColumns": 2, + "Inputs": [ + { + "OperatorType": "Projection", + "Expressions": [ + "[COLUMN 0] as a", + "[COLUMN 2] * [COLUMN 3] as count(*)", + "[COLUMN 1]" + ], + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:1,L:2,L:0,R:1", + "TableName": "`user`_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select count(*), `user`.a, weight_string(`user`.a) from user where 1 != 1 group by `user`.a, weight_string(`user`.a)", + "OrderBy": "(1|2) ASC", + "Query": "select count(*), `user`.a, weight_string(`user`.a) from user group by `user`.a, weight_string(`user`.a) order by `user`.a asc", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1, count(*) from user_extra where 1 != 1 group by 1", + "Query": "select 1, count(*) from user_extra group by 1", + "Table": "user_extra" + } + ] + } + ] + } + ] + } +} + +# Aggregate on other table in join +"select user.a, count(user_extra.a) from user join user_extra group by user.a" +"unsupported: cross-shard query with aggregates" +{ + "QueryType": "SELECT", + "Original": "select user.a, count(user_extra.a) from user join user_extra group by user.a", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count(1) AS count(user_extra.a)", + "GroupBy": "(0|2)", + "ResultColumns": 2, + "Inputs": [ + { + "OperatorType": "Projection", + "Expressions": [ + "[COLUMN 0] as a", + "[COLUMN 2] * [COLUMN 3] as count(user_extra.a)", + "[COLUMN 1]" + ], + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:1,L:2,L:0,R:1", + "TableName": "`user`_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select count(*), `user`.a, weight_string(`user`.a) from user where 1 != 1 group by `user`.a, weight_string(`user`.a)", + "OrderBy": "(1|2) ASC", + "Query": "select count(*), `user`.a, weight_string(`user`.a) from user group by `user`.a, weight_string(`user`.a) order by `user`.a asc", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1, count(user_extra.a) from user_extra where 1 != 1 group by 1", + "Query": "select 1, count(user_extra.a) from user_extra group by 1", + "Table": "user_extra" + } + ] + } + ] + } + ] + } +} + +# Aggregates and joins +"select count(*) from user join user_extra" +"unsupported: cross-shard query with aggregates" +{ + "QueryType": "SELECT", + "Original": "select count(*) from user join user_extra", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Scalar", + "Aggregates": "sum_count_star(0) AS count(*)", + "Inputs": [ + { + "OperatorType": "Projection", + "Expressions": [ + "[COLUMN 0] * [COLUMN 1] as count(*)" + ], + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:0,R:1", + "TableName": "`user`_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select count(*) from user where 1 != 1", + "Query": "select count(*) from user", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1, count(*) from user_extra where 1 != 1 group by 1", + "Query": "select 1, count(*) from user_extra group by 1", + "Table": "user_extra" + } + ] + } + ] + } + ] + } +} + +# Grouping on join +"select user.a from user join user_extra group by user.a" +"unsupported: cross-shard query with aggregates" +{ + "QueryType": "SELECT", + "Original": "select user.a from user join user_extra group by user.a", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "GroupBy": "(0|1)", + "ResultColumns": 1, + "Inputs": [ + { + "OperatorType": "Projection", + "Expressions": [ + "[COLUMN 0] as a", + "[COLUMN 1]" + ], + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:0,L:1", + "TableName": "`user`_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select `user`.a, weight_string(`user`.a) from user where 1 != 1 group by `user`.a, weight_string(`user`.a)", + "OrderBy": "(0|1) ASC", + "Query": "select `user`.a, weight_string(`user`.a) from user group by `user`.a, weight_string(`user`.a) order by `user`.a asc", + "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", + "Table": "user_extra" + } + ] + } + ] + } + ] + } +} + +# aggregate query with order by NULL +"select col, count(*) k from user group by col order by null" +{ + "QueryType": "SELECT", + "Original": "select col, count(*) k from user group by col order by null", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "count(1)", + "Distinct":false, + "GroupBy": "0", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col, count(*) as k from user where 1 != 1 group by col", + "Query": "select col, count(*) as k from user group by col order by col asc", + "Table": "user" + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select col, count(*) k from user group by col order by null", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count_star(1) AS k", + "GroupBy": "0", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col, count(*) as k from user where 1 != 1 group by col", + "OrderBy": "0 ASC", + "Query": "select col, count(*) as k from user group by col order by col asc", + "Table": "user" + } + ] + } +} + +# aggregate query with order by aggregate column along with NULL +"select col, count(*) k from user group by col order by null, k" +"column number is not an int" +{ + "QueryType": "SELECT", + "Original": "select col, count(*) k from user group by col order by null, k", + "Instructions": { + "OperatorType": "Sort", + "Variant": "Memory", + "OrderBy": "1 ASC", + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count_star(1) AS k", + "GroupBy": "0", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col, count(*) as k from user where 1 != 1 group by col", + "OrderBy": "0 ASC", + "Query": "select col, count(*) as k from user group by col order by col asc", + "Table": "user" + } + ] + } + ] + } +} + +# aggregation on top of semijoin +"select count(*) from user where exists (select 0 from user_extra where user.apa = user_extra.bar)" +"unsupported: cross-shard correlated subquery" +{ + "QueryType": "SELECT", + "Original": "select count(*) from user where exists (select 0 from user_extra where user.apa = user_extra.bar)", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Scalar", + "Aggregates": "sum_count_star(0) AS count(*)", + "Inputs": [ + { + "OperatorType": "Projection", + "Expressions": [ + "[COLUMN 1] as count(*)" + ], + "Inputs": [ + { + "OperatorType": "SemiJoin", + "JoinVars": { + "user_apa": 0 + }, + "TableName": "`user`_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select `user`.apa, count(*), weight_string(`user`.apa) from user where 1 != 1 group by `user`.apa, weight_string(`user`.apa)", + "Query": "select `user`.apa, count(*), weight_string(`user`.apa) from user group by `user`.apa, weight_string(`user`.apa)", + "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.bar = :user_apa", + "Table": "user_extra" + } + ] + } + ] + } + ] + } +} + +# aggregation spread out across three routes +"select count(u.textcol1), count(ue.foo), us.bar from user u join user_extra ue on u.foo = ue.bar join unsharded us on ue.bar = us.baz group by us.bar" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select count(u.textcol1), count(ue.foo), us.bar from user u join user_extra ue on u.foo = ue.bar join unsharded us on ue.bar = us.baz group by us.bar", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count(0) AS count(u.textcol1), sum_count(1) AS count(ue.foo)", + "GroupBy": "(2|3)", + "ResultColumns": 3, + "Inputs": [ + { + "OperatorType": "Projection", + "Expressions": [ + "([COLUMN 2] * [COLUMN 3]) * [COLUMN 4] as count(u.textcol1)", + "([COLUMN 5] * [COLUMN 6]) * [COLUMN 7] as count(ue.foo)", + "[COLUMN 0] as bar", + "[COLUMN 1]" + ], + "Inputs": [ + { + "OperatorType": "Sort", + "Variant": "Memory", + "OrderBy": "(0|1) ASC", + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "R:0,R:1,L:1,R:2,R:3,L:2,R:4,R:5", + "JoinVars": { + "u_foo": 0 + }, + "TableName": "`user`_user_extra_unsharded", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select u.foo, count(u.textcol1), count(*), weight_string(u.foo) from user as u where 1 != 1 group by u.foo, weight_string(u.foo)", + "Query": "select u.foo, count(u.textcol1), count(*), weight_string(u.foo) from user as u group by u.foo, weight_string(u.foo)", + "Table": "user" + }, + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "R:1,R:2,L:1,R:0,L:2,R:0", + "JoinVars": { + "ue_bar": 0 + }, + "TableName": "user_extra_unsharded", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select ue.bar, count(*), count(ue.foo), weight_string(ue.bar) from user_extra as ue where 1 != 1 group by ue.bar, weight_string(ue.bar)", + "Query": "select ue.bar, count(*), count(ue.foo), weight_string(ue.bar) from user_extra as ue where ue.bar = :u_foo group by ue.bar, weight_string(ue.bar)", + "Table": "user_extra" + }, + { + "OperatorType": "Route", + "Variant": "Unsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select count(*), us.bar, weight_string(us.bar) from unsharded as us where 1 != 1 group by us.bar, weight_string(us.bar)", + "Query": "select count(*), us.bar, weight_string(us.bar) from unsharded as us where us.baz = :ue_bar group by us.bar, weight_string(us.bar)", + "Table": "unsharded" + } + ] + } + ] + } + ] + } + ] + } + ] + } +} + +# correlated subquery in exists clause with an ordering +"select col, id from user where exists(select user_id from user_extra where user_id = 3 and user_id < user.id) order by id" +"unsupported: cross-shard correlated subquery" +{ + "QueryType": "SELECT", + "Original": "select col, id from user where exists(select user_id from user_extra where user_id = 3 and user_id \u003c user.id) order by id", + "Instructions": { + "OperatorType": "SemiJoin", + "JoinVars": { + "user_id": 0 + }, + "ProjectedIndexes": "-2,-1", + "TableName": "`user`_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select `user`.id, col, weight_string(id) from user where 1 != 1", + "OrderBy": "(0|2) ASC", + "Query": "select `user`.id, col, weight_string(id) from user order by id asc", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from user_extra where 1 != 1", + "Query": "select 1 from user_extra where user_id = 3 and user_id \u003c :user_id", + "Table": "user_extra", + "Values": [ + "INT64(3)" + ], + "Vindex": "user_index" + } + ] + } +} + +# correlated subquery on sharding key with group by a unique vindex with having clause. +"select count(*) from user where exists (select 1 from user_extra where user_id = user.id group by user_id having max(col) > 10)" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select count(*) from user where exists (select 1 from user_extra where user_id = user.id group by user_id having max(col) \u003e 10)", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Scalar", + "Aggregates": "sum_count_star(0) AS count(*)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select count(*) from user where 1 != 1", + "Query": "select count(*) from user where exists (select 1 from user_extra where user_id = `user`.id group by user_id having max(col) \u003e 10 limit 1)", + "Table": "user" + } + ] + } +} + +# count non-null columns incoming from outer joins should work well +"select count(col) from (select user_extra.col as col from user left join user_extra on user.id = user_extra.id limit 10) as x" +"unsupported: cross-shard query with aggregates" +{ + "QueryType": "SELECT", + "Original": "select count(col) from (select user_extra.col as col from user left join user_extra on user.id = user_extra.id limit 10) as x", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Scalar", + "Aggregates": "count(0) AS count(col)", + "Inputs": [ + { + "OperatorType": "Projection", + "Expressions": [ + "[COLUMN 0] as count(col)" + ], + "Inputs": [ + { + "OperatorType": "Limit", + "Count": "INT64(10)", + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "LeftJoin", + "JoinColumnIndexes": "R:0", + "JoinVars": { + "user_id": 0 + }, + "TableName": "`user`_user_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": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user_extra.col as col from user_extra where 1 != 1", + "Query": "select user_extra.col as col from user_extra where user_extra.id = :user_id", + "Table": "user_extra" + } + ] + } + ] + } + ] + } + ] + } +} + +# count on column from LIMIT +"select count(city) from (select phone, id, city from user where id > 12 limit 10) as x" +"unsupported: cross-shard query with aggregates" +{ + "QueryType": "SELECT", + "Original": "select count(city) from (select phone, id, city from user where id \u003e 12 limit 10) as x", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Scalar", + "Aggregates": "count(0) AS count(city)", + "Inputs": [ + { + "OperatorType": "Projection", + "Expressions": [ + "[COLUMN 2] as count(city)" + ], + "Inputs": [ + { + "OperatorType": "Limit", + "Count": "INT64(10)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select phone, id, city from user where 1 != 1", + "Query": "select phone, id, city from user where id \u003e 12 limit :__upper_limit", + "Table": "user" + } + ] + } + ] + } + ] + } +} + +# count(*) on column from LIMIT +"select count(*) from (select phone, id, city from user where id > 12 limit 10) as x" +"unsupported: cross-shard query with aggregates" +{ + "QueryType": "SELECT", + "Original": "select count(*) from (select phone, id, city from user where id \u003e 12 limit 10) as x", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Scalar", + "Aggregates": "count_star(0) AS count(*)", + "Inputs": [ + { + "OperatorType": "Projection", + "Expressions": [ + "[COLUMN 0] as count(*)" + ], + "Inputs": [ + { + "OperatorType": "Limit", + "Count": "INT64(10)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select phone, id, city from user where 1 != 1", + "Query": "select phone, id, city from user where id \u003e 12 limit :__upper_limit", + "Table": "user" + } + ] + } + ] + } + ] + } +} + +# distinct aggregation will 3 table join query +"select u.textcol1, count(distinct u.val2) from user u join user u2 on u.val2 = u2.id join music m on u2.val2 = m.id group by u.textcol1" +"unsupported: cross-shard query with aggregates" +{ + "QueryType": "SELECT", + "Original": "select u.textcol1, count(distinct u.val2) from user u join user u2 on u.val2 = u2.id join music m on u2.val2 = m.id group by u.textcol1", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "count_distinct(1|2) AS count(distinct u.val2)", + "GroupBy": "0 COLLATE latin1_swedish_ci", + "ResultColumns": 2, + "Inputs": [ + { + "OperatorType": "Projection", + "Expressions": [ + "[COLUMN 0] as textcol1", + "[COLUMN 1] as val2", + "[COLUMN 2]" + ], + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:2,L:3,L:5", + "JoinVars": { + "u2_val2": 0 + }, + "TableName": "`user`_`user`_music", + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "R:0,R:0,L:2,L:0,R:1,L:1", + "JoinVars": { + "u_val2": 0 + }, + "TableName": "`user`_`user`", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select u.val2, weight_string(u.val2), u.textcol1 from user as u where 1 != 1 group by u.val2, weight_string(u.val2), u.textcol1", + "OrderBy": "2 ASC COLLATE latin1_swedish_ci, (0|1) ASC", + "Query": "select u.val2, weight_string(u.val2), u.textcol1 from user as u group by u.val2, weight_string(u.val2), u.textcol1 order by u.textcol1 asc, u.val2 asc", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select u2.val2, weight_string(u2.val2) from user as u2 where 1 != 1 group by u2.val2, weight_string(u2.val2)", + "Query": "select u2.val2, weight_string(u2.val2) from user as u2 where u2.id = :u_val2 group by u2.val2, weight_string(u2.val2)", + "Table": "user", + "Values": [ + ":u_val2" + ], + "Vindex": "user_index" + } + ] + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from music as m where 1 != 1", + "Query": "select 1 from music as m where m.id = :u2_val2", + "Table": "music", + "Values": [ + ":u2_val2" + ], + "Vindex": "music_user_map" + } + ] + } + ] + } + ] + } +} + +# distinct on text column with collation +"select col, count(distinct textcol1) from user group by col" +{ + "QueryType": "SELECT", + "Original": "select col, count(distinct textcol1) from user group by col", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "count_distinct(1) AS count(distinct textcol1)", + "Distinct":true, + "GroupBy": "0", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col, textcol1, weight_string(textcol1) from user where 1 != 1 group by col, textcol1", + "Query": "select col, textcol1, weight_string(textcol1) from user group by col, textcol1 order by col asc, textcol1 asc", + "Table": "user" + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select col, count(distinct textcol1) from user group by col", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "count_distinct(1) AS count(distinct textcol1)", + "GroupBy": "0", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col, textcol1 from user where 1 != 1 group by col, textcol1", + "OrderBy": "0 ASC, 1 ASC", + "Query": "select col, textcol1 from user group by col, textcol1 order by col asc, textcol1 asc", + "Table": "user" + } + ] + } +} + +# do not use distinct when using only aggregates and no group by +"select distinct count(*) from user" +"unsupported: distinct cannot be combined with aggregate functions" +{ + "QueryType": "SELECT", + "Original": "select distinct count(*) from user", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Scalar", + "Aggregates": "sum_count_star(0) AS count(*)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select count(*) from user where 1 != 1", + "Query": "select count(*) from user", + "Table": "user" + } + ] + } +} + +# group by a unique vindex should revert to simple route, and having clause should find the correct symbols. +"select name, count(*) c from user group by name having max(col) > 10" +{ + "QueryType": "SELECT", + "Original": "select name, count(*) c from user group by name having max(col) \u003e 10", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select name, count(*) as c from user where 1 != 1 group by name", + "Query": "select name, count(*) as c from user group by name having max(col) \u003e 10", + "Table": "user" + } +} +Gen4 plan same as above + +# group by and ',' joins +"select user.id from user, user_extra group by id" +"unsupported: cross-shard query with aggregates" +{ + "QueryType": "SELECT", + "Original": "select user.id from user, user_extra group by id", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "random(0) AS id", + "GroupBy": "(2|1)", + "ResultColumns": 1, + "Inputs": [ + { + "OperatorType": "Projection", + "Expressions": [ + "[COLUMN 2] * [COLUMN 3] as id", + "[COLUMN 1]", + "[COLUMN 0] as id" + ], + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:0,L:1,L:0,R:1", + "TableName": "`user`_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select `user`.id, weight_string(id) from user where 1 != 1 group by id, weight_string(id)", + "OrderBy": "(0|1) ASC", + "Query": "select `user`.id, weight_string(id) from user group by id, weight_string(id) order by id asc", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1, count(*) from user_extra where 1 != 1 group by 1", + "Query": "select 1, count(*) from user_extra group by 1", + "Table": "user_extra" + } + ] + } + ] + } + ] + } +} + +# group by and ',' joins with condition +"select user.col from user join user_extra on user_extra.col = user.col group by user.id" +"unsupported: cross-shard query with aggregates" +{ + "QueryType": "SELECT", + "Original": "select user.col from user join user_extra on user_extra.col = user.col group by user.id", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "random(0) AS col", + "GroupBy": "(2|1)", + "ResultColumns": 1, + "Inputs": [ + { + "OperatorType": "Projection", + "Expressions": [ + "[COLUMN 2] * [COLUMN 3] as col", + "[COLUMN 1]", + "[COLUMN 0] as id" + ], + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:1,L:2,L:0,R:1", + "JoinVars": { + "user_col": 0 + }, + "TableName": "`user`_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select `user`.col, `user`.id, weight_string(`user`.id) from user where 1 != 1 group by `user`.col, `user`.id, weight_string(`user`.id)", + "OrderBy": "(1|2) ASC", + "Query": "select `user`.col, `user`.id, weight_string(`user`.id) from user group by `user`.col, `user`.id, weight_string(`user`.id) order by `user`.id asc", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1, count(*) from user_extra where 1 != 1 group by 1", + "Query": "select 1, count(*) from user_extra where user_extra.col = :user_col group by 1", + "Table": "user_extra" + } + ] + } + ] + } + ] + } +} + +# group by column alias +"select ascii(val1) as a, count(*) from user group by a" +{ + "QueryType": "SELECT", + "Original": "select ascii(val1) as a, count(*) from user group by a", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "count(1)", + "Distinct":false, + "GroupBy": "0", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select ascii(val1) as a, count(*) from user where 1 != 1 group by a", + "Query": "select ascii(val1) as a, count(*) from user group by a order by a asc", + "Table": "user" + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select ascii(val1) as a, count(*) from user group by a", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count_star(1) AS count(*)", + "GroupBy": "(0|2)", + "ResultColumns": 2, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select ascii(val1) as a, count(*), weight_string(ascii(val1)) from user where 1 != 1 group by a, weight_string(ascii(val1))", + "OrderBy": "(0|2) ASC", + "Query": "select ascii(val1) as a, count(*), weight_string(ascii(val1)) from user group by a, weight_string(ascii(val1)) order by a asc", + "Table": "user" + } + ] + } +} + +# group by must only reference expressions in the select list +"select col, count(*) from user group by col, baz" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select col, count(*) from user group by col, baz", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count_star(1) AS count(*)", + "GroupBy": "0, (2|3)", + "ResultColumns": 2, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col, count(*), baz, weight_string(baz) from user where 1 != 1 group by col, baz, weight_string(baz)", + "OrderBy": "0 ASC, (2|3) ASC", + "Query": "select col, count(*), baz, weight_string(baz) from user group by col, baz, weight_string(baz) order by col asc, baz asc", + "Table": "user" + } + ] + } +} + +# here it is safe to remove the order by on the derived table since it will not influence the output of the count(*) +"select count(*) from (select user.col, user_extra.extra from user join user_extra on user.id = user_extra.user_id order by user_extra.extra) a" +"unsupported: cross-shard query with aggregates" +{ + "QueryType": "SELECT", + "Original": "select count(*) from (select user.col, user_extra.extra from user join user_extra on user.id = user_extra.user_id order by user_extra.extra) a", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Scalar", + "Aggregates": "sum_count_star(0) AS count(*)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select count(*) from (select `user`.col, user_extra.extra, weight_string(user_extra.extra) from user, user_extra where 1 != 1) as a where 1 != 1", + "OrderBy": "(1|2) ASC", + "Query": "select count(*) from (select `user`.col, user_extra.extra, weight_string(user_extra.extra) from user, user_extra where `user`.id = user_extra.user_id order by user_extra.extra asc) as a", + "Table": "`user`, user_extra" + } + ] + } +} + +# here we keep the order since the column is visible on the outside, and used by the orderedAggregate +"select col, count(*) from (select user.col, user_extra.extra from user join user_extra on user.id = user_extra.user_id order by user_extra.extra) a group by col" +"unsupported: cross-shard query with aggregates" +{ + "QueryType": "SELECT", + "Original": "select col, count(*) from (select user.col, user_extra.extra from user join user_extra on user.id = user_extra.user_id order by user_extra.extra) a group by col", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count_star(1) AS count(*)", + "GroupBy": "0", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col, count(*) from (select `user`.col, user_extra.extra, weight_string(user_extra.extra) from user, user_extra where 1 != 1) as a where 1 != 1 group by col", + "OrderBy": "(1|2) ASC, 0 ASC", + "Query": "select col, count(*) from (select `user`.col, user_extra.extra, weight_string(user_extra.extra) from user, user_extra where `user`.id = user_extra.user_id order by user_extra.extra asc) as a group by col order by col asc", + "Table": "`user`, user_extra" + } + ] + } +} + +# interleaving grouping, aggregation and join +"select user.col, min(user_extra.foo), user.bar, max(user_extra.bar) from user join user_extra on user.col = user_extra.bar group by user.col, user.bar" +"unsupported: cross-shard query with aggregates" +{ + "QueryType": "SELECT", + "Original": "select user.col, min(user_extra.foo), user.bar, max(user_extra.bar) from user join user_extra on user.col = user_extra.bar group by user.col, user.bar", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "min(1) AS min(user_extra.foo), max(3) AS max(user_extra.bar)", + "GroupBy": "0, (2|4)", + "ResultColumns": 4, + "Inputs": [ + { + "OperatorType": "Projection", + "Expressions": [ + "[COLUMN 0] as col", + "[COLUMN 3] as min(user_extra.foo)", + "[COLUMN 1] as bar", + "[COLUMN 4] as max(user_extra.bar)", + "[COLUMN 2]" + ], + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:0,L:1,L:2,R:1,R:2", + "JoinVars": { + "user_col": 0 + }, + "TableName": "`user`_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select `user`.col, `user`.bar, weight_string(`user`.bar) from user where 1 != 1 group by `user`.col, `user`.bar, weight_string(`user`.bar)", + "OrderBy": "0 ASC, (1|2) ASC", + "Query": "select `user`.col, `user`.bar, weight_string(`user`.bar) from user group by `user`.col, `user`.bar, weight_string(`user`.bar) order by `user`.col asc, `user`.bar asc", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1, min(user_extra.foo), max(user_extra.bar) from user_extra where 1 != 1 group by 1", + "Query": "select 1, min(user_extra.foo), max(user_extra.bar) from user_extra where user_extra.bar = :user_col group by 1", + "Table": "user_extra" + } + ] + } + ] + } + ] + } +} + +# join query on sharding key with group by a unique vindex with having clause. +"select user.id, count(*) c from user, user_extra where user.id = user_extra.user_id group by user.id having max(user.col) > 10" +"unsupported: cross-shard query with aggregates" +{ + "QueryType": "SELECT", + "Original": "select user.id, count(*) c from user, user_extra where user.id = user_extra.user_id group by user.id having max(user.col) \u003e 10", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select `user`.id, count(*) as c from user, user_extra where 1 != 1 group by `user`.id", + "Query": "select `user`.id, count(*) as c from user, user_extra where `user`.id = user_extra.user_id group by `user`.id having max(`user`.col) \u003e 10", + "Table": "`user`, user_extra" + } +} + +# only extract the aggregation once, even if used twice +"select u.id from user u join user_extra ue on ue.id = u.id group by u.id having count(*) < 3 and count(*) > 5" +"unsupported: cross-shard query with aggregates" +{ + "QueryType": "SELECT", + "Original": "select u.id from user u join user_extra ue on ue.id = u.id group by u.id having count(*) \u003c 3 and count(*) \u003e 5", + "Instructions": { + "OperatorType": "SimpleProjection", + "Columns": [ + 0 + ], + "Inputs": [ + { + "OperatorType": "Filter", + "Predicate": "OFFSET(1, 'count(*)') \u003c 3 and OFFSET(1, 'count(*)') \u003e 5", + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count_star(1) AS count(*)", + "GroupBy": "(0|2)", + "Inputs": [ + { + "OperatorType": "Projection", + "Expressions": [ + "[COLUMN 0] as id", + "[COLUMN 2] * [COLUMN 3] as count(*)", + "[COLUMN 1]" + ], + "Inputs": [ + { + "OperatorType": "Sort", + "Variant": "Memory", + "OrderBy": "(0|1) ASC", + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "R:1,R:2,L:1,R:0", + "JoinVars": { + "ue_id": 0 + }, + "TableName": "user_extra_`user`", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select ue.id, count(*), weight_string(ue.id) from user_extra as ue where 1 != 1 group by ue.id, weight_string(ue.id)", + "Query": "select ue.id, count(*), weight_string(ue.id) from user_extra as ue group by ue.id, weight_string(ue.id)", + "Table": "user_extra" + }, + { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select count(*), u.id, weight_string(u.id) from user as u where 1 != 1 group by u.id, weight_string(u.id)", + "Query": "select count(*), u.id, weight_string(u.id) from user as u where u.id = :ue_id group by u.id, weight_string(u.id)", + "Table": "user", + "Values": [ + ":ue_id" + ], + "Vindex": "user_index" + } + ] + } + ] + } + ] + } + ] + } + ] + } + ] + } +} + +# optimize group by when using distinct with no aggregation +"select distinct col1, col2 from user group by col1, col2" +{ + "QueryType": "SELECT", + "Original": "select distinct col1, col2 from user group by col1, col2", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Distinct":false, + "GroupBy": "0, 1, 0, 1", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col1, col2 from user where 1 != 1 group by col1, col2", + "Query": "select distinct col1, col2 from user group by col1, col2 order by col1 asc, col2 asc, col1 asc, col2 asc", + "ResultColumns": 2, + "Table": "user" + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select distinct col1, col2 from user group by col1, col2", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "GroupBy": "(0|2), (1|3)", + "ResultColumns": 2, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col1, col2, weight_string(col1), weight_string(col2) from user where 1 != 1 group by col1, col2", + "OrderBy": "(0|2) ASC, (1|3) ASC", + "Query": "select distinct col1, col2, weight_string(col1), weight_string(col2) from user group by col1, col2 order by col1 asc, col2 asc", + "Table": "user" + } + ] + } +} + +# scatter aggregate multiple group by columns inverse order +"select a, b, count(*) from user group by b, a" +{ + "QueryType": "SELECT", + "Original": "select a, b, count(*) from user group by b, a", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "count(2)", + "Distinct":false, + "GroupBy": "1, 0", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select a, b, count(*) from user where 1 != 1 group by b, a", + "Query": "select a, b, count(*) from user group by b, a order by b asc, a asc", + "Table": "user" + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select a, b, count(*) from user group by b, a", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count_star(2) AS count(*)", + "GroupBy": "(0|3), (1|4)", + "ResultColumns": 3, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select a, b, count(*), weight_string(a), weight_string(b) from user where 1 != 1 group by a, weight_string(a), b, weight_string(b)", + "OrderBy": "(0|3) ASC, (1|4) ASC", + "Query": "select a, b, count(*), weight_string(a), weight_string(b) from user group by a, weight_string(a), b, weight_string(b) order by a asc, b asc", + "Table": "user" + } + ] + } +} + +# scatter aggregate using distinctdistinct +"select distinct col from user" +{ + "QueryType": "SELECT", + "Original": "select distinct col from user", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Distinct":false, + "GroupBy": "0", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col from user where 1 != 1", + "Query": "select distinct col from user order by col asc", + "Table": "user" + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select distinct col from user", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "GroupBy": "0", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col from user where 1 != 1", + "OrderBy": "0 ASC", + "Query": "select distinct col from user order by col asc", + "Table": "user" + } + ] + } +} + +# we have to track the order of distinct aggregation expressions +"select val2, count(distinct val1), count(*) from user group by val2" +{ + "QueryType": "SELECT", + "Original": "select val2, count(distinct val1), count(*) from user group by val2", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "count(2)", + "Distinct":true, + "GroupBy": "0", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select val2, val1, count(*) from user where 1 != 1 group by val2, val1", + "Query": "select val2, val1, count(*) from user group by val2, val1 order by val2 asc, val1 asc", + "Table": "user" + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select val2, count(distinct val1), count(*) from user group by val2", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "count_distinct(1|4) AS count(distinct val1), sum_count_star(2) AS count(*)", + "GroupBy": "(0|3)", + "ResultColumns": 3, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select val2, val1, count(*), weight_string(val2), weight_string(val1) from user where 1 != 1 group by val2, weight_string(val2), val1, weight_string(val1)", + "OrderBy": "(0|3) ASC, (1|4) ASC", + "Query": "select val2, val1, count(*), weight_string(val2), weight_string(val1) from user group by val2, weight_string(val2), val1, weight_string(val1) order by val2 asc, val1 asc", + "Table": "user" + } + ] + } +} + +# weight_string addition to group by +"select lower(textcol1) as v, count(*) from user group by v" +{ + "QueryType": "SELECT", + "Original": "select lower(textcol1) as v, count(*) from user group by v", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "count(1)", + "Distinct":false, + "GroupBy": "0", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select lower(textcol1) as v, count(*) from user where 1 != 1 group by v", + "Query": "select lower(textcol1) as v, count(*) from user group by v order by v asc", + "Table": "user" + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select lower(textcol1) as v, count(*) from user group by v", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count_star(1) AS count(*)", + "GroupBy": "(0|2)", + "ResultColumns": 2, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select lower(textcol1) as v, count(*), weight_string(lower(textcol1)) from user where 1 != 1 group by v, weight_string(lower(textcol1))", + "OrderBy": "(0|2) ASC", + "Query": "select lower(textcol1) as v, count(*), weight_string(lower(textcol1)) from user group by v, weight_string(lower(textcol1)) order by v asc", + "Table": "user" + } + ] + } +} + +# weight_string addition to group by when also there in order by +"select char_length(texcol1) as a, count(*) from user group by a order by a" +{ + "QueryType": "SELECT", + "Original": "select char_length(texcol1) as a, count(*) from user group by a order by a", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "count(1)", + "Distinct":false, + "GroupBy": "0", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select char_length(texcol1) as a, count(*) from user where 1 != 1 group by a", + "Query": "select char_length(texcol1) as a, count(*) from user group by a order by a asc", + "Table": "user" + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select char_length(texcol1) as a, count(*) from user group by a order by a", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count_star(1) AS count(*)", + "GroupBy": "(0|2)", + "ResultColumns": 2, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select char_length(texcol1) as a, count(*), weight_string(char_length(texcol1)) from user where 1 != 1 group by a, weight_string(char_length(texcol1))", + "OrderBy": "(0|2) ASC", + "Query": "select char_length(texcol1) as a, count(*), weight_string(char_length(texcol1)) from user group by a, weight_string(char_length(texcol1)) order by a asc", + "Table": "user" + } + ] + } +} + +# scatter aggregate symtab lookup error +"select id, b as id, count(*) from user order by id" +"ambiguous symbol reference: id" +{ + "QueryType": "SELECT", + "Original": "select id, b as id, count(*) from user order by id", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Scalar", + "Aggregates": "random(0) AS id, random(1) AS b AS id, sum_count_star(2) AS count(*)", + "ResultColumns": 3, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id, b as id, count(*), weight_string(b) from user where 1 != 1", + "OrderBy": "(1|3) ASC", + "Query": "select id, b as id, count(*), weight_string(b) from user order by id asc", + "Table": "user" + } + ] + } +} + +# scatter group by a integer column. Do not add weight strings for this. +"select count(*), intcol from user group by intcol" +{ + "QueryType": "SELECT", + "Original": "select count(*), intcol from user group by intcol", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "count(0)", + "Distinct":false, + "GroupBy": "1", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select count(*), intcol from user where 1 != 1 group by intcol", + "Query": "select count(*), intcol from user group by intcol order by intcol asc", + "Table": "user" + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select count(*), intcol from user group by intcol", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count_star(0) AS count(*)", + "GroupBy": "1", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select count(*), intcol from user where 1 != 1 group by intcol", + "OrderBy": "1 ASC", + "Query": "select count(*), intcol from user group by intcol order by intcol asc", + "Table": "user" + } + ] + } +} + +# group by a non-unique vindex column should use an OrderedAggregate primitive +"select id, count(*) from user group by id" +{ + "QueryType": "SELECT", + "Original": "select id, count(*) from user group by id", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "count(1)", + "Distinct":false, + "GroupBy": "0", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id, count(*) from user where 1 != 1 group by id", + "Query": "select id, count(*) from user group by id order by id asc", + "Table": "user" + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select id, count(*) from user group by id", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count_star(1) AS count(*)", + "GroupBy": "(0|2)", + "ResultColumns": 2, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id, count(*), weight_string(id) from user where 1 != 1 group by id, weight_string(id)", + "OrderBy": "(0|2) ASC", + "Query": "select id, count(*), weight_string(id) from user group by id, weight_string(id) order by id asc", + "Table": "user" + } + ] + } +} + +# Cannot have more than one aggr(distinct... +"select count(distinct a), count(distinct b) from user" +"unsupported: only one distinct aggregation allowed in a select: count(distinct b)" +Gen4 plan same as above + +# multiple distinct aggregations on the same column in different positions +"select count(distinct tcol2), tcol1, count(*), sum(distinct tcol2) from user group by tcol1" +"unsupported: only one distinct aggregation allowed in a select: sum(distinct tcol2)" +{ + "QueryType": "SELECT", + "Original": "select count(distinct tcol2), tcol1, count(*), sum(distinct tcol2) from user group by tcol1", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "count_distinct(0|4) AS count(distinct tcol2), sum_count_star(2) AS count(*), sum_distinct(3|4) AS sum(distinct tcol2)", + "GroupBy": "(1|5)", + "ResultColumns": 4, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select tcol2, tcol1, count(*), tcol2, weight_string(tcol2), weight_string(tcol1) from user where 1 != 1 group by tcol2, weight_string(tcol2), tcol1, weight_string(tcol1)", + "OrderBy": "(1|5) ASC, (0|4) ASC", + "Query": "select tcol2, tcol1, count(*), tcol2, weight_string(tcol2), weight_string(tcol1) from user group by tcol2, weight_string(tcol2), tcol1, weight_string(tcol1) order by tcol1 asc, tcol2 asc", + "Table": "user" + } + ] + } +} + +# multiple distinct aggregations on the same column is allowed +"select tcol1, count(distinct tcol2), sum(distinct tcol2) from user group by tcol1" +"unsupported: only one distinct aggregation allowed in a select: sum(distinct tcol2)" +{ + "QueryType": "SELECT", + "Original": "select tcol1, count(distinct tcol2), sum(distinct tcol2) from user group by tcol1", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "count_distinct(1|4) AS count(distinct tcol2), sum_distinct(2|4) AS sum(distinct tcol2)", + "GroupBy": "(0|3)", + "ResultColumns": 3, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select tcol1, tcol2, tcol2, weight_string(tcol1), weight_string(tcol2) from user where 1 != 1 group by tcol1, weight_string(tcol1), tcol2, weight_string(tcol2)", + "OrderBy": "(0|3) ASC, (1|4) ASC", + "Query": "select tcol1, tcol2, tcol2, weight_string(tcol1), weight_string(tcol2) from user group by tcol1, weight_string(tcol1), tcol2, weight_string(tcol2) order by tcol1 asc, tcol2 asc", + "Table": "user" + } + ] + } +} + +# multiple distinct functions with grouping. +"select col1, count(distinct col2), sum(distinct col2) from user group by col1" +"unsupported: only one distinct aggregation allowed in a select: sum(distinct col2)" +{ + "QueryType": "SELECT", + "Original": "select col1, count(distinct col2), sum(distinct col2) from user group by col1", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "count_distinct(1|4) AS count(distinct col2), sum_distinct(2|4) AS sum(distinct col2)", + "GroupBy": "(0|3)", + "ResultColumns": 3, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col1, col2, col2, weight_string(col1), weight_string(col2) from user where 1 != 1 group by col1, weight_string(col1), col2, weight_string(col2)", + "OrderBy": "(0|3) ASC, (1|4) ASC", + "Query": "select col1, col2, col2, weight_string(col1), weight_string(col2) from user group by col1, weight_string(col1), col2, weight_string(col2) order by col1 asc, col2 asc", + "Table": "user" + } + ] + } +} + +# Column and Literal not equal filter on scatter aggregates +"select count(*) a from user having a != 10" +"unsupported: filtering on results of aggregates" +{ + "QueryType": "SELECT", + "Original": "select count(*) a from user having a != 10", + "Instructions": { + "OperatorType": "Filter", + "Predicate": "a != 10", + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Scalar", + "Aggregates": "sum_count_star(0) AS a", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select count(*) as a from user where 1 != 1", + "Query": "select count(*) as a from user", + "Table": "user" + } + ] + } + ] + } +} + +# Column and Literal equality filter on scatter aggregates +"select count(*) a from user having a = 10" +"unsupported: filtering on results of aggregates" +{ + "QueryType": "SELECT", + "Original": "select count(*) a from user having a = 10", + "Instructions": { + "OperatorType": "Filter", + "Predicate": "a = 10", + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Scalar", + "Aggregates": "sum_count_star(0) AS a", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select count(*) as a from user where 1 != 1", + "Query": "select count(*) as a from user", + "Table": "user" + } + ] + } + ] + } +} + +# Equality filtering with column and string literal on scatter aggregates +"select count(*) a from user having a = '1'" +"unsupported: filtering on results of aggregates" +{ + "QueryType": "SELECT", + "Original": "select count(*) a from user having a = '1'", + "Instructions": { + "OperatorType": "Filter", + "Predicate": "a = '1'", + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Scalar", + "Aggregates": "sum_count_star(0) AS a", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select count(*) as a from user where 1 != 1", + "Query": "select count(*) as a from user", + "Table": "user" + } + ] + } + ] + } +} + +# Greater Equal filter on scatter aggregates +"select count(*) a from user having a >= 10" +"unsupported: filtering on results of aggregates" +{ + "QueryType": "SELECT", + "Original": "select count(*) a from user having a \u003e= 10", + "Instructions": { + "OperatorType": "Filter", + "Predicate": "a \u003e= 10", + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Scalar", + "Aggregates": "sum_count_star(0) AS a", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select count(*) as a from user where 1 != 1", + "Query": "select count(*) as a from user", + "Table": "user" + } + ] + } + ] + } +} + +# Greater than filter on scatter aggregates +"select count(*) a from user having a > 10" +"unsupported: filtering on results of aggregates" +{ + "QueryType": "SELECT", + "Original": "select count(*) a from user having a \u003e 10", + "Instructions": { + "OperatorType": "Filter", + "Predicate": "a \u003e 10", + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Scalar", + "Aggregates": "sum_count_star(0) AS a", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select count(*) as a from user where 1 != 1", + "Query": "select count(*) as a from user", + "Table": "user" + } + ] + } + ] + } +} + +# Less Equal filter on scatter aggregates +"select count(*) a from user having a <= 10" +"unsupported: filtering on results of aggregates" +{ + "QueryType": "SELECT", + "Original": "select count(*) a from user having a \u003c= 10", + "Instructions": { + "OperatorType": "Filter", + "Predicate": "a \u003c= 10", + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Scalar", + "Aggregates": "sum_count_star(0) AS a", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select count(*) as a from user where 1 != 1", + "Query": "select count(*) as a from user", + "Table": "user" + } + ] + } + ] + } +} + +# Less Equal filter on scatter with grouping +"select col, count(*) a from user group by col having a <= 10" +"unsupported: filtering on results of aggregates" +{ + "QueryType": "SELECT", + "Original": "select col, count(*) a from user group by col having a \u003c= 10", + "Instructions": { + "OperatorType": "Filter", + "Predicate": "a \u003c= 10", + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count_star(1) AS a", + "GroupBy": "0", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col, count(*) as a from user where 1 != 1 group by col", + "OrderBy": "0 ASC", + "Query": "select col, count(*) as a from user group by col order by col asc", + "Table": "user" + } + ] + } + ] + } +} + +# Less than filter on scatter aggregates +"select count(*) a from user having a < 10" +"unsupported: filtering on results of aggregates" +{ + "QueryType": "SELECT", + "Original": "select count(*) a from user having a \u003c 10", + "Instructions": { + "OperatorType": "Filter", + "Predicate": "a \u003c 10", + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Scalar", + "Aggregates": "sum_count_star(0) AS a", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select count(*) as a from user where 1 != 1", + "Query": "select count(*) as a from user", + "Table": "user" + } + ] + } + ] + } +} + +# Not equal filter with column and string literal on scatter aggregates +"select count(*) a from user having a != '1'" +"unsupported: filtering on results of aggregates" +{ + "QueryType": "SELECT", + "Original": "select count(*) a from user having a != '1'", + "Instructions": { + "OperatorType": "Filter", + "Predicate": "a != '1'", + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Scalar", + "Aggregates": "sum_count_star(0) AS a", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select count(*) as a from user where 1 != 1", + "Query": "select count(*) as a from user", + "Table": "user" + } + ] + } + ] + } +} + +# We should be able to find grouping keys on ordered aggregates +"select count(*) as a, val1 from user group by val1 having a = 1.00" +"unsupported: filtering on results of aggregates" +{ + "QueryType": "SELECT", + "Original": "select count(*) as a, val1 from user group by val1 having a = 1.00", + "Instructions": { + "OperatorType": "SimpleProjection", + "Columns": [ + 0, + 1 + ], + "Inputs": [ + { + "OperatorType": "Filter", + "Predicate": "a = 1.00", + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count_star(0) AS a", + "GroupBy": "(1|2)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select count(*) as a, val1, weight_string(val1) from user where 1 != 1 group by val1, weight_string(val1)", + "OrderBy": "(1|2) ASC", + "Query": "select count(*) as a, val1, weight_string(val1) from user group by val1, weight_string(val1) order by val1 asc", + "Table": "user" + } + ] } ] } @@ -1046,60 +4145,337 @@ } } -# invalid order by column numner for scatter -"select col, count(*) from user group by col order by 5 limit 10" -"column number out of range: 5" +# aggr and non-aggr without group by (with query does not give useful result out) +"select id, count(*) from user" +{ + "QueryType": "SELECT", + "Original": "select id, count(*) from user", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "count(1)", + "Distinct":false, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id, count(*) from user where 1 != 1", + "Query": "select id, count(*) from user", + "Table": "user" + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select id, count(*) from user", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Scalar", + "Aggregates": "random(0) AS id, sum_count_star(1) AS count(*)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id, count(*) from user where 1 != 1", + "Query": "select id, count(*) from user", + "Table": "user" + } + ] + } +} -# Group by with collate operator -"select user.col1 as a from user where user.name = 5 group by a collate utf8_general_ci" +# aggregation filtering by having on a route +"select name from user group by name having count(name) = 10" { "QueryType": "SELECT", - "Original": "select user.col1 as a from user where user.name = 5 group by a collate utf8_general_ci", + "Original": "select name from user group by name having count(name) = 10", "Instructions": { "OperatorType": "Route", - "Variant": "SelectEqualUnique", + "Variant": "SelectScatter", "Keyspace": { "Name": "user", "Sharded": true }, - "FieldQuery": "select user.col1 as a from user where 1 != 1 group by a collate utf8_general_ci", - "Query": "select user.col1 as a from user where user.name = 5 group by a collate utf8_general_ci", - "Table": "user", - "Values": [ - "INT64(5)" + "FieldQuery": "select name from user where 1 != 1 group by name", + "Query": "select name from user group by name having count(name) = 10", + "Table": "user" + } +} +Gen4 plan same as above + +# aggregation filtering by having on a route with no group by +"select 1 from user having count(name) = 10" +{ + "QueryType": "SELECT", + "Original": "select 1 from user having count(name) = 10", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from user where 1 != 1", + "Query": "select 1 from user having count(name) = 10", + "Table": "user" + } +} +{ + "QueryType": "SELECT", + "Original": "select 1 from user having count(name) = 10", + "Instructions": { + "OperatorType": "SimpleProjection", + "Columns": [ + 0 ], - "Vindex": "user_index" + "Inputs": [ + { + "OperatorType": "Filter", + "Predicate": "OFFSET(1, 'COUNT(name)') = 10", + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Scalar", + "Aggregates": "random(0) AS 1, sum_count(1) AS count(name)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1, count(name) from user where 1 != 1", + "Query": "select 1, count(name) from user", + "Table": "user" + } + ] + } + ] + } + ] } } -# Group by invalid column number (code is duplicated from symab). -"select id from user group by 1.1" -"column number is not an int" +# group_concat on single shards +# vitess gen3报错,VtDriver gen4竟然可以执行,并且和gen4生成的一样 +"select group_concat(user_id order by name), name from user group by name" +{ + "QueryType": "SELECT", + "Original": "select group_concat(user_id order by name), name from user group by name", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select group_concat(user_id order by name), name from user where 1 != 1 group by name", + "Query": "select group_concat(user_id order by name), name from user group by name", + "Table": "user" + } +} +Gen4 plan same as above -# Group by out of range column number (code is duplicated from symab). -"select id from user group by 2" -"column number out of range: 2" +# having should be able to add new aggregation expressions in having +"select foo from user group by foo having count(*) = 3" +"unsupported: filtering on results of aggregates" +{ + "QueryType": "SELECT", + "Original": "select foo from user group by foo having count(*) = 3", + "Instructions": { + "OperatorType": "SimpleProjection", + "Columns": [ + 0 + ], + "Inputs": [ + { + "OperatorType": "Filter", + "Predicate": "OFFSET(1, 'COUNT(*)') = 3", + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count_star(1) AS count(*)", + "GroupBy": "(0|2)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select foo, count(*), weight_string(foo) from user where 1 != 1 group by foo, weight_string(foo)", + "OrderBy": "(0|2) ASC", + "Query": "select foo, count(*), weight_string(foo) from user group by foo, weight_string(foo) order by foo asc", + "Table": "user" + } + ] + } + ] + } + ] + } +} -# syntax error detected by planbuilder -"select count(distinct *) from user" -"syntax error: count(distinct *)" +# find aggregation expression and use column offset in filter +"select foo, count(*) from user group by foo having count(*) = 3" +"unsupported: filtering on results of aggregates" +{ + "QueryType": "SELECT", + "Original": "select foo, count(*) from user group by foo having count(*) = 3", + "Instructions": { + "OperatorType": "SimpleProjection", + "Columns": [ + 0, + 1 + ], + "Inputs": [ + { + "OperatorType": "Filter", + "Predicate": "OFFSET(1, 'COUNT(*)') = 3", + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count_star(1) AS count(*)", + "GroupBy": "(0|2)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select foo, count(*), weight_string(foo) from user where 1 != 1 group by foo, weight_string(foo)", + "OrderBy": "(0|2) ASC", + "Query": "select foo, count(*), weight_string(foo) from user group by foo, weight_string(foo) order by foo asc", + "Table": "user" + } + ] + } + ] + } + ] + } +} -# scatter aggregate in a subquery -"select a from (select count(*) as a from user) t" +# find aggregation expression and use column offset in filter times two +"select foo, sum(foo), sum(bar) from user group by foo having sum(foo)+sum(bar) = 42" +"unsupported: filtering on results of aggregates" { "QueryType": "SELECT", - "Original": "select a from (select count(*) as a from user) t", + "Original": "select foo, sum(foo), sum(bar) from user group by foo having sum(foo)+sum(bar) = 42", "Instructions": { - "OperatorType": "Subquery", + "OperatorType": "SimpleProjection", "Columns": [ - 0 + 0, + 1, + 2 + ], + "Inputs": [ + { + "OperatorType": "Filter", + "Predicate": "OFFSET(1, 'SUM(foo)') + OFFSET(2, 'SUM(bar)') = 42", + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum(1) AS sum(foo), sum(2) AS sum(bar)", + "GroupBy": "(0|3)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select foo, sum(foo), sum(bar), weight_string(foo) from user where 1 != 1 group by foo, weight_string(foo)", + "OrderBy": "(0|3) ASC", + "Query": "select foo, sum(foo), sum(bar), weight_string(foo) from user group by foo, weight_string(foo) order by foo asc", + "Table": "user" + } + ] + } + ] + } + ] + } +} + +# find aggregation expression and use column offset in filter times three +"select foo, sum(foo) as fooSum, sum(bar) as barSum from user group by foo having fooSum+sum(bar) = 42" +"unsupported: filtering on results of aggregates" +{ + "QueryType": "SELECT", + "Original": "select foo, sum(foo) as fooSum, sum(bar) as barSum from user group by foo having fooSum+sum(bar) = 42", + "Instructions": { + "OperatorType": "SimpleProjection", + "Columns": [ + 0, + 1, + 2 ], + "Inputs": [ + { + "OperatorType": "Filter", + "Predicate": "fooSum + OFFSET(2, 'SUM(bar)') = 42", + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum(1) AS foosum, sum(2) AS barsum", + "GroupBy": "(0|3)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select foo, sum(foo) as fooSum, sum(bar) as barSum, weight_string(foo) from user where 1 != 1 group by foo, weight_string(foo)", + "OrderBy": "(0|3) ASC", + "Query": "select foo, sum(foo) as fooSum, sum(bar) as barSum, weight_string(foo) from user group by foo, weight_string(foo) order by foo asc", + "Table": "user" + } + ] + } + ] + } + ] + } +} + +# distinct and aggregate functions +"select distinct a, count(*) from user group by a" +"unsupported: distinct cannot be combined with aggregate functions" +{ + "QueryType": "SELECT", + "Original": "select distinct a, count(*) from user group by a", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "GroupBy": "(0|2), 1", + "ResultColumns": 2, "Inputs": [ { "OperatorType": "Aggregate", "Variant": "Ordered", - "Aggregates": "count(0)", - "Distinct": "false", + "Aggregates": "sum_count_star(1) AS count(*)", + "GroupBy": "(0|2)", "Inputs": [ { "OperatorType": "Route", @@ -1108,8 +4484,9 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select count(*) as a from user where 1 != 1", - "Query": "select count(*) as a from user", + "FieldQuery": "select a, count(*), weight_string(a) from user where 1 != 1 group by a, weight_string(a)", + "OrderBy": "(0|2) ASC", + "Query": "select a, count(*), weight_string(a) from user group by a, weight_string(a) order by a asc", "Table": "user" } ] @@ -1118,42 +4495,49 @@ } } -# scatter avg -"select avg(costly) from user" +# distinct and aggregate functions missing group by +"select distinct a, count(*) from user" +"unsupported: distinct cannot be combined with aggregate functions" { "QueryType": "SELECT", - "Original": "select avg(costly) from user", + "Original": "select distinct a, count(*) from user", "Instructions": { "OperatorType": "Aggregate", "Variant": "Ordered", - "Aggregates": "avg_sum(0) AS avg(costly);avg_count(1)", - "Distinct": "false", + "GroupBy": "0, 1", "Inputs": [ { - "OperatorType": "Route", - "Variant": "SelectScatter", - "Keyspace": { - "Name": "user", - "Sharded": true - }, - "FieldQuery": "select sum(costly), count(costly) from user where 1 != 1", - "Query": "select sum(costly), count(costly) from user", - "Table": "user" + "OperatorType": "Aggregate", + "Variant": "Scalar", + "Aggregates": "random(0) AS a, sum_count_star(1) AS count(*)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select a, count(*) from user where 1 != 1", + "Query": "select a, count(*) from user", + "Table": "user" + } + ] } ] } } -# scatter avg group by non-vindex column -"select id, avg(costly) from user group by id" +# using two distinct columns - min with distinct vindex, sum with distinct without vindex +"select col1, min(distinct name), sum(distinct col3) from user group by col1" { "QueryType": "SELECT", - "Original": "select id, avg(costly) from user group by id", + "Original": "select col1, min(distinct name), sum(distinct col3) from user group by col1", "Instructions": { "OperatorType": "Aggregate", "Variant": "Ordered", - "Aggregates": "avg_sum(1) AS avg(costly);avg_count(2)", - "Distinct": "false", + "Aggregates": "sum_distinct(2) AS sum(distinct col3)", + "Distinct":true, "GroupBy": "0", "Inputs": [ { @@ -1163,53 +4547,44 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select id, sum(costly), count(costly) from user where 1 != 1 group by id", - "Query": "select id, sum(costly), count(costly) from user group by id order by id asc", + "FieldQuery": "select col1, min(DISTINCT name), col3 from user where 1 != 1 group by col1, col3", + "Query": "select col1, min(DISTINCT name), col3 from user group by col1, col3 order by col1 asc, col3 asc", "Table": "user" } ] } } - -# scatter avg group by non-vindex column order by avg_cost -"select id, avg(costly) as avg_cost from user group by id order by avg_cost" { "QueryType": "SELECT", - "Original": "select id, avg(costly) as avg_cost from user group by id order by avg_cost", + "Original": "select col1, min(distinct name), sum(distinct col3) from user group by col1", "Instructions": { - "OperatorType": "Sort", - "Variant": "Memory", - "OrderBy": "1 ASC", + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "min(1) AS min(distinct name), sum_distinct(2|4) AS sum(distinct col3)", + "GroupBy": "(0|3)", + "ResultColumns": 3, "Inputs": [ { - "OperatorType": "Aggregate", - "Variant": "Ordered", - "Aggregates": "avg_sum(1) AS avg_cost;avg_count(2)", - "Distinct": "false", - "GroupBy": "0", - "Inputs": [ - { - "OperatorType": "Route", - "Variant": "SelectScatter", - "Keyspace": { - "Name": "user", - "Sharded": true - }, - "FieldQuery": "select id, sum(costly), count(costly) from user where 1 != 1 group by id", - "Query": "select id, sum(costly), count(costly) from user group by id order by id asc", - "Table": "user" - } - ] + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col1, min(DISTINCT name), col3, weight_string(col1), weight_string(col3) from user where 1 != 1 group by col1, weight_string(col1), col3, weight_string(col3)", + "OrderBy": "(0|3) ASC, (2|4) ASC", + "Query": "select col1, min(DISTINCT name), col3, weight_string(col1), weight_string(col3) from user group by col1, weight_string(col1), col3, weight_string(col3) order by col1 asc, col3 asc", + "Table": "user" } ] } } -# scatter avg group by vindex column -"select name, avg(costly) from user group by name" +# distinct and group by together for single route - group by is redundant +"select distinct col1, name from user group by col1" { "QueryType": "SELECT", - "Original": "select name, avg(costly) from user group by name", + "Original": "select distinct col1, name from user group by col1", "Instructions": { "OperatorType": "Route", "Variant": "SelectScatter", @@ -1217,8 +4592,9 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select name, avg(costly) from user where 1 != 1 group by name", - "Query": "select name, avg(costly) from user group by name", + "FieldQuery": "select col1, name from user where 1 != 1 group by col1", + "Query": "select distinct col1, name from user group by col1", "Table": "user" } -} \ No newline at end of file +} +Gen4 plan same as above \ No newline at end of file diff --git a/src/test/resources/plan/aggr_cases_unsupported.txt b/src/test/resources/plan/aggr_cases_unsupported.txt index 5553895..05ab7f2 100644 --- a/src/test/resources/plan/aggr_cases_unsupported.txt +++ b/src/test/resources/plan/aggr_cases_unsupported.txt @@ -7,8 +7,8 @@ "Instructions": { "OperatorType": "Join", "Variant": "Join", - "JoinColumnIndexes": [-1,1], - "TableName": "user_unsharded", + "JoinColumnIndexes": "L:0,R:0", + "TableName": "`user`_unsharded", "Inputs": [ { "OperatorType": "Route", @@ -17,13 +17,13 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select user.a from user where 1 != 1", - "Query": "select user.a from user", + "FieldQuery": "select `user`.a from user where 1 != 1", + "Query": "select `user`.a from user", "Table": "user" }, { "OperatorType": "Route", - "Variant": "SelectUnsharded", + "Variant": "Unsharded", "Keyspace": { "Name": "main", "Sharded": false @@ -35,6 +35,7 @@ ] } } +Gen4 plan same as above # Reason: routing rules # routing rules for aggregates @@ -44,7 +45,7 @@ "Original": "select id, count(*) from route2 group by id", "Instructions": { "OperatorType": "Route", - "Variant": "SelectUnsharded", + "Variant": "Unsharded", "Keyspace": { "Name": "main", "Sharded": false @@ -54,6 +55,7 @@ "Table": "unsharded" } } +Gen4 plan same as above # Reason: reference table # order by on a reference table @@ -63,7 +65,7 @@ "Original": "select col from ref order by col", "Instructions": { "OperatorType": "Route", - "Variant": "SelectReference", + "Variant": "Reference", "Keyspace": { "Name": "user", "Sharded": true @@ -72,4 +74,116 @@ "Query": "select col from ref order by col asc", "Table": "ref" } +} +Gen4 plan same as above + +# aggregation filtering by having on a route with no group by with non-unique vindex filter +"select 1 from user having count(id) = 10 and name = 'a'" +{ + "QueryType": "SELECT", + "Original": "select 1 from user having count(id) = 10 and name = 'a'", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from user where 1 != 1", + "Query": "select 1 from user having count(id) = 10 and name = 'a'", + "Table": "user", + "Values": [ + "VARCHAR(\"a\")" + ], + "Vindex": "name_user_map" + } +} +{ + "QueryType": "SELECT", + "Original": "select 1 from user having count(id) = 10 and name = 'a'", + "Instructions": { + "OperatorType": "SimpleProjection", + "Columns": [ + 0 + ], + "Inputs": [ + { + "OperatorType": "Filter", + "Predicate": "OFFSET(1, 'count(id)') = 10", + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Scalar", + "Aggregates": "random(0) AS 1, sum_count(1) AS count(id)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "Equal", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1, count(id) from user where 1 != 1", + "Query": "select 1, count(id) from user where `name` = 'a'", + "Table": "user", + "Values": [ + "VARCHAR(\"a\")" + ], + "Vindex": "name_user_map" + } + ] + } + ] + } + ] + } +} + +# order by inside and outside parenthesis select +"(select id from user order by 1 desc) order by 1 asc limit 2" +{ + "QueryType": "SELECT", + "Original": "(select id from user order by 1 desc) order by 1 asc limit 2", + "Instructions": { + "OperatorType": "Limit", + "Count": "INT64(2)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id, weight_string(id) from user where 1 != 1", + "OrderBy": "(0|1) ASC", + "Query": "select id, weight_string(id) from user order by 1 asc limit :__upper_limit", + "ResultColumns": 1, + "Table": "user" + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "(select id from user order by 1 desc) order by 1 asc limit 2", + "Instructions": { + "OperatorType": "Limit", + "Count": "INT64(2)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id, weight_string(id) from user where 1 != 1", + "OrderBy": "(0|1) ASC", + "Query": "select id, weight_string(id) from user order by id asc limit :__upper_limit", + "ResultColumns": 1, + "Table": "user" + } + ] + } } \ No newline at end of file diff --git a/src/test/resources/plan/filter_cases.txt b/src/test/resources/plan/filter_cases.txt index 0ffae1f..c559426 100644 --- a/src/test/resources/plan/filter_cases.txt +++ b/src/test/resources/plan/filter_cases.txt @@ -1,5 +1,4 @@ # No where clause - "select id from user" { "QueryType": "SELECT", @@ -16,6 +15,7 @@ "Table": "user" } } +Gen4 plan same as above # Query that always return empty "select id from user where someColumn = null" @@ -34,6 +34,7 @@ "Table": "user" } } +Gen4 plan same as above # Single table unique vindex route "select name from user where user.name = 5" @@ -56,6 +57,7 @@ "Vindex": "user_index" } } +Gen4 plan same as above # Single table unique vindex route, but complex expr "select name from user where user.name = 5+5" @@ -74,6 +76,25 @@ "Table": "user" } } +{ + "QueryType": "SELECT", + "Original": "select name from user where user.name = 5+5", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select name from user where 1 != 1", + "Query": "select name from user where user.name = 5 + 5", + "Table": "user", + "Values": [ + "INT64(10)" + ], + "Vindex": "user_index" + } +} # Single table multiple unique vindex match "select id from music where id = 5 and user_id = 4" @@ -96,6 +117,7 @@ "Vindex": "user_index" } } +Gen4 plan same as above # Single table complex in clause "select id from user where name in (col, 'bb')" @@ -114,6 +136,7 @@ "Table": "user" } } +Gen4 plan same as above # Route with multiple route constraints, SelectIN is the best constraint. "select name from user where user.col = 5 and user.name in (1, 2)" @@ -136,6 +159,7 @@ "Vindex": "user_index" } } +Gen4 plan same as above # Route with multiple route constraints and boolean, SelectIN is the best constraint. "select name from user where user.col = case user.col when 'foo' then true else false end and user.name in (1, 2)" @@ -158,6 +182,7 @@ "Vindex": "user_index" } } +Gen4 plan same as above # 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" @@ -180,6 +205,7 @@ "Vindex": "user_index" } } +Gen4 plan same as above # 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" @@ -202,6 +228,7 @@ "Vindex": "user_index" } } +Gen4 plan same as above # 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)" @@ -220,6 +247,7 @@ "Table": "user" } } +Gen4 plan same as above # SELECT with IS NULL "select user_id from music where user_id is null" @@ -241,6 +269,60 @@ "Vindex": "music_user_map" } } +{ + "QueryType": "SELECT", + "Original": "select user_id from music where user_id is null", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user_id from music where 1 != 1", + "Query": "select user_id from music where user_id is null", + "Table": "music", + "Values": [ + "null" + ], + "Vindex": "music_user_map" + } +} + +# SELECT with IS NULL +"select id from music where id is null" +{ + "QueryType": "SELECT", + "Original": "select id from music where id is null", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from music where 1 != 1", + "Query": "select id from music where id is null", + "Table": "music", + "Vindex": "music_user_map" + } +} +{ + "QueryType": "SELECT", + "Original": "select id from music where id is null", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from music where 1 != 1", + "Query": "select id from music where id is null", + "Table": "music", + "Vindex": "music_user_map" + } +} # SELECT with IS NOT NULL "select id from music where id is not null" @@ -259,6 +341,7 @@ "Table": "music" } } +Gen4 plan same as above # Single table with unique vindex match and null match "select id from music where user_id = 4 and id = null" @@ -277,6 +360,7 @@ "Table": "music" } } +Gen4 plan same as above # Single table with unique vindex match and IN (null) "select id from music where user_id = 4 and user_id IN (null)" @@ -295,6 +379,7 @@ "Table": "music" } } +Gen4 plan same as above # Single table with unique vindex match and IN (null, 1, 2) "select id from music where user_id = 4 and id IN (null, 1, 2)" @@ -317,6 +402,7 @@ "Vindex": "user_index" } } +Gen4 plan same as above # 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)" @@ -335,6 +421,7 @@ "Table": "music" } } +Gen4 plan same as above # Multi-table unique vindex constraint "select user_extra.id from user join user_extra on user.name = user_extra.user_id where user.name = 5" @@ -357,6 +444,25 @@ "Vindex": "user_index" } } +{ + "QueryType": "SELECT", + "Original": "select user_extra.id from user join user_extra on user.name = user_extra.user_id where user.name = 5", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user_extra.id from user, user_extra where 1 != 1", + "Query": "select user_extra.id from user, user_extra where user.name = 5 and user.name = user_extra.user_id", + "Table": "user,user_extra", + "Values": [ + "INT64(5)" + ], + "Vindex": "user_index" + } +} # Multi-table unique vindex constraint on right table "select user_extra.id from user join user_extra on user.name = user_extra.user_id where user_extra.user_id = 5" @@ -379,6 +485,25 @@ "Vindex": "user_index" } } +{ + "QueryType": "SELECT", + "Original": "select user_extra.id from user join user_extra on user.name = user_extra.user_id where user_extra.user_id = 5", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user_extra.id from user, user_extra where 1 != 1", + "Query": "select user_extra.id from user, user_extra where user_extra.user_id = 5 and user.name = user_extra.user_id", + "Table": "user,user_extra", + "Values": [ + "INT64(5)" + ], + "Vindex": "user_index" + } +} # Multi-table unique vindex constraint on left table of left join "select user_extra.id from user left join user_extra on user.name = user_extra.user_id where user.name = 5" @@ -401,6 +526,25 @@ "Vindex": "user_index" } } +{ + "QueryType": "SELECT", + "Original": "select user_extra.id from user left join user_extra on user.name = user_extra.user_id where user.name = 5", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user_extra.id from user left join user_extra on user.name = user_extra.user_id where 1 != 1", + "Query": "select user_extra.id from user left join user_extra on user.name = user_extra.user_id where user.name = 5", + "Table": "user,user_extra", + "Values": [ + "INT64(5)" + ], + "Vindex": "user_index" + } +} # Multi-table unique vindex constraint on left-joined right table "select user_extra.id from user left join user_extra on user.name = user_extra.user_id where user_extra.user_id = 5" @@ -423,6 +567,25 @@ "Vindex": "user_index" } } +{ + "QueryType": "SELECT", + "Original": "select user_extra.id from user left join user_extra on user.name = user_extra.user_id where user_extra.user_id = 5", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user_extra.id from user, user_extra where 1 != 1", + "Query": "select user_extra.id from user, user_extra where user_extra.user_id = 5 and user.name = user_extra.user_id", + "Table": "user,user_extra", + "Values": [ + "INT64(5)" + ], + "Vindex": "user_index" + } +} # Multi-route unique vindex constraint "select user_extra.id from user join user_extra on user.col = user_extra.col where user.name = 5" @@ -464,6 +627,7 @@ ] } } +Gen4 plan same as above # 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.name = 5 and user_extra.user_id = 5" @@ -509,6 +673,21 @@ ] } } +{ + "QueryType": "SELECT", + "Original": "select user_extra.id from user join user_extra on user.col = user_extra.col where user.name = 5 and user_extra.user_id = 5", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user_extra.id from user, user_extra where 1 != 1", + "Query": "select user_extra.id from user, user_extra where user.name = 5 and user_extra.user_id = 5 and user.col = user_extra.col", + "Table": "user,user_extra" + } +} # Multi-route with cross-route constraint "select user_extra.id from user join user_extra on user.col = user_extra.col where user_extra.user_id = user.col" @@ -550,6 +729,7 @@ ] } } +Gen4 plan same as above # Multi-route with non-route constraint, should use first route. "select user_extra.id from user join user_extra on user.col = user_extra.col where 1 = 1" @@ -587,6 +767,40 @@ ] } } +{ + "QueryType": "SELECT", + "Original": "select user_extra.id from user join user_extra on user.col = user_extra.col where 1 = 1", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": [1], + "TableName": "user_user_extra", + "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 where 1 = 1", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user_extra.id from user_extra where 1 != 1", + "Query": "select user_extra.id from user_extra where 1 = 1 and user_extra.col = :user_col", + "Table": "user_extra" + } + ] + } +} # Case preservation test "select user_extra.Id from user join user_extra on user.nAME = user_extra.User_Id where user.Name = 5" @@ -609,45 +823,43 @@ "Vindex": "user_index" } } - -# subquery of information_schema with itself -"select * from information_schema.a where id in (select * from information_schema.b)" { "QueryType": "SELECT", - "Original": "select * from information_schema.a where id in (select * from information_schema.b)", + "Original": "select user_extra.Id from user join user_extra on user.nAME = user_extra.User_Id where user.Name = 5", "Instructions": { "OperatorType": "Route", - "Variant": "SelectDBA", + "Variant": "SelectEqualUnique", "Keyspace": { - "Name": "main", - "Sharded": false + "Name": "user", + "Sharded": true }, - "FieldQuery": "select * from information_schema.a where 1 != 1", - "Query": "select * from information_schema.a where id in ( select * from information_schema.b )" + "FieldQuery": "select user_extra.Id from user, user_extra where 1 != 1", + "Query": "select user_extra.Id from user, user_extra where user.Name = 5 and user.nAME = user_extra.User_Id", + "Table": "user,user_extra", + "Values": [ + "INT64(5)" + ], + "Vindex": "user_index" } } -# Single table equality route with unsigned value -"select id from user where name = 18446744073709551615" +# subquery of information_schema with itself +"select * from information_schema.a where id in (select * from information_schema.b)" { "QueryType": "SELECT", - "Original": "select id from user where name = 18446744073709551615", + "Original": "select * from information_schema.a where id in (select * from information_schema.b)", "Instructions": { "OperatorType": "Route", - "Variant": "SelectEqualUnique", + "Variant": "SelectDBA", "Keyspace": { - "Name": "user", - "Sharded": true + "Name": "main", + "Sharded": false }, - "FieldQuery": "select id from user where 1 != 1", - "Query": "select id from user where name = 18446744073709551615", - "Table": "user", - "Values": [ - "UINT64(18446744073709551615)" - ], - "Vindex": "name_user_map" + "FieldQuery": "select * from information_schema.a where 1 != 1", + "Query": "select * from information_schema.a where id in ( select * from information_schema.b )" } } +Gen4 plan same as above # Single table multiple non-unique vindex match "select id from user where costly = 'aa' and name = 'bb'" @@ -670,6 +882,7 @@ "Vindex": "name_user_map" } } +Gen4 plan same as above # Single table multiple non-unique vindex match for IN clause "select id from user where costly in ('aa', 'bb') and name in ('aa', 'bb')" @@ -692,6 +905,7 @@ "Vindex": "name_user_map" } } +Gen4 plan same as above # Route with multiple route constraints and boolean, SelectEqual is the best constraint. "select (id or col) as val from user where user.col = 5 and user.id in (1, 2) and user.name = 'aa'" @@ -714,6 +928,7 @@ "Vindex": "name_user_map" } } +Gen4 plan same as above # Single table equality route with val arg "select id from user where name = :a" @@ -736,6 +951,7 @@ "Vindex": "name_user_map" } } +Gen4 plan same as above # 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'" @@ -758,6 +974,7 @@ "Vindex": "name_user_map" } } +Gen4 plan same as above # subquery "select u.m from user_extra join user u where u.name in (select m2 from user where user.name = u.name and user_extra.col = user.col) and u.name in (user_extra.col, 1)" @@ -799,16 +1016,16 @@ ] } } - -# ensure subquery reordering gets us a better plan -"select u.m from user_extra join user u where u.name in (select m2 from user where user.name = 5) and u.name = 5" { "QueryType": "SELECT", - "Original": "select u.m from user_extra join user u where u.name in (select m2 from user where user.name = 5) and u.name = 5", + "Original": "select u.m from user_extra join user u where u.id in (select m2 from user where user.id = u.id and user_extra.col = user.col) and u.id in (user_extra.col, 1)", "Instructions": { "OperatorType": "Join", "Variant": "Join", - "JoinColumnIndexes": [1], + "JoinColumnIndexes": "R:0", + "JoinVars": { + "user_extra_col": 0 + }, "TableName": "user_extra_user", "Inputs": [ { @@ -818,34 +1035,34 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select 1 from user_extra where 1 != 1", - "Query": "select 1 from user_extra", + "FieldQuery": "select user_extra.col from user_extra where 1 != 1", + "Query": "select user_extra.col from user_extra", "Table": "user_extra" }, { "OperatorType": "Route", - "Variant": "SelectEqualUnique", + "Variant": "SelectIN", "Keyspace": { "Name": "user", "Sharded": true }, "FieldQuery": "select u.m from user as u where 1 != 1", - "Query": "select u.m from user as u where u.name = 5 and u.name in ( select m2 from user where user.name = 5 )", + "Query": "select u.m from user as u where u.id in (select m2 from user where user.id = u.id and user.col = :user_extra_col) and u.id in ::__vals", "Table": "user", "Values": [ - "INT64(5)" + "(:user_extra_col, INT64(1))" ], - "Vindex": "hash" + "Vindex": "user_index" } ] } } -# nested subquery -"select u.m from user_extra join user u where u.name in (select m2 from user where user.name = u.name and user_extra.col = user.col and user.name in (select m3 from user_extra where user_extra.user_id = user.name)) and u.name in (user_extra.col, 1)" +# ensure subquery reordering gets us a better plan +"select u.m from user_extra join user u where u.name in (select m2 from user where user.name = 5) and u.name = 5" { "QueryType": "SELECT", - "Original": "select u.m from user_extra join user u where u.name in (select m2 from user where user.name = u.name and user_extra.col = user.col and user.name in (select m3 from user_extra where user_extra.user_id = user.name)) and u.name in (user_extra.col, 1)", + "Original": "select u.m from user_extra join user u where u.name in (select m2 from user where user.name = 5) and u.name = 5", "Instructions": { "OperatorType": "Join", "Variant": "Join", @@ -859,46 +1076,168 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select user_extra.col from user_extra where 1 != 1", - "Query": "select user_extra.col from user_extra", + "FieldQuery": "select 1 from user_extra where 1 != 1", + "Query": "select 1 from user_extra", "Table": "user_extra" }, { "OperatorType": "Route", - "Variant": "SelectIN", + "Variant": "SelectEqualUnique", "Keyspace": { "Name": "user", "Sharded": true }, "FieldQuery": "select u.m from user as u where 1 != 1", - "Query": "select u.m from user as u where u.name in (::__vals) and u.name in ( select m2 from user where user.name = u.name and user.col = :user_extra_col and user.name in ( select m3 from user_extra where user_extra.user_id = user.name ) )", + "Query": "select u.m from user as u where u.name = 5 and u.name in ( select m2 from user where user.name = 5 )", "Table": "user", "Values": [ - "(:user_extra_col, INT64(1))" + "INT64(5)" ], - "Vindex": "user_index" + "Vindex": "hash" } ] } } - -# Correlated subquery in where clause -"select id from user where user.col in (select user_extra.col from user_extra where user_extra.user_id = user.name)" { "QueryType": "SELECT", - "Original": "select id from user where user.col in (select user_extra.col from user_extra where user_extra.user_id = user.name)", + "Original": "select u.m from user_extra join user u where u.id in (select m2 from user where user.id = 5) and u.id = 5", "Instructions": { - "OperatorType": "Route", - "Variant": "SelectScatter", - "Keyspace": { - "Name": "user", - "Sharded": true - }, + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "R:0", + "TableName": "user_extra_user", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from user_extra where 1 != 1", + "Query": "select 1 from user_extra", + "Table": "user_extra" + }, + { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select u.m from user as u where 1 != 1", + "Query": "select u.m from user as u where u.id in (select m2 from user where user.id = 5) and u.id = 5", + "Table": "user", + "Values": [ + 5 + ], + "Vindex": "user_index" + } + ] + } +} + +# nested subquery +"select u.m from user_extra join user u where u.name in (select m2 from user where user.name = u.name and user_extra.col = user.col and user.name in (select m3 from user_extra where user_extra.user_id = user.name)) and u.name in (user_extra.col, 1)" +{ + "QueryType": "SELECT", + "Original": "select u.m from user_extra join user u where u.name in (select m2 from user where user.name = u.name and user_extra.col = user.col and user.name in (select m3 from user_extra where user_extra.user_id = user.name)) and u.name in (user_extra.col, 1)", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": [1], + "TableName": "user_extra_user", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user_extra.col from user_extra where 1 != 1", + "Query": "select user_extra.col from user_extra", + "Table": "user_extra" + }, + { + "OperatorType": "Route", + "Variant": "SelectIN", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select u.m from user as u where 1 != 1", + "Query": "select u.m from user as u where u.name in (::__vals) and u.name in ( select m2 from user where user.name = u.name and user.col = :user_extra_col and user.name in ( select m3 from user_extra where user_extra.user_id = user.name ) )", + "Table": "user", + "Values": [ + "(:user_extra_col, INT64(1))" + ], + "Vindex": "user_index" + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select u.m from user_extra join user u where u.id in (select m2 from user where user.id = u.id and user_extra.col = user.col and user.id in (select m3 from user_extra where user_extra.user_id = user.id)) and u.id in (user_extra.col, 1)", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "R:0", + "JoinVars": { + "user_extra_col": 0 + }, + "TableName": "user_extra_user", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user_extra.col from user_extra where 1 != 1", + "Query": "select user_extra.col from user_extra", + "Table": "user_extra" + }, + { + "OperatorType": "Route", + "Variant": "SelectIN", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select u.m from user as u where 1 != 1", + "Query": "select u.m from user as u where u.id in (select m2 from user where user.id = u.id and user.col = :user_extra_col and user.id in (select m3 from user_extra where user_extra.user_id = user.id)) and u.id in ::__vals", + "Table": "user", + "Values": [ + ":user_extra_col", + 1 + ], + "Vindex": "user_index" + } + ] + } +} + +# Correlated subquery in where clause +"select id from user where user.col in (select user_extra.col from user_extra where user_extra.user_id = user.name)" +{ + "QueryType": "SELECT", + "Original": "select id from user where user.col in (select user_extra.col from user_extra where user_extra.user_id = user.name)", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, "FieldQuery": "select id from user where 1 != 1", "Query": "select id from user where user.col in ( select user_extra.col from user_extra where user_extra.user_id = user.name )", "Table": "user" } } +Gen4 plan same as above # outer and inner subquery route by same int val "select id from user where name = 5 and user.col in (select user_extra.col from user_extra where user_extra.user_id = 5)" @@ -921,6 +1260,7 @@ "Vindex": "user_index" } } +Gen4 plan same as above # outer and inner subquery route by same str val "select id from user where name = 'aa' and user.col in (select user_extra.col from user_extra where user_extra.user_id = 'aa')" @@ -943,6 +1283,7 @@ "Vindex": "user_index" } } +Gen4 plan same as above # outer and inner subquery route by same val arg "select id from user where name = :a and user.col in (select user_extra.col from user_extra where user_extra.user_id = :a)" @@ -965,10 +1306,12 @@ "Vindex": "user_index" } } +Gen4 plan same as above # 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)" "symbol foo.id not found" +Gen4 plan same as above # outer and inner subquery route by same outermost column value "select id2 from user uu where name in (select name from user where name = uu.name and user.col in (select user_extra.col from user_extra where user_extra.user_id = uu.name))" @@ -987,6 +1330,7 @@ "Table": "user" } } +Gen4 plan same as above # cross-shard subquery in IN clause. # Note the improved Underlying plan as SelectIN. @@ -1027,6 +1371,7 @@ ] } } +Gen4 plan same as above # cross-shard subquery in NOT IN clause. "select name from user where name not in (select col from user)" @@ -1062,6 +1407,7 @@ ] } } +Gen4 plan same as above # cross-shard subquery in EXISTS clause. "select id from user where exists (select col from user)" @@ -1097,6 +1443,38 @@ ] } } +{ + "QueryType": "SELECT", + "Original": "select id from user where exists (select col from user)", + "Instructions": { + "OperatorType": "Subquery", + "Variant": "PulloutExists", + "Inputs": [ + { + "OperatorType": "Limit", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col from user where 1 != 1", + "Query": "select col from user", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from user where 1 != 1", + "Query": "select id from user where :__sq_has_values1", + "Table": "user" + } + ] + } +} # cross-shard subquery as expression "select name from user where name = (select col from user)" @@ -1136,6 +1514,7 @@ ] } } +Gen4 plan same as above # multi-level pullout "select id1 from user where name = (select id2 from user where id2 in (select id3 from user))" @@ -1192,6 +1571,67 @@ ] } } +{ + "QueryType": "SELECT", + "Original": "select id1 from user where id = (select id2 from user where id2 in (select id3 from user))", + "Instructions": { + "OperatorType": "Subquery", + "Variant": "PulloutValue", + "PulloutVars": [ + "__sq_has_values1", + "__sq1" + ], + "Inputs": [ + { + "OperatorType": "Subquery", + "Variant": "PulloutIn", + "PulloutVars": [ + "__sq_has_values2", + "__sq2" + ], + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id3 from user where 1 != 1", + "Query": "select id3 from user", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id2 from user where 1 != 1", + "Query": "select id2 from user where :__sq_has_values2 = 1 and id2 in ::__sq2", + "Table": "user" + } + ] + }, + { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id1 from user where 1 != 1", + "Query": "select id1 from user where id = :__sq1", + "Table": "user", + "Values": [ + ":__sq1" + ], + "Vindex": "user_index" + } + ] + } +} # database() call in where clause. "select id from user where database()" @@ -1210,12 +1650,14 @@ "Table": "user" } } +Gen4 plan same as above # 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, # and the second reference is to the innermost 'from' subquery. "select id2 from user uu where id in (select id from user where id = uu.id and user.col in (select col from (select id from user_extra where user_id = 5) uu where uu.user_id = uu.id))" "unsupported: cross-shard correlated subquery" +Gen4 plan same as above # Select with equals null "select id from music where id = null" @@ -1234,6 +1676,7 @@ "Table": "music" } } +Gen4 plan same as above # query trying to query two different keyspaces at the same time "SELECT B.TABLE_NAME FROM INFORMATION_SCHEMA.TABLES AS A, INFORMATION_SCHEMA.COLUMNS AS B WHERE A.TABLE_SCHEMA = 'user' AND A.TABLE_SCHEMA = 'main' AND B.TABLE_NAME = A.TABLE_NAME" @@ -1255,9 +1698,6 @@ ] } } - -# query trying to query two different keyspaces at the same time -"SELECT B.TABLE_NAME FROM INFORMATION_SCHEMA.TABLES AS A, INFORMATION_SCHEMA.COLUMNS AS B WHERE A.TABLE_SCHEMA = 'user' AND A.TABLE_SCHEMA = 'main' AND B.TABLE_NAME = A.TABLE_NAME" { "QueryType": "SELECT", "Original": "SELECT B.TABLE_NAME FROM INFORMATION_SCHEMA.TABLES AS A, INFORMATION_SCHEMA.COLUMNS AS B WHERE A.TABLE_SCHEMA = 'user' AND A.TABLE_SCHEMA = 'main' AND B.TABLE_NAME = A.TABLE_NAME", @@ -1273,7 +1713,8 @@ "SysTableKeyspaceExpr": [ "VARBINARY(\"user\")", "VARBINARY(\"main\")" - ] + ], + "Table":"INFORMATION_SCHEMA.`COLUMNS`,INFORMATION_SCHEMA.`TABLES`" } } @@ -1293,3 +1734,1019 @@ "Query": "select * from INFORMATION_SCHEMA.TABLES where TABLE_SCHEMA = DATABASE()" } } +{ + "QueryType": "SELECT", + "Original": "SELECT * FROM INFORMATION_SCHEMA.TABLES WHERE TABLE_SCHEMA = DATABASE()", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectDBA", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select * from INFORMATION_SCHEMA.TABLES where 1 != 1", + "Query": "select * from INFORMATION_SCHEMA.TABLES where TABLE_SCHEMA = DATABASE()", + "Table": "INFORMATION_SCHEMA.`TABLES`" + } +} + +# Composite IN clause +"select id from user where (name, col) in (('aa', 'bb'), ('cc', 'dd'))" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select id from user where (name, col) in (('aa', 'bb'), ('cc', 'dd'))", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectIN", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from user where 1 != 1", + "Query": "select id from user where (name, col) in (('aa', 'bb'), ('cc', 'dd'))", + "Table": "user", + "Values": [ + "(VARCHAR(\"aa\"), VARCHAR(\"cc\"))" + ], + "Vindex": "name_user_map" + } +} + +# Composite IN clause, swapped columns +"select id from user where (col, name) in (('aa', 'bb'), ('cc', 'dd'))" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select id from user where (col, name) in (('aa', 'bb'), ('cc', 'dd'))", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectIN", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from user where 1 != 1", + "Query": "select id from user where (col, name) in (('aa', 'bb'), ('cc', 'dd'))", + "Table": "user", + "Values": [ + "(VARCHAR(\"bb\"), VARCHAR(\"dd\"))" + ], + "Vindex": "name_user_map" + } +} + +# Composite IN clause, choose cost within tuple +"select id from user where (costly, name) in (('aa', 'bb'), ('cc', 'dd'))" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select id from user where (costly, name) in (('aa', 'bb'), ('cc', 'dd'))", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectIN", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from user where 1 != 1", + "Query": "select id from user where (costly, name) in (('aa', 'bb'), ('cc', 'dd'))", + "Table": "user", + "Values": [ + "(VARCHAR(\"bb\"), VARCHAR(\"dd\"))" + ], + "Vindex": "name_user_map" + } +} + +# Composite IN clause, choose cost within tuple, swapped +"select id from user where (name, costly) in (('aa', 'bb'), ('cc', 'dd'))" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select id from user where (name, costly) in (('aa', 'bb'), ('cc', 'dd'))", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectIN", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from user where 1 != 1", + "Query": "select id from user where (name, costly) in (('aa', 'bb'), ('cc', 'dd'))", + "Table": "user", + "Values": [ + "(VARCHAR(\"aa\"), VARCHAR(\"cc\"))" + ], + "Vindex": "name_user_map" + } +} + +# Composite IN clause, choose cost +"select id from user where (col, costly) in (('aa', 'bb')) and (col, name) in (('cc', 'dd'))" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select id from user where (col, costly) in (('aa', 'bb')) and (col, name) in (('cc', 'dd'))", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectIN", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from user where 1 != 1", + "Query": "select id from user where (col, costly) in (('aa', 'bb')) and (col, name) in (('cc', 'dd'))", + "Table": "user", + "Values": [ + "(VARCHAR(\"dd\"))" + ], + "Vindex": "name_user_map" + } +} + + +# Composite IN clause vs equality +"select id from user where (col, id) in (('aa', 'bb')) and name = 5" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select id from user where (col, id) in (('aa', 'bb')) and name = 5", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from user where 1 != 1", + "Query": "select id from user where (col, id) in (('aa', 'bb')) and name = 5", + "Table": "user", + "Values": [ + "INT64(5)" + ], + "Vindex": "user_index" + } +} + +# Composite IN: multiple vindex matches +"select id from user where (costly, name) in (('aa', 'bb'), ('cc', 'dd'))" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select id from user where (costly, name) in (('aa', 'bb'), ('cc', 'dd'))", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectIN", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from user where 1 != 1", + "Query": "select id from user where (costly, name) in (('aa', 'bb'), ('cc', 'dd'))", + "Table": "user", + "Values": [ + "(VARCHAR(\"bb\"), VARCHAR(\"dd\"))" + ], + "Vindex": "name_user_map" + } +} + +# Composite IN: tuple inside tuple +"select id from user where ((col1, name), col2) in ((('aa', 'bb'), 'cc'), (('dd', 'ee'), 'ff'))" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select id from user where ((col1, name), col2) in ((('aa', 'bb'), 'cc'), (('dd', 'ee'), 'ff'))", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectIN", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from user where 1 != 1", + "Query": "select id from user where ((col1, name), col2) in ((('aa', 'bb'), 'cc'), (('dd', 'ee'), 'ff'))", + "Table": "user", + "Values": [ + "(VARCHAR(\"bb\"), VARCHAR(\"ee\"))" + ], + "Vindex": "name_user_map" + } +} + +# Composite IN: tuple inside tuple, but no match in tuple +"select id from user where (name, (col1, col2)) in (('aa', ('bb', 'cc')), ('dd', ('ee', 'ff')))" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select id from user where (name, (col1, col2)) in (('aa', ('bb', 'cc')), ('dd', ('ee', 'ff')))", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectIN", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from user where 1 != 1", + "Query": "select id from user where (name, (col1, col2)) in (('aa', ('bb', 'cc')), ('dd', ('ee', 'ff')))", + "Table": "user", + "Values": [ + "(VARCHAR(\"aa\"), VARCHAR(\"dd\"))" + ], + "Vindex": "name_user_map" + } +} + +# Composite IN: tuple inside tuple, mismiatched values +"select id from user where ((col1, name), col2) in (('aa', 'bb', 'cc'), (('dd', 'ee'), 'ff'))" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select id from user where ((col1, name), col2) in (('aa', 'bb', 'cc'), (('dd', 'ee'), 'ff'))", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from user where 1 != 1", + "Query": "select id from user where ((col1, name), col2) in (('aa', 'bb', 'cc'), (('dd', 'ee'), 'ff'))", + "Table": "user" + } +} + +# Composite IN: RHS not tuple +"select id from user where (col1, name) in (select * from music where music.user_id=user.id)" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select id from user where (col1, name) in (select * from music where music.user_id=user.id)", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from user where 1 != 1", + "Query": "select id from user where (col1, name) in (select * from music where music.user_id = user.id)", + "Table": "user" + } +} + +# Composite IN: RHS has no simple values +"select id from user where (col1, name) in (('aa', 1+1))" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select id from user where (col1, name) in (('aa', 1+1))", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectIN", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from user where 1 != 1", + "Query": "select id from user where (col1, name) in (('aa', 1 + 1))", + "Table": "user", + "Values": [ + "(INT64(2))" + ], + "Vindex": "name_user_map" + } +} + +# routing rules subquery merge with alias +"select col from user where id = (select id from route1 as a where a.id = user.id)" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select col from user where id = (select id from route1 as a where a.id = user.id)", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col from user where 1 != 1", + "Query": "select col from user where id = (select id from user as a where a.id = user.id)", + "Table": "user" + } +} + +#subquery on other table +"select distinct user.id, user.col from user where user.col in (select id from music where col2 = 'a')" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select distinct user.id, user.col from user where user.col in (select id from music where col2 = 'a')", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "GroupBy": "(0|2), 1", + "ResultColumns": 2, + "Inputs": [ + { + "OperatorType": "Subquery", + "Variant": "PulloutIn", + "PulloutVars": [ + "__sq_has_values1", + "__sq1" + ], + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from music where 1 != 1", + "Query": "select id from music where col2 = 'a'", + "Table": "music" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.id, user.col, weight_string(user.id) from user where 1 != 1", + "OrderBy": "(0|2) ASC, 1 ASC", + "Query": "select user.id, user.col, weight_string(user.id) from user where :__sq_has_values1 = 1 and user.col in ::__sq1 order by user.id asc, user.col asc", + "Table": "user" + } + ] + } + ] + } +} + +# two correlated subqueries that can be merge in a single route +"select u.id from user as u where u.col in (select ue.user_id from user_extra as ue where ue.user_id = u.id) and u.col2 in (select ue.user_id from user_extra as ue where ue.user_id = u.id)" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select u.id from user as u where u.col in (select ue.user_id from user_extra as ue where ue.user_id = u.id) and u.col2 in (select ue.user_id from user_extra as ue where ue.user_id = u.id)", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select u.id from user as u where 1 != 1", + "Query": "select u.id from user as u where u.col in (select ue.user_id from user_extra as ue where ue.user_id = u.id) and u.col2 in (select ue.user_id from user_extra as ue where ue.user_id = u.id)", + "Table": "user" + } +} + +# transitive closures for the win +"select id from user where user.name = user.col and user.col = 5" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select id from user where user.name = user.col and user.col = 5", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from user where 1 != 1", + "Query": "select id from user where user.name = user.col and user.col = 5", + "Table": "user", + "Values": [ + "INT64(5)" + ], + "Vindex": "user_index" + } +} + +# select from unsharded keyspace with uncorrelated subquery which should be merged to a single route +"select unsharded.id from unsharded where unsharded.name in (select name from unsharded_a)" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select unsharded.id from unsharded where unsharded.name in (select name from unsharded_a)", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select unsharded.id from unsharded where 1 != 1", + "Query": "select unsharded.id from unsharded where unsharded.name in (select name from unsharded_a)", + "Table": "unsharded, unsharded_a" + } +} + +# pullout sq after pullout sq +"select id from user where not id in (select user_extra.col from user_extra where user_extra.user_id = 42) and id in (select user_extra.col from user_extra where user_extra.user_id = 411)" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select id from user where not id in (select user_extra.col from user_extra where user_extra.user_id = 42) and id in (select user_extra.col from user_extra where user_extra.user_id = 411)", + "Instructions": { + "OperatorType": "Subquery", + "Variant": "PulloutIn", + "PulloutVars": [ + "__sq_has_values2", + "__sq2" + ], + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user_extra.col from user_extra where 1 != 1", + "Query": "select user_extra.col from user_extra where user_extra.user_id = 411", + "Table": "user_extra", + "Values": [ + "INT64(411)" + ], + "Vindex": "user_index" + }, + { + "OperatorType": "Subquery", + "Variant": "PulloutNotIn", + "PulloutVars": [ + "__sq_has_values1", + "__sq1" + ], + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user_extra.col from user_extra where 1 != 1", + "Query": "select user_extra.col from user_extra where user_extra.user_id = 42", + "Table": "user_extra", + "Values": [ + "INT64(42)" + ], + "Vindex": "user_index" + }, + { + "OperatorType": "Route", + "Variant": "SelectIN", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from user where 1 != 1", + "Query": "select id from user where (:__sq_has_values1 = 0 or id not in ::__sq1) and (:__sq_has_values2 = 1 and id in ::__vals)", + "Table": "user", + "Values": [ + ":__sq2" + ], + "Vindex": "user_index" + } + ] + } + ] + } +} + +# optimize ORs to IN route op codes #1 +"select col from user where name = 1 or name = 2" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select col from user where name = 1 or name = 2", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectIN", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col from user where 1 != 1", + "Query": "select col from user where name = 1 or name = 2", + "Table": "user", + "Values": [ + "(INT64(1), INT64(2))" + ], + "Vindex": "user_index" + } +} + +# optimize ORs to IN route op codes #2 +"select col from user where name = 1 or name = 2 or name = 3" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select col from user where name = 1 or name = 2 or name = 3", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectIN", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col from user where 1 != 1", + "Query": "select col from user where name = 1 or name = 2 or name = 3", + "Table": "user", + "Values": [ + "(INT64(1), INT64(2), INT64(3))" + ], + "Vindex": "user_index" + } +} + +# optimize ORs to IN route op codes #3 +"select col from user where (name = 1 or name = 2) or (name = 3 or name = 4)" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select col from user where (name = 1 or name = 2) or (name = 3 or name = 4)", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectIN", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col from user where 1 != 1", + "Query": "select col from user where name = 1 or name = 2 or (name = 3 or name = 4)", + "Table": "user", + "Values": [ + "(INT64(1), INT64(2), INT64(3), INT64(4))" + ], + "Vindex": "user_index" + } +} + +# non unique predicate on vindex +"select id from user where user.id > 5" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select id from user where user.id \u003e 5", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from user where 1 != 1", + "Query": "select id from user where user.id \u003e 5", + "Table": "user" + } +} + +# left join with where clause - should be handled by gen4 but still isn't +"select 0 from unsharded_a left join unsharded_b on unsharded_a.col = unsharded_b.col where coalesce(unsharded_b.col, 4) = 5" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select 0 from unsharded_a left join unsharded_b on unsharded_a.col = unsharded_b.col where coalesce(unsharded_b.col, 4) = 5", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select 0 from unsharded_a left join unsharded_b on unsharded_a.col = unsharded_b.col where 1 != 1", + "Query": "select 0 from unsharded_a left join unsharded_b on unsharded_a.col = unsharded_b.col where coalesce(unsharded_b.col, 4) = 5", + "Table": "unsharded_a,unsharded_b" + } +} + +# in subquery the id will be scoped to local table as there is no qualifier associated with it. +"select id from user where id in (select col from unsharded where col = id)" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select id from user where id in (select col from unsharded where col = id)", + "Instructions": { + "OperatorType": "Subquery", + "Variant": "PulloutIn", + "PulloutVars": [ + "__sq_has_values1", + "__sq1" + ], + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select col from unsharded where 1 != 1", + "Query": "select col from unsharded where col = id", + "Table": "SelectUnsharded" + }, + { + "OperatorType": "Route", + "Variant": "SelectIN", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from user where 1 != 1", + "Query": "select id from user where :__sq_has_values1 = 1 and id in ::__vals", + "Table": "user", + "Values": [ + ":__sq1" + ], + "Vindex": "user_index" + } + ] + } +} + +# filter on outer join should not be used for routing +"select user.col from user_extra left outer join user on user_extra.user_id = user.name WHERE user.name IS NULL" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select user.col from user_extra left outer join user on user_extra.user_id = user.name WHERE user.name IS NULL", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.col from user_extra left join user on user_extra.user_id = user.name where 1 != 1", + "Query": "select user.col from user_extra left join user on user_extra.user_id = user.name where user.name is null", + "Table": "user,user_extra" + } +} + +# cross-shard subquery in IN clause. +# Note the improved Underlying plan as SelectIN. +"select id from user where id in (select col from user)" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select id from user where id in (select col from user)", + "Instructions": { + "OperatorType": "Subquery", + "Variant": "PulloutIn", + "PulloutVars": [ + "__sq_has_values1", + "__sq1" + ], + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col from user where 1 != 1", + "Query": "select col from user", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "SelectIN", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from user where 1 != 1", + "Query": "select id from user where :__sq_has_values1 = 1 and id in ::__vals", + "Table": "user", + "Values": [ + ":__sq1" + ], + "Vindex": "user_index" + } + ] + } +} + +# correlated subquery with same keyspace +"select u.id from user as u where u.col in (select ue.user_id from user_extra as ue where ue.user_id = u.id)" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select u.id from user as u where u.col in (select ue.user_id from user_extra as ue where ue.user_id = u.id)", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select u.id from user as u where 1 != 1", + "Query": "select u.id from user as u where u.col in (select ue.user_id from user_extra as ue where ue.user_id = u.id)", + "Table": "user" + } +} + +# correlated subquery with different keyspace tables involved +"select id from user where id in (select col from unsharded where col = user.id)" +Gen3 skip +"unsupported: cross-shard correlated subquery" + +# correlated subquery merge-able into a route of a join tree +"select u.m from user_extra join user u where u.id in (select m2 from user where user.id = u.id) and u.id in (user_extra.col, 1)" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select u.m from user_extra join user u where u.id in (select m2 from user where user.id = u.id) and u.id in (user_extra.col, 1)", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "R:0", + "JoinVars": { + "user_extra_col": 0 + }, + "TableName": "user_extra_user", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user_extra.col from user_extra where 1 != 1", + "Query": "select user_extra.col from user_extra", + "Table": "user_extra" + }, + { + "OperatorType": "Route", + "Variant": "SelectIN", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select u.m from user as u where 1 != 1", + "Query": "select u.m from user as u where u.id in (select m2 from user where user.id = u.id) and u.id in ::__vals", + "Table": "user", + "Values": [ + "(:user_extra_col, INT64(1))" + ], + "Vindex": "user_index" + } + ] + } +} + +# The outer and second inner are SelectEqualUnique with same Vindex value, the first inner has different Vindex value +"select id from user where id = 5 and not id in (select user_extra.col from user_extra where user_extra.user_id = 4) and id in (select user_extra.col from user_extra where user_extra.user_id = 5)" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select id from user where id = 5 and not id in (select user_extra.col from user_extra where user_extra.user_id = 4) and id in (select user_extra.col from user_extra where user_extra.user_id = 5)", + "Instructions": { + "OperatorType": "Subquery", + "Variant": "PulloutNotIn", + "PulloutVars": [ + "__sq_has_values1", + "__sq1" + ], + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user_extra.col from user_extra where 1 != 1", + "Query": "select user_extra.col from user_extra where user_extra.user_id = 4", + "Table": "user_extra", + "Values": [ + "INT64(4)" + ], + "Vindex": "user_index" + }, + { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from user where 1 != 1", + "Query": "select id from user where id = 5 and (:__sq_has_values1 = 0 or id not in ::__sq1) and id in (select user_extra.col from user_extra where user_extra.user_id = 5)", + "Table": "user", + "Values": [ + "INT64(5)" + ], + "Vindex": "user_index" + } + ] + } +} + +# The outer and first inner are SelectEqualUnique with same Vindex value, the second inner has different Vindex value +"select id from user where id = 5 and not id in (select user_extra.col from user_extra where user_extra.user_id = 5) and id in (select user_extra.col from user_extra where user_extra.user_id = 4)" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select id from user where id = 5 and not id in (select user_extra.col from user_extra where user_extra.user_id = 5) and id in (select user_extra.col from user_extra where user_extra.user_id = 4)", + "Instructions": { + "OperatorType": "Subquery", + "Variant": "PulloutIn", + "PulloutVars": [ + "__sq_has_values2", + "__sq2" + ], + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user_extra.col from user_extra where 1 != 1", + "Query": "select user_extra.col from user_extra where user_extra.user_id = 4", + "Table": "user_extra", + "Values": [ + "INT64(4)" + ], + "Vindex": "user_index" + }, + { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from user where 1 != 1", + "Query": "select id from user where id = 5 and id not in (select user_extra.col from user_extra where user_extra.user_id = 5) and (:__sq_has_values2 = 1 and id in ::__sq2)", + "Table": "user", + "Values": [ + "INT64(5)" + ], + "Vindex": "user_index" + } + ] + } +} + +# SelectScatter with NOT EXISTS uncorrelated subquery +"select u1.col from user as u1 where not exists (select u2.name from user u2 where u2.id = 5)" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select u1.col from user as u1 where not exists (select u2.name from user u2 where u2.id = 5)", + "Instructions": { + "OperatorType": "Subquery", + "Variant": "PulloutExists", + "PulloutVars": [ + "__sq_has_values1" + ], + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from user as u2 where 1 != 1", + "Query": "select 1 from user as u2 where u2.id = 5 limit 1", + "Table": "user", + "Values": [ + "INT64(5)" + ], + "Vindex": "user_index" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select u1.col from user as u1 where 1 != 1", + "Query": "select u1.col from user as u1 where not :__sq_has_values1", + "Table": "user" + } + ] + } +} + +# SelectEqualUnique with uncorrelated subqueries +"select u1.col from user as u1 where u1.id = 5 and u1.name in (select u2.name from user u2 where u2.id = 5)" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select u1.col from user as u1 where u1.id = 5 and u1.name in (select u2.name from user u2 where u2.id = 5)", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select u1.col from user as u1 where 1 != 1", + "Query": "select u1.col from user as u1 where u1.id = 5 and u1.name in (select u2.name from user as u2 where u2.id = 5)", + "Table": "user", + "Values": [ + "INT64(5)" + ], + "Vindex": "user_index" + } +} + +# SelectEqualUnique with EXISTS uncorrelated subquery +"select u1.col from user as u1 where u1.id = 5 and exists (select u2.name from user u2 where u2.id = 5)" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select u1.col from user as u1 where u1.id = 5 and exists (select u2.name from user u2 where u2.id = 5)", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select u1.col from user as u1 where 1 != 1", + "Query": "select u1.col from user as u1 where u1.id = 5 and exists (select 1 from user as u2 where u2.id = 5 limit 1)", + "Table": "user", + "Values": [ + "INT64(5)" + ], + "Vindex": "user_index" + } +} + +# SelectDBA with uncorrelated subqueries +"select t.table_schema from information_schema.tables as t where t.table_schema in (select c.column_name from information_schema.columns as c)" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select t.table_schema from information_schema.tables as t where t.table_schema in (select c.column_name from information_schema.columns as c)", + "Instructions": { + "OperatorType": "Route", + "Variant": "DBA", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select t.table_schema from information_schema.`tables` as t where 1 != 1", + "Query": "select t.table_schema from information_schema.`tables` as t where t.table_schema in (select c.column_name from information_schema.`columns` as c)", + "Table": "information_schema.`tables`" + } +} + +# Don't pick a vindex for an IS NULL predicate if it's a lookup vindex +"select id from music where id is null and user_id in (1,2)" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select id from music where id is null and user_id in (1,2)", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectIN", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from music where 1 != 1", + "Query": "select id from music where id is null and user_id in (::__vals)", + "Table": "music", + "Values": [ + "(INT64(1), INT64(2))" + ], + "Vindex": "user_index" + } +} + +# IN clause: LHS is neither column nor composite tuple +"select Id from user where 1 in ('aa', 'bb')" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select Id from user where 1 in ('aa', 'bb')", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select Id from user where 1 != 1", + "Query": "select Id from user where 1 in ('aa', 'bb')", + "Table": "user" + } +} \ No newline at end of file diff --git a/src/test/resources/plan/filter_cases_unsupported.txt b/src/test/resources/plan/filter_cases_unsupported.txt index e1b15e5..5d1bc35 100644 --- a/src/test/resources/plan/filter_cases_unsupported.txt +++ b/src/test/resources/plan/filter_cases_unsupported.txt @@ -140,3 +140,329 @@ "Vindex": "user_index" } } + +# this query lead to a nil pointer error +"select user.id from user left join user_extra on user.col = user_extra.col where foo(user_extra.foobar)" +Gen3 skip +"expr cannot be translated, not supported: foo(user_extra.foobar)" + +# solving LIKE query with a CFC prefix vindex +"select c2 from cfc_vindex_col where c1 like 'A%'" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select c2 from cfc_vindex_col where c1 like 'A%'", + "Instructions": { + "OperatorType": "Route", + "Variant": "Equal", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select c2 from cfc_vindex_col where 1 != 1", + "Query": "select c2 from cfc_vindex_col where c1 like 'A%'", + "Table": "cfc_vindex_col", + "Values": [ + "VARCHAR(\"A%\")" + ], + "Vindex": "cfc" + } +} + +# v3 takes cola_map, gen4 takes colb_colc_map, may be based on map key ordering +"select * from multicolvin where column_a = 3 and column_b = 1" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select * from multicolvin where column_a = 3 and column_b = 1", + "Instructions": { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select * from multicolvin where 1 != 1", + "Query": "select * from multicolvin where column_a = 3 and column_b = 1", + "Table": "multicolvin", + "Values": [ + "INT64(1)" + ], + "Vindex": "colb_colc_map" + } +} + +# uses vindex colb_colc_map +"select * from multicolvin where column_b = 1 and column_c = 2 and column_a = 3" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select * from multicolvin where column_b = 1 and column_c = 2 and column_a = 3", + "Instructions": { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select * from multicolvin where 1 != 1", + "Query": "select * from multicolvin where column_b = 1 and column_c = 2 and column_a = 3", + "Table": "multicolvin", + "Values": [ + "INT64(1)" + ], + "Vindex": "colb_colc_map" + } +} + +# should use colb_colc_map as first column of the vindex is present in predicate +"select * from multicolvin where column_b = 1" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select * from multicolvin where column_b = 1", + "Instructions": { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select * from multicolvin where 1 != 1", + "Query": "select * from multicolvin where column_b = 1", + "Table": "multicolvin", + "Values": [ + "INT64(1)" + ], + "Vindex": "colb_colc_map" + } +} + +# should only use first column of the vindex colb_colc_map +"select * from multicolvin where column_b = 1 and column_c = 2" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select * from multicolvin where column_b = 1 and column_c = 2", + "Instructions": { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select * from multicolvin where 1 != 1", + "Query": "select * from multicolvin where column_b = 1 and column_c = 2", + "Table": "multicolvin", + "Values": [ + "INT64(1)" + ], + "Vindex": "colb_colc_map" + } +} + +# not supported transitive closures with equality inside of an OR +"select id from user, user_extra where user.id = user_extra.col and (user_extra.col = user_extra.user_id or user_extra.col2 = user_extra.name)" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select id from user, user_extra where user.id = user_extra.col and (user_extra.col = user_extra.user_id or user_extra.col2 = user_extra.name)", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "R:0", + "JoinVars": { + "user_extra_col": 0 + }, + "TableName": "user_extra_user", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user_extra.col from user_extra where 1 != 1", + "Query": "select user_extra.col from user_extra where user_extra.col = user_extra.user_id or user_extra.col2 = user_extra.name", + "Table": "user_extra" + }, + { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from user where 1 != 1", + "Query": "select id from user where user.id = :user_extra_col", + "Table": "user", + "Values": [ + ":user_extra_col" + ], + "Vindex": "user_index" + } + ] + } +} + +# left join where clauses where we can optimize into an inner join +"select user.id from user left join user_extra on user.col = user_extra.col where user_extra.foobar = 5" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select user.id from user left join user_extra on user.col = user_extra.col where user_extra.foobar = 5", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:1", + "JoinVars": { + "user_col": 0 + }, + "TableName": "user_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.col, user.id from user where 1 != 1", + "Query": "select user.col, 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.foobar = 5 and user_extra.col = :user_col", + "Table": "user_extra" + } + ] + } +} + +# join with transitive closures +"select id from user, user_extra where user.id = user_extra.col and user_extra.col = user_extra.user_id" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select id from user, user_extra where user.id = user_extra.col and user_extra.col = user_extra.user_id", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from user, user_extra where 1 != 1", + "Query": "select id from user, user_extra where user_extra.col = user_extra.user_id and user.id = user_extra.col", + "Table": "user, user_extra" + } +} + +# filter after outer join +"select user.id from user left join user_extra on user.col = user_extra.col where user_extra.id is null" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select user.id from user left join user_extra on user.col = user_extra.col where user_extra.id is null", + "Instructions": { + "OperatorType": "SimpleProjection", + "Columns": [ + 1 + ], + "Inputs": [ + { + "OperatorType": "Filter", + "Predicate": "user_extra.id is null", + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "LeftJoin", + "JoinColumnIndexes": "R:0,L:1", + "JoinVars": { + "user_col": 0 + }, + "TableName": "user_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.col, user.id from user where 1 != 1", + "Query": "select user.col, user.id from user", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user_extra.id from user_extra where 1 != 1", + "Query": "select user_extra.id from user_extra where user_extra.col = :user_col", + "Table": "user_extra" + } + ] + } + ] + } + ] + } +} + +# SelectReference with uncorrelated subqueries +"select ref.col from ref where ref.col in (select ref.col from ref)" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select ref.col from ref where ref.col in (select ref.col from ref)", + "Instructions": { + "OperatorType": "Route", + "Variant": "Reference", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select ref.col from ref where 1 != 1", + "Query": "select ref.col from ref where ref.col in (select ref.col from ref)", + "Table": "ref" + } +} + +# Single table equality route with unsigned value +"select id from user where name = 18446744073709551615" +{ + "QueryType": "SELECT", + "Original": "select id from user where name = 18446744073709551615", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from user where 1 != 1", + "Query": "select id from user where name = 18446744073709551615", + "Table": "user", + "Values": [ + "UINT64(18446744073709551615)" + ], + "Vindex": "name_user_map" + } +} +Gen4 plan same as above + + + + diff --git a/src/test/resources/plan/from_cases.txt b/src/test/resources/plan/from_cases.txt index eb7b3f0..5695600 100644 --- a/src/test/resources/plan/from_cases.txt +++ b/src/test/resources/plan/from_cases.txt @@ -15,6 +15,7 @@ "Table": "user" } } +Gen4 plan same as above # Single table unsharded "select col from unsharded" @@ -33,6 +34,7 @@ "Table": "unsharded" } } +Gen4 plan same as above # Single table unsharded and no metadata "select col from unsharded_no_metadata" @@ -69,6 +71,7 @@ "Table": "foo" } } +Gen4 plan same as above # Single information_schema query "select col from information_schema.foo" @@ -86,6 +89,21 @@ "Query": "select col from information_schema.foo" } } +{ + "QueryType": "SELECT", + "Original": "select col from information_schema.foo", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectDBA", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select col from information_schema.foo where 1 != 1", + "Query": "select col from information_schema.foo", + "Table": "information_schema.`foo`" + } +} # access to unqualified column names in information_schema "select * from information_schema.a where b=10" @@ -103,6 +121,21 @@ "Query": "select * from information_schema.a where b = 10" } } +{ + "QueryType": "SELECT", + "Original": "select * from information_schema.a where b=10", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectDBA", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select * from information_schema.a where 1 != 1", + "Query": "select * from information_schema.a where b = 10", + "Table": "information_schema.`a`" + } +} # access to qualified column names in information_schema "select * from information_schema.a where information_schema.a.b=10" @@ -120,6 +153,21 @@ "Query": "select * from information_schema.a where information_schema.a.b = 10" } } +{ + "QueryType": "SELECT", + "Original": "select * from information_schema.a where information_schema.a.b=10", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectDBA", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select * from information_schema.a where 1 != 1", + "Query": "select * from information_schema.a where a.b = 10", + "Table": "information_schema.`a`" + } +} # Multi-table unsharded "select m1.col from unsharded as m1 join unsharded as m2" @@ -138,6 +186,7 @@ "Table": "unsharded" } } +Gen4 plan same as above # ',' join unsharded "select u1.a, u2.a from unsharded u1, unsharded u2" @@ -156,6 +205,7 @@ "Table": "unsharded" } } +Gen4 plan same as above # ',' join information_schema "select * from information_schema.a, information_schema.b" @@ -173,6 +223,21 @@ "Query": "select * from information_schema.a, information_schema.b" } } +{ + "QueryType": "SELECT", + "Original": "select * from information_schema.a, information_schema.b", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectDBA", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select * from information_schema.a, information_schema.b where 1 != 1", + "Query": "select * from information_schema.a, information_schema.b", + "Table": "information_schema.`a`,information_schema.`b`" + } +} # ',' 3-way join unsharded "select u1.a, u2.a from unsharded u1, unsharded u2, unsharded u3" @@ -191,6 +256,7 @@ "Table": "unsharded" } } +Gen4 plan same as above # Left join, single chunk "select m1.col from unsharded as m1 left join unsharded as m2 on m1.a=m2.b" @@ -209,6 +275,7 @@ "Table": "unsharded" } } +Gen4 plan same as above # Right join "select m1.col from unsharded as m1 right join unsharded as m2 on m1.a=m2.b" @@ -227,6 +294,21 @@ "Table": "unsharded" } } +{ + "QueryType": "SELECT", + "Original": "select m1.col from unsharded as m1 right join unsharded as m2 on m1.a=m2.b", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select m1.col from unsharded as m1 right join unsharded as m2 on m1.a = m2.b where 1 != 1", + "Query": "select m1.col from unsharded as m1 right join unsharded as m2 on m1.a = m2.b", + "Table": "unsharded" + } +} # Right join with a join LHS "select m1.col from unsharded as m1 join unsharded as m2 right join unsharded as m3 on m1.a=m2.b" @@ -245,6 +327,21 @@ "Table": "unsharded" } } +{ + "QueryType": "SELECT", + "Original": "select m1.col from unsharded as m1 join unsharded as m2 right join unsharded as m3 on m1.a=m2.b", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select m1.col from unsharded as m1 join unsharded as m2 right join unsharded as m3 on m1.a = m2.b where 1 != 1", + "Query": "select m1.col from unsharded as m1 join unsharded as m2 right join unsharded as m3 on m1.a = m2.b", + "Table": "unsharded" + } +} # Straight-join "select m1.col from unsharded as m1 straight_join unsharded as m2" @@ -263,6 +360,21 @@ "Table": "unsharded" } } +{ + "QueryType": "SELECT", + "Original": "select m1.col from unsharded as m1 straight_join unsharded as m2", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select m1.col from unsharded as m1 join unsharded as m2 where 1 != 1", + "Query": "select m1.col from unsharded as m1 join unsharded as m2", + "Table": "unsharded" + } +} # Multi-table, multi-chunk "select music.col from user join music" @@ -300,6 +412,7 @@ ] } } +Gen4 plan same as above # ',' join "select music.col from user, music" @@ -337,6 +450,7 @@ ] } } +Gen4 plan same as above # mergeable sharded join on unique vindex "select user.col from user join user_extra on user.name = user_extra.user_id" @@ -355,6 +469,21 @@ "Table": "user" } } +{ + "QueryType": "SELECT", + "Original": "select user.col from user join user_extra on user.name = user_extra.user_id", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.col from user, user_extra where 1 != 1", + "Query": "select user.col from user, user_extra where user.name = 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.name = user_extra.user_id)" @@ -373,6 +502,21 @@ "Table": "user" } } +{ + "QueryType": "SELECT", + "Original": "select user.col from user join user_extra on (user.name = user_extra.user_id)", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.col from user, user_extra where 1 != 1", + "Query": "select user.col from user, user_extra where user.name = 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.name = user_extra.user_id" @@ -391,6 +535,21 @@ "Table": "user" } } +{ + "QueryType": "SELECT", + "Original": "select user.col from user join user_extra on user.col between 1 and 2 and user.name = user_extra.user_id", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "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.name = 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.name" @@ -409,6 +568,21 @@ "Table": "user" } } +{ + "QueryType": "SELECT", + "Original": "select user.col from user join user_extra on user_extra.user_id = user.name", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "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.name", + "Table": "user,user_extra" + } +} # mergeable sharded join on unique vindex, and condition "select user.col from user join user_extra on user.name = 5 and user.name = user_extra.user_id" @@ -431,6 +605,21 @@ "Vindex": "user_index" } } +{ + "QueryType": "SELECT", + "Original": "select user.col from user join user_extra on user.name = 5 and user.name = user_extra.user_id", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.col from user, user_extra where 1 != 1", + "Query": "select user.col from user, user_extra where user.name = 5 and user.name = user_extra.user_id", + "Table": "user,user_extra" + } +} # sharded join on unique vindex, inequality "select user.col from user join user_extra on user.name < user_extra.user_id" @@ -468,6 +657,40 @@ ] } } +{ + "QueryType": "SELECT", + "Original": "select user.col from user join user_extra on user.name \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.name, user.col from user where 1 != 1", + "Query": "select user.name, 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_name \u003c user_extra.user_id", + "Table": "user_extra" + } + ] + } +} # sharded join, non-col reference RHS "select user.col from user join user_extra on user.name = 5" @@ -509,6 +732,7 @@ ] } } +Gen4 plan same as above # sharded join, non-col reference LHS "select user.col from user join user_extra on 5 = user.name" @@ -550,12 +774,13 @@ ] } } +Gen4 plan same as above # 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.name = user_extra.col" { "QueryType": "SELECT", - "Original": "select user.col from user join user_extra on user.id = user_extra.col", + "Original": "select user.col from user join user_extra on user.name = user_extra.col", "Instructions": { "OperatorType": "Join", "Variant": "Join", @@ -569,8 +794,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select user.col, user.id from user where 1 != 1", - "Query": "select user.col, user.id from user", + "FieldQuery": "select user.col, user.name from user where 1 != 1", + "Query": "select user.col, user.name from user", "Table": "user" }, { @@ -581,8 +806,42 @@ "Sharded": true }, "FieldQuery": "select 1 from user_extra where 1 != 1", - "Query": "select 1 from user_extra where user_extra.col = :user_id", + "Query": "select 1 from user_extra where user_extra.col = :user_name", + "Table": "user_extra" + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select user.col from user join user_extra on user.name = user_extra.col", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": [1], + "TableName": "user_extra_user", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user_extra.col from user_extra where 1 != 1", + "Query": "select user_extra.col from user_extra", "Table": "user_extra" + }, + { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.col from user where 1 != 1", + "Query": "select user.col from user where user.name = :user_extra_col", + "Table": "user" } ] } @@ -605,6 +864,21 @@ "Table": "user" } } +{ + "QueryType": "SELECT", + "Original": "select user.col from user join user_extra on user.NAME = user_extra.User_Id", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.col from user, user_extra where 1 != 1", + "Query": "select user.col from user, user_extra where user.NAME = user_extra.User_Id", + "Table": "user,user_extra" + } +} # database call in ON clause. # The on clause is weird because the substitution must even for root expressions. @@ -624,6 +898,7 @@ "Table": "unsharded" } } +Gen4 plan same as above # index hints, make sure they are not stripped. "select user.col from user use index(a)" @@ -642,6 +917,7 @@ "Table": "user" } } +Gen4 plan same as above # subquery "select id from (select id, col from user where name = 5) as t" @@ -664,6 +940,7 @@ "Vindex": "hash" } } +Gen4 plan same as above # subquery with join "select t.name from ( select name from user where name = 5 ) as t join user_extra on t.name = user_extra.user_id" @@ -686,6 +963,7 @@ "Vindex": "hash" } } +Gen4 plan same as above # subquery with join, and aliased references "select t.name from ( select user.name from user where user.name = 5 ) as t join user_extra on t.name = user_extra.user_id" @@ -708,10 +986,12 @@ "Vindex": "user_index" } } +Gen4 plan same as above # subquery with join, duplicate columns "select t.name from (select user.name, name from user where user.name = 5) as t join user_extra on t.name = user_extra.user_id" "duplicate column aliases: name" +Gen4 plan same as above # subquery in RHS of join "select t.name from user_extra join ( select name from user where name = 5 ) as t on t.name = user_extra.user_id" @@ -730,6 +1010,7 @@ "Table": "user_extra" } } +Gen4 plan same as above # subquery in FROM with cross-shard join "select t.name from ( select name from user where name = 5 ) as t join user_extra on t.name = user_extra.col" @@ -771,6 +1052,7 @@ ] } } +Gen4 plan same as above # merge subqueries with single-shard routes "select u.col, e.col from ( select col from user where name = 5 ) as u join ( select col from user_extra where user_id = 5 ) as e" @@ -793,6 +1075,7 @@ "Vindex": "hash" } } +Gen4 plan same as above # join of information_schema with normal table "select unsharded.foo from information_schema.a join unsharded" @@ -829,51 +1112,119 @@ ] } } - -# join of normal table with information_schema -"select unsharded.foo from unsharded join information_schema.a" { "QueryType": "SELECT", - "Original": "select unsharded.foo from unsharded join information_schema.a", + "Original": "select unsharded.foo from information_schema.a join unsharded", "Instructions": { "OperatorType": "Join", "Variant": "Join", - "JoinColumnIndexes": [-1], - "TableName": "unsharded_", + "JoinColumnIndexes": [1], + "TableName": "information_schema.`a`_unsharded", "Inputs": [ { "OperatorType": "Route", - "Variant": "SelectUnsharded", + "Variant": "SelectDBA", "Keyspace": { "Name": "main", "Sharded": false }, - "FieldQuery": "select unsharded.foo from unsharded where 1 != 1", - "Query": "select unsharded.foo from unsharded", - "Table": "unsharded" + "FieldQuery": "select 1 from information_schema.a where 1 != 1", + "Query": "select 1 from information_schema.a", + "Table": "information_schema.`a`" }, { "OperatorType": "Route", - "Variant": "SelectDBA", + "Variant": "SelectUnsharded", "Keyspace": { "Name": "main", "Sharded": false }, - "FieldQuery": "select 1 from information_schema.a where 1 != 1", - "Query": "select 1 from information_schema.a" + "FieldQuery": "select unsharded.foo from unsharded where 1 != 1", + "Query": "select unsharded.foo from unsharded", + "Table": "unsharded" } ] } } -# wire-up on within cross-shard subquery -"select t.id from ( select user.id, user.col1 from user join user_extra on user_extra.col = user.col ) as t" +# join of normal table with information_schema +"select unsharded.foo from unsharded join information_schema.a" { "QueryType": "SELECT", - "Original": "select t.id from ( select user.id, user.col1 from user join user_extra on user_extra.col = user.col ) as t", + "Original": "select unsharded.foo from unsharded join information_schema.a", "Instructions": { - "OperatorType": "Subquery", - "Columns": [ + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": [-1], + "TableName": "unsharded_", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select unsharded.foo from unsharded where 1 != 1", + "Query": "select unsharded.foo from unsharded", + "Table": "unsharded" + }, + { + "OperatorType": "Route", + "Variant": "SelectDBA", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select 1 from information_schema.a where 1 != 1", + "Query": "select 1 from information_schema.a" + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select unsharded.foo from unsharded join information_schema.a", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": [-1], + "TableName": "unsharded_information_schema.`a`", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select unsharded.foo from unsharded where 1 != 1", + "Query": "select unsharded.foo from unsharded", + "Table": "unsharded" + }, + { + "OperatorType": "Route", + "Variant": "SelectDBA", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select 1 from information_schema.a where 1 != 1", + "Query": "select 1 from information_schema.a", + "Table": "information_schema.`a`" + } + ] + } +} + +# wire-up on within cross-shard subquery +"select t.id from ( select user.id, user.col1 from user join user_extra on user_extra.col = user.col ) as t" +{ + "QueryType": "SELECT", + "Original": "select t.id from ( select user.id, user.col1 from user join user_extra on user_extra.col = user.col ) as t", + "Instructions": { + "OperatorType": "Subquery", + "Columns": [ 0 ], "Inputs": [ @@ -910,6 +1261,7 @@ ] } } +Gen4 plan same as above # subquery with join primitive (FROM) "select id, t.id from ( select user.id from user join user_extra ) as t" @@ -956,28 +1308,1457 @@ ] } } +Gen4 plan same as above # first expression fails for ',' join (code coverage: ensure error is returned) "select user.foo.col from user.foo, user" "table foo not found" +Gen4 plan same as above # duplicate symbols "select user.id from user join user" "duplicate symbol: user" +"Not unique table/alias: 'user'==='user'" # duplicate symbols for merging routes "select user.id from user join user_extra user on user.id = user.user_id" "duplicate symbol: user" +"Not unique table/alias: 'user'==='user'" # non-existent table on left of join "select c from t join user" "table t not found" +Gen4 plan same as above # non-existent table on right of join "select c from user join t" "table t not found" +Gen4 plan same as above # non-existent table # success when t in sharded keyspace "select c from t" "table t not found" +Gen4 plan same as above + +# query with parens is planned correctly +"select m1.col from (unsharded as m1, unsharded as m2)" +{ + "QueryType": "SELECT", + "Original": "select m1.col from (unsharded as m1, 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" + } +} +Gen4 plan same as above + +# gen4 - optimise plan by merging user_extra and music first, and then querying for user info +"select 1 from user u join user_extra ue on ue.id = u.id join music m on m.user_id = ue.user_id" +{ + "QueryType":"SELECT", + "Original":"select 1 from user u join user_extra ue on ue.id = u.id join music m on m.user_id = ue.user_id", + "Instructions":{ + "OperatorType":"Join", + "Variant":"Join", + "JoinColumnIndexes":[-1], + "TableName":"user_user_extra_music", + "Inputs":[ + { + "OperatorType":"Route", + "Variant":"SelectEqualUnique", + "Keyspace":{ + "Name":"user", + "Sharded":true + }, + "FieldQuery":"select 1 from music as m where 1 != 1", + "Query":"select 1 from music as m where m.user_id = :ue_user_id", + "Table":"music", + "Values":[":ue_user_id"] + }] + } +} +{ + "QueryType": "SELECT", + "Original": "select 1 from user u join user_extra ue on ue.id = u.id join music m on m.user_id = ue.user_id", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:1", + "JoinVars": { + "ue_id": 0 + }, + "TableName": "music, user_extra_`user`", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select ue.id, 1 from user_extra as ue, music as m where 1 != 1", + "Query": "select ue.id, 1 from user_extra as ue, music as m where m.user_id = ue.user_id", + "Table": "music, user_extra" + }, + { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from `user` as u where 1 != 1", + "Query": "select 1 from `user` as u where u.id = :ue_id", + "Table": "`user`", + "Values": [ + ":ue_id" + ], + "Vindex": "user_index" + } + ] + } +} + +# join column selected as alias +"SELECT u.id as uid, ue.id as ueid FROM user u join user_extra ue where u.id = ue.id" +{ + "QueryType": "SELECT", + "Original": "SELECT u.id as uid, ue.id as ueid FROM user u join user_extra ue where u.id = ue.id", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": [-1,1], + "TableName": "user_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select u.id as uid from user as u where 1 != 1", + "Query": "select u.id as uid from user as u", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select ue.id as ueid from user_extra as ue where 1 != 1", + "Query": "select ue.id as ueid from user_extra as ue where ue.id = :u_id", + "Table": "user_extra" + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "SELECT u.id as uid, ue.id as ueid FROM user u join user_extra ue where u.id = ue.id", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "R:0,L:1", + "JoinVars": { + "ue_id": 0 + }, + "TableName": "user_extra_`user`", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select ue.id, ue.id as ueid from user_extra as ue where 1 != 1", + "Query": "select ue.id, ue.id as ueid from user_extra as ue", + "Table": "user_extra" + }, + { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select u.id as uid from `user` as u where 1 != 1", + "Query": "select u.id as uid from `user` as u where u.id = :ue_id", + "Table": "`user`", + "Values": [ + ":ue_id" + ], + "Vindex": "user_index" + } + ] + } +} + +# alias on column from derived table. TODO: to support alias in SimpleProjection engine primitive. +"select a as k from (select count(*) as a from user) t" +{ + "QueryType": "SELECT", + "Original": "select a as k from (select count(*) as a from user) t", + "Instructions": { + "OperatorType": "Subquery", + "Columns": [0], + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant":"Ordered", + "Aggregates":"count(0)", + "Distinct":false, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery":"select count(*) as a from user where 1 != 1", + "Query":"select count(*) as a from user", + "Table":"user" + } + ] + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select a as k from (select count(*) as a from user) t", + "Instructions": { + "OperatorType": "SimpleProjection", + "Columns": [ + 0 + ], + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Scalar", + "Aggregates": "sum_count_star(0) AS a", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select count(*) as a from `user` where 1 != 1", + "Query": "select count(*) as a from `user`", + "Table": "`user`" + } + ] + } + ] + } +} + +# select star from derived table on expandable and unsharded table +"select u.* from (select * from unsharded) u" +{ + "QueryType": "SELECT", + "Original": "select u.* from (select * from unsharded) u", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select u.* from ( select * from unsharded where 1 != 1 ) u where 1 != 1", + "Query": "select u.* from ( select * from unsharded ) u", + "Table": "unsharded" + } +} +Gen4 plan same as above + +# filtering on a cross-shard derived table +"select id from (select user.id, user.col from user join user_extra) as t where id=5" +"unsupported: filtering on results of cross-shard subquery" +{ + "QueryType": "SELECT", + "Original": "select id from (select user.id, user.col from user join user_extra) as t where id=5", + "Instructions": { + "OperatorType": "SimpleProjection", + "Columns": [ + 0 + ], + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:0,L:1", + "TableName": "`user`_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select `user`.id, `user`.col from `user` where 1 != 1", + "Query": "select `user`.id, `user`.col from `user` where `user`.id = 5", + "Table": "`user`", + "Values": [ + "INT64(5)" + ], + "Vindex": "user_index" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from user_extra where 1 != 1", + "Query": "select 1 from user_extra", + "Table": "user_extra" + } + ] + } + ] + } +} + +# expression on a cross-shard derived table +"select id+1 from (select user.id, user.col from user join user_extra) as t" +"unsupported: expression on results of a cross-shard subquery" +{ + "QueryType": "SELECT", + "Original": "select id+1 from (select user.id, user.col from user join user_extra) as t", + "Instructions": { + "OperatorType": "SimpleProjection", + "Columns": [ + 2 + ], + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:0,L:1,L:2", + "TableName": "`user`_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select `user`.id, `user`.col, `user`.id + 1 from `user` where 1 != 1", + "Query": "select `user`.id, `user`.col, `user`.id + 1 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", + "Table": "user_extra" + } + ] + } + ] + } +} + +# join on int columns +"select u.id from user as u join user as uu on u.intcol = uu.intcol" +{ + "QueryType": "SELECT", + "Original": "select u.id from user as u join user as uu on u.intcol = uu.intcol", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": [-1], + "TableName": "user_user", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select u.id, u.intcol from user as u where 1 != 1", + "Query": "select u.id, u.intcol from user as u", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from user as uu where 1 != 1", + "Query": "select 1 from user as uu where uu.intcol = :u_intcol", + "Table": "user" + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select u.id from user as u join user as uu on u.intcol = uu.intcol", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:1", + "JoinVars": { + "u_intcol": 0 + }, + "TableName": "user_user", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select u.intcol, u.id from user as u where 1 != 1", + "Query": "select u.intcol, u.id from user as u", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from user as uu where 1 != 1", + "Query": "select 1 from user as uu where uu.intcol = :u_intcol", + "Table": "user" + } + ] + } +} + +# left join where clauses #2 +"select user.id from user left join user_extra on user.col = user_extra.col where coalesce(user_extra.col, 4) = 5" +"unsupported: cross-shard left join and where clause" +{ + "QueryType": "SELECT", + "Original": "select user.id from user left join user_extra on user.col = user_extra.col where coalesce(user_extra.col, 4) = 5", + "Instructions": { + "OperatorType": "SimpleProjection", + "Columns": [ + 1 + ], + "Inputs": [ + { + "OperatorType": "Filter", + "Predicate": "coalesce(user_extra.col, 4) = 5", + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "LeftJoin", + "JoinColumnIndexes": [-1,2], + "TableName": "`user`_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select `user`.col, `user`.id from `user` where 1 != 1", + "Query": "select `user`.col, `user`.id from `user`", + "Table": "`user`" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user_extra.col from user_extra where 1 != 1", + "Query": "select user_extra.col from user_extra where user_extra.col = :user_col", + "Table": "user_extra" + } + ] + } + ] + } + ] + } +} + +# dont merge unsharded tables from different keyspaces +"select 1 from main.unsharded join main_2.unsharded_tab" +{ + "QueryType":"SELECT", + "Original":"select 1 from main.unsharded join main_2.unsharded_tab", + "Instructions":{ + "OperatorType":"Route", + "Variant":"SelectUnsharded", + "Keyspace":{ + "Name":"main", + "Sharded":false + }, + "FieldQuery":"select 1 from main.unsharded join main_2.unsharded_tab where 1 != 1", + "Query":"select 1 from main.unsharded join main_2.unsharded_tab", + "Table":"unsharded", + "JoinColumnIndexes":null, + "TableName":null + } +} +Gen4 plan same as above + +# single unsharded keyspace with derived table +"select col from (select col from unsharded join unsharded_b) as u join unsharded_a ua limit 1" +{ + "QueryType": "SELECT", + "Original": "select col from (select col from unsharded join unsharded_b) as u join unsharded_a ua limit 1", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select col from ( select col from unsharded join unsharded_b where 1 != 1 ) u join unsharded_a as ua where 1 != 1", + "Query": "select col from ( select col from unsharded join unsharded_b ) u join unsharded_a as ua limit 1", + "Table": "unsharded" + } +} +{ + "QueryType": "SELECT", + "Original": "select col from (select col from unsharded join unsharded_b) as u join unsharded_a ua limit 1", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select col from (select col from unsharded join unsharded_b where 1 != 1) as u join unsharded_a as ua where 1 != 1", + "Query": "select col from (select col from unsharded join unsharded_b) as u join unsharded_a as ua limit 1", + "Table": "unsharded, unsharded_a, unsharded_b" + } +} + +# left join with expressions +"select user_extra.col+1 from user left join user_extra on user.col = user_extra.col" +"unsupported: cross-shard left join and column expressions" +{ + "QueryType": "SELECT", + "Original": "select user_extra.col+1 from user left join user_extra on user.col = user_extra.col", + "Instructions": { + "OperatorType": "Join", + "Variant": "LeftJoin", + "JoinColumnIndexes": [1], + "TableName": "user_user_extra", + "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": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user_extra.col + 1 from user_extra where 1 != 1", + "Query": "select user_extra.col + 1 from user_extra where user_extra.col = :user_col", + "Table": "user_extra" + } + ] + } +} + +# left join with expressions, with three-way join (different code path) +"select user.id, user_extra.col+1 from user left join user_extra on user.col = user_extra.col join user_extra e" +"unsupported: cross-shard left join and column expressions" +{ + "QueryType": "SELECT", + "Original": "select user.id, user_extra.col+1 from user left join user_extra on user.col = user_extra.col join user_extra e", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": [-1,2], + "TableName": "`user`_user_extra_user_extra", + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "LeftJoin", + "JoinColumnIndexes": [-2,1], + "TableName": "`user`_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select `user`.col, `user`.id from `user` where 1 != 1", + "Query": "select `user`.col, `user`.id from `user`", + "Table": "`user`" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user_extra.col + 1 from user_extra where 1 != 1", + "Query": "select user_extra.col + 1 from user_extra where user_extra.col = :user_col", + "Table": "user_extra" + } + ] + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from user_extra as e where 1 != 1", + "Query": "select 1 from user_extra as e", + "Table": "user_extra" + } + ] + } +} + +# left join with expressions coming from both sides +"select user.foo+user_extra.col+1 from user left join user_extra on user.col = user_extra.col" +"unsupported: cross-shard left join and column expressions" +{ + "QueryType": "SELECT", + "Original": "select user.foo+user_extra.col+1 from user left join user_extra on user.col = user_extra.col", + "Instructions": { + "OperatorType": "Join", + "Variant": "LeftJoin", + "JoinColumnIndexes": [1], + "TableName": "user_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.col, user.foo from user where 1 != 1", + "Query": "select user.col, user.foo from user", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select :user_foo + user_extra.col + 1 from user_extra where 1 != 1", + "Query": "select :user_foo + user_extra.col + 1 from user_extra where user_extra.col = :user_col", + "Table": "user_extra" + } + ] + } +} + +# Do not rewrite derived expressions when the derived table is merged with the outer +"select col1, count(*) from (select colC+colD as col1 from user) as tbl group by col1" +{ + "QueryType": "SELECT", + "Original": "select col1, count(*) from (select colC+colD as col1 from user) as tbl group by col1", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "count(1)", + "Distinct":false, + "GroupBy": "0", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col1, count(*) from ( select colC + colD as col1 from user where 1 != 1 ) tbl where 1 != 1 group by col1", + "Query": "select col1, count(*) from ( select colC + colD as col1 from user ) tbl group by col1 order by col1 asc", + "Table": "user" + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select col1, count(*) from (select colC+colD as col1 from user) as tbl group by col1", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count_star(1) AS count(*)", + "GroupBy": "(0|2)", + "ResultColumns": 2, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col1, count(*), weight_string(col1) from (select colC + colD as col1 from user where 1 != 1) as tbl where 1 != 1 group by col1, weight_string(col1)", + "OrderBy": "(0|2) ASC", + "Query": "select col1, count(*), weight_string(col1) from (select colC + colD as col1 from user) as tbl group by col1, weight_string(col1) order by col1 asc", + "Table": "user" + } + ] + } +} + +# Join with cross-shard derived table on rhs - push down join predicate to derived table +"select t.col1 from unsharded_a ua join (select user.id, user.col1 from user join user_extra) as t on t.id = ua.id" +{ + "QueryType": "SELECT", + "Original": "select t.col1 from unsharded_a ua join (select user.id, user.col1 from user join user_extra) as t on t.id = ua.id", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery":"select t.col1 from unsharded_a as ua join ( select user.id, user.col1 from user join user_extra where 1 != 1 ) t on t.id = ua.id where 1 != 1", + "Query":"select t.col1 from unsharded_a as ua join ( select user.id, user.col1 from user join user_extra ) t on t.id = ua.id", + "Table": "unsharded_a" + } +} +Gen4 plan same as above + +# Join with cross-shard derived table on rhs +"select t.col1 from unsharded_a ua join (select user.id, user.col1 from user join user_extra) as t" +{ + "QueryType": "SELECT", + "Original": "select t.col1 from unsharded_a ua join (select user.id, user.col1 from user join user_extra) as t", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery":"select t.col1 from unsharded_a as ua join ( select user.id, user.col1 from user join user_extra where 1 != 1 ) t where 1 != 1", + "Query":"select t.col1 from unsharded_a as ua join ( select user.id, user.col1 from user join user_extra ) t", + "Table":"unsharded_a" + } +} +Gen4 plan same as above + +# Straight-join (Gen4 ignores the straight_join hint) +"select m1.col from unsharded as m1 straight_join unsharded as m2" +{ + "QueryType": "SELECT", + "Original": "select m1.col from unsharded as m1 straight_join unsharded as m2", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select m1.col from unsharded as m1 straight_join unsharded as m2 where 1 != 1", + "Query": "select m1.col from unsharded as m1 straight_join unsharded as m2", + "Table": "unsharded" + } +} +{ + "QueryType": "SELECT", + "Original": "select m1.col from unsharded as m1 straight_join unsharded as m2", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select m1.col from unsharded as m1 join unsharded as m2 where 1 != 1", + "Query": "select m1.col from unsharded as m1 join unsharded as m2", + "Table": "unsharded" + } +} + +# derived table in FROM with cross-shard join +"select t.id from (select id from user where id = 5) as t join user_extra on t.id = user_extra.col" +{ + "QueryType": "SELECT", + "Original": "select t.id from (select id from user where id = 5) as t join user_extra on t.id = user_extra.col", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": [-1], + "TableName": "user_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant":"SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery":"select t.id from ( select id from user where 1 != 1 ) t where 1 != 1", + "Query":"select t.id from ( select id from user where id = 5 ) t", + "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 = :t_id", + "Table": "user_extra" + } + ] + } +} +Gen4 plan same as above + +# derived table missing columns +"select t.id from (select id from user) as t join user_extra on t.id = user_extra.user_id where t.col = 42" +{ + "QueryType": "SELECT", + "Original": "select t.id from (select id from user) as t join user_extra on t.id = user_extra.user_id where t.col = 42", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes":[ + -1], + "TableName":"user_user_extra", + "Inputs":[ + { + "OperatorType":"Route", + "Variant":"SelectScatter", + "Keyspace":{ + "Name":"user", + "Sharded":true + }, + "FieldQuery":"select t.id from ( select id from user where 1 != 1 ) t where 1 != 1", + "Query":"select t.id from ( select id from user ) t where t.col = 42", + "Table":"user" + }, + { + "OperatorType":"Route", + "Variant":"SelectEqualUnique", + "Keyspace":{ + "Name":"user", + "Sharded":true + }, + "FieldQuery":"select 1 from user_extra where 1 != 1", + "Query":"select 1 from user_extra where user_extra.user_id = :t_id", + "Table":"user_extra", + "Values":[ + ":t_id"] + }] + } +} +Gen4 error: symbol t.col not found + +# derived table with join +"select t.id from (select id from user where id = 5) as t join user_extra on t.id = user_extra.user_id" +{ + "QueryType":"SELECT", + "Original":"select t.id from (select id from user where id = 5) as t join user_extra on t.id = user_extra.user_id", + "Instructions":{ + "OperatorType":"Join", + "Variant":"Join", + "JoinColumnIndexes":[ + -1], + "TableName":"user_user_extra", + "Inputs":[ + { + "OperatorType":"Route", + "Variant":"SelectScatter", + "Keyspace":{ + "Name":"user", + "Sharded":true + }, + "TargetTabletType":null, + "MultiShardAutocommit":null, + "FieldQuery":"select t.id from ( select id from user where 1 != 1 ) t where 1 != 1", + "Query":"select t.id from ( select id from user where id = 5 ) t", + "Table":"user" + }, + { + "OperatorType":"Route", + "Variant":"SelectEqualUnique", + "Keyspace":{ + "Name":"user", + "Sharded":true + }, + "TargetTabletType":null, + "MultiShardAutocommit":null, + "FieldQuery":"select 1 from user_extra where 1 != 1", + "Query":"select 1 from user_extra where user_extra.user_id = :t_id", + "Table":"user_extra", + "Values":[ + ":t_id"] + }] + } +} +{ + "QueryType": "SELECT", + "Original": "select t.id from (select id from user where id = 5) as t join user_extra on t.id = user_extra.user_id", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select t.id from (select id from user where 1 != 1) as t, user_extra where 1 != 1", + "Query": "select t.id from (select id from user where id = 5) as t, user_extra where t.id = user_extra.user_id", + "Table": "user, user_extra", + "Values": [ + "INT64(5)" + ], + "Vindex": "user_index" + } +} + +# derived table with join primitive (FROM) +"select id, t.id from (select user.id from user join user_extra) as t" +{ + "QueryType": "SELECT", + "Original": "select id, t.id from (select user.id from user join user_extra) as t", + "Instructions": { + "OperatorType": "Subquery", + "Columns": [ + 0, + 0 + ], + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": [-1], + "TableName": "user_user_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": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from user_extra where 1 != 1", + "Query": "select 1 from user_extra", + "Table": "user_extra" + } + ] + } + ] + } +} +Gen4 plan same as above + +# derived table with join, and aliased references +"select t.id from (select user.id from user where user.id = 5) as t join user_extra on t.id = user_extra.user_id" +{ + "QueryType":"SELECT", + "Original":"select t.id from (select user.id from user where user.id = 5) as t join user_extra on t.id = user_extra.user_id", + "Instructions":{ + "OperatorType":"Join", + "Variant":"Join", + "JoinColumnIndexes":[ + -1], + "TableName":"user_user_extra", + "Inputs":[ + { + "OperatorType":"Route", + "Variant":"SelectScatter", + "Keyspace":{ + "Name":"user", + "Sharded":true + }, + "FieldQuery":"select t.id from ( select user.id from user where 1 != 1 ) t where 1 != 1", + "Query":"select t.id from ( select user.id from user where user.id = 5 ) t", + "Table":"user" + }, + { + "OperatorType":"Route", + "Variant":"SelectEqualUnique", + "Keyspace":{ + "Name":"user", + "Sharded":true + }, + "FieldQuery":"select 1 from user_extra where 1 != 1", + "Query":"select 1 from user_extra where user_extra.user_id = :t_id", + "Table":"user_extra", + "Values":[ + ":t_id"] + }] + } +} +{ + "QueryType": "SELECT", + "Original": "select t.id from (select user.id from user where user.id = 5) as t join user_extra on t.id = user_extra.user_id", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select t.id from (select `user`.id from `user` where 1 != 1) as t, user_extra where 1 != 1", + "Query": "select t.id from (select `user`.id from `user` where `user`.id = 5) as t, user_extra where t.id = user_extra.user_id", + "Table": "`user`, user_extra", + "Values": [ + "INT64(5)" + ], + "Vindex": "user_index" + } +} + +# derived table with join, duplicate columns +"select t.id from (select user.id, id from user where user.id = 5) as t join user_extra on t.id = user_extra.user_id" +{ + "QueryType":"SELECT", + "Original":"select t.id from (select user.id, id from user where user.id = 5) as t join user_extra on t.id = user_extra.user_id", + "Instructions":{ + "OperatorType":"Join", + "Variant":"Join", + "JoinColumnIndexes":[ + -1], + "TableName":"user_user_extra", + "Inputs":[ + { + "OperatorType":"Route", + "Variant":"SelectScatter", + "Keyspace":{ + "Name":"user", + "Sharded":true + }, + "FieldQuery":"select t.id from ( select user.id, id from user where 1 != 1 ) t where 1 != 1", + "Query":"select t.id from ( select user.id, id from user where user.id = 5 ) t", + "Table":"user" + }, + { + "OperatorType":"Route", + "Variant":"SelectEqualUnique", + "Keyspace":{ + "Name":"user", + "Sharded":true + }, + "FieldQuery":"select 1 from user_extra where 1 != 1", + "Query":"select 1 from user_extra where user_extra.user_id = :t_id", + "Table":"user_extra", + "Values":[ + ":t_id"] + }] + } +} +Gen4 error: Duplicate column name 'id' + +# derived table in RHS of join +"select t.id from user_extra join (select id from user where id = 5) as t on t.id = user_extra.user_id" +{ + "QueryType":"SELECT", + "Original":"select t.id from user_extra join (select id from user where id = 5) as t on t.id = user_extra.user_id", + "Instructions":{ + "OperatorType":"Join", + "Variant":"Join", + "JoinColumnIndexes":[ + 1], + "TableName":"user_extra_user", + "Inputs":[ + { + "OperatorType":"Route", + "Variant":"SelectScatter", + "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", + "Table":"user_extra" + }, + { + "OperatorType":"Route", + "Variant":"SelectScatter", + "Keyspace":{ + "Name":"user", + "Sharded":true + }, + "FieldQuery":"select t.id from ( select id from user where 1 != 1 ) t where 1 != 1", + "Query":"select t.id from ( select id from user where id = 5 ) t where t.id = :user_extra_user_id", + "Table":"user" + }] + } +} +{ + "QueryType": "SELECT", + "Original": "select t.id from user_extra join (select id from user where id = 5) as t on t.id = user_extra.user_id", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select t.id from user_extra, (select id from user where 1 != 1) as t where 1 != 1", + "Query": "select t.id from user_extra, (select id from user where id = 5) as t where t.id = user_extra.user_id", + "Table": "user, user_extra", + "Values": [ + "INT64(5)" + ], + "Vindex": "user_index" + } +} + +# join with bindvariables +"SELECT user.id FROM user INNER JOIN user_extra ON user.id = user_extra.assembly_id WHERE user_extra.user_id = 2" +{ + "QueryType": "SELECT", + "Original": "SELECT user.id FROM user INNER JOIN user_extra ON user.id = user_extra.assembly_id WHERE user_extra.user_id = 2", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": [-1], + "TableName": "user_user_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": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from user_extra where 1 != 1", + "Query": "select 1 from user_extra where user_extra.assembly_id = :user_id and user_extra.user_id = 2", + "Table": "user_extra", + "Values": [ + "INT64(2)" + ] + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "SELECT user.id FROM user INNER JOIN user_extra ON user.id = user_extra.assembly_id WHERE user_extra.user_id = 2", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "R:0", + "JoinVars": { + "user_extra_assembly_id": 0 + }, + "TableName": "user_extra_user", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user_extra.assembly_id from user_extra where 1 != 1", + "Query": "select user_extra.assembly_id from user_extra where user_extra.user_id = 2", + "Table": "user_extra", + "Values": [ + "INT64(2)" + ], + "Vindex": "user_index" + }, + { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.id from user where 1 != 1", + "Query": "select user.id from user where user.id = :user_extra_assembly_id", + "Table": "user", + "Values": [ + ":user_extra_assembly_id" + ], + "Vindex": "user_index" + } + ] + } +} + +# merge derived tables with single-shard routes +"select u.col, e.col from (select col from user where id = 5) as u join (select col from user_extra where user_id = 5) as e" +{ + "QueryType":"SELECT", + "Original":"select u.col, e.col from (select col from user where id = 5) as u join (select col from user_extra where user_id = 5) as e", + "Instructions":{ + "OperatorType":"Join", + "Variant":"Join", + "JoinColumnIndexes":[ + -1, + 1], + "TableName":"user_user_extra", + "Inputs":[ + { + "OperatorType":"Route", + "Variant":"SelectScatter", + "Keyspace":{ + "Name":"user", + "Sharded":true + }, + "FieldQuery":"select u.col from ( select col from user where 1 != 1 ) u where 1 != 1", + "Query":"select u.col from ( select col from user where id = 5 ) u", + "Table":"user" + }, + { + "OperatorType":"Route", + "Variant":"SelectEqualUnique", + "Keyspace":{ + "Name":"user", + "Sharded":true + }, + "FieldQuery":"select e.col from ( select col from user_extra where 1 != 1 ) e where 1 != 1", + "Query":"select e.col from ( select col from user_extra where user_id = 5 ) e", + "Table":"user_extra", + "Values":[ + "INT64(5)"] + }] + } +} +{ + "QueryType": "SELECT", + "Original": "select u.col, e.col from (select col from user where id = 5) as u join (select col from user_extra where user_id = 5) as e", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select u.col, e.col from (select col from user where 1 != 1) as u, (select col from user_extra where 1 != 1) as e where 1 != 1", + "Query": "select u.col, e.col from (select col from user where id = 5) as u, (select col from user_extra where user_id = 5) as e", + "Table": "user, user_extra", + "Values": [ + "INT64(5)" + ], + "Vindex": "user_index" + } +} + +# recursive derived table lookups +"select id from (select id from (select id from user) as u) as t where id = 5" +{ + "QueryType": "SELECT", + "Original": "select id from (select id from (select id from user) as u) as t where id = 5", + "Instructions": { + "OperatorType": "Route", + "Variant":"SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery":"select id from ( select id from ( select id from user where 1 != 1 ) u where 1 != 1 ) t where 1 != 1", + "Query":"select id from ( select id from ( select id from user ) u ) t where id = 5", + "Table": "user" + } +} +{ + "QueryType": "SELECT", + "Original": "select id from (select id from (select id from user) as u) as t where id = 5", + "Instructions": { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from (select id from (select id from user where 1 != 1) as u where 1 != 1) as t where 1 != 1", + "Query": "select id from (select id from (select id from user where id = 5) as u) as t", + "Table": "user", + "Values": [ + "INT64(5)" + ], + "Vindex": "user_index" + } +} + +# recursive derived table predicate push down +"select bar from (select foo+4 as bar from (select colA+colB as foo from user) as u) as t where bar = 5" +{ + "QueryType": "SELECT", + "Original": "select bar from (select foo+4 as bar from (select colA+colB as foo from user) as u) as t where bar = 5", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery":"select bar from ( select foo + 4 as bar from ( select colA + colB as foo from user where 1 != 1 ) u where 1 != 1 ) t where 1 != 1", + "Query":"select bar from ( select foo + 4 as bar from ( select colA + colB as foo from user ) u ) t where bar = 5", + "Table": "user" + } +} +{ + "QueryType": "SELECT", + "Original": "select bar from (select foo+4 as bar from (select colA+colB as foo from user) as u) as t where bar = 5", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select bar from (select foo + 4 as bar from (select colA + colB as foo from user where 1 != 1) as u where 1 != 1) as t where 1 != 1", + "Query": "select bar from (select foo + 4 as bar from (select colA + colB as foo from user where colA + colB + 4 = 5) as u) as t", + "Table": "user" + } +} + +# wire-up on within cross-shard derived table +"select t.id from (select user.id, user.col1 from user join user_extra on user_extra.col = user.col) as t" +{ + "QueryType": "SELECT", + "Original": "select t.id from (select user.id, user.col1 from user join user_extra on user_extra.col = user.col) as t", + "Instructions": { + "OperatorType":"Subquery", + "Columns": [ + 0 + ], + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": [-1,-2], + "TableName": "user_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.id, user.col1, user.col from user where 1 != 1", + "Query": "select user.id, user.col1, 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_col", + "Table": "user_extra" + } + ] + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select t.id from (select user.id, user.col1 from user join user_extra on user_extra.col = user.col) as t", + "Instructions": { + "OperatorType": "SimpleProjection", + "Columns": [ + 0 + ], + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:1,L:2", + "JoinVars": { + "user_col": 0 + }, + "TableName": "user_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.col, user.id, user.col1 from user where 1 != 1", + "Query": "select user.col, user.id, user.col1 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_col", + "Table": "user_extra" + } + ] + } + ] + } +} diff --git a/src/test/resources/plan/from_cases_unsupported.txt b/src/test/resources/plan/from_cases_unsupported.txt index a43a8e6..de08d08 100644 --- a/src/test/resources/plan/from_cases_unsupported.txt +++ b/src/test/resources/plan/from_cases_unsupported.txt @@ -16,6 +16,139 @@ "Table": "seq" } } +Gen4 plan same as above + +# Reason: syntax error +# select next from non-sequence table +"select next value from user" +"NEXT used on a non-sequence table" +Gen4 plan same as above + +# Reason: syntax error +# select next in derived table +"select 1 from (select next value from seq) t" +{ + "QueryType": "SELECT", + "Original": "select 1 from (select next value from seq) t", + "Instructions": { + "OperatorType": "Route", + "Variant": "Next", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select 1 from (select next 1 values from seq where 1 != 1) as t where 1 != 1", + "Query": "select 1 from (select next 1 values from seq) as t", + "Table": "seq" + } +} +Gen4 error: Incorrect usage/placement of 'NEXT' + +# Reason: syntax error +# select next in derived table +"select * from (select next value from seq) t" +{ + "QueryType": "SELECT", + "Original": "select * from (select next value from seq) t", + "Instructions": { + "OperatorType": "Route", + "Variant": "Next", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select * from (select next 1 values from seq where 1 != 1) as t where 1 != 1", + "Query": "select * from (select next 1 values from seq) as t", + "Table": "seq" + } +} +Gen4 error: Incorrect usage/placement of 'NEXT' + +# Reason: syntax error +# select next in subquery +"select 1 from user where id in (select next value from seq)" +{ + "QueryType": "SELECT", + "Original": "select 1 from user where id in (select next value from seq)", + "Instructions": { + "OperatorType": "Subquery", + "Variant": "PulloutIn", + "PulloutVars": [ + "__sq_has_values1", + "__sq1" + ], + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "Next", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select next 1 values from seq where 1 != 1", + "Query": "select next 1 values from seq", + "Table": "seq" + }, + { + "OperatorType": "Route", + "Variant": "IN", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from `user` where 1 != 1", + "Query": "select 1 from `user` where :__sq_has_values1 = 1 and id in ::__vals", + "Table": "`user`", + "Values": [ + ":__sq1" + ], + "Vindex": "user_index" + } + ] + } +} +Gen4 error: Incorrect usage/placement of 'NEXT' + +# Reason: syntax error +# select next in projection +"select (select next value from seq) from user" +{ + "QueryType": "SELECT", + "Original": "select (select next value from seq) from user", + "Instructions": { + "OperatorType": "Subquery", + "Variant": "PulloutValue", + "PulloutVars": [ + "__sq_has_values1", + "__sq1" + ], + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "Next", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select next 1 values from seq where 1 != 1", + "Query": "select next 1 values from seq", + "Table": "seq" + }, + { + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select :__sq1 from `user` where 1 != 1", + "Query": "select :__sq1 from `user`", + "Table": "`user`" + } + ] + } +} +Gen4 error: Incorrect usage/placement of 'NEXT' # Reason: reference table # Select from reference @@ -1187,3 +1320,573 @@ # implicit table reference for sharded keyspace "select user.foo.col from user.foo" "table foo not found" + +# multiple index hints, make sure they are not stripped. +"select user.col from user use index(a) use index for group by (b)" +{ + "QueryType": "SELECT", + "Original": "select user.col from user use index(a) use index for group by (b)", + "Instructions": { + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select `user`.col from `user` use index (a) use index for group by (b) where 1 != 1", + "Query": "select `user`.col from `user` use index (a) use index for group by (b)", + "Table": "`user`" + } +} +Gen4 plan same as above + +# derived table with aliased columns and a join that requires pushProjection +"select i+1 from (select user.id from user join user_extra) t(i)" +"unsupported: column aliases in derived table" +{ + "QueryType": "SELECT", + "Original": "select i+1 from (select user.id from user join user_extra) t(i)", + "Instructions": { + "OperatorType": "SimpleProjection", + "Columns": [ + 1 + ], + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:0,L:1", + "TableName": "`user`_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select `user`.id, `user`.id + 1 from `user` where 1 != 1", + "Query": "select `user`.id, `user`.id + 1 from `user`", + "Table": "`user`" + }, + { + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from user_extra where 1 != 1", + "Query": "select 1 from user_extra", + "Table": "user_extra" + } + ] + } + ] + } +} + +# derived table with aliased columns and outer predicate pushed in derived table +"select u.a from (select id as b, name from user) u(a, n) where u.n = 1" +"unsupported: column aliases in derived table" +{ + "QueryType": "SELECT", + "Original": "select u.a from (select id as b, name from user) u(a, n) where u.n = 1", + "Instructions": { + "OperatorType": "Route", + "Variant": "Equal", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select u.a from (select id as b, `name` from `user` where 1 != 1) as u(a, n) where 1 != 1", + "Query": "select u.a from (select id as b, `name` from `user` where `name` = 1) as u(a, n)", + "Table": "`user`", + "Values": [ + "INT64(1)" + ], + "Vindex": "name_user_map" + } +} + +# Reason: syntax error +# derived table with aliased columns predicate in both the outer and inner +"select u.a from (select id as b, name from user where b = 1) u(a, n) where u.n = 1" +"unsupported: column aliases in derived table" +{ + "QueryType": "SELECT", + "Original": "select u.a from (select id as b, name from user where b = 1) u(a, n) where u.n = 1", + "Instructions": { + "OperatorType": "Route", + "Variant": "Equal", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select u.a from (select id as b, `name` from `user` where 1 != 1) as u(a, n) where 1 != 1", + "Query": "select u.a from (select id as b, `name` from `user` where b = 1 and `name` = 1) as u(a, n)", + "Table": "`user`", + "Values": [ + "INT64(1)" + ], + "Vindex": "name_user_map" + } +} + +# Column aliases in Derived Table +"select id2 from (select id from user) as x (id2)" +"unsupported: column aliases in derived table" +{ + "QueryType": "SELECT", + "Original": "select id2 from (select id from user) as x (id2)", + "Instructions": { + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id2 from (select id from `user` where 1 != 1) as x(id2) where 1 != 1", + "Query": "select id2 from (select id from `user`) as x(id2)", + "Table": "`user`" + } +} + +# two subqueries with different Select and OpCode +"select id from user where id in (select id from user_extra) and col = (select user_id from user_extra limit 1)" +{ + "QueryType": "SELECT", + "Original": "select id from user where id in (select id from user_extra) and col = (select user_id from user_extra limit 1)", + "Instructions": { + "OperatorType": "Subquery", + "Variant": "PulloutIn", + "PulloutVars": [ + "__sq_has_values2", + "__sq2" + ], + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from user_extra where 1 != 1", + "Query": "select id from user_extra", + "Table": "user_extra" + }, + { + "OperatorType": "Subquery", + "Variant": "PulloutValue", + "PulloutVars": [ + "__sq_has_values1", + "__sq1" + ], + "Inputs": [ + { + "OperatorType": "Limit", + "Count": "INT64(1)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user_id from user_extra where 1 != 1", + "Query": "select user_id from user_extra limit :__upper_limit", + "Table": "user_extra" + } + ] + }, + { + "OperatorType": "Route", + "Variant": "IN", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from `user` where 1 != 1", + "Query": "select id from `user` where col = :__sq1 and :__sq_has_values2 = 1 and id in ::__vals", + "Table": "`user`", + "Values": [ + ":__sq2" + ], + "Vindex": "user_index" + } + ] + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select id from user where id in (select id from user_extra) and col = (select user_id from user_extra limit 1)", + "Instructions": { + "OperatorType": "Subquery", + "Variant": "PulloutValue", + "PulloutVars": [ + "__sq_has_values2", + "__sq2" + ], + "Inputs": [ + { + "OperatorType": "Limit", + "Count": "INT64(1)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user_id from user_extra where 1 != 1", + "Query": "select user_id from user_extra limit :__upper_limit", + "Table": "user_extra" + } + ] + }, + { + "OperatorType": "Subquery", + "Variant": "PulloutIn", + "PulloutVars": [ + "__sq_has_values1", + "__sq1" + ], + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from user_extra where 1 != 1", + "Query": "select id from user_extra", + "Table": "user_extra" + }, + { + "OperatorType": "Route", + "Variant": "IN", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from `user` where 1 != 1", + "Query": "select id from `user` where :__sq_has_values1 = 1 and id in ::__vals and col = :__sq2", + "Table": "`user`", + "Values": [ + ":__sq1" + ], + "Vindex": "user_index" + } + ] + } + ] + } +} + +# Reason: multiple keyspace +# Duplicate output column from derived table having a join +"select 0 from (select `user`.col1 from `user` join unsharded) as t join unsharded on unsharded.col1 = t.col1 and unsharded.a = t.col1" +"unsupported: expression on results of a cross-shard subquery" +{ + "QueryType": "SELECT", + "Original": "select 0 from (select `user`.col1 from `user` join unsharded) as t join unsharded on unsharded.col1 = t.col1 and unsharded.a = t.col1", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": [-2], + "TableName": "`user`_unsharded_unsharded", + "Inputs": [ + { + "OperatorType": "SimpleProjection", + "Columns": [ + 0, + 1 + ], + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:0,L:1", + "TableName": "`user`_unsharded", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select `user`.col1, 0 from `user` where 1 != 1", + "Query": "select `user`.col1, 0 from `user`", + "Table": "`user`" + }, + { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select 1 from unsharded where 1 != 1", + "Query": "select 1 from unsharded", + "Table": "unsharded" + } + ] + } + ] + }, + { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select 1 from unsharded where 1 != 1", + "Query": "select 1 from unsharded where unsharded.col1 = :t_col1 and unsharded.a = :t_col1", + "Table": "unsharded" + } + ] + } +} + +# Reason: `using(column)` syntax unsupported +# Unsharded join with using +"select * from unsharded_a join unsharded_b using (propertyId);" +{ + "QueryType": "SELECT", + "Original": "select * from unsharded_a join unsharded_b using (propertyId);", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select * from unsharded_a join unsharded_b using (propertyId) where 1 != 1", + "Query": "select * from unsharded_a join unsharded_b using (propertyId)", + "Table": "unsharded_a, unsharded_b" + } +} +Gen4 plan same as above + + +# query builder with derived table having join inside it +"select u.col from (select user.col from user join user_extra) as u join user_extra ue limit 1" +{ + "QueryType": "SELECT", + "Original": "select u.col from (select user.col from user join user_extra) as u join user_extra ue limit 1", + "Instructions": { + "OperatorType": "Limit", + "Count": 1, + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": [-1], + "TableName": "user_user_extra_user_extra", + "Inputs": [ + { + "OperatorType": "SimpleProjection", + "Columns": [ + 0 + ], + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": [-1], + "TableName": "user_user_extra", + "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": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from user_extra where 1 != 1", + "Query": "select 1 from user_extra", + "Table": "user_extra" + } + ] + } + ] + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from user_extra as ue where 1 != 1", + "Query": "select 1 from user_extra as ue", + "Table": "user_extra" + } + ] + } + ] + } +} +Gen4 plan same as above + +# Reason: `using(column)` syntax unsupported +# join with USING construct +"select * from authoritative join unsharded_authoritative using(col1)" +"unsupported: join with USING(column_list) clause for complex queries" +{ + "QueryType": "SELECT", + "Original": "select * from authoritative join unsharded_authoritative using(col1)", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:1,L:2,L:3,R:0", + "JoinVars": { + "authoritative_col1": 0 + }, + "TableName": "authoritative_unsharded_authoritative", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select authoritative.col1, authoritative.col1 as col1, authoritative.user_id as user_id, authoritative.col2 as col2 from authoritative where 1 != 1", + "Query": "select authoritative.col1, authoritative.col1 as col1, authoritative.user_id as user_id, authoritative.col2 as col2 from authoritative", + "Table": "authoritative" + }, + { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select unsharded_authoritative.col2 as col2 from unsharded_authoritative where 1 != 1", + "Query": "select unsharded_authoritative.col2 as col2 from unsharded_authoritative where unsharded_authoritative.col1 = :authoritative_col1", + "Table": "unsharded_authoritative" + } + ] + } +} + +# Reason: unsupported table route1 +# push predicate on joined derived tables +"select t.id from (select id, textcol1 as baz from route1) as t join (select id, textcol1+textcol1 as baz from user) as s ON t.id = s.id WHERE t.baz = '3' AND s.baz = '3'" +{ + "QueryType": "SELECT", + "Original": "select t.id from (select id, textcol1 as baz from route1) as t join (select id, textcol1+textcol1 as baz from user) as s ON t.id = s.id WHERE t.baz = '3' AND s.baz = '3'", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select t.id from (select id, textcol1 as baz from user as route1 where 1 != 1) as t join (select id, textcol1 + textcol1 as baz from user where 1 != 1) as s on t.id = s.id where 1 != 1", + "Query": "select t.id from (select id, textcol1 as baz from user as route1) as t join (select id, textcol1 + textcol1 as baz from user) as s on t.id = s.id where t.baz = '3' and s.baz = '3'", + "Table": "user" + } +} +{ + "QueryType": "SELECT", + "Original": "select t.id from (select id, textcol1 as baz from route1) as t join (select id, textcol1+textcol1 as baz from user) as s ON t.id = s.id WHERE t.baz = '3' AND s.baz = '3'", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select t.id from (select id, textcol1 as baz from user as route1 where 1 != 1) as t, (select id, textcol1 + textcol1 as baz from user where 1 != 1) as s where 1 != 1", + "Query": "select t.id from (select id, textcol1 as baz from user as route1 where textcol1 = '3') as t, (select id, textcol1 + textcol1 as baz from user where textcol1 + textcol1 = '3') as s where t.id = s.id", + "Table": "user" + } +} + +# Reason: multiple keyspace +# wire-up on join with cross-shard derived table +"select t.col1 from (select user.id, user.col1 from user join user_extra) as t join unsharded on unsharded.col1 = t.col1 and unsharded.id = t.id" +{ + "QueryType": "SELECT", + "Original": "select t.col1 from (select user.id, user.col1 from user join user_extra) as t join unsharded on unsharded.col1 = t.col1 and unsharded.id = t.id", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": [-1], + "TableName": "user_user_extra_unsharded", + "Inputs": [ + { + "OperatorType": "SimpleProjection", + "Columns": [ + 1, + 0 + ], + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": [-1,-2], + "TableName": "user_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.id, user.col1 from user where 1 != 1", + "Query": "select user.id, user.col1 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", + "Table": "user_extra" + } + ] + } + ] + }, + { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select 1 from unsharded where 1 != 1", + "Query": "select 1 from unsharded where unsharded.col1 = :t_col1 and unsharded.id = :t_id", + "Table": "unsharded" + } + ] + } +} +Gen4 plan same as above diff --git a/src/test/resources/plan/memory_sort_cases.txt b/src/test/resources/plan/memory_sort_cases.txt index 1a6f8b8..cbe020d 100644 --- a/src/test/resources/plan/memory_sort_cases.txt +++ b/src/test/resources/plan/memory_sort_cases.txt @@ -31,6 +31,31 @@ ] } } +{ + "QueryType": "SELECT", + "Original": "select a, b, count(*) from user group by a order by b", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "random(1) AS b, sum_count_star(2) AS count(*)", + "GroupBy": "(0|3)", + "ResultColumns": 3, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select a, b, count(*), weight_string(a), weight_string(b) from user where 1 != 1 group by a, weight_string(a)", + "OrderBy": "(1|4) ASC, (0|3) ASC", + "Query": "select a, b, count(*), weight_string(a), weight_string(b) from user group by a, weight_string(a) order by b asc, a asc", + "Table": "user" + } + ] + } +} # scatter aggregate order by references aggregate expression "select a, b, count(*) k from user group by a order by k" @@ -65,6 +90,38 @@ ] } } +{ + "QueryType": "SELECT", + "Original": "select a, b, count(*) k from user group by a order by k", + "Instructions": { + "OperatorType": "Sort", + "Variant": "Memory", + "OrderBy": "2 ASC", + "ResultColumns": 3, + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "random(1) AS b, sum_count_star(2) AS k", + "GroupBy": "(0|3)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select a, b, count(*) as k, weight_string(a) from user where 1 != 1 group by a, weight_string(a)", + "OrderBy": "(0|3) ASC", + "Query": "select a, b, count(*) as k, weight_string(a) from user group by a, weight_string(a) order by a asc", + "Table": "user" + } + ] + } + ] + } +} # scatter aggregate order by references multiple non-group-by expressions "select a, b, count(*) k from user group by a order by b, a, k" @@ -99,6 +156,38 @@ ] } } +{ + "QueryType": "SELECT", + "Original": "select a, b, count(*) k from user group by a order by b, a, k", + "Instructions": { + "OperatorType": "Sort", + "Variant": "Memory", + "OrderBy": "1 ASC, (0|3) ASC, 2 ASC", + "ResultColumns": 3, + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "random(1) AS b, sum_count_star(2) AS k", + "GroupBy": "(0|3)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select a, b, count(*) as k, weight_string(a) from user where 1 != 1 group by a, weight_string(a)", + "OrderBy": "(0|3) ASC", + "Query": "select a, b, count(*) as k, weight_string(a) from user group by a, weight_string(a) order by a asc", + "Table": "user" + } + ] + } + ] + } +} # scatter aggregate with memory sort and limit "select a, b, count(*) k from user group by a order by k desc limit 10" @@ -139,6 +228,134 @@ ] } } +{ + "QueryType":"SELECT", + "Original":"select a, b, count(*) k from user group by a order by k desc limit 10", + "Instructions":{ + "OperatorType":"Limit", + "Count":10, + "Offset":null, + "Variant":null, + "Aggregates":null, + "Distinct":null, + "GroupBy":null, + "OrderBy":null, + "Keyspace":null, + "TargetTabletType":null, + "MultiShardAutocommit":null, + "FieldQuery":null, + "Query":null, + "Table":null, + "Values":null, + "Vindex":null, + "JoinColumnIndexes":null, + "TableName":null, + "Columns":null, + "StrColumns":null, + "SysTableKeyspaceExpr":null, + "Expressions":null, + "ResultColumns":null, + "Inputs":[ + { + "OperatorType":"Sort", + "Count":null, + "Offset":null, + "Variant":"Memory", + "Aggregates":null, + "Distinct":null, + "GroupBy":null, + "OrderBy":"2 DESC", + "Keyspace":null, + "TargetTabletType":null, + "MultiShardAutocommit":null, + "FieldQuery":null, + "Query":null, + "Table":null, + "Values":null, + "Vindex":null, + "JoinColumnIndexes":null, + "TableName":null, + "Columns":null, + "StrColumns":null, + "SysTableKeyspaceExpr":null, + "Expressions":null, + "ResultColumns":3, + "Inputs":[ + { + "OperatorType":"Aggregate", + "Count":null, + "Offset":null, + "Variant":"Ordered", + "Aggregates":"random(1) AS b, sum_count_star(2) AS k", + "Distinct":null, + "GroupBy":"(0|3)", + "OrderBy":null, + "Keyspace":null, + "TargetTabletType":null, + "MultiShardAutocommit":null, + "FieldQuery":null, + "Query":null, + "Table":null, + "Values":null, + "Vindex":null, + "JoinColumnIndexes":null, + "TableName":null, + "Columns":null, + "StrColumns":null, + "SysTableKeyspaceExpr":null, + "Expressions":null, + "ResultColumns":null, + "Inputs":[ + { + "OperatorType":"Route", + "Count":null, + "Offset":null, + "Variant":"SelectScatter", + "Aggregates":null, + "Distinct":null, + "GroupBy":null, + "OrderBy":"(0|3) ASC", + "Keyspace":{ + "Name":"user", + "Sharded":true + }, + "TargetTabletType":null, + "MultiShardAutocommit":null, + "FieldQuery":"select a, b, count(*) as k, weight_string(a) from user where 1 != 1 group by a, weight_string(a)", + "Query":"select a, b, count(*) as k, weight_string(a) from user group by a, weight_string(a) order by a asc", + "Table":"user", + "Values":null, + "Vindex":null, + "JoinColumnIndexes":null, + "TableName":null, + "Columns":null, + "StrColumns":null, + "SysTableKeyspaceExpr":null, + "Expressions":null, + "ResultColumns":null, + "Inputs":null, + "TargetDestination":null, + "IsDML":false, + "SingleShardOnly":false, + "ShardNameNeeded":false + }], + "TargetDestination":null, + "IsDML":false, + "SingleShardOnly":false, + "ShardNameNeeded":false + }], + "TargetDestination":null, + "IsDML":false, + "SingleShardOnly":false, + "ShardNameNeeded":false + }], + "TargetDestination":null, + "IsDML":false, + "SingleShardOnly":false, + "ShardNameNeeded":false + }, + "errorMessage":null +} # scatter aggregate with memory sort and order by number "select a, b, count(*) k from user group by a order by 1,3" @@ -173,24 +390,20 @@ ] } } - -# scatter aggregate with memory sort and order by number, reuse weight_string -# we have to use a meaningless construct to test this. -"select textcol1, count(*) k from user group by textcol1 order by textcol1, k, textcol1" { "QueryType": "SELECT", - "Original": "select textcol1, count(*) k from user group by textcol1 order by textcol1, k, textcol1", + "Original": "select a, b, count(*) k from user group by a order by 1,3", "Instructions": { "OperatorType": "Sort", "Variant": "Memory", - "OrderBy": "2 ASC, 1 ASC, 2 ASC", + "OrderBy": "(0|3) ASC, 2 ASC", + "ResultColumns": 3, "Inputs": [ { "OperatorType": "Aggregate", "Variant": "Ordered", - "Aggregates": "count(1)", - "Distinct": "false", - "GroupBy": "2", + "Aggregates": "random(1) AS b, sum_count_star(2) AS k", + "GroupBy": "(0|3)", "Inputs": [ { "OperatorType": "Route", @@ -199,8 +412,9 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select textcol1, count(*) as k, weight_string(textcol1) from user where 1 != 1 group by textcol1", - "Query": "select textcol1, count(*) as k, weight_string(textcol1) from user group by textcol1 order by textcol1 asc, textcol1 asc", + "FieldQuery": "select a, b, count(*) as k, weight_string(a) from user where 1 != 1 group by a, weight_string(a)", + "OrderBy": "(0|3) ASC", + "Query": "select a, b, count(*) as k, weight_string(a) from user group by a, weight_string(a) order by a asc", "Table": "user" } ] @@ -210,6 +424,7 @@ } # order by on a cross-shard query. Note: this happens only when an order by column is from the second table +# same vindex type ,push down it "select user.col1 as a, user.col2 b, music.col3 c from user, music where user.name = music.user_id and user.name = 1 order by c" { "QueryType": "SELECT", @@ -260,6 +475,21 @@ ] } } +{ + "QueryType": "SELECT", + "Original": "select user.col1 as a, user.col2 b, music.col3 c from user, music where user.name = music.user_id and user.name = 1 order by c", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.col1 as a, user.col2 as b, music.col3 as c from user, music where 1 != 1", + "Query": "select user.col1 as a, user.col2 as b, music.col3 as c from user, music where user.name = 1 and user.name = music.user_id order by c asc", + "Table": "music,user" + } +} # Order by for join, with mixed cross-shard ordering "select user.col1 as a, user.col2, music.col3 from user join music on user.name = music.id where user.name = 1 order by 1 asc, 3 desc, 2 asc" @@ -308,6 +538,48 @@ ] } } +{ + "QueryType": "SELECT", + "Original": "select user.col1 as a, user.col2, music.col3 from user join music on user.name = music.id where user.name = 1 order by 1 asc, 3 desc, 2 asc", + "Instructions": { + "OperatorType": "Sort", + "Variant": "Memory", + "OrderBy": "(0|3) ASC, (2|4) DESC, (1|5) ASC", + "ResultColumns": 3, + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": [-2,-3,1,-4,2,-5], + "TableName": "user_music", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.name, user.col1 as a, user.col2, weight_string(user.col1), weight_string(user.col2) from user where 1 != 1", + "Query": "select user.name, user.col1 as a, user.col2, weight_string(user.col1), weight_string(user.col2) from user where user.name = 1", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select music.col3, weight_string(music.col3) from music where 1 != 1", + "Query": "select music.col3, weight_string(music.col3) from music where music.id = :user_name", + "Table": "music" + } + ] + } + ] + } +} # order by on a cross-shard subquery "select name from ( select user.name, user.col from user join user_extra ) as t order by name" @@ -360,3 +632,167 @@ ] } } + +# scatter aggregate with memory sort and order by number, reuse weight_string +# we have to use a meaningless construct to test this. TODO: improve to do ordering once for textcol1 +"select textcol1 as t, count(*) k from user group by textcol1 order by textcol1, k, textcol1" +{ + "QueryType": "SELECT", + "Original": "select textcol1 as t, count(*) k from user group by textcol1 order by textcol1, k, textcol1", + "Instructions": { + "OperatorType": "Sort", + "Variant": "Memory", + "OrderBy": "2 ASC, 1 ASC, 2 ASC", + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "count(1)", + "Distinct":false, + "GroupBy": "2", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery":"select textcol1 as t, count(*) as k, weight_string(textcol1) from user where 1 != 1 group by textcol1", + "Query":"select textcol1 as t, count(*) as k, weight_string(textcol1) from user group by textcol1 order by textcol1 asc, textcol1 asc", + "Table": "user" + } + ] + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select textcol1 as t, count(*) k from user group by textcol1 order by textcol1, k, textcol1", + "Instructions": { + "OperatorType": "Sort", + "Variant": "Memory", + "OrderBy": "0 ASC, 1 ASC, 0 ASC", + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count_star(1) AS k", + "GroupBy": "0", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select textcol1 as t, count(*) as k from user where 1 != 1 group by textcol1", + "OrderBy": "0 ASC", + "Query": "select textcol1 as t, count(*) as k from user group by textcol1 order by textcol1 asc", + "Table": "user" + } + ] + } + ] + } +} + +# intcol order by +"select id, intcol from user order by intcol" +{ + "QueryType": "SELECT", + "Original": "select id, intcol from user order by intcol", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id, intcol from user where 1 != 1", + "Query": "select id, intcol from user order by intcol asc", + "Table": "user" + } +} +{ + "QueryType": "SELECT", + "Original": "select id, intcol from user order by intcol", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id, intcol from user where 1 != 1", + "OrderBy": "1 ASC", + "Query": "select id, intcol from user order by intcol asc", + "Table": "user" + } +} + +# unary expression in join query +# gen3 NPE报错 +"select u.a from user u join music m on u.a = m.a order by binary a desc" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select u.a from user u join music m on u.a = m.a order by binary a desc", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:0", + "JoinVars": { + "u_a": 0 + }, + "TableName": "`user`_music", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select u.a, convert(a, binary), weight_string(convert(a, binary)) from user as u where 1 != 1", + "OrderBy": "(1|2) DESC", + "Query": "select u.a, convert(a, binary), weight_string(convert(a, binary)) from user as u order by convert(a, binary) desc", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from music as m where 1 != 1", + "Query": "select 1 from music as m where m.a = :u_a", + "Table": "music" + } + ] + } +} + +# scatter order by with order by column not present +"select col from user order by id" +"unsupported: in scatter query: order by must reference a column in the select list:id" +{ + "QueryType": "SELECT", + "Original": "select col from user order by id", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col, id, weight_string(id) from user where 1 != 1", + "OrderBy": "(1|2) ASC", + "Query": "select col, id, weight_string(id) from user order by id asc", + "ResultColumns": 1, + "Table": "user" + } +} \ No newline at end of file diff --git a/src/test/resources/plan/memory_sort_cases_unsupported.txt b/src/test/resources/plan/memory_sort_cases_unsupported.txt index e21746a..3aac930 100644 --- a/src/test/resources/plan/memory_sort_cases_unsupported.txt +++ b/src/test/resources/plan/memory_sort_cases_unsupported.txt @@ -7,13 +7,14 @@ "Instructions": { "OperatorType": "Sort", "Variant": "Memory", - "OrderBy": "3 ASC, 2 ASC", + "OrderBy": "(1|3) ASC, (2|4) ASC", + "ResultColumns": 3, "Inputs": [ { "OperatorType": "Join", "Variant": "Join", - "JoinColumnIndexes": [-1,-2,1,-3], - "TableName": "user_unsharded", + "JoinColumnIndexes": "L:0,L:1,R:0,L:2,R:1", + "TableName": "`user`_unsharded", "Inputs": [ { "OperatorType": "Route", @@ -28,13 +29,55 @@ }, { "OperatorType": "Route", - "Variant": "SelectUnsharded", + "Variant": "Unsharded", "Keyspace": { "Name": "main", "Sharded": false }, - "FieldQuery": "select un.col2 from unsharded as un where 1 != 1", - "Query": "select un.col2 from unsharded as un", + "FieldQuery": "select un.col2, weight_string(un.col2) from unsharded as un where 1 != 1", + "Query": "select un.col2, weight_string(un.col2) from unsharded as un", + "Table": "unsharded" + } + ] + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select u.a, u.textcol1, un.col2 from user u join unsharded un order by u.textcol1, un.col2", + "Instructions": { + "OperatorType": "Sort", + "Variant": "Memory", + "OrderBy": "(1|3) ASC COLLATE latin1_swedish_ci, (2|4) ASC", + "ResultColumns": 3, + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:0,L:1,R:0,L:2,R:1", + "TableName": "`user`_unsharded", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select u.a, u.textcol1, weight_string(u.textcol1) from user as u where 1 != 1", + "Query": "select u.a, u.textcol1, weight_string(u.textcol1) from user as u", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "Unsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select un.col2, weight_string(un.col2) from unsharded as un where 1 != 1", + "Query": "select un.col2, weight_string(un.col2) from unsharded as un", "Table": "unsharded" } ] @@ -52,23 +95,66 @@ "Instructions": { "OperatorType": "Sort", "Variant": "Memory", - "OrderBy": "3 ASC, 2 ASC", + "OrderBy": "(1|3) ASC, (2|4) ASC", + "ResultColumns": 3, "Inputs": [ { "OperatorType": "Join", "Variant": "Join", - "JoinColumnIndexes": [1,2,-1,3], - "TableName": "unsharded_user", + "JoinColumnIndexes": "R:0,R:1,L:0,R:2,L:1", + "TableName": "unsharded_`user`", "Inputs": [ { "OperatorType": "Route", - "Variant": "SelectUnsharded", + "Variant": "Unsharded", "Keyspace": { "Name": "main", "Sharded": false }, - "FieldQuery": "select un.col2 from unsharded as un where 1 != 1", - "Query": "select un.col2 from unsharded as un", + "FieldQuery": "select un.col2, weight_string(un.col2) from unsharded as un where 1 != 1", + "Query": "select un.col2, weight_string(un.col2) from unsharded as un", + "Table": "unsharded" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select u.a, u.textcol1, weight_string(u.textcol1) from user as u where 1 != 1", + "Query": "select u.a, u.textcol1, weight_string(u.textcol1) from user as u", + "Table": "user" + } + ] + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select u.a, u.textcol1, un.col2 from unsharded un join user u order by u.textcol1, un.col2", + "Instructions": { + "OperatorType": "Sort", + "Variant": "Memory", + "OrderBy": "(1|3) ASC COLLATE latin1_swedish_ci, (2|4) ASC", + "ResultColumns": 3, + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "R:0,R:1,L:0,R:2,L:1", + "TableName": "unsharded_`user`", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "Unsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select un.col2, weight_string(un.col2) from unsharded as un where 1 != 1", + "Query": "select un.col2, weight_string(un.col2) from unsharded as un", "Table": "unsharded" }, { @@ -119,4 +205,158 @@ } ] } +} +{ + "QueryType": "SELECT", + "Original": "select id, keyspace_id, range_start, range_end from user_index where id = :id order by range_start", + "Instructions": { + "OperatorType": "Sort", + "Variant": "Memory", + "OrderBy": "2 ASC", + "Inputs": [ + { + "OperatorType": "VindexFunc", + "Variant": "VindexMap", + "Columns": [ + 0, + 1, + 2, + 3 + ], + "Fields": { + "id": "VARBINARY", + "keyspace_id": "VARBINARY", + "range_end": "VARBINARY", + "range_start": "VARBINARY" + }, + "Value": ":id", + "Vindex": "user_index" + } + ] + } +} + +# order by on a cross-shard derived table +"select id from (select user.id, user.col from user join user_extra) as t order by id" +{ + "QueryType": "SELECT", + "Original": "select id from (select user.id, user.col from user join user_extra) as t order by id", + "Instructions": { + "OperatorType": "Sort", + "Variant": "Memory", + "OrderBy": "0 ASC", + "Inputs": [ + { + "OperatorType": "Subquery", + "Columns": [ + 0 + ], + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes":[-1,-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", + "Table": "user_extra" + } + ] + } + ] + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select id from (select user.id, user.col from user join user_extra) as t order by id", + "Instructions": { + "OperatorType": "Sort", + "Variant": "Memory", + "OrderBy": "(0|1) ASC", + "ResultColumns": 1, + "Inputs": [ + { + "OperatorType": "SimpleProjection", + "Columns": [ + 0, + 2 + ], + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:0,L:1,L:2", + "TableName": "`user`_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select `user`.id, `user`.col, weight_string(`user`.id) from user where 1 != 1", + "Query": "select `user`.id, `user`.col, weight_string(`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", + "Table": "user_extra" + } + ] + } + ] + } + ] + } +} + +# unary expression +# gen3未实现对应的toString,报错unsupported: in scatter query: complex order by expression:com.jd.jdbc.sqlparser.ast.expr.sqlunaryexpr@62886aeb +"select a from user order by binary a desc" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select a from user order by binary a desc", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select a, convert(a, binary), weight_string(convert(a, binary)) from user where 1 != 1", + "OrderBy": "(1|2) DESC", + "Query": "select a, convert(a, binary), weight_string(convert(a, binary)) from user order by convert(a, binary) desc", + "ResultColumns": 1, + "Table": "user" + } } \ No newline at end of file diff --git a/src/test/resources/plan/plan_schema.json b/src/test/resources/plan/plan_schema.json index 4b987f9..41c8e73 100644 --- a/src/test/resources/plan/plan_schema.json +++ b/src/test/resources/plan/plan_schema.json @@ -24,6 +24,10 @@ "sequence": "seq" }, "columns": [ + { + "name": "col", + "type": "INT16" + }, { "name": "costly" }, @@ -37,6 +41,10 @@ "name": "textcol1", "type": "VARCHAR" }, + { + "name": "intcol", + "type": "INT16" + }, { "name": "textcol2", "type": "VARCHAR" @@ -72,7 +80,13 @@ "auto_increment": { "column": "extra_id", "sequence": "seq" - } + }, + "columns": [ + { + "name": "col", + "type": "INT16" + } + ] }, "music": { "column_vindexes": [ diff --git a/src/test/resources/plan/postprocess_cases.txt b/src/test/resources/plan/postprocess_cases.txt index 71204d6..b5edd68 100644 --- a/src/test/resources/plan/postprocess_cases.txt +++ b/src/test/resources/plan/postprocess_cases.txt @@ -15,6 +15,21 @@ "Table": "user" } } +{ + "QueryType": "SELECT", + "Original": "select user.col1 from user having col2 = 2", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.col1 from user where 1 != 1", + "Query": "select user.col1 from user where col2 = 2", + "Table": "user" + } +} # ORDER BY, reference col from local table. "select col from user where name = 5 order by aa" @@ -37,6 +52,7 @@ "Vindex": "user_index" } } +Gen4 plan same as above # ORDER BY uses column numbers "select col from user where name = 1 order by 1" @@ -59,6 +75,25 @@ "Vindex": "user_index" } } +{ + "QueryType": "SELECT", + "Original": "select col from user where name = 1 order by 1", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col from user where 1 != 1", + "Query": "select col from user where name = 1 order by col asc", + "Table": "user", + "Values": [ + "INT64(1)" + ], + "Vindex": "user_index" + } +} # ORDER BY on scatter "select col from user order by col" @@ -77,6 +112,22 @@ "Table": "user" } } +{ + "QueryType": "SELECT", + "Original": "select col from user order by col", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col from user where 1 != 1", + "OrderBy": "0 ASC", + "Query": "select col from user order by col asc", + "Table": "user" + } +} # ORDER BY works for select * from authoritative table "select * from authoritative order by user_id" @@ -95,6 +146,7 @@ "Table": "authoritative" } } +Gen4 error: unsupported: '*' expression in cross-shard query # ORDER BY works for select * from authoritative table "select * from authoritative order by col1" @@ -113,6 +165,7 @@ "Table": "authoritative" } } +Gen4 error: unsupported: '*' expression in cross-shard query # ORDER BY on scatter with text column "select a, textcol1, b from user order by a, textcol1, b" @@ -131,8 +184,25 @@ "Table": "user" } } +{ + "QueryType": "SELECT", + "Original": "select a, textcol1, b from user order by a, textcol1, b", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select a, textcol1, b, weight_string(a), weight_string(b) from user where 1 != 1", + "OrderBy": "(0|3) ASC, 1 ASC, (2|4) ASC", + "Query": "select a, textcol1, b, weight_string(a), weight_string(b) from user order by a asc, textcol1 asc, b asc", + "ResultColumns": 3, + "Table": "user" + } +} -# ORDER BY on scatter with text column, qualified name +# ORDER BY on scatter with text column, qualified name TODO: can plan better "select a, user.textcol1, b from user order by a, textcol1, b" { "QueryType": "SELECT", @@ -149,6 +219,23 @@ "Table": "user" } } +{ + "QueryType": "SELECT", + "Original": "select a, user.textcol1, b from user order by a, textcol1, b", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select a, user.textcol1, b, weight_string(a), weight_string(b) from user where 1 != 1", + "OrderBy": "(0|3) ASC, 1 ASC, (2|4) ASC", + "Query": "select a, user.textcol1, b, weight_string(a), weight_string(b) from user order by a asc, textcol1 asc, b asc", + "ResultColumns": 3, + "Table": "user" + } +} # ORDER BY on scatter with multiple text columns "select a, textcol1, b, textcol2 from user order by a, textcol1, b, textcol2" @@ -167,10 +254,28 @@ "Table": "user" } } +{ + "QueryType": "SELECT", + "Original": "select a, textcol1, b, textcol2 from user order by a, textcol1, b, textcol2", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select a, textcol1, b, textcol2, weight_string(a), weight_string(b) from user where 1 != 1", + "OrderBy": "(0|4) ASC, 1 ASC, (2|5) ASC, 3 ASC", + "Query": "select a, textcol1, b, textcol2, weight_string(a), weight_string(b) from user order by a asc, textcol1 asc, b asc, textcol2 asc", + "ResultColumns": 4, + "Table": "user" + } +} # ORDER BY invalid col number on scatter "select col from user order by 2" "column number out of range: 2" +Gen4 error: Unknown column '2' in 'order clause' # ORDER BY NULL "select col from user order by null" @@ -189,6 +294,7 @@ "Table": "user" } } +Gen4 plan same as above # ORDER BY RAND() "select col from user order by RAND()" @@ -207,6 +313,7 @@ "Table": "user" } } +Gen4 plan same as above # Order by, '*' expression "select * from user where name = 5 order by col" @@ -229,6 +336,7 @@ "Vindex": "user_index" } } +Gen4 plan same as above # Order by, qualified '*' expression "select user.* from user where name = 5 order by user.col" @@ -251,6 +359,7 @@ "Vindex": "user_index" } } +Gen4 plan same as above # Order by, '*' expression with qualified reference "select * from user where name = 5 order by user.col" @@ -273,18 +382,22 @@ "Vindex": "user_index" } } +Gen4 plan same as above # Order by, qualified '*' expression, name mismatched. "select user.* from user where id = 5 order by e.col" "symbol e.col not found" +Gen4 plan same as above # Order by, invalid column number "select col from user order by 18446744073709551616" "error parsing column number: 18446744073709551616" +Gen4 plan same as above # Order by, out of range column number "select col from user order by 2" "column number out of range: 2" +Gen4 error: unknown column '2' in 'order clause' # Order by, '*' expression with qualified reference and using collate "select * from user where name = 5 order by user.col collate utf8_general_ci" @@ -307,6 +420,7 @@ "Vindex": "user_index" } } +Gen4 plan same as above #Order by with math functions "select * from user where name = 5 order by -col1" @@ -329,6 +443,7 @@ "Vindex": "user_index" } } +Gen4 plan same as above #Order by with string operations "select * from user where name = 5 order by concat(col,col1) collate utf8_general_ci desc" @@ -351,6 +466,7 @@ "Vindex": "user_index" } } +Gen4 plan same as above #Order by with math operations "select * from user where name = 5 order by name+col collate utf8_general_ci desc" @@ -373,6 +489,7 @@ "Vindex": "user_index" } } +Gen4 plan same as above # LIMIT "select col1 from user where name = 1 limit 1" @@ -395,6 +512,7 @@ "Vindex": "user_index" } } +Gen4 plan same as above # limit for scatter "select col from user limit 1" @@ -419,6 +537,7 @@ ] } } +Gen4 plan same as above # cross-shard expression in parenthesis with limit "select * from user where (id1 = 4 AND name1 ='abc') limit 5" @@ -443,11 +562,34 @@ ] } } +Gen4 plan same as above # invalid limit expression "select id from user limit 1+1" "unexpected expression in LIMIT: expression is too complex 1 + 1" +{ + "QueryType": "SELECT", + "Original": "select id from user limit 1+1", + "Instructions": { + "OperatorType": "Limit", + "Count": "2", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id from user where 1 != 1", + "Query": "select id from user limit :__upper_limit", + "Table": "user" + } + ] + } +} +# TODO: this should be 'Column 'col1' in having clause is ambiguous' # non-ambiguous symbol reference "select user.col1, user_extra.col1 from user join user_extra having user_extra.col1 = 2" { @@ -484,6 +626,40 @@ ] } } +{ + "QueryType": "SELECT", + "Original": "select user.col1, user_extra.col1 from user join user_extra having user_extra.col1 = 2", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": [-1,1], + "TableName": "user_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.col1 from user where 1 != 1", + "Query": "select user.col1 from user", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user_extra.col1 from user_extra where 1 != 1", + "Query": "select user_extra.col1 from user_extra where user_extra.col1 = 2", + "Table": "user_extra" + } + ] + } +} # HAVING multi-route "select user.col1 as a, user.col2, user_extra.col3 from user join user_extra having 1 = 1 and a = 1 and a = user.col2 and user_extra.col3 = 1" @@ -521,6 +697,40 @@ ] } } +{ + "QueryType": "SELECT", + "Original": "select user.col1 as a, user.col2, user_extra.col3 from user join user_extra having 1 = 1 and a = 1 and a = user.col2 and user_extra.col3 = 1", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": [-1,-2,1], + "TableName": "user_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.col1 as a, user.col2 from user where 1 != 1", + "Query": "select user.col1 as a, user.col2 from user where user.col1 = 1 and user.col1 = user.col2 and 1 = 1", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user_extra.col3 from user_extra where 1 != 1", + "Query": "select user_extra.col3 from user_extra where user_extra.col3 = 1 and 1 = 1", + "Table": "user_extra" + } + ] + } +} # ORDER BY NULL for join "select user.col1 as a, user.col2, music.col3 from user join music on user.name = music.id where user.name = 1 order by null" @@ -562,6 +772,40 @@ ] } } +{ + "QueryType": "SELECT", + "Original": "select user.col1 as a, user.col2, music.col3 from user join music on user.name = music.id where user.name = 1 order by null", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": [-2,-3,1], + "TableName": "user_music", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.name, user.col1 as a, user.col2 from user where 1 != 1", + "Query": "select user.name, user.col1 as a, user.col2 from user where user.name = 1", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select music.col3 from music where 1 != 1", + "Query": "select music.col3 from music where music.id = :user_name", + "Table": "music" + } + ] + } +} # ORDER BY non-key column for join "select user.col1 as a, user.col2, music.col3 from user join music on user.name = music.id where user.name = 1 order by a" @@ -603,6 +847,41 @@ ] } } +{ + "QueryType": "SELECT", + "Original": "select user.col1 as a, user.col2, music.col3 from user join music on user.name = music.id where user.name = 1 order by a", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": [-2,-3,1], + "TableName": "user_music", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.name, user.col1 as a, user.col2, weight_string(user.col1) from user where 1 != 1", + "OrderBy": "(1|3) ASC", + "Query": "select user.name, user.col1 as a, user.col2, weight_string(user.col1) from user where user.name = 1 order by a asc", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select music.col3 from music where 1 != 1", + "Query": "select music.col3 from music where music.id = :user_name", + "Table": "music" + } + ] + } +} # ORDER BY non-key column for implicit join "select user.col1 as a, user.col2, music.col3 from user, music where user.name = music.id and user.name = 1 order by a" @@ -644,16 +923,13 @@ ] } } - -# ORDER BY RAND() for join -"select user.col1 as a, user.col2, music.col3 from user join music on user.name = music.id where user.name = 1 order by RAND()" { "QueryType": "SELECT", - "Original": "select user.col1 as a, user.col2, music.col3 from user join music on user.name = music.id where user.name = 1 order by RAND()", + "Original": "select user.col1 as a, user.col2, music.col3 from user, music where user.name = music.id and user.name = 1 order by a", "Instructions": { "OperatorType": "Join", "Variant": "Join", - "JoinColumnIndexes": [-1,-2,1], + "JoinColumnIndexes": [-2,-3,1], "TableName": "user_music", "Inputs": [ { @@ -663,13 +939,10 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select user.col1 as a, user.col2, user.name from user where 1 != 1", - "Query": "select user.col1 as a, user.col2, user.name from user where user.name = 1 order by RAND()", - "Table": "user", - "Values": [ - "INT64(1)" - ], - "Vindex": "hash" + "FieldQuery": "select user.name, user.col1 as a, user.col2, weight_string(user.col1) from user where 1 != 1", + "OrderBy": "(1|3) ASC", + "Query": "select user.name, user.col1 as a, user.col2, weight_string(user.col1) from user where user.name = 1 order by a asc", + "Table": "user" }, { "OperatorType": "Route", @@ -679,23 +952,98 @@ "Sharded": true }, "FieldQuery": "select music.col3 from music where 1 != 1", - "Query": "select music.col3 from music where music.id = :user_name order by RAND()", + "Query": "select music.col3 from music where music.id = :user_name", "Table": "music" } ] } } -# limit for joins. Can't push down the limit because result -# counts get multiplied by join operations. -"select user.col from user join user_extra limit 1" +# ORDER BY RAND() for join +"select user.col1 as a, user.col2, music.col3 from user join music on user.name = music.id where user.name = 1 order by RAND()" { "QueryType": "SELECT", - "Original": "select user.col from user join user_extra limit 1", + "Original": "select user.col1 as a, user.col2, music.col3 from user join music on user.name = music.id where user.name = 1 order by RAND()", "Instructions": { - "OperatorType": "Limit", - "Count": 1, - "Inputs": [ + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": [-1,-2,1], + "TableName": "user_music", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.col1 as a, user.col2, user.name from user where 1 != 1", + "Query": "select user.col1 as a, user.col2, user.name from user where user.name = 1 order by RAND()", + "Table": "user", + "Values": [ + "INT64(1)" + ], + "Vindex": "hash" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select music.col3 from music where 1 != 1", + "Query": "select music.col3 from music where music.id = :user_name order by RAND()", + "Table": "music" + } + ] + } +} +{ + "QueryType": "SELECT", + "Original": "select user.col1 as a, user.col2, music.col3 from user join music on user.name = music.id where user.name = 1 order by RAND()", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": [-2,-3,1], + "TableName": "user_music", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select user.name, user.col1 as a, user.col2 from user where 1 != 1", + "Query": "select user.name, user.col1 as a, user.col2 from user where user.name = 1 order by RAND()", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select music.col3 from music where 1 != 1", + "Query": "select music.col3 from music where music.id = :user_name order by RAND()", + "Table": "music" + } + ] + } +} + +# limit for joins. Can't push down the limit because result +# counts get multiplied by join operations. +"select user.col from user join user_extra limit 1" +{ + "QueryType": "SELECT", + "Original": "select user.col from user join user_extra limit 1", + "Instructions": { + "OperatorType": "Limit", + "Count": 1, + "Inputs": [ { "OperatorType": "Join", "Variant": "Join", @@ -729,10 +1077,12 @@ ] } } +Gen4 plan same as above # ambiguous symbol reference "select user.col1, user_extra.col1 from user join user_extra having col1 = 2" "ambiguous symbol reference: col1" +Gen4 error: Column 'col1' in field list is ambiguous # HAVING uses subquery "select name from user having name in ( select col from user )" @@ -843,6 +1193,7 @@ } } + # ORDER BY RAND() after pull-out subquery "select col from user where col in ( select col2 from user ) order by rand()" { @@ -877,6 +1228,7 @@ ] } } +Gen4 plan same as above # Order by, '*' expression in a subquery "select u.name, e.name from user u join user_extra e where u.col = e.col and u.col in ( select * from user where user.name = u.name order by col )" @@ -914,11 +1266,46 @@ ] } } +{ + "QueryType": "SELECT", + "Original": "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.name = u.id order by col)", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": [-2,1], + "TableName": "user_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select u.col, u.id from user as u where 1 != 1", + "Query": "select u.col, u.id from user as u where u.col in (select * from user where user.name = u.id order by col asc)", + "Table": "user" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select e.id from user_extra as e where 1 != 1", + "Query": "select e.id from user_extra as e where e.col = :u_col", + "Table": "user_extra" + } + ] + } +} # Order by, verify outer symtab is searched according to its own context. "select u.name from user u having u.name in (select col2 from user where user.name = u.name order by u.col)" "symbol u.col not found in subquery" + # Order by subquery column "select * from user u join ( select user_id from user_extra where user_id = 5 ) eu on u.name = eu.user_id where u.name = 5 order by eu.user_id" { @@ -940,6 +1327,7 @@ "Vindex": "user_index" } } +Gen4 plan same as above # limit for scatter with bind var "select col from user limit :a" @@ -964,6 +1352,7 @@ ] } } +Gen4 plan same as above # scatter limit after pullout subquery "select col from user where col in ( select col1 from user ) limit 1" @@ -1005,3 +1394,526 @@ ] } } +Gen4 plan same as above + +# Distinct with column alias +"select distinct a as c, a from user" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select distinct a as c, a from user", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "GroupBy": "(0|2), (1|2)", + "ResultColumns": 2, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select a as c, a, weight_string(a) from user where 1 != 1", + "OrderBy": "(0|2) ASC, (0|2) ASC", + "Query": "select distinct a as c, a, weight_string(a) from user order by c asc, a asc", + "Table": "user" + } + ] + } +} + +# Distinct with cross shard query +"select distinct user.a from user join user_extra" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select distinct user.a from user join user_extra", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "GroupBy": "(0|1)", + "ResultColumns": 1, + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:0,L:1", + "TableName": "`user`_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select `user`.a, weight_string(`user`.a) from `user` where 1 != 1", + "OrderBy": "(0|1) ASC", + "Query": "select `user`.a, weight_string(`user`.a) from `user` order by `user`.a asc", + "Table": "`user`" + }, + { + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from user_extra where 1 != 1", + "Query": "select 1 from user_extra", + "Table": "user_extra" + } + ] + } + ] + } +} + +# Distinct with same column +"select distinct a, a from user" +"generating order by clause: ambiguous symbol reference: a" +{ + "QueryType": "SELECT", + "Original": "select distinct a, a from user", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "GroupBy": "(0|2), (1|2)", + "ResultColumns": 2, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select a, a, weight_string(a) from user where 1 != 1", + "OrderBy": "(0|2) ASC, (0|2) ASC", + "Query": "select distinct a, a, weight_string(a) from user order by a asc, a asc", + "Table": "user" + } + ] + } +} + +#Order by derived table 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" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "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", + "Instructions": { + "OperatorType": "Route", + "Variant": "EqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select * from `user` as u, (select user_id from user_extra where 1 != 1) as eu where 1 != 1", + "Query": "select * from `user` as u, (select user_id from user_extra where user_id = 5) as eu where u.id = 5 and u.id = eu.user_id order by eu.user_id asc", + "Table": "`user`, user_extra", + "Values": [ + "INT64(5)" + ], + "Vindex": "user_index" + } +} + +# ordering on the left side of the join +"select name from user, music order by name" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select name from user, music order by name", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:0", + "TableName": "`user`_music", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select `name`, weight_string(`name`) from `user` where 1 != 1", + "OrderBy": "(0|1) ASC", + "Query": "select `name`, weight_string(`name`) from `user` order by `name` asc", + "Table": "`user`" + }, + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from music where 1 != 1", + "Query": "select 1 from music", + "Table": "music" + } + ] + } +} + +# order by with ambiguous column reference ; valid in MySQL +"select id, id from user order by id" +"ambiguous symbol reference: id" +{ + "QueryType": "SELECT", + "Original": "select id, id from user order by id", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id, id, weight_string(id) from user where 1 != 1", + "OrderBy": "(0|2) ASC", + "Query": "select id, id, weight_string(id) from user order by id asc", + "ResultColumns": 2, + "Table": "user" + } +} + +# order by column alias +"select id as foo from music order by foo" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select id as foo from music order by foo", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id as foo, weight_string(id) from music where 1 != 1", + "OrderBy": "(0|1) ASC", + "Query": "select id as foo, weight_string(id) from music order by foo asc", + "ResultColumns": 1, + "Table": "music" + } +} + +# join order by with ambiguous column reference ; valid in MySQL +"select name, name from user, music order by `name`" +"ambiguous symbol reference: `name`" +{ + "QueryType": "SELECT", + "Original": "select name, name from user, music order by name", + "Instructions": { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:0,L:0", + "TableName": "`user`_music", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select `name`, weight_string(`name`) from `user` where 1 != 1", + "OrderBy": "(0|1) ASC", + "Query": "select `name`, weight_string(`name`) from `user` order by `name` asc", + "Table": "`user`" + }, + { + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from music where 1 != 1", + "Query": "select 1 from music", + "Table": "music" + } + ] + } +} + +# column alias for a table column in order by +"select id as foo, id2 as id from music order by id" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select id as foo, id2 as id from music order by id", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id as foo, id2 as id, weight_string(id2) from music where 1 != 1", + "OrderBy": "(1|2) ASC", + "Query": "select id as foo, id2 as id, weight_string(id2) from music order by id asc", + "ResultColumns": 2, + "Table": "music" + } +} + +# aggregation and non-aggregations column without group by +"select count(id), num from user" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select count(id), num from user", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Scalar", + "Aggregates": "sum_count(0) AS count(id), random(1) AS num", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select count(id), num from user where 1 != 1", + "Query": "select count(id), num from user", + "Table": "user" + } + ] + } +} + +# aggregation and non-aggregations column with order by +"select count(id), num from user order by 2" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select count(id), num from user order by 2", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Scalar", + "Aggregates": "sum_count(0) AS count(id), random(1) AS num", + "ResultColumns": 2, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select count(id), num, weight_string(num) from user where 1 != 1", + "OrderBy": "(1|2) ASC", + "Query": "select count(id), num, weight_string(num) from user order by num asc", + "Table": "user" + } + ] + } +} + +# aggregation and non-aggregations column with group by and order by +"select count(id), num from user group by 2 order by 1" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select count(id), num from user group by 2 order by 1", + "Instructions": { + "OperatorType": "Sort", + "Variant": "Memory", + "OrderBy": "0 ASC", + "ResultColumns": 2, + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count(0) AS count(id)", + "GroupBy": "(1|2)", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select count(id), num, weight_string(num) from user where 1 != 1 group by num, weight_string(num)", + "OrderBy": "(1|2) ASC", + "Query": "select count(id), num, weight_string(num) from user group by num, weight_string(num) order by num asc", + "Table": "user" + } + ] + } + ] + } +} + +# aggregation and non-aggregations column with group by +"select count(id), num from user group by 2" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select count(id), num from user group by 2", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count(0) AS count(id)", + "GroupBy": "(1|2)", + "ResultColumns": 2, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select count(id), num, weight_string(num) from user where 1 != 1 group by num, weight_string(num)", + "OrderBy": "(1|2) ASC", + "Query": "select count(id), num, weight_string(num) from user group by num, weight_string(num) order by num asc", + "Table": "user" + } + ] + } +} + +# Scatter order by and aggregation: order by column must reference column from select list +"select col, count(*) from user group by col order by c1" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select col, count(*) from user group by col order by c1", + "Instructions": { + "OperatorType": "Aggregate", + "Variant": "Ordered", + "Aggregates": "sum_count_star(1) AS count(*), random(2) AS c1", + "GroupBy": "0", + "ResultColumns": 2, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select col, count(*), c1, weight_string(c1) from user where 1 != 1 group by col", + "OrderBy": "(2|3) ASC, 0 ASC", + "Query": "select col, count(*), c1, weight_string(c1) from user group by col order by c1 asc, col asc", + "Table": "user" + } + ] + } +} + +# ORDER BY on select t.* TODO:*后面的表达式排序 原测试用例第三代是报错,vtdriver执行不报错 +"select t.*, t.col from user t order by t.col" +Gen3 skip +Gen4 error: unsupported: '*' expression in cross-shard query + +# ORDER BY on select multi * +"select *, name, *, col from user order by col" +Gen3 skip +Gen4 error: unsupported: '*' expression in cross-shard query + +# ORDER BY on select multi t.* +"select t.*, t.name, t.*, t.col from user t order by t.col" +Gen3 skip +Gen4 error: unsupported: '*' expression in cross-shard query + +# ORDER BY on select * +"select *, col from user order by col" +Gen3 skip +Gen4 error: unsupported: '*' expression in cross-shard query + +# ORDER BY column offset +"select id as foo from music order by 1" +Gen3 skip +{ + "QueryType": "SELECT", + "Original": "select id as foo from music order by 1", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select id as foo, weight_string(id) from music where 1 != 1", + "OrderBy": "(0|1) ASC", + "Query": "select id as foo, weight_string(id) from music order by foo asc", + "ResultColumns": 1, + "Table": "music" + } +} + +# Equal filter with hexadecimal value +"select count(*) a from user having a = 0x01" +"unsupported: filtering on results of aggregates" +{ + "QueryType": "SELECT", + "Original": "select count(*) a from user having a = 0x01", + "Instructions": { + "OperatorType": "Filter", + "Predicate": "a = 0x01", + "Inputs": [ + { + "OperatorType": "Aggregate", + "Variant": "Scalar", + "Aggregates": "sum_count_star(0) AS a", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select count(*) as a from user where 1 != 1", + "Query": "select count(*) as a from user", + "Table": "user" + } + ] + } + ] + } +} + +# Order by has subqueries +"select id from unsharded order by (select id from unsharded)" +{ + "QueryType": "SELECT", + "Original": "select id from unsharded order by (select id from unsharded)", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select id from unsharded where 1 != 1", + "Query": "select id from unsharded order by ( select id from unsharded ) asc", + "Table": "unsharded" + } +} +{ + "QueryType": "SELECT", + "Original": "select id from unsharded order by (select id from unsharded)", + "Instructions": { + "OperatorType": "Route", + "Variant": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select id from unsharded where 1 != 1", + "Query": "select id from unsharded order by ( select id from unsharded )", + "Table": "unsharded" + } +} + diff --git a/src/test/resources/plan/postprocess_cases_unsupported.txt b/src/test/resources/plan/postprocess_cases_unsupported.txt index 622300c..aaf6ae2 100644 --- a/src/test/resources/plan/postprocess_cases_unsupported.txt +++ b/src/test/resources/plan/postprocess_cases_unsupported.txt @@ -6,20 +6,21 @@ "Original": "select col from route1 where id = 1 order by col", "Instructions": { "OperatorType": "Route", - "Variant": "SelectEqualUnique", + "Variant": "EqualUnique", "Keyspace": { "Name": "user", "Sharded": true }, - "FieldQuery": "select col from user as route1 where 1 != 1", - "Query": "select col from user as route1 where id = 1 order by col asc", - "Table": "user", + "FieldQuery": "select col from `user` as route1 where 1 != 1", + "Query": "select col from `user` as route1 where id = 1 order by col asc", + "Table": "`user`", "Values": [ - 1 + "INT64(1)" ], "Vindex": "user_index" } } +Gen4 plan same as above # Reason: reference table # limit on reference table @@ -38,4 +39,5 @@ "Query": "select col from ref limit 1", "Table": "ref" } -} \ No newline at end of file +} + diff --git a/src/test/resources/plan/select_cases.txt b/src/test/resources/plan/select_cases.txt index ee4f7c4..c445c7d 100644 --- a/src/test/resources/plan/select_cases.txt +++ b/src/test/resources/plan/select_cases.txt @@ -442,8 +442,8 @@ } # Cannot auto-resolve for cross-shard joins -"select col from user join user_extra" -"symbol col not found" +"select col1 from user join user_extra" +"symbol col1 not found" # Auto-resolve should work if unique vindex columns are referenced "select name, user_id from user join user_extra" diff --git a/src/test/resources/plan/union_cases.txt b/src/test/resources/plan/union_cases.txt index 74b5bcb..c2eda4a 100644 --- a/src/test/resources/plan/union_cases.txt +++ b/src/test/resources/plan/union_cases.txt @@ -31,6 +31,7 @@ ] } } +Gen4 plan same as above # union all between two SelectEqualUnique "select id from user where name = 1 union all select id from user where name = 5" @@ -73,6 +74,33 @@ ] } } +Gen4 plan same as above + +# union distinct between two scatter selects +# Vitess支持,VtDriver gen3报错 +"select name from user union select user_id from music" +"unsupported: UNION cannot be executed as a single route" +{ + "QueryType": "SELECT", + "Original": "select name from user union select user_id from music", + "Instructions": { + "OperatorType": "Distinct", + "ResultColumns": 1, + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select name, weight_string(name) from user where 1 != 1 union select user_id, weight_string(user_id) from music where 1 != 1", + "Query": "select name, weight_string(name) from user union select user_id, weight_string(user_id) from music", + "Table": "user" + } + ] + } +} #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)" @@ -119,6 +147,7 @@ ] } } +Gen4 plan same as above # Union all "select col1, col2 from user union all select col1, col2 from user_extra" @@ -153,6 +182,7 @@ ] } } +Gen4 plan same as above # union operations in subqueries (FROM) "select * from (select * from user union all select * from user_extra) as t" @@ -195,6 +225,7 @@ ] } } +Gen4 plan same as above # union all between two scatter selects, with order by "(select id from user order by id limit 5) union all (select id from music order by id desc limit 5)" @@ -241,6 +272,7 @@ ] } } +Gen4 plan same as above # union all on scatter and single route "select id from user where name = 1 union select id from user where name = 1 union all select id from user" @@ -279,3 +311,4 @@ ] } } +Gen4 plan same as above