diff --git a/core/src/main/java/org/opensearch/sql/calcite/CalcitePlanContext.java b/core/src/main/java/org/opensearch/sql/calcite/CalcitePlanContext.java index d9cc7251e14..6ad935e59da 100644 --- a/core/src/main/java/org/opensearch/sql/calcite/CalcitePlanContext.java +++ b/core/src/main/java/org/opensearch/sql/calcite/CalcitePlanContext.java @@ -21,9 +21,9 @@ import org.apache.calcite.rex.RexLambdaRef; import org.apache.calcite.rex.RexNode; import org.apache.calcite.tools.FrameworkConfig; -import org.apache.calcite.tools.RelBuilder; import org.opensearch.sql.ast.expression.UnresolvedExpression; import org.opensearch.sql.calcite.utils.CalciteToolsHelper; +import org.opensearch.sql.calcite.utils.CalciteToolsHelper.OpenSearchRelBuilder; import org.opensearch.sql.common.setting.Settings; import org.opensearch.sql.executor.QueryType; import org.opensearch.sql.expression.function.FunctionProperties; @@ -32,7 +32,7 @@ public class CalcitePlanContext { public FrameworkConfig config; public final Connection connection; - public final RelBuilder relBuilder; + public final OpenSearchRelBuilder relBuilder; public final ExtendedRexBuilder rexBuilder; public final FunctionProperties functionProperties; public final QueryType queryType; diff --git a/core/src/main/java/org/opensearch/sql/calcite/CalciteRelNodeVisitor.java b/core/src/main/java/org/opensearch/sql/calcite/CalciteRelNodeVisitor.java index 19dce3e3609..937a35b98cb 100644 --- a/core/src/main/java/org/opensearch/sql/calcite/CalciteRelNodeVisitor.java +++ b/core/src/main/java/org/opensearch/sql/calcite/CalciteRelNodeVisitor.java @@ -14,8 +14,8 @@ import static org.opensearch.sql.ast.tree.Sort.SortOption.DEFAULT_DESC; import static org.opensearch.sql.ast.tree.Sort.SortOrder.ASC; import static org.opensearch.sql.ast.tree.Sort.SortOrder.DESC; -import static org.opensearch.sql.calcite.utils.PlanUtils.ROW_NUMBER_COLUMN_FOR_DEDUP; -import static org.opensearch.sql.calcite.utils.PlanUtils.ROW_NUMBER_COLUMN_FOR_JOIN_MAX_DEDUP; +import static org.opensearch.sql.calcite.plan.rule.PPLDedupConvertRule.buildDedupNotNull; +import static org.opensearch.sql.calcite.plan.rule.PPLDedupConvertRule.buildDedupOrNull; import static org.opensearch.sql.calcite.utils.PlanUtils.ROW_NUMBER_COLUMN_FOR_MAIN; import static org.opensearch.sql.calcite.utils.PlanUtils.ROW_NUMBER_COLUMN_FOR_RARE_TOP; import static org.opensearch.sql.calcite.utils.PlanUtils.ROW_NUMBER_COLUMN_FOR_STREAMSTATS; @@ -146,9 +146,9 @@ import org.opensearch.sql.ast.tree.Values; import org.opensearch.sql.ast.tree.Window; import org.opensearch.sql.calcite.plan.AliasFieldsWrappable; -import org.opensearch.sql.calcite.plan.LogicalSystemLimit; -import org.opensearch.sql.calcite.plan.LogicalSystemLimit.SystemLimitType; import org.opensearch.sql.calcite.plan.OpenSearchConstants; +import org.opensearch.sql.calcite.plan.rel.LogicalSystemLimit; +import org.opensearch.sql.calcite.plan.rel.LogicalSystemLimit.SystemLimitType; import org.opensearch.sql.calcite.utils.BinUtils; import org.opensearch.sql.calcite.utils.JoinAndLookupUtils; import org.opensearch.sql.calcite.utils.PPLHintUtils; @@ -1330,7 +1330,7 @@ public RelNode visitJoin(Join node, CalcitePlanContext context) { : duplicatedFieldNames.stream() .map(a -> (RexNode) context.relBuilder.field(a)) .toList(); - buildDedupNotNull(context, dedupeFields, allowedDuplication, true); + buildDedupNotNull(context.relBuilder, dedupeFields, allowedDuplication); } // add LogicalSystemLimit after dedup addSysLimitForJoinSubsearch(context); @@ -1388,7 +1388,7 @@ public RelNode visitJoin(Join node, CalcitePlanContext context) { List dedupeFields = getRightColumnsInJoinCriteria(context.relBuilder, joinCondition); - buildDedupNotNull(context, dedupeFields, allowedDuplication, true); + buildDedupNotNull(context.relBuilder, dedupeFields, allowedDuplication); } // add LogicalSystemLimit after dedup addSysLimitForJoinSubsearch(context); @@ -1565,81 +1565,13 @@ public RelNode visitDedupe(Dedupe node, CalcitePlanContext context) { List dedupeFields = node.getFields().stream().map(f -> rexVisitor.analyze(f, context)).toList(); if (keepEmpty) { - buildDedupOrNull(context, dedupeFields, allowedDuplication); + buildDedupOrNull(context.relBuilder, dedupeFields, allowedDuplication); } else { - buildDedupNotNull(context, dedupeFields, allowedDuplication, false); + buildDedupNotNull(context.relBuilder, dedupeFields, allowedDuplication); } return context.relBuilder.peek(); } - private static void buildDedupOrNull( - CalcitePlanContext context, List dedupeFields, Integer allowedDuplication) { - /* - * | dedup 2 a, b keepempty=true - * LogicalProject(...) - * +- LogicalFilter(condition=[OR(IS NULL(a), IS NULL(b), <=(_row_number_dedup_, 1))]) - * +- LogicalProject(..., _row_number_dedup_=[ROW_NUMBER() OVER (PARTITION BY a, b ORDER BY a, b)]) - * +- ... - */ - RexNode rowNumber = - context - .relBuilder - .aggregateCall(SqlStdOperatorTable.ROW_NUMBER) - .over() - .partitionBy(dedupeFields) - .rowsTo(RexWindowBounds.CURRENT_ROW) - .as(ROW_NUMBER_COLUMN_FOR_DEDUP); - context.relBuilder.projectPlus(rowNumber); - RexNode _row_number_dedup_ = context.relBuilder.field(ROW_NUMBER_COLUMN_FOR_DEDUP); - // Filter (isnull('a) OR isnull('b) OR '_row_number_dedup_ <= n) - context.relBuilder.filter( - context.relBuilder.or( - context.relBuilder.or(dedupeFields.stream().map(context.relBuilder::isNull).toList()), - context.relBuilder.lessThanOrEqual( - _row_number_dedup_, context.relBuilder.literal(allowedDuplication)))); - // DropColumns('_row_number_dedup_) - context.relBuilder.projectExcept(_row_number_dedup_); - } - - private static void buildDedupNotNull( - CalcitePlanContext context, - List dedupeFields, - Integer allowedDuplication, - boolean fromJoinMaxOption) { - /* - * | dedup 2 a, b keepempty=false - * LogicalProject(...) - * +- LogicalFilter(condition=[<=(_row_number_dedup_, n)])) - * +- LogicalProject(..., _row_number_dedup_=[ROW_NUMBER() OVER (PARTITION BY a, b ORDER BY a, b)]) - * +- LogicalFilter(condition=[AND(IS NOT NULL(a), IS NOT NULL(b))]) - * +- ... - */ - // Filter (isnotnull('a) AND isnotnull('b)) - String rowNumberAlias = - fromJoinMaxOption ? ROW_NUMBER_COLUMN_FOR_JOIN_MAX_DEDUP : ROW_NUMBER_COLUMN_FOR_DEDUP; - context.relBuilder.filter( - context.relBuilder.and(dedupeFields.stream().map(context.relBuilder::isNotNull).toList())); - // Window [row_number() windowspecdefinition('a, 'b, 'a ASC NULLS FIRST, 'b ASC NULLS FIRST, - // specifiedwindowoundedpreceding$(), currentrow$())) AS _row_number_dedup_], ['a, 'b], ['a ASC - // NULLS FIRST, 'b ASC NULLS FIRST] - RexNode rowNumber = - context - .relBuilder - .aggregateCall(SqlStdOperatorTable.ROW_NUMBER) - .over() - .partitionBy(dedupeFields) - .rowsTo(RexWindowBounds.CURRENT_ROW) - .as(rowNumberAlias); - context.relBuilder.projectPlus(rowNumber); - RexNode rowNumberField = context.relBuilder.field(rowNumberAlias); - // Filter ('_row_number_dedup_ <= n) - context.relBuilder.filter( - context.relBuilder.lessThanOrEqual( - rowNumberField, context.relBuilder.literal(allowedDuplication))); - // DropColumns('_row_number_dedup_) - context.relBuilder.projectExcept(rowNumberField); - } - @Override public RelNode visitWindow(Window node, CalcitePlanContext context) { visitChildren(node, context); diff --git a/core/src/main/java/org/opensearch/sql/calcite/CalciteRexNodeVisitor.java b/core/src/main/java/org/opensearch/sql/calcite/CalciteRexNodeVisitor.java index 632b67df9c5..9354bcc3329 100644 --- a/core/src/main/java/org/opensearch/sql/calcite/CalciteRexNodeVisitor.java +++ b/core/src/main/java/org/opensearch/sql/calcite/CalciteRexNodeVisitor.java @@ -70,8 +70,8 @@ import org.opensearch.sql.ast.expression.subquery.ScalarSubquery; import org.opensearch.sql.ast.expression.subquery.SubqueryExpression; import org.opensearch.sql.ast.tree.UnresolvedPlan; -import org.opensearch.sql.calcite.plan.LogicalSystemLimit; -import org.opensearch.sql.calcite.plan.LogicalSystemLimit.SystemLimitType; +import org.opensearch.sql.calcite.plan.rel.LogicalSystemLimit; +import org.opensearch.sql.calcite.plan.rel.LogicalSystemLimit.SystemLimitType; import org.opensearch.sql.calcite.utils.OpenSearchTypeFactory; import org.opensearch.sql.calcite.utils.PlanUtils; import org.opensearch.sql.calcite.utils.SubsearchUtils; diff --git a/core/src/main/java/org/opensearch/sql/calcite/plan/rel/Dedup.java b/core/src/main/java/org/opensearch/sql/calcite/plan/rel/Dedup.java new file mode 100644 index 00000000000..6d593787eb5 --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/calcite/plan/rel/Dedup.java @@ -0,0 +1,89 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.calcite.plan.rel; + +import java.util.List; +import lombok.Getter; +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelOptPlanner; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.RelWriter; +import org.apache.calcite.rel.SingleRel; +import org.apache.calcite.rex.RexNode; +import org.opensearch.sql.exception.CalciteUnsupportedException; + +/** Relational expression representing a dedup command. */ +@Getter +public abstract class Dedup extends SingleRel { + final List dedupeFields; + final Integer allowedDuplication; + final Boolean keepEmpty; + final Boolean consecutive; + + /** */ + protected Dedup( + RelOptCluster cluster, + RelTraitSet traitSet, + RelNode input, + List dedupeFields, + Integer allowedDuplication, + Boolean keepEmpty, + Boolean consecutive) { + super(cluster, traitSet, input); + if (allowedDuplication <= 0) { + throw new IllegalArgumentException("Number of duplicate events must be greater than 0"); + } + if (consecutive) { + throw new CalciteUnsupportedException("Consecutive deduplication is unsupported in Calcite"); + } + this.dedupeFields = dedupeFields; + this.allowedDuplication = allowedDuplication; + this.keepEmpty = keepEmpty; + this.consecutive = consecutive; + } + + @Override + public final RelNode copy(RelTraitSet traitSet, List inputs) { + return copy( + traitSet, + sole(inputs), + this.dedupeFields, + this.allowedDuplication, + this.keepEmpty, + this.consecutive); + } + + public abstract Dedup copy( + RelTraitSet traitSet, + RelNode input, + List dedupeFields, + Integer allowedDuplication, + Boolean keepEmpty, + Boolean consecutive); + + public Dedup copy(RelNode input, List dedupeFields) { + return this.copy( + this.getTraitSet(), + input, + dedupeFields, + this.allowedDuplication, + this.keepEmpty, + this.consecutive); + } + + @Override + public RelWriter explainTerms(RelWriter pw) { + return super.explainTerms(pw) + .item("dedup_fields", dedupeFields) + .item("allowed_dedup", allowedDuplication) + .item("keepEmpty", keepEmpty) + .item("consecutive", consecutive); + } + + @Override + public void register(RelOptPlanner planner) {} +} diff --git a/core/src/main/java/org/opensearch/sql/calcite/plan/rel/LogicalDedup.java b/core/src/main/java/org/opensearch/sql/calcite/plan/rel/LogicalDedup.java new file mode 100644 index 00000000000..2a8eb5038d6 --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/calcite/plan/rel/LogicalDedup.java @@ -0,0 +1,60 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.calcite.plan.rel; + +import static org.opensearch.sql.calcite.plan.rule.PPLDedupConvertRule.DEDUP_CONVERT_RULE; + +import java.util.List; +import org.apache.calcite.plan.Convention; +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelOptPlanner; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rex.RexNode; + +public class LogicalDedup extends Dedup { + + protected LogicalDedup( + RelOptCluster cluster, + RelTraitSet traitSet, + RelNode input, + List dedupeFields, + Integer allowedDuplication, + Boolean keepEmpty, + Boolean consecutive) { + super(cluster, traitSet, input, dedupeFields, allowedDuplication, keepEmpty, consecutive); + } + + @Override + public Dedup copy( + RelTraitSet traitSet, + RelNode input, + List dedupeFields, + Integer allowedDuplication, + Boolean keepEmpty, + Boolean consecutive) { + assert traitSet.containsIfApplicable(Convention.NONE); + return new LogicalDedup( + getCluster(), traitSet, input, dedupeFields, allowedDuplication, keepEmpty, consecutive); + } + + public static LogicalDedup create( + RelNode input, + List dedupeFields, + Integer allowedDuplication, + Boolean keepEmpty, + Boolean consecutive) { + final RelOptCluster cluster = input.getCluster(); + RelTraitSet traitSet = cluster.traitSetOf(Convention.NONE); + return new LogicalDedup( + cluster, traitSet, input, dedupeFields, allowedDuplication, keepEmpty, consecutive); + } + + @Override + public void register(RelOptPlanner planner) { + planner.addRule(DEDUP_CONVERT_RULE); + } +} diff --git a/core/src/main/java/org/opensearch/sql/calcite/plan/LogicalSystemLimit.java b/core/src/main/java/org/opensearch/sql/calcite/plan/rel/LogicalSystemLimit.java similarity index 98% rename from core/src/main/java/org/opensearch/sql/calcite/plan/LogicalSystemLimit.java rename to core/src/main/java/org/opensearch/sql/calcite/plan/rel/LogicalSystemLimit.java index 84c448c8387..4f999cf0792 100644 --- a/core/src/main/java/org/opensearch/sql/calcite/plan/LogicalSystemLimit.java +++ b/core/src/main/java/org/opensearch/sql/calcite/plan/rel/LogicalSystemLimit.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.sql.calcite.plan; +package org.opensearch.sql.calcite.plan.rel; import java.util.Collections; import java.util.List; diff --git a/core/src/main/java/org/opensearch/sql/calcite/plan/OpenSearchTableScan.java b/core/src/main/java/org/opensearch/sql/calcite/plan/rel/OpenSearchTableScan.java similarity index 92% rename from core/src/main/java/org/opensearch/sql/calcite/plan/OpenSearchTableScan.java rename to core/src/main/java/org/opensearch/sql/calcite/plan/rel/OpenSearchTableScan.java index 03678713ef4..0794d8f4d0f 100644 --- a/core/src/main/java/org/opensearch/sql/calcite/plan/OpenSearchTableScan.java +++ b/core/src/main/java/org/opensearch/sql/calcite/plan/rel/OpenSearchTableScan.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.sql.calcite.plan; +package org.opensearch.sql.calcite.plan.rel; import com.google.common.collect.ImmutableList; import org.apache.calcite.adapter.enumerable.EnumerableConvention; @@ -14,6 +14,7 @@ import org.apache.calcite.plan.RelOptTable; import org.apache.calcite.rel.core.TableScan; import org.apache.calcite.rel.rules.CoreRules; +import org.opensearch.sql.calcite.plan.rule.OpenSearchRules; /** Relational expression representing a scan of an OpenSearch type. */ public abstract class OpenSearchTableScan extends TableScan implements EnumerableRel { diff --git a/core/src/main/java/org/opensearch/sql/calcite/plan/OpenSearchRuleConfig.java b/core/src/main/java/org/opensearch/sql/calcite/plan/rule/OpenSearchRuleConfig.java similarity index 93% rename from core/src/main/java/org/opensearch/sql/calcite/plan/OpenSearchRuleConfig.java rename to core/src/main/java/org/opensearch/sql/calcite/plan/rule/OpenSearchRuleConfig.java index ca031afbb51..3786ebf19ff 100644 --- a/core/src/main/java/org/opensearch/sql/calcite/plan/OpenSearchRuleConfig.java +++ b/core/src/main/java/org/opensearch/sql/calcite/plan/rule/OpenSearchRuleConfig.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.sql.calcite.plan; +package org.opensearch.sql.calcite.plan.rule; import org.apache.calcite.plan.Contexts; import org.apache.calcite.plan.RelRule; diff --git a/core/src/main/java/org/opensearch/sql/calcite/plan/OpenSearchRules.java b/core/src/main/java/org/opensearch/sql/calcite/plan/rule/OpenSearchRules.java similarity index 93% rename from core/src/main/java/org/opensearch/sql/calcite/plan/OpenSearchRules.java rename to core/src/main/java/org/opensearch/sql/calcite/plan/rule/OpenSearchRules.java index 8d41b30ab91..9eb30bb51d7 100644 --- a/core/src/main/java/org/opensearch/sql/calcite/plan/OpenSearchRules.java +++ b/core/src/main/java/org/opensearch/sql/calcite/plan/rule/OpenSearchRules.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.sql.calcite.plan; +package org.opensearch.sql.calcite.plan.rule; import com.google.common.collect.ImmutableList; import java.util.List; diff --git a/core/src/main/java/org/opensearch/sql/calcite/plan/PPLAggGroupMergeRule.java b/core/src/main/java/org/opensearch/sql/calcite/plan/rule/PPLAggGroupMergeRule.java similarity index 99% rename from core/src/main/java/org/opensearch/sql/calcite/plan/PPLAggGroupMergeRule.java rename to core/src/main/java/org/opensearch/sql/calcite/plan/rule/PPLAggGroupMergeRule.java index 97fde04768f..4be6f466ce2 100644 --- a/core/src/main/java/org/opensearch/sql/calcite/plan/PPLAggGroupMergeRule.java +++ b/core/src/main/java/org/opensearch/sql/calcite/plan/rule/PPLAggGroupMergeRule.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.sql.calcite.plan; +package org.opensearch.sql.calcite.plan.rule; import java.util.ArrayList; import java.util.Collection; diff --git a/core/src/main/java/org/opensearch/sql/calcite/plan/PPLAggregateConvertRule.java b/core/src/main/java/org/opensearch/sql/calcite/plan/rule/PPLAggregateConvertRule.java similarity index 99% rename from core/src/main/java/org/opensearch/sql/calcite/plan/PPLAggregateConvertRule.java rename to core/src/main/java/org/opensearch/sql/calcite/plan/rule/PPLAggregateConvertRule.java index fee2d02e086..81e2c295de7 100644 --- a/core/src/main/java/org/opensearch/sql/calcite/plan/PPLAggregateConvertRule.java +++ b/core/src/main/java/org/opensearch/sql/calcite/plan/rule/PPLAggregateConvertRule.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.sql.calcite.plan; +package org.opensearch.sql.calcite.plan.rule; import com.google.common.collect.ImmutableList; import java.util.ArrayList; diff --git a/core/src/main/java/org/opensearch/sql/calcite/plan/rule/PPLDedupConvertRule.java b/core/src/main/java/org/opensearch/sql/calcite/plan/rule/PPLDedupConvertRule.java new file mode 100644 index 00000000000..c1b452a2ac0 --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/calcite/plan/rule/PPLDedupConvertRule.java @@ -0,0 +1,137 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.calcite.plan.rule; + +import static org.opensearch.sql.calcite.utils.PlanUtils.ROW_NUMBER_COLUMN_FOR_DEDUP; + +import java.util.List; +import java.util.stream.Collectors; +import org.apache.calcite.plan.RelOptRuleCall; +import org.apache.calcite.plan.RelRule; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexWindowBounds; +import org.apache.calcite.sql.fun.SqlStdOperatorTable; +import org.apache.calcite.tools.RelBuilder; +import org.immutables.value.Value; +import org.opensearch.sql.calcite.plan.rel.LogicalDedup; + +/** + * Planner rule that converts a logical dedup into equivalent composite of logical operators, e.g. + * + *
+ * | dedup 2 a, b keepempty=true
+ *
+ * becomes:
+ *
+ * LogicalDedup(dedupeFields=[a, b], allowedDuplication=2, keepempty=true)
+ *
+ * which is then converted to:
+ *
+ * LogicalProject(...)
+ * +- LogicalFilter(condition=[OR(IS NULL(a), IS NULL(b), <=(_row_number_dedup_, 2))])
+ *    +- LogicalProject(..., _row_number_dedup_=[ROW_NUMBER() OVER (PARTITION BY a, b ORDER BY a, b)])
+ * 
+ */ +@Value.Enclosing +public class PPLDedupConvertRule extends RelRule { + /** Creates a PPLDedupConvertRule. */ + protected PPLDedupConvertRule(Config config) { + super(config); + } + + @Override + public void onMatch(RelOptRuleCall call) { + final LogicalDedup dedup = call.rel(0); + RelBuilder relBuilder = call.builder(); + relBuilder.push(dedup.getInput()); + if (dedup.getKeepEmpty()) { + buildDedupOrNull(relBuilder, dedup.getDedupeFields(), dedup.getAllowedDuplication()); + } else { + buildDedupNotNull(relBuilder, dedup.getDedupeFields(), dedup.getAllowedDuplication()); + } + call.transformTo(relBuilder.build()); + } + + public static void buildDedupOrNull( + RelBuilder relBuilder, List dedupeFields, Integer allowedDuplication) { + /* + * | dedup 2 a, b keepempty=true + * LogicalProject(...) + * +- LogicalFilter(condition=[OR(IS NULL(a), IS NULL(b), <=(_row_number_dedup_, 1))]) + * +- LogicalProject(..., _row_number_dedup_=[ROW_NUMBER() OVER (PARTITION BY a, b ORDER BY a, b)]) + * +- ... + */ + RexNode rowNumber = + relBuilder + .aggregateCall(SqlStdOperatorTable.ROW_NUMBER) + .over() + .partitionBy(dedupeFields) + .rowsTo(RexWindowBounds.CURRENT_ROW) + .as(ROW_NUMBER_COLUMN_FOR_DEDUP); + relBuilder.projectPlus(rowNumber); + RexNode _row_number_dedup_ = relBuilder.field(ROW_NUMBER_COLUMN_FOR_DEDUP); + // Filter (isnull('a) OR isnull('b) OR '_row_number_dedup_ <= n) + relBuilder.filter( + relBuilder.or( + relBuilder.or( + dedupeFields.stream().map(relBuilder::isNull).collect(Collectors.toList())), + relBuilder.lessThanOrEqual( + _row_number_dedup_, relBuilder.literal(allowedDuplication)))); + // DropColumns('_row_number_dedup_) + relBuilder.projectExcept(_row_number_dedup_); + } + + public static void buildDedupNotNull( + RelBuilder relBuilder, List dedupeFields, Integer allowedDuplication) { + /* + * | dedup 2 a, b keepempty=false + * LogicalProject(...) + * +- LogicalFilter(condition=[<=(_row_number_dedup_, n)])) + * +- LogicalProject(..., _row_number_dedup_=[ROW_NUMBER() OVER (PARTITION BY a, b ORDER BY a, b)]) + * +- LogicalFilter(condition=[AND(IS NOT NULL(a), IS NOT NULL(b))]) + * +- ... + */ + // Filter (isnotnull('a) AND isnotnull('b)) + String rowNumberAlias = ROW_NUMBER_COLUMN_FOR_DEDUP; + relBuilder.filter( + relBuilder.and( + dedupeFields.stream().map(relBuilder::isNotNull).collect(Collectors.toList()))); + // Window [row_number() windowspecdefinition('a, 'b, 'a ASC NULLS FIRST, 'b ASC NULLS FIRST, + // specifiedwindowoundedpreceding$(), currentrow$())) AS _row_number_dedup_], ['a, 'b], ['a ASC + // NULLS FIRST, 'b ASC NULLS FIRST] + RexNode rowNumber = + relBuilder + .aggregateCall(SqlStdOperatorTable.ROW_NUMBER) + .over() + .partitionBy(dedupeFields) + .rowsTo(RexWindowBounds.CURRENT_ROW) + .as(rowNumberAlias); + relBuilder.projectPlus(rowNumber); + RexNode rowNumberField = relBuilder.field(rowNumberAlias); + // Filter ('_row_number_dedup_ <= n) + relBuilder.filter( + relBuilder.lessThanOrEqual(rowNumberField, relBuilder.literal(allowedDuplication))); + // DropColumns('_row_number_dedup_) + relBuilder.projectExcept(rowNumberField); + } + + /** Rule configuration. */ + @Value.Immutable + public interface Config extends OpenSearchRuleConfig { + Config DEDUP_CONVERTER = + ImmutablePPLDedupConvertRule.Config.builder() + .build() + .withOperandSupplier(b0 -> b0.operand(LogicalDedup.class).anyInputs()); + + @Override + default PPLDedupConvertRule toRule() { + return new PPLDedupConvertRule(this); + } + } + + public static final PPLDedupConvertRule DEDUP_CONVERT_RULE = + PPLDedupConvertRule.Config.DEDUP_CONVERTER.toRule(); +} diff --git a/core/src/main/java/org/opensearch/sql/calcite/plan/rule/PPLSimplifyDedupRule.java b/core/src/main/java/org/opensearch/sql/calcite/plan/rule/PPLSimplifyDedupRule.java new file mode 100644 index 00000000000..054141371b9 --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/calcite/plan/rule/PPLSimplifyDedupRule.java @@ -0,0 +1,185 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.calcite.plan.rule; + +import java.util.List; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import org.apache.calcite.plan.RelOptRuleCall; +import org.apache.calcite.plan.RelRule; +import org.apache.calcite.rel.logical.LogicalFilter; +import org.apache.calcite.rel.logical.LogicalProject; +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.rex.RexLiteral; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexWindow; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.calcite.tools.RelBuilder; +import org.apache.calcite.util.Pair; +import org.immutables.value.Value; +import org.opensearch.sql.calcite.plan.rel.LogicalDedup; +import org.opensearch.sql.calcite.utils.PlanUtils; + +/** + * Planner rule that simplify a composite of logical operators into a logical dedup, e.g. + * + *
+ * | dedup 2 a, b keepempty=true
+ *
+ * becomes:
+ * LogicalProject(...)
+ * +- LogicalFilter(condition=[OR(IS NULL(a), IS NULL(b), <=(_row_number_dedup_, 2))])
+ *    +- LogicalProject(..., _row_number_dedup_=[ROW_NUMBER() OVER (PARTITION BY a, b ORDER BY a, b)])
+ *
+ * which is simplified to:
+ *
+ * LogicalDedup(dedupeFields=[a, b], allowedDuplication=2, keepempty=true)
+ * 
+ */ +@Value.Enclosing +public class PPLSimplifyDedupRule extends RelRule { + /** Creates a PPLSimplifyDedupRule. */ + protected PPLSimplifyDedupRule(Config config) { + super(config); + } + + @Override + public void onMatch(RelOptRuleCall call) { + final LogicalProject finalProject = call.rel(0); + final LogicalFilter numOfDedupFilter = call.rel(1); + final LogicalProject projectWithWindow = call.rel(2); + final LogicalFilter bucketNonNullFilter = call.rel(3); + apply(call, finalProject, numOfDedupFilter, projectWithWindow, bucketNonNullFilter); + } + + /** + * Applies the simplification rule to transform a composite pattern into a LogicalDedup. + * + * @param call the rule call context + * @param finalProject the outer projection + * @param numOfDedupFilter the filter containing the row number condition + * @param projectWithWindow the projection containing the ROW_NUMBER window function + * @param bucketNonNullFilter the filter for non-null partition keys + */ + protected void apply( + RelOptRuleCall call, + LogicalProject finalProject, + LogicalFilter numOfDedupFilter, + LogicalProject projectWithWindow, + LogicalFilter bucketNonNullFilter) { + List windows = PlanUtils.getRexWindowFromProject(projectWithWindow); + if (windows.size() != 1) { + return; + } + + List dedupColumns = windows.get(0).partitionKeys; + if (dedupColumns.stream() + .filter(rex -> rex.isA(SqlKind.INPUT_REF)) + .anyMatch( + rex -> + rex.getType().getSqlTypeName() == SqlTypeName.MAP + || rex.getType().getSqlTypeName() == SqlTypeName.ARRAY)) { + return; + } + + // must be row_number <= number. + // Since we cannot push down dedup with keepEmpty=true, we don't simplify that pattern + RexNode condition = numOfDedupFilter.getCondition(); + if (!(condition instanceof RexCall)) { + return; + } + List operands = ((RexCall) condition).getOperands(); + if (operands.isEmpty()) { + return; + } + RexNode lastOperand = operands.get(operands.size() - 1); + if (!(lastOperand instanceof RexLiteral)) { + return; + } + RexLiteral literal = (RexLiteral) lastOperand; + Integer dedupNumber = literal.getValueAs(Integer.class); + if (dedupNumber == null) { + return; + } + + RelBuilder relBuilder = call.builder(); + relBuilder.push(bucketNonNullFilter.getInput()); + List> targetProjections = + projectWithWindow.getNamedProjects().stream() + .filter(p -> !p.getKey().isA(SqlKind.ROW_NUMBER)) + .collect(Collectors.toList()); + relBuilder.project( + targetProjections.stream().map(Pair::getKey).collect(Collectors.toList()), + targetProjections.stream().map(Pair::getValue).collect(Collectors.toList())); + + LogicalDedup dedup = + LogicalDedup.create(relBuilder.build(), dedupColumns, dedupNumber, false, false); + relBuilder.push(dedup); + relBuilder.project(finalProject.getProjects(), finalProject.getRowType().getFieldNames()); + + call.transformTo(relBuilder.build()); + } + + /** Rule configuration. */ + @Value.Immutable + public interface Config extends OpenSearchRuleConfig { + Config DEFAULT = + ImmutablePPLSimplifyDedupRule.Config.builder() + .build() + .withOperandSupplier( + b0 -> + b0.operand(LogicalProject.class) + .predicate(Predicate.not(PlanUtils::containsRowNumberDedup)) + .oneInput( + b1 -> + b1.operand(LogicalFilter.class) + .predicate(Config::validDedupNumberChecker) + .oneInput( + b2 -> + b2.operand(LogicalProject.class) + .predicate(PlanUtils::containsRowNumberDedup) + .oneInput( + b3 -> + b3.operand(LogicalFilter.class) + .predicate( + PlanUtils + ::mayBeFilterFromBucketNonNull) + .anyInputs())))); + + @Override + default PPLSimplifyDedupRule toRule() { + return new PPLSimplifyDedupRule(this); + } + + private static boolean validDedupNumberChecker(LogicalFilter filter) { + return filter.getCondition().isA(SqlKind.LESS_THAN_OR_EQUAL) + && PlanUtils.containsRowNumberDedup(filter); + } + + /** + * Check if the condition is null or less than. Should be useful if we can push down Dedup with + * keepEmpty=true in the future. + */ + private static boolean isNullOrLessThan(RexNode node) { + if (node.isA(SqlKind.LESS_THAN_OR_EQUAL)) return true; + if (!node.isA(SqlKind.OR)) return false; + boolean hasLessThan = false; + for (RexNode operand : ((RexCall) node).getOperands()) { + if (operand.isA(SqlKind.LESS_THAN_OR_EQUAL)) { + if (hasLessThan) return false; // only one less than + hasLessThan = true; + } else if (!operand.isA(SqlKind.IS_NULL)) { + return false; // only null if not less_than + } + } + return hasLessThan; // should be one less than + } + } + + public static final PPLSimplifyDedupRule DEDUP_SIMPLIFY_RULE = + PPLSimplifyDedupRule.Config.DEFAULT.toRule(); +} diff --git a/core/src/main/java/org/opensearch/sql/calcite/utils/CalciteToolsHelper.java b/core/src/main/java/org/opensearch/sql/calcite/utils/CalciteToolsHelper.java index b87ffea04d1..50b935ac8ad 100644 --- a/core/src/main/java/org/opensearch/sql/calcite/utils/CalciteToolsHelper.java +++ b/core/src/main/java/org/opensearch/sql/calcite/utils/CalciteToolsHelper.java @@ -36,6 +36,7 @@ import java.sql.PreparedStatement; import java.sql.SQLException; import java.time.Instant; +import java.util.List; import java.util.Properties; import java.util.function.Consumer; import org.apache.calcite.adapter.enumerable.EnumerableConvention; @@ -58,16 +59,23 @@ import org.apache.calcite.plan.Convention; import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.plan.RelOptPlanner; +import org.apache.calcite.plan.RelOptRule; import org.apache.calcite.plan.RelOptSchema; import org.apache.calcite.plan.RelOptTable; +import org.apache.calcite.plan.RelOptTable.ViewExpander; +import org.apache.calcite.plan.hep.HepPlanner; +import org.apache.calcite.plan.hep.HepProgram; +import org.apache.calcite.plan.hep.HepProgramBuilder; import org.apache.calcite.prepare.CalciteCatalogReader; import org.apache.calcite.prepare.CalcitePrepareImpl; +import org.apache.calcite.prepare.Prepare.CatalogReader; import org.apache.calcite.rel.RelHomogeneousShuttle; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.RelRoot; import org.apache.calcite.rel.RelShuttle; import org.apache.calcite.rel.core.TableScan; import org.apache.calcite.rel.logical.LogicalTableScan; +import org.apache.calcite.rel.rules.FilterMergeRule; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.rel.type.RelDataTypeSystem; @@ -79,7 +87,10 @@ import org.apache.calcite.server.CalciteServerStatement; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.calcite.sql.validate.SqlValidator; +import org.apache.calcite.sql2rel.RelFieldTrimmer; import org.apache.calcite.sql2rel.SqlRexConvertletTable; +import org.apache.calcite.sql2rel.SqlToRelConverter; import org.apache.calcite.tools.FrameworkConfig; import org.apache.calcite.tools.Frameworks; import org.apache.calcite.tools.RelBuilder; @@ -87,9 +98,11 @@ import org.apache.calcite.tools.RelRunner; import org.apache.calcite.util.Holder; import org.apache.calcite.util.Util; +import org.checkerframework.checker.nullness.qual.Nullable; import org.opensearch.sql.calcite.CalcitePlanContext; -import org.opensearch.sql.calcite.plan.OpenSearchRules; import org.opensearch.sql.calcite.plan.Scannable; +import org.opensearch.sql.calcite.plan.rule.OpenSearchRules; +import org.opensearch.sql.calcite.plan.rule.PPLSimplifyDedupRule; import org.opensearch.sql.expression.function.PPLBuiltinOperators; import org.opensearch.sql.monitor.profile.ProfileMetric; import org.opensearch.sql.monitor.profile.QueryProfiling; @@ -99,14 +112,13 @@ * 3. RelBuilder 4. RelRunner 5. CalcitePreparingStmt. TODO delete it in future if possible. */ public class CalciteToolsHelper { - /** Create a RelBuilder with testing */ public static RelBuilder create(FrameworkConfig config) { return RelBuilder.create(config); } /** Create a RelBuilder with typeFactory */ - public static RelBuilder create( + public static OpenSearchRelBuilder create( FrameworkConfig config, JavaTypeFactory typeFactory, Connection connection) { return withPrepare( config, @@ -279,6 +291,8 @@ protected CalcitePrepareImpl.CalcitePreparingStmt getPreparingStmt( public static class OpenSearchCalcitePreparingStmt extends CalcitePrepareImpl.CalcitePreparingStmt { + protected final RelOptCluster cluster; + public OpenSearchCalcitePreparingStmt( CalcitePrepareImpl prepare, CalcitePrepare.Context context, @@ -299,6 +313,7 @@ public OpenSearchCalcitePreparingStmt( cluster, resultConvention, convertletTable); + this.cluster = cluster; } @Override @@ -337,6 +352,41 @@ public Type getElementType() { } return super.implement(root); } + + @Override + protected SqlToRelConverter getSqlToRelConverter( + SqlValidator validator, CatalogReader catalogReader, SqlToRelConverter.Config config) { + return new OpenSearchSqlToRelConverter( + this, validator, catalogReader, this.cluster, convertletTable, config); + } + } + + public static class OpenSearchSqlToRelConverter extends SqlToRelConverter { + protected final RelBuilder relBuilder; + + public OpenSearchSqlToRelConverter( + ViewExpander viewExpander, + @Nullable SqlValidator validator, + CatalogReader catalogReader, + RelOptCluster cluster, + SqlRexConvertletTable convertletTable, + Config config) { + super(viewExpander, validator, catalogReader, cluster, convertletTable, config); + this.relBuilder = + config + .getRelBuilderFactory() + .create( + cluster, + validator != null + ? validator.getCatalogReader().unwrap(RelOptSchema.class) + : null) + .transform(config.getRelBuilderConfigTransform()); + } + + @Override + protected RelFieldTrimmer newFieldTrimmer() { + return new OpenSearchRelFieldTrimmer(validator, this.relBuilder); + } } public static class OpenSearchRelRunners { @@ -347,6 +397,8 @@ public static class OpenSearchRelRunners { public static PreparedStatement run(CalcitePlanContext context, RelNode rel) { ProfileMetric optimizeTime = QueryProfiling.current().getOrCreateMetric(OPTIMIZE); long startTime = System.nanoTime(); + // Optimize the plan by Calcite's HepPlanner before using VolcanoPlanner in prepareStatement. + rel = CalciteToolsHelper.optimize(rel, context); final RelShuttle shuttle = new RelHomogeneousShuttle() { @Override @@ -384,4 +436,18 @@ public RelNode visit(TableScan scan) { } } } + + /** Try to optimize the plan by using HepPlanner */ + private static final List hepRuleList = + List.of(FilterMergeRule.Config.DEFAULT.toRule(), PPLSimplifyDedupRule.DEDUP_SIMPLIFY_RULE); + + private static final HepProgram HEP_PROGRAM = + new HepProgramBuilder().addRuleCollection(hepRuleList).build(); + + public static RelNode optimize(RelNode plan, CalcitePlanContext context) { + Util.discard(context); + HepPlanner planner = new HepPlanner(HEP_PROGRAM); + planner.setRoot(plan); + return planner.findBestExp(); + } } diff --git a/core/src/main/java/org/opensearch/sql/calcite/utils/OpenSearchRelFieldTrimmer.java b/core/src/main/java/org/opensearch/sql/calcite/utils/OpenSearchRelFieldTrimmer.java new file mode 100644 index 00000000000..398b82530c9 --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/calcite/utils/OpenSearchRelFieldTrimmer.java @@ -0,0 +1,70 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.calcite.utils; + +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Set; +import org.apache.calcite.plan.RelOptUtil; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexPermuteInputsShuttle; +import org.apache.calcite.rex.RexVisitor; +import org.apache.calcite.sql.validate.SqlValidator; +import org.apache.calcite.sql2rel.RelFieldTrimmer; +import org.apache.calcite.tools.RelBuilder; +import org.apache.calcite.util.ImmutableBitSet; +import org.apache.calcite.util.mapping.Mapping; +import org.apache.calcite.util.mapping.Mappings; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.opensearch.sql.calcite.plan.rel.Dedup; + +/** + * Trims fields from a relational expression. + * + *

This class extends Calcite's RelFieldTrimmer to support trimming customized operators. + */ +public class OpenSearchRelFieldTrimmer extends RelFieldTrimmer { + + public OpenSearchRelFieldTrimmer(@Nullable SqlValidator validator, RelBuilder relBuilder) { + super(validator, relBuilder); + } + + public TrimResult trimFields( + Dedup dedup, ImmutableBitSet fieldsUsed, Set extraFields) { + final RelDataType rowType = dedup.getRowType(); + final int fieldCount = rowType.getFieldCount(); + final List dedupFields = dedup.getDedupeFields(); + RelNode input = dedup.getInput(); + + final Set inputExtraFields = new LinkedHashSet<>(extraFields); + RelOptUtil.InputFinder inputFinder = new RelOptUtil.InputFinder(inputExtraFields, fieldsUsed); + inputFinder.visitEach(dedup.getDedupeFields()); + final ImmutableBitSet inputFieldsUsed = inputFinder.build(); + + // Create input with trimmed columns. + TrimResult trimResult = trimChild(dedup, input, inputFieldsUsed, inputExtraFields); + RelNode newInput = trimResult.left; + final Mapping inputMapping = trimResult.right; + + // If the input is unchanged, and we need to project all columns, + // there's nothing we can do. + if (newInput == input && fieldsUsed.cardinality() == fieldCount) { + return result(dedup, Mappings.createIdentity(fieldCount)); + } + + // Build new project expressions, and populate the mapping. + final RexVisitor shuttle = new RexPermuteInputsShuttle(inputMapping, newInput); + List newDedupFields = shuttle.visitList(dedupFields); + + // The result has the same mapping as the input gave us. Sometimes we + // return fields that the consumer didn't ask for, because the filter + // needs them for its condition. + return result(dedup.copy(newInput, newDedupFields), inputMapping); + } +} diff --git a/core/src/main/java/org/opensearch/sql/calcite/utils/PlanUtils.java b/core/src/main/java/org/opensearch/sql/calcite/utils/PlanUtils.java index b5d22a19601..ec0eed5a97d 100644 --- a/core/src/main/java/org/opensearch/sql/calcite/utils/PlanUtils.java +++ b/core/src/main/java/org/opensearch/sql/calcite/utils/PlanUtils.java @@ -75,7 +75,6 @@ public interface PlanUtils { /** this is only for dedup command, do not reuse it in other command */ String ROW_NUMBER_COLUMN_FOR_DEDUP = "_row_number_dedup_"; - String ROW_NUMBER_COLUMN_FOR_JOIN_MAX_DEDUP = "_row_number_join_max_dedup_"; String ROW_NUMBER_COLUMN_FOR_RARE_TOP = "_row_number_rare_top_"; String ROW_NUMBER_COLUMN_FOR_MAIN = "_row_number_main_"; String ROW_NUMBER_COLUMN_FOR_SUBSEARCH = "_row_number_subsearch_"; @@ -463,13 +462,10 @@ static RexNode derefMapCall(RexNode rexNode) { return rexNode; } - /** Check if contains dedup */ + /** Check if contains dedup, it should be put in the last position */ static boolean containsRowNumberDedup(RelNode node) { - return node.getRowType().getFieldNames().stream() - .anyMatch( - name -> - name.equals(ROW_NUMBER_COLUMN_FOR_DEDUP) - || name.equals(ROW_NUMBER_COLUMN_FOR_JOIN_MAX_DEDUP)); + List fieldNames = node.getRowType().getFieldNames(); + return fieldNames.get(fieldNames.size() - 1).equals(ROW_NUMBER_COLUMN_FOR_DEDUP); } /** Check if contains dedup for top/rare */ diff --git a/core/src/main/java/org/opensearch/sql/calcite/utils/SubsearchUtils.java b/core/src/main/java/org/opensearch/sql/calcite/utils/SubsearchUtils.java index 221e4275f64..abc23b524a1 100644 --- a/core/src/main/java/org/opensearch/sql/calcite/utils/SubsearchUtils.java +++ b/core/src/main/java/org/opensearch/sql/calcite/utils/SubsearchUtils.java @@ -22,7 +22,7 @@ import org.apache.calcite.rex.RexUtil; import org.apache.commons.lang3.tuple.Pair; import org.opensearch.sql.calcite.CalcitePlanContext; -import org.opensearch.sql.calcite.plan.LogicalSystemLimit; +import org.opensearch.sql.calcite.plan.rel.LogicalSystemLimit; @UtilityClass public class SubsearchUtils { diff --git a/core/src/main/java/org/opensearch/sql/executor/QueryService.java b/core/src/main/java/org/opensearch/sql/executor/QueryService.java index 27539c0d46c..d40d8dcaa71 100644 --- a/core/src/main/java/org/opensearch/sql/executor/QueryService.java +++ b/core/src/main/java/org/opensearch/sql/executor/QueryService.java @@ -14,15 +14,11 @@ import lombok.extern.log4j.Log4j2; import org.apache.calcite.jdbc.CalciteSchema; import org.apache.calcite.plan.RelTraitDef; -import org.apache.calcite.plan.hep.HepPlanner; -import org.apache.calcite.plan.hep.HepProgram; -import org.apache.calcite.plan.hep.HepProgramBuilder; import org.apache.calcite.rel.RelCollation; import org.apache.calcite.rel.RelCollations; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.core.Sort; import org.apache.calcite.rel.logical.LogicalSort; -import org.apache.calcite.rel.rules.FilterMergeRule; import org.apache.calcite.schema.SchemaPlus; import org.apache.calcite.sql.parser.SqlParser; import org.apache.calcite.tools.FrameworkConfig; @@ -36,8 +32,8 @@ import org.opensearch.sql.calcite.CalciteRelNodeVisitor; import org.opensearch.sql.calcite.OpenSearchSchema; import org.opensearch.sql.calcite.SysLimit; -import org.opensearch.sql.calcite.plan.LogicalSystemLimit; -import org.opensearch.sql.calcite.plan.LogicalSystemLimit.SystemLimitType; +import org.opensearch.sql.calcite.plan.rel.LogicalSystemLimit; +import org.opensearch.sql.calcite.plan.rel.LogicalSystemLimit.SystemLimitType; import org.opensearch.sql.common.response.ResponseListener; import org.opensearch.sql.common.setting.Settings; import org.opensearch.sql.common.utils.QueryContext; @@ -59,9 +55,6 @@ @AllArgsConstructor @Log4j2 public class QueryService { - private static final HepProgram FILTER_MERGE_PROGRAM = - new HepProgramBuilder().addRuleInstance(FilterMergeRule.Config.DEFAULT.toRule()).build(); - private final Analyzer analyzer; private final ExecutionEngine executionEngine; private final Planner planner; @@ -111,9 +104,7 @@ public void executeWithCalcite( CalcitePlanContext.create( buildFrameworkConfig(), SysLimit.fromSettings(settings), queryType); RelNode relNode = analyze(plan, context); - relNode = mergeAdjacentFilters(relNode); - RelNode optimized = optimize(relNode, context); - RelNode calcitePlan = convertToCalcitePlan(optimized); + RelNode calcitePlan = convertToCalcitePlan(relNode, context); analyzeMetric.set(System.nanoTime() - analyzeStart); executionEngine.execute(calcitePlan, context, listener); } catch (Throwable t) { @@ -154,9 +145,7 @@ public void explainWithCalcite( context.run( () -> { RelNode relNode = analyze(plan, context); - relNode = mergeAdjacentFilters(relNode); - RelNode optimized = optimize(relNode, context); - RelNode calcitePlan = convertToCalcitePlan(optimized); + RelNode calcitePlan = convertToCalcitePlan(relNode, context); executionEngine.explain(calcitePlan, format, context, listener); }, settings); @@ -267,16 +256,6 @@ public RelNode analyze(UnresolvedPlan plan, CalcitePlanContext context) { return getRelNodeVisitor().analyze(plan, context); } - /** - * Run Calcite FILTER_MERGE once so adjacent filters created during analysis can collapse before - * the rest of optimization. - */ - private RelNode mergeAdjacentFilters(RelNode relNode) { - HepPlanner planner = new HepPlanner(FILTER_MERGE_PROGRAM); - planner.setRoot(relNode); - return planner.findBestExp(); - } - /** Analyze {@link UnresolvedPlan}. */ public LogicalPlan analyze(UnresolvedPlan plan, QueryType queryType) { return analyzer.analyze(plan, new AnalysisContext(queryType)); @@ -287,17 +266,6 @@ public PhysicalPlan plan(LogicalPlan plan) { return planner.plan(plan); } - /** - * Try to optimize the plan by appending a limit operator for QUERY_SIZE_LIMIT Don't add for - * `EXPLAIN` to avoid changing its output plan. - */ - public RelNode optimize(RelNode plan, CalcitePlanContext context) { - return LogicalSystemLimit.create( - SystemLimitType.QUERY_SIZE_LIMIT, - plan, - context.relBuilder.literal(context.sysLimit.querySizeLimit())); - } - private boolean isCalciteFallbackAllowed(@Nullable Throwable t) { // We always allow fallback the query failed with CalciteUnsupportedException. // This is for avoiding breaking changes when enable Calcite by default. @@ -351,9 +319,15 @@ private FrameworkConfig buildFrameworkConfig() { * are some differences in the topological structures or semantics between them. * * @param osPlan Logical Plan derived from OpenSearch PPL + * @param context Calcite context */ - private static RelNode convertToCalcitePlan(RelNode osPlan) { - RelNode calcitePlan = osPlan; + private static RelNode convertToCalcitePlan(RelNode osPlan, CalcitePlanContext context) { + // Explicitly add a limit operator to enforce query size limit + RelNode calcitePlan = + LogicalSystemLimit.create( + SystemLimitType.QUERY_SIZE_LIMIT, + osPlan, + context.relBuilder.literal(context.sysLimit.querySizeLimit())); /* Calcite only ensures collation of the final result produced from the root sort operator. * While we expect that the collation can be preserved through the pipes over PPL, we need to * explicitly add a sort operator on top of the original plan @@ -361,9 +335,9 @@ private static RelNode convertToCalcitePlan(RelNode osPlan) { * See logic in ${@link CalcitePrepareImpl} * For the redundant sort, we rely on Calcite optimizer to eliminate */ - RelCollation collation = osPlan.getTraitSet().getCollation(); - if (!(osPlan instanceof Sort) && collation != RelCollations.EMPTY) { - calcitePlan = LogicalSort.create(osPlan, collation, null, null); + RelCollation collation = calcitePlan.getTraitSet().getCollation(); + if (!(calcitePlan instanceof Sort) && collation != RelCollations.EMPTY) { + calcitePlan = LogicalSort.create(calcitePlan, collation, null, null); } return calcitePlan; } diff --git a/core/src/test/java/org/opensearch/sql/calcite/CalciteRexNodeVisitorTest.java b/core/src/test/java/org/opensearch/sql/calcite/CalciteRexNodeVisitorTest.java index 1ec8d005716..9be542f2086 100644 --- a/core/src/test/java/org/opensearch/sql/calcite/CalciteRexNodeVisitorTest.java +++ b/core/src/test/java/org/opensearch/sql/calcite/CalciteRexNodeVisitorTest.java @@ -17,7 +17,6 @@ import org.apache.calcite.sql.type.ArraySqlType; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.tools.FrameworkConfig; -import org.apache.calcite.tools.RelBuilder; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -29,6 +28,7 @@ import org.opensearch.sql.ast.expression.LambdaFunction; import org.opensearch.sql.ast.expression.QualifiedName; import org.opensearch.sql.calcite.utils.CalciteToolsHelper; +import org.opensearch.sql.calcite.utils.CalciteToolsHelper.OpenSearchRelBuilder; import org.opensearch.sql.datasource.DataSourceService; import org.opensearch.sql.executor.QueryType; @@ -52,7 +52,7 @@ public class CalciteRexNodeVisitorTest { @Mock static FrameworkConfig frameworkConfig; @Mock static Connection connection; - @Mock static RelBuilder relBuilder; + @Mock static OpenSearchRelBuilder relBuilder; @Mock static ExtendedRexBuilder rexBuilder; static CalcitePlanContext context; MockedStatic mockedStatic; diff --git a/core/src/test/java/org/opensearch/sql/calcite/PPLAggGroupMergeRuleTest.java b/core/src/test/java/org/opensearch/sql/calcite/PPLAggGroupMergeRuleTest.java index 4960d9c73b6..c2520bcc6cc 100644 --- a/core/src/test/java/org/opensearch/sql/calcite/PPLAggGroupMergeRuleTest.java +++ b/core/src/test/java/org/opensearch/sql/calcite/PPLAggGroupMergeRuleTest.java @@ -41,8 +41,8 @@ import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; -import org.opensearch.sql.calcite.plan.OpenSearchRules; -import org.opensearch.sql.calcite.plan.PPLAggGroupMergeRule; +import org.opensearch.sql.calcite.plan.rule.OpenSearchRules; +import org.opensearch.sql.calcite.plan.rule.PPLAggGroupMergeRule; import org.opensearch.sql.calcite.utils.CalciteToolsHelper.OpenSearchRelBuilder; @ExtendWith(MockitoExtension.class) diff --git a/core/src/test/java/org/opensearch/sql/calcite/PPLAggregateConvertRuleTest.java b/core/src/test/java/org/opensearch/sql/calcite/PPLAggregateConvertRuleTest.java index 1086fc5f0c1..e6461d10cbe 100644 --- a/core/src/test/java/org/opensearch/sql/calcite/PPLAggregateConvertRuleTest.java +++ b/core/src/test/java/org/opensearch/sql/calcite/PPLAggregateConvertRuleTest.java @@ -37,8 +37,8 @@ import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; -import org.opensearch.sql.calcite.plan.OpenSearchRules; -import org.opensearch.sql.calcite.plan.PPLAggregateConvertRule; +import org.opensearch.sql.calcite.plan.rule.OpenSearchRules; +import org.opensearch.sql.calcite.plan.rule.PPLAggregateConvertRule; import org.opensearch.sql.calcite.utils.CalciteToolsHelper.OpenSearchRelBuilder; @ExtendWith(MockitoExtension.class) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/access_struct_subfield_with_item.yaml b/integ-test/src/test/resources/expectedOutput/calcite/access_struct_subfield_with_item.yaml index a3726ad6126..42c39458bdf 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/access_struct_subfield_with_item.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/access_struct_subfield_with_item.yaml @@ -4,5 +4,5 @@ calcite: LogicalProject(host=[$0], info=[GEOIP('dummy-datasource':VARCHAR, $0)], info.dummy_sub_field=[ITEM(GEOIP('dummy-datasource':VARCHAR, $0), 'dummy_sub_field')]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_weblogs]]) physical: | - EnumerableCalc(expr#0=[{inputs}], expr#1=['dummy-datasource':VARCHAR], expr#2=[GEOIP($t1, $t0)], expr#3=['dummy_sub_field'], expr#4=[ITEM($t2, $t3)], host=[$t0], $f1=[$t2], $f2=[$t4]) + EnumerableCalc(expr#0=[{inputs}], expr#1=['dummy-datasource':VARCHAR], expr#2=[GEOIP($t1, $t0)], expr#3=['dummy_sub_field'], expr#4=[ITEM($t2, $t3)], host=[$t0], info=[$t2], info.dummy_sub_field=[$t4]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_weblogs]], PushDownContext=[[PROJECT->[host], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10000,"timeout":"1m","_source":{"includes":["host"],"excludes":[]}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/big5/composite_date_histogram_daily.yaml b/integ-test/src/test/resources/expectedOutput/calcite/big5/composite_date_histogram_daily.yaml index 1f3949c6ce3..b3918a1c22d 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/big5/composite_date_histogram_daily.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/big5/composite_date_histogram_daily.yaml @@ -5,7 +5,8 @@ calcite: LogicalProject(count()=[$1], span(`@timestamp`,1d)=[$0]) LogicalAggregate(group=[{0}], count()=[COUNT()]) LogicalProject(span(`@timestamp`,1d)=[SPAN($17, 1, 'd')]) - LogicalFilter(condition=[AND(>=($17, TIMESTAMP('2022-12-30 00:00:00':VARCHAR)), <($17, TIMESTAMP('2023-01-07 12:00:00':VARCHAR)))]) - CalciteLogicalIndexScan(table=[[OpenSearch, big5]]) + LogicalFilter(condition=[IS NOT NULL($17)]) + LogicalFilter(condition=[AND(>=($17, TIMESTAMP('2022-12-30 00:00:00':VARCHAR)), <($17, TIMESTAMP('2023-01-07 12:00:00':VARCHAR)))]) + CalciteLogicalIndexScan(table=[[OpenSearch, big5]]) physical: | CalciteEnumerableIndexScan(table=[[OpenSearch, big5]], PushDownContext=[[FILTER->SEARCH($0, Sarg[['2022-12-30 00:00:00':VARCHAR..'2023-01-07 12:00:00':VARCHAR)]:VARCHAR), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},count()=COUNT()), PROJECT->[count(), span(`@timestamp`,1d)], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"range":{"@timestamp":{"from":"2022-12-30T00:00:00.000Z","to":"2023-01-07T12:00:00.000Z","include_lower":true,"include_upper":false,"format":"date_time","boost":1.0}}},"aggregations":{"composite_buckets":{"composite":{"size":10,"sources":[{"span(`@timestamp`,1d)":{"date_histogram":{"field":"@timestamp","missing_bucket":false,"order":"asc","fixed_interval":"1d"}}}]}}}}, requestedTotalSize=10, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/big5/composite_terms.yaml b/integ-test/src/test/resources/expectedOutput/calcite/big5/composite_terms.yaml index 199988f7e3b..5957b6a4acf 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/big5/composite_terms.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/big5/composite_terms.yaml @@ -5,7 +5,8 @@ calcite: LogicalProject(count()=[$2], process.name=[$0], cloud.region=[$1]) LogicalAggregate(group=[{0, 1}], count()=[COUNT()]) LogicalProject(process.name=[$7], cloud.region=[$14]) - LogicalFilter(condition=[AND(>=($17, TIMESTAMP('2023-01-02 00:00:00':VARCHAR)), <($17, TIMESTAMP('2023-01-02 10:00:00':VARCHAR)), IS NOT NULL($7), IS NOT NULL($14))]) - CalciteLogicalIndexScan(table=[[OpenSearch, big5]]) + LogicalFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($14))]) + LogicalFilter(condition=[AND(>=($17, TIMESTAMP('2023-01-02 00:00:00':VARCHAR)), <($17, TIMESTAMP('2023-01-02 10:00:00':VARCHAR)))]) + CalciteLogicalIndexScan(table=[[OpenSearch, big5]]) physical: | CalciteEnumerableIndexScan(table=[[OpenSearch, big5]], PushDownContext=[[FILTER->AND(SEARCH($2, Sarg[['2023-01-02 00:00:00':VARCHAR..'2023-01-02 10:00:00':VARCHAR)]:VARCHAR), IS NOT NULL($0), IS NOT NULL($1)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},count()=COUNT()), PROJECT->[count(), process.name, cloud.region], SORT->[1 DESC LAST, 2 ASC FIRST], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"range":{"@timestamp":{"from":"2023-01-02T00:00:00.000Z","to":"2023-01-02T10:00:00.000Z","include_lower":true,"include_upper":false,"format":"date_time","boost":1.0}}},{"exists":{"field":"process.name","boost":1.0}},{"exists":{"field":"cloud.region","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"composite_buckets":{"composite":{"size":10,"sources":[{"process.name":{"terms":{"field":"process.name","missing_bucket":false,"order":"desc"}}},{"cloud.region":{"terms":{"field":"cloud.region","missing_bucket":false,"order":"asc"}}}]}}}}, requestedTotalSize=10, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/big5/composite_terms_keyword.yaml b/integ-test/src/test/resources/expectedOutput/calcite/big5/composite_terms_keyword.yaml index 9c34a7ecd56..eb9cf711f55 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/big5/composite_terms_keyword.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/big5/composite_terms_keyword.yaml @@ -5,7 +5,8 @@ calcite: LogicalProject(count()=[$3], process.name=[$0], cloud.region=[$1], aws.cloudwatch.log_stream=[$2]) LogicalAggregate(group=[{0, 1, 2}], count()=[COUNT()]) LogicalProject(process.name=[$7], cloud.region=[$14], aws.cloudwatch.log_stream=[$34]) - LogicalFilter(condition=[AND(>=($17, TIMESTAMP('2023-01-02 00:00:00':VARCHAR)), <($17, TIMESTAMP('2023-01-02 10:00:00':VARCHAR)), IS NOT NULL($7), IS NOT NULL($14), IS NOT NULL($34))]) - CalciteLogicalIndexScan(table=[[OpenSearch, big5]]) + LogicalFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($14), IS NOT NULL($34))]) + LogicalFilter(condition=[AND(>=($17, TIMESTAMP('2023-01-02 00:00:00':VARCHAR)), <($17, TIMESTAMP('2023-01-02 10:00:00':VARCHAR)))]) + CalciteLogicalIndexScan(table=[[OpenSearch, big5]]) physical: | CalciteEnumerableIndexScan(table=[[OpenSearch, big5]], PushDownContext=[[FILTER->AND(SEARCH($2, Sarg[['2023-01-02 00:00:00':VARCHAR..'2023-01-02 10:00:00':VARCHAR)]:VARCHAR), IS NOT NULL($0), IS NOT NULL($1), IS NOT NULL($3)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1, 3},count()=COUNT()), PROJECT->[count(), process.name, cloud.region, aws.cloudwatch.log_stream], SORT->[1 DESC LAST, 2 ASC FIRST, 3 ASC FIRST], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"range":{"@timestamp":{"from":"2023-01-02T00:00:00.000Z","to":"2023-01-02T10:00:00.000Z","include_lower":true,"include_upper":false,"format":"date_time","boost":1.0}}},{"exists":{"field":"process.name","boost":1.0}},{"exists":{"field":"cloud.region","boost":1.0}},{"exists":{"field":"aws.cloudwatch.log_stream","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"composite_buckets":{"composite":{"size":10,"sources":[{"process.name":{"terms":{"field":"process.name","missing_bucket":false,"order":"desc"}}},{"cloud.region":{"terms":{"field":"cloud.region","missing_bucket":false,"order":"asc"}}},{"aws.cloudwatch.log_stream":{"terms":{"field":"aws.cloudwatch.log_stream","missing_bucket":false,"order":"asc"}}}]}}}}, requestedTotalSize=10, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/big5/date_histogram_minute_agg.yaml b/integ-test/src/test/resources/expectedOutput/calcite/big5/date_histogram_minute_agg.yaml index 41303581efc..f1f6208d9c3 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/big5/date_histogram_minute_agg.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/big5/date_histogram_minute_agg.yaml @@ -4,7 +4,8 @@ calcite: LogicalProject(count()=[$1], span(`@timestamp`,1m)=[$0]) LogicalAggregate(group=[{0}], count()=[COUNT()]) LogicalProject(span(`@timestamp`,1m)=[SPAN($17, 1, 'm')]) - LogicalFilter(condition=[AND(>=($17, TIMESTAMP('2023-01-01 00:00:00':VARCHAR)), <($17, TIMESTAMP('2023-01-03 00:00:00':VARCHAR)))]) - CalciteLogicalIndexScan(table=[[OpenSearch, big5]]) + LogicalFilter(condition=[IS NOT NULL($17)]) + LogicalFilter(condition=[AND(>=($17, TIMESTAMP('2023-01-01 00:00:00':VARCHAR)), <($17, TIMESTAMP('2023-01-03 00:00:00':VARCHAR)))]) + CalciteLogicalIndexScan(table=[[OpenSearch, big5]]) physical: | CalciteEnumerableIndexScan(table=[[OpenSearch, big5]], PushDownContext=[[FILTER->SEARCH($0, Sarg[['2023-01-01 00:00:00':VARCHAR..'2023-01-03 00:00:00':VARCHAR)]:VARCHAR), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},count()=COUNT()), PROJECT->[count(), span(`@timestamp`,1m)], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"range":{"@timestamp":{"from":"2023-01-01T00:00:00.000Z","to":"2023-01-03T00:00:00.000Z","include_lower":true,"include_upper":false,"format":"date_time","boost":1.0}}},"aggregations":{"composite_buckets":{"composite":{"size":10000,"sources":[{"span(`@timestamp`,1m)":{"date_histogram":{"field":"@timestamp","missing_bucket":false,"order":"asc","fixed_interval":"1m"}}}]}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/big5/dedup_metrics_size_field.yaml b/integ-test/src/test/resources/expectedOutput/calcite/big5/dedup_metrics_size_field.yaml index 4387e72fd96..4bc2da21669 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/big5/dedup_metrics_size_field.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/big5/dedup_metrics_size_field.yaml @@ -9,5 +9,6 @@ calcite: LogicalFilter(condition=[IS NOT NULL($28)]) CalciteLogicalIndexScan(table=[[OpenSearch, big5]]) physical: | - CalciteEnumerableTopK(sort0=[$7], dir0=[DESC-nulls-last], fetch=[10000]) - CalciteEnumerableIndexScan(table=[[OpenSearch, big5]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0},agg#0=LITERAL_AGG(1))], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":10000,"sources":[{"metrics.size":{"terms":{"field":"metrics.size","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f1":{"top_hits":{"from":0,"size":1,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["metrics.size","agent","process","log","message","tags","cloud","input","@timestamp","ecs","data_stream","meta","host","metrics","aws","event"],"excludes":[]},"script_fields":{}}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file + EnumerableCalc(expr#0..15=[{inputs}], proj#0..12=[{exprs}], aws=[$t14], event=[$t15]) + CalciteEnumerableTopK(sort0=[$7], dir0=[DESC-nulls-last], fetch=[10000]) + CalciteEnumerableIndexScan(table=[[OpenSearch, big5]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0},agg#0=LITERAL_AGG(1))], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":10000,"sources":[{"metrics.size":{"terms":{"field":"metrics.size","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f1":{"top_hits":{"from":0,"size":1,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["metrics.size","agent","process","log","message","tags","cloud","input","@timestamp","ecs","data_stream","meta","host","metrics","aws","event"],"excludes":[]},"script_fields":{}}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/big5/keyword_in_range.yaml b/integ-test/src/test/resources/expectedOutput/calcite/big5/keyword_in_range.yaml index e1f0873de61..c7db35fbaa1 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/big5/keyword_in_range.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/big5/keyword_in_range.yaml @@ -3,7 +3,8 @@ calcite: LogicalSystemLimit(fetch=[10000], type=[QUERY_SIZE_LIMIT]) LogicalProject(agent=[$0], process=[$6], log=[$8], message=[$11], tags=[$12], cloud=[$13], input=[$15], @timestamp=[$17], ecs=[$18], data_stream=[$20], meta=[$24], host=[$26], metrics=[$27], aws=[$30], event=[$35]) LogicalSort(fetch=[10]) - LogicalFilter(condition=[AND(query_string(MAP('query', 'process.name:kernel':VARCHAR)), >=($17, TIMESTAMP('2023-01-01 00:00:00':VARCHAR)), <($17, TIMESTAMP('2023-01-03 00:00:00':VARCHAR)))]) - CalciteLogicalIndexScan(table=[[OpenSearch, big5]]) + LogicalFilter(condition=[AND(>=($17, TIMESTAMP('2023-01-01 00:00:00':VARCHAR)), <($17, TIMESTAMP('2023-01-03 00:00:00':VARCHAR)))]) + LogicalFilter(condition=[query_string(MAP('query', 'process.name:kernel':VARCHAR))]) + CalciteLogicalIndexScan(table=[[OpenSearch, big5]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, big5]], PushDownContext=[[PROJECT->[agent, process, log, message, tags, cloud, input, @timestamp, ecs, data_stream, meta, host, metrics, aws, event], FILTER->AND(query_string(MAP('query', 'process.name:kernel':VARCHAR)), SEARCH($7, Sarg[['2023-01-01 00:00:00':VARCHAR..'2023-01-03 00:00:00':VARCHAR)]:VARCHAR)), LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10,"timeout":"1m","query":{"bool":{"must":[{"query_string":{"query":"process.name:kernel","fields":[],"type":"best_fields","default_operator":"or","max_determinized_states":10000,"enable_position_increments":true,"fuzziness":"AUTO","fuzzy_prefix_length":0,"fuzzy_max_expansions":50,"phrase_slop":0,"escape":false,"auto_generate_synonyms_phrase_query":true,"fuzzy_transpositions":true,"boost":1.0}},{"range":{"@timestamp":{"from":"2023-01-01T00:00:00.000Z","to":"2023-01-03T00:00:00.000Z","include_lower":true,"include_upper":false,"format":"date_time","boost":1.0}}}],"adjust_pure_negative":true,"boost":1.0}},"_source":{"includes":["agent","process","log","message","tags","cloud","input","@timestamp","ecs","data_stream","meta","host","metrics","aws","event"],"excludes":[]}}, requestedTotalSize=10, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, big5]], PushDownContext=[[PROJECT->[agent, process, log, message, tags, cloud, input, @timestamp, ecs, data_stream, meta, host, metrics, aws, event], FILTER->AND(query_string(MAP('query', 'process.name:kernel':VARCHAR)), SEARCH($7, Sarg[['2023-01-01 00:00:00':VARCHAR..'2023-01-03 00:00:00':VARCHAR)]:VARCHAR)), LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10,"timeout":"1m","query":{"bool":{"must":[{"query_string":{"query":"process.name:kernel","fields":[],"type":"best_fields","default_operator":"or","max_determinized_states":10000,"enable_position_increments":true,"fuzziness":"AUTO","fuzzy_prefix_length":0,"fuzzy_max_expansions":50,"phrase_slop":0,"escape":false,"auto_generate_synonyms_phrase_query":true,"fuzzy_transpositions":true,"boost":1.0}},{"range":{"@timestamp":{"from":"2023-01-01T00:00:00.000Z","to":"2023-01-03T00:00:00.000Z","include_lower":true,"include_upper":false,"format":"date_time","boost":1.0}}}],"adjust_pure_negative":true,"boost":1.0}},"_source":{"includes":["agent","process","log","message","tags","cloud","input","@timestamp","ecs","data_stream","meta","host","metrics","aws","event"],"excludes":[]}}, requestedTotalSize=10, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/big5/multi_terms_keyword.yaml b/integ-test/src/test/resources/expectedOutput/calcite/big5/multi_terms_keyword.yaml index b01761d7dee..b0c896f61fb 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/big5/multi_terms_keyword.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/big5/multi_terms_keyword.yaml @@ -5,7 +5,8 @@ calcite: LogicalProject(count()=[$2], process.name=[$0], cloud.region=[$1]) LogicalAggregate(group=[{0, 1}], count()=[COUNT()]) LogicalProject(process.name=[$7], cloud.region=[$14]) - LogicalFilter(condition=[AND(>=($17, TIMESTAMP('2023-01-05 00:00:00':VARCHAR)), <($17, TIMESTAMP('2023-01-05 05:00:00':VARCHAR)), IS NOT NULL($7), IS NOT NULL($14))]) - CalciteLogicalIndexScan(table=[[OpenSearch, big5]]) + LogicalFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($14))]) + LogicalFilter(condition=[AND(>=($17, TIMESTAMP('2023-01-05 00:00:00':VARCHAR)), <($17, TIMESTAMP('2023-01-05 05:00:00':VARCHAR)))]) + CalciteLogicalIndexScan(table=[[OpenSearch, big5]]) physical: | CalciteEnumerableIndexScan(table=[[OpenSearch, big5]], PushDownContext=[[FILTER->AND(SEARCH($2, Sarg[['2023-01-05 00:00:00':VARCHAR..'2023-01-05 05:00:00':VARCHAR)]:VARCHAR), IS NOT NULL($0), IS NOT NULL($1)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},count()=COUNT()), SORT_AGG_METRICS->[2 DESC LAST], PROJECT->[count(), process.name, cloud.region], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"range":{"@timestamp":{"from":"2023-01-05T00:00:00.000Z","to":"2023-01-05T05:00:00.000Z","include_lower":true,"include_upper":false,"format":"date_time","boost":1.0}}},{"exists":{"field":"process.name","boost":1.0}},{"exists":{"field":"cloud.region","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"process.name|cloud.region":{"multi_terms":{"terms":[{"field":"process.name"},{"field":"cloud.region"}],"size":10,"min_doc_count":1,"shard_min_doc_count":0,"show_term_doc_count_error":false,"order":[{"_count":"desc"},{"_key":"asc"}]}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q11.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q11.yaml index af6c337fbbb..7a8da847554 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q11.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q11.yaml @@ -5,7 +5,8 @@ calcite: LogicalProject(u=[$1], MobilePhoneModel=[$0]) LogicalAggregate(group=[{0}], u=[COUNT(DISTINCT $1)]) LogicalProject(MobilePhoneModel=[$31], UserID=[$84]) - LogicalFilter(condition=[<>($31, '')]) - CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) + LogicalFilter(condition=[IS NOT NULL($31)]) + LogicalFilter(condition=[<>($31, '')]) + CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER-><>($0, ''), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},u=COUNT(DISTINCT $1)), SORT_AGG_METRICS->[1 DESC LAST], PROJECT->[u, MobilePhoneModel], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"exists":{"field":"MobilePhoneModel","boost":1.0}}],"must_not":[{"term":{"MobilePhoneModel":{"value":"","boost":1.0}}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"MobilePhoneModel":{"terms":{"field":"MobilePhoneModel","size":10,"min_doc_count":1,"shard_min_doc_count":0,"show_term_doc_count_error":false,"order":[{"u":"desc"},{"_key":"asc"}]},"aggregations":{"u":{"cardinality":{"field":"UserID"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q12.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q12.yaml index 9e72d78f9cb..be24923eeea 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q12.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q12.yaml @@ -5,7 +5,8 @@ calcite: LogicalProject(u=[$2], MobilePhone=[$0], MobilePhoneModel=[$1]) LogicalAggregate(group=[{0, 1}], u=[COUNT(DISTINCT $2)]) LogicalProject(MobilePhone=[$62], MobilePhoneModel=[$31], UserID=[$84]) - LogicalFilter(condition=[AND(<>($31, ''), IS NOT NULL($62))]) - CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) + LogicalFilter(condition=[AND(IS NOT NULL($62), IS NOT NULL($31))]) + LogicalFilter(condition=[<>($31, '')]) + CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER->AND(<>($0, ''), IS NOT NULL($1)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},u=COUNT(DISTINCT $2)), SORT_AGG_METRICS->[2 DESC LAST], PROJECT->[u, MobilePhone, MobilePhoneModel], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"bool":{"must":[{"exists":{"field":"MobilePhoneModel","boost":1.0}}],"must_not":[{"term":{"MobilePhoneModel":{"value":"","boost":1.0}}}],"adjust_pure_negative":true,"boost":1.0}},{"exists":{"field":"MobilePhone","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"MobilePhoneModel|MobilePhone":{"multi_terms":{"terms":[{"field":"MobilePhoneModel"},{"field":"MobilePhone"}],"size":10,"min_doc_count":1,"shard_min_doc_count":0,"show_term_doc_count_error":false,"order":[{"u":"desc"},{"_key":"asc"}]},"aggregations":{"u":{"cardinality":{"field":"UserID"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q13.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q13.yaml index eb0def79d01..dd4f502bbde 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q13.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q13.yaml @@ -5,7 +5,8 @@ calcite: LogicalProject(c=[$1], SearchPhrase=[$0]) LogicalAggregate(group=[{0}], c=[COUNT()]) LogicalProject(SearchPhrase=[$63]) - LogicalFilter(condition=[<>($63, '')]) - CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) + LogicalFilter(condition=[IS NOT NULL($63)]) + LogicalFilter(condition=[<>($63, '')]) + CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER-><>($0, ''), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},c=COUNT()), SORT_AGG_METRICS->[1 DESC LAST], PROJECT->[c, SearchPhrase], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"exists":{"field":"SearchPhrase","boost":1.0}}],"must_not":[{"term":{"SearchPhrase":{"value":"","boost":1.0}}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"SearchPhrase":{"terms":{"field":"SearchPhrase","size":10,"min_doc_count":1,"shard_min_doc_count":0,"show_term_doc_count_error":false,"order":[{"_count":"desc"},{"_key":"asc"}]}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q14.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q14.yaml index 18f0015a42f..d5c34e6a7f2 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q14.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q14.yaml @@ -5,7 +5,8 @@ calcite: LogicalProject(u=[$1], SearchPhrase=[$0]) LogicalAggregate(group=[{0}], u=[COUNT(DISTINCT $1)]) LogicalProject(SearchPhrase=[$63], UserID=[$84]) - LogicalFilter(condition=[<>($63, '')]) - CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) + LogicalFilter(condition=[IS NOT NULL($63)]) + LogicalFilter(condition=[<>($63, '')]) + CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER-><>($0, ''), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},u=COUNT(DISTINCT $1)), SORT_AGG_METRICS->[1 DESC LAST], PROJECT->[u, SearchPhrase], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"exists":{"field":"SearchPhrase","boost":1.0}}],"must_not":[{"term":{"SearchPhrase":{"value":"","boost":1.0}}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"SearchPhrase":{"terms":{"field":"SearchPhrase","size":10,"min_doc_count":1,"shard_min_doc_count":0,"show_term_doc_count_error":false,"order":[{"u":"desc"},{"_key":"asc"}]},"aggregations":{"u":{"cardinality":{"field":"UserID"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q15.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q15.yaml index a69a320e81a..b13cc1a62ca 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q15.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q15.yaml @@ -5,7 +5,8 @@ calcite: LogicalProject(c=[$2], SearchEngineID=[$0], SearchPhrase=[$1]) LogicalAggregate(group=[{0, 1}], c=[COUNT()]) LogicalProject(SearchEngineID=[$65], SearchPhrase=[$63]) - LogicalFilter(condition=[AND(<>($63, ''), IS NOT NULL($65))]) - CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) + LogicalFilter(condition=[AND(IS NOT NULL($65), IS NOT NULL($63))]) + LogicalFilter(condition=[<>($63, '')]) + CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER->AND(<>($0, ''), IS NOT NULL($1)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},c=COUNT()), SORT_AGG_METRICS->[2 DESC LAST], PROJECT->[c, SearchEngineID, SearchPhrase], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"bool":{"must":[{"exists":{"field":"SearchPhrase","boost":1.0}}],"must_not":[{"term":{"SearchPhrase":{"value":"","boost":1.0}}}],"adjust_pure_negative":true,"boost":1.0}},{"exists":{"field":"SearchEngineID","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"SearchPhrase|SearchEngineID":{"multi_terms":{"terms":[{"field":"SearchPhrase"},{"field":"SearchEngineID"}],"size":10,"min_doc_count":1,"shard_min_doc_count":0,"show_term_doc_count_error":false,"order":[{"_count":"desc"},{"_key":"asc"}]}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q22.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q22.yaml index cb0630aaba2..a28945e87d0 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q22.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q22.yaml @@ -5,7 +5,8 @@ calcite: LogicalProject(c=[$1], SearchPhrase=[$0]) LogicalAggregate(group=[{0}], c=[COUNT()]) LogicalProject(SearchPhrase=[$63]) - LogicalFilter(condition=[AND(LIKE($26, '%google%', '\'), <>($63, ''))]) - CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) + LogicalFilter(condition=[IS NOT NULL($63)]) + LogicalFilter(condition=[AND(LIKE($26, '%google%', '\'), <>($63, ''))]) + CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER->AND(LIKE($0, '%google%', '\'), <>($1, '')), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={1},c=COUNT()), SORT_AGG_METRICS->[1 DESC LAST], PROJECT->[c, SearchPhrase], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"wildcard":{"URL":{"wildcard":"*google*","boost":1.0}}},{"bool":{"must":[{"exists":{"field":"SearchPhrase","boost":1.0}}],"must_not":[{"term":{"SearchPhrase":{"value":"","boost":1.0}}}],"adjust_pure_negative":true,"boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"SearchPhrase":{"terms":{"field":"SearchPhrase","size":10,"min_doc_count":1,"shard_min_doc_count":0,"show_term_doc_count_error":false,"order":[{"_count":"desc"},{"_key":"asc"}]}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q23.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q23.yaml index 252361aee86..f5b8ec9c184 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q23.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q23.yaml @@ -5,7 +5,8 @@ calcite: LogicalProject(c=[$1], dc(UserID)=[$2], SearchPhrase=[$0]) LogicalAggregate(group=[{0}], c=[COUNT()], dc(UserID)=[COUNT(DISTINCT $1)]) LogicalProject(SearchPhrase=[$63], UserID=[$84]) - LogicalFilter(condition=[AND(LIKE($97, '%Google%', '\'), <>($63, ''), NOT(LIKE($26, '%.google.%', '\')))]) - CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) + LogicalFilter(condition=[IS NOT NULL($63)]) + LogicalFilter(condition=[AND(LIKE($97, '%Google%', '\'), <>($63, ''), NOT(LIKE($26, '%.google.%', '\')))]) + CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER->AND(LIKE($3, '%Google%', '\'), <>($1, ''), NOT(LIKE($0, '%.google.%', '\'))), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={1},c=COUNT(),dc(UserID)=COUNT(DISTINCT $2)), SORT_AGG_METRICS->[1 DESC LAST], PROJECT->[c, dc(UserID), SearchPhrase], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"wildcard":{"Title":{"wildcard":"*Google*","boost":1.0}}},{"bool":{"must":[{"exists":{"field":"SearchPhrase","boost":1.0}}],"must_not":[{"term":{"SearchPhrase":{"value":"","boost":1.0}}}],"adjust_pure_negative":true,"boost":1.0}},{"bool":{"must_not":[{"wildcard":{"URL":{"wildcard":"*.google.*","boost":1.0}}}],"adjust_pure_negative":true,"boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"SearchPhrase":{"terms":{"field":"SearchPhrase","size":10,"min_doc_count":1,"shard_min_doc_count":0,"show_term_doc_count_error":false,"order":[{"c":"desc"},{"_key":"asc"}]},"aggregations":{"dc(UserID)":{"cardinality":{"field":"UserID"}},"c":{"value_count":{"field":"_index"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q28.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q28.yaml index 3b02510db8b..4e5b41348cd 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q28.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q28.yaml @@ -6,10 +6,11 @@ calcite: LogicalProject(l=[$1], c=[$2], CounterID=[$0]) LogicalAggregate(group=[{0}], l=[AVG($1)], c=[COUNT()]) LogicalProject(CounterID=[$103], $f2=[CHAR_LENGTH($26)]) - LogicalFilter(condition=[AND(<>($26, ''), IS NOT NULL($103))]) - CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) + LogicalFilter(condition=[IS NOT NULL($103)]) + LogicalFilter(condition=[<>($26, '')]) + CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | EnumerableLimit(fetch=[10000]) CalciteEnumerableTopK(sort0=[$0], dir0=[DESC-nulls-last], fetch=[25]) EnumerableCalc(expr#0..2=[{inputs}], expr#3=[100000], expr#4=[>($t1, $t3)], proj#0..2=[{exprs}], $condition=[$t4]) - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER->AND(<>($0, ''), IS NOT NULL($1)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},l=AVG($1),c=COUNT()), PROJECT->[l, c, CounterID]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"bool":{"must":[{"exists":{"field":"URL","boost":1.0}}],"must_not":[{"term":{"URL":{"value":"","boost":1.0}}}],"adjust_pure_negative":true,"boost":1.0}},{"exists":{"field":"CounterID","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"composite_buckets":{"composite":{"size":10000,"sources":[{"CounterID":{"terms":{"field":"CounterID","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"l":{"avg":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/3sKICAib3AiOiB7CiAgICAibmFtZSI6ICJDSEFSX0xFTkdUSCIsCiAgICAia2luZCI6ICJDSEFSX0xFTkdUSCIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["URL"]}}}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER->AND(<>($0, ''), IS NOT NULL($1)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},l=AVG($1),c=COUNT()), PROJECT->[l, c, CounterID]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"bool":{"must":[{"exists":{"field":"URL","boost":1.0}}],"must_not":[{"term":{"URL":{"value":"","boost":1.0}}}],"adjust_pure_negative":true,"boost":1.0}},{"exists":{"field":"CounterID","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"composite_buckets":{"composite":{"size":10000,"sources":[{"CounterID":{"terms":{"field":"CounterID","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"l":{"avg":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/3sKICAib3AiOiB7CiAgICAibmFtZSI6ICJDSEFSX0xFTkdUSCIsCiAgICAia2luZCI6ICJDSEFSX0xFTkdUSCIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["URL"]}}}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q31.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q31.yaml index 460a3b8b582..a8ac7eaf9b7 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q31.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q31.yaml @@ -5,7 +5,8 @@ calcite: LogicalProject(c=[$2], sum(IsRefresh)=[$3], avg(ResolutionWidth)=[$4], SearchEngineID=[$0], ClientIP=[$1]) LogicalAggregate(group=[{0, 1}], c=[COUNT()], sum(IsRefresh)=[SUM($2)], avg(ResolutionWidth)=[AVG($3)]) LogicalProject(SearchEngineID=[$65], ClientIP=[$76], IsRefresh=[$72], ResolutionWidth=[$80]) - LogicalFilter(condition=[AND(<>($63, ''), IS NOT NULL($65), IS NOT NULL($76))]) - CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) + LogicalFilter(condition=[AND(IS NOT NULL($65), IS NOT NULL($76))]) + LogicalFilter(condition=[<>($63, '')]) + CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER->AND(<>($0, ''), IS NOT NULL($1), IS NOT NULL($3)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={1, 3},c=COUNT(),sum(IsRefresh)=SUM($2),avg(ResolutionWidth)=AVG($4)), SORT_AGG_METRICS->[2 DESC LAST], PROJECT->[c, sum(IsRefresh), avg(ResolutionWidth), SearchEngineID, ClientIP], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"bool":{"must":[{"exists":{"field":"SearchPhrase","boost":1.0}}],"must_not":[{"term":{"SearchPhrase":{"value":"","boost":1.0}}}],"adjust_pure_negative":true,"boost":1.0}},{"exists":{"field":"SearchEngineID","boost":1.0}},{"exists":{"field":"ClientIP","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"SearchEngineID|ClientIP":{"multi_terms":{"terms":[{"field":"SearchEngineID"},{"field":"ClientIP"}],"size":10,"min_doc_count":1,"shard_min_doc_count":0,"show_term_doc_count_error":false,"order":[{"c":"desc"},{"_key":"asc"}]},"aggregations":{"sum(IsRefresh)":{"sum":{"field":"IsRefresh"}},"avg(ResolutionWidth)":{"avg":{"field":"ResolutionWidth"}},"c":{"value_count":{"field":"_index"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q32.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q32.yaml index 9028b51a6d5..5cf8f54b258 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q32.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q32.yaml @@ -5,7 +5,8 @@ calcite: LogicalProject(c=[$2], sum(IsRefresh)=[$3], avg(ResolutionWidth)=[$4], WatchID=[$0], ClientIP=[$1]) LogicalAggregate(group=[{0, 1}], c=[COUNT()], sum(IsRefresh)=[SUM($2)], avg(ResolutionWidth)=[AVG($3)]) LogicalProject(WatchID=[$41], ClientIP=[$76], IsRefresh=[$72], ResolutionWidth=[$80]) - LogicalFilter(condition=[AND(<>($63, ''), IS NOT NULL($41), IS NOT NULL($76))]) - CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) + LogicalFilter(condition=[AND(IS NOT NULL($41), IS NOT NULL($76))]) + LogicalFilter(condition=[<>($63, '')]) + CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER->AND(<>($1, ''), IS NOT NULL($0), IS NOT NULL($3)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 3},c=COUNT(),sum(IsRefresh)=SUM($2),avg(ResolutionWidth)=AVG($4)), SORT_AGG_METRICS->[2 DESC LAST], PROJECT->[c, sum(IsRefresh), avg(ResolutionWidth), WatchID, ClientIP], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"bool":{"must":[{"exists":{"field":"SearchPhrase","boost":1.0}}],"must_not":[{"term":{"SearchPhrase":{"value":"","boost":1.0}}}],"adjust_pure_negative":true,"boost":1.0}},{"exists":{"field":"WatchID","boost":1.0}},{"exists":{"field":"ClientIP","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"WatchID|ClientIP":{"multi_terms":{"terms":[{"field":"WatchID"},{"field":"ClientIP"}],"size":10,"min_doc_count":1,"shard_min_doc_count":0,"show_term_doc_count_error":false,"order":[{"c":"desc"},{"_key":"asc"}]},"aggregations":{"sum(IsRefresh)":{"sum":{"field":"IsRefresh"}},"avg(ResolutionWidth)":{"avg":{"field":"ResolutionWidth"}},"c":{"value_count":{"field":"_index"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q37.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q37.yaml index 5cf2fb92261..44a4218baf5 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q37.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q37.yaml @@ -5,7 +5,8 @@ calcite: LogicalProject(PageViews=[$1], URL=[$0]) LogicalAggregate(group=[{0}], PageViews=[COUNT()]) LogicalProject(URL=[$26]) - LogicalFilter(condition=[AND(=($103, 62), >=($0, TIMESTAMP('2013-07-01 00:00:00':VARCHAR)), <=($0, TIMESTAMP('2013-07-31 00:00:00':VARCHAR)), =($42, 0), =($72, 0), <>($26, ''))]) - CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) + LogicalFilter(condition=[IS NOT NULL($26)]) + LogicalFilter(condition=[AND(=($103, 62), >=($0, TIMESTAMP('2013-07-01 00:00:00':VARCHAR)), <=($0, TIMESTAMP('2013-07-31 00:00:00':VARCHAR)), =($42, 0), =($72, 0), <>($26, ''))]) + CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER->AND(=($4, 62), SEARCH($0, Sarg[['2013-07-01 00:00:00':VARCHAR..'2013-07-31 00:00:00':VARCHAR]]:VARCHAR), =($2, 0), =($3, 0), <>($1, '')), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={1},PageViews=COUNT()), SORT_AGG_METRICS->[1 DESC LAST], PROJECT->[PageViews, URL], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"term":{"CounterID":{"value":62,"boost":1.0}}},{"range":{"EventDate":{"from":"2013-07-01T00:00:00.000Z","to":"2013-07-31T00:00:00.000Z","include_lower":true,"include_upper":true,"format":"date_time","boost":1.0}}},{"term":{"DontCountHits":{"value":0,"boost":1.0}}},{"term":{"IsRefresh":{"value":0,"boost":1.0}}},{"bool":{"must":[{"exists":{"field":"URL","boost":1.0}}],"must_not":[{"term":{"URL":{"value":"","boost":1.0}}}],"adjust_pure_negative":true,"boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"URL":{"terms":{"field":"URL","size":10,"min_doc_count":1,"shard_min_doc_count":0,"show_term_doc_count_error":false,"order":[{"_count":"desc"},{"_key":"asc"}]}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q38.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q38.yaml index 5ae72b31e00..cd15e03f941 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q38.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q38.yaml @@ -5,7 +5,8 @@ calcite: LogicalProject(PageViews=[$1], Title=[$0]) LogicalAggregate(group=[{0}], PageViews=[COUNT()]) LogicalProject(Title=[$97]) - LogicalFilter(condition=[AND(=($103, 62), >=($0, TIMESTAMP('2013-07-01 00:00:00':VARCHAR)), <=($0, TIMESTAMP('2013-07-31 00:00:00':VARCHAR)), =($42, 0), =($72, 0), <>($97, ''))]) - CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) + LogicalFilter(condition=[IS NOT NULL($97)]) + LogicalFilter(condition=[AND(=($103, 62), >=($0, TIMESTAMP('2013-07-01 00:00:00':VARCHAR)), <=($0, TIMESTAMP('2013-07-31 00:00:00':VARCHAR)), =($42, 0), =($72, 0), <>($97, ''))]) + CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER->AND(=($4, 62), SEARCH($0, Sarg[['2013-07-01 00:00:00':VARCHAR..'2013-07-31 00:00:00':VARCHAR]]:VARCHAR), =($1, 0), =($2, 0), <>($3, '')), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={3},PageViews=COUNT()), SORT_AGG_METRICS->[1 DESC LAST], PROJECT->[PageViews, Title], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"term":{"CounterID":{"value":62,"boost":1.0}}},{"range":{"EventDate":{"from":"2013-07-01T00:00:00.000Z","to":"2013-07-31T00:00:00.000Z","include_lower":true,"include_upper":true,"format":"date_time","boost":1.0}}},{"term":{"DontCountHits":{"value":0,"boost":1.0}}},{"term":{"IsRefresh":{"value":0,"boost":1.0}}},{"bool":{"must":[{"exists":{"field":"Title","boost":1.0}}],"must_not":[{"term":{"Title":{"value":"","boost":1.0}}}],"adjust_pure_negative":true,"boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"Title":{"terms":{"field":"Title","size":10,"min_doc_count":1,"shard_min_doc_count":0,"show_term_doc_count_error":false,"order":[{"_count":"desc"},{"_key":"asc"}]}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q39.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q39.yaml index 6f08007b702..c4bc303bfb2 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q39.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q39.yaml @@ -5,8 +5,9 @@ calcite: LogicalProject(PageViews=[$1], URL=[$0]) LogicalAggregate(group=[{0}], PageViews=[COUNT()]) LogicalProject(URL=[$26]) - LogicalFilter(condition=[AND(=($103, 62), >=($0, TIMESTAMP('2013-07-01 00:00:00':VARCHAR)), <=($0, TIMESTAMP('2013-07-31 00:00:00':VARCHAR)), =($72, 0), <>($49, 0), =($35, 0), IS NOT NULL($26))]) - CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) + LogicalFilter(condition=[IS NOT NULL($26)]) + LogicalFilter(condition=[AND(=($103, 62), >=($0, TIMESTAMP('2013-07-01 00:00:00':VARCHAR)), <=($0, TIMESTAMP('2013-07-31 00:00:00':VARCHAR)), =($72, 0), <>($49, 0), =($35, 0))]) + CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | EnumerableLimit(fetch=[10000]) EnumerableLimit(offset=[1000], fetch=[10]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q41.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q41.yaml index dc908d4df76..c23839c1674 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q41.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q41.yaml @@ -5,8 +5,9 @@ calcite: LogicalProject(PageViews=[$2], URLHash=[$0], EventDate=[$1]) LogicalAggregate(group=[{0, 1}], PageViews=[COUNT()]) LogicalProject(URLHash=[$25], EventDate=[$0]) - LogicalFilter(condition=[AND(=($103, 62), >=($0, TIMESTAMP('2013-07-01 00:00:00':VARCHAR)), <=($0, TIMESTAMP('2013-07-31 00:00:00':VARCHAR)), =($72, 0), SEARCH($12, Sarg[-1, 6]), =($11, 3594120000172545465), IS NOT NULL($25))]) - CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) + LogicalFilter(condition=[AND(IS NOT NULL($25), IS NOT NULL($0))]) + LogicalFilter(condition=[AND(=($103, 62), >=($0, TIMESTAMP('2013-07-01 00:00:00':VARCHAR)), <=($0, TIMESTAMP('2013-07-31 00:00:00':VARCHAR)), =($72, 0), SEARCH($12, Sarg[-1, 6]), =($11, 3594120000172545465))]) + CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | EnumerableLimit(fetch=[10000]) EnumerableLimit(offset=[100], fetch=[10]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q42.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q42.yaml index 3b0bf781c00..7a7d97c857a 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q42.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q42.yaml @@ -5,8 +5,9 @@ calcite: LogicalProject(PageViews=[$2], WindowClientWidth=[$0], WindowClientHeight=[$1]) LogicalAggregate(group=[{0, 1}], PageViews=[COUNT()]) LogicalProject(WindowClientWidth=[$104], WindowClientHeight=[$57]) - LogicalFilter(condition=[AND(=($103, 62), >=($0, TIMESTAMP('2013-07-01 00:00:00':VARCHAR)), <=($0, TIMESTAMP('2013-07-31 00:00:00':VARCHAR)), =($72, 0), =($42, 0), =($25, 2868770270353813622), IS NOT NULL($104), IS NOT NULL($57))]) - CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) + LogicalFilter(condition=[AND(IS NOT NULL($104), IS NOT NULL($57))]) + LogicalFilter(condition=[AND(=($103, 62), >=($0, TIMESTAMP('2013-07-01 00:00:00':VARCHAR)), <=($0, TIMESTAMP('2013-07-31 00:00:00':VARCHAR)), =($72, 0), =($42, 0), =($25, 2868770270353813622))]) + CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | EnumerableLimit(fetch=[10000]) EnumerableLimit(offset=[10000], fetch=[10]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q43.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q43.yaml index 5b2c5816127..62977946daf 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q43.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q43.yaml @@ -5,8 +5,9 @@ calcite: LogicalProject(PageViews=[$1], M=[$0]) LogicalAggregate(group=[{0}], PageViews=[COUNT()]) LogicalProject(M=[SPAN($17, 1, 'm')]) - LogicalFilter(condition=[AND(=($103, 62), >=($0, TIMESTAMP('2013-07-01 00:00:00':VARCHAR)), <=($0, TIMESTAMP('2013-07-15 00:00:00':VARCHAR)), =($72, 0), =($42, 0), IS NOT NULL($17))]) - CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) + LogicalFilter(condition=[IS NOT NULL($17)]) + LogicalFilter(condition=[AND(=($103, 62), >=($0, TIMESTAMP('2013-07-01 00:00:00':VARCHAR)), <=($0, TIMESTAMP('2013-07-15 00:00:00':VARCHAR)), =($72, 0), =($42, 0))]) + CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | EnumerableLimit(fetch=[10000]) EnumerableLimit(offset=[1000], fetch=[10]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q8.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q8.yaml index e4ba0272b81..a71532d4271 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q8.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q8.yaml @@ -5,7 +5,8 @@ calcite: LogicalProject(count()=[$1], AdvEngineID=[$0]) LogicalAggregate(group=[{0}], count()=[COUNT()]) LogicalProject(AdvEngineID=[$19]) - LogicalFilter(condition=[<>($19, 0)]) - CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) + LogicalFilter(condition=[IS NOT NULL($19)]) + LogicalFilter(condition=[<>($19, 0)]) + CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER-><>($0, 0), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},count()=COUNT()), SORT_AGG_METRICS->[1 DESC LAST], PROJECT->[count(), AdvEngineID], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"exists":{"field":"AdvEngineID","boost":1.0}}],"must_not":[{"term":{"AdvEngineID":{"value":0,"boost":1.0}}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"AdvEngineID":{"terms":{"field":"AdvEngineID","size":10000,"min_doc_count":1,"shard_min_doc_count":0,"show_term_doc_count_error":false,"order":[{"_count":"desc"},{"_key":"asc"}]}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_bin_aligntime.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_bin_aligntime.yaml index 9a82afe29ec..dce92478a61 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_bin_aligntime.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_bin_aligntime.yaml @@ -6,5 +6,5 @@ calcite: LogicalProject(category=[$1], value=[$2], timestamp=[$3], _id=[$4], _index=[$5], _score=[$6], _maxscore=[$7], _sort=[$8], _routing=[$9], @timestamp=[FROM_UNIXTIME(*(*(FLOOR(DIVIDE(DIVIDE(UNIX_TIMESTAMP($0), 3600), 2)), 2), 3600))]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]]) physical: | - EnumerableCalc(expr#0..3=[{inputs}], expr#4=[UNIX_TIMESTAMP($t3)], expr#5=[3600], expr#6=[DIVIDE($t4, $t5)], expr#7=[2], expr#8=[DIVIDE($t6, $t7)], expr#9=[FLOOR($t8)], expr#10=[*($t9, $t7)], expr#11=[*($t10, $t5)], expr#12=[FROM_UNIXTIME($t11)], proj#0..2=[{exprs}], $f3=[$t12]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]], PushDownContext=[[PROJECT->[category, value, timestamp, @timestamp], LIMIT->5, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":5,"timeout":"1m","_source":{"includes":["category","value","timestamp","@timestamp"],"excludes":[]}}, requestedTotalSize=5, pageSize=null, startFrom=0)]) \ No newline at end of file + EnumerableCalc(expr#0..3=[{inputs}], expr#4=[UNIX_TIMESTAMP($t3)], expr#5=[3600], expr#6=[DIVIDE($t4, $t5)], expr#7=[2], expr#8=[DIVIDE($t6, $t7)], expr#9=[FLOOR($t8)], expr#10=[*($t9, $t7)], expr#11=[*($t10, $t5)], expr#12=[FROM_UNIXTIME($t11)], proj#0..2=[{exprs}], @timestamp=[$t12]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]], PushDownContext=[[PROJECT->[category, value, timestamp, @timestamp], LIMIT->5, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":5,"timeout":"1m","_source":{"includes":["category","value","timestamp","@timestamp"],"excludes":[]}}, requestedTotalSize=5, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_bin_span.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_bin_span.yaml index 3a1cee955d3..35b422b93f8 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_bin_span.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_bin_span.yaml @@ -6,5 +6,5 @@ calcite: LogicalProject(account_number=[$0], firstname=[$1], address=[$2], balance=[$3], gender=[$4], city=[$5], employer=[$6], state=[$7], email=[$9], lastname=[$10], _id=[$11], _index=[$12], _score=[$13], _maxscore=[$14], _sort=[$15], _routing=[$16], age=[SPAN_BUCKET($8, 10)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - EnumerableCalc(expr#0..10=[{inputs}], expr#11=[10], expr#12=[SPAN_BUCKET($t10, $t11)], proj#0..9=[{exprs}], $f10=[$t12]) + EnumerableCalc(expr#0..10=[{inputs}], expr#11=[10], expr#12=[SPAN_BUCKET($t10, $t11)], proj#0..9=[{exprs}], age=[$t12]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[account_number, firstname, address, balance, gender, city, employer, state, email, lastname, age], LIMIT->5, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":5,"timeout":"1m","_source":{"includes":["account_number","firstname","address","balance","gender","city","employer","state","email","lastname","age"],"excludes":[]}}, requestedTotalSize=5, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_complex_sort_expr_project_then_sort.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_complex_sort_expr_project_then_sort.yaml index 1b78dc58db8..2f9e75af3dd 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_complex_sort_expr_project_then_sort.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_complex_sort_expr_project_then_sort.yaml @@ -5,5 +5,5 @@ calcite: LogicalProject(age=[$10], age2=[+($10, $7)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]]) physical: | - EnumerableCalc(expr#0..1=[{inputs}], expr#2=[+($t0, $t1)], age=[$t0], $f1=[$t2]) + EnumerableCalc(expr#0..1=[{inputs}], expr#2=[+($t0, $t1)], age=[$t0], age2=[$t2]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[PROJECT->[age, balance], SORT_EXPR->[+($0, $1) ASCENDING NULLS_FIRST], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10000,"timeout":"1m","_source":{"includes":["age","balance"],"excludes":[]},"sort":[{"_script":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQBQ3sKICAib3AiOiB7CiAgICAibmFtZSI6ICIrIiwKICAgICJraW5kIjogIlBMVVMiLAogICAgInN5bnRheCI6ICJCSU5BUlkiCiAgfSwKICAib3BlcmFuZHMiOiBbCiAgICB7CiAgICAgICJkeW5hbWljUGFyYW0iOiAwLAogICAgICAidHlwZSI6IHsKICAgICAgICAidHlwZSI6ICJCSUdJTlQiLAogICAgICAgICJudWxsYWJsZSI6IHRydWUKICAgICAgfQogICAgfSwKICAgIHsKICAgICAgImR5bmFtaWNQYXJhbSI6IDEsCiAgICAgICJ0eXBlIjogewogICAgICAgICJ0eXBlIjogIkJJR0lOVCIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZQogICAgICB9CiAgICB9CiAgXQp9\"}","lang":"opensearch_compounded_script","params":{"MISSING_MAX":false,"utcTimestamp": 0,"SOURCES":[0,0],"DIGESTS":["age","balance"]}},"type":"number","order":"asc"}}]}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_complex_sort_expr_push.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_complex_sort_expr_push.yaml index 3d57195be33..695e6c17c7c 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_complex_sort_expr_push.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_complex_sort_expr_push.yaml @@ -6,5 +6,5 @@ calcite: LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12], _id=[$13], _index=[$14], _score=[$15], _maxscore=[$16], _sort=[$17], _routing=[$18], age2=[+($10, $7)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]]) physical: | - EnumerableCalc(expr#0..1=[{inputs}], expr#2=[+($t0, $t1)], age=[$t0], $f1=[$t2]) + EnumerableCalc(expr#0..1=[{inputs}], expr#2=[+($t0, $t1)], age=[$t0], age2=[$t2]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[PROJECT->[age, balance], SORT_EXPR->[+($0, $1) ASCENDING NULLS_FIRST], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10000,"timeout":"1m","_source":{"includes":["age","balance"],"excludes":[]},"sort":[{"_script":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQBQ3sKICAib3AiOiB7CiAgICAibmFtZSI6ICIrIiwKICAgICJraW5kIjogIlBMVVMiLAogICAgInN5bnRheCI6ICJCSU5BUlkiCiAgfSwKICAib3BlcmFuZHMiOiBbCiAgICB7CiAgICAgICJkeW5hbWljUGFyYW0iOiAwLAogICAgICAidHlwZSI6IHsKICAgICAgICAidHlwZSI6ICJCSUdJTlQiLAogICAgICAgICJudWxsYWJsZSI6IHRydWUKICAgICAgfQogICAgfSwKICAgIHsKICAgICAgImR5bmFtaWNQYXJhbSI6IDEsCiAgICAgICJ0eXBlIjogewogICAgICAgICJ0eXBlIjogIkJJR0lOVCIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZQogICAgICB9CiAgICB9CiAgXQp9\"}","lang":"opensearch_compounded_script","params":{"MISSING_MAX":false,"utcTimestamp": 0,"SOURCES":[0,0],"DIGESTS":["age","balance"]}},"type":"number","order":"asc"}}]}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_complex_sort_expr_pushdown_for_smj.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_complex_sort_expr_pushdown_for_smj.yaml index b747b30f714..56183ad9939 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_complex_sort_expr_pushdown_for_smj.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_complex_sort_expr_pushdown_for_smj.yaml @@ -11,7 +11,7 @@ calcite: physical: | EnumerableLimit(fetch=[10000]) EnumerableMergeJoin(condition=[=($13, $15)], joinType=[inner]) - EnumerableCalc(expr#0..12=[{inputs}], expr#13=['(?^[A-Z])'], expr#14=['initial'], expr#15=[REX_EXTRACT($t6, $t13, $t14)], proj#0..12=[{exprs}], $f13=[$t15]) + EnumerableCalc(expr#0..12=[{inputs}], expr#13=['(?^[A-Z])'], expr#14=['initial'], expr#15=[REX_EXTRACT($t6, $t13, $t14)], proj#0..12=[{exprs}], initial=[$t15]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[PROJECT->[account_number, firstname, address, birthdate, gender, city, lastname, balance, employer, state, age, email, male], SORT_EXPR->[REX_EXTRACT($6, '(?^[A-Z])', 'initial') ASCENDING NULLS_LAST]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["account_number","firstname","address","birthdate","gender","city","lastname","balance","employer","state","age","email","male"],"excludes":[]},"sort":[{"_script":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQC63sKICAib3AiOiB7CiAgICAibmFtZSI6ICJSRVhfRVhUUkFDVCIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfSwKICAgIHsKICAgICAgImR5bmFtaWNQYXJhbSI6IDEsCiAgICAgICJ0eXBlIjogewogICAgICAgICJ0eXBlIjogIlZBUkNIQVIiLAogICAgICAgICJudWxsYWJsZSI6IHRydWUsCiAgICAgICAgInByZWNpc2lvbiI6IC0xCiAgICAgIH0KICAgIH0sCiAgICB7CiAgICAgICJkeW5hbWljUGFyYW0iOiAyLAogICAgICAidHlwZSI6IHsKICAgICAgICAidHlwZSI6ICJWQVJDSEFSIiwKICAgICAgICAibnVsbGFibGUiOiB0cnVlLAogICAgICAgICJwcmVjaXNpb24iOiAtMQogICAgICB9CiAgICB9CiAgXSwKICAiY2xhc3MiOiAib3JnLm9wZW5zZWFyY2guc3FsLmV4cHJlc3Npb24uZnVuY3Rpb24uVXNlckRlZmluZWRGdW5jdGlvbkJ1aWxkZXIkMSIsCiAgInR5cGUiOiB7CiAgICAidHlwZSI6ICJWQVJDSEFSIiwKICAgICJudWxsYWJsZSI6IHRydWUsCiAgICAicHJlY2lzaW9uIjogMjAwMAogIH0sCiAgImRldGVybWluaXN0aWMiOiB0cnVlLAogICJkeW5hbWljIjogZmFsc2UKfQ==\"}","lang":"opensearch_compounded_script","params":{"MISSING_MAX":true,"utcTimestamp": 0,"SOURCES":[0,2,2],"DIGESTS":["lastname","(?^[A-Z])","initial"]}},"type":"string","order":"asc"}}]}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[PROJECT->[account_number, firstname, address, birthdate, gender, city, lastname, balance, employer, state, age, email, male], LIMIT->50000, SORT->[{ "firstname" : { diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_complex_sort_expr_pushdown_for_smj_w_max_option.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_complex_sort_expr_pushdown_for_smj_w_max_option.yaml index 6a9d9fd09b2..96a86472e47 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_complex_sort_expr_pushdown_for_smj_w_max_option.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_complex_sort_expr_pushdown_for_smj_w_max_option.yaml @@ -8,7 +8,7 @@ calcite: LogicalSystemLimit(fetch=[50000], type=[JOIN_SUBSEARCH_MAXOUT]) LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12]) LogicalFilter(condition=[<=($13, 1)]) - LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12], _row_number_join_max_dedup_=[ROW_NUMBER() OVER (PARTITION BY $6)]) + LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12], _row_number_dedup_=[ROW_NUMBER() OVER (PARTITION BY $6)]) LogicalFilter(condition=[IS NOT NULL($6)]) LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]]) @@ -16,6 +16,6 @@ calcite: EnumerableCalc(expr#0..13=[{inputs}], account_number=[$t1], firstname=[$t2], address=[$t3], birthdate=[$t4], gender=[$t5], city=[$t6], lastname=[$t7], balance=[$t8], employer=[$t9], state=[$t10], age=[$t11], email=[$t12], male=[$t13]) EnumerableLimit(fetch=[10000]) EnumerableMergeJoin(condition=[=($0, $7)], joinType=[left]) - EnumerableCalc(expr#0=[{inputs}], expr#1=['(?^[A-Z])'], expr#2=['lastname'], expr#3=[REX_EXTRACT($t0, $t1, $t2)], $f0=[$t3]) + EnumerableCalc(expr#0=[{inputs}], expr#1=['(?^[A-Z])'], expr#2=['lastname'], expr#3=[REX_EXTRACT($t0, $t1, $t2)], lastname=[$t3]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[PROJECT->[lastname], LIMIT->10000, SORT_EXPR->[REX_EXTRACT($0, '(?^[A-Z])', 'lastname') ASCENDING NULLS_LAST]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10000,"timeout":"1m","_source":{"includes":["lastname"],"excludes":[]},"sort":[{"_script":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQC63sKICAib3AiOiB7CiAgICAibmFtZSI6ICJSRVhfRVhUUkFDVCIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfSwKICAgIHsKICAgICAgImR5bmFtaWNQYXJhbSI6IDEsCiAgICAgICJ0eXBlIjogewogICAgICAgICJ0eXBlIjogIlZBUkNIQVIiLAogICAgICAgICJudWxsYWJsZSI6IHRydWUsCiAgICAgICAgInByZWNpc2lvbiI6IC0xCiAgICAgIH0KICAgIH0sCiAgICB7CiAgICAgICJkeW5hbWljUGFyYW0iOiAyLAogICAgICAidHlwZSI6IHsKICAgICAgICAidHlwZSI6ICJWQVJDSEFSIiwKICAgICAgICAibnVsbGFibGUiOiB0cnVlLAogICAgICAgICJwcmVjaXNpb24iOiAtMQogICAgICB9CiAgICB9CiAgXSwKICAiY2xhc3MiOiAib3JnLm9wZW5zZWFyY2guc3FsLmV4cHJlc3Npb24uZnVuY3Rpb24uVXNlckRlZmluZWRGdW5jdGlvbkJ1aWxkZXIkMSIsCiAgInR5cGUiOiB7CiAgICAidHlwZSI6ICJWQVJDSEFSIiwKICAgICJudWxsYWJsZSI6IHRydWUsCiAgICAicHJlY2lzaW9uIjogMjAwMAogIH0sCiAgImRldGVybWluaXN0aWMiOiB0cnVlLAogICJkeW5hbWljIjogZmFsc2UKfQ==\"}","lang":"opensearch_compounded_script","params":{"MISSING_MAX":true,"utcTimestamp": 0,"SOURCES":[0,2,2],"DIGESTS":["lastname","(?^[A-Z])","lastname"]}},"type":"string","order":"asc"}}]}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0},agg#0=LITERAL_AGG(1)), LIMIT->50000, SORT->[6]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"lastname":{"terms":{"field":"lastname","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f1":{"top_hits":{"from":0,"size":1,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["lastname","account_number","firstname","address","birthdate","gender","city","balance","employer","state","age","email","male"],"excludes":[]},"script_fields":{}}}}}}}, requestedTotalSize=50000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_complex_sort_expr_single_expr_output_push.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_complex_sort_expr_single_expr_output_push.yaml index aee7f7bce3d..be22682ff85 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_complex_sort_expr_single_expr_output_push.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_complex_sort_expr_single_expr_output_push.yaml @@ -6,5 +6,5 @@ calcite: LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12], _id=[$13], _index=[$14], _score=[$15], _maxscore=[$16], _sort=[$17], _routing=[$18], age2=[+($10, $7)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]]) physical: | - EnumerableCalc(expr#0..1=[{inputs}], expr#2=[+($t0, $t1)], $f0=[$t2]) + EnumerableCalc(expr#0..1=[{inputs}], expr#2=[+($t0, $t1)], age2=[$t2]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[PROJECT->[age, balance], SORT_EXPR->[+($0, $1) ASCENDING NULLS_FIRST], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10000,"timeout":"1m","_source":{"includes":["age","balance"],"excludes":[]},"sort":[{"_script":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQBQ3sKICAib3AiOiB7CiAgICAibmFtZSI6ICIrIiwKICAgICJraW5kIjogIlBMVVMiLAogICAgInN5bnRheCI6ICJCSU5BUlkiCiAgfSwKICAib3BlcmFuZHMiOiBbCiAgICB7CiAgICAgICJkeW5hbWljUGFyYW0iOiAwLAogICAgICAidHlwZSI6IHsKICAgICAgICAidHlwZSI6ICJCSUdJTlQiLAogICAgICAgICJudWxsYWJsZSI6IHRydWUKICAgICAgfQogICAgfSwKICAgIHsKICAgICAgImR5bmFtaWNQYXJhbSI6IDEsCiAgICAgICJ0eXBlIjogewogICAgICAgICJ0eXBlIjogIkJJR0lOVCIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZQogICAgICB9CiAgICB9CiAgXQp9\"}","lang":"opensearch_compounded_script","params":{"MISSING_MAX":false,"utcTimestamp": 0,"SOURCES":[0,0],"DIGESTS":["age","balance"]}},"type":"number","order":"asc"}}]}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr1.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr1.yaml index a84b664804e..39ef7e23a9b 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr1.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr1.yaml @@ -8,4 +8,4 @@ calcite: LogicalProject(account_number=[$0], firstname=[$1], address=[$2], balance=[$3], gender=[$4], city=[$5], employer=[$6], state=[$7], age=[$8], email=[$9], lastname=[$10], _id=[$11], _index=[$12], _score=[$13], _maxscore=[$14], _sort=[$15], _routing=[$16], new_gender=[LOWER($4)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0},agg#0=LITERAL_AGG(1)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"new_gender":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f1":{"top_hits":{"from":0,"size":1,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["account_number","firstname","address","balance","gender","city","employer","state","age","email","lastname"],"excludes":[]},"script_fields":{"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0},agg#0=LITERAL_AGG(1)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"new_gender":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f1":{"top_hits":{"from":0,"size":1,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["account_number","firstname","address","balance","gender","city","employer","state","age","email","lastname"],"excludes":[]},"script_fields":{"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr2.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr2.yaml index a892b5c4e26..6edfbc9bdac 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr2.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr2.yaml @@ -8,4 +8,4 @@ calcite: LogicalProject(account_number=[$0], gender=[$4], age=[$8], state=[$7], new_gender=[LOWER($4)], new_state=[LOWER($7)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(1)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"new_gender":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"missing_bucket":false,"order":"asc"}}},{"new_state":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":1,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["account_number","gender","age","state"],"excludes":[]},"script_fields":{"new_state":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"ignore_failure":false},"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(1)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"new_gender":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"missing_bucket":false,"order":"asc"}}},{"new_state":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":1,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["account_number","gender","age","state"],"excludes":[]},"script_fields":{"new_state":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"ignore_failure":false},"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr2_alternative.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr2_alternative.yaml index ffa6bc1aa52..29b7041b623 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr2_alternative.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr2_alternative.yaml @@ -8,4 +8,4 @@ calcite: LogicalProject(account_number=[$0], gender=[$4], age=[$8], state=[$7], new_gender=[LOWER($4)], new_state=[LOWER($7)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(1)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"new_gender":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"missing_bucket":false,"order":"asc"}}},{"new_state":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":1,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["account_number","gender","age","state"],"excludes":[]},"script_fields":{"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false},"new_state":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(1)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"new_gender":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"missing_bucket":false,"order":"asc"}}},{"new_state":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":1,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["account_number","gender","age","state"],"excludes":[]},"script_fields":{"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false},"new_state":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr3.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr3.yaml index ce4f25d3892..95cf9873d44 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr3.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr3.yaml @@ -8,4 +8,4 @@ calcite: LogicalProject(account_number=[$0], firstname=[$1], address=[$2], balance=[$3], gender=[$4], city=[$5], employer=[$6], state=[$7], age=[$8], email=[$9], lastname=[$10], _id=[$11], _index=[$12], _score=[$13], _maxscore=[$14], _sort=[$15], _routing=[$16], new_gender=[LOWER($4)], new_state=[LOWER($7)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(2)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"new_gender":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"missing_bucket":false,"order":"asc"}}},{"new_state":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":2,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["account_number","firstname","address","balance","gender","city","employer","state","age","email","lastname"],"excludes":[]},"script_fields":{"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false},"new_state":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(2)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"new_gender":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"missing_bucket":false,"order":"asc"}}},{"new_state":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":2,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["account_number","firstname","address","balance","gender","city","employer","state","age","email","lastname"],"excludes":[]},"script_fields":{"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false},"new_state":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr3_alternative.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr3_alternative.yaml index a7009a3751b..744c1fcdf68 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr3_alternative.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr3_alternative.yaml @@ -8,4 +8,4 @@ calcite: LogicalProject(account_number=[$0], firstname=[$1], address=[$2], balance=[$3], gender=[$4], city=[$5], employer=[$6], state=[$7], age=[$8], email=[$9], lastname=[$10], _id=[$11], _index=[$12], _score=[$13], _maxscore=[$14], _sort=[$15], _routing=[$16], new_gender=[LOWER($4)], new_state=[LOWER($7)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(2)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"new_gender":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"missing_bucket":false,"order":"asc"}}},{"new_state":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":2,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["account_number","firstname","address","balance","gender","city","employer","state","age","email","lastname"],"excludes":[]},"script_fields":{"new_state":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"ignore_failure":false},"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(2)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"new_gender":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"missing_bucket":false,"order":"asc"}}},{"new_state":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":2,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["account_number","firstname","address","balance","gender","city","employer","state","age","email","lastname"],"excludes":[]},"script_fields":{"new_state":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"ignore_failure":false},"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr4.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr4.yaml index 54b69192b13..ea9c46e976c 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr4.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr4.yaml @@ -9,4 +9,4 @@ calcite: LogicalProject(account_number=[$0], gender=[$4], age=[$8], state=[$7], new_gender=[LOWER($4)], new_state=[LOWER($7)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(2)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"new_gender":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"missing_bucket":false,"order":"asc"}}},{"new_state":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":2,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["account_number","gender","age","state"],"excludes":[]},"script_fields":{"new_state":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"ignore_failure":false},"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(2)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"new_gender":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"missing_bucket":false,"order":"asc"}}},{"new_state":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":2,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["account_number","gender","age","state"],"excludes":[]},"script_fields":{"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false},"new_state":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr4_alternative.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr4_alternative.yaml index ea9c46e976c..54b69192b13 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr4_alternative.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr4_alternative.yaml @@ -9,4 +9,4 @@ calcite: LogicalProject(account_number=[$0], gender=[$4], age=[$8], state=[$7], new_gender=[LOWER($4)], new_state=[LOWER($7)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(2)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"new_gender":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"missing_bucket":false,"order":"asc"}}},{"new_state":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":2,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["account_number","gender","age","state"],"excludes":[]},"script_fields":{"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false},"new_state":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(2)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"new_gender":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"missing_bucket":false,"order":"asc"}}},{"new_state":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":2,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["account_number","gender","age","state"],"excludes":[]},"script_fields":{"new_state":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"ignore_failure":false},"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr_complex1.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr_complex1.yaml index d7354c8b6d1..32cb2772354 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr_complex1.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr_complex1.yaml @@ -9,5 +9,4 @@ calcite: LogicalProject(account_number=[$0], gender=[$4], age=[$8], state=[$7], new_gender=[LOWER($4)], new_state=[LOWER($7)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - EnumerableLimit(fetch=[10000]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(2)), SORT->[4 ASC FIRST, 5 DESC LAST]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"new_gender":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"missing_bucket":false,"order":"asc"}}},{"new_state":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"missing_bucket":false,"order":"desc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":2,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["account_number","gender","age","state"],"excludes":[]},"script_fields":{"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false},"new_state":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(2)), SORT->[4 ASC FIRST, 5 DESC LAST], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"new_gender":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"missing_bucket":false,"order":"asc"}}},{"new_state":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"missing_bucket":false,"order":"desc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":2,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["account_number","gender","age","state"],"excludes":[]},"script_fields":{"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false},"new_state":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr_complex1_alternative.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr_complex1_alternative.yaml index fdd427f5f9b..7539f76dd94 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr_complex1_alternative.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr_complex1_alternative.yaml @@ -9,5 +9,4 @@ calcite: LogicalProject(account_number=[$0], gender=[$4], age=[$8], state=[$7], new_gender=[LOWER($4)], new_state=[LOWER($7)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - EnumerableLimit(fetch=[10000]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(2)), SORT->[4 ASC FIRST, 5 DESC LAST]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"new_gender":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"missing_bucket":false,"order":"asc"}}},{"new_state":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"missing_bucket":false,"order":"desc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":2,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["account_number","gender","age","state"],"excludes":[]},"script_fields":{"new_state":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"ignore_failure":false},"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(2)), SORT->[4 ASC FIRST, 5 DESC LAST], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"new_gender":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"missing_bucket":false,"order":"asc"}}},{"new_state":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"missing_bucket":false,"order":"desc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":2,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["account_number","gender","age","state"],"excludes":[]},"script_fields":{"new_state":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"ignore_failure":false},"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr_complex2.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr_complex2.yaml index 53462af4bce..92783e80233 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr_complex2.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr_complex2.yaml @@ -9,4 +9,4 @@ calcite: LogicalProject(account_number=[$0], gender=[$4], age=[$8], state=[$7], new_gender=[LOWER($4)], new_state=[LOWER($7)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(2)), SORT->[4 ASC FIRST, 5 DESC LAST], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"new_gender":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"missing_bucket":false,"order":"asc"}}},{"new_state":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"missing_bucket":false,"order":"desc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":2,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["account_number","gender","age","state"],"excludes":[]},"script_fields":{"new_state":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"ignore_failure":false},"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(2)), SORT->[4 ASC FIRST, 5 DESC LAST], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"new_gender":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"missing_bucket":false,"order":"asc"}}},{"new_state":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"missing_bucket":false,"order":"desc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":2,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["account_number","gender","age","state"],"excludes":[]},"script_fields":{"new_state":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"ignore_failure":false},"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr_complex2_alternative.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr_complex2_alternative.yaml index b9cb684053f..4b0564b3e7c 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr_complex2_alternative.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr_complex2_alternative.yaml @@ -9,4 +9,4 @@ calcite: LogicalProject(account_number=[$0], gender=[$4], age=[$8], state=[$7], new_gender=[LOWER($4)], new_state=[LOWER($7)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(2)), SORT->[4 ASC FIRST, 5 DESC LAST], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"new_gender":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"missing_bucket":false,"order":"asc"}}},{"new_state":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"missing_bucket":false,"order":"desc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":2,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["account_number","gender","age","state"],"excludes":[]},"script_fields":{"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false},"new_state":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(2)), SORT->[4 ASC FIRST, 5 DESC LAST], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"new_gender":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"missing_bucket":false,"order":"asc"}}},{"new_state":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"missing_bucket":false,"order":"desc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":2,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["account_number","gender","age","state"],"excludes":[]},"script_fields":{"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false},"new_state":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_keepempty_true_not_pushed.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_keepempty_true_not_pushed.yaml index 9f15c0d120b..99ab0811bda 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_keepempty_true_not_pushed.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_keepempty_true_not_pushed.yaml @@ -9,4 +9,4 @@ calcite: EnumerableLimit(fetch=[10000]) EnumerableCalc(expr#0..3=[{inputs}], expr#4=[IS NULL($t1)], expr#5=[1], expr#6=[<=($t3, $t5)], expr#7=[OR($t4, $t6)], proj#0..2=[{exprs}], $condition=[$t7]) EnumerableWindow(window#0=[window(partition {1} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[account_number, gender, age]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["account_number","gender","age"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[account_number, gender, age]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["account_number","gender","age"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_text_type_no_push.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_text_type_no_push.yaml index 80599e8a338..c2c1e672087 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_text_type_no_push.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_text_type_no_push.yaml @@ -10,4 +10,4 @@ calcite: EnumerableLimit(fetch=[10000]) EnumerableCalc(expr#0..13=[{inputs}], expr#14=[1], expr#15=[<=($t13, $t14)], proj#0..12=[{exprs}], $condition=[$t15]) EnumerableWindow(window#0=[window(partition {11} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[PROJECT->[account_number, firstname, address, birthdate, gender, city, lastname, balance, employer, state, age, email, male], FILTER->IS NOT NULL($11)], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","query":{"exists":{"field":"email","boost":1.0}},"_source":{"includes":["account_number","firstname","address","birthdate","gender","city","lastname","balance","employer","state","age","email","male"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[PROJECT->[account_number, firstname, address, birthdate, gender, city, lastname, balance, employer, state, age, email, male], FILTER->IS NOT NULL($11)], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","query":{"exists":{"field":"email","boost":1.0}},"_source":{"includes":["account_number","firstname","address","birthdate","gender","city","lastname","balance","employer","state","age","email","male"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_expr1.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_expr1.yaml index 45034087d0f..4f2359fa280 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_expr1.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_expr1.yaml @@ -8,4 +8,4 @@ calcite: LogicalProject(account_number=[$0], firstname=[$1], address=[$2], balance=[$3], gender=[$4], city=[$5], employer=[$6], state=[$7], age=[$8], email=[$9], lastname=[$10], _id=[$11], _index=[$12], _score=[$13], _maxscore=[$14], _sort=[$15], _routing=[$16], new_gender=[LOWER($4)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0},agg#0=LITERAL_AGG(1)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"age":{"terms":{"field":"age","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f1":{"top_hits":{"from":0,"size":1,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["age","account_number","firstname","address","balance","gender","city","employer","state","email","lastname"],"excludes":[]},"script_fields":{"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0},agg#0=LITERAL_AGG(1)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"age":{"terms":{"field":"age","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f1":{"top_hits":{"from":0,"size":1,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["age","account_number","firstname","address","balance","gender","city","employer","state","email","lastname"],"excludes":[]},"script_fields":{"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_expr2.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_expr2.yaml index eadbe165e38..f82817f5777 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_expr2.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_expr2.yaml @@ -8,4 +8,4 @@ calcite: LogicalProject(account_number=[$0], gender=[$4], age=[$8], state=[$7], new_gender=[LOWER($4)], new_state=[LOWER($7)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(1)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"age":{"terms":{"field":"age","missing_bucket":false,"order":"asc"}}},{"new_state":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":1,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["age","account_number","gender","state"],"excludes":[]},"script_fields":{"new_state":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"ignore_failure":false},"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(1)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"age":{"terms":{"field":"age","missing_bucket":false,"order":"asc"}}},{"new_state":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":1,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["age","account_number","gender","state"],"excludes":[]},"script_fields":{"new_state":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"ignore_failure":false},"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_expr2_alternative.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_expr2_alternative.yaml index 4c7afb236a8..595619d6130 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_expr2_alternative.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_expr2_alternative.yaml @@ -8,4 +8,4 @@ calcite: LogicalProject(account_number=[$0], gender=[$4], age=[$8], state=[$7], new_gender=[LOWER($4)], new_state=[LOWER($7)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(1)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"age":{"terms":{"field":"age","missing_bucket":false,"order":"asc"}}},{"new_state":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":1,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["age","account_number","gender","state"],"excludes":[]},"script_fields":{"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false},"new_state":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(1)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"age":{"terms":{"field":"age","missing_bucket":false,"order":"asc"}}},{"new_state":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":1,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["age","account_number","gender","state"],"excludes":[]},"script_fields":{"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false},"new_state":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_expr4.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_expr4.yaml index 0767a8357e1..bd618ccdc00 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_expr4.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_expr4.yaml @@ -9,4 +9,4 @@ calcite: LogicalProject(account_number=[$0], gender=[$4], age=[$8], state=[$7], new_gender=[LOWER($4)], new_state=[LOWER($7)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(2)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"gender":{"terms":{"field":"gender.keyword","missing_bucket":false,"order":"asc"}}},{"state":{"terms":{"field":"state.keyword","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":2,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["gender","state","account_number","age"],"excludes":[]},"script_fields":{"new_state":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"ignore_failure":false},"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(2)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"gender":{"terms":{"field":"gender.keyword","missing_bucket":false,"order":"asc"}}},{"state":{"terms":{"field":"state.keyword","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":2,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["gender","state","account_number","age"],"excludes":[]},"script_fields":{"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false},"new_state":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_expr4_alternative.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_expr4_alternative.yaml index 4527da4f140..5bcfb7c6b46 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_expr4_alternative.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_expr4_alternative.yaml @@ -9,4 +9,4 @@ calcite: LogicalProject(account_number=[$0], gender=[$4], age=[$8], state=[$7], new_gender=[LOWER($4)], new_state=[LOWER($7)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(2)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"gender":{"terms":{"field":"gender.keyword","missing_bucket":false,"order":"asc"}}},{"state":{"terms":{"field":"state.keyword","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":2,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["gender","state","account_number","age"],"excludes":[]},"script_fields":{"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false},"new_state":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(2)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"gender":{"terms":{"field":"gender.keyword","missing_bucket":false,"order":"asc"}}},{"state":{"terms":{"field":"state.keyword","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":2,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["gender","state","account_number","age"],"excludes":[]},"script_fields":{"new_state":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"ignore_failure":false},"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_eval_max.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_eval_max.yaml index bd517738a0b..4d371b4d427 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_eval_max.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_eval_max.yaml @@ -4,5 +4,5 @@ calcite: LogicalProject(account_number=[$0], firstname=[$1], address=[$2], balance=[$3], gender=[$4], city=[$5], employer=[$6], state=[$7], age=[$8], email=[$9], lastname=[$10], new=[SCALAR_MAX(1, 2, 3, $8, 'banana':VARCHAR)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - EnumerableCalc(expr#0..10=[{inputs}], expr#11=[1], expr#12=[2], expr#13=[3], expr#14=['banana':VARCHAR], expr#15=[SCALAR_MAX($t11, $t12, $t13, $t8, $t14)], proj#0..10=[{exprs}], $f11=[$t15]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[account_number, firstname, address, balance, gender, city, employer, state, age, email, lastname], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10000,"timeout":"1m","_source":{"includes":["account_number","firstname","address","balance","gender","city","employer","state","age","email","lastname"],"excludes":[]}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) \ No newline at end of file + EnumerableCalc(expr#0..10=[{inputs}], expr#11=[1], expr#12=[2], expr#13=[3], expr#14=['banana':VARCHAR], expr#15=[SCALAR_MAX($t11, $t12, $t13, $t8, $t14)], proj#0..10=[{exprs}], new=[$t15]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[account_number, firstname, address, balance, gender, city, employer, state, age, email, lastname], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10000,"timeout":"1m","_source":{"includes":["account_number","firstname","address","balance","gender","city","employer","state","age","email","lastname"],"excludes":[]}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_eval_min.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_eval_min.yaml index d28a4ef30ec..cc83bbd8f71 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_eval_min.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_eval_min.yaml @@ -4,5 +4,5 @@ calcite: LogicalProject(account_number=[$0], firstname=[$1], address=[$2], balance=[$3], gender=[$4], city=[$5], employer=[$6], state=[$7], age=[$8], email=[$9], lastname=[$10], new=[SCALAR_MIN(1, 2, 3, $8, 'banana':VARCHAR)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - EnumerableCalc(expr#0..10=[{inputs}], expr#11=[1], expr#12=[2], expr#13=[3], expr#14=['banana':VARCHAR], expr#15=[SCALAR_MIN($t11, $t12, $t13, $t8, $t14)], proj#0..10=[{exprs}], $f11=[$t15]) + EnumerableCalc(expr#0..10=[{inputs}], expr#11=[1], expr#12=[2], expr#13=[3], expr#14=['banana':VARCHAR], expr#15=[SCALAR_MIN($t11, $t12, $t13, $t8, $t14)], proj#0..10=[{exprs}], new=[$t15]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[account_number, firstname, address, balance, gender, city, employer, state, age, email, lastname], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10000,"timeout":"1m","_source":{"includes":["account_number","firstname","address","balance","gender","city","employer","state","age","email","lastname"],"excludes":[]}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_fillnull_push.json b/integ-test/src/test/resources/expectedOutput/calcite/explain_fillnull_push.json index cdc0c4c27fd..9bdc21993e9 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_fillnull_push.json +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_fillnull_push.json @@ -1,6 +1,6 @@ { "calcite": { "logical": "LogicalSystemLimit(fetch=[10000], type=[QUERY_SIZE_LIMIT])\n LogicalProject(age=[COALESCE($8, -1)], balance=[COALESCE($3, -1)])\n CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]])\n", - "physical": "EnumerableCalc(expr#0..1=[{inputs}], expr#2=[-1], expr#3=[COALESCE($t0, $t2)], expr#4=[COALESCE($t1, $t2)], $f0=[$t3], $f1=[$t4])\n CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[age, balance], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={\"from\":0,\"size\":10000,\"timeout\":\"1m\",\"_source\":{\"includes\":[\"age\",\"balance\"],\"excludes\":[]}}, requestedTotalSize=10000, pageSize=null, startFrom=0)])\n" + "physical": "EnumerableCalc(expr#0..1=[{inputs}], expr#2=[-1], expr#3=[COALESCE($t0, $t2)], expr#4=[COALESCE($t1, $t2)], age=[$t3], balance=[$t4])\n CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[age, balance], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={\"from\":0,\"size\":10000,\"timeout\":\"1m\",\"_source\":{\"includes\":[\"age\",\"balance\"],\"excludes\":[]}}, requestedTotalSize=10000, pageSize=null, startFrom=0)])\n" } } diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_fillnull_value_syntax.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_fillnull_value_syntax.yaml index 0f2ba239d73..e677364b7c5 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_fillnull_value_syntax.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_fillnull_value_syntax.yaml @@ -4,5 +4,5 @@ calcite: LogicalProject(age=[COALESCE($8, 0)], balance=[COALESCE($3, 0)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - EnumerableCalc(expr#0..1=[{inputs}], expr#2=[0], expr#3=[COALESCE($t0, $t2)], expr#4=[COALESCE($t1, $t2)], $f0=[$t3], $f1=[$t4]) + EnumerableCalc(expr#0..1=[{inputs}], expr#2=[0], expr#3=[COALESCE($t0, $t2)], expr#4=[COALESCE($t1, $t2)], age=[$t3], balance=[$t4]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[age, balance], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10000,"timeout":"1m","_source":{"includes":["age","balance"],"excludes":[]}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_filter_push.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_filter_push.yaml index eb020cf0f92..11bf9baa46c 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_filter_push.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_filter_push.yaml @@ -2,7 +2,9 @@ calcite: logical: | LogicalSystemLimit(fetch=[10000], type=[QUERY_SIZE_LIMIT]) LogicalProject(age=[$8]) - LogicalFilter(condition=[AND(SEARCH($8, Sarg[(30..40)]), >($3, 10000))]) - CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) + LogicalFilter(condition=[>($3, 10000)]) + LogicalFilter(condition=[<($8, 40)]) + LogicalFilter(condition=[>($8, 30)]) + CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[balance, age], FILTER->AND(SEARCH($1, Sarg[(30..40)]), >($0, 10000)), PROJECT->[age], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10000,"timeout":"1m","query":{"bool":{"must":[{"range":{"age":{"from":30.0,"to":40.0,"include_lower":false,"include_upper":false,"boost":1.0}}},{"range":{"balance":{"from":10000,"to":null,"include_lower":false,"include_upper":true,"boost":1.0}}}],"adjust_pure_negative":true,"boost":1.0}},"_source":{"includes":["age"],"excludes":[]}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_filter_push_compare_date_string.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_filter_push_compare_date_string.yaml index 7f604d806ee..55951816ff7 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_filter_push_compare_date_string.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_filter_push_compare_date_string.yaml @@ -1,8 +1,9 @@ calcite: logical: | LogicalSystemLimit(fetch=[10000], type=[QUERY_SIZE_LIMIT]) - LogicalFilter(condition=[AND(>($0, DATE('2016-12-08 00:00:00.123456789':VARCHAR)), <($0, DATE('2018-11-09 00:00:00.000000000':VARCHAR)))]) - LogicalProject(yyyy-MM-dd=[$83]) - CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_date_formats]]) + LogicalFilter(condition=[<($0, DATE('2018-11-09 00:00:00.000000000':VARCHAR))]) + LogicalFilter(condition=[>($0, DATE('2016-12-08 00:00:00.123456789':VARCHAR))]) + LogicalProject(yyyy-MM-dd=[$83]) + CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_date_formats]]) physical: | CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_date_formats]], PushDownContext=[[PROJECT->[yyyy-MM-dd], FILTER->SEARCH($0, Sarg[('2016-12-08':VARCHAR..'2018-11-09':VARCHAR)]:VARCHAR), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10000,"timeout":"1m","query":{"range":{"yyyy-MM-dd":{"from":"2016-12-08","to":"2018-11-09","include_lower":false,"include_upper":false,"boost":1.0}}},"_source":{"includes":["yyyy-MM-dd"],"excludes":[]}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_filter_push_compare_time_string.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_filter_push_compare_time_string.yaml index 475bc85ecb2..faf6a3764c5 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_filter_push_compare_time_string.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_filter_push_compare_time_string.yaml @@ -1,8 +1,9 @@ calcite: logical: | LogicalSystemLimit(fetch=[10000], type=[QUERY_SIZE_LIMIT]) - LogicalFilter(condition=[AND(>($0, TIME('2016-12-08 12:00:00.123456789':VARCHAR)), <($0, TIME('2018-11-09 19:00:00.123456789':VARCHAR)))]) - LogicalProject(custom_time=[$49]) - CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_date_formats]]) + LogicalFilter(condition=[<($0, TIME('2018-11-09 19:00:00.123456789':VARCHAR))]) + LogicalFilter(condition=[>($0, TIME('2016-12-08 12:00:00.123456789':VARCHAR))]) + LogicalProject(custom_time=[$49]) + CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_date_formats]]) physical: | CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_date_formats]], PushDownContext=[[PROJECT->[custom_time], FILTER->SEARCH($0, Sarg[('12:00:00.123456789':VARCHAR..'19:00:00.123456789':VARCHAR)]:VARCHAR), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10000,"timeout":"1m","query":{"range":{"custom_time":{"from":"12:00:00.123456789","to":"19:00:00.123456789","include_lower":false,"include_upper":false,"boost":1.0}}},"_source":{"includes":["custom_time"],"excludes":[]}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_filter_push_compare_timestamp_string.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_filter_push_compare_timestamp_string.yaml index 954fab09d05..e0a3fc8a7d3 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_filter_push_compare_timestamp_string.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_filter_push_compare_timestamp_string.yaml @@ -2,7 +2,8 @@ calcite: logical: | LogicalSystemLimit(fetch=[10000], type=[QUERY_SIZE_LIMIT]) LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12]) - LogicalFilter(condition=[AND(>($3, TIMESTAMP('2016-12-08 00:00:00.000000000':VARCHAR)), <($3, TIMESTAMP('2018-11-09 00:00:00.000000000':VARCHAR)))]) - CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]]) + LogicalFilter(condition=[<($3, TIMESTAMP('2018-11-09 00:00:00.000000000':VARCHAR))]) + LogicalFilter(condition=[>($3, TIMESTAMP('2016-12-08 00:00:00.000000000':VARCHAR))]) + CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]]) physical: | CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[PROJECT->[account_number, firstname, address, birthdate, gender, city, lastname, balance, employer, state, age, email, male], FILTER->SEARCH($3, Sarg[('2016-12-08 00:00:00':VARCHAR..'2018-11-09 00:00:00':VARCHAR)]:VARCHAR), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10000,"timeout":"1m","query":{"range":{"birthdate":{"from":"2016-12-08T00:00:00.000Z","to":"2018-11-09T00:00:00.000Z","include_lower":false,"include_upper":false,"format":"date_time","boost":1.0}}},"_source":{"includes":["account_number","firstname","address","birthdate","gender","city","lastname","balance","employer","state","age","email","male"],"excludes":[]}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_filter_with_search.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_filter_with_search.yaml index b6dd067dc2e..7757094a517 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_filter_with_search.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_filter_with_search.yaml @@ -4,7 +4,8 @@ calcite: LogicalProject(count()=[$1], span(birthdate,1d)=[$0]) LogicalAggregate(group=[{0}], count()=[COUNT()]) LogicalProject(span(birthdate,1d)=[SPAN($3, 1, 'd')]) - LogicalFilter(condition=[AND(>=($3, TIMESTAMP('2023-01-01 00:00:00':VARCHAR)), <($3, TIMESTAMP('2023-01-03 00:00:00':VARCHAR)))]) - CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]]) + LogicalFilter(condition=[IS NOT NULL($3)]) + LogicalFilter(condition=[AND(>=($3, TIMESTAMP('2023-01-01 00:00:00':VARCHAR)), <($3, TIMESTAMP('2023-01-03 00:00:00':VARCHAR)))]) + CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]]) physical: | CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[FILTER->SEARCH($0, Sarg[['2023-01-01 00:00:00':VARCHAR..'2023-01-03 00:00:00':VARCHAR)]:VARCHAR), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},count()=COUNT()), PROJECT->[count(), span(birthdate,1d)], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"range":{"birthdate":{"from":"2023-01-01T00:00:00.000Z","to":"2023-01-03T00:00:00.000Z","include_lower":true,"include_upper":false,"format":"date_time","boost":1.0}}},"aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"span(birthdate,1d)":{"date_histogram":{"field":"birthdate","missing_bucket":false,"order":"asc","fixed_interval":"1d"}}}]}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_join_with_criteria_max_option.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_join_with_criteria_max_option.yaml index 0c3f8565335..359b5e203ac 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_join_with_criteria_max_option.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_join_with_criteria_max_option.yaml @@ -8,7 +8,7 @@ calcite: LogicalSystemLimit(fetch=[50000], type=[JOIN_SUBSEARCH_MAXOUT]) LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12]) LogicalFilter(condition=[<=($13, 1)]) - LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12], _row_number_join_max_dedup_=[ROW_NUMBER() OVER (PARTITION BY $0)]) + LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12], _row_number_dedup_=[ROW_NUMBER() OVER (PARTITION BY $0)]) LogicalFilter(condition=[IS NOT NULL($0)]) LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_join_with_fields_max_option.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_join_with_fields_max_option.yaml index 4b5597cc6b2..a3c14feafc9 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_join_with_fields_max_option.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_join_with_fields_max_option.yaml @@ -8,7 +8,7 @@ calcite: LogicalSystemLimit(fetch=[50000], type=[JOIN_SUBSEARCH_MAXOUT]) LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12]) LogicalFilter(condition=[<=($13, 1)]) - LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12], _row_number_join_max_dedup_=[ROW_NUMBER() OVER (PARTITION BY $0)]) + LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12], _row_number_dedup_=[ROW_NUMBER() OVER (PARTITION BY $0)]) LogicalFilter(condition=[IS NOT NULL($0)]) LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_limit_push.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_limit_push.yaml index 187f948a6e4..09636db30ad 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_limit_push.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_limit_push.yaml @@ -6,5 +6,5 @@ calcite: LogicalProject(account_number=[$0], firstname=[$1], address=[$2], balance=[$3], gender=[$4], city=[$5], employer=[$6], state=[$7], age=[$8], email=[$9], lastname=[$10], _id=[$11], _index=[$12], _score=[$13], _maxscore=[$14], _sort=[$15], _routing=[$16], ageMinus=[-($8, 30)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - EnumerableCalc(expr#0=[{inputs}], expr#1=[30], expr#2=[-($t0, $t1)], $f0=[$t2]) + EnumerableCalc(expr#0=[{inputs}], expr#1=[30], expr#2=[-($t0, $t1)], ageMinus=[$t2]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[age], LIMIT->5, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":5,"timeout":"1m","_source":{"includes":["age"],"excludes":[]}}, requestedTotalSize=5, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_nested_agg_dedup_not_push.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_nested_agg_dedup_not_push.yaml index e5a235e9f98..c9f8b484aed 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_nested_agg_dedup_not_push.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_nested_agg_dedup_not_push.yaml @@ -10,4 +10,4 @@ calcite: EnumerableLimit(fetch=[10000]) EnumerableCalc(expr#0..5=[{inputs}], expr#6=[1], expr#7=[<=($t5, $t6)], proj#0..1=[{exprs}], id=[$t3], age=[$t4], $condition=[$t7]) EnumerableWindow(window#0=[window(partition {2} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_nested_simple]], PushDownContext=[[PROJECT->[name, address, address.city, id, age], FILTER->IS NOT NULL($2)], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","query":{"nested":{"query":{"exists":{"field":"address.city","boost":1.0}},"path":"address","ignore_unmapped":false,"score_mode":"none","boost":1.0}},"_source":{"includes":["name","address","address.city","id","age"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_nested_simple]], PushDownContext=[[PROJECT->[name, address, address.city, id, age], FILTER->IS NOT NULL($2)], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","query":{"nested":{"query":{"exists":{"field":"address.city","boost":1.0}},"path":"address","ignore_unmapped":false,"score_mode":"none","boost":1.0}},"_source":{"includes":["name","address","address.city","id","age"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_output.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_output.yaml index 73a3fefe705..862a45dc617 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_output.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_output.yaml @@ -13,8 +13,9 @@ calcite: LogicalFilter(condition=[>($8, 30)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - EnumerableLimit(fetch=[10000]) - EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], expr#3=[<=($t1, $t2)], age2=[$t0], $condition=[$t3]) - EnumerableWindow(window#0=[window(partition {0} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) - EnumerableCalc(expr#0=[{inputs}], expr#1=[2], expr#2=[+($t0, $t1)], expr#3=[IS NOT NULL($t0)], age2=[$t2], $condition=[$t3]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[FILTER->>($2, 30), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},avg_age=AVG($2)), SORT->[0 ASC FIRST], PROJECT->[avg_age]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"range":{"age":{"from":30,"to":null,"include_lower":false,"include_upper":true,"boost":1.0}}},"aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"state":{"terms":{"field":"state.keyword","missing_bucket":true,"missing_order":"first","order":"asc"}}},{"city":{"terms":{"field":"city.keyword","missing_bucket":true,"missing_order":"first","order":"asc"}}}]},"aggregations":{"avg_age":{"avg":{"field":"age"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + EnumerableCalc(expr#0..1=[{inputs}], age2=[$t0]) + EnumerableLimit(fetch=[10000]) + EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], expr#3=[<=($t1, $t2)], proj#0..1=[{exprs}], $condition=[$t3]) + EnumerableWindow(window#0=[window(partition {0} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) + EnumerableCalc(expr#0=[{inputs}], expr#1=[2], expr#2=[+($t0, $t1)], expr#3=[IS NOT NULL($t0)], $0=[$t2], $condition=[$t3]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[FILTER->>($2, 30), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},avg_age=AVG($2)), PROJECT->[avg_age, state], SORT->[1 ASC FIRST], PROJECT->[avg_age]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"range":{"age":{"from":30,"to":null,"include_lower":false,"include_upper":true,"boost":1.0}}},"aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"state":{"terms":{"field":"state.keyword","missing_bucket":true,"missing_order":"first","order":"asc"}}},{"city":{"terms":{"field":"city.keyword","missing_bucket":true,"missing_order":"first","order":"asc"}}}]},"aggregations":{"avg_age":{"avg":{"field":"age"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_patterns_simple_pattern.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_patterns_simple_pattern.yaml index 5b6fdebfd14..71da341abfc 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_patterns_simple_pattern.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_patterns_simple_pattern.yaml @@ -4,5 +4,5 @@ calcite: LogicalProject(account_number=[$0], firstname=[$1], address=[$2], balance=[$3], gender=[$4], city=[$5], employer=[$6], state=[$7], age=[$8], email=[$9], lastname=[$10], patterns_field=[CASE(SEARCH($9, Sarg['':VARCHAR; NULL AS TRUE]:VARCHAR), '':VARCHAR, REGEXP_REPLACE($9, '[a-zA-Z0-9]+':VARCHAR, '<*>':VARCHAR))]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - EnumerableCalc(expr#0..10=[{inputs}], expr#11=[Sarg['':VARCHAR; NULL AS TRUE]:VARCHAR], expr#12=[SEARCH($t9, $t11)], expr#13=['':VARCHAR], expr#14=['[a-zA-Z0-9]+':VARCHAR], expr#15=['<*>':VARCHAR], expr#16=[REGEXP_REPLACE($t9, $t14, $t15)], expr#17=[CASE($t12, $t13, $t16)], proj#0..10=[{exprs}], $f11=[$t17]) + EnumerableCalc(expr#0..10=[{inputs}], expr#11=[Sarg['':VARCHAR; NULL AS TRUE]:VARCHAR], expr#12=[SEARCH($t9, $t11)], expr#13=['':VARCHAR], expr#14=['[a-zA-Z0-9]+':VARCHAR], expr#15=['<*>':VARCHAR], expr#16=[REGEXP_REPLACE($t9, $t14, $t15)], expr#17=[CASE($t12, $t13, $t16)], proj#0..10=[{exprs}], patterns_field=[$t17]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[account_number, firstname, address, balance, gender, city, employer, state, age, email, lastname], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10000,"timeout":"1m","_source":{"includes":["account_number","firstname","address","balance","gender","city","employer","state","age","email","lastname"],"excludes":[]}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_regexp_match_in_eval.json b/integ-test/src/test/resources/expectedOutput/calcite/explain_regexp_match_in_eval.json index cbd608971fd..bcaaec137b6 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_regexp_match_in_eval.json +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_regexp_match_in_eval.json @@ -1,6 +1,6 @@ { "calcite": { "logical": "LogicalSystemLimit(fetch=[10000], type=[QUERY_SIZE_LIMIT])\n LogicalProject(has_hello=[REGEXP_CONTAINS($0, 'hello':VARCHAR)])\n CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_strings]])\n", - "physical": "EnumerableCalc(expr#0=[{inputs}], expr#1=['hello':VARCHAR], expr#2=[REGEXP_CONTAINS($t0, $t1)], $f0=[$t2])\n CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_strings]], PushDownContext=[[PROJECT->[name], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={\"from\":0,\"size\":10000,\"timeout\":\"1m\",\"_source\":{\"includes\":[\"name\"],\"excludes\":[]}}, requestedTotalSize=10000, pageSize=null, startFrom=0)])\n" + "physical": "EnumerableCalc(expr#0=[{inputs}], expr#1=['hello':VARCHAR], expr#2=[REGEXP_CONTAINS($t0, $t1)], has_hello=[$t2])\n CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_strings]], PushDownContext=[[PROJECT->[name], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={\"from\":0,\"size\":10000,\"timeout\":\"1m\",\"_source\":{\"includes\":[\"name\"],\"excludes\":[]}}, requestedTotalSize=10000, pageSize=null, startFrom=0)])\n" } -} \ No newline at end of file +} diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_replace_command.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_replace_command.yaml index a867c569168..7e951ffecf8 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_replace_command.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_replace_command.yaml @@ -4,5 +4,5 @@ calcite: LogicalProject(state=[REPLACE($7, 'IL':VARCHAR, 'Illinois':VARCHAR)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - EnumerableCalc(expr#0=[{inputs}], expr#1=['IL':VARCHAR], expr#2=['Illinois':VARCHAR], expr#3=[REPLACE($t0, $t1, $t2)], $f0=[$t3]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[state], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10000,"timeout":"1m","_source":{"includes":["state"],"excludes":[]}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) \ No newline at end of file + EnumerableCalc(expr#0=[{inputs}], expr#1=['IL':VARCHAR], expr#2=['Illinois':VARCHAR], expr#3=[REPLACE($t0, $t1, $t2)], state=[$t3]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[state], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10000,"timeout":"1m","_source":{"includes":["state"],"excludes":[]}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_replace_wildcard.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_replace_wildcard.yaml index 0407849a472..784b05c63a5 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_replace_wildcard.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_replace_wildcard.yaml @@ -4,5 +4,5 @@ calcite: LogicalProject(state=[REGEXP_REPLACE($7, '^\Q\E(.*?)\QL\E$':VARCHAR, 'STATE_IL':VARCHAR)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - EnumerableCalc(expr#0=[{inputs}], expr#1=['^\Q\E(.*?)\QL\E$':VARCHAR], expr#2=['STATE_IL':VARCHAR], expr#3=[REGEXP_REPLACE($t0, $t1, $t2)], $f0=[$t3]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[state], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10000,"timeout":"1m","_source":{"includes":["state"],"excludes":[]}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) \ No newline at end of file + EnumerableCalc(expr#0=[{inputs}], expr#1=['^\Q\E(.*?)\QL\E$':VARCHAR], expr#2=['STATE_IL':VARCHAR], expr#3=[REGEXP_REPLACE($t0, $t1, $t2)], state=[$t3]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[state], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10000,"timeout":"1m","_source":{"includes":["state"],"excludes":[]}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_rex.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_rex.yaml index d420fca0baf..2c18345f7fb 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_rex.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_rex.yaml @@ -6,5 +6,5 @@ calcite: LogicalProject(account_number=[$0], firstname=[$1], address=[$2], balance=[$3], gender=[$4], city=[$5], employer=[$6], state=[$7], age=[$8], email=[$9], lastname=[$10], _id=[$11], _index=[$12], _score=[$13], _maxscore=[$14], _sort=[$15], _routing=[$16], initial=[REX_EXTRACT($10, '(?^[A-Z])', 'initial')]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - EnumerableCalc(expr#0..10=[{inputs}], expr#11=['(?^[A-Z])'], expr#12=['initial'], expr#13=[REX_EXTRACT($t10, $t11, $t12)], proj#0..10=[{exprs}], $f11=[$t13]) + EnumerableCalc(expr#0..10=[{inputs}], expr#11=['(?^[A-Z])'], expr#12=['initial'], expr#13=[REX_EXTRACT($t10, $t11, $t12)], proj#0..10=[{exprs}], initial=[$t13]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[account_number, firstname, address, balance, gender, city, employer, state, age, email, lastname], LIMIT->5, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":5,"timeout":"1m","_source":{"includes":["account_number","firstname","address","balance","gender","city","employer","state","age","email","lastname"],"excludes":[]}}, requestedTotalSize=5, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_simple_sort_expr_push.json b/integ-test/src/test/resources/expectedOutput/calcite/explain_simple_sort_expr_push.json index 0e826fc3fd2..779add87f49 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_simple_sort_expr_push.json +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_simple_sort_expr_push.json @@ -1,6 +1,6 @@ { "calcite": { "logical": "LogicalSystemLimit(sort0=[$1], dir0=[ASC-nulls-first], fetch=[10000], type=[QUERY_SIZE_LIMIT])\n LogicalProject(age=[$10], age2=[$19])\n LogicalSort(sort0=[$19], dir0=[ASC-nulls-first])\n LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12], _id=[$13], _index=[$14], _score=[$15], _maxscore=[$16], _sort=[$17], _routing=[$18], age2=[+($10, 2)])\n CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]])\n", - "physical": "EnumerableCalc(expr#0=[{inputs}], expr#1=[2], expr#2=[+($t0, $t1)], age=[$t0], $f1=[$t2])\n CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[PROJECT->[age], SORT->[{\n \"age\" : {\n \"order\" : \"asc\",\n \"missing\" : \"_first\"\n }\n}], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={\"from\":0,\"size\":10000,\"timeout\":\"1m\",\"_source\":{\"includes\":[\"age\"],\"excludes\":[]},\"sort\":[{\"age\":{\"order\":\"asc\",\"missing\":\"_first\"}}]}, requestedTotalSize=10000, pageSize=null, startFrom=0)])\n" + "physical": "EnumerableCalc(expr#0=[{inputs}], expr#1=[2], expr#2=[+($t0, $t1)], age=[$t0], age2=[$t2])\n CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[PROJECT->[age], SORT->[{\n \"age\" : {\n \"order\" : \"asc\",\n \"missing\" : \"_first\"\n }\n}], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={\"from\":0,\"size\":10000,\"timeout\":\"1m\",\"_source\":{\"includes\":[\"age\"],\"excludes\":[]},\"sort\":[{\"age\":{\"order\":\"asc\",\"missing\":\"_first\"}}]}, requestedTotalSize=10000, pageSize=null, startFrom=0)])\n" } -} \ No newline at end of file +} diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_simple_sort_expr_single_expr_output_push.json b/integ-test/src/test/resources/expectedOutput/calcite/explain_simple_sort_expr_single_expr_output_push.json index 722ef5379f0..ff07a6cddf6 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_simple_sort_expr_single_expr_output_push.json +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_simple_sort_expr_single_expr_output_push.json @@ -1,6 +1,6 @@ { "calcite": { "logical": "LogicalSystemLimit(sort0=[$0], dir0=[ASC-nulls-first], fetch=[10000], type=[QUERY_SIZE_LIMIT])\n LogicalProject(b=[$19])\n LogicalSort(sort0=[$19], dir0=[ASC-nulls-first])\n LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12], _id=[$13], _index=[$14], _score=[$15], _maxscore=[$16], _sort=[$17], _routing=[$18], b=[+($7, 1)])\n CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]])\n", - "physical": "EnumerableCalc(expr#0=[{inputs}], expr#1=[1], expr#2=[+($t0, $t1)], $f0=[$t2])\n CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[PROJECT->[balance], SORT->[{\n \"balance\" : {\n \"order\" : \"asc\",\n \"missing\" : \"_first\"\n }\n}], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={\"from\":0,\"size\":10000,\"timeout\":\"1m\",\"_source\":{\"includes\":[\"balance\"],\"excludes\":[]},\"sort\":[{\"balance\":{\"order\":\"asc\",\"missing\":\"_first\"}}]}, requestedTotalSize=10000, pageSize=null, startFrom=0)])\n" + "physical": "EnumerableCalc(expr#0=[{inputs}], expr#1=[1], expr#2=[+($t0, $t1)], b=[$t2])\n CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[PROJECT->[balance], SORT->[{\n \"balance\" : {\n \"order\" : \"asc\",\n \"missing\" : \"_first\"\n }\n}], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={\"from\":0,\"size\":10000,\"timeout\":\"1m\",\"_source\":{\"includes\":[\"balance\"],\"excludes\":[]},\"sort\":[{\"balance\":{\"order\":\"asc\",\"missing\":\"_first\"}}]}, requestedTotalSize=10000, pageSize=null, startFrom=0)])\n" } -} \ No newline at end of file +} diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_sort_complex_and_simple_expr.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_sort_complex_and_simple_expr.yaml index eaea0e0abcf..616e24c3b71 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_sort_complex_and_simple_expr.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_sort_complex_and_simple_expr.yaml @@ -6,5 +6,5 @@ calcite: LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12], _id=[$13], _index=[$14], _score=[$15], _maxscore=[$16], _sort=[$17], _routing=[$18], age2=[+($10, $7)], balance2=[+($7, 1)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]]) physical: | - EnumerableCalc(expr#0..12=[{inputs}], expr#13=[+($t10, $t7)], expr#14=[1], expr#15=[+($t7, $t14)], proj#0..13=[{exprs}], $f14=[$t15]) + EnumerableCalc(expr#0..12=[{inputs}], expr#13=[+($t10, $t7)], expr#14=[1], expr#15=[+($t7, $t14)], proj#0..13=[{exprs}], balance2=[$t15]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[PROJECT->[account_number, firstname, address, birthdate, gender, city, lastname, balance, employer, state, age, email, male], SORT_EXPR->[+($10, $7) ASCENDING NULLS_FIRST, balance ASCENDING NULLS_FIRST], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10000,"timeout":"1m","_source":{"includes":["account_number","firstname","address","birthdate","gender","city","lastname","balance","employer","state","age","email","male"],"excludes":[]},"sort":[{"_script":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQBQ3sKICAib3AiOiB7CiAgICAibmFtZSI6ICIrIiwKICAgICJraW5kIjogIlBMVVMiLAogICAgInN5bnRheCI6ICJCSU5BUlkiCiAgfSwKICAib3BlcmFuZHMiOiBbCiAgICB7CiAgICAgICJkeW5hbWljUGFyYW0iOiAwLAogICAgICAidHlwZSI6IHsKICAgICAgICAidHlwZSI6ICJCSUdJTlQiLAogICAgICAgICJudWxsYWJsZSI6IHRydWUKICAgICAgfQogICAgfSwKICAgIHsKICAgICAgImR5bmFtaWNQYXJhbSI6IDEsCiAgICAgICJ0eXBlIjogewogICAgICAgICJ0eXBlIjogIkJJR0lOVCIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZQogICAgICB9CiAgICB9CiAgXQp9\"}","lang":"opensearch_compounded_script","params":{"MISSING_MAX":false,"utcTimestamp": 0,"SOURCES":[0,0],"DIGESTS":["age","balance"]}},"type":"number","order":"asc"}},{"balance":{"order":"asc","missing":"_first"}}]}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_sort_pass_through_join_then_pushdown.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_sort_pass_through_join_then_pushdown.yaml index 36ae1037150..f3ba3bd9694 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_sort_pass_through_join_then_pushdown.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_sort_pass_through_join_then_pushdown.yaml @@ -12,11 +12,11 @@ calcite: physical: | CalciteEnumerableTopK(sort0=[$13], dir0=[ASC-nulls-first], fetch=[10000]) EnumerableMergeJoin(condition=[=($13, $15)], joinType=[left]) - EnumerableCalc(expr#0..12=[{inputs}], expr#13=['(?^[A-Z])'], expr#14=['initial'], expr#15=[REX_EXTRACT($t6, $t13, $t14)], proj#0..12=[{exprs}], $f13=[$t15]) + EnumerableCalc(expr#0..12=[{inputs}], expr#13=['(?^[A-Z])'], expr#14=['initial'], expr#15=[REX_EXTRACT($t6, $t13, $t14)], proj#0..12=[{exprs}], initial=[$t15]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[PROJECT->[account_number, firstname, address, birthdate, gender, city, lastname, balance, employer, state, age, email, male], SORT_EXPR->[REX_EXTRACT($6, '(?^[A-Z])', 'initial') ASCENDING NULLS_LAST]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["account_number","firstname","address","birthdate","gender","city","lastname","balance","employer","state","age","email","male"],"excludes":[]},"sort":[{"_script":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQC63sKICAib3AiOiB7CiAgICAibmFtZSI6ICJSRVhfRVhUUkFDVCIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfSwKICAgIHsKICAgICAgImR5bmFtaWNQYXJhbSI6IDEsCiAgICAgICJ0eXBlIjogewogICAgICAgICJ0eXBlIjogIlZBUkNIQVIiLAogICAgICAgICJudWxsYWJsZSI6IHRydWUsCiAgICAgICAgInByZWNpc2lvbiI6IC0xCiAgICAgIH0KICAgIH0sCiAgICB7CiAgICAgICJkeW5hbWljUGFyYW0iOiAyLAogICAgICAidHlwZSI6IHsKICAgICAgICAidHlwZSI6ICJWQVJDSEFSIiwKICAgICAgICAibnVsbGFibGUiOiB0cnVlLAogICAgICAgICJwcmVjaXNpb24iOiAtMQogICAgICB9CiAgICB9CiAgXSwKICAiY2xhc3MiOiAib3JnLm9wZW5zZWFyY2guc3FsLmV4cHJlc3Npb24uZnVuY3Rpb24uVXNlckRlZmluZWRGdW5jdGlvbkJ1aWxkZXIkMSIsCiAgInR5cGUiOiB7CiAgICAidHlwZSI6ICJWQVJDSEFSIiwKICAgICJudWxsYWJsZSI6IHRydWUsCiAgICAicHJlY2lzaW9uIjogMjAwMAogIH0sCiAgImRldGVybWluaXN0aWMiOiB0cnVlLAogICJkeW5hbWljIjogZmFsc2UKfQ==\"}","lang":"opensearch_compounded_script","params":{"MISSING_MAX":true,"utcTimestamp": 0,"SOURCES":[0,2,2],"DIGESTS":["lastname","(?^[A-Z])","initial"]}},"type":"string","order":"asc"}}]}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[PROJECT->[account_number, firstname, address, birthdate, gender, city, lastname, balance, employer, state, age, email, male], LIMIT->50000, SORT->[{ "firstname" : { "order" : "asc", "missing" : "_last" } - }]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":50000,"timeout":"1m","_source":{"includes":["account_number","firstname","address","birthdate","gender","city","lastname","balance","employer","state","age","email","male"],"excludes":[]},"sort":[{"firstname":{"order":"asc","missing":"_last"}}]}, requestedTotalSize=50000, pageSize=null, startFrom=0)]) \ No newline at end of file + }]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":50000,"timeout":"1m","_source":{"includes":["account_number","firstname","address","birthdate","gender","city","lastname","balance","employer","state","age","email","male"],"excludes":[]},"sort":[{"firstname":{"order":"asc","missing":"_last"}}]}, requestedTotalSize=50000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_global.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_global.yaml index 97703d849a7..b1b492f44a7 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_global.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_global.yaml @@ -16,7 +16,7 @@ calcite: EnumerableLimit(fetch=[10000]) EnumerableMergeJoin(condition=[AND(=($11, $15), =($12, $16), =($13, $17), IS NOT DISTINCT FROM($4, $14))], joinType=[left]) EnumerableSort(sort0=[$11], sort1=[$12], sort2=[$13], dir0=[ASC], dir1=[ASC], dir2=[ASC]) - EnumerableCalc(expr#0..11=[{inputs}], expr#12=[1], expr#13=[-($t11, $t12)], expr#14=[IS NULL($t4)], proj#0..11=[{exprs}], $f12=[$t13], $f13=[$t14]) + EnumerableCalc(expr#0..11=[{inputs}], expr#12=[1], expr#13=[-($t11, $t12)], expr#14=[IS NULL($t4)], proj#0..11=[{exprs}], $f12=[$t13], $f15=[$t14]) EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[account_number, firstname, address, balance, gender, city, employer, state, age, email, lastname]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["account_number","firstname","address","balance","gender","city","employer","state","age","email","lastname"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) EnumerableSort(sort0=[$1], sort1=[$2], sort2=[$3], dir0=[ASC], dir1=[ASC], dir2=[ASC]) @@ -24,8 +24,8 @@ calcite: EnumerableAggregate(group=[{0, 1, 2, 3}], agg#0=[$SUM0($5)], agg#1=[COUNT($5)]) EnumerableNestedLoopJoin(condition=[AND(>=($6, $2), <=($6, $1), OR(=($4, $0), AND(IS NULL($4), $3)))], joinType=[inner]) EnumerableAggregate(group=[{0, 1, 2, 3}]) - EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], expr#3=[-($t1, $t2)], expr#4=[IS NULL($t0)], proj#0..1=[{exprs}], $f2=[$t3], $f3=[$t4]) + EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], expr#3=[-($t1, $t2)], expr#4=[IS NULL($t0)], proj#0..1=[{exprs}], $f12=[$t3], $f15=[$t4]) EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[gender]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["gender"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[gender, age]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["gender","age"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[gender, age]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["gender","age"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_global_null_bucket.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_global_null_bucket.yaml index 3959777f707..24425578af2 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_global_null_bucket.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_global_null_bucket.yaml @@ -24,8 +24,8 @@ calcite: EnumerableAggregate(group=[{0, 1, 2}], agg#0=[$SUM0($4)], agg#1=[COUNT($4)]) EnumerableHashJoin(condition=[AND(=($0, $3), >=($5, $2), <=($5, $1))], joinType=[inner]) EnumerableAggregate(group=[{0, 1, 2}]) - EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], expr#3=[-($t1, $t2)], proj#0..1=[{exprs}], $f2=[$t3]) + EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], expr#3=[-($t1, $t2)], proj#0..1=[{exprs}], $f12=[$t3]) EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[gender]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["gender"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[gender, age]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["gender","age"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[gender, age]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["gender","age"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_timechart.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_timechart.yaml index 5dd080d68b5..b4acb2b0530 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_timechart.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_timechart.yaml @@ -35,5 +35,5 @@ calcite: EnumerableAggregate(group=[{0}], __grand_total__=[SUM($1)]) EnumerableCalc(expr#0..3=[{inputs}], expr#4=[0], expr#5=[=($t3, $t4)], expr#6=[null:DOUBLE], expr#7=[CASE($t5, $t6, $t2)], expr#8=[/($t7, $t3)], host=[$t0], avg(cpu_usage)=[$t8]) EnumerableAggregate(group=[{0, 2}], agg#0=[$SUM0($1)], agg#1=[COUNT($1)]) - EnumerableCalc(expr#0..2=[{inputs}], expr#3=[1], expr#4=['m'], expr#5=[SPAN($t2, $t3, $t4)], proj#0..1=[{exprs}], $f2=[$t5]) - CalciteEnumerableIndexScan(table=[[OpenSearch, events]], PushDownContext=[[PROJECT->[@timestamp, host, cpu_usage], FILTER->AND(IS NOT NULL($0), IS NOT NULL($2)), PROJECT->[host, cpu_usage, @timestamp], FILTER->IS NOT NULL($0)], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","query":{"bool":{"filter":[{"bool":{"must":[{"exists":{"field":"@timestamp","boost":1.0}},{"exists":{"field":"cpu_usage","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},{"exists":{"field":"host","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"_source":{"includes":["host","cpu_usage","@timestamp"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file + EnumerableCalc(expr#0..2=[{inputs}], expr#3=[1], expr#4=['m'], expr#5=[SPAN($t2, $t3, $t4)], proj#0..1=[{exprs}], @timestamp0=[$t5]) + CalciteEnumerableIndexScan(table=[[OpenSearch, events]], PushDownContext=[[PROJECT->[@timestamp, host, cpu_usage], FILTER->AND(IS NOT NULL($0), IS NOT NULL($2)), PROJECT->[host, cpu_usage, @timestamp], FILTER->IS NOT NULL($0)], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","query":{"bool":{"filter":[{"bool":{"must":[{"exists":{"field":"@timestamp","boost":1.0}},{"exists":{"field":"cpu_usage","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},{"exists":{"field":"host","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"_source":{"includes":["host","cpu_usage","@timestamp"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_complex_sort_expr_pushdown_for_smj_w_max_option.yaml b/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_complex_sort_expr_pushdown_for_smj_w_max_option.yaml index 07684889396..49cb168d0b7 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_complex_sort_expr_pushdown_for_smj_w_max_option.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_complex_sort_expr_pushdown_for_smj_w_max_option.yaml @@ -8,7 +8,7 @@ calcite: LogicalSystemLimit(fetch=[50000], type=[JOIN_SUBSEARCH_MAXOUT]) LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12]) LogicalFilter(condition=[<=($13, 1)]) - LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12], _row_number_join_max_dedup_=[ROW_NUMBER() OVER (PARTITION BY $6)]) + LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12], _row_number_dedup_=[ROW_NUMBER() OVER (PARTITION BY $6)]) LogicalFilter(condition=[IS NOT NULL($6)]) LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]]) @@ -24,4 +24,4 @@ calcite: EnumerableCalc(expr#0..19=[{inputs}], expr#20=[1], expr#21=[<=($t19, $t20)], proj#0..12=[{exprs}], $condition=[$t21]) EnumerableWindow(window#0=[window(partition {6} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) EnumerableCalc(expr#0..18=[{inputs}], expr#19=[IS NOT NULL($t6)], proj#0..18=[{exprs}], $condition=[$t19]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_dedup_keepempty_false_push.yaml b/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_dedup_keepempty_false_push.yaml index 6bb96baa243..8687247de22 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_dedup_keepempty_false_push.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_dedup_keepempty_false_push.yaml @@ -12,4 +12,4 @@ calcite: EnumerableCalc(expr#0..17=[{inputs}], expr#18=[1], expr#19=[<=($t17, $t18)], account_number=[$t0], gender=[$t4], age=[$t8], $condition=[$t19]) EnumerableWindow(window#0=[window(partition {4} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) EnumerableCalc(expr#0..16=[{inputs}], expr#17=[IS NOT NULL($t4)], proj#0..16=[{exprs}], $condition=[$t17]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_dedup_keepempty_true_not_pushed.yaml b/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_dedup_keepempty_true_not_pushed.yaml index 52d30ddf078..60ed46dbe9b 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_dedup_keepempty_true_not_pushed.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_dedup_keepempty_true_not_pushed.yaml @@ -9,4 +9,4 @@ calcite: EnumerableLimit(fetch=[10000]) EnumerableCalc(expr#0..17=[{inputs}], expr#18=[IS NULL($t4)], expr#19=[1], expr#20=[<=($t17, $t19)], expr#21=[OR($t18, $t20)], account_number=[$t0], gender=[$t4], age=[$t8], $condition=[$t21]) EnumerableWindow(window#0=[window(partition {4} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_dedup_push.yaml b/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_dedup_push.yaml index 6bb96baa243..8687247de22 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_dedup_push.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_dedup_push.yaml @@ -12,4 +12,4 @@ calcite: EnumerableCalc(expr#0..17=[{inputs}], expr#18=[1], expr#19=[<=($t17, $t18)], account_number=[$t0], gender=[$t4], age=[$t8], $condition=[$t19]) EnumerableWindow(window#0=[window(partition {4} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) EnumerableCalc(expr#0..16=[{inputs}], expr#17=[IS NOT NULL($t4)], proj#0..16=[{exprs}], $condition=[$t17]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_filter_push.yaml b/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_filter_push.yaml index d1f0cead8f6..ff9e2ed0ec1 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_filter_push.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_filter_push.yaml @@ -2,8 +2,10 @@ calcite: logical: | LogicalSystemLimit(fetch=[10000], type=[QUERY_SIZE_LIMIT]) LogicalProject(age=[$8]) - LogicalFilter(condition=[AND(SEARCH($8, Sarg[(30..40)]), >($3, 10000))]) - CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) + LogicalFilter(condition=[>($3, 10000)]) + LogicalFilter(condition=[<($8, 40)]) + LogicalFilter(condition=[>($8, 30)]) + CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | EnumerableLimit(fetch=[10000]) EnumerableCalc(expr#0..16=[{inputs}], expr#17=[Sarg[(30..40)]], expr#18=[SEARCH($t8, $t17)], expr#19=[10000], expr#20=[>($t3, $t19)], expr#21=[AND($t18, $t20)], age=[$t8], $condition=[$t21]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_filter_push_compare_date_string.yaml b/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_filter_push_compare_date_string.yaml index f8fcc3a23a7..a8f52a8ac7e 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_filter_push_compare_date_string.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_filter_push_compare_date_string.yaml @@ -1,9 +1,10 @@ calcite: logical: | LogicalSystemLimit(fetch=[10000], type=[QUERY_SIZE_LIMIT]) - LogicalFilter(condition=[AND(>($0, DATE('2016-12-08 00:00:00.123456789':VARCHAR)), <($0, DATE('2018-11-09 00:00:00.000000000':VARCHAR)))]) - LogicalProject(yyyy-MM-dd=[$83]) - CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_date_formats]]) + LogicalFilter(condition=[<($0, DATE('2018-11-09 00:00:00.000000000':VARCHAR))]) + LogicalFilter(condition=[>($0, DATE('2016-12-08 00:00:00.123456789':VARCHAR))]) + LogicalProject(yyyy-MM-dd=[$83]) + CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_date_formats]]) physical: | EnumerableLimit(fetch=[10000]) EnumerableCalc(expr#0..94=[{inputs}], expr#95=[Sarg[('2016-12-08':VARCHAR..'2018-11-09':VARCHAR)]:VARCHAR], expr#96=[SEARCH($t83, $t95)], yyyy-MM-dd=[$t83], $condition=[$t96]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_filter_push_compare_time_string.yaml b/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_filter_push_compare_time_string.yaml index 4634cfaaa47..72c738eaed0 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_filter_push_compare_time_string.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_filter_push_compare_time_string.yaml @@ -1,9 +1,10 @@ calcite: logical: | LogicalSystemLimit(fetch=[10000], type=[QUERY_SIZE_LIMIT]) - LogicalFilter(condition=[AND(>($0, TIME('2016-12-08 12:00:00.123456789':VARCHAR)), <($0, TIME('2018-11-09 19:00:00.123456789':VARCHAR)))]) - LogicalProject(custom_time=[$49]) - CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_date_formats]]) + LogicalFilter(condition=[<($0, TIME('2018-11-09 19:00:00.123456789':VARCHAR))]) + LogicalFilter(condition=[>($0, TIME('2016-12-08 12:00:00.123456789':VARCHAR))]) + LogicalProject(custom_time=[$49]) + CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_date_formats]]) physical: | EnumerableLimit(fetch=[10000]) EnumerableCalc(expr#0..94=[{inputs}], expr#95=[Sarg[('12:00:00.123456789':VARCHAR..'19:00:00.123456789':VARCHAR)]:VARCHAR], expr#96=[SEARCH($t49, $t95)], custom_time=[$t49], $condition=[$t96]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_filter_push_compare_timestamp_string.yaml b/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_filter_push_compare_timestamp_string.yaml index 20f0cbf4238..424444f8dc4 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_filter_push_compare_timestamp_string.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_filter_push_compare_timestamp_string.yaml @@ -2,8 +2,9 @@ calcite: logical: | LogicalSystemLimit(fetch=[10000], type=[QUERY_SIZE_LIMIT]) LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12]) - LogicalFilter(condition=[AND(>($3, TIMESTAMP('2016-12-08 00:00:00.000000000':VARCHAR)), <($3, TIMESTAMP('2018-11-09 00:00:00.000000000':VARCHAR)))]) - CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]]) + LogicalFilter(condition=[<($3, TIMESTAMP('2018-11-09 00:00:00.000000000':VARCHAR))]) + LogicalFilter(condition=[>($3, TIMESTAMP('2016-12-08 00:00:00.000000000':VARCHAR))]) + CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]]) physical: | EnumerableLimit(fetch=[10000]) EnumerableCalc(expr#0..18=[{inputs}], expr#19=[Sarg[('2016-12-08 00:00:00':VARCHAR..'2018-11-09 00:00:00':VARCHAR)]:VARCHAR], expr#20=[SEARCH($t3, $t19)], proj#0..12=[{exprs}], $condition=[$t20]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_join_with_criteria_max_option.yaml b/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_join_with_criteria_max_option.yaml index be2c210caf2..6a3cc19bd3d 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_join_with_criteria_max_option.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_join_with_criteria_max_option.yaml @@ -8,7 +8,7 @@ calcite: LogicalSystemLimit(fetch=[50000], type=[JOIN_SUBSEARCH_MAXOUT]) LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12]) LogicalFilter(condition=[<=($13, 1)]) - LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12], _row_number_join_max_dedup_=[ROW_NUMBER() OVER (PARTITION BY $0)]) + LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12], _row_number_dedup_=[ROW_NUMBER() OVER (PARTITION BY $0)]) LogicalFilter(condition=[IS NOT NULL($0)]) LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]]) @@ -23,4 +23,4 @@ calcite: EnumerableCalc(expr#0..19=[{inputs}], expr#20=[1], expr#21=[<=($t19, $t20)], proj#0..12=[{exprs}], $condition=[$t21]) EnumerableWindow(window#0=[window(partition {0} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) EnumerableCalc(expr#0..18=[{inputs}], expr#19=[IS NOT NULL($t0)], proj#0..18=[{exprs}], $condition=[$t19]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_join_with_fields_max_option.yaml b/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_join_with_fields_max_option.yaml index 84ce94c02e8..46129c1c5b0 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_join_with_fields_max_option.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_join_with_fields_max_option.yaml @@ -8,7 +8,7 @@ calcite: LogicalSystemLimit(fetch=[50000], type=[JOIN_SUBSEARCH_MAXOUT]) LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12]) LogicalFilter(condition=[<=($13, 1)]) - LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12], _row_number_join_max_dedup_=[ROW_NUMBER() OVER (PARTITION BY $0)]) + LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12], _row_number_dedup_=[ROW_NUMBER() OVER (PARTITION BY $0)]) LogicalFilter(condition=[IS NOT NULL($0)]) LogicalProject(account_number=[$0], firstname=[$1], address=[$2], birthdate=[$3], gender=[$4], city=[$5], lastname=[$6], balance=[$7], employer=[$8], state=[$9], age=[$10], email=[$11], male=[$12]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]]) @@ -24,4 +24,4 @@ calcite: EnumerableCalc(expr#0..19=[{inputs}], expr#20=[1], expr#21=[<=($t19, $t20)], proj#0..12=[{exprs}], $condition=[$t21]) EnumerableWindow(window#0=[window(partition {0} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) EnumerableCalc(expr#0..18=[{inputs}], expr#19=[IS NOT NULL($t0)], proj#0..18=[{exprs}], $condition=[$t19]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_output.yaml b/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_output.yaml index 2a1449e4b1f..4cb67a380a5 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_output.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite_no_pushdown/explain_output.yaml @@ -13,12 +13,11 @@ calcite: LogicalFilter(condition=[>($8, 30)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - EnumerableCalc(expr#0..2=[{inputs}], age2=[$t1]) - EnumerableLimit(fetch=[10000]) - EnumerableCalc(expr#0..2=[{inputs}], expr#3=[1], expr#4=[<=($t2, $t3)], proj#0..2=[{exprs}], $condition=[$t4]) - EnumerableWindow(window#0=[window(partition {1} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) - EnumerableCalc(expr#0..3=[{inputs}], expr#4=[0], expr#5=[=($t3, $t4)], expr#6=[null:BIGINT], expr#7=[CASE($t5, $t6, $t2)], expr#8=[CAST($t7):DOUBLE], expr#9=[/($t8, $t3)], expr#10=[2], expr#11=[+($t9, $t10)], expr#12=[IS NOT NULL($t8)], state=[$t1], age2=[$t11], $condition=[$t12]) - EnumerableSort(sort0=[$1], dir0=[ASC-nulls-first]) - EnumerableAggregate(group=[{5, 7}], agg#0=[$SUM0($8)], agg#1=[COUNT($8)]) - EnumerableCalc(expr#0..16=[{inputs}], expr#17=[30], expr#18=[>($t8, $t17)], proj#0..16=[{exprs}], $condition=[$t18]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) \ No newline at end of file + EnumerableLimit(fetch=[10000]) + EnumerableCalc(expr#0..2=[{inputs}], expr#3=[1], expr#4=[<=($t2, $t3)], age2=[$t1], $condition=[$t4]) + EnumerableWindow(window#0=[window(partition {1} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) + EnumerableCalc(expr#0..3=[{inputs}], expr#4=[0], expr#5=[=($t3, $t4)], expr#6=[null:BIGINT], expr#7=[CASE($t5, $t6, $t2)], expr#8=[CAST($t7):DOUBLE], expr#9=[/($t8, $t3)], expr#10=[2], expr#11=[+($t9, $t10)], expr#12=[IS NOT NULL($t8)], state=[$t1], age2=[$t11], $condition=[$t12]) + EnumerableSort(sort0=[$1], dir0=[ASC-nulls-first]) + EnumerableAggregate(group=[{5, 7}], agg#0=[$SUM0($8)], agg#1=[COUNT($8)]) + EnumerableCalc(expr#0..16=[{inputs}], expr#17=[30], expr#18=[>($t8, $t17)], proj#0..16=[{exprs}], $condition=[$t18]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/AggregateIndexScanRule.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/AggregateIndexScanRule.java index 4154a10ce51..bd9f17abd2a 100644 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/AggregateIndexScanRule.java +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/AggregateIndexScanRule.java @@ -23,7 +23,7 @@ import org.apache.calcite.rex.RexNode; import org.apache.calcite.tools.RelBuilder; import org.immutables.value.Value; -import org.opensearch.sql.calcite.plan.OpenSearchRuleConfig; +import org.opensearch.sql.calcite.plan.rule.OpenSearchRuleConfig; import org.opensearch.sql.calcite.utils.PlanUtils; import org.opensearch.sql.expression.function.udf.binning.WidthBucketFunction; import org.opensearch.sql.opensearch.storage.scan.AbstractCalciteIndexScan; diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/DedupPushdownRule.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/DedupPushdownRule.java index 57e8045f2fe..f8899ead2eb 100644 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/DedupPushdownRule.java +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/DedupPushdownRule.java @@ -6,27 +6,25 @@ package org.opensearch.sql.opensearch.planner.rules; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; +import java.util.Set; import java.util.function.Predicate; import java.util.stream.IntStream; import org.apache.calcite.plan.RelOptRuleCall; import org.apache.calcite.rel.logical.LogicalAggregate; -import org.apache.calcite.rel.logical.LogicalFilter; import org.apache.calcite.rel.logical.LogicalProject; import org.apache.calcite.rel.rules.SubstitutionRule; -import org.apache.calcite.rex.RexCall; import org.apache.calcite.rex.RexInputRef; -import org.apache.calcite.rex.RexLiteral; import org.apache.calcite.rex.RexNode; -import org.apache.calcite.rex.RexWindow; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.tools.RelBuilder; -import org.apache.calcite.util.Pair; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.immutables.value.Value; -import org.opensearch.sql.calcite.plan.OpenSearchRuleConfig; +import org.opensearch.sql.calcite.plan.rel.LogicalDedup; +import org.opensearch.sql.calcite.plan.rule.OpenSearchRuleConfig; import org.opensearch.sql.calcite.utils.PPLHintUtils; import org.opensearch.sql.calcite.utils.PlanUtils; import org.opensearch.sql.opensearch.storage.scan.AbstractCalciteIndexScan; @@ -44,28 +42,19 @@ protected DedupPushdownRule(Config config) { @Override protected void onMatchImpl(RelOptRuleCall call) { - final LogicalProject finalProject = call.rel(0); - // TODO Used when number of duplication is more than 1 - final LogicalFilter numOfDedupFilter = call.rel(1); - final LogicalProject projectWithWindow = call.rel(2); - final LogicalProject projectWithExpr = call.rel(4); - final CalciteLogicalIndexScan scan = call.rel(5); - apply(call, finalProject, numOfDedupFilter, projectWithWindow, projectWithExpr, scan); + final LogicalDedup logicalDedup = call.rel(0); + final LogicalProject projectWithExpr = call.rel(1); + final CalciteLogicalIndexScan scan = call.rel(2); + apply(call, logicalDedup, projectWithExpr, scan); } protected void apply( RelOptRuleCall call, - LogicalProject finalProject, - LogicalFilter numOfDedupFilter, - LogicalProject projectWithWindow, - LogicalProject bottomProject, + LogicalDedup dedup, + LogicalProject project, CalciteLogicalIndexScan scan) { - List windows = PlanUtils.getRexWindowFromProject(projectWithWindow); - if (windows.size() != 1) { - return; - } - List dedupColumns = windows.get(0).partitionKeys; + List dedupColumns = dedup.getDedupeFields(); if (dedupColumns.stream() .filter(rex -> rex.isA(SqlKind.INPUT_REF)) .anyMatch( @@ -77,37 +66,32 @@ protected void apply( // fallback to non-pushdown return; } - // must be row_number <= number - assert numOfDedupFilter.getCondition().isA(SqlKind.LESS_THAN_OR_EQUAL); - RexLiteral literal = - (RexLiteral) ((RexCall) numOfDedupFilter.getCondition()).getOperands().getLast(); - Integer dedupNumer = literal.getValueAs(Integer.class); RelBuilder relBuilder = call.builder(); - relBuilder.push(bottomProject); - - // 1 Build the target projections - // 1.1 The dedup columns should be put first as it will be used in the aggregation below; - // The dedup columns may source from the current projectWithWindow or the bottomProject - // 1.2 The rest of the columns should be put after the dedup columns, which is force needed in - // dedup. - List> targetProjections = new ArrayList<>(); + relBuilder.push(project); + + List targetProjections = new ArrayList<>(); + Set dedupFieldsIndexSet = new HashSet<>(); for (RexNode dedupColumn : dedupColumns) { - if (projectWithWindow.getProjects().contains(dedupColumn)) { - targetProjections.add( - projectWithWindow - .getNamedProjects() - .get(projectWithWindow.getProjects().indexOf(dedupColumn))); - } else if (dedupColumn instanceof RexInputRef ref) { - targetProjections.add( - Pair.of( - dedupColumn, relBuilder.peek().getRowType().getFieldNames().get(ref.getIndex()))); + if (dedupColumn instanceof RexInputRef ref) { + targetProjections.add(dedupColumn); + dedupFieldsIndexSet.add(ref.getIndex()); } else { LOG.warn("The dedup column {} is illegal.", dedupColumn); return; } } - if (targetProjections.stream() + IntStream.range(0, project.getProjects().size()) + .boxed() + .filter(index -> !dedupFieldsIndexSet.contains(index)) + .map(relBuilder::field) + .forEach(targetProjections::add); + + relBuilder.project(targetProjections); + LogicalProject targetChildProject = (LogicalProject) relBuilder.peek(); + + if (targetChildProject.getNamedProjects().stream() + .limit(dedupColumns.size()) .anyMatch( pair -> Utils.resolveNestedPath(pair.getValue(), scan.getOsIndex().getFieldTypes()) @@ -115,16 +99,6 @@ protected void apply( // fallback to non-pushdown if the dedup columns contain nested fields. return; } - for (Pair project : projectWithWindow.getNamedProjects()) { - if (!project.getKey().isA(SqlKind.ROW_NUMBER) && !targetProjections.contains(project)) { - targetProjections.add(project); - } - } - - relBuilder.project( - targetProjections.stream().map(Pair::getKey).toList(), - targetProjections.stream().map(Pair::getValue).toList()); - LogicalProject targetChildProject = (LogicalProject) relBuilder.peek(); // 2 Push an Aggregate // We push down a LITERAL_AGG with dedupNumer for converting the dedup command to aggregate: @@ -133,7 +107,8 @@ protected void apply( // (LITERAL_AGG is used in optimization normally, see {@link SqlKind#LITERAL_AGG}) List newGroupByList = IntStream.range(0, dedupColumns.size()).boxed().toList(); relBuilder.aggregate( - relBuilder.groupKey(relBuilder.fields(newGroupByList)), relBuilder.literalAgg(dedupNumer)); + relBuilder.groupKey(relBuilder.fields(newGroupByList)), + relBuilder.literalAgg(dedup.getAllowedDuplication())); // add bucket_nullable = false hint PPLHintUtils.addIgnoreNullBucketHintToAggregate(relBuilder); @@ -146,53 +121,33 @@ protected void apply( (CalciteLogicalIndexScan) scan.pushDownAggregate(aggregate, targetChildProject); if (newScan != null) { // Back to original project order - call.transformTo(newScan.copyWithNewSchema(finalProject.getRowType())); + call.transformTo(newScan.copyWithNewSchema(dedup.getRowType())); PlanUtils.tryPruneRelNodes(call); } } @Value.Immutable public interface Config extends OpenSearchRuleConfig { - // +- LogicalProject(no _row_number_dedup_) - // +- LogicalFilter(condition contains _row_number_dedup_) - // +- LogicalProject(contains _row_number_dedup_) - // +- LogicalFilter(condition IS NOT NULL(dedupColumn)) - // +- LogicalProject(dedupColumn is call or ref) - // +- CalciteLogicalIndexScan + // +- LogicalDedup + // +- LogicalProject + // +- CalciteLogicalIndexScan Config DEFAULT = ImmutableDedupPushdownRule.Config.builder() .build() - .withDescription("DedupWithExpression-to-Aggregate") + .withDescription("Dedup-to-Aggregate") .withOperandSupplier( b0 -> - b0.operand(LogicalProject.class) - .predicate(Predicate.not(PlanUtils::containsRowNumberDedup)) + b0.operand(LogicalDedup.class) + // Cannot push dedup operator if keepEmpty=true + .predicate(dedup -> !dedup.getKeepEmpty()) .oneInput( b1 -> - b1.operand(LogicalFilter.class) - .predicate(Config::validDedupNumberChecker) + b1.operand(LogicalProject.class) .oneInput( b2 -> - b2.operand(LogicalProject.class) - .predicate(PlanUtils::containsRowNumberDedup) - .oneInput( - b3 -> - b3.operand(LogicalFilter.class) - .predicate( - PlanUtils - ::mayBeFilterFromBucketNonNull) - .oneInput( - b4 -> - b4.operand(LogicalProject.class) - .oneInput( - b5 -> - b5.operand( - CalciteLogicalIndexScan - .class) - .predicate( - Config - ::tableScanChecker) - .noInputs())))))); + b2.operand(CalciteLogicalIndexScan.class) + .predicate(Config::tableScanChecker) + .noInputs()))); /** * Project must be not pushed since the name of expression would lose after project pushed. E.g. @@ -208,10 +163,5 @@ private static boolean tableScanChecker(AbstractCalciteIndexScan scan) { default DedupPushdownRule toRule() { return new DedupPushdownRule(this); } - - private static boolean validDedupNumberChecker(LogicalFilter filter) { - return filter.getCondition().isA(SqlKind.LESS_THAN_OR_EQUAL) - && PlanUtils.containsRowNumberDedup(filter); - } } } diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/EnumerableTopKMergeRule.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/EnumerableTopKMergeRule.java index e0d034fb5ae..e904a728439 100644 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/EnumerableTopKMergeRule.java +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/EnumerableTopKMergeRule.java @@ -9,7 +9,7 @@ import org.apache.calcite.adapter.enumerable.EnumerableSort; import org.apache.calcite.plan.RelOptRuleCall; import org.immutables.value.Value; -import org.opensearch.sql.calcite.plan.OpenSearchRuleConfig; +import org.opensearch.sql.calcite.plan.rule.OpenSearchRuleConfig; import org.opensearch.sql.opensearch.planner.physical.CalciteEnumerableTopK; /** diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/ExpandCollationOnProjectExprRule.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/ExpandCollationOnProjectExprRule.java index cc7890ddc78..a84598b834a 100644 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/ExpandCollationOnProjectExprRule.java +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/ExpandCollationOnProjectExprRule.java @@ -24,7 +24,7 @@ import org.apache.calcite.rel.core.Project; import org.apache.commons.lang3.tuple.Pair; import org.immutables.value.Value; -import org.opensearch.sql.calcite.plan.OpenSearchRuleConfig; +import org.opensearch.sql.calcite.plan.rule.OpenSearchRuleConfig; import org.opensearch.sql.calcite.utils.PlanUtils; import org.opensearch.sql.opensearch.storage.scan.CalciteEnumerableIndexScan; import org.opensearch.sql.opensearch.util.OpenSearchRelOptUtil; diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/FilterIndexScanRule.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/FilterIndexScanRule.java index 9cf6d297ada..f74cefce7df 100644 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/FilterIndexScanRule.java +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/FilterIndexScanRule.java @@ -11,7 +11,7 @@ import org.apache.calcite.rel.core.Filter; import org.apache.calcite.rel.logical.LogicalFilter; import org.immutables.value.Value; -import org.opensearch.sql.calcite.plan.OpenSearchRuleConfig; +import org.opensearch.sql.calcite.plan.rule.OpenSearchRuleConfig; import org.opensearch.sql.calcite.utils.PlanUtils; import org.opensearch.sql.opensearch.storage.scan.AbstractCalciteIndexScan; import org.opensearch.sql.opensearch.storage.scan.CalciteLogicalIndexScan; diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/InterruptibleRelRule.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/InterruptibleRelRule.java index 59e94a4757c..8046f3fe090 100644 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/InterruptibleRelRule.java +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/InterruptibleRelRule.java @@ -8,7 +8,7 @@ import org.apache.calcite.plan.RelOptRuleCall; import org.apache.calcite.plan.RelRule; import org.opensearch.OpenSearchTimeoutException; -import org.opensearch.sql.calcite.plan.OpenSearchRuleConfig; +import org.opensearch.sql.calcite.plan.rule.OpenSearchRuleConfig; /** * Base class for OpenSearch planner rules that automatically checks for thread interruption during diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/LimitIndexScanRule.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/LimitIndexScanRule.java index be84c8e5c4e..355e2c1dfd4 100644 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/LimitIndexScanRule.java +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/LimitIndexScanRule.java @@ -12,7 +12,7 @@ import org.apache.calcite.rex.RexLiteral; import org.apache.calcite.rex.RexNode; import org.immutables.value.Value; -import org.opensearch.sql.calcite.plan.OpenSearchRuleConfig; +import org.opensearch.sql.calcite.plan.rule.OpenSearchRuleConfig; import org.opensearch.sql.calcite.utils.PlanUtils; import org.opensearch.sql.opensearch.storage.scan.CalciteLogicalIndexScan; diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/ProjectIndexScanRule.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/ProjectIndexScanRule.java index 4561691ed9e..8248ad93030 100644 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/ProjectIndexScanRule.java +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/ProjectIndexScanRule.java @@ -20,7 +20,7 @@ import org.apache.calcite.util.mapping.Mapping; import org.apache.calcite.util.mapping.Mappings; import org.immutables.value.Value; -import org.opensearch.sql.calcite.plan.OpenSearchRuleConfig; +import org.opensearch.sql.calcite.plan.rule.OpenSearchRuleConfig; import org.opensearch.sql.calcite.utils.PlanUtils; import org.opensearch.sql.opensearch.storage.OpenSearchIndex; import org.opensearch.sql.opensearch.storage.scan.CalciteLogicalIndexScan; @@ -78,7 +78,11 @@ public Void visitInputRef(RexInputRef inputRef) { if (RexUtil.isIdentity(newProjectRexNodes, newScan.getRowType())) { call.transformTo(newScan); } else { - call.transformTo(call.builder().push(newScan).project(newProjectRexNodes).build()); + call.transformTo( + call.builder() + .push(newScan) + .project(newProjectRexNodes, project.getRowType().getFieldNames()) + .build()); } PlanUtils.tryPruneRelNodes(call); } diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/RareTopPushdownRule.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/RareTopPushdownRule.java index 370b4131ed7..5a815e87456 100644 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/RareTopPushdownRule.java +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/RareTopPushdownRule.java @@ -17,7 +17,7 @@ import org.apache.calcite.rex.RexWindow; import org.apache.calcite.sql.SqlKind; import org.immutables.value.Value; -import org.opensearch.sql.calcite.plan.OpenSearchRuleConfig; +import org.opensearch.sql.calcite.plan.rule.OpenSearchRuleConfig; import org.opensearch.sql.calcite.utils.PlanUtils; import org.opensearch.sql.opensearch.storage.scan.AbstractCalciteIndexScan; import org.opensearch.sql.opensearch.storage.scan.CalciteLogicalIndexScan; diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/RelevanceFunctionPushdownRule.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/RelevanceFunctionPushdownRule.java index 2e27e0e871b..b49b62e0163 100644 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/RelevanceFunctionPushdownRule.java +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/RelevanceFunctionPushdownRule.java @@ -18,7 +18,7 @@ import org.apache.calcite.rex.RexVisitorImpl; import org.apache.calcite.sql.SqlOperator; import org.immutables.value.Value; -import org.opensearch.sql.calcite.plan.OpenSearchRuleConfig; +import org.opensearch.sql.calcite.plan.rule.OpenSearchRuleConfig; import org.opensearch.sql.calcite.utils.PlanUtils; import org.opensearch.sql.opensearch.storage.scan.CalciteLogicalIndexScan; diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/SortAggregateMeasureRule.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/SortAggregateMeasureRule.java index b05db3203a3..9dacc1bccdc 100644 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/SortAggregateMeasureRule.java +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/SortAggregateMeasureRule.java @@ -10,7 +10,7 @@ import org.apache.calcite.rel.core.Sort; import org.apache.calcite.rel.logical.LogicalSort; import org.immutables.value.Value; -import org.opensearch.sql.calcite.plan.OpenSearchRuleConfig; +import org.opensearch.sql.calcite.plan.rule.OpenSearchRuleConfig; import org.opensearch.sql.calcite.utils.PlanUtils; import org.opensearch.sql.opensearch.storage.scan.AbstractCalciteIndexScan; import org.opensearch.sql.opensearch.storage.scan.CalciteLogicalIndexScan; diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/SortExprIndexScanRule.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/SortExprIndexScanRule.java index 5c49dd6b7d2..b7c25912bca 100644 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/SortExprIndexScanRule.java +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/SortExprIndexScanRule.java @@ -25,7 +25,7 @@ import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.Pair; import org.immutables.value.Value; -import org.opensearch.sql.calcite.plan.OpenSearchRuleConfig; +import org.opensearch.sql.calcite.plan.rule.OpenSearchRuleConfig; import org.opensearch.sql.calcite.utils.PlanUtils; import org.opensearch.sql.opensearch.storage.scan.AbstractCalciteIndexScan; import org.opensearch.sql.opensearch.storage.scan.CalciteLogicalIndexScan; diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/SortIndexScanRule.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/SortIndexScanRule.java index ca1d5384f59..c10c7cf488d 100644 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/SortIndexScanRule.java +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/SortIndexScanRule.java @@ -9,7 +9,7 @@ import org.apache.calcite.plan.RelOptRuleCall; import org.apache.calcite.rel.core.Sort; import org.immutables.value.Value; -import org.opensearch.sql.calcite.plan.OpenSearchRuleConfig; +import org.opensearch.sql.calcite.plan.rule.OpenSearchRuleConfig; import org.opensearch.sql.calcite.utils.PlanUtils; import org.opensearch.sql.opensearch.planner.physical.CalciteEnumerableTopK; import org.opensearch.sql.opensearch.storage.scan.AbstractCalciteIndexScan; diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/SortProjectExprTransposeRule.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/SortProjectExprTransposeRule.java index 9136b72a246..ad0ad0f2d0b 100644 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/SortProjectExprTransposeRule.java +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/SortProjectExprTransposeRule.java @@ -23,7 +23,7 @@ import org.apache.calcite.rex.RexNode; import org.apache.commons.lang3.tuple.Pair; import org.immutables.value.Value; -import org.opensearch.sql.calcite.plan.OpenSearchRuleConfig; +import org.opensearch.sql.calcite.plan.rule.OpenSearchRuleConfig; import org.opensearch.sql.calcite.utils.PlanUtils; import org.opensearch.sql.opensearch.util.OpenSearchRelOptUtil; diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/storage/scan/CalciteEnumerableIndexScan.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/storage/scan/CalciteEnumerableIndexScan.java index 9bbc80155de..7ba75e46ba0 100644 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/storage/scan/CalciteEnumerableIndexScan.java +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/storage/scan/CalciteEnumerableIndexScan.java @@ -27,8 +27,8 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.checkerframework.checker.nullness.qual.Nullable; -import org.opensearch.sql.calcite.plan.OpenSearchRules; import org.opensearch.sql.calcite.plan.Scannable; +import org.opensearch.sql.calcite.plan.rule.OpenSearchRules; import org.opensearch.sql.opensearch.request.OpenSearchRequestBuilder; import org.opensearch.sql.opensearch.storage.OpenSearchIndex; import org.opensearch.sql.opensearch.storage.scan.context.PushDownContext; diff --git a/ppl/src/test/java/org/opensearch/sql/ppl/calcite/CalcitePPLJoinTest.java b/ppl/src/test/java/org/opensearch/sql/ppl/calcite/CalcitePPLJoinTest.java index a8b6873b55b..e7ad1f6448c 100644 --- a/ppl/src/test/java/org/opensearch/sql/ppl/calcite/CalcitePPLJoinTest.java +++ b/ppl/src/test/java/org/opensearch/sql/ppl/calcite/CalcitePPLJoinTest.java @@ -1002,7 +1002,7 @@ public void testJoinWithFieldListMaxGreaterThanZero() { + " LogicalProject(DEPTNO=[$0], DNAME=[$1], LOC=[$2])\n" + " LogicalFilter(condition=[<=($3, 1)])\n" + " LogicalProject(DEPTNO=[$0], DNAME=[$1], LOC=[$2]," - + " _row_number_join_max_dedup_=[ROW_NUMBER() OVER (PARTITION BY $0)])\n" + + " _row_number_dedup_=[ROW_NUMBER() OVER (PARTITION BY $0)])\n" + " LogicalTableScan(table=[[scott, DEPT]])\n"; verifyLogical(root, expectedLogical); verifyResultCount(root, 14); @@ -1013,9 +1013,9 @@ public void testJoinWithFieldListMaxGreaterThanZero() { + "FROM `scott`.`EMP`\n" + "LEFT JOIN (SELECT `DEPTNO`, `DNAME`, `LOC`\n" + "FROM (SELECT `DEPTNO`, `DNAME`, `LOC`, ROW_NUMBER() OVER (PARTITION BY `DEPTNO`)" - + " `_row_number_join_max_dedup_`\n" + + " `_row_number_dedup_`\n" + "FROM `scott`.`DEPT`) `t`\n" - + "WHERE `_row_number_join_max_dedup_` <= 1) `t1` ON `EMP`.`DEPTNO` = `t1`.`DEPTNO`"; + + "WHERE `_row_number_dedup_` <= 1) `t1` ON `EMP`.`DEPTNO` = `t1`.`DEPTNO`"; verifyPPLToSparkSQL(root, expectedSparkSql); } @@ -1031,7 +1031,7 @@ public void testJoinWithCriteriaMaxGreaterThanZero() { + " LogicalProject(DEPTNO=[$0], DNAME=[$1], LOC=[$2])\n" + " LogicalFilter(condition=[<=($3, 1)])\n" + " LogicalProject(DEPTNO=[$0], DNAME=[$1], LOC=[$2]," - + " _row_number_join_max_dedup_=[ROW_NUMBER() OVER (PARTITION BY $0)])\n" + + " _row_number_dedup_=[ROW_NUMBER() OVER (PARTITION BY $0)])\n" + " LogicalTableScan(table=[[scott, DEPT]])\n"; verifyLogical(root, expectedLogical); verifyResultCount(root, 14); @@ -1043,9 +1043,9 @@ public void testJoinWithCriteriaMaxGreaterThanZero() { + "FROM `scott`.`EMP`\n" + "LEFT JOIN (SELECT `DEPTNO`, `DNAME`, `LOC`\n" + "FROM (SELECT `DEPTNO`, `DNAME`, `LOC`, ROW_NUMBER() OVER (PARTITION BY `DEPTNO`)" - + " `_row_number_join_max_dedup_`\n" + + " `_row_number_dedup_`\n" + "FROM `scott`.`DEPT`) `t`\n" - + "WHERE `_row_number_join_max_dedup_` <= 1) `t1` ON `EMP`.`DEPTNO` = `t1`.`DEPTNO`"; + + "WHERE `_row_number_dedup_` <= 1) `t1` ON `EMP`.`DEPTNO` = `t1`.`DEPTNO`"; verifyPPLToSparkSQL(root, expectedSparkSql); }