diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java index 7be8acb3ed5a7..3122d3c99d0fb 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java @@ -33,8 +33,6 @@ import org.apache.calcite.plan.ConventionTraitDef; import org.apache.calcite.plan.RelTraitDef; import org.apache.calcite.rel.RelCollationTraitDef; -import org.apache.calcite.rel.core.Aggregate; -import org.apache.calcite.rel.hint.HintStrategyTable; import org.apache.calcite.schema.SchemaPlus; import org.apache.calcite.sql.SqlCall; import org.apache.calcite.sql.SqlDynamicParam; @@ -77,6 +75,7 @@ import org.apache.ignite.internal.processors.query.calcite.exec.QueryTaskExecutor; import org.apache.ignite.internal.processors.query.calcite.exec.QueryTaskExecutorImpl; import org.apache.ignite.internal.processors.query.calcite.exec.exp.RexExecutorImpl; +import org.apache.ignite.internal.processors.query.calcite.hint.HintsConfig; import org.apache.ignite.internal.processors.query.calcite.message.MessageService; import org.apache.ignite.internal.processors.query.calcite.message.MessageServiceImpl; import org.apache.ignite.internal.processors.query.calcite.metadata.AffinityService; @@ -143,14 +142,7 @@ public class CalciteQueryProcessor extends GridProcessorAdapter implements Query .withInSubQueryThreshold(Integer.MAX_VALUE) .withDecorrelationEnabled(true) .withExpand(false) - .withHintStrategyTable( - HintStrategyTable.builder() - .hintStrategy("DISABLE_RULE", (hint, rel) -> true) - .hintStrategy("EXPAND_DISTINCT_AGG", (hint, rel) -> rel instanceof Aggregate) - // QUERY_ENGINE hint preprocessed by regexp, but to avoid warnings should be also in HintStrategyTable. - .hintStrategy("QUERY_ENGINE", (hint, rel) -> true) - .build() - ) + .withHintStrategyTable(HintsConfig.buildHintTable()) ) .convertletTable(IgniteConvertletTable.INSTANCE) .parserConfig( diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/hint/HintDefinition.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/hint/HintDefinition.java new file mode 100644 index 0000000000000..6ad756d8fb0b8 --- /dev/null +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/hint/HintDefinition.java @@ -0,0 +1,86 @@ +/* + * 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.hint; + +import org.apache.calcite.rel.hint.HintPredicate; +import org.apache.calcite.rel.hint.HintPredicates; +import org.apache.ignite.internal.processors.query.calcite.rel.logical.IgniteLogicalTableScan; + +/** + * Holds supported SQL hints and their settings. + */ +public enum HintDefinition { + /** Sets the query engine like H2 or Calcite. Is preprocessed by regexp. */ + QUERY_ENGINE, + + /** Disables planner rules. */ + DISABLE_RULE, + + /** Forces expanding of distinct aggregates to join. */ + EXPAND_DISTINCT_AGG { + /** {@inheritDoc} */ + @Override public HintPredicate predicate() { + return HintPredicates.AGGREGATE; + } + + /** {@inheritDoc} */ + @Override public HintOptionsChecker optionsChecker() { + return HintsConfig.OPTS_CHECK_EMPTY; + } + }, + + /** Disables indexes. */ + NO_INDEX { + /** {@inheritDoc} */ + @Override public HintPredicate predicate() { + return (hint, rel) -> rel instanceof IgniteLogicalTableScan; + } + + /** {@inheritDoc} */ + @Override public HintOptionsChecker optionsChecker() { + return HintsConfig.OPTS_CHECK_NO_KV; + } + }, + + /** Forces index usage. */ + FORCE_INDEX { + /** {@inheritDoc} */ + @Override public HintPredicate predicate() { + return NO_INDEX.predicate(); + } + + /** {@inheritDoc} */ + @Override public HintOptionsChecker optionsChecker() { + return NO_INDEX.optionsChecker(); + } + }; + + /** + * @return Hint predicate which limits redundant hint copying and reduces mem/cpu consumption. + */ + HintPredicate predicate() { + return HintPredicates.SET_VAR; + } + + /** + * @return {@link HintOptionsChecker}. + */ + HintOptionsChecker optionsChecker() { + return HintsConfig.OPTS_CHECK_PLAIN; + } +} diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/hint/HintOptionsChecker.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/hint/HintOptionsChecker.java new file mode 100644 index 0000000000000..1e5cff47f8cf3 --- /dev/null +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/hint/HintOptionsChecker.java @@ -0,0 +1,30 @@ +/* + * 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.hint; + +import java.util.function.Function; +import org.apache.calcite.rel.hint.RelHint; +import org.jetbrains.annotations.Nullable; + +/** + * Hint options validator. Returns error text if the rel hint can't accept passed options. + * Otherwise, returns {@code null}. + */ +interface HintOptionsChecker extends Function { + /* No-op. */ +} diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/hint/HintUtils.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/hint/HintUtils.java new file mode 100644 index 0000000000000..dc3da9a72276d --- /dev/null +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/hint/HintUtils.java @@ -0,0 +1,175 @@ +/* + * 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.hint; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.calcite.plan.RelOptUtil; +import org.apache.calcite.rel.AbstractRelNode; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.RelWriter; +import org.apache.calcite.rel.core.AggregateCall; +import org.apache.calcite.rel.hint.HintStrategyTable; +import org.apache.calcite.rel.hint.Hintable; +import org.apache.calcite.rel.hint.RelHint; +import org.apache.calcite.rel.logical.LogicalAggregate; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.sql.SqlExplainLevel; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.processors.query.calcite.prepare.BaseQueryContext; +import org.apache.ignite.internal.processors.query.calcite.util.Commons; +import org.apache.ignite.internal.util.typedef.F; + +import static org.apache.ignite.internal.processors.query.calcite.hint.HintDefinition.EXPAND_DISTINCT_AGG; + +/** + * Base class for working with Calcite's SQL hints. + */ +public final class HintUtils { + /** */ + private HintUtils() { + // No-op. + } + + /** + * @return Combined list options of all {@code hints} filtered with {@code hintDef}. + * @see #filterHints(RelNode, Collection, List) + */ + public static Collection options(RelNode rel, Collection hints, HintDefinition hintDef) { + return F.flatCollections(filterHints(rel, hints, Collections.singletonList(hintDef)).stream() + .map(h -> h.listOptions).collect(Collectors.toList())); + } + + /** + * @return Hints filtered with {@code hintDefs} and suitable for {@code rel}. + * @see HintStrategyTable#apply(List, RelNode) + * @see #filterHints(RelNode, Collection, List) + */ + public static List hints(RelNode rel, HintDefinition... hintDefs) { + return rel.getCluster().getHintStrategies() + .apply(filterHints(rel, allRelHints(rel), Arrays.asList(hintDefs)), rel); + } + + /** + * @return Hints of {@code rel} if it is a {@code Hintable}. If is not or has no hints, empty collection. + * @see Hintable#getHints() + */ + public static List allRelHints(RelNode rel) { + return rel instanceof Hintable ? ((Hintable)rel).getHints() : Collections.emptyList(); + } + + /** + * @return Distinct hints within {@code hints} filtered with {@code hintDefs}, {@link HintOptionsChecker} and + * removed inherit pathes. + * @see HintOptionsChecker + * @see RelHint#inheritPath + */ + private static List filterHints(RelNode rel, Collection hints, List hintDefs) { + Set requiredHintDefs = hintDefs.stream().map(Enum::name).collect(Collectors.toSet()); + + List res = hints.stream().filter(h -> requiredHintDefs.contains(h.hintName)) + .map(h -> { + RelHint.Builder rb = RelHint.builder(h.hintName); + + if (!h.listOptions.isEmpty()) + rb.hintOptions(h.listOptions); + else if (!h.kvOptions.isEmpty()) + rb.hintOptions(h.kvOptions); + + return rb.build(); + }).distinct().collect(Collectors.toList()); + + // Validate hint options. + Iterator it = res.iterator(); + + while (it.hasNext()) { + RelHint hint = it.next(); + + String optsErr = HintDefinition.valueOf(hint.hintName).optionsChecker().apply(hint); + + if (!F.isEmpty(optsErr)) { + skippedHint(rel, hint, optsErr); + + it.remove(); + } + } + + return res; + } + + /** + * @return {@code True} if {@code rel} is hinted with {@link HintDefinition#EXPAND_DISTINCT_AGG}. + * {@code False} otherwise. + */ + public static boolean isExpandDistinctAggregate(LogicalAggregate rel) { + return !hints(rel, EXPAND_DISTINCT_AGG).isEmpty() + && rel.getAggCallList().stream().anyMatch(AggregateCall::isDistinct); + } + + /** + * Logs skipped hint. + */ + public static void skippedHint(RelNode relNode, RelHint hint, String reason) { + IgniteLogger log = Commons.context(relNode).unwrap(BaseQueryContext.class).logger(); + + if (log.isDebugEnabled()) { + String hintOptions = hint.listOptions.isEmpty() ? "" : "with options " + + hint.listOptions.stream().map(o -> '\'' + o + '\'').collect(Collectors.joining(",")) + + ' '; + + if (!relNode.getInputs().isEmpty()) + relNode = new NoInputsRelNodeWrap(relNode); + + log.debug(String.format("Skipped hint '%s' %sfor relation operator '%s'. %s", hint.hintName, + hintOptions, RelOptUtil.toString(relNode, SqlExplainLevel.EXPPLAN_ATTRIBUTES).trim(), reason)); + } + } + + /** */ + private static final class NoInputsRelNodeWrap extends AbstractRelNode { + /** Original rel. */ + private final RelNode rel; + + /** Ctor. */ + private NoInputsRelNodeWrap(RelNode relNode) { + super(relNode.getCluster(), relNode.getTraitSet()); + + this.rel = relNode; + } + + /** {@inheritDoc} */ + @Override public List getInputs() { + return Collections.emptyList(); + } + + /** {@inheritDoc} */ + @Override protected RelDataType deriveRowType() { + return rel.getRowType(); + } + + /** {@inheritDoc} */ + @Override public void explain(RelWriter pw) { + rel.explain(pw); + } + } +} diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/hint/HintsConfig.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/hint/HintsConfig.java new file mode 100644 index 0000000000000..2f4e902ef9d77 --- /dev/null +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/hint/HintsConfig.java @@ -0,0 +1,85 @@ +/* + * 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.hint; + +import java.util.Arrays; +import org.apache.calcite.rel.hint.HintStrategy; +import org.apache.calcite.rel.hint.HintStrategyTable; +import org.apache.calcite.rel.hint.RelHint; +import org.apache.calcite.util.Litmus; +import org.jetbrains.annotations.Nullable; + +/** + * Provides configuration of the supported SQL hints. + */ +public final class HintsConfig { + /** */ + private HintsConfig() { + // No-op. + } + + /** Allows no key-value option. */ + static final HintOptionsChecker OPTS_CHECK_NO_KV = new HintOptionsChecker() { + /** {@inheritDoc} */ + @Override public @Nullable String apply(RelHint hint) { + return hint.kvOptions.isEmpty() + ? null + : String.format("Hint '%s' can't have any key-value option (not supported).", hint.hintName); + } + }; + + /** Allows no option. */ + static final HintOptionsChecker OPTS_CHECK_EMPTY = new HintOptionsChecker() { + @Override public @Nullable String apply(RelHint hint) { + String noKv = OPTS_CHECK_NO_KV.apply(hint); + + if (noKv != null) + return noKv; + + return hint.kvOptions.isEmpty() && hint.listOptions.isEmpty() + ? null + : String.format("Hint '%s' can't have any option.", hint.hintName); + } + }; + + /** Allows only plain options. */ + static final HintOptionsChecker OPTS_CHECK_PLAIN = new HintOptionsChecker() { + @Override public @Nullable String apply(RelHint hint) { + String noKv = OPTS_CHECK_NO_KV.apply(hint); + + if (noKv != null) + return noKv; + + return !hint.listOptions.isEmpty() + ? null + : String.format("Hint '%s' must have at least one option.", hint.hintName); + } + }; + + /** + * @return Configuration of all the supported hints. + */ + public static HintStrategyTable buildHintTable() { + HintStrategyTable.Builder b = HintStrategyTable.builder().errorHandler(Litmus.IGNORE); + + Arrays.stream(HintDefinition.values()).forEach(hintDef -> + b.hintStrategy(hintDef.name(), HintStrategy.builder(hintDef.predicate()).build())); + + return b.build(); + } +} diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgnitePlanner.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgnitePlanner.java index 399d248749e60..1af54fa999822 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgnitePlanner.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgnitePlanner.java @@ -21,6 +21,7 @@ import java.io.Reader; import java.io.StringWriter; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -596,7 +597,10 @@ private SqlToRelConverter sqlToRelConverter(SqlValidator validator, CalciteCatal } /** */ - public void setDisabledRules(Set disabledRuleNames) { + public void setDisabledRules(Collection disabledRuleNames) { + if (F.isEmpty(disabledRuleNames)) + return; + ctx.rulesFilter(rulesSet -> { List newSet = new ArrayList<>(); diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerHelper.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerHelper.java index f5344631dfc8d..b979c7997c687 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerHelper.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerHelper.java @@ -18,21 +18,28 @@ package org.apache.ignite.internal.processors.query.calcite.prepare; import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; import java.util.List; - +import java.util.stream.Collectors; import com.google.common.collect.ImmutableSet; +import org.apache.calcite.plan.RelOptUtil; import org.apache.calcite.plan.RelTraitSet; import org.apache.calcite.rel.RelCollations; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.RelRoot; +import org.apache.calcite.rel.core.SetOp; import org.apache.calcite.rel.core.Spool; import org.apache.calcite.rel.core.TableScan; +import org.apache.calcite.rel.hint.RelHint; import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.rex.RexNode; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; import org.apache.calcite.util.Pair; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.processors.query.calcite.hint.HintDefinition; +import org.apache.ignite.internal.processors.query.calcite.hint.HintUtils; import org.apache.ignite.internal.processors.query.calcite.rel.AbstractIndexScan; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexScan; @@ -45,7 +52,7 @@ import org.apache.ignite.internal.processors.query.calcite.schema.IgniteTable; import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions; import org.apache.ignite.internal.processors.query.calcite.util.Commons; -import org.apache.ignite.internal.processors.query.calcite.util.HintUtils; +import org.apache.ignite.internal.util.typedef.F; /** */ public class PlannerHelper { @@ -63,17 +70,23 @@ private PlannerHelper() { */ public static IgniteRel optimize(SqlNode sqlNode, IgnitePlanner planner, IgniteLogger log) { try { - // Convert to Relational operators graph + // Convert to Relational operators graph. RelRoot root = planner.rel(sqlNode); - RelNode rel = root.rel; + planner.setDisabledRules(HintUtils.options(root.rel, extractRootHints(root.rel), + HintDefinition.DISABLE_RULE)); - if (HintUtils.containsDisabledRules(root.hints)) - planner.setDisabledRules(HintUtils.disabledRules(root.hints)); + RelNode rel = root.rel; // Transformation chain rel = planner.transform(PlannerPhase.HEP_DECORRELATE, rel.getTraitSet(), rel); + // RelOptUtil#propagateRelHints(RelNode, equiv) may skip hints because current RelNode has no hints. + // Or if hints reside in a child nodes which are not inputs of the current node. Like LogicalFlter#condition. + // Such hints may appear or be required below in the tree, after rules applying. + // In Calcite, RelDecorrelator#decorrelateQuery(...) can re-propagate hints. + rel = RelOptUtil.propagateRelHints(rel, false); + rel = planner.replaceCorrelatesCollisions(rel); rel = planner.trimUnusedFields(root.withRel(rel)).rel; @@ -113,6 +126,21 @@ public static IgniteRel optimize(SqlNode sqlNode, IgnitePlanner planner, IgniteL } } + /** + * Extracts planner-level hints like 'DISABLE_RULE' if the root node is a combining node like 'UNION'. + */ + private static Collection extractRootHints(RelNode rel) { + if (!HintUtils.allRelHints(rel).isEmpty()) + return HintUtils.allRelHints(rel); + + if (rel instanceof SetOp) { + return F.flatCollections(rel.getInputs().stream() + .map(PlannerHelper::extractRootHints).collect(Collectors.toList())); + } + + return Collections.emptyList(); + } + /** * This shuttle analyzes a relational tree and inserts an eager spool node * just under the TableModify node in case latter depends upon a table used diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/AbstractIndexScan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/AbstractIndexScan.java index b5a5a7e695d25..40efc54172ef1 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/AbstractIndexScan.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/AbstractIndexScan.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.query.calcite.rel; +import java.util.Collections; import java.util.List; import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.plan.RelOptCost; @@ -25,7 +26,6 @@ import org.apache.calcite.plan.RelTraitSet; import org.apache.calcite.rel.RelInput; import org.apache.calcite.rel.RelWriter; -import org.apache.calcite.rel.hint.RelHint; import org.apache.calcite.rel.metadata.RelMetadataQuery; import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.rex.RexNode; @@ -65,7 +65,6 @@ protected AbstractIndexScan(RelInput input) { protected AbstractIndexScan( RelOptCluster cluster, RelTraitSet traitSet, - List hints, RelOptTable table, String idxName, @Nullable List proj, @@ -73,7 +72,7 @@ protected AbstractIndexScan( @Nullable List searchBounds, @Nullable ImmutableBitSet reqColumns ) { - super(cluster, traitSet, hints, table, proj, cond, reqColumns); + super(cluster, traitSet, Collections.emptyList(), table, proj, cond, reqColumns); this.idxName = idxName; this.searchBounds = searchBounds; diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteIndexScan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteIndexScan.java index a5a3c82c9b325..066e1b9d37c41 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteIndexScan.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteIndexScan.java @@ -18,7 +18,6 @@ package org.apache.ignite.internal.processors.query.calcite.rel; import java.util.List; -import com.google.common.collect.ImmutableList; import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.plan.RelOptTable; import org.apache.calcite.plan.RelTraitSet; @@ -107,7 +106,7 @@ private IgniteIndexScan( @Nullable ImmutableBitSet requiredCols, RelCollation collation ) { - super(cluster, traits, ImmutableList.of(), tbl, idxName, proj, cond, searchBounds, requiredCols); + super(cluster, traits, tbl, idxName, proj, cond, searchBounds, requiredCols); this.sourceId = sourceId; this.collation = collation; diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/logical/IgniteLogicalIndexScan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/logical/IgniteLogicalIndexScan.java index 1cb7dd0cd8059..2602c21ded6d3 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/logical/IgniteLogicalIndexScan.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/logical/IgniteLogicalIndexScan.java @@ -18,7 +18,6 @@ package org.apache.ignite.internal.processors.query.calcite.rel.logical; import java.util.List; -import com.google.common.collect.ImmutableList; import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.plan.RelOptTable; import org.apache.calcite.plan.RelTraitSet; @@ -79,6 +78,6 @@ private IgniteLogicalIndexScan( @Nullable List searchBounds, @Nullable ImmutableBitSet requiredCols ) { - super(cluster, traits, ImmutableList.of(), tbl, idxName, proj, cond, searchBounds, requiredCols); + super(cluster, traits, tbl, idxName, proj, cond, searchBounds, requiredCols); } } diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/logical/IgniteLogicalTableScan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/logical/IgniteLogicalTableScan.java index 4992795464684..0756b5342519f 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/logical/IgniteLogicalTableScan.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/logical/IgniteLogicalTableScan.java @@ -19,10 +19,10 @@ import java.util.List; -import com.google.common.collect.ImmutableList; import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.plan.RelOptTable; import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.hint.RelHint; import org.apache.calcite.rex.RexNode; import org.apache.calcite.util.ImmutableBitSet; import org.apache.ignite.internal.processors.query.calcite.rel.ProjectableFilterableTableScan; @@ -30,23 +30,25 @@ /** */ public class IgniteLogicalTableScan extends ProjectableFilterableTableScan { - /** Creates a IgniteTableScan. */ + /** Creates a IgniteLogicalTableScan. */ public static IgniteLogicalTableScan create( RelOptCluster cluster, RelTraitSet traits, RelOptTable tbl, + List hints, @Nullable List proj, @Nullable RexNode cond, @Nullable ImmutableBitSet requiredColumns ) { - return new IgniteLogicalTableScan(cluster, traits, tbl, proj, cond, requiredColumns); + return new IgniteLogicalTableScan(cluster, traits, tbl, hints, proj, cond, requiredColumns); } /** - * Creates a TableScan. - * @param cluster Cluster that this relational expression belongs to - * @param traits Traits of this relational expression + * Creates a IgniteLogicalTableScan. + * @param cluster Cluster that this relational expression belongs to. + * @param traits Traits of this relational expression. * @param tbl Table definition. + * @param hints Hints. * @param proj Projects. * @param cond Filters. * @param requiredColunms Participating colunms. @@ -55,10 +57,17 @@ private IgniteLogicalTableScan( RelOptCluster cluster, RelTraitSet traits, RelOptTable tbl, + List hints, @Nullable List proj, @Nullable RexNode cond, @Nullable ImmutableBitSet requiredColunms ) { - super(cluster, traits, ImmutableList.of(), tbl, proj, cond, requiredColunms); + super(cluster, traits, hints, tbl, proj, cond, requiredColunms); + } + + /** {@inheritDoc} */ + @Override public IgniteLogicalTableScan withHints(List hints) { + return new IgniteLogicalTableScan(getCluster(), getTraitSet(), getTable(), hints, projects(), condition(), + requiredColumns()); } } diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/HashAggregateConverterRule.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/HashAggregateConverterRule.java index 374132b16e832..38f5b41fc1482 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/HashAggregateConverterRule.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/HashAggregateConverterRule.java @@ -25,12 +25,12 @@ import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.logical.LogicalAggregate; import org.apache.calcite.rel.metadata.RelMetadataQuery; +import org.apache.ignite.internal.processors.query.calcite.hint.HintUtils; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention; import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteColocatedHashAggregate; import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteMapHashAggregate; import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteReduceHashAggregate; import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions; -import org.apache.ignite.internal.processors.query.calcite.util.HintUtils; /** * diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/SortAggregateConverterRule.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/SortAggregateConverterRule.java index 13c427876ef23..8b4f29580c911 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/SortAggregateConverterRule.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/SortAggregateConverterRule.java @@ -26,13 +26,13 @@ import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.logical.LogicalAggregate; import org.apache.calcite.rel.metadata.RelMetadataQuery; +import org.apache.ignite.internal.processors.query.calcite.hint.HintUtils; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention; import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteColocatedSortAggregate; import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteMapSortAggregate; import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteReduceSortAggregate; import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions; import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils; -import org.apache.ignite.internal.processors.query.calcite.util.HintUtils; import org.apache.ignite.internal.util.typedef.F; /** diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/logical/ExposeIndexRule.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/logical/ExposeIndexRule.java index 9fbed19ed3235..8c3f48bbe5e1d 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/logical/ExposeIndexRule.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/logical/ExposeIndexRule.java @@ -17,9 +17,12 @@ package org.apache.ignite.internal.processors.query.calcite.rule.logical; +import java.util.Collection; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.plan.RelOptRule; @@ -27,14 +30,22 @@ import org.apache.calcite.plan.RelOptTable; import org.apache.calcite.plan.RelRule; import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.TableScan; +import org.apache.calcite.rel.hint.RelHint; import org.apache.calcite.rex.RexNode; import org.apache.calcite.util.ImmutableBitSet; +import org.apache.ignite.internal.processors.query.calcite.hint.HintDefinition; +import org.apache.ignite.internal.processors.query.calcite.hint.HintUtils; +import org.apache.ignite.internal.processors.query.calcite.rel.AbstractIndexScan; import org.apache.ignite.internal.processors.query.calcite.rel.logical.IgniteLogicalIndexScan; import org.apache.ignite.internal.processors.query.calcite.rel.logical.IgniteLogicalTableScan; import org.apache.ignite.internal.processors.query.calcite.schema.IgniteTable; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.lang.IgniteBiTuple; import org.immutables.value.Value; +import static org.apache.calcite.util.Util.last; + /** * */ @@ -77,6 +88,16 @@ private static boolean preMatch(IgniteLogicalTableScan scan) { assert !indexes.isEmpty(); + IgniteBiTuple, Boolean> hintedIndexes = processHints(scan, indexes); + + indexes = hintedIndexes.get1(); + + if (indexes.isEmpty()) + return; + + if (hintedIndexes.get2()) + cluster.getPlanner().prune(scan); + Map equivMap = new HashMap<>(indexes.size()); for (int i = 1; i < indexes.size(); i++) equivMap.put(indexes.get(i), scan); @@ -84,6 +105,48 @@ private static boolean preMatch(IgniteLogicalTableScan scan) { call.transformTo(F.first(indexes), equivMap); } + /** + * @return Actual indixes list and prune-table-scan flag if any index is forced to use. + */ + private IgniteBiTuple, Boolean> processHints( + TableScan scan, + List indexes + ) { + assert !F.isEmpty(indexes); + + Set tblIdxNames = indexes.stream().map(AbstractIndexScan::indexName).collect(Collectors.toSet()); + Set idxToSkip = new HashSet<>(); + Set idxToUse = new HashSet<>(); + + for (RelHint hint : HintUtils.hints(scan, HintDefinition.NO_INDEX, HintDefinition.FORCE_INDEX)) { + boolean skip = !hint.hintName.equals(HintDefinition.FORCE_INDEX.name()); + + Collection hintIdxNames = hint.listOptions.isEmpty() ? tblIdxNames : hint.listOptions; + + for (String hintIdxName : hintIdxNames) { + if (!tblIdxNames.contains(hintIdxName)) + continue; + + if (idxToSkip.contains(hintIdxName) || idxToUse.contains(hintIdxName)) { + HintUtils.skippedHint(scan, hint, "Index '" + hintIdxName + + "' of table '" + last(scan.getTable().getQualifiedName()) + + "' has already been excluded or forced to use by other options or other hints before."); + + continue; + } + + if (skip) + idxToSkip.add(hintIdxName); + else + idxToUse.add(hintIdxName); + } + } + + return new IgniteBiTuple<>(indexes.stream().filter(idx -> !idxToSkip.contains(idx.indexName()) + && (idxToUse.isEmpty() || idxToUse.contains(idx.indexName()))).collect(Collectors.toList()), + !idxToUse.isEmpty()); + } + /** */ @SuppressWarnings("ClassNameSameAsAncestorName") @Value.Immutable diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/logical/FilterScanMergeRule.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/logical/FilterScanMergeRule.java index 7254435b7ede0..f1c2ca78d3ef1 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/logical/FilterScanMergeRule.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/logical/FilterScanMergeRule.java @@ -177,7 +177,7 @@ private FilterTableScanMergeRule(FilterScanMergeRule.Config cfg) { RelTraitSet traits, RexNode cond ) { - return IgniteLogicalTableScan.create(cluster, traits, scan.getTable(), scan.projects(), + return IgniteLogicalTableScan.create(cluster, traits, scan.getTable(), scan.getHints(), scan.projects(), cond, scan.requiredColumns()); } } diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/logical/LogicalOrToUnionRule.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/logical/LogicalOrToUnionRule.java index f6a145e37c243..78315018970c3 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/logical/LogicalOrToUnionRule.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/logical/LogicalOrToUnionRule.java @@ -164,6 +164,7 @@ private void buildInput(RelBuilder relBldr, RelNode input, RexNode condition) { scan.getCluster(), trait, scan.getTable(), + scan.getHints(), scan.projects(), condition, scan.requiredColumns() diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/logical/ProjectScanMergeRule.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/logical/ProjectScanMergeRule.java index bfe20194d0138..739e17f868ac3 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/logical/ProjectScanMergeRule.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/logical/ProjectScanMergeRule.java @@ -175,6 +175,7 @@ private ProjectTableScanMergeRule(ProjectScanMergeRule.Config config) { cluster, traits, scan.getTable(), + scan.getHints(), projections, cond, requiredColumns diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/CacheTableImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/CacheTableImpl.java index 784201d0099c5..1fc923767fd78 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/CacheTableImpl.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/CacheTableImpl.java @@ -101,7 +101,8 @@ public CacheTableImpl(GridKernalContext ctx, CacheTableDescriptor desc) { @Nullable RexNode cond, @Nullable ImmutableBitSet requiredColumns ) { - return IgniteLogicalTableScan.create(cluster, cluster.traitSet(), relOptTbl, proj, cond, requiredColumns); + return IgniteLogicalTableScan.create(cluster, cluster.traitSet(), relOptTbl, Collections.emptyList(), proj, + cond, requiredColumns); } /** {@inheritDoc} */ diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SystemViewIndexImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SystemViewIndexImpl.java index d5eca929902dc..38657fec1e691 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SystemViewIndexImpl.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SystemViewIndexImpl.java @@ -73,7 +73,8 @@ public SystemViewIndexImpl(SystemViewTableImpl tbl) { @Nullable RexNode cond, @Nullable ImmutableBitSet requiredColumns ) { - return IgniteLogicalIndexScan.create(cluster, cluster.traitSet(), relOptTbl, idxName, proj, cond, requiredColumns); + return IgniteLogicalIndexScan.create(cluster, cluster.traitSet(), relOptTbl, idxName, proj, cond, + requiredColumns); } /** */ diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SystemViewTableImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SystemViewTableImpl.java index f54d56b50053c..132f663b6476a 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SystemViewTableImpl.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SystemViewTableImpl.java @@ -94,7 +94,8 @@ public SystemViewTableImpl(SystemViewTableDescriptorImpl desc) { @Nullable RexNode cond, @Nullable ImmutableBitSet requiredColumns ) { - return IgniteLogicalTableScan.create(cluster, cluster.traitSet(), relOptTbl, proj, cond, requiredColumns); + return IgniteLogicalTableScan.create(cluster, cluster.traitSet(), relOptTbl, Collections.emptyList(), proj, + cond, requiredColumns); } /** {@inheritDoc} */ diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/HintUtils.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/HintUtils.java deleted file mode 100644 index eaf33931c3014..0000000000000 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/HintUtils.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * 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.util; - -import java.util.Collections; -import java.util.Set; -import java.util.stream.Collectors; - -import com.google.common.collect.ImmutableList; -import org.apache.calcite.rel.core.AggregateCall; -import org.apache.calcite.rel.hint.RelHint; -import org.apache.calcite.rel.logical.LogicalAggregate; -import org.apache.ignite.internal.util.typedef.F; - -/** */ -public class HintUtils { - /** */ - private HintUtils() { - // No-op. - } - - /** */ - public static boolean containsDisabledRules(ImmutableList hints) { - return hints.stream() - .anyMatch(h -> "DISABLE_RULE".equals(h.hintName) && !h.listOptions.isEmpty()); - } - - /** */ - public static Set disabledRules(ImmutableList hints) { - if (F.isEmpty(hints)) - return Collections.emptySet(); - - return hints.stream() - .filter(h -> "DISABLE_RULE".equals(h.hintName)) - .flatMap(h -> h.listOptions.stream()) - .collect(Collectors.toSet()); - } - - /** */ - public static boolean isExpandDistinctAggregate(LogicalAggregate rel) { - return rel.getHints().stream() - .anyMatch(h -> "EXPAND_DISTINCT_AGG".equals(h.hintName)) - && rel.getAggCallList().stream().anyMatch(AggregateCall::isDistinct); - } -} diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/IndexScanlIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/IndexScanlIntegrationTest.java index 875e5e760a37f..f66c495149f7e 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/IndexScanlIntegrationTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/IndexScanlIntegrationTest.java @@ -43,6 +43,7 @@ import org.apache.ignite.internal.processors.query.schema.management.SchemaManager; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.G; +import org.hamcrest.CoreMatchers; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import org.junit.Test; @@ -336,6 +337,83 @@ public void checkMultiColumnsInlineScan( } } + /** */ + @Test + public void testNoIndexHint() { + executeSql("CREATE TABLE t1(i1 INTEGER) WITH TEMPLATE=PARTITIONED"); + executeSql("INSERT INTO t1 VALUES (1), (2), (30), (40)"); + executeSql("CREATE INDEX t1_idx ON t1(i1)"); + + executeSql("CREATE TABLE t2(i2 INTEGER, i3 INTEGER) WITH TEMPLATE=PARTITIONED"); + + for (int i = 0; i < 100; ++i) + executeSql("INSERT INTO t2 VALUES (?, ?)", i, i); + + executeSql("CREATE INDEX t2_idx ON t2(i2)"); + + assertQuery("SELECT /*+ NO_INDEX(T2_IDX) */ i3 FROM t2 where i2=2") + .matches(CoreMatchers.not(QueryChecker.containsIndexScan("PUBLIC", "T2", "T2_IDX"))) + .returns(2) + .check(); + + assertQuery("SELECT /*+ NO_INDEX(T1_IDX,T2_IDX) */ i1, i3 FROM t1, t2 where i2=i1") + .matches(CoreMatchers.not(QueryChecker.containsIndexScan("PUBLIC", "T1", "T1_IDX"))) + .matches(CoreMatchers.not(QueryChecker.containsIndexScan("PUBLIC", "T2", "T2_IDX"))) + .returns(1, 1) + .returns(2, 2) + .returns(30, 30) + .returns(40, 40) + .check(); + + assertQuery("SELECT * FROM t1 WHERE i1 = (SELECT /*+ NO_INDEX(T2_IDX) */ i3 from t2 where i2=40)") + .matches(CoreMatchers.not(QueryChecker.containsIndexScan("PUBLIC", "T2", "T2_IDX"))) + .returns(40) + .check(); + } + + /** */ + @Test + public void testForcedIndexHint() { + executeSql("CREATE TABLE t1(i1 INTEGER, i2 INTEGER, i3 INTEGER) WITH TEMPLATE=PARTITIONED"); + + executeSql("CREATE INDEX t1_idx1 ON t1(i1)"); + executeSql("CREATE INDEX t1_idx2 ON t1(i2)"); + executeSql("CREATE INDEX t1_idx3 ON t1(i3)"); + + executeSql("INSERT INTO t1 VALUES (1, 2, 3)"); + + assertQuery("SELECT /*+ FORCE_INDEX(T1_IDX1) */ i1 FROM t1 where i1=1 and i2=2 and i3=3") + .matches(QueryChecker.containsIndexScan("PUBLIC", "T1", "T1_IDX1")) + .returns(1) + .check(); + + assertQuery("SELECT /*+ FORCE_INDEX(T1_IDX2) */ i1 FROM t1 where i1=1 and i2=2 and i3=3") + .matches(QueryChecker.containsIndexScan("PUBLIC", "T1", "T1_IDX2")) + .returns(1) + .check(); + + assertQuery("SELECT /*+ FORCE_INDEX(T1_IDX3) */ i1 FROM t1 where i1=1 and i2=2 and i3=3") + .matches(QueryChecker.containsIndexScan("PUBLIC", "T1", "T1_IDX3")) + .returns(1) + .check(); + + executeSql("CREATE TABLE t2(i21 INTEGER, i22 INTEGER, i23 INTEGER) WITH TEMPLATE=PARTITIONED"); + + for (int i = 99; i < 300; ++i) + executeSql("INSERT INTO t2 VALUES (?, ?, ?)", i + 1, i + 1, i + 1); + + executeSql("CREATE INDEX t2_idx1 ON t2(i21)"); + executeSql("CREATE INDEX t2_idx2 ON t2(i22)"); + executeSql("CREATE INDEX t2_idx3 ON t2(i23)"); + + assertQuery("SELECT /*+ FORCE_INDEX(T1_IDX2), FORCE_INDEX(T2_IDX2) */ i1, i22 FROM t1, t2 where i2=i22 " + + "and i3=i23 + 1") + .matches(QueryChecker.containsIndexScan("PUBLIC", "T1", "T1_IDX2")) + .matches(QueryChecker.containsIndexScan("PUBLIC", "T2", "T2_IDX2")) + .resultSize(0) + .check(); + } + /** */ private RowCountingIndex injectRowCountingIndex(IgniteEx node, String tableName, String idxName) { RowCountingIndex idx = null; diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/AbstractPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/AbstractPlannerTest.java index 328345e652343..e87ac9520ab79 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/AbstractPlannerTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/AbstractPlannerTest.java @@ -89,6 +89,7 @@ import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.ListeningTestLogger; import org.apache.ignite.testframework.junits.GridTestKernalContext; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.jetbrains.annotations.NotNull; @@ -128,6 +129,9 @@ public abstract class AbstractPlannerTest extends GridCommonAbstractTest { /** Last error message. */ protected String lastErrorMsg; + /** */ + protected ListeningTestLogger lsnrLog = new ListeningTestLogger(log); + /** */ @Before public void setup() { @@ -681,7 +685,7 @@ protected BaseQueryContext baseQueryContext(Collection schemas) { .defaultSchema(dfltSchema) .build() ) - .logger(log) + .logger(lsnrLog) .build(); } diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/TestTable.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/TestTable.java index 530d9fa1ffc2c..d34841c94539f 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/TestTable.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/TestTable.java @@ -142,7 +142,8 @@ public TestTable setStatistics(IgniteStatisticsImpl statistics) { @Nullable RexNode cond, @Nullable ImmutableBitSet requiredColumns ) { - return IgniteLogicalTableScan.create(cluster, cluster.traitSet(), relOptTbl, proj, cond, requiredColumns); + return IgniteLogicalTableScan.create(cluster, cluster.traitSet(), relOptTbl, Collections.emptyList(), proj, + cond, requiredColumns); } /** {@inheritDoc} */ diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/hints/CommonHintsPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/hints/CommonHintsPlannerTest.java new file mode 100644 index 0000000000000..e268f61fb948d --- /dev/null +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/hints/CommonHintsPlannerTest.java @@ -0,0 +1,118 @@ +/* + * 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.hints; + +import org.apache.ignite.internal.processors.query.QueryUtils; +import org.apache.ignite.internal.processors.query.calcite.planner.AbstractPlannerTest; +import org.apache.ignite.internal.processors.query.calcite.planner.TestTable; +import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexScan; +import org.apache.ignite.internal.processors.query.calcite.schema.IgniteSchema; +import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions; +import org.apache.ignite.testframework.LogListener; +import org.apache.ignite.testframework.junits.logger.GridTestLog4jLogger; +import org.apache.logging.log4j.Level; +import org.junit.Test; + +/** + * Common test for SQL hints. + */ +public class CommonHintsPlannerTest extends AbstractPlannerTest { + /** */ + private IgniteSchema schema; + + /** */ + private TestTable tbl; + + /** {@inheritDoc} */ + @Override public void setup() { + super.setup(); + + tbl = createTable("TBL", 100, IgniteDistributions.random(), "ID", Integer.class, "VAL", + Integer.class).addIndex(QueryUtils.PRIMARY_KEY_INDEX, 0).addIndex("IDX", 1); + + schema = createSchema(tbl); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + ((GridTestLog4jLogger)log).setLevel(Level.INFO); + } + + /** + * Tests hint 'DISABLE_RULE' works for whole query despite it is not set for the root node. + */ + @Test + public void testDisableRuleInHeader() throws Exception { + assertPlan("SELECT /*+ DISABLE_RULE('ExposeIndexRule') */ VAL FROM TBL UNION ALL " + + "SELECT VAL FROM TBL", schema, nodeOrAnyChild(isInstanceOf(IgniteIndexScan.class)).negate()); + + assertPlan("SELECT VAL FROM TBL where val=1 UNION ALL " + + "SELECT /*+ DISABLE_RULE('ExposeIndexRule') */ VAL FROM TBL", schema, + nodeOrAnyChild(isInstanceOf(IgniteIndexScan.class)).negate()); + } + + /** */ + @Test + public void testWrongParamsDisableRule() throws Exception { + LogListener lsnr = LogListener.matches("Hint 'DISABLE_RULE' must have at least one option").build(); + + lsnrLog.registerListener(lsnr); + + ((GridTestLog4jLogger)log).setLevel(Level.DEBUG); + + physicalPlan("SELECT /*+ DISABLE_RULE */ VAL FROM TBL", schema); + + assertTrue(lsnr.check()); + + lsnrLog.clearListeners(); + + lsnr = LogListener.matches("Hint 'DISABLE_RULE' can't have any key-value option").build(); + + lsnrLog.registerListener(lsnr); + + physicalPlan("SELECT /*+ DISABLE_RULE(a='b') */ VAL FROM TBL", schema); + + assertTrue(lsnr.check()); + } + + /** */ + @Test + public void testWrongParamsExpandDistinct() throws Exception { + LogListener lsnr = LogListener.matches("Hint 'EXPAND_DISTINCT_AGG' can't have any option").build(); + + lsnrLog.registerListener(lsnr); + + ((GridTestLog4jLogger)log).setLevel(Level.DEBUG); + + physicalPlan("SELECT /*+ EXPAND_DISTINCT_AGG(OPTION) */ MAX(VAL) FROM TBL", schema); + + assertTrue(lsnr.check()); + + lsnrLog.clearListeners(); + + lsnr = LogListener.matches("Hint 'EXPAND_DISTINCT_AGG' can't have any key-value option").build(); + + lsnrLog.registerListener(lsnr); + + physicalPlan("SELECT /*+ EXPAND_DISTINCT_AGG(a='b') */ MAX(VAL) FROM TBL", schema); + + assertTrue(lsnr.check()); + } +} diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/hints/ForceIndexHintPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/hints/ForceIndexHintPlannerTest.java new file mode 100644 index 0000000000000..3283436df5b6e --- /dev/null +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/hints/ForceIndexHintPlannerTest.java @@ -0,0 +1,208 @@ +/* + * 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.hints; + +import org.apache.ignite.internal.processors.query.QueryUtils; +import org.apache.ignite.internal.processors.query.calcite.planner.AbstractPlannerTest; +import org.apache.ignite.internal.processors.query.calcite.planner.TestTable; +import org.apache.ignite.internal.processors.query.calcite.schema.IgniteSchema; +import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions; +import org.apache.ignite.testframework.LogListener; +import org.apache.ignite.testframework.junits.logger.GridTestLog4jLogger; +import org.apache.logging.log4j.Level; +import org.junit.Test; + +/** + * Planner test for force index hint. + */ +public class ForceIndexHintPlannerTest extends AbstractPlannerTest { + /** */ + private IgniteSchema schema; + + /** */ + private TestTable tbl1; + + /** */ + private TestTable tbl2; + + /** {@inheritDoc} */ + @Override public void setup() { + super.setup(); + + // A tiny table. + tbl1 = createTable("TBL1", 1, IgniteDistributions.single(), "ID", Integer.class, + "VAL1", Integer.class, "VAL2", Integer.class, "VAL3", Integer.class) + .addIndex(QueryUtils.PRIMARY_KEY_INDEX, 0) + .addIndex("IDX1_1", 1) + .addIndex("IDX1_2", 2) + .addIndex("IDX1_3", 3); + + // A large table. Has the same first inndex name 'IDX1' as of TBL1. + tbl2 = createTable("TBL2", 10_000, IgniteDistributions.single(), "ID", Integer.class, + "VAL21", Integer.class, "VAL22", Integer.class, "VAL23", Integer.class) + .addIndex(QueryUtils.PRIMARY_KEY_INDEX, 0) + .addIndex("IDX2_1", 1) + .addIndex("IDX2_2", 2) + .addIndex("IDX2_3", 3); + + schema = createSchema(tbl1, tbl2); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + ((GridTestLog4jLogger)log).setLevel(Level.INFO); + } + + /** */ + @Test + public void testBasicIndexSelection() throws Exception { + assertPlan("SELECT /*+ FORCE_INDEX(IDX2_3) */ * FROM TBL2 WHERE val23=1 and val21=2 and val22=3", + schema, nodeOrAnyChild(isIndexScan("TBL2", "IDX2_3"))); + + assertPlan("SELECT /*+ FORCE_INDEX(UNEXISTING,IDX2_3,UNEXISTING) */ * FROM TBL2 WHERE val23=1 and val21=2 " + + "and val22=3", schema, nodeOrAnyChild(isIndexScan("TBL2", "IDX2_3"))); + + assertPlan("SELECT /*+ FORCE_INDEX(IDX2_3,IDX2_1) */ * FROM TBL2 WHERE val23=1 and val21=2 " + + "and val22=3", schema, nodeOrAnyChild(isIndexScan("TBL2", "IDX2_1") + .or(isIndexScan("TBL2", "IDX2_3")))); + + assertPlan("SELECT /*+ FORCE_INDEX(IDX2_2,IDX2_3) */ * FROM TBL2 WHERE val23=1 and val21=2 and val22=3", + schema, nodeOrAnyChild(isIndexScan("TBL2", "IDX2_2")) + .or(nodeOrAnyChild(isIndexScan("TBL2", "IDX2_3")))); + + assertPlan("SELECT /*+ FORCE_INDEX(IDX2_3), FORCE_INDEX(IDX2_3) */ * FROM TBL2 WHERE val23=1 and val21=2 " + + "and val22=3", schema, nodeOrAnyChild(isIndexScan("TBL2", "IDX2_3"))); + } + + /** */ + @Test + public void testJoins() throws Exception { + doTestJoins("LEFT"); + doTestJoins("RIGHT"); + doTestJoins("INNER"); + } + + /** */ + private void doTestJoins(String jt) throws Exception { + assertPlan("SELECT /*+ FORCE_INDEX(IDX2_2) */ t1.val1, t2.val22 FROM TBL1 t1 " + jt + " JOIN TBL2 " + + "t2 on t1.val3=t2.val23 and t1.val1=t2.val22", schema, + nodeOrAnyChild(isIndexScan("TBL2", "IDX2_2"))); + + assertPlan("SELECT /*+ FORCE_INDEX(IDX2_3) */ t1.val1, t2.val22 FROM TBL1 t1 " + jt + " JOIN TBL2 " + + "t2 on t1.val3=t2.val23 and t1.val1=t2.val22", schema, + nodeOrAnyChild(isIndexScan("TBL2", "IDX2_3"))); + + assertPlan("SELECT /*+ FORCE_INDEX(IDX2_2,IDX2_3) */ t1.val1, t2.val22 FROM TBL1 t1 " + jt + " JOIN TBL2 " + + "t2 on t1.val3=t2.val23 and t1.val1=t2.val22", schema, + nodeOrAnyChild(isIndexScan("TBL2", "IDX2_2")) + .or(nodeOrAnyChild(isIndexScan("TBL2", "IDX2_3")))); + + // With additional filter. + assertPlan("SELECT /*+ FORCE_INDEX(IDX2_2,IDX2_3) */ t1.val1, t2.val22 FROM TBL1 t1 " + jt + " JOIN TBL2 " + + "t2 on t1.val3=t2.val23 and t1.val1=t2.val22 where t2.val22=2 and t1.val3=3 and t2.val21=1", schema, + nodeOrAnyChild(isIndexScan("TBL2", "IDX2_2")) + .or(nodeOrAnyChild(isIndexScan("TBL2", "IDX2_3")))); + } + + /** */ + @Test + public void testOrderBy() throws Exception { + assertPlan("SELECT /*+ FORCE_INDEX(IDX1_3) */ val2, val3 FROM TBL1 ORDER by val2, val1, val3", schema, + nodeOrAnyChild(isIndexScan("TBL1", "IDX1_3"))); + + assertPlan("SELECT /*+ FORCE_INDEX(IDX1_2) */ val2, val3 FROM TBL1 ORDER by val2, val1, val3", schema, + nodeOrAnyChild(isIndexScan("TBL1", "IDX1_2"))); + + assertPlan("SELECT /*+ FORCE_INDEX(IDX1_1) */ val2, val3 FROM TBL1 ORDER by val2, val1, val3", schema, + nodeOrAnyChild(isIndexScan("TBL1", "IDX1_1"))); + } + + /** */ + @Test + public void testAggregates() throws Exception { + doTestAggregates("sum"); + doTestAggregates("avg"); + doTestAggregates("min"); + doTestAggregates("max"); + } + + /** */ + private void doTestAggregates(String op) throws Exception { + assertPlan("SELECT /*+ FORCE_INDEX(IDX1_2) */ " + op + "(val1) FROM TBL1 where val1=1 group by val2", schema, + nodeOrAnyChild(isIndexScan("TBL1", "IDX1_2"))); + + assertPlan("SELECT /*+ FORCE_INDEX(IDX1_2) */ " + op + "(val1) FROM TBL1 group by val2", schema, + nodeOrAnyChild(isIndexScan("TBL1", "IDX1_2"))); + + assertPlan("SELECT /*+ FORCE_INDEX(IDX1_1) */ " + op + "(val1) FROM TBL1 where val1=1 group by val2", schema, + nodeOrAnyChild(isIndexScan("TBL1", "IDX1_1"))); + + assertPlan("SELECT /*+ FORCE_INDEX(IDX1_2) */ " + op + "(val1) FROM TBL1 where val1=1 group by val2", schema, + nodeOrAnyChild(isIndexScan("TBL1", "IDX1_2"))); + + assertPlan("SELECT /*+ FORCE_INDEX(IDX1_1,IDX1_2) */ " + op + "(val1) FROM TBL1 where val1=1 group by val2", + schema, nodeOrAnyChild(isIndexScan("TBL1", "IDX1_2")) + .or(nodeOrAnyChild(isIndexScan("TBL1", "IDX1_1")))); + } + + /** */ + @Test + public void testWithNoIndexHint() throws Exception { + LogListener lsnr = LogListener.matches("Skipped hint 'NO_INDEX' with options 'IDX2_3'") + .andMatches("Index 'IDX2_3' of table 'TBL2' has already been excluded").times(1).build(); + + lsnrLog.registerListener(lsnr); + + ((GridTestLog4jLogger)log).setLevel(Level.DEBUG); + + assertPlan("SELECT /*+ NO_INDEX(IDX2_1), FORCE_INDEX(IDX2_3), NO_INDEX(IDX2_3) */ * FROM TBL2 where " + + "val21=1 and val22=2 and val23=3", schema, nodeOrAnyChild(isIndexScan("TBL2", "IDX2_3"))); + + assertTrue(lsnr.check()); + + assertPlan("SELECT /*+ NO_INDEX(IDX2_1), FORCE_INDEX(IDX2_1), FORCE_INDEX(IDX2_3) */ * FROM TBL2 where " + + "val21=1 and val22=2 and val23=3", schema, nodeOrAnyChild(isIndexScan("TBL2", "IDX2_3"))); + + assertPlan("SELECT /*+ FORCE_INDEX(IDX2_3), NO_INDEX */ * FROM TBL2 where " + + "val21=1 and val22=2 and val23=3", schema, nodeOrAnyChild(isIndexScan("TBL2", "IDX2_3"))); + } + + /** */ + @Test + public void testSubquery() throws Exception { + assertPlan("SELECT /*+ FORCE_INDEX(IDX1_2,IDX2_3) */ t1.val1 FROM TBL1 t1 where t1.val2 = " + + "(SELECT t2.val23 from TBL2 t2 where t2.val21=10 and t2.val23=10 and t2.val21=10)", schema, + nodeOrAnyChild(isIndexScan("TBL1", "IDX1_2")) + .and(nodeOrAnyChild(isIndexScan("TBL2", "IDX2_3")))); + + assertPlan("SELECT t1.val1 FROM TBL1 t1 where t1.val2 = (SELECT /*+ FORCE_INDEX(IDX1_2,IDX2_3) */ " + + "t2.val23 from TBL2 t2 where t2.val21=10 and t2.val23=10 and t2.val21=10)", schema, + nodeOrAnyChild(isIndexScan("TBL2", "IDX2_3"))); + } + + /** */ + @Test + public void testTwoTables() throws Exception { + assertPlan("SELECT /*+ FORCE_INDEX(IDX1_1,IDX2_1,IDX2_2) */ val1 FROM TBL1, TBL2 WHERE val1=val21 and " + + "val2=val22 and val3=val23", schema, nodeOrAnyChild(isIndexScan("TBL1", "IDX1_1")) + .and(nodeOrAnyChild(isIndexScan("TBL2", "IDX2_1") + .or(nodeOrAnyChild(isIndexScan("TBL2", "IDX2_2")))))); + } +} diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/hints/HintsTestSuite.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/hints/HintsTestSuite.java new file mode 100644 index 0000000000000..4271935d626e1 --- /dev/null +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/hints/HintsTestSuite.java @@ -0,0 +1,33 @@ +/* + * 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.hints; + +import org.junit.runner.RunWith; +import org.junit.runners.Suite; + +/** + * SQL hints tests. + */ +@RunWith(Suite.class) +@Suite.SuiteClasses({ + CommonHintsPlannerTest.class, + NoIndexHintPlannerTest.class, + ForceIndexHintPlannerTest.class +}) +public class HintsTestSuite { +} diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/hints/NoIndexHintPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/hints/NoIndexHintPlannerTest.java new file mode 100644 index 0000000000000..39b8d9e754813 --- /dev/null +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/hints/NoIndexHintPlannerTest.java @@ -0,0 +1,288 @@ +/* + * 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.hints; + +import org.apache.ignite.internal.processors.query.QueryUtils; +import org.apache.ignite.internal.processors.query.calcite.planner.AbstractPlannerTest; +import org.apache.ignite.internal.processors.query.calcite.planner.TestTable; +import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexScan; +import org.apache.ignite.internal.processors.query.calcite.schema.IgniteSchema; +import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions; +import org.apache.ignite.testframework.LogListener; +import org.apache.ignite.testframework.junits.logger.GridTestLog4jLogger; +import org.apache.logging.log4j.Level; +import org.junit.Test; + +/** + * Planner test for index hints. + */ +public class NoIndexHintPlannerTest extends AbstractPlannerTest { + /** */ + private IgniteSchema schema; + + /** */ + private TestTable tbl1; + + /** */ + private TestTable tbl2; + + /** {@inheritDoc} */ + @Override public void setup() { + super.setup(); + + tbl1 = createTable("TBL1", 100, IgniteDistributions.single(), "ID", Integer.class, + "VAL1", String.class, "VAL2", String.class, "VAL3", String.class) + .addIndex(QueryUtils.PRIMARY_KEY_INDEX, 0) + .addIndex("IDX1_1", 1) + .addIndex("IDX1_23", 2, 3) + .addIndex("IDX1_3", 3); + + tbl2 = createTable("TBL2", 100_000, IgniteDistributions.single(), "ID", Integer.class, + "VAL1", String.class, "VAL2", String.class, "VAL3", String.class) + .addIndex(QueryUtils.PRIMARY_KEY_INDEX, 0) + .addIndex("IDX2_1", 1) + .addIndex("IDX2_2", 2) + .addIndex("IDX2_3", 3); + + schema = createSchema(tbl1, tbl2); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + ((GridTestLog4jLogger)log).setLevel(Level.INFO); + } + + /** */ + @Test + public void testWrongParams() throws Exception { + LogListener lsnr = LogListener.matches("Skipped hint 'NO_INDEX' with options 'IDX2_1','IDX2_1'") + .times(1).build(); + + lsnrLog.registerListener(lsnr); + + ((GridTestLog4jLogger)log).setLevel(Level.DEBUG); + + physicalPlan("SELECT /*+ NO_INDEX(IDX2_1,IDX2_1) */ * FROM TBL2 WHERE val2='v'", schema); + + assertTrue(lsnr.check()); + + lsnrLog.clearListeners(); + + lsnr = LogListener.matches("Skipped hint 'NO_INDEX' with options 'IDX2_1'").times(1).build(); + + lsnrLog.registerListener(lsnr); + + physicalPlan("SELECT /*+ NO_INDEX, NO_INDEX(IDX2_1) */ * FROM TBL2 WHERE val2='v'", schema); + + assertTrue(lsnr.check()); + } + + /** */ + @Test + public void testCertainIndex() throws Exception { + assertNoAnyIndex("SELECT /*+ NO_INDEX */ * FROM TBL2 WHERE val2='v'"); + + // Checks lower-case idx name. + assertCertainIndex("SELECT /*+ NO_INDEX('idx1_1') */ * FROM TBL1 WHERE val1='v'", "TBL1", "IDX1_1"); + + // Without quotes, Calcite's parser makes lower-case upper. + assertNoCertainIndex("SELECT /*+ NO_INDEX(idx1_1) */ * FROM TBL1 WHERE val1='v'", "TBL1", "IDX1_1"); + + assertCertainIndex("SELECT /*+ NO_INDEX(" + QueryUtils.PRIMARY_KEY_INDEX + + ") */ * FROM TBL1 WHERE id = 0", "TBL1", QueryUtils.PRIMARY_KEY_INDEX); + assertNoCertainIndex("SELECT /*+ NO_INDEX('" + QueryUtils.PRIMARY_KEY_INDEX + "') */ * FROM TBL1 WHERE id = 0", + "TBL1", QueryUtils.PRIMARY_KEY_INDEX); + + assertNoAnyIndex("SELECT /*+ NO_INDEX('idx1_1','IDX1_1','IDX1_23','IDX1_3') */ * FROM TBL1 WHERE val1='v' " + + "and val2='v' and val3='v'"); + + // Wrong names should be just skipped. + assertNoAnyIndex("SELECT " + + "/*+ NO_INDEX('UNEXISTING','idx1_1','UNEXISTING2','IDX1_1','UNEXISTING3','IDX1_23','IDX1_3') */ * " + + "FROM TBL1 WHERE val1='v' and val2='v' and val3='v'"); + + // Dedicated hint for each index. + assertNoAnyIndex("SELECT /*+ NO_INDEX('idx1_1'), NO_INDEX(IDX1_1), NO_INDEX(IDX1_23), NO_INDEX(IDX1_3) */ * " + + "FROM TBL1 WHERE val1='v' and val2='v' and val3='v'"); + + // Index of the second table. + assertNoCertainIndex("SELECT /*+ NO_INDEX(IDX2_3) */ t1.val3, t2.val3 FROM TBL1 t1, TBL2 t2 WHERE " + + "t1.val3='v' and t2.val3='v'", "TBL2", "IDX2_3"); + } + + /** */ + @Test + public void testSecondQuery() throws Exception { + assertNoCertainIndex("SELECT /*+ NO_INDEX(IDX1_23) */ * FROM TBL1 t1, (select * FROM TBL2 WHERE " + + "val2='v') t2 WHERE t1.val2='v'", "TBL1", "IDX1_23"); + + // Propagated, pushed-down hint. + assertNoCertainIndex("SELECT /*+ NO_INDEX(IDX2_3) */ * FROM TBL1 t1, (select * FROM TBL2 WHERE " + + "val3='v') t2 WHERE t1.val2='v'", "TBL2", "IDX2_3"); + + // Not-root hint. + assertNoCertainIndex("SELECT * FROM TBL1 t1, (select /*+ NO_INDEX(IDX2_3) */ * FROM TBL2 WHERE " + + "val3='v') t2 WHERE t1.val2='v'", "TBL2", "IDX2_3"); + } + + /** */ + @Test + public void testCorrelatedSubquery() throws Exception { + doTestDisabledInTable2Val3("t1.val3"); + } + + /** */ + @Test + public void testSubquery() throws Exception { + doTestDisabledInTable2Val3("'v'"); + } + + /** */ + @Test + public void testOrderBy() throws Exception { + assertNoCertainIndex("SELECT /*+ NO_INDEX(IDX1_23) */ val3 FROM TBL1 order by val2, val3", "TBL1", "IDX1_23"); + } + + /** */ + @Test + public void testAggregates() throws Exception { + doTestAggregate("sum"); + doTestAggregate("avg"); + doTestAggregate("min"); + doTestAggregate("max"); + } + + /** */ + private void doTestAggregate(String op) throws Exception { + assertNoCertainIndex("SELECT /*+ NO_INDEX(IDX2_3) */ " + op + "(val1) FROM TBL2 group by val3", "TBL2", "IDX2_3"); + + assertNoAnyIndex("SELECT /*+ NO_INDEX */ " + op + "(val1) FROM TBL2 group by val3"); + } + + /** */ + @Test + public void testJoins() throws Exception { + assertNoAnyIndex("SELECT /*+ NO_INDEX */ t1.val1, t2.val2 FROM TBL1 t1, TBL2 t2 where " + + "t2.val3=t1.val3"); + + assertNoCertainIndex("SELECT /*+ NO_INDEX(IDX1_3,IDX2_3) */ t1.val1, t2.val2 FROM TBL1 t1, TBL2 t2 where " + + "t2.val3=t1.val3", "TBL1", "IDX1_3"); + assertNoCertainIndex("SELECT /*+ NO_INDEX(IDX1_3,IDX2_3) */ t1.val1, t2.val2 FROM TBL1 t1, TBL2 t2 where " + + "t2.val3=t1.val3", "TBL2", "IDX2_3"); + + assertNoCertainIndex("SELECT /*+ NO_INDEX(IDX1_3) */ t1.val1, t2.val2 FROM TBL1 t1 JOIN TBL2 t2 on " + + "t1.val3=t2.val3", "TBL1", "IDX1_3"); + + assertNoCertainIndex("SELECT /*+ NO_INDEX(IDX2_3) */ t1.val1, t2.val2 FROM TBL1 t1 JOIN TBL2 t2 on " + + "t1.val3=t2.val3", "TBL2", "IDX2_3"); + + // With a filter + assertNoCertainIndex("SELECT /*+ NO_INDEX(IDX2_2) */ t1.val1, t2.val2 FROM TBL1 t1, TBL2 t2 where t1.val2='v' " + + "and t2.val2=t1.val2", "TBL2", "IDX2_2"); + } + + /** */ + @Test + public void testUnion() throws Exception { + doTestSetOps("UNION"); + } + + /** */ + @Test + public void testIntersect() throws Exception { + doTestSetOps("INTERSECT"); + } + + /** */ + private void doTestSetOps(String operation) throws Exception { + assertPlan(String.format("SELECT /*+ NO_INDEX(IDX2_3) */ t1.* FROM TBL1 t1 where t1.val2='v' %s " + + "SELECT t2.* FROM TBL2 t2 where t2.val3='v'", operation), schema, + nodeOrAnyChild(isIndexScan("TBL2", "IDX2_3")) + .and(nodeOrAnyChild(isIndexScan("TBL1", "IDX1_23")))); + + assertPlan(String.format("SELECT /*+ NO_INDEX(IDX1_23) */ t1.* FROM TBL1 t1 where t1.val2='v' %s " + + "SELECT t2.* FROM TBL2 t2 where t2.val3='v'", operation), schema, + nodeOrAnyChild(isIndexScan("TBL2", "IDX2_3")) + .and(nodeOrAnyChild(isIndexScan("TBL1", "IDX1_23")).negate())); + + assertPlan(String.format("SELECT /*+ NO_INDEX */ t1.* FROM TBL1 t1 where t1.val2='v' %s " + + "SELECT t2.* FROM TBL2 t2 where t2.val3='v'", operation), schema, + nodeOrAnyChild(isIndexScan("TBL2", "IDX2_3")) + .and(nodeOrAnyChild(isIndexScan("TBL1", "IDX1_23")).negate())); + + assertPlan(String.format("SELECT t1.* FROM TBL1 t1 where t1.val2='v' %s " + + "SELECT /*+ NO_INDEX(IDX2_3) */ t2.* FROM TBL2 t2 where t2.val3='v'", operation), schema, + nodeOrAnyChild(isIndexScan("TBL2", "IDX2_3")).negate() + .and(nodeOrAnyChild(isIndexScan("TBL1", "IDX1_23")))); + + assertPlan(String.format("SELECT t1.* FROM TBL1 t1 where t1.val2='v' %s " + + "SELECT /*+ NO_INDEX */ t2.* FROM TBL2 t2 where t2.val3='v'", operation), schema, + nodeOrAnyChild(isIndexScan("TBL2", "IDX2_3")).negate() + .and(nodeOrAnyChild(isIndexScan("TBL1", "IDX1_23")))); + + assertNoCertainIndex(String.format("SELECT /*+ NO_INDEX(IDX1_23) */ t1.* FROM TBL1 t1 where t1.val2='v' %s " + + "SELECT /*+ NO_INDEX(IDX2_3) */ t2.* FROM TBL2 t2 where t2.val3='v'", operation), "TBL1", "IDX1_23"); + assertNoCertainIndex(String.format("SELECT /*+ NO_INDEX(IDX1_23) */ t1.* FROM TBL1 t1 where t1.val2='v' %s " + + "SELECT /*+ NO_INDEX(IDX2_3) */ t2.* FROM TBL2 t2 where t2.val3='v'", operation), "TBL2", "IDX2_3"); + } + + /** + * Tests whether index 'IDX3' of table 'TBL2' in subquery of query to 'TBL1' is disabled. + * + * @param valueOfT2Val3 Value to use in 'WHERE TBL2.val2=' in the subquery. Can refer to 'TBL1'. + */ + private void doTestDisabledInTable2Val3(String valueOfT2Val3) throws Exception { + assertNoCertainIndex("SELECT /*+ NO_INDEX(IDX1_23) */ * FROM TBL1 t1 WHERE t1.val2 = " + + "(SELECT val2 from TBL2 WHERE val3=" + valueOfT2Val3 + ')', "TBL1", "IDX1_23"); + + assertNoCertainIndex("SELECT /*+ NO_INDEX(IDX2_3) */ * FROM TBL1 t1 WHERE t1.val2 = " + + "(SELECT val2 from TBL2 WHERE val3=" + valueOfT2Val3 + ')', "TBL2", "IDX2_3"); + + assertNoCertainIndex("SELECT * FROM TBL1 t1 WHERE t1.val2 = " + + "(SELECT /*+ NO_INDEX(IDX2_3) */ val2 from TBL2 WHERE val3=" + valueOfT2Val3 + ')', "TBL2", "IDX2_3"); + + assertCertainIndex("SELECT t2.val3 FROM TBL2 t2 WHERE t2.val2 = " + + "(SELECT /*+ NO_INDEX(IDX2_2) */ t1.val2 from TBL1 t1 WHERE t1.val3=" + valueOfT2Val3 + ')', "TBL2", "IDX2_2"); + + assertNoAnyIndex("SELECT /*+ NO_INDEX(IDX1_3), NO_INDEX(IDX2_3) */ * FROM TBL1 t1 WHERE t1.val3 = " + + "(SELECT val2 from TBL2 WHERE val3=" + valueOfT2Val3 + ')'); + assertNoAnyIndex("SELECT /*+ NO_INDEX(IDX1_3, IDX2_3) */ * FROM TBL1 t1 WHERE t1.val3 = " + + "(SELECT val2 from TBL2 WHERE val3=" + valueOfT2Val3 + ')'); + assertNoAnyIndex("SELECT /*+ NO_INDEX */ * FROM TBL1 t1 WHERE t1.val3 = " + + "(SELECT val2 from TBL2 WHERE val3=" + valueOfT2Val3 + ')'); + assertNoAnyIndex("SELECT /*+ NO_INDEX(IDX1_3) */ * FROM TBL1 t1 WHERE t1.val3 = " + + "(SELECT /*+ NO_INDEX(IDX2_3) */ val2 from TBL2 WHERE val3=" + valueOfT2Val3 + ')'); + } + + /** */ + private void assertNoAnyIndex(String sql) throws Exception { + assertPlan(sql, schema, nodeOrAnyChild(isInstanceOf(IgniteIndexScan.class)).negate()); + } + + /** */ + private void assertNoCertainIndex(String sql, String tblName, String idxName) throws Exception { + assertPlan(sql, schema, nodeOrAnyChild(isIndexScan(tblName, idxName)).negate()); + } + + /** */ + private void assertCertainIndex(String sql, String tblName, String idxName) throws Exception { + assertPlan(sql, schema, nodeOrAnyChild(isIndexScan(tblName, idxName))); + } +} diff --git a/modules/calcite/src/test/java/org/apache/ignite/testsuites/PlannerTestSuite.java b/modules/calcite/src/test/java/org/apache/ignite/testsuites/PlannerTestSuite.java index fb42d1457eb1a..d7f185e32cee3 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/testsuites/PlannerTestSuite.java +++ b/modules/calcite/src/test/java/org/apache/ignite/testsuites/PlannerTestSuite.java @@ -42,6 +42,7 @@ import org.apache.ignite.internal.processors.query.calcite.planner.TableFunctionPlannerTest; import org.apache.ignite.internal.processors.query.calcite.planner.TableSpoolPlannerTest; import org.apache.ignite.internal.processors.query.calcite.planner.UnionPlannerTest; +import org.apache.ignite.internal.processors.query.calcite.planner.hints.HintsTestSuite; import org.junit.runner.RunWith; import org.junit.runners.Suite; @@ -75,6 +76,8 @@ PlannerTimeoutTest.class, IndexSearchBoundsPlannerTest.class, InlineIndexScanPlannerTest.class, + + HintsTestSuite.class }) public class PlannerTestSuite { }