Berkof commented on a change in pull request #9276: URL: https://github.com/apache/ignite/pull/9276#discussion_r693547556
########## File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdColumnOrigins.java ########## @@ -0,0 +1,345 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.ignite.internal.processors.query.calcite.metadata; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import org.apache.calcite.plan.RelOptTable; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.Aggregate; +import org.apache.calcite.rel.core.AggregateCall; +import org.apache.calcite.rel.core.Calc; +import org.apache.calcite.rel.core.Exchange; +import org.apache.calcite.rel.core.Filter; +import org.apache.calcite.rel.core.Join; +import org.apache.calcite.rel.core.Project; +import org.apache.calcite.rel.core.SetOp; +import org.apache.calcite.rel.core.Sort; +import org.apache.calcite.rel.core.TableFunctionScan; +import org.apache.calcite.rel.core.TableModify; +import org.apache.calcite.rel.metadata.BuiltInMetadata; +import org.apache.calcite.rel.metadata.MetadataDef; +import org.apache.calcite.rel.metadata.MetadataHandler; +import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider; +import org.apache.calcite.rel.metadata.RelColumnMapping; +import org.apache.calcite.rel.metadata.RelColumnOrigin; +import org.apache.calcite.rel.metadata.RelMetadataProvider; +import org.apache.calcite.rel.metadata.RelMetadataQuery; +import org.apache.calcite.rex.RexInputRef; +import org.apache.calcite.rex.RexLocalRef; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexShuttle; +import org.apache.calcite.rex.RexVisitor; +import org.apache.calcite.rex.RexVisitorImpl; +import org.apache.calcite.util.BuiltInMethod; +import org.apache.ignite.internal.processors.query.calcite.rel.ProjectableFilterableTableScan; +import org.jetbrains.annotations.Nullable; + +/** + * RelMdColumnOrigins supplies a default implementation of + * {@link RelMetadataQuery#getColumnOrigins} for the standard logical algebra. + */ +public class IgniteMdColumnOrigins implements MetadataHandler<BuiltInMetadata.ColumnOrigin> { + /** */ + public static final RelMetadataProvider SOURCE = ReflectiveRelMetadataProvider.reflectiveSource( + BuiltInMethod.COLUMN_ORIGIN.method, new IgniteMdColumnOrigins()); + + /** {@inheritDoc} */ + @Override public MetadataDef<BuiltInMetadata.ColumnOrigin> getDef() { + return BuiltInMetadata.ColumnOrigin.DEF; + } + + /** */ + public @Nullable Set<RelColumnOrigin> getColumnOrigins(Aggregate rel, RelMetadataQuery mq, int iOutputColumn) { + if (iOutputColumn < rel.getGroupCount()) { + // get actual index of Group columns. + return mq.getColumnOrigins(rel.getInput(), rel.getGroupSet().asList().get(iOutputColumn)); + } + + // Aggregate columns are derived from input columns + AggregateCall call = rel.getAggCallList().get(iOutputColumn - rel.getGroupCount()); + + final Set<RelColumnOrigin> set = new HashSet<>(); + + for (Integer iInput : call.getArgList()) { + Set<RelColumnOrigin> inputSet = mq.getColumnOrigins(rel.getInput(), iInput); + inputSet = createDerivedColumnOrigins(inputSet); + + if (inputSet != null) + set.addAll(inputSet); + + } + return set; + } + + /** */ + public @Nullable Set<RelColumnOrigin> getColumnOrigins(Join rel, RelMetadataQuery mq, + int iOutputColumn) { + int nLeftColumns = rel.getLeft().getRowType().getFieldList().size(); + Set<RelColumnOrigin> set; + boolean derived = false; + + if (iOutputColumn < nLeftColumns) { + set = mq.getColumnOrigins(rel.getLeft(), iOutputColumn); + + if (rel.getJoinType().generatesNullsOnLeft()) + derived = true; + + } + else { + set = mq.getColumnOrigins(rel.getRight(), iOutputColumn - nLeftColumns); + + if (rel.getJoinType().generatesNullsOnRight()) + derived = true; + } + + if (derived) { + // nulls are generated due to outer join; that counts + // as derivation + set = createDerivedColumnOrigins(set); + } + return set; + } + + /** */ + public @Nullable Set<RelColumnOrigin> getColumnOrigins(SetOp rel, RelMetadataQuery mq, int iOutputColumn) { + final Set<RelColumnOrigin> set = new HashSet<>(); + + for (RelNode input : rel.getInputs()) { + Set<RelColumnOrigin> inputSet = mq.getColumnOrigins(input, iOutputColumn); + + if (inputSet == null) + return null; + + set.addAll(inputSet); + } + + return set; + } + + /** */ + public @Nullable Set<RelColumnOrigin> getColumnOrigins(Project rel, + final RelMetadataQuery mq, int iOutputColumn) { + final RelNode input = rel.getInput(); + RexNode rexNode = rel.getProjects().get(iOutputColumn); + + if (rexNode instanceof RexInputRef) { + // Direct reference: no derivation added. + RexInputRef inputRef = (RexInputRef) rexNode; + + return mq.getColumnOrigins(input, inputRef.getIndex()); + } + // Anything else is a derivation, possibly from multiple columns. + final Set<RelColumnOrigin> set = getMultipleColumns(rexNode, input, mq); + + return createDerivedColumnOrigins(set); + } + + /** */ + public @Nullable Set<RelColumnOrigin> getColumnOrigins(Calc rel, final RelMetadataQuery mq, int iOutputColumn) { + final RelNode input = rel.getInput(); + final RexShuttle rexShuttle = new RexShuttle() { + @Override public RexNode visitLocalRef(RexLocalRef localRef) { + return rel.getProgram().expandLocalRef(localRef); + } + }; + final List<RexNode> projects = new ArrayList<>(); + + for (RexNode rex: rexShuttle.apply(rel.getProgram().getProjectList())) + projects.add(rex); + + final RexNode rexNode = projects.get(iOutputColumn); + + if (rexNode instanceof RexInputRef) { + // Direct reference: no derivation added. + RexInputRef inputRef = (RexInputRef) rexNode; + return mq.getColumnOrigins(input, inputRef.getIndex()); + } + // Anything else is a derivation, possibly from multiple columns. + final Set<RelColumnOrigin> set = getMultipleColumns(rexNode, input, mq); + + return createDerivedColumnOrigins(set); + } + + /** */ + public @Nullable Set<RelColumnOrigin> getColumnOrigins(Filter rel, RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + /** */ + public @Nullable Set<RelColumnOrigin> getColumnOrigins(Sort rel, RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + /** */ + public @Nullable Set<RelColumnOrigin> getColumnOrigins(TableModify rel, RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + /** */ + public @Nullable Set<RelColumnOrigin> getColumnOrigins(Exchange rel, RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + /** */ + public @Nullable Set<RelColumnOrigin> getColumnOrigins( + TableFunctionScan rel, + RelMetadataQuery mq, + int iOutputColumn + ) { + Set<RelColumnMapping> mappings = rel.getColumnMappings(); + + if (mappings == null) { + if (!rel.getInputs().isEmpty()) { + // This is a non-leaf transformation: say we don't + // know about origins, because there are probably + // columns below. + return null; + } + else { + // This is a leaf transformation: say there are for sure no + // column origins. + return Collections.emptySet(); + } + } + + final Set<RelColumnOrigin> set = new HashSet<>(); + + for (RelColumnMapping mapping : mappings) { + if (mapping.iOutputColumn != iOutputColumn) + continue; + + final RelNode input = rel.getInputs().get(mapping.iInputRel); + final int column = mapping.iInputColumn; + Set<RelColumnOrigin> origins = mq.getColumnOrigins(input, column); + + if (origins == null) + return null; + + if (mapping.derived) + origins = createDerivedColumnOrigins(origins); + + set.addAll(origins); + } + + return set; + } + + /** + * Get column origins. + * + * @param rel Rel to get origins from. + * @param mq Rel metadata query. + * @param iOutputColumn Column idx. + * @return Set of column origins. + */ + public @Nullable Set<RelColumnOrigin> getColumnOrigins( + ProjectableFilterableTableScan rel, Review comment: no ########## File path: modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/ServerStatisticsIntegrationTest.java ########## @@ -0,0 +1,695 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.ignite.internal.processors.query.calcite.integration; + +import java.math.BigInteger; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; +import java.util.Arrays; +import java.util.HashSet; +import java.util.Set; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cache.QueryEntity; +import org.apache.ignite.cache.query.annotations.QuerySqlField; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.query.QueryEngine; +import org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor; +import org.apache.ignite.internal.processors.query.calcite.QueryChecker; +import org.apache.ignite.internal.processors.query.calcite.util.Commons; +import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing; +import org.apache.ignite.internal.processors.query.stat.IgniteStatisticsManager; +import org.apache.ignite.internal.processors.query.stat.StatisticsKey; +import org.apache.ignite.internal.processors.query.stat.StatisticsTarget; +import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.testframework.GridTestUtils; +import org.junit.Test; + +/** + * Tests for server side statistics usage. + */ +public class ServerStatisticsIntegrationTest extends AbstractBasicIntegrationTest { Review comment: Added some planer test too. ########## File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdSelectivity.java ########## @@ -110,6 +121,646 @@ public Double getSelectivity(IgniteSortedIndexSpool rel, RelMetadataQuery mq, Re } /** */ + public Double getSelectivity(RelSubset rel, RelMetadataQuery mq, RexNode predicate) { + RelNode best = rel.getBest(); + if (best == null) + return super.getSelectivity(rel, mq, predicate); + + return getSelectivity(best, mq, predicate); + } + + /** + * Convert specified value into comparable type: BigDecimal, + * + * @param val Value to convert to comparable form. + * @return Comparable form of value. + */ + private BigDecimal toComparableValue(RexLiteral val) { + RelDataType type = val.getType(); + + if (type instanceof BasicSqlType) { + BasicSqlType bType = (BasicSqlType)type; + + switch ((SqlTypeFamily)bType.getFamily()) { + case NULL: + return null; + + case NUMERIC: + return val.getValueAs(BigDecimal.class); + + case DATE: + return new BigDecimal(val.getValueAs(DateString.class).getMillisSinceEpoch()); + + case TIME: + return new BigDecimal(val.getValueAs(TimeString.class).getMillisOfDay()); + + case TIMESTAMP: + return new BigDecimal(val.getValueAs(TimestampString.class).getMillisSinceEpoch()); + + case BOOLEAN: + return (val.getValueAs(Boolean.class)) ? BigDecimal.ONE : BigDecimal.ZERO; + + default: + return null; + } + } + + return null; + } + + /** + * Convert specified value into comparable type: BigDecimal, + * + * @param val Value to convert to comparable form. + * @return Comparable form of value. + */ + private BigDecimal toComparableValue(Value val) { + if (val == null) + return null; + + switch (val.getType()) { + case Value.NULL: + throw new IllegalArgumentException("Can't compare null values"); + + case Value.BOOLEAN: + return (val.getBoolean()) ? BigDecimal.ONE : BigDecimal.ZERO; + + case Value.BYTE: + return new BigDecimal(val.getByte()); + + case Value.SHORT: + return new BigDecimal(val.getShort()); + + case Value.INT: + return new BigDecimal(val.getInt()); + + case Value.LONG: + return new BigDecimal(val.getLong()); + + case Value.DECIMAL: + return val.getBigDecimal(); + + case Value.DOUBLE: + return BigDecimal.valueOf(val.getDouble()); + + case Value.FLOAT: + return BigDecimal.valueOf(val.getFloat()); + + case Value.DATE: + return BigDecimal.valueOf(val.getDate().getTime()); + + case Value.TIME: + return BigDecimal.valueOf(val.getTime().getTime()); + + case Value.TIMESTAMP: + return BigDecimal.valueOf(val.getTimestamp().getTime()); + + case Value.BYTES: + BigInteger bigInteger = new BigInteger(1, val.getBytes()); + return new BigDecimal(bigInteger); + + case Value.STRING: + case Value.STRING_FIXED: + case Value.STRING_IGNORECASE: + case Value.ARRAY: + case Value.JAVA_OBJECT: + case Value.GEOMETRY: + return null; + + case Value.UUID: + BigInteger bigInt = new BigInteger(1, val.getBytes()); + return new BigDecimal(bigInt); + + default: + throw new IllegalStateException("Unsupported H2 type: " + val.getType()); + } + } + + /** + * Predicate based selectivity for table. Estimate condition on each column taking in comparison it's statistics. + * + * @param rel Original rel node to fallback calculation by. + * @param tbl Underlying IgniteTable. + * @param mq RelMetadataQuery. + * @param predicate Predicate to estimate selectivity by. + * @return Selectivity. + */ + private double getTablePredicateBasedSelectivity( + RelNode rel, + IgniteTable tbl, + RelMetadataQuery mq, + RexNode predicate + ) { + if (tbl == null) + return RelMdUtil.guessSelectivity(predicate); + + double sel = 1.0; + + Map<RexSlot, Boolean> addNotNull = new HashMap<>(); + + for (RexNode pred : RelOptUtil.conjunctions(predicate)) { + SqlKind predKind = pred.getKind(); + RexLocalRef op = getOperand(pred, RexLocalRef.class); + + if (predKind == SqlKind.OR) { + double orSelTotal = 1; + + for (RexNode orPred : RelOptUtil.disjunctions(pred)) + orSelTotal *= 1 - getTablePredicateBasedSelectivity(rel, tbl, mq, orPred); + + + sel *= 1 - orSelTotal; + } + else if (predKind == SqlKind.NOT) { + if (op == null) + sel *= guessSelectivity(pred); + else { + tryAddNotNull(addNotNull, tbl, op); + + sel *= 1 - getTablePredicateBasedSelectivity(rel, tbl, mq, op); + } + } + else if (predKind == SqlKind.LOCAL_REF) { + if (op != null) + addNotNull.put(op, Boolean.TRUE); + + sel *= estimateRefSelectivity(rel, mq, tbl, (RexLocalRef)pred); + } else if (predKind == SqlKind.IS_NULL) { + if (op != null) + addNotNull.put(op, Boolean.FALSE); + + sel *= estimateIsNullSelectivity(rel, mq, tbl, pred); + + } else if (predKind == SqlKind.IS_NOT_NULL) { + if (op != null) + addNotNull.put(op, Boolean.FALSE); + + sel *= estimateIsNotNullSelectivity(rel, mq, tbl, pred); + } else if (predKind == SqlKind.EQUALS) { + if (op != null) + addNotNull.put(op, Boolean.TRUE); + + sel *= estimateEqualsSelectivity(rel, mq, tbl, pred); + } else if (predKind.belongsTo(SqlKind.COMPARISON)) { + if (op != null) + addNotNull.put(op, Boolean.TRUE); + + sel *= estimateComparisonSelectivity(rel, mq, tbl, pred); + } else + sel *= .25; + } + + // Estimate not null selectivity in addition to comparison. + for (Map.Entry<RexSlot, Boolean> colAddNotNull : addNotNull.entrySet()) { + if (colAddNotNull.getValue()) { + ColumnStatistics colStat = getColStatBySlot(rel, mq, tbl, colAddNotNull.getKey()); + + sel *= (colStat == null) ? NOT_NULL_SELECTIVITY : estimateNotNullSelectivity(colStat); + } + } + + return sel; + } + + /** + * Try to add operand "not null" flag if there are no false flag for it. + * + * @param addNotNull Map with "add not null" flags for operands. + * @param tbl IgniteTable. + * @param op RexSlot to add operand by. + */ + private void tryAddNotNull(Map<RexSlot, Boolean> addNotNull, IgniteTable tbl, RexSlot op) { Review comment: Put if absent or not False. ########## File path: modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/QueryChecker.java ########## @@ -89,6 +90,54 @@ return containsSubPlan("IgniteIndexScan(table=[[" + schema + ", " + tblName + "]], index=[" + idxName + ']'); } + /** + * Ignite result row count mather. + * + * @param rowCount Expected result row count. + * @return Mather. + */ + public static Matcher<String> containsResultRowCount(double rowCount) { + String rowCountStr = String.format(".*rowcount = %s,.*", printCostVal(rowCount)); + + return new RegexpMather(rowCountStr); + } + + /** + * Regexp string mather. + */ + private static class RegexpMather extends CustomTypeSafeMatcher<String> { + /** Compilled pathern. */ + private Pattern pattern; + + /** + * Constructor. + * + * @param regexp Regexp to search. + */ + public RegexpMather(String regexp) { + super(regexp); + + pattern = Pattern.compile(regexp, Pattern.DOTALL); + } + + /** {@inheritDoc} */ + @Override protected boolean matchesSafely(String item) { + java.util.regex.Matcher matcher = pattern.matcher(item); + + return matcher.matches(); + } + } + + /** + * Print cost value to regexp. + * + * @param val Value to print. + * @return String representation for regexp. + */ + private static String printCostVal(Double val) { Review comment: removed ########## File path: modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/QueryCheckerTest.java ########## @@ -20,24 +20,50 @@ import org.hamcrest.Matcher; import org.junit.Test; +import static org.apache.ignite.internal.processors.query.calcite.QueryChecker.containsResultRowCount; import static org.apache.ignite.internal.processors.query.calcite.QueryChecker.matchesOnce; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; /** Query checker tests. */ public class QueryCheckerTest { + /** */ + private static final String PLAN = "IgniteSingleHashAggregate(group=[{}], COUNT(NAME)=[COUNT($0)]): rowcount = 1.0, " + Review comment: removed ########## File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdSelectivity.java ########## @@ -110,6 +121,646 @@ public Double getSelectivity(IgniteSortedIndexSpool rel, RelMetadataQuery mq, Re } /** */ + public Double getSelectivity(RelSubset rel, RelMetadataQuery mq, RexNode predicate) { + RelNode best = rel.getBest(); + if (best == null) + return super.getSelectivity(rel, mq, predicate); + + return getSelectivity(best, mq, predicate); + } + + /** + * Convert specified value into comparable type: BigDecimal, + * + * @param val Value to convert to comparable form. + * @return Comparable form of value. + */ + private BigDecimal toComparableValue(RexLiteral val) { + RelDataType type = val.getType(); + + if (type instanceof BasicSqlType) { + BasicSqlType bType = (BasicSqlType)type; + + switch ((SqlTypeFamily)bType.getFamily()) { + case NULL: + return null; + + case NUMERIC: + return val.getValueAs(BigDecimal.class); + + case DATE: + return new BigDecimal(val.getValueAs(DateString.class).getMillisSinceEpoch()); + + case TIME: + return new BigDecimal(val.getValueAs(TimeString.class).getMillisOfDay()); + + case TIMESTAMP: + return new BigDecimal(val.getValueAs(TimestampString.class).getMillisSinceEpoch()); + + case BOOLEAN: + return (val.getValueAs(Boolean.class)) ? BigDecimal.ONE : BigDecimal.ZERO; + + default: + return null; + } + } + + return null; + } + + /** + * Convert specified value into comparable type: BigDecimal, + * + * @param val Value to convert to comparable form. + * @return Comparable form of value. + */ + private BigDecimal toComparableValue(Value val) { + if (val == null) + return null; + + switch (val.getType()) { + case Value.NULL: + throw new IllegalArgumentException("Can't compare null values"); + + case Value.BOOLEAN: + return (val.getBoolean()) ? BigDecimal.ONE : BigDecimal.ZERO; + + case Value.BYTE: + return new BigDecimal(val.getByte()); + + case Value.SHORT: + return new BigDecimal(val.getShort()); + + case Value.INT: + return new BigDecimal(val.getInt()); + + case Value.LONG: + return new BigDecimal(val.getLong()); + + case Value.DECIMAL: + return val.getBigDecimal(); + + case Value.DOUBLE: + return BigDecimal.valueOf(val.getDouble()); + + case Value.FLOAT: + return BigDecimal.valueOf(val.getFloat()); + + case Value.DATE: + return BigDecimal.valueOf(val.getDate().getTime()); + + case Value.TIME: + return BigDecimal.valueOf(val.getTime().getTime()); + + case Value.TIMESTAMP: + return BigDecimal.valueOf(val.getTimestamp().getTime()); + + case Value.BYTES: + BigInteger bigInteger = new BigInteger(1, val.getBytes()); + return new BigDecimal(bigInteger); + + case Value.STRING: + case Value.STRING_FIXED: + case Value.STRING_IGNORECASE: + case Value.ARRAY: + case Value.JAVA_OBJECT: + case Value.GEOMETRY: + return null; + + case Value.UUID: + BigInteger bigInt = new BigInteger(1, val.getBytes()); + return new BigDecimal(bigInt); + + default: + throw new IllegalStateException("Unsupported H2 type: " + val.getType()); + } + } + + /** + * Predicate based selectivity for table. Estimate condition on each column taking in comparison it's statistics. + * + * @param rel Original rel node to fallback calculation by. + * @param tbl Underlying IgniteTable. + * @param mq RelMetadataQuery. + * @param predicate Predicate to estimate selectivity by. + * @return Selectivity. + */ + private double getTablePredicateBasedSelectivity( + RelNode rel, + IgniteTable tbl, + RelMetadataQuery mq, + RexNode predicate + ) { + if (tbl == null) + return RelMdUtil.guessSelectivity(predicate); + + double sel = 1.0; + + Map<RexSlot, Boolean> addNotNull = new HashMap<>(); + + for (RexNode pred : RelOptUtil.conjunctions(predicate)) { + SqlKind predKind = pred.getKind(); + RexLocalRef op = getOperand(pred, RexLocalRef.class); + + if (predKind == SqlKind.OR) { + double orSelTotal = 1; + + for (RexNode orPred : RelOptUtil.disjunctions(pred)) + orSelTotal *= 1 - getTablePredicateBasedSelectivity(rel, tbl, mq, orPred); + Review comment: removed ########## File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdColumnOrigins.java ########## @@ -0,0 +1,362 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.ignite.internal.processors.query.calcite.metadata; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import org.apache.calcite.plan.RelOptTable; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.Aggregate; +import org.apache.calcite.rel.core.AggregateCall; +import org.apache.calcite.rel.core.Calc; +import org.apache.calcite.rel.core.Exchange; +import org.apache.calcite.rel.core.Filter; +import org.apache.calcite.rel.core.Join; +import org.apache.calcite.rel.core.Project; +import org.apache.calcite.rel.core.SetOp; +import org.apache.calcite.rel.core.Sort; +import org.apache.calcite.rel.core.TableFunctionScan; +import org.apache.calcite.rel.core.TableModify; +import org.apache.calcite.rel.metadata.BuiltInMetadata; +import org.apache.calcite.rel.metadata.MetadataDef; +import org.apache.calcite.rel.metadata.MetadataHandler; +import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider; +import org.apache.calcite.rel.metadata.RelColumnMapping; +import org.apache.calcite.rel.metadata.RelColumnOrigin; +import org.apache.calcite.rel.metadata.RelMetadataProvider; +import org.apache.calcite.rel.metadata.RelMetadataQuery; +import org.apache.calcite.rex.RexInputRef; +import org.apache.calcite.rex.RexLocalRef; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexShuttle; +import org.apache.calcite.rex.RexVisitor; +import org.apache.calcite.rex.RexVisitorImpl; +import org.apache.calcite.util.BuiltInMethod; +import org.apache.ignite.internal.processors.query.calcite.rel.ProjectableFilterableTableScan; +import org.jetbrains.annotations.Nullable; + +/** + * RelMdColumnOrigins supplies a default implementation of + * {@link RelMetadataQuery#getColumnOrigins} for the standard logical algebra. + */ +public class IgniteMdColumnOrigins implements MetadataHandler<BuiltInMetadata.ColumnOrigin> { + /** */ + public static final RelMetadataProvider SOURCE = ReflectiveRelMetadataProvider.reflectiveSource( + BuiltInMethod.COLUMN_ORIGIN.method, new IgniteMdColumnOrigins()); + + /** {@inheritDoc} */ + @Override public MetadataDef<BuiltInMetadata.ColumnOrigin> getDef() { + return BuiltInMetadata.ColumnOrigin.DEF; + } + + /** */ + public @Nullable Set<RelColumnOrigin> getColumnOrigins(Aggregate rel, RelMetadataQuery mq, int iOutputColumn) { + if (iOutputColumn < rel.getGroupCount()) { + // get actual index of Group columns. + return mq.getColumnOrigins(rel.getInput(), rel.getGroupSet().asList().get(iOutputColumn)); + } + + // Aggregate columns are derived from input columns + AggregateCall call = rel.getAggCallList().get(iOutputColumn - rel.getGroupCount()); + + final Set<RelColumnOrigin> set = new HashSet<>(); + + for (Integer iInput : call.getArgList()) { + Set<RelColumnOrigin> inputSet = mq.getColumnOrigins(rel.getInput(), iInput); + inputSet = createDerivedColumnOrigins(inputSet); + + if (inputSet != null) + set.addAll(inputSet); + + } + return set; + } + + /** */ + public @Nullable Set<RelColumnOrigin> getColumnOrigins(Join rel, RelMetadataQuery mq, + int iOutputColumn) { + int nLeftColumns = rel.getLeft().getRowType().getFieldList().size(); + Set<RelColumnOrigin> set; + boolean derived = false; + + if (iOutputColumn < nLeftColumns) { + set = mq.getColumnOrigins(rel.getLeft(), iOutputColumn); + + if (rel.getJoinType().generatesNullsOnLeft()) + derived = true; + + } + else { + set = mq.getColumnOrigins(rel.getRight(), iOutputColumn - nLeftColumns); + + if (rel.getJoinType().generatesNullsOnRight()) + derived = true; + } + + if (derived) { + // nulls are generated due to outer join; that counts + // as derivation + set = createDerivedColumnOrigins(set); + } + return set; + } + + /** */ + public @Nullable Set<RelColumnOrigin> getColumnOrigins(SetOp rel, RelMetadataQuery mq, int iOutputColumn) { + final Set<RelColumnOrigin> set = new HashSet<>(); + + for (RelNode input : rel.getInputs()) { + Set<RelColumnOrigin> inputSet = mq.getColumnOrigins(input, iOutputColumn); + + if (inputSet == null) + return null; + + set.addAll(inputSet); + } + + return set; + } + + /** */ + public @Nullable Set<RelColumnOrigin> getColumnOrigins(Project rel, + final RelMetadataQuery mq, int iOutputColumn) { + final RelNode input = rel.getInput(); + RexNode rexNode = rel.getProjects().get(iOutputColumn); + + if (rexNode instanceof RexInputRef) { + // Direct reference: no derivation added. + RexInputRef inputRef = (RexInputRef) rexNode; + + return mq.getColumnOrigins(input, inputRef.getIndex()); + } + // Anything else is a derivation, possibly from multiple columns. + final Set<RelColumnOrigin> set = getMultipleColumns(rexNode, input, mq); + + return createDerivedColumnOrigins(set); + } + + /** */ + public @Nullable Set<RelColumnOrigin> getColumnOrigins(Calc rel, final RelMetadataQuery mq, int iOutputColumn) { + final RelNode input = rel.getInput(); + final RexShuttle rexShuttle = new RexShuttle() { + @Override public RexNode visitLocalRef(RexLocalRef localRef) { + return rel.getProgram().expandLocalRef(localRef); + } + }; + final List<RexNode> projects = new ArrayList<>(); + + for (RexNode rex: rexShuttle.apply(rel.getProgram().getProjectList())) + projects.add(rex); + + final RexNode rexNode = projects.get(iOutputColumn); + + if (rexNode instanceof RexInputRef) { + // Direct reference: no derivation added. + RexInputRef inputRef = (RexInputRef) rexNode; + return mq.getColumnOrigins(input, inputRef.getIndex()); + } + // Anything else is a derivation, possibly from multiple columns. + final Set<RelColumnOrigin> set = getMultipleColumns(rexNode, input, mq); + + return createDerivedColumnOrigins(set); + } + + /** */ + public @Nullable Set<RelColumnOrigin> getColumnOrigins(Filter rel, RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + /** */ + public @Nullable Set<RelColumnOrigin> getColumnOrigins(Sort rel, RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + /** */ + public @Nullable Set<RelColumnOrigin> getColumnOrigins(TableModify rel, RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + /** */ + public @Nullable Set<RelColumnOrigin> getColumnOrigins(Exchange rel, RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + /** */ + public @Nullable Set<RelColumnOrigin> getColumnOrigins( + TableFunctionScan rel, + RelMetadataQuery mq, + int iOutputColumn + ) { + Set<RelColumnMapping> mappings = rel.getColumnMappings(); + + if (mappings == null) { + if (!rel.getInputs().isEmpty()) { + // This is a non-leaf transformation: say we don't + // know about origins, because there are probably + // columns below. + return null; + } + else { + // This is a leaf transformation: say there are for sure no + // column origins. + return Collections.emptySet(); + } + } + + final Set<RelColumnOrigin> set = new HashSet<>(); + + for (RelColumnMapping mapping : mappings) { + if (mapping.iOutputColumn != iOutputColumn) + continue; + + final RelNode input = rel.getInputs().get(mapping.iInputRel); + final int column = mapping.iInputColumn; + Set<RelColumnOrigin> origins = mq.getColumnOrigins(input, column); + + if (origins == null) + return null; + + if (mapping.derived) + origins = createDerivedColumnOrigins(origins); + + set.addAll(origins); + } + + return set; + } + + /** + * Get column origins. + * + * @param rel Rel to get origins from. + * @param mq Rel metadata query. + * @param iOutputColumn Column idx. + * @return Set of column origins. + */ + public @Nullable Set<RelColumnOrigin> getColumnOrigins( + ProjectableFilterableTableScan rel, + RelMetadataQuery mq, + int iOutputColumn + ) { + if (rel.projects() != null) { + if (rel.getInputs().isEmpty() || rel.getInputs().size() > 1) Review comment: so let's just return null if there are projections. While we implement it - we can handle it here. ########## File path: modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/rules/OrToUnionRuleTest.java ########## @@ -203,7 +204,8 @@ public void testRangeOrToUnionAllRewrite() { checkQuery("SELECT * " + "FROM products " + "WHERE cat_id > 1 " + - "OR subcat_id < 10") + "OR subcat_id < 10 " + + "OR id in (5,6,7,9)") Review comment: removed ########## File path: modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/StatisticsPlannerTest.java ########## @@ -0,0 +1,237 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.ignite.internal.processors.query.calcite.planner; + +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; +import java.util.Arrays; +import java.util.HashMap; +import org.apache.calcite.rel.RelCollations; +import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.util.ImmutableIntList; +import org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup; +import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexScan; +import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel; +import org.apache.ignite.internal.processors.query.calcite.schema.IgniteIndex; +import org.apache.ignite.internal.processors.query.calcite.schema.IgniteSchema; +import org.apache.ignite.internal.processors.query.calcite.schema.IgniteStatisticsImpl; +import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions; +import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory; +import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeSystem; +import org.apache.ignite.internal.processors.query.stat.ColumnStatistics; +import org.apache.ignite.internal.processors.query.stat.ObjectStatisticsImpl; +import org.h2.value.ValueBoolean; +import org.h2.value.ValueByte; +import org.h2.value.ValueDate; +import org.h2.value.ValueDouble; +import org.h2.value.ValueFloat; +import org.h2.value.ValueInt; +import org.h2.value.ValueLong; +import org.h2.value.ValueShort; +import org.h2.value.ValueString; +import org.h2.value.ValueTime; +import org.h2.value.ValueTimestamp; +import org.junit.Before; +import org.junit.Test; + +/** + * Statistic related simple tests. + */ +public class StatisticsPlannerTest extends AbstractPlannerTest { Review comment: Fixed -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
