diff --git a/core/src/main/java/org/opensearch/sql/calcite/plan/PPLAggGroupMergeRule.java b/core/src/main/java/org/opensearch/sql/calcite/plan/PPLAggGroupMergeRule.java index 019de7fec01..97fde04768f 100644 --- a/core/src/main/java/org/opensearch/sql/calcite/plan/PPLAggGroupMergeRule.java +++ b/core/src/main/java/org/opensearch/sql/calcite/plan/PPLAggGroupMergeRule.java @@ -15,6 +15,7 @@ import org.apache.calcite.plan.RelRule; import org.apache.calcite.rel.logical.LogicalAggregate; 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.RexNode; @@ -27,6 +28,7 @@ import org.apache.commons.lang3.tuple.Pair; import org.immutables.value.Value; import org.opensearch.sql.calcite.utils.CalciteUtils; +import org.opensearch.sql.calcite.utils.PlanUtils; /** * Planner rule that merge multiple agg group fields into a single one, on which all other group @@ -41,7 +43,8 @@ * these UDFs' output must have equivalent cardinality as `a`. */ @Value.Enclosing -public class PPLAggGroupMergeRule extends RelRule { +public class PPLAggGroupMergeRule extends RelRule + implements SubstitutionRule { /** Creates a OpenSearchAggregateConvertRule. */ protected PPLAggGroupMergeRule(Config config) { @@ -101,6 +104,7 @@ public void apply(RelOptRuleCall call, LogicalAggregate aggregate, LogicalProjec parentProjections.addAll(aggCallRefs); relBuilder.project(parentProjections); call.transformTo(relBuilder.build()); + PlanUtils.tryPruneRelNodes(call); } /** Rule configuration. */ diff --git a/core/src/main/java/org/opensearch/sql/calcite/plan/PPLAggregateConvertRule.java b/core/src/main/java/org/opensearch/sql/calcite/plan/PPLAggregateConvertRule.java index c3069d85668..fee2d02e086 100644 --- a/core/src/main/java/org/opensearch/sql/calcite/plan/PPLAggregateConvertRule.java +++ b/core/src/main/java/org/opensearch/sql/calcite/plan/PPLAggregateConvertRule.java @@ -17,6 +17,7 @@ import org.apache.calcite.rel.core.Project; import org.apache.calcite.rel.logical.LogicalAggregate; import org.apache.calcite.rel.logical.LogicalProject; +import org.apache.calcite.rel.rules.SubstitutionRule; import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.rex.RexCall; import org.apache.calcite.rex.RexInputRef; @@ -28,6 +29,7 @@ import org.apache.calcite.tools.RelBuilder; import org.apache.commons.lang3.tuple.Pair; import org.immutables.value.Value; +import org.opensearch.sql.calcite.utils.PlanUtils; /** * Planner rule that converts specific aggCall to a more efficient expressions, which includes: @@ -45,8 +47,8 @@ *

- AVG/MAX/MIN(FIELD [+|-|*|+|/] NUMBER) -> AVG/MAX/MIN(FIELD) [+|-|*|+|/] NUMBER */ @Value.Enclosing -public class PPLAggregateConvertRule extends RelRule { - +public class PPLAggregateConvertRule extends RelRule + implements SubstitutionRule { /** Creates a OpenSearchAggregateConvertRule. */ protected PPLAggregateConvertRule(Config config) { super(config); @@ -97,6 +99,8 @@ public void apply(RelOptRuleCall call, LogicalAggregate aggregate, LogicalProjec return ref; }) .toList(); + // Stop processing if there is no converted agg call args + if (convertedAggCallArgs.isEmpty()) return; relBuilder.project(newChildProjects); RelNode newInput = relBuilder.peek(); @@ -193,6 +197,7 @@ public void apply(RelOptRuleCall call, LogicalAggregate aggregate, LogicalProjec aliasMaybe(relBuilder, constructor.apply(relBuilder.peek()), name))); relBuilder.project(parentProjects); call.transformTo(relBuilder.build()); + PlanUtils.tryPruneRelNodes(call); } interface OperatorConstructor { 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 55f911cc4c8..d89c36601ef 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 @@ -10,21 +10,28 @@ import static org.apache.calcite.rex.RexWindowBounds.UNBOUNDED_PRECEDING; import static org.apache.calcite.rex.RexWindowBounds.following; import static org.apache.calcite.rex.RexWindowBounds.preceding; +import static org.opensearch.sql.calcite.utils.OpenSearchTypeFactory.isTimeBasedType; import com.google.common.collect.ImmutableList; import java.lang.reflect.Method; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashSet; import java.util.List; import java.util.Objects; import java.util.Set; +import java.util.function.Function; import java.util.function.Predicate; import java.util.stream.Collectors; import javax.annotation.Nullable; +import org.apache.calcite.plan.Convention; +import org.apache.calcite.plan.RelOptRuleCall; import org.apache.calcite.plan.RelOptTable; +import org.apache.calcite.plan.volcano.VolcanoPlanner; import org.apache.calcite.rel.RelHomogeneousShuttle; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.RelShuttle; +import org.apache.calcite.rel.core.Aggregate; import org.apache.calcite.rel.core.AggregateCall; import org.apache.calcite.rel.core.Project; import org.apache.calcite.rel.core.Sort; @@ -41,6 +48,7 @@ import org.apache.calcite.rex.RexLiteral; import org.apache.calcite.rex.RexNode; import org.apache.calcite.rex.RexOver; +import org.apache.calcite.rex.RexSlot; import org.apache.calcite.rex.RexVisitorImpl; import org.apache.calcite.rex.RexWindow; import org.apache.calcite.rex.RexWindowBound; @@ -646,4 +654,97 @@ private static boolean isNotNullOnRef(RexNode rex) { && rexCall.isA(SqlKind.IS_NOT_NULL) && rexCall.getOperands().get(0) instanceof RexInputRef; } + + Predicate aggIgnoreNullBucket = + agg -> + agg.getHints().stream() + .anyMatch( + hint -> + hint.hintName.equals("stats_args") + && hint.kvOptions.get(Argument.BUCKET_NULLABLE).equals("false")); + + Predicate maybeTimeSpanAgg = + agg -> + agg.getGroupSet().stream() + .allMatch( + group -> + isTimeBasedType( + agg.getInput().getRowType().getFieldList().get(group).getType())); + + static boolean isTimeSpan(RexNode rex) { + return rex instanceof RexCall rexCall + && rexCall.getKind() == SqlKind.OTHER_FUNCTION + && rexCall.getOperator().getName().equalsIgnoreCase(BuiltinFunctionName.SPAN.name()) + && rexCall.getOperands().size() == 3 + && rexCall.getOperands().get(2) instanceof RexLiteral unitLiteral + && unitLiteral.getTypeName() != SqlTypeName.NULL; + } + + /** + * Check if the condition is NOT NULL derived from an aggregate. + * + * @param condition the condition to check, composite of single or multiple NOT NULL conditions + * @param aggregate the aggregate where the condition is derived from + * @param project the project between the aggregate and the filter + * @param otherMapping the other mapping generated from ProjectIndexScanRule when applied on the + * above project with non-ref expressions. + * @return true if the condition is single or multiple NOT NULL derived from an aggregate, false + * otherwise + */ + static boolean isNotNullDerivedFromAgg( + RexNode condition, + Aggregate aggregate, + @Nullable Project project, + @Nullable List otherMapping) { + boolean ignoreNullBucket = aggIgnoreNullBucket.test(aggregate); + if (!ignoreNullBucket && project == null) return false; + List groupRefList = aggregate.getGroupSet().asList(); + if (project != null) { + groupRefList = + groupRefList.stream() + .map(project.getProjects()::get) + .filter(rex -> ignoreNullBucket || isTimeSpan(rex)) + .flatMap(expr -> PlanUtils.getInputRefs(expr).stream()) + .map(RexSlot::getIndex) + .toList(); + } + if (otherMapping != null) { + groupRefList = groupRefList.stream().map(otherMapping::get).toList(); + } + List finalGroupRefList = groupRefList; + Function isNotNullFromAgg = + rex -> + rex instanceof RexCall rexCall + && rexCall.isA(SqlKind.IS_NOT_NULL) + && rexCall.getOperands().get(0) instanceof RexInputRef ref + && finalGroupRefList.contains(ref.getIndex()); + return isNotNullFromAgg.apply(condition) + || (condition instanceof RexCall rexCall + && rexCall.getOperator() == SqlStdOperatorTable.AND + && rexCall.getOperands().stream().allMatch(isNotNullFromAgg::apply)); + } + + /** + * Try to prune all RelNodes in the RuleCall from top to down. We can prune a RelNode if: + * + *

1. It's the root RelNode of the current RuleCall. Or, + * + *

2. It's logical RelNode and it only has one parent which is pruned. TODO: To be more + * precisely, we can prun a RelNode whose parents are all pruned, but `prunedNodes` in + * VolcanoPlanner is not available. + * + * @param call the RuleCall to prune + */ + static void tryPruneRelNodes(RelOptRuleCall call) { + if (call.getPlanner() instanceof VolcanoPlanner volcanoPlanner) { + Arrays.stream(call.rels) + .takeWhile( + rel -> + // Don't prune the physical RelNode as it may prevent sort expr push down + rel.getConvention() == Convention.NONE + && (rel == call.rels[0] + || volcanoPlanner.getSubsetNonNull(rel).getParentRels().size() == 1)) + .forEach(volcanoPlanner::prune); + } + } } diff --git a/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_date_range_push.yaml b/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_date_range_push.yaml index 3c116fc7b44..792b065745d 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_date_range_push.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_date_range_push.yaml @@ -8,4 +8,4 @@ calcite: LogicalProject(@timestamp=[$0], category=[$1], value=[$2], timestamp=[$3], _id=[$4], _index=[$5], _score=[$6], _maxscore=[$7], _sort=[$8], _routing=[$9], value_range=[CASE(<($2, 7000), 'small':VARCHAR, 'large':VARCHAR)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]], PushDownContext=[[FILTER->IS NOT NULL($0), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 2},avg(value)=AVG($1)), PROJECT->[avg(value), span(@timestamp,1h), value_range], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"exists":{"field":"@timestamp","boost":1.0}},"aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"span(@timestamp,1h)":{"date_histogram":{"field":"@timestamp","missing_bucket":false,"order":"asc","fixed_interval":"1h"}}}]},"aggregations":{"value_range":{"range":{"field":"value","ranges":[{"key":"small","to":7000.0},{"key":"large","from":7000.0}],"keyed":true},"aggregations":{"avg(value)":{"avg":{"field":"value"}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 2},avg(value)=AVG($1)), PROJECT->[avg(value), span(@timestamp,1h), value_range], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"span(@timestamp,1h)":{"date_histogram":{"field":"@timestamp","missing_bucket":false,"order":"asc","fixed_interval":"1h"}}}]},"aggregations":{"value_range":{"range":{"field":"value","ranges":[{"key":"small","to":7000.0},{"key":"large","from":7000.0}],"keyed":true},"aggregations":{"avg(value)":{"avg":{"field":"value"}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/big5/cardinality_agg_high.yaml b/integ-test/src/test/resources/expectedOutput/calcite/big5/cardinality_agg_high.yaml index bd3889e2926..cbfad5eb117 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/big5/cardinality_agg_high.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/big5/cardinality_agg_high.yaml @@ -6,4 +6,4 @@ calcite: LogicalFilter(condition=[IS NOT NULL($3)]) CalciteLogicalIndexScan(table=[[OpenSearch, big5]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, big5]], PushDownContext=[[FILTER->IS NOT NULL($3), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={},dc(`agent.name`)=COUNT(DISTINCT $0)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"exists":{"field":"agent.name","boost":1.0}},"aggregations":{"dc(`agent.name`)":{"cardinality":{"field":"agent.name"}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, big5]], PushDownContext=[[FILTER->IS NOT NULL($0), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={},dc(`agent.name`)=COUNT(DISTINCT $0)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"exists":{"field":"agent.name","boost":1.0}},"aggregations":{"dc(`agent.name`)":{"cardinality":{"field":"agent.name"}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/big5/cardinality_agg_high_2.yaml b/integ-test/src/test/resources/expectedOutput/calcite/big5/cardinality_agg_high_2.yaml index 6d5c2b7448e..8ac7fee451b 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/big5/cardinality_agg_high_2.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/big5/cardinality_agg_high_2.yaml @@ -6,4 +6,4 @@ calcite: LogicalFilter(condition=[IS NOT NULL($37)]) CalciteLogicalIndexScan(table=[[OpenSearch, big5]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, big5]], PushDownContext=[[FILTER->IS NOT NULL($37), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={},dc(`event.id`)=COUNT(DISTINCT $0)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"exists":{"field":"event.id","boost":1.0}},"aggregations":{"dc(`event.id`)":{"cardinality":{"field":"event.id"}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, big5]], PushDownContext=[[FILTER->IS NOT NULL($0), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={},dc(`event.id`)=COUNT(DISTINCT $0)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"exists":{"field":"event.id","boost":1.0}},"aggregations":{"dc(`event.id`)":{"cardinality":{"field":"event.id"}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/big5/cardinality_agg_low.yaml b/integ-test/src/test/resources/expectedOutput/calcite/big5/cardinality_agg_low.yaml index dec25a78628..d0bcca5bc4c 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/big5/cardinality_agg_low.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/big5/cardinality_agg_low.yaml @@ -6,4 +6,4 @@ calcite: LogicalFilter(condition=[IS NOT NULL($14)]) CalciteLogicalIndexScan(table=[[OpenSearch, big5]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, big5]], PushDownContext=[[FILTER->IS NOT NULL($14), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={},dc(`cloud.region`)=COUNT(DISTINCT $0)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"exists":{"field":"cloud.region","boost":1.0}},"aggregations":{"dc(`cloud.region`)":{"cardinality":{"field":"cloud.region"}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, big5]], PushDownContext=[[FILTER->IS NOT NULL($0), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={},dc(`cloud.region`)=COUNT(DISTINCT $0)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"exists":{"field":"cloud.region","boost":1.0}},"aggregations":{"dc(`cloud.region`)":{"cardinality":{"field":"cloud.region"}}}}, requestedTotalSize=2147483647, 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 6f0e2fdae76..1f3949c6ce3 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 @@ -8,4 +8,4 @@ calcite: 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=[[PROJECT->[@timestamp], 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}}},"_source":{"includes":["@timestamp"],"excludes":[]},"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)]) \ No newline at end of file + 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 9be838e68b8..199988f7e3b 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 @@ -8,4 +8,4 @@ calcite: 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]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, big5]], PushDownContext=[[PROJECT->[process.name, cloud.region, @timestamp], 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}},"_source":{"includes":["process.name","cloud.region","@timestamp"],"excludes":[]},"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)]) \ No newline at end of file + 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 9cab2b0de11..9c34a7ecd56 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 @@ -8,4 +8,4 @@ calcite: 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]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, big5]], PushDownContext=[[PROJECT->[process.name, cloud.region, @timestamp, aws.cloudwatch.log_stream], 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, 2},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}},"_source":{"includes":["process.name","cloud.region","@timestamp","aws.cloudwatch.log_stream"],"excludes":[]},"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)]) \ No newline at end of file + 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 7bccacdf5af..41303581efc 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 @@ -7,4 +7,4 @@ calcite: 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=[[PROJECT->[@timestamp], 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}}},"_source":{"includes":["@timestamp"],"excludes":[]},"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)]) \ No newline at end of file + 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 9ab1bb342c9..7550a82cca4 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 @@ -11,4 +11,4 @@ calcite: physical: | EnumerableLimit(fetch=[10000]) EnumerableSort(sort0=[$7], dir0=[DESC-nulls-last]) - 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","agent.ephemeral_id","agent.id","agent.name","agent.type","agent.version","process","process.name","log","log.file","log.file.path","message","tags","cloud","cloud.region","input","input.type","@timestamp","ecs","ecs.version","data_stream","data_stream.dataset","data_stream.namespace","data_stream.type","meta","meta.file","host","metrics","metrics.tmin","aws","aws.cloudwatch","aws.cloudwatch.ingestion_time","aws.cloudwatch.log_group","aws.cloudwatch.log_stream","event","event.dataset","event.id","event.ingested","_id","_index","_score","_maxscore","_sort","_routing"],"excludes":[]},"script_fields":{}}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file + 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_terms.yaml b/integ-test/src/test/resources/expectedOutput/calcite/big5/keyword_terms.yaml index 30eb5cfddab..7718e89b02f 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/big5/keyword_terms.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/big5/keyword_terms.yaml @@ -8,4 +8,4 @@ calcite: LogicalFilter(condition=[IS NOT NULL($34)]) CalciteLogicalIndexScan(table=[[OpenSearch, big5]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, big5]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},station=COUNT()), PROJECT->[station, aws.cloudwatch.log_stream], SORT_AGG_METRICS->[0 DESC LAST], LIMIT->500, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"aws.cloudwatch.log_stream":{"terms":{"field":"aws.cloudwatch.log_stream","size":500,"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)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, big5]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},station=COUNT()), SORT_AGG_METRICS->[1 DESC LAST], PROJECT->[station, aws.cloudwatch.log_stream], LIMIT->500, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"aws.cloudwatch.log_stream":{"terms":{"field":"aws.cloudwatch.log_stream","size":500,"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/big5/keyword_terms_low_cardinality.yaml b/integ-test/src/test/resources/expectedOutput/calcite/big5/keyword_terms_low_cardinality.yaml index ddb924bedf5..2ce4f996236 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/big5/keyword_terms_low_cardinality.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/big5/keyword_terms_low_cardinality.yaml @@ -8,4 +8,4 @@ calcite: LogicalFilter(condition=[IS NOT NULL($34)]) CalciteLogicalIndexScan(table=[[OpenSearch, big5]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, big5]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},country=COUNT()), PROJECT->[country, aws.cloudwatch.log_stream], SORT_AGG_METRICS->[0 DESC LAST], LIMIT->50, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"aws.cloudwatch.log_stream":{"terms":{"field":"aws.cloudwatch.log_stream","size":50,"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)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, big5]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},country=COUNT()), SORT_AGG_METRICS->[1 DESC LAST], PROJECT->[country, aws.cloudwatch.log_stream], LIMIT->50, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"aws.cloudwatch.log_stream":{"terms":{"field":"aws.cloudwatch.log_stream","size":50,"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/big5/multi_terms_keyword.yaml b/integ-test/src/test/resources/expectedOutput/calcite/big5/multi_terms_keyword.yaml index 5659aca68dd..b01761d7dee 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 @@ -8,4 +8,4 @@ calcite: 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]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, big5]], PushDownContext=[[PROJECT->[process.name, cloud.region, @timestamp], 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}},"_source":{"includes":["process.name","cloud.region","@timestamp"],"excludes":[]},"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)]) \ No newline at end of file + 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/big5/terms_significant_1.yaml b/integ-test/src/test/resources/expectedOutput/calcite/big5/terms_significant_1.yaml index 6e5b68f495c..938181676fb 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/big5/terms_significant_1.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/big5/terms_significant_1.yaml @@ -8,4 +8,4 @@ calcite: 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=[[PROJECT->[process.name, @timestamp, aws.cloudwatch.log_stream], FILTER->SEARCH($1, Sarg[['2023-01-01 00:00:00':VARCHAR..'2023-01-03 00:00:00':VARCHAR)]:VARCHAR), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},count()=COUNT()), PROJECT->[count(), aws.cloudwatch.log_stream, process.name], LIMIT->10, 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}}},"_source":{"includes":["process.name","@timestamp","aws.cloudwatch.log_stream"],"excludes":[]},"aggregations":{"composite_buckets":{"composite":{"size":10,"sources":[{"aws.cloudwatch.log_stream":{"terms":{"field":"aws.cloudwatch.log_stream","missing_bucket":true,"missing_order":"first","order":"asc"}}},{"process.name":{"terms":{"field":"process.name","missing_bucket":true,"missing_order":"first","order":"asc"}}}]}}}}, requestedTotalSize=10, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, big5]], PushDownContext=[[FILTER->SEARCH($1, Sarg[['2023-01-01 00:00:00':VARCHAR..'2023-01-03 00:00:00':VARCHAR)]:VARCHAR), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 2},count()=COUNT()), PROJECT->[count(), aws.cloudwatch.log_stream, process.name], LIMIT->10, 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":10,"sources":[{"process.name":{"terms":{"field":"process.name","missing_bucket":true,"missing_order":"first","order":"asc"}}},{"aws.cloudwatch.log_stream":{"terms":{"field":"aws.cloudwatch.log_stream","missing_bucket":true,"missing_order":"first","order":"asc"}}}]}}}}, requestedTotalSize=10, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/big5/terms_significant_2.yaml b/integ-test/src/test/resources/expectedOutput/calcite/big5/terms_significant_2.yaml index 5d9f9d1f579..da9a207307d 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/big5/terms_significant_2.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/big5/terms_significant_2.yaml @@ -8,4 +8,4 @@ calcite: 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=[[PROJECT->[process.name, @timestamp, aws.cloudwatch.log_stream], FILTER->SEARCH($1, Sarg[['2023-01-01 00:00:00':VARCHAR..'2023-01-03 00:00:00':VARCHAR)]:VARCHAR), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},count()=COUNT()), PROJECT->[count(), process.name, aws.cloudwatch.log_stream], LIMIT->10, 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}}},"_source":{"includes":["process.name","@timestamp","aws.cloudwatch.log_stream"],"excludes":[]},"aggregations":{"composite_buckets":{"composite":{"size":10,"sources":[{"process.name":{"terms":{"field":"process.name","missing_bucket":true,"missing_order":"first","order":"asc"}}},{"aws.cloudwatch.log_stream":{"terms":{"field":"aws.cloudwatch.log_stream","missing_bucket":true,"missing_order":"first","order":"asc"}}}]}}}}, requestedTotalSize=10, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, big5]], PushDownContext=[[FILTER->SEARCH($1, Sarg[['2023-01-01 00:00:00':VARCHAR..'2023-01-03 00:00:00':VARCHAR)]:VARCHAR), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 2},count()=COUNT()), PROJECT->[count(), process.name, aws.cloudwatch.log_stream], LIMIT->10, 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":10,"sources":[{"process.name":{"terms":{"field":"process.name","missing_bucket":true,"missing_order":"first","order":"asc"}}},{"aws.cloudwatch.log_stream":{"terms":{"field":"aws.cloudwatch.log_stream","missing_bucket":true,"missing_order":"first","order":"asc"}}}]}}}}, requestedTotalSize=10, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/chart_multiple_group_keys.yaml b/integ-test/src/test/resources/expectedOutput/calcite/chart_multiple_group_keys.yaml index d692ba70f69..42f0d27e392 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/chart_multiple_group_keys.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/chart_multiple_group_keys.yaml @@ -27,10 +27,10 @@ calcite: EnumerableMergeJoin(condition=[=($1, $3)], joinType=[left]) EnumerableSort(sort0=[$1], dir0=[ASC]) EnumerableCalc(expr#0..2=[{inputs}], expr#3=[SAFE_CAST($t1)], gender=[$t0], age=[$t3], avg(balance)=[$t2]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[PROJECT->[gender, balance, age], FILTER->AND(IS NOT NULL($0), IS NOT NULL($1)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},avg(balance)=AVG($2))], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"exists":{"field":"gender","boost":1.0}},{"exists":{"field":"balance","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"_source":{"includes":["gender","balance","age"],"excludes":[]},"aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"gender":{"terms":{"field":"gender.keyword","missing_bucket":true,"missing_order":"first","order":"asc"}}},{"age":{"terms":{"field":"age","missing_bucket":true,"missing_order":"first","order":"asc"}}}]},"aggregations":{"avg(balance)":{"avg":{"field":"balance"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[FILTER->AND(IS NOT NULL($0), IS NOT NULL($1)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 2},avg(balance)=AVG($1))], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"exists":{"field":"gender","boost":1.0}},{"exists":{"field":"balance","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"gender":{"terms":{"field":"gender.keyword","missing_bucket":true,"missing_order":"first","order":"asc"}}},{"age":{"terms":{"field":"age","missing_bucket":true,"missing_order":"first","order":"asc"}}}]},"aggregations":{"avg(balance)":{"avg":{"field":"balance"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) EnumerableSort(sort0=[$0], dir0=[ASC]) EnumerableCalc(expr#0..2=[{inputs}], age=[$t0], $1=[$t2]) EnumerableWindow(window#0=[window(order by [1 DESC-nulls-last] rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) EnumerableAggregate(group=[{0}], __grand_total__=[SUM($1)]) EnumerableCalc(expr#0..1=[{inputs}], expr#2=[SAFE_CAST($t0)], expr#3=[IS NOT NULL($t2)], age=[$t2], avg(balance)=[$t1], $condition=[$t3]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[PROJECT->[gender, balance, age], FILTER->AND(IS NOT NULL($0), IS NOT NULL($1)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},avg(balance)=AVG($2)), PROJECT->[age, avg(balance)]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"exists":{"field":"gender","boost":1.0}},{"exists":{"field":"balance","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"_source":{"includes":["gender","balance","age"],"excludes":[]},"aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"gender":{"terms":{"field":"gender.keyword","missing_bucket":true,"missing_order":"first","order":"asc"}}},{"age":{"terms":{"field":"age","missing_bucket":true,"missing_order":"first","order":"asc"}}}]},"aggregations":{"avg(balance)":{"avg":{"field":"balance"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[FILTER->AND(IS NOT NULL($0), IS NOT NULL($1)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 2},avg(balance)=AVG($1)), PROJECT->[age, avg(balance)]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"exists":{"field":"gender","boost":1.0}},{"exists":{"field":"balance","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"gender":{"terms":{"field":"gender.keyword","missing_bucket":true,"missing_order":"first","order":"asc"}}},{"age":{"terms":{"field":"age","missing_bucket":true,"missing_order":"first","order":"asc"}}}]},"aggregations":{"avg(balance)":{"avg":{"field":"balance"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/chart_single_group_key.yaml b/integ-test/src/test/resources/expectedOutput/calcite/chart_single_group_key.yaml index f8200b10a81..c21e8c468e9 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/chart_single_group_key.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/chart_single_group_key.yaml @@ -7,4 +7,4 @@ calcite: LogicalFilter(condition=[AND(IS NOT NULL($4), IS NOT NULL($7))]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[FILTER->AND(IS NOT NULL($4), IS NOT NULL($7)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},avg(balance)=AVG($1)), SORT->[0], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"exists":{"field":"gender","boost":1.0}},{"exists":{"field":"balance","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"gender":{"terms":{"field":"gender.keyword","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"avg(balance)":{"avg":{"field":"balance"}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[FILTER->AND(IS NOT NULL($0), IS NOT NULL($1)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},avg(balance)=AVG($1)), SORT->[0], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"exists":{"field":"gender","boost":1.0}},{"exists":{"field":"balance","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"gender":{"terms":{"field":"gender.keyword","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"avg(balance)":{"avg":{"field":"balance"}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/chart_timestamp_span_and_category.yaml b/integ-test/src/test/resources/expectedOutput/calcite/chart_timestamp_span_and_category.yaml index ef2320d13c5..f3e8cae5ba3 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/chart_timestamp_span_and_category.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/chart_timestamp_span_and_category.yaml @@ -24,9 +24,9 @@ calcite: EnumerableAggregate(group=[{0, 1}], max(value)=[MAX($2)]) EnumerableCalc(expr#0..4=[{inputs}], expr#5=[IS NULL($t1)], expr#6=['NULL'], expr#7=[10], expr#8=[<=($t4, $t7)], expr#9=['OTHER'], expr#10=[CASE($t5, $t6, $t8, $t1, $t9)], timestamp=[$t0], category=[$t10], max(value)=[$t2]) EnumerableMergeJoin(condition=[=($1, $3)], joinType=[left]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]], PushDownContext=[[PROJECT->[category, value, timestamp], FILTER->AND(IS NOT NULL($2), IS NOT NULL($1)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 2},max(value)=MAX($1)), PROJECT->[timestamp0, category, max(value)], SORT->[1]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"exists":{"field":"timestamp","boost":1.0}},{"exists":{"field":"value","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"_source":{"includes":["category","value","timestamp"],"excludes":[]},"aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"category":{"terms":{"field":"category","missing_bucket":true,"missing_order":"last","order":"asc"}}},{"timestamp0":{"date_histogram":{"field":"timestamp","missing_bucket":false,"order":"asc","calendar_interval":"1w"}}}]},"aggregations":{"max(value)":{"max":{"field":"value"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]], PushDownContext=[[FILTER->AND(IS NOT NULL($2), IS NOT NULL($1)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 2},max(value)=MAX($1)), PROJECT->[timestamp0, category, max(value)], SORT->[1]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"exists":{"field":"timestamp","boost":1.0}},{"exists":{"field":"value","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"category":{"terms":{"field":"category","missing_bucket":true,"missing_order":"last","order":"asc"}}},{"timestamp0":{"date_histogram":{"field":"timestamp","missing_bucket":false,"order":"asc","calendar_interval":"1w"}}}]},"aggregations":{"max(value)":{"max":{"field":"value"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) EnumerableSort(sort0=[$0], dir0=[ASC]) EnumerableCalc(expr#0..2=[{inputs}], category=[$t0], $1=[$t2]) EnumerableWindow(window#0=[window(order by [1 DESC-nulls-last] rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) EnumerableAggregate(group=[{0}], __grand_total__=[SUM($1)]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]], PushDownContext=[[PROJECT->[category, value, timestamp], FILTER->AND(IS NOT NULL($2), IS NOT NULL($1)), FILTER->IS NOT NULL($0), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 2},max(value)=MAX($1)), PROJECT->[category, max(value)]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"filter":[{"bool":{"must":[{"exists":{"field":"timestamp","boost":1.0}},{"exists":{"field":"value","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},{"exists":{"field":"category","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"_source":{"includes":["category","value","timestamp"],"excludes":[]},"aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"category":{"terms":{"field":"category","missing_bucket":true,"missing_order":"first","order":"asc"}}},{"timestamp0":{"date_histogram":{"field":"timestamp","missing_bucket":false,"order":"asc","calendar_interval":"1w"}}}]},"aggregations":{"max(value)":{"max":{"field":"value"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]], PushDownContext=[[FILTER->AND(IS NOT NULL($2), IS NOT NULL($1)), FILTER->IS NOT NULL($0), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 2},max(value)=MAX($1)), PROJECT->[category, max(value)]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"filter":[{"bool":{"must":[{"exists":{"field":"timestamp","boost":1.0}},{"exists":{"field":"value","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},{"exists":{"field":"category","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"category":{"terms":{"field":"category","missing_bucket":true,"missing_order":"first","order":"asc"}}},{"timestamp0":{"date_histogram":{"field":"timestamp","missing_bucket":false,"order":"asc","calendar_interval":"1w"}}}]},"aggregations":{"max(value)":{"max":{"field":"value"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/chart_use_other.yaml b/integ-test/src/test/resources/expectedOutput/calcite/chart_use_other.yaml index 4ee0759468f..953aab7ef47 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/chart_use_other.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/chart_use_other.yaml @@ -23,9 +23,9 @@ calcite: EnumerableAggregate(group=[{0, 1}], max(severityNumber)=[MAX($2)]) EnumerableCalc(expr#0..4=[{inputs}], expr#5=[IS NULL($t1)], expr#6=['NULL'], expr#7=[2], expr#8=[<=($t4, $t7)], expr#9=['max_among_other'], expr#10=[CASE($t5, $t6, $t8, $t1, $t9)], flags=[$t0], severityText=[$t10], max(severityNumber)=[$t2]) EnumerableMergeJoin(condition=[=($1, $3)], joinType=[left]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_otel_logs]], PushDownContext=[[PROJECT->[severityText, flags, severityNumber], FILTER->AND(IS NOT NULL($1), IS NOT NULL($2)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},max(severityNumber)=MAX($2)), SORT->[1]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"exists":{"field":"flags","boost":1.0}},{"exists":{"field":"severityNumber","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"_source":{"includes":["severityText","flags","severityNumber"],"excludes":[]},"aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"severityText":{"terms":{"field":"severityText","missing_bucket":true,"missing_order":"last","order":"asc"}}},{"flags":{"terms":{"field":"flags","missing_bucket":true,"missing_order":"first","order":"asc"}}}]},"aggregations":{"max(severityNumber)":{"max":{"field":"severityNumber"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_otel_logs]], PushDownContext=[[FILTER->AND(IS NOT NULL($1), IS NOT NULL($2)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},max(severityNumber)=MAX($2)), SORT->[1]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"exists":{"field":"flags","boost":1.0}},{"exists":{"field":"severityNumber","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"severityText":{"terms":{"field":"severityText","missing_bucket":true,"missing_order":"last","order":"asc"}}},{"flags":{"terms":{"field":"flags","missing_bucket":true,"missing_order":"first","order":"asc"}}}]},"aggregations":{"max(severityNumber)":{"max":{"field":"severityNumber"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) EnumerableSort(sort0=[$0], dir0=[ASC]) EnumerableCalc(expr#0..2=[{inputs}], severityText=[$t0], $1=[$t2]) EnumerableWindow(window#0=[window(order by [1 DESC-nulls-last] rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) EnumerableAggregate(group=[{0}], __grand_total__=[SUM($1)]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_otel_logs]], PushDownContext=[[PROJECT->[severityText, flags, severityNumber], FILTER->AND(IS NOT NULL($1), IS NOT NULL($2)), FILTER->IS NOT NULL($0), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},max(severityNumber)=MAX($2)), PROJECT->[severityText, max(severityNumber)]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"filter":[{"bool":{"must":[{"exists":{"field":"flags","boost":1.0}},{"exists":{"field":"severityNumber","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},{"exists":{"field":"severityText","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"_source":{"includes":["severityText","flags","severityNumber"],"excludes":[]},"aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"flags":{"terms":{"field":"flags","missing_bucket":true,"missing_order":"first","order":"asc"}}},{"severityText":{"terms":{"field":"severityText","missing_bucket":true,"missing_order":"first","order":"asc"}}}]},"aggregations":{"max(severityNumber)":{"max":{"field":"severityNumber"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_otel_logs]], PushDownContext=[[FILTER->AND(IS NOT NULL($1), IS NOT NULL($2)), FILTER->IS NOT NULL($1), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},max(severityNumber)=MAX($2)), PROJECT->[severityText, max(severityNumber)]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"filter":[{"bool":{"must":[{"exists":{"field":"flags","boost":1.0}},{"exists":{"field":"severityNumber","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},{"exists":{"field":"severityText","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"flags":{"terms":{"field":"flags","missing_bucket":true,"missing_order":"first","order":"asc"}}},{"severityText":{"terms":{"field":"severityText","missing_bucket":true,"missing_order":"first","order":"asc"}}}]},"aggregations":{"max(severityNumber)":{"max":{"field":"severityNumber"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/chart_with_integer_span.yaml b/integ-test/src/test/resources/expectedOutput/calcite/chart_with_integer_span.yaml index 4fcc802063f..5a37ec9b92c 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/chart_with_integer_span.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/chart_with_integer_span.yaml @@ -8,4 +8,4 @@ calcite: LogicalFilter(condition=[AND(IS NOT NULL($10), IS NOT NULL($7))]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[PROJECT->[balance, age], FILTER->AND(IS NOT NULL($1), IS NOT NULL($0)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={1},max(balance)=MAX($0)), SORT->[0], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"exists":{"field":"age","boost":1.0}},{"exists":{"field":"balance","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"_source":{"includes":["balance","age"],"excludes":[]},"aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"age0":{"histogram":{"field":"age","missing_bucket":false,"order":"asc","interval":10.0}}}]},"aggregations":{"max(balance)":{"max":{"field":"balance"}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[FILTER->AND(IS NOT NULL($1), IS NOT NULL($0)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={1},max(balance)=MAX($0)), SORT->[0], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"exists":{"field":"age","boost":1.0}},{"exists":{"field":"balance","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"age0":{"histogram":{"field":"age","missing_bucket":false,"order":"asc","interval":10.0}}}]},"aggregations":{"max(balance)":{"max":{"field":"balance"}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/chart_with_limit.yaml b/integ-test/src/test/resources/expectedOutput/calcite/chart_with_limit.yaml index e4578f21fcd..de69b108758 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/chart_with_limit.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/chart_with_limit.yaml @@ -7,4 +7,4 @@ calcite: LogicalFilter(condition=[AND(IS NOT NULL($9), IS NOT NULL($7))]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[PROJECT->[gender, balance, state], FILTER->AND(IS NOT NULL($2), IS NOT NULL($1)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},avg(balance)=AVG($2)), SORT->[0], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"exists":{"field":"state","boost":1.0}},{"exists":{"field":"balance","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"_source":{"includes":["gender","balance","state"],"excludes":[]},"aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"state":{"terms":{"field":"state.keyword","missing_bucket":true,"missing_order":"last","order":"asc"}}},{"gender":{"terms":{"field":"gender.keyword","missing_bucket":true,"missing_order":"first","order":"asc"}}}]},"aggregations":{"avg(balance)":{"avg":{"field":"balance"}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[FILTER->AND(IS NOT NULL($2), IS NOT NULL($1)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},avg(balance)=AVG($2)), SORT->[0], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"exists":{"field":"state","boost":1.0}},{"exists":{"field":"balance","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"state":{"terms":{"field":"state.keyword","missing_bucket":true,"missing_order":"last","order":"asc"}}},{"gender":{"terms":{"field":"gender.keyword","missing_bucket":true,"missing_order":"first","order":"asc"}}}]},"aggregations":{"avg(balance)":{"avg":{"field":"balance"}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q10.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q10.yaml index 7fc8c3d5a7f..ae4cade06e5 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q10.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q10.yaml @@ -8,4 +8,4 @@ calcite: LogicalFilter(condition=[IS NOT NULL($68)]) CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},sum(AdvEngineID)=SUM($1),c=COUNT(),avg(ResolutionWidth)=AVG($2),dc(UserID)=COUNT(DISTINCT $3)), PROJECT->[sum(AdvEngineID), c, avg(ResolutionWidth), dc(UserID), RegionID], SORT_AGG_METRICS->[1 DESC LAST], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"RegionID":{"terms":{"field":"RegionID","size":10,"min_doc_count":1,"shard_min_doc_count":0,"show_term_doc_count_error":false,"order":[{"c":"desc"},{"_key":"asc"}]},"aggregations":{"sum(AdvEngineID)":{"sum":{"field":"AdvEngineID"}},"avg(ResolutionWidth)":{"avg":{"field":"ResolutionWidth"}},"dc(UserID)":{"cardinality":{"field":"UserID"}},"c":{"value_count":{"field":"_index"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={1},sum(AdvEngineID)=SUM($0),c=COUNT(),avg(ResolutionWidth)=AVG($2),dc(UserID)=COUNT(DISTINCT $3)), SORT_AGG_METRICS->[2 DESC LAST], PROJECT->[sum(AdvEngineID), c, avg(ResolutionWidth), dc(UserID), RegionID], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"RegionID":{"terms":{"field":"RegionID","size":10,"min_doc_count":1,"shard_min_doc_count":0,"show_term_doc_count_error":false,"order":[{"c":"desc"},{"_key":"asc"}]},"aggregations":{"sum(AdvEngineID)":{"sum":{"field":"AdvEngineID"}},"avg(ResolutionWidth)":{"avg":{"field":"ResolutionWidth"}},"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/q11.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q11.yaml index 0f137b73605..af6c337fbbb 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q11.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q11.yaml @@ -8,4 +8,4 @@ calcite: LogicalFilter(condition=[<>($31, '')]) CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER-><>($31, ''), 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)]) + 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 6b47ac718b6..9e72d78f9cb 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q12.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q12.yaml @@ -8,4 +8,4 @@ calcite: LogicalFilter(condition=[AND(<>($31, ''), IS NOT NULL($62))]) CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[PROJECT->[MobilePhoneModel, MobilePhone, UserID], 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}},"_source":{"includes":["MobilePhoneModel","MobilePhone","UserID"],"excludes":[]},"aggregations":{"MobilePhone|MobilePhoneModel":{"multi_terms":{"terms":[{"field":"MobilePhone"},{"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)]) \ 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, 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 3c2c0f9dfb5..eb0def79d01 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q13.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q13.yaml @@ -8,4 +8,4 @@ calcite: LogicalFilter(condition=[<>($63, '')]) CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER-><>($63, ''), 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)]) \ No newline at end of file + 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 c3678470be8..18f0015a42f 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q14.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q14.yaml @@ -8,4 +8,4 @@ calcite: LogicalFilter(condition=[<>($63, '')]) CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER-><>($63, ''), 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)]) + 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 7218a23bc29..a69a320e81a 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q15.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q15.yaml @@ -8,4 +8,4 @@ calcite: LogicalFilter(condition=[AND(<>($63, ''), IS NOT NULL($65))]) CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[PROJECT->[SearchPhrase, SearchEngineID], 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}},"_source":{"includes":["SearchPhrase","SearchEngineID"],"excludes":[]},"aggregations":{"SearchEngineID|SearchPhrase":{"multi_terms":{"terms":[{"field":"SearchEngineID"},{"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)]) \ 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, 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/q16.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q16.yaml index 4110e1a12d0..3f0fb7644a9 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q16.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q16.yaml @@ -8,4 +8,4 @@ calcite: LogicalFilter(condition=[IS NOT NULL($84)]) CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},count()=COUNT()), PROJECT->[count(), UserID], SORT_AGG_METRICS->[0 DESC LAST], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"UserID":{"terms":{"field":"UserID","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)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},count()=COUNT()), SORT_AGG_METRICS->[1 DESC LAST], PROJECT->[count(), UserID], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"UserID":{"terms":{"field":"UserID","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/q17.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q17.yaml index 46855028754..d9efea667c2 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q17.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q17.yaml @@ -8,4 +8,4 @@ calcite: LogicalFilter(condition=[AND(IS NOT NULL($84), IS NOT NULL($63))]) CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},count()=COUNT()), PROJECT->[count(), UserID, SearchPhrase], SORT_AGG_METRICS->[0 DESC LAST], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"UserID|SearchPhrase":{"multi_terms":{"terms":[{"field":"UserID"},{"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)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},count()=COUNT()), SORT_AGG_METRICS->[2 DESC LAST], PROJECT->[count(), UserID, SearchPhrase], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"SearchPhrase|UserID":{"multi_terms":{"terms":[{"field":"SearchPhrase"},{"field":"UserID"}],"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/q18.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q18.yaml index 7b3603d66aa..f1af61a6090 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q18.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q18.yaml @@ -8,4 +8,4 @@ calcite: LogicalFilter(condition=[AND(IS NOT NULL($84), IS NOT NULL($63))]) CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},count()=COUNT()), PROJECT->[count(), UserID, SearchPhrase], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":10,"sources":[{"UserID":{"terms":{"field":"UserID","missing_bucket":false,"order":"asc"}}},{"SearchPhrase":{"terms":{"field":"SearchPhrase","missing_bucket":false,"order":"asc"}}}]}}}}, requestedTotalSize=10, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},count()=COUNT()), PROJECT->[count(), UserID, SearchPhrase], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":10,"sources":[{"SearchPhrase":{"terms":{"field":"SearchPhrase","missing_bucket":false,"order":"asc"}}},{"UserID":{"terms":{"field":"UserID","missing_bucket":false,"order":"asc"}}}]}}}}, requestedTotalSize=10, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q19.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q19.yaml index 74f1ebab575..c4005b2ea4f 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q19.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q19.yaml @@ -9,4 +9,4 @@ calcite: LogicalProject(EventDate=[$0], URLRegionID=[$1], HasGCLID=[$2], Income=[$3], Interests=[$4], Robotness=[$5], BrowserLanguage=[$6], CounterClass=[$7], BrowserCountry=[$8], OriginalURL=[$9], ClientTimeZone=[$10], RefererHash=[$11], TraficSourceID=[$12], HitColor=[$13], RefererRegionID=[$14], URLCategoryID=[$15], LocalEventTime=[$16], EventTime=[$17], UTMTerm=[$18], AdvEngineID=[$19], UserAgentMinor=[$20], UserAgentMajor=[$21], RemoteIP=[$22], Sex=[$23], JavaEnable=[$24], URLHash=[$25], URL=[$26], ParamOrderID=[$27], OpenstatSourceID=[$28], HTTPError=[$29], SilverlightVersion3=[$30], MobilePhoneModel=[$31], SilverlightVersion4=[$32], SilverlightVersion1=[$33], SilverlightVersion2=[$34], IsDownload=[$35], IsParameter=[$36], CLID=[$37], FlashMajor=[$38], FlashMinor=[$39], UTMMedium=[$40], WatchID=[$41], DontCountHits=[$42], CookieEnable=[$43], HID=[$44], SocialAction=[$45], WindowName=[$46], ConnectTiming=[$47], PageCharset=[$48], IsLink=[$49], IsArtifical=[$50], JavascriptEnable=[$51], ClientEventTime=[$52], DNSTiming=[$53], CodeVersion=[$54], ResponseEndTiming=[$55], FUniqID=[$56], WindowClientHeight=[$57], OpenstatServiceName=[$58], UTMContent=[$59], HistoryLength=[$60], IsOldCounter=[$61], MobilePhone=[$62], SearchPhrase=[$63], FlashMinor2=[$64], SearchEngineID=[$65], IsEvent=[$66], UTMSource=[$67], RegionID=[$68], OpenstatAdID=[$69], UTMCampaign=[$70], GoodEvent=[$71], IsRefresh=[$72], ParamCurrency=[$73], Params=[$74], ResolutionHeight=[$75], ClientIP=[$76], FromTag=[$77], ParamCurrencyID=[$78], ResponseStartTiming=[$79], ResolutionWidth=[$80], SendTiming=[$81], RefererCategoryID=[$82], OpenstatCampaignID=[$83], UserID=[$84], WithHash=[$85], UserAgent=[$86], ParamPrice=[$87], ResolutionDepth=[$88], IsMobile=[$89], Age=[$90], SocialSourceNetworkID=[$91], OpenerName=[$92], OS=[$93], IsNotBounce=[$94], Referer=[$95], NetMinor=[$96], Title=[$97], NetMajor=[$98], IPNetworkID=[$99], FetchTiming=[$100], SocialNetwork=[$101], SocialSourcePage=[$102], CounterID=[$103], WindowClientWidth=[$104], _id=[$105], _index=[$106], _score=[$107], _maxscore=[$108], _sort=[$109], _routing=[$110], m=[EXTRACT('minute':VARCHAR, $17)]) CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1, 2},count()=COUNT()), PROJECT->[count(), UserID, m, SearchPhrase], SORT_AGG_METRICS->[0 DESC LAST], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"UserID|m|SearchPhrase":{"multi_terms":{"terms":[{"field":"UserID"},{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQCZXsKICAib3AiOiB7CiAgICAibmFtZSI6ICJFWFRSQUNUIiwKICAgICJraW5kIjogIk9USEVSX0ZVTkNUSU9OIiwKICAgICJzeW50YXgiOiAiRlVOQ1RJT04iCiAgfSwKICAib3BlcmFuZHMiOiBbCiAgICB7CiAgICAgICJkeW5hbWljUGFyYW0iOiAwLAogICAgICAidHlwZSI6IHsKICAgICAgICAidHlwZSI6ICJWQVJDSEFSIiwKICAgICAgICAibnVsbGFibGUiOiB0cnVlLAogICAgICAgICJwcmVjaXNpb24iOiAtMQogICAgICB9CiAgICB9LAogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMSwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInVkdCI6ICJFWFBSX1RJTUVTVEFNUCIsCiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0sCiAgImNsYXNzIjogIm9yZy5vcGVuc2VhcmNoLnNxbC5leHByZXNzaW9uLmZ1bmN0aW9uLlVzZXJEZWZpbmVkRnVuY3Rpb25CdWlsZGVyJDEiLAogICJ0eXBlIjogewogICAgInR5cGUiOiAiQklHSU5UIiwKICAgICJudWxsYWJsZSI6IHRydWUKICB9LAogICJkZXRlcm1pbmlzdGljIjogdHJ1ZSwKICAiZHluYW1pYyI6IGZhbHNlCn0=\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[2,0],"DIGESTS":["minute","EventTime"]}},"value_type":"long"},{"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)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1, 2},count()=COUNT()), SORT_AGG_METRICS->[3 DESC LAST], PROJECT->[count(), UserID, m, SearchPhrase], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"UserID|m|SearchPhrase":{"multi_terms":{"terms":[{"field":"UserID"},{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQCZXsKICAib3AiOiB7CiAgICAibmFtZSI6ICJFWFRSQUNUIiwKICAgICJraW5kIjogIk9USEVSX0ZVTkNUSU9OIiwKICAgICJzeW50YXgiOiAiRlVOQ1RJT04iCiAgfSwKICAib3BlcmFuZHMiOiBbCiAgICB7CiAgICAgICJkeW5hbWljUGFyYW0iOiAwLAogICAgICAidHlwZSI6IHsKICAgICAgICAidHlwZSI6ICJWQVJDSEFSIiwKICAgICAgICAibnVsbGFibGUiOiB0cnVlLAogICAgICAgICJwcmVjaXNpb24iOiAtMQogICAgICB9CiAgICB9LAogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMSwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInVkdCI6ICJFWFBSX1RJTUVTVEFNUCIsCiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0sCiAgImNsYXNzIjogIm9yZy5vcGVuc2VhcmNoLnNxbC5leHByZXNzaW9uLmZ1bmN0aW9uLlVzZXJEZWZpbmVkRnVuY3Rpb25CdWlsZGVyJDEiLAogICJ0eXBlIjogewogICAgInR5cGUiOiAiQklHSU5UIiwKICAgICJudWxsYWJsZSI6IHRydWUKICB9LAogICJkZXRlcm1pbmlzdGljIjogdHJ1ZSwKICAiZHluYW1pYyI6IGZhbHNlCn0=\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[2,0],"DIGESTS":["minute","EventTime"]}},"value_type":"long"},{"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/q2.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q2.yaml index 65149a1553b..d80d488a167 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q2.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q2.yaml @@ -5,4 +5,4 @@ calcite: LogicalFilter(condition=[<>($19, 0)]) CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[PROJECT->[AdvEngineID], FILTER-><>($0, 0), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={},count()=COUNT()), 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}},"_source":{"includes":["AdvEngineID"],"excludes":[]},"track_total_hits":2147483647}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER-><>($0, 0), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={},count()=COUNT()), 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}},"track_total_hits":2147483647}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q21.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q21.yaml index db43556e0cb..ea3d4d6863e 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q21.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q21.yaml @@ -5,4 +5,4 @@ calcite: LogicalFilter(condition=[LIKE($26, '%google%', '\')]) CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[PROJECT->[URL], FILTER->LIKE($0, '%google%', '\'), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={},count()=COUNT()), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"wildcard":{"URL":{"wildcard":"*google*","boost":1.0}}},"_source":{"includes":["URL"],"excludes":[]},"track_total_hits":2147483647}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER->LIKE($0, '%google%', '\'), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={},count()=COUNT()), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"wildcard":{"URL":{"wildcard":"*google*","boost":1.0}}},"track_total_hits":2147483647}, 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 a4cc2248e1e..cb0630aaba2 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q22.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q22.yaml @@ -8,4 +8,4 @@ calcite: LogicalFilter(condition=[AND(LIKE($26, '%google%', '\'), <>($63, ''))]) CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[PROJECT->[URL, SearchPhrase], FILTER->AND(LIKE($0, '%google%', '\'), <>($1, '')), 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":[{"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}},"_source":{"includes":["URL","SearchPhrase"],"excludes":[]},"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)]) \ No newline at end of file + 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 5ec734b13ea..252361aee86 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q23.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q23.yaml @@ -8,4 +8,4 @@ calcite: LogicalFilter(condition=[AND(LIKE($97, '%Google%', '\'), <>($63, ''), NOT(LIKE($26, '%.google.%', '\')))]) CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[PROJECT->[URL, SearchPhrase, UserID, Title], FILTER->AND(LIKE($3, '%Google%', '\'), <>($1, ''), NOT(LIKE($0, '%.google.%', '\'))), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},c=COUNT(),dc(UserID)=COUNT(DISTINCT $1)), 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}},"_source":{"includes":["URL","SearchPhrase","UserID","Title"],"excludes":[]},"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)]) \ No newline at end of file + 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 fbc4dd965dd..346d1bb3a91 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q28.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q28.yaml @@ -13,4 +13,4 @@ calcite: EnumerableLimit(fetch=[25]) EnumerableSort(sort0=[$0], dir0=[DESC-nulls-last]) EnumerableCalc(expr#0..2=[{inputs}], expr#3=[100000], expr#4=[>($t1, $t3)], proj#0..2=[{exprs}], $condition=[$t4]) - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[PROJECT->[URL, CounterID], 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}},"_source":{"includes":["URL","CounterID"],"excludes":[]},"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/q29.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q29.yaml index 0d882c3067c..5a47cee4a15 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q29.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q29.yaml @@ -15,4 +15,4 @@ calcite: EnumerableLimit(fetch=[25]) EnumerableSort(sort0=[$0], dir0=[DESC-nulls-last]) EnumerableCalc(expr#0..3=[{inputs}], expr#4=[100000], expr#5=[>($t1, $t4)], proj#0..3=[{exprs}], $condition=[$t5]) - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[PROJECT->[Referer], FILTER-><>($0, ''), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},l=AVG($2),c=COUNT(),min(Referer)=MIN($1)), PROJECT->[l, c, min(Referer), k]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"exists":{"field":"Referer","boost":1.0}}],"must_not":[{"term":{"Referer":{"value":"","boost":1.0}}}],"adjust_pure_negative":true,"boost":1.0}},"_source":{"includes":["Referer"],"excludes":[]},"aggregations":{"composite_buckets":{"composite":{"size":10000,"sources":[{"k":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQCG3sKICAib3AiOiB7CiAgICAibmFtZSI6ICJSRUdFWFBfUkVQTEFDRSIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfSwKICAgIHsKICAgICAgImR5bmFtaWNQYXJhbSI6IDEsCiAgICAgICJ0eXBlIjogewogICAgICAgICJ0eXBlIjogIlZBUkNIQVIiLAogICAgICAgICJudWxsYWJsZSI6IHRydWUsCiAgICAgICAgInByZWNpc2lvbiI6IC0xCiAgICAgIH0KICAgIH0sCiAgICB7CiAgICAgICJkeW5hbWljUGFyYW0iOiAyLAogICAgICAidHlwZSI6IHsKICAgICAgICAidHlwZSI6ICJWQVJDSEFSIiwKICAgICAgICAibnVsbGFibGUiOiB0cnVlLAogICAgICAgICJwcmVjaXNpb24iOiAtMQogICAgICB9CiAgICB9CiAgXQp9\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0,2,2],"DIGESTS":["Referer","^https?://(?:www\\.)?([^/]+)/.*$","$1"]}},"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":["Referer"]}}}},"min(Referer)":{"top_hits":{"from":0,"size":1,"version":false,"seq_no_primary_term":false,"explain":false,"fields":[{"field":"Referer"}],"sort":[{"Referer":{"order":"asc"}}]}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER-><>($0, ''), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},l=AVG($2),c=COUNT(),min(Referer)=MIN($1)), PROJECT->[l, c, min(Referer), k]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"exists":{"field":"Referer","boost":1.0}}],"must_not":[{"term":{"Referer":{"value":"","boost":1.0}}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"composite_buckets":{"composite":{"size":10000,"sources":[{"k":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQCG3sKICAib3AiOiB7CiAgICAibmFtZSI6ICJSRUdFWFBfUkVQTEFDRSIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfSwKICAgIHsKICAgICAgImR5bmFtaWNQYXJhbSI6IDEsCiAgICAgICJ0eXBlIjogewogICAgICAgICJ0eXBlIjogIlZBUkNIQVIiLAogICAgICAgICJudWxsYWJsZSI6IHRydWUsCiAgICAgICAgInByZWNpc2lvbiI6IC0xCiAgICAgIH0KICAgIH0sCiAgICB7CiAgICAgICJkeW5hbWljUGFyYW0iOiAyLAogICAgICAidHlwZSI6IHsKICAgICAgICAidHlwZSI6ICJWQVJDSEFSIiwKICAgICAgICAibnVsbGFibGUiOiB0cnVlLAogICAgICAgICJwcmVjaXNpb24iOiAtMQogICAgICB9CiAgICB9CiAgXQp9\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0,2,2],"DIGESTS":["Referer","^https?://(?:www\\.)?([^/]+)/.*$","$1"]}},"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":["Referer"]}}}},"min(Referer)":{"top_hits":{"from":0,"size":1,"version":false,"seq_no_primary_term":false,"explain":false,"fields":[{"field":"Referer"}],"sort":[{"Referer":{"order":"asc"}}]}}}}}}, 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 b6a513d31ee..460a3b8b582 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q31.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q31.yaml @@ -8,4 +8,4 @@ calcite: LogicalFilter(condition=[AND(<>($63, ''), IS NOT NULL($65), IS NOT NULL($76))]) CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[PROJECT->[SearchPhrase, SearchEngineID, IsRefresh, ClientIP, ResolutionWidth], FILTER->AND(<>($0, ''), IS NOT NULL($1), IS NOT NULL($3)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},c=COUNT(),sum(IsRefresh)=SUM($2),avg(ResolutionWidth)=AVG($3)), 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}},"_source":{"includes":["SearchPhrase","SearchEngineID","IsRefresh","ClientIP","ResolutionWidth"],"excludes":[]},"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)]) + 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 e20758eed71..9028b51a6d5 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q32.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q32.yaml @@ -8,4 +8,4 @@ calcite: LogicalFilter(condition=[AND(<>($63, ''), IS NOT NULL($41), IS NOT NULL($76))]) CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[PROJECT->[WatchID, SearchPhrase, IsRefresh, ClientIP, ResolutionWidth], FILTER->AND(<>($1, ''), IS NOT NULL($0), IS NOT NULL($3)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},c=COUNT(),sum(IsRefresh)=SUM($2),avg(ResolutionWidth)=AVG($3)), 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}},"_source":{"includes":["WatchID","SearchPhrase","IsRefresh","ClientIP","ResolutionWidth"],"excludes":[]},"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)]) + 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/q33.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q33.yaml index c143a17262a..e9b5c203f20 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q33.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q33.yaml @@ -8,4 +8,4 @@ calcite: LogicalFilter(condition=[AND(IS NOT NULL($41), IS NOT NULL($76))]) CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},c=COUNT(),sum(IsRefresh)=SUM($2),avg(ResolutionWidth)=AVG($3)), PROJECT->[c, sum(IsRefresh), avg(ResolutionWidth), WatchID, ClientIP], SORT_AGG_METRICS->[0 DESC LAST], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","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)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 2},c=COUNT(),sum(IsRefresh)=SUM($1),avg(ResolutionWidth)=AVG($3)), 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","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/q34.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q34.yaml index 97f48a87f28..e2fd395e0ec 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q34.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q34.yaml @@ -8,4 +8,4 @@ calcite: LogicalFilter(condition=[IS NOT NULL($26)]) CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},c=COUNT()), PROJECT->[c, URL], SORT_AGG_METRICS->[0 DESC LAST], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","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)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},c=COUNT()), SORT_AGG_METRICS->[1 DESC LAST], PROJECT->[c, URL], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","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/q36.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q36.yaml index 55560215771..5f1b457eaa1 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q36.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q36.yaml @@ -10,4 +10,4 @@ calcite: CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], expr#3=[-($t0, $t2)], expr#4=[2], expr#5=[-($t0, $t4)], expr#6=[3], expr#7=[-($t0, $t6)], c=[$t1], ClientIP=[$t0], ClientIP - 1=[$t3], ClientIP - 2=[$t5], ClientIP - 3=[$t7]) - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER->IS NOT NULL($76), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},c=COUNT()), SORT_AGG_METRICS->[1 DESC LAST], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"exists":{"field":"ClientIP","boost":1.0}},"aggregations":{"ClientIP":{"terms":{"field":"ClientIP","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)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},c=COUNT()), SORT_AGG_METRICS->[1 DESC LAST], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"ClientIP":{"terms":{"field":"ClientIP","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/q37.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q37.yaml index cea77806c98..5cf2fb92261 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q37.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q37.yaml @@ -8,4 +8,4 @@ calcite: 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=[[PROJECT->[EventDate, URL, DontCountHits, IsRefresh, CounterID], 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={0},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}},"_source":{"includes":["EventDate","URL","DontCountHits","IsRefresh","CounterID"],"excludes":[]},"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)]) \ No newline at end of file + 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 711cf5bc29e..5ae72b31e00 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q38.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q38.yaml @@ -8,4 +8,4 @@ calcite: 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=[[PROJECT->[EventDate, DontCountHits, IsRefresh, Title, CounterID], 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={0},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}},"_source":{"includes":["EventDate","DontCountHits","IsRefresh","Title","CounterID"],"excludes":[]},"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)]) + 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 08ea4e0a45b..6f08007b702 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q39.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q39.yaml @@ -10,4 +10,4 @@ calcite: physical: | EnumerableLimit(fetch=[10000]) EnumerableLimit(offset=[1000], fetch=[10]) - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[PROJECT->[EventDate, URL, IsDownload, IsLink, IsRefresh, CounterID], FILTER->AND(=($5, 62), SEARCH($0, Sarg[['2013-07-01 00:00:00':VARCHAR..'2013-07-31 00:00:00':VARCHAR]]:VARCHAR), =($4, 0), <>($3, 0), =($2, 0), IS NOT NULL($1)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},PageViews=COUNT()), SORT_AGG_METRICS->[1 DESC LAST], PROJECT->[PageViews, URL]], 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":{"IsRefresh":{"value":0,"boost":1.0}}},{"bool":{"must":[{"exists":{"field":"IsLink","boost":1.0}}],"must_not":[{"term":{"IsLink":{"value":0,"boost":1.0}}}],"adjust_pure_negative":true,"boost":1.0}},{"term":{"IsDownload":{"value":0,"boost":1.0}}},{"exists":{"field":"URL","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"_source":{"includes":["EventDate","URL","IsDownload","IsLink","IsRefresh","CounterID"],"excludes":[]},"aggregations":{"URL":{"terms":{"field":"URL","size":1010,"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)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER->AND(=($5, 62), SEARCH($0, Sarg[['2013-07-01 00:00:00':VARCHAR..'2013-07-31 00:00:00':VARCHAR]]:VARCHAR), =($4, 0), <>($3, 0), =($2, 0), IS NOT NULL($1)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={1},PageViews=COUNT()), SORT_AGG_METRICS->[1 DESC LAST], PROJECT->[PageViews, URL], LIMIT->[10 from 1000]], 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":{"IsRefresh":{"value":0,"boost":1.0}}},{"bool":{"must":[{"exists":{"field":"IsLink","boost":1.0}}],"must_not":[{"term":{"IsLink":{"value":0,"boost":1.0}}}],"adjust_pure_negative":true,"boost":1.0}},{"term":{"IsDownload":{"value":0,"boost":1.0}}},{"exists":{"field":"URL","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"URL":{"terms":{"field":"URL","size":1010,"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/q40.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q40.yaml index 0500f94da78..e9eefc046b2 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q40.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q40.yaml @@ -10,7 +10,6 @@ calcite: 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))]) CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | - EnumerableCalc(expr#0..5=[{inputs}], PageViews=[$t5], TraficSourceID=[$t0], SearchEngineID=[$t1], AdvEngineID=[$t2], Src=[$t3], Dst=[$t4]) - EnumerableLimit(fetch=[10000]) - EnumerableLimit(offset=[1000], fetch=[10]) - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[PROJECT->[EventDate, TraficSourceID, AdvEngineID, URL, SearchEngineID, IsRefresh, Referer, CounterID], FILTER->AND(=($7, 62), SEARCH($0, Sarg[['2013-07-01 00:00:00':VARCHAR..'2013-07-31 00:00:00':VARCHAR]]:VARCHAR), =($5, 0)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1, 2, 3, 4},PageViews=COUNT()), SORT_AGG_METRICS->[5 DESC LAST], LIMIT->[10 from 1000]], 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":{"IsRefresh":{"value":0,"boost":1.0}}}],"adjust_pure_negative":true,"boost":1.0}},"_source":{"includes":["EventDate","TraficSourceID","AdvEngineID","URL","SearchEngineID","IsRefresh","Referer","CounterID"],"excludes":[]},"aggregations":{"TraficSourceID|SearchEngineID|AdvEngineID|Src|Dst":{"multi_terms":{"terms":[{"field":"TraficSourceID"},{"field":"SearchEngineID"},{"field":"AdvEngineID"},{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQGAXsKICAib3AiOiB7CiAgICAibmFtZSI6ICJDQVNFIiwKICAgICJraW5kIjogIkNBU0UiLAogICAgInN5bnRheCI6ICJTUEVDSUFMIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAib3AiOiB7CiAgICAgICAgIm5hbWUiOiAiQU5EIiwKICAgICAgICAia2luZCI6ICJBTkQiLAogICAgICAgICJzeW50YXgiOiAiQklOQVJZIgogICAgICB9LAogICAgICAib3BlcmFuZHMiOiBbCiAgICAgICAgewogICAgICAgICAgIm9wIjogewogICAgICAgICAgICAibmFtZSI6ICI9IiwKICAgICAgICAgICAgImtpbmQiOiAiRVFVQUxTIiwKICAgICAgICAgICAgInN5bnRheCI6ICJCSU5BUlkiCiAgICAgICAgICB9LAogICAgICAgICAgIm9wZXJhbmRzIjogWwogICAgICAgICAgICB7CiAgICAgICAgICAgICAgImR5bmFtaWNQYXJhbSI6IDAsCiAgICAgICAgICAgICAgInR5cGUiOiB7CiAgICAgICAgICAgICAgICAidHlwZSI6ICJCSUdJTlQiLAogICAgICAgICAgICAgICAgIm51bGxhYmxlIjogdHJ1ZQogICAgICAgICAgICAgIH0KICAgICAgICAgICAgfSwKICAgICAgICAgICAgewogICAgICAgICAgICAgICJkeW5hbWljUGFyYW0iOiAxLAogICAgICAgICAgICAgICJ0eXBlIjogewogICAgICAgICAgICAgICAgInR5cGUiOiAiQklHSU5UIiwKICAgICAgICAgICAgICAgICJudWxsYWJsZSI6IHRydWUKICAgICAgICAgICAgICB9CiAgICAgICAgICAgIH0KICAgICAgICAgIF0KICAgICAgICB9LAogICAgICAgIHsKICAgICAgICAgICJvcCI6IHsKICAgICAgICAgICAgIm5hbWUiOiAiPSIsCiAgICAgICAgICAgICJraW5kIjogIkVRVUFMUyIsCiAgICAgICAgICAgICJzeW50YXgiOiAiQklOQVJZIgogICAgICAgICAgfSwKICAgICAgICAgICJvcGVyYW5kcyI6IFsKICAgICAgICAgICAgewogICAgICAgICAgICAgICJkeW5hbWljUGFyYW0iOiAyLAogICAgICAgICAgICAgICJ0eXBlIjogewogICAgICAgICAgICAgICAgInR5cGUiOiAiQklHSU5UIiwKICAgICAgICAgICAgICAgICJudWxsYWJsZSI6IHRydWUKICAgICAgICAgICAgICB9CiAgICAgICAgICAgIH0sCiAgICAgICAgICAgIHsKICAgICAgICAgICAgICAiZHluYW1pY1BhcmFtIjogMywKICAgICAgICAgICAgICAidHlwZSI6IHsKICAgICAgICAgICAgICAgICJ0eXBlIjogIkJJR0lOVCIsCiAgICAgICAgICAgICAgICAibnVsbGFibGUiOiB0cnVlCiAgICAgICAgICAgICAgfQogICAgICAgICAgICB9CiAgICAgICAgICBdCiAgICAgICAgfQogICAgICBdCiAgICB9LAogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogNCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfSwKICAgIHsKICAgICAgImR5bmFtaWNQYXJhbSI6IDUsCiAgICAgICJ0eXBlIjogewogICAgICAgICJ0eXBlIjogIlZBUkNIQVIiLAogICAgICAgICJudWxsYWJsZSI6IHRydWUsCiAgICAgICAgInByZWNpc2lvbiI6IC0xCiAgICAgIH0KICAgIH0KICBdCn0=\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0,2,0,2,0,2],"DIGESTS":["AdvEngineID",0,"SearchEngineID",0,"Referer",""]}}},{"field":"URL"}],"size":1010,"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)]) + EnumerableLimit(fetch=[10000]) + EnumerableLimit(offset=[1000], fetch=[10]) + CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER->AND(=($7, 62), SEARCH($0, Sarg[['2013-07-01 00:00:00':VARCHAR..'2013-07-31 00:00:00':VARCHAR]]:VARCHAR), =($5, 0)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1, 2, 3, 4},PageViews=COUNT()), SORT_AGG_METRICS->[5 DESC LAST], PROJECT->[PageViews, TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst], LIMIT->[10 from 1000]], 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":{"IsRefresh":{"value":0,"boost":1.0}}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"TraficSourceID|AdvEngineID|SearchEngineID|Src|Dst":{"multi_terms":{"terms":[{"field":"TraficSourceID"},{"field":"AdvEngineID"},{"field":"SearchEngineID"},{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQGAXsKICAib3AiOiB7CiAgICAibmFtZSI6ICJDQVNFIiwKICAgICJraW5kIjogIkNBU0UiLAogICAgInN5bnRheCI6ICJTUEVDSUFMIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAib3AiOiB7CiAgICAgICAgIm5hbWUiOiAiQU5EIiwKICAgICAgICAia2luZCI6ICJBTkQiLAogICAgICAgICJzeW50YXgiOiAiQklOQVJZIgogICAgICB9LAogICAgICAib3BlcmFuZHMiOiBbCiAgICAgICAgewogICAgICAgICAgIm9wIjogewogICAgICAgICAgICAibmFtZSI6ICI9IiwKICAgICAgICAgICAgImtpbmQiOiAiRVFVQUxTIiwKICAgICAgICAgICAgInN5bnRheCI6ICJCSU5BUlkiCiAgICAgICAgICB9LAogICAgICAgICAgIm9wZXJhbmRzIjogWwogICAgICAgICAgICB7CiAgICAgICAgICAgICAgImR5bmFtaWNQYXJhbSI6IDAsCiAgICAgICAgICAgICAgInR5cGUiOiB7CiAgICAgICAgICAgICAgICAidHlwZSI6ICJCSUdJTlQiLAogICAgICAgICAgICAgICAgIm51bGxhYmxlIjogdHJ1ZQogICAgICAgICAgICAgIH0KICAgICAgICAgICAgfSwKICAgICAgICAgICAgewogICAgICAgICAgICAgICJkeW5hbWljUGFyYW0iOiAxLAogICAgICAgICAgICAgICJ0eXBlIjogewogICAgICAgICAgICAgICAgInR5cGUiOiAiQklHSU5UIiwKICAgICAgICAgICAgICAgICJudWxsYWJsZSI6IHRydWUKICAgICAgICAgICAgICB9CiAgICAgICAgICAgIH0KICAgICAgICAgIF0KICAgICAgICB9LAogICAgICAgIHsKICAgICAgICAgICJvcCI6IHsKICAgICAgICAgICAgIm5hbWUiOiAiPSIsCiAgICAgICAgICAgICJraW5kIjogIkVRVUFMUyIsCiAgICAgICAgICAgICJzeW50YXgiOiAiQklOQVJZIgogICAgICAgICAgfSwKICAgICAgICAgICJvcGVyYW5kcyI6IFsKICAgICAgICAgICAgewogICAgICAgICAgICAgICJkeW5hbWljUGFyYW0iOiAyLAogICAgICAgICAgICAgICJ0eXBlIjogewogICAgICAgICAgICAgICAgInR5cGUiOiAiQklHSU5UIiwKICAgICAgICAgICAgICAgICJudWxsYWJsZSI6IHRydWUKICAgICAgICAgICAgICB9CiAgICAgICAgICAgIH0sCiAgICAgICAgICAgIHsKICAgICAgICAgICAgICAiZHluYW1pY1BhcmFtIjogMywKICAgICAgICAgICAgICAidHlwZSI6IHsKICAgICAgICAgICAgICAgICJ0eXBlIjogIkJJR0lOVCIsCiAgICAgICAgICAgICAgICAibnVsbGFibGUiOiB0cnVlCiAgICAgICAgICAgICAgfQogICAgICAgICAgICB9CiAgICAgICAgICBdCiAgICAgICAgfQogICAgICBdCiAgICB9LAogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogNCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfSwKICAgIHsKICAgICAgImR5bmFtaWNQYXJhbSI6IDUsCiAgICAgICJ0eXBlIjogewogICAgICAgICJ0eXBlIjogIlZBUkNIQVIiLAogICAgICAgICJudWxsYWJsZSI6IHRydWUsCiAgICAgICAgInByZWNpc2lvbiI6IC0xCiAgICAgIH0KICAgIH0KICBdCn0=\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0,2,0,2,0,2],"DIGESTS":["SearchEngineID",0,"AdvEngineID",0,"Referer",""]}}},{"field":"URL"}],"size":1010,"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/q41.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q41.yaml index cb31cbd45db..dc908d4df76 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q41.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q41.yaml @@ -10,4 +10,4 @@ calcite: physical: | EnumerableLimit(fetch=[10000]) EnumerableLimit(offset=[100], fetch=[10]) - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[PROJECT->[EventDate, RefererHash, TraficSourceID, URLHash, IsRefresh, CounterID], FILTER->AND(=($5, 62), SEARCH($0, Sarg[['2013-07-01 00:00:00':VARCHAR..'2013-07-31 00:00:00':VARCHAR]]:VARCHAR), =($4, 0), SEARCH($2, Sarg[-1, 6]), =($1, 3594120000172545465), IS NOT NULL($3)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},PageViews=COUNT()), SORT_AGG_METRICS->[2 DESC LAST], PROJECT->[PageViews, URLHash, EventDate]], 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":{"IsRefresh":{"value":0,"boost":1.0}}},{"terms":{"TraficSourceID":[-1.0,6.0],"boost":1.0}},{"term":{"RefererHash":{"value":3594120000172545465,"boost":1.0}}},{"exists":{"field":"URLHash","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"_source":{"includes":["EventDate","RefererHash","TraficSourceID","URLHash","IsRefresh","CounterID"],"excludes":[]},"aggregations":{"URLHash|EventDate":{"multi_terms":{"terms":[{"field":"URLHash"},{"field":"EventDate","value_type":"long"}],"size":110,"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)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER->AND(=($5, 62), SEARCH($0, Sarg[['2013-07-01 00:00:00':VARCHAR..'2013-07-31 00:00:00':VARCHAR]]:VARCHAR), =($4, 0), SEARCH($2, Sarg[-1, 6]), =($1, 3594120000172545465), IS NOT NULL($3)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 3},PageViews=COUNT()), SORT_AGG_METRICS->[2 DESC LAST], PROJECT->[PageViews, URLHash, EventDate], LIMIT->[10 from 100]], 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":{"IsRefresh":{"value":0,"boost":1.0}}},{"terms":{"TraficSourceID":[-1.0,6.0],"boost":1.0}},{"term":{"RefererHash":{"value":3594120000172545465,"boost":1.0}}},{"exists":{"field":"URLHash","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"EventDate|URLHash":{"multi_terms":{"terms":[{"field":"EventDate","value_type":"long"},{"field":"URLHash"}],"size":110,"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/q42.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q42.yaml index 6e9ced0f691..3b0bf781c00 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q42.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q42.yaml @@ -10,4 +10,4 @@ calcite: physical: | EnumerableLimit(fetch=[10000]) EnumerableLimit(offset=[10000], fetch=[10]) - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[PROJECT->[EventDate, URLHash, DontCountHits, WindowClientHeight, IsRefresh, CounterID, WindowClientWidth], FILTER->AND(=($5, 62), SEARCH($0, Sarg[['2013-07-01 00:00:00':VARCHAR..'2013-07-31 00:00:00':VARCHAR]]:VARCHAR), =($4, 0), =($2, 0), =($1, 2868770270353813622), IS NOT NULL($6), IS NOT NULL($3)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},PageViews=COUNT()), SORT_AGG_METRICS->[2 DESC LAST], PROJECT->[PageViews, WindowClientWidth, WindowClientHeight]], 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":{"IsRefresh":{"value":0,"boost":1.0}}},{"term":{"DontCountHits":{"value":0,"boost":1.0}}},{"term":{"URLHash":{"value":2868770270353813622,"boost":1.0}}},{"exists":{"field":"WindowClientWidth","boost":1.0}},{"exists":{"field":"WindowClientHeight","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"_source":{"includes":["EventDate","URLHash","DontCountHits","WindowClientHeight","IsRefresh","CounterID","WindowClientWidth"],"excludes":[]},"aggregations":{"WindowClientWidth|WindowClientHeight":{"multi_terms":{"terms":[{"field":"WindowClientWidth"},{"field":"WindowClientHeight"}],"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)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER->AND(=($5, 62), SEARCH($0, Sarg[['2013-07-01 00:00:00':VARCHAR..'2013-07-31 00:00:00':VARCHAR]]:VARCHAR), =($4, 0), =($2, 0), =($1, 2868770270353813622), IS NOT NULL($6), IS NOT NULL($3)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={3, 6},PageViews=COUNT()), SORT_AGG_METRICS->[2 DESC LAST], PROJECT->[PageViews, WindowClientWidth, WindowClientHeight]], 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":{"IsRefresh":{"value":0,"boost":1.0}}},{"term":{"DontCountHits":{"value":0,"boost":1.0}}},{"term":{"URLHash":{"value":2868770270353813622,"boost":1.0}}},{"exists":{"field":"WindowClientWidth","boost":1.0}},{"exists":{"field":"WindowClientHeight","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"WindowClientHeight|WindowClientWidth":{"multi_terms":{"terms":[{"field":"WindowClientHeight"},{"field":"WindowClientWidth"}],"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/clickbench/q43.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q43.yaml index d261f22d6f5..5b2c5816127 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q43.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q43.yaml @@ -8,7 +8,6 @@ calcite: 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]]) physical: | - EnumerableCalc(expr#0..1=[{inputs}], PageViews=[$t1], M=[$t0]) - EnumerableLimit(fetch=[10000]) - EnumerableLimit(offset=[1000], fetch=[10]) - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[PROJECT->[EventDate, EventTime, DontCountHits, IsRefresh, CounterID], FILTER->AND(=($4, 62), SEARCH($0, Sarg[['2013-07-01 00:00:00':VARCHAR..'2013-07-15 00:00:00':VARCHAR]]:VARCHAR), =($3, 0), =($2, 0), IS NOT NULL($1)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},PageViews=COUNT()), SORT->[0 ASC FIRST], LIMIT->[10 from 1000]], 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-15T00:00:00.000Z","include_lower":true,"include_upper":true,"format":"date_time","boost":1.0}}},{"term":{"IsRefresh":{"value":0,"boost":1.0}}},{"term":{"DontCountHits":{"value":0,"boost":1.0}}},{"exists":{"field":"EventTime","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"_source":{"includes":["EventDate","EventTime","DontCountHits","IsRefresh","CounterID"],"excludes":[]},"aggregations":{"composite_buckets":{"composite":{"size":1010,"sources":[{"M":{"date_histogram":{"field":"EventTime","missing_bucket":false,"order":"asc","fixed_interval":"1m"}}}]}}}}, requestedTotalSize=1010, pageSize=null, startFrom=0)]) \ No newline at end of file + EnumerableLimit(fetch=[10000]) + EnumerableLimit(offset=[1000], fetch=[10]) + CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER->AND(=($4, 62), SEARCH($0, Sarg[['2013-07-01 00:00:00':VARCHAR..'2013-07-15 00:00:00':VARCHAR]]:VARCHAR), =($3, 0), =($2, 0), IS NOT NULL($1)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},PageViews=COUNT()), PROJECT->[PageViews, M], SORT->[1 ASC FIRST], LIMIT->[10 from 1000]], 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-15T00:00:00.000Z","include_lower":true,"include_upper":true,"format":"date_time","boost":1.0}}},{"term":{"IsRefresh":{"value":0,"boost":1.0}}},{"term":{"DontCountHits":{"value":0,"boost":1.0}}},{"exists":{"field":"EventTime","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"composite_buckets":{"composite":{"size":1010,"sources":[{"M":{"date_histogram":{"field":"EventTime","missing_bucket":false,"order":"asc","fixed_interval":"1m"}}}]}}}}, requestedTotalSize=1010, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q5.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q5.yaml index 3edc0813519..74426be60cc 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q5.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q5.yaml @@ -6,4 +6,4 @@ calcite: LogicalFilter(condition=[IS NOT NULL($84)]) CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER->IS NOT NULL($84), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={},dc(UserID)=COUNT(DISTINCT $0)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"exists":{"field":"UserID","boost":1.0}},"aggregations":{"dc(UserID)":{"cardinality":{"field":"UserID"}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER->IS NOT NULL($0), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={},dc(UserID)=COUNT(DISTINCT $0)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"exists":{"field":"UserID","boost":1.0}},"aggregations":{"dc(UserID)":{"cardinality":{"field":"UserID"}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q6.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q6.yaml index 9a8d579de98..e74504e544d 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q6.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q6.yaml @@ -6,4 +6,4 @@ calcite: LogicalFilter(condition=[IS NOT NULL($63)]) CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER->IS NOT NULL($63), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={},dc(SearchPhrase)=COUNT(DISTINCT $0)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"exists":{"field":"SearchPhrase","boost":1.0}},"aggregations":{"dc(SearchPhrase)":{"cardinality":{"field":"SearchPhrase"}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER->IS NOT NULL($0), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={},dc(SearchPhrase)=COUNT(DISTINCT $0)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"exists":{"field":"SearchPhrase","boost":1.0}},"aggregations":{"dc(SearchPhrase)":{"cardinality":{"field":"SearchPhrase"}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) 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 343aade48c4..e4ba0272b81 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q8.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q8.yaml @@ -8,4 +8,4 @@ calcite: LogicalFilter(condition=[<>($19, 0)]) CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER-><>($19, 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)]) \ No newline at end of file + 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/clickbench/q9.yaml b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q9.yaml index 5e6bc1617c5..6ea001905ce 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q9.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q9.yaml @@ -8,4 +8,4 @@ calcite: LogicalFilter(condition=[IS NOT NULL($68)]) CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},u=COUNT(DISTINCT $1)), PROJECT->[u, RegionID], SORT_AGG_METRICS->[0 DESC LAST], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"RegionID":{"terms":{"field":"RegionID","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)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},u=COUNT(DISTINCT $1)), SORT_AGG_METRICS->[1 DESC LAST], PROJECT->[u, RegionID], LIMIT->10, LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"RegionID":{"terms":{"field":"RegionID","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/explain_agg_paginating_head_from.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_agg_paginating_head_from.yaml index cfe1a0a29df..ccb30a4d6e4 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_agg_paginating_head_from.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_agg_paginating_head_from.yaml @@ -9,4 +9,4 @@ calcite: physical: | EnumerableLimit(fetch=[10000]) EnumerableLimit(offset=[2], fetch=[10]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},c=COUNT()), PROJECT->[c, state]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":2,"sources":[{"state":{"terms":{"field":"state.keyword","missing_bucket":true,"missing_order":"first","order":"asc"}}}]}}}}, 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=RelSubset#,group={0},c=COUNT()), PROJECT->[c, state], LIMIT->[10 from 2]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":2,"sources":[{"state":{"terms":{"field":"state.keyword","missing_bucket":true,"missing_order":"first","order":"asc"}}}]}}}}, requestedTotalSize=12, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_agg_paginating_join4.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_agg_paginating_join4.yaml index 51d82f37450..36bf1245a2d 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_agg_paginating_join4.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_agg_paginating_join4.yaml @@ -15,6 +15,7 @@ calcite: CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | EnumerableLimit(fetch=[10000]) - EnumerableHashJoin(condition=[=($1, $2)], joinType=[semi]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},c=COUNT()), PROJECT->[c, state], LIMIT->50000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":2,"sources":[{"state":{"terms":{"field":"state.keyword","missing_bucket":true,"missing_order":"last","order":"asc"}}}]}}}}, requestedTotalSize=50000, pageSize=null, startFrom=0)]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0}), LIMIT->10], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":2,"sources":[{"state":{"terms":{"field":"state.keyword","missing_bucket":true,"missing_order":"last","order":"asc"}}}]}}}}, requestedTotalSize=10, pageSize=null, startFrom=0)]) \ No newline at end of file + EnumerableCalc(expr#0..2=[{inputs}], c=[$t1], state=[$t2]) + EnumerableHashJoin(condition=[=($0, $2)], joinType=[inner]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0}), LIMIT->10], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":2,"sources":[{"state":{"terms":{"field":"state.keyword","missing_bucket":true,"missing_order":"last","order":"asc"}}}]}}}}, requestedTotalSize=10, pageSize=null, startFrom=0)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},c=COUNT()), PROJECT->[c, state], LIMIT->50000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":2,"sources":[{"state":{"terms":{"field":"state.keyword","missing_bucket":true,"missing_order":"last","order":"asc"}}}]}}}}, requestedTotalSize=50000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_agg_sort_on_measure2.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_agg_sort_on_measure2.yaml index 734e6ea1571..be021c55e23 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_agg_sort_on_measure2.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_agg_sort_on_measure2.yaml @@ -8,4 +8,4 @@ calcite: LogicalFilter(condition=[IS NOT NULL($7)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},sum=SUM($1)), SORT_AGG_METRICS->[1 DESC LAST], PROJECT->[sum, state], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"state":{"terms":{"field":"state.keyword","size":1000,"min_doc_count":1,"shard_min_doc_count":0,"show_term_doc_count_error":false,"order":[{"sum":"desc"},{"_key":"asc"}]},"aggregations":{"sum":{"sum":{"field":"balance"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={1},sum=SUM($0)), SORT_AGG_METRICS->[1 DESC LAST], PROJECT->[sum, state], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"state":{"terms":{"field":"state.keyword","size":1000,"min_doc_count":1,"shard_min_doc_count":0,"show_term_doc_count_error":false,"order":[{"sum":"desc"},{"_key":"asc"}]},"aggregations":{"sum":{"sum":{"field":"balance"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_agg_sort_on_measure_complex1.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_agg_sort_on_measure_complex1.yaml index 6a1e114acc5..3215115297a 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_agg_sort_on_measure_complex1.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_agg_sort_on_measure_complex1.yaml @@ -8,4 +8,4 @@ calcite: LogicalFilter(condition=[IS NOT NULL($7)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},sum(balance)=SUM($1),c=COUNT(),dc(employer)=COUNT(DISTINCT $2)), SORT_AGG_METRICS->[2 DESC LAST], PROJECT->[sum(balance), c, dc(employer), state], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"state":{"terms":{"field":"state.keyword","size":1000,"min_doc_count":1,"shard_min_doc_count":0,"show_term_doc_count_error":false,"order":[{"c":"desc"},{"_key":"asc"}]},"aggregations":{"sum(balance)":{"sum":{"field":"balance"}},"dc(employer)":{"cardinality":{"field":"employer.keyword"}},"c":{"value_count":{"field":"_index"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={2},sum(balance)=SUM($0),c=COUNT(),dc(employer)=COUNT(DISTINCT $1)), SORT_AGG_METRICS->[2 DESC LAST], PROJECT->[sum(balance), c, dc(employer), state], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"state":{"terms":{"field":"state.keyword","size":1000,"min_doc_count":1,"shard_min_doc_count":0,"show_term_doc_count_error":false,"order":[{"c":"desc"},{"_key":"asc"}]},"aggregations":{"sum(balance)":{"sum":{"field":"balance"}},"dc(employer)":{"cardinality":{"field":"employer.keyword"}},"c":{"value_count":{"field":"_index"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_agg_with_script.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_agg_with_script.yaml index f5a0b3abcd9..c543431c519 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_agg_with_script.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_agg_with_script.yaml @@ -6,5 +6,5 @@ calcite: LogicalProject(len=[CHAR_LENGTH($4)], gender=[$4], $f3=[+($7, 100)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]]) physical: | - EnumerableCalc(expr#0..1=[{inputs}], expr#2=[CHAR_LENGTH($t0)], sum=[$t1], len=[$t2], gender=[$t0]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},sum=SUM($1)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"gender":{"terms":{"field":"gender.keyword","missing_bucket":true,"missing_order":"first","order":"asc"}}}]},"aggregations":{"sum":{"sum":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQBQ3sKICAib3AiOiB7CiAgICAibmFtZSI6ICIrIiwKICAgICJraW5kIjogIlBMVVMiLAogICAgInN5bnRheCI6ICJCSU5BUlkiCiAgfSwKICAib3BlcmFuZHMiOiBbCiAgICB7CiAgICAgICJkeW5hbWljUGFyYW0iOiAwLAogICAgICAidHlwZSI6IHsKICAgICAgICAidHlwZSI6ICJCSUdJTlQiLAogICAgICAgICJudWxsYWJsZSI6IHRydWUKICAgICAgfQogICAgfSwKICAgIHsKICAgICAgImR5bmFtaWNQYXJhbSI6IDEsCiAgICAgICJ0eXBlIjogewogICAgICAgICJ0eXBlIjogIkJJR0lOVCIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZQogICAgICB9CiAgICB9CiAgXQp9\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0,2],"DIGESTS":["balance",100]}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) + EnumerableCalc(expr#0..2=[{inputs}], expr#3=[100], expr#4=[*($t2, $t3)], expr#5=[+($t1, $t4)], expr#6=[CHAR_LENGTH($t0)], sum=[$t5], len=[$t6], gender=[$t0]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},sum_SUM=SUM($1),sum_COUNT=COUNT($1)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"gender":{"terms":{"field":"gender.keyword","missing_bucket":true,"missing_order":"first","order":"asc"}}}]},"aggregations":{"sum_SUM":{"sum":{"field":"balance"}},"sum_COUNT":{"value_count":{"field":"balance"}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_complex_sort_expr_no_expr_output_push.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_complex_sort_expr_no_expr_output_push.yaml index 4b8055f4919..c7b818113f5 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_complex_sort_expr_no_expr_output_push.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_complex_sort_expr_no_expr_output_push.yaml @@ -6,4 +6,4 @@ 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: | - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[SORT_EXPR->[+($10, $7) ASCENDING NULLS_FIRST], LIMIT->10000, PROJECT->[age]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10000,"timeout":"1m","_source":{"includes":["age"],"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)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[PROJECT->[age, balance], SORT_EXPR->[+($0, $1) ASCENDING NULLS_FIRST], LIMIT->10000, PROJECT->[age]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10000,"timeout":"1m","_source":{"includes":["age"],"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_w_max_option.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_complex_sort_expr_pushdown_for_smj_w_max_option.yaml index 2e1eaf42f88..6a9d9fd09b2 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 @@ -18,4 +18,4 @@ calcite: 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]) 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","_id","_index","_score","_maxscore","_sort","_routing"],"excludes":[]},"script_fields":{}}}}}}}, requestedTotalSize=50000, pageSize=null, startFrom=0)]) \ No newline at end of file + 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_count_agg_push2.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_count_agg_push2.yaml index 17b847a9261..77542ba4570 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_count_agg_push2.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_count_agg_push2.yaml @@ -6,4 +6,4 @@ calcite: LogicalFilter(condition=[IS NOT NULL($10)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[FILTER->IS NOT NULL($10), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={},cnt=COUNT($0)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"exists":{"field":"lastname","boost":1.0}},"track_total_hits":2147483647}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[FILTER->IS NOT NULL($0), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={},cnt=COUNT($0)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"exists":{"field":"lastname","boost":1.0}},"track_total_hits":2147483647}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_count_agg_push3.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_count_agg_push3.yaml index c2d092293b8..bd1fff2449a 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_count_agg_push3.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_count_agg_push3.yaml @@ -7,4 +7,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], name=[$10]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[FILTER->IS NOT NULL($10), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={},cnt=COUNT($1)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"exists":{"field":"lastname","boost":1.0}},"track_total_hits":2147483647}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[FILTER->IS NOT NULL($0), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={},cnt=COUNT($1)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"exists":{"field":"lastname","boost":1.0}},"track_total_hits":2147483647}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_count_agg_push5.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_count_agg_push5.yaml index a89a1923ee2..933c7bab9b8 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_count_agg_push5.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_count_agg_push5.yaml @@ -8,4 +8,4 @@ calcite: CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | EnumerableCalc(expr#0=[{inputs}], c1=[$t0], c2=[$t0]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[FILTER->IS NOT NULL($10), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={},c1=COUNT($0)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"exists":{"field":"lastname","boost":1.0}},"track_total_hits":2147483647}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[FILTER->IS NOT NULL($0), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={},c1=COUNT($0)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"exists":{"field":"lastname","boost":1.0}},"track_total_hits":2147483647}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_count_agg_push6.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_count_agg_push6.yaml index 603ef7a563d..5a3eb44057f 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_count_agg_push6.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_count_agg_push6.yaml @@ -7,4 +7,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], name=[$10]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[FILTER->IS NOT NULL($10), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={},count(lastname)=COUNT($0),count(name)=COUNT($1)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"exists":{"field":"lastname","boost":1.0}},"track_total_hits":2147483647}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[FILTER->IS NOT NULL($0), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={},count(lastname)=COUNT($0),count(name)=COUNT($1)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"exists":{"field":"lastname","boost":1.0}},"track_total_hits":2147483647}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_complex1.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_complex1.yaml index de7047f0161..17aef474306 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_complex1.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_complex1.yaml @@ -7,4 +7,4 @@ calcite: LogicalFilter(condition=[IS NOT NULL($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":[{"gender":{"terms":{"field":"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":["gender","account_number","firstname","address","balance","city","employer","state","age","email","lastname","_id","_index","_score","_maxscore","_sort","_routing"],"excludes":[]},"script_fields":{}}}}}}}, 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":[{"gender":{"terms":{"field":"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":["gender","account_number","firstname","address","balance","city","employer","state","age","email","lastname"],"excludes":[]},"script_fields":{}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_complex2.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_complex2.yaml index 0d045fb7a5f..ec8172633bb 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_complex2.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_complex2.yaml @@ -8,4 +8,4 @@ calcite: LogicalProject(account_number=[$0], gender=[$4], age=[$8], state=[$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":[{"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":1,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["gender","state","account_number","firstname","address","balance","city","employer","age","email","lastname","_id","_index","_score","_maxscore","_sort","_routing"],"excludes":[]},"script_fields":{}}}}}}}, 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":[{"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":1,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["gender","state","account_number","age"],"excludes":[]},"script_fields":{}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_complex3.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_complex3.yaml index d5b816ff644..c31ff1a3eb3 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_complex3.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_complex3.yaml @@ -7,4 +7,4 @@ calcite: LogicalFilter(condition=[AND(IS NOT NULL($4), IS NOT NULL($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","firstname","address","balance","city","employer","age","email","lastname","_id","_index","_score","_maxscore","_sort","_routing"],"excludes":[]},"script_fields":{}}}}}}}, 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","firstname","address","balance","city","employer","age","email","lastname"],"excludes":[]},"script_fields":{}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_complex4.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_complex4.yaml index cbe56d53c56..345d3829266 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_complex4.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_complex4.yaml @@ -8,4 +8,4 @@ calcite: LogicalProject(account_number=[$0], gender=[$4], age=[$8], state=[$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","firstname","address","balance","city","employer","age","email","lastname","_id","_index","_score","_maxscore","_sort","_routing"],"excludes":[]},"script_fields":{}}}}}}}, 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":{}}}}}}}, 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 e8dffe4a7a8..54b69192b13 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,6 +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]) - EnumerableSort(sort0=[$1], sort1=[$3], dir0=[ASC-nulls-first], dir1=[DESC-nulls-last]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(2))], 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=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)), 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_expr4_alternative.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_expr4_alternative.yaml index f441f5d8e12..ea9c46e976c 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,6 +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]) - EnumerableSort(sort0=[$1], sort1=[$3], dir0=[ASC-nulls-first], dir1=[DESC-nulls-last]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(2))], 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=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)), 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_keepempty_false_push.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_keepempty_false_push.yaml index b35dfe6adb3..ea1dbba807d 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_keepempty_false_push.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_keepempty_false_push.yaml @@ -8,4 +8,4 @@ calcite: LogicalProject(account_number=[$0], gender=[$4], age=[$8]) 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":[{"gender":{"terms":{"field":"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":["gender","account_number","firstname","address","balance","city","employer","state","age","email","lastname","_id","_index","_score","_maxscore","_sort","_routing"],"excludes":[]},"script_fields":{}}}}}}}, 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":[{"gender":{"terms":{"field":"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":["gender","account_number","age"],"excludes":[]},"script_fields":{}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_push.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_push.yaml index b35dfe6adb3..ea1dbba807d 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_push.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_push.yaml @@ -8,4 +8,4 @@ calcite: LogicalProject(account_number=[$0], gender=[$4], age=[$8]) 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":[{"gender":{"terms":{"field":"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":["gender","account_number","firstname","address","balance","city","employer","state","age","email","lastname","_id","_index","_score","_maxscore","_sort","_routing"],"excludes":[]},"script_fields":{}}}}}}}, 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":[{"gender":{"terms":{"field":"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":["gender","account_number","age"],"excludes":[]},"script_fields":{}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_expr3.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_expr3.yaml index 95a1627f212..6f84e2f856d 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_expr3.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_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":[{"account_number":{"terms":{"field":"account_number","missing_bucket":false,"order":"asc"}}},{"age":{"terms":{"field":"age","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","age","firstname","address","balance","gender","city","employer","state","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":[{"age":{"terms":{"field":"age","missing_bucket":false,"order":"asc"}}},{"account_number":{"terms":{"field":"account_number","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":2,"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_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_expr3_alternative.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_expr3_alternative.yaml index be3cbf82623..3e647b1d4b6 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_expr3_alternative.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_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":[{"account_number":{"terms":{"field":"account_number","missing_bucket":false,"order":"asc"}}},{"age":{"terms":{"field":"age","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","age","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},"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":[{"age":{"terms":{"field":"age","missing_bucket":false,"order":"asc"}}},{"account_number":{"terms":{"field":"account_number","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":2,"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},"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 1c583d8f79d..5bcfb7c6b46 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,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->[1 ASC FIRST, 3 DESC LAST]], 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":"desc"}}}]},"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=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)), 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_dedup_with_expr4_alternative.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_expr4_alternative.yaml index c205d98dbad..bd618ccdc00 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,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->[1 ASC FIRST, 3 DESC LAST]], 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":"desc"}}}]},"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=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)), 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_filter_agg_push.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_filter_agg_push.yaml index f1b2c6a434f..c3db9d4f6b9 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_filter_agg_push.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_filter_agg_push.yaml @@ -7,4 +7,4 @@ calcite: LogicalFilter(condition=[>($8, 30)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[city, state, age], FILTER->>($2, 30), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},avg_age=AVG($2)), PROJECT->[avg_age, state, city], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"range":{"age":{"from":30,"to":null,"include_lower":false,"include_upper":true,"boost":1.0}}},"_source":{"includes":["city","state","age"],"excludes":[]},"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=10000, pageSize=null, startFrom=0)]) + 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, city], LIMIT->10000], 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":[{"city":{"terms":{"field":"city.keyword","missing_bucket":true,"missing_order":"first","order":"asc"}}},{"state":{"terms":{"field":"state.keyword","missing_bucket":true,"missing_order":"first","order":"asc"}}}]},"aggregations":{"avg_age":{"avg":{"field":"age"}}}}}}, 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 9ac2557ab2e..b6dd067dc2e 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 @@ -7,4 +7,4 @@ calcite: 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=[[PROJECT->[birthdate], 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}}},"_source":{"includes":["birthdate"],"excludes":[]},"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)]) \ No newline at end of file + 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 d86d374ec2e..0c3f8565335 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 @@ -21,4 +21,4 @@ calcite: "missing" : "_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":[{"account_number":{"order":"asc","missing":"_last"}}]}, requestedTotalSize=2147483647, 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->[0]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"account_number":{"terms":{"field":"account_number","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","birthdate","gender","city","lastname","balance","employer","state","age","email","male","_id","_index","_score","_maxscore","_sort","_routing"],"excludes":[]},"script_fields":{}}}}}}}, requestedTotalSize=50000, pageSize=null, startFrom=0)]) \ No newline at end of file + 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->[0]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"account_number":{"terms":{"field":"account_number","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","birthdate","gender","city","lastname","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_join_with_fields_max_option.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_join_with_fields_max_option.yaml index 715ba0c9882..4b5597cc6b2 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 @@ -16,5 +16,5 @@ calcite: EnumerableCalc(expr#0..13=[{inputs}], proj#0..12=[{exprs}]) EnumerableLimit(fetch=[10000]) EnumerableHashJoin(condition=[=($0, $13)], joinType=[inner]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0},agg#0=LITERAL_AGG(1)), LIMIT->50000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"account_number":{"terms":{"field":"account_number","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","birthdate","gender","city","lastname","balance","employer","state","age","email","male","_id","_index","_score","_maxscore","_sort","_routing"],"excludes":[]},"script_fields":{}}}}}}}, requestedTotalSize=50000, pageSize=null, startFrom=0)]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[PROJECT->[account_number]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["account_number"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0},agg#0=LITERAL_AGG(1)), LIMIT->50000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"account_number":{"terms":{"field":"account_number","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","birthdate","gender","city","lastname","balance","employer","state","age","email","male"],"excludes":[]},"script_fields":{}}}}}}}, requestedTotalSize=50000, pageSize=null, startFrom=0)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[PROJECT->[account_number]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["account_number"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_limit_agg_pushdown3.json b/integ-test/src/test/resources/expectedOutput/calcite/explain_limit_agg_pushdown3.json index 0f391511f22..aa70977eaa1 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_limit_agg_pushdown3.json +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_limit_agg_pushdown3.json @@ -1,6 +1,6 @@ { "calcite": { "logical": "LogicalSystemLimit(fetch=[10000], type=[QUERY_SIZE_LIMIT])\n LogicalSort(offset=[10], fetch=[10])\n LogicalSort(fetch=[100])\n LogicalProject(count()=[$1], state=[$0])\n LogicalAggregate(group=[{0}], count()=[COUNT()])\n LogicalProject(state=[$7])\n CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]])\n", - "physical": "EnumerableLimit(fetch=[10000])\n EnumerableCalc(expr#0..1=[{inputs}], count()=[$t1], state=[$t0])\n EnumerableLimit(offset=[10], fetch=[10])\n CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},count()=COUNT()), LIMIT->100, LIMIT->[10 from 10]], OpenSearchRequestBuilder(sourceBuilder={\"from\":0,\"size\":0,\"timeout\":\"1m\",\"aggregations\":{\"composite_buckets\":{\"composite\":{\"size\":20,\"sources\":[{\"state\":{\"terms\":{\"field\":\"state.keyword\",\"missing_bucket\":true,\"missing_order\":\"first\",\"order\":\"asc\"}}}]}}}}, requestedTotalSize=20, pageSize=null, startFrom=0)])\n" + "physical": "EnumerableLimit(fetch=[10000])\n EnumerableLimit(offset=[10], fetch=[10])\n CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},count()=COUNT()), PROJECT->[count(), state], LIMIT->100, LIMIT->[10 from 10]], OpenSearchRequestBuilder(sourceBuilder={\"from\":0,\"size\":0,\"timeout\":\"1m\",\"aggregations\":{\"composite_buckets\":{\"composite\":{\"size\":20,\"sources\":[{\"state\":{\"terms\":{\"field\":\"state.keyword\",\"missing_bucket\":true,\"missing_order\":\"first\",\"order\":\"asc\"}}}]}}}}, requestedTotalSize=20, pageSize=null, startFrom=0)])\n" } } diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_limit_agg_pushdown_bucket_nullable1.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_limit_agg_pushdown_bucket_nullable1.yaml index 4ff9804f458..bab348f5897 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_limit_agg_pushdown_bucket_nullable1.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_limit_agg_pushdown_bucket_nullable1.yaml @@ -10,6 +10,5 @@ calcite: CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | EnumerableLimit(fetch=[10000]) - EnumerableCalc(expr#0..1=[{inputs}], count()=[$t1], state=[$t0]) - EnumerableLimit(offset=[10], fetch=[10]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},count()=COUNT()), LIMIT->100, LIMIT->[10 from 10]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":20,"sources":[{"state":{"terms":{"field":"state.keyword","missing_bucket":false,"order":"asc"}}}]}}}}, requestedTotalSize=20, pageSize=null, startFrom=0)]) + EnumerableLimit(offset=[10], fetch=[10]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},count()=COUNT()), PROJECT->[count(), state], LIMIT->100, LIMIT->[10 from 10]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":20,"sources":[{"state":{"terms":{"field":"state.keyword","missing_bucket":false,"order":"asc"}}}]}}}}, requestedTotalSize=20, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_multiple_agg_with_sort_on_one_measure_not_push1.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_multiple_agg_with_sort_on_one_measure_not_push1.yaml index 2f63567329f..4d460b3b361 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_multiple_agg_with_sort_on_one_measure_not_push1.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_multiple_agg_with_sort_on_one_measure_not_push1.yaml @@ -10,4 +10,4 @@ calcite: physical: | EnumerableLimit(fetch=[10000]) EnumerableSort(sort0=[$0], sort1=[$2], dir0=[ASC-nulls-first], dir1=[ASC-nulls-first]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},c=COUNT(),s=SUM($1)), PROJECT->[c, s, state]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"state":{"terms":{"field":"state.keyword","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"s":{"sum":{"field":"balance"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={1},c=COUNT(),s=SUM($0)), PROJECT->[c, s, state]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"state":{"terms":{"field":"state.keyword","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"s":{"sum":{"field":"balance"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_multiple_agg_with_sort_on_one_measure_not_push2.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_multiple_agg_with_sort_on_one_measure_not_push2.yaml index 4177d6616be..de74d7f7f78 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_multiple_agg_with_sort_on_one_measure_not_push2.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_multiple_agg_with_sort_on_one_measure_not_push2.yaml @@ -10,4 +10,4 @@ calcite: physical: | EnumerableLimit(fetch=[10000]) EnumerableSort(sort0=[$0], sort1=[$1], dir0=[ASC-nulls-first], dir1=[ASC-nulls-first]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},c=COUNT(),s=SUM($1)), PROJECT->[c, s, state]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"state":{"terms":{"field":"state.keyword","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"s":{"sum":{"field":"balance"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={1},c=COUNT(),s=SUM($0)), PROJECT->[c, s, state]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"state":{"terms":{"field":"state.keyword","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"s":{"sum":{"field":"balance"}}}}}}, 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 b9c090616e5..73a3fefe705 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_output.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_output.yaml @@ -13,9 +13,8 @@ calcite: LogicalFilter(condition=[>($8, 30)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - 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)], age2=[$t2], $condition=[$t3]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[city, state, age], 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}}},"_source":{"includes":["city","state","age"],"excludes":[]},"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)]) \ No newline at end of file + 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)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_scalar_correlated_subquery_in_select.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_scalar_correlated_subquery_in_select.yaml index 39410dca951..027e7e40c43 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_scalar_correlated_subquery_in_select.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_scalar_correlated_subquery_in_select.yaml @@ -17,4 +17,4 @@ calcite: EnumerableCalc(expr#0..2=[{inputs}], expr#3=[IS NOT NULL($t2)], expr#4=[0], expr#5=[CASE($t3, $t2, $t4)], uid=[$t0], count(name)=[$t5]) EnumerableNestedLoopJoin(condition=[IS NOT DISTINCT FROM($0, $1)], joinType=[left]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_worker]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0})], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"id":{"terms":{"field":"id","missing_bucket":true,"missing_order":"first","order":"asc"}}}]}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_work_information]], PushDownContext=[[PROJECT->[name, uid], FILTER->AND(IS NOT NULL($1), IS NOT NULL($0)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},count(name)=COUNT($1))], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"exists":{"field":"uid","boost":1.0}},{"exists":{"field":"name","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"_source":{"includes":["name","uid"],"excludes":[]},"aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"uid":{"terms":{"field":"uid","missing_bucket":true,"missing_order":"first","order":"asc"}}}]},"aggregations":{"count(name)":{"value_count":{"field":"name"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_work_information]], PushDownContext=[[FILTER->AND(IS NOT NULL($1), IS NOT NULL($0)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={1},count(name)=COUNT($0))], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"exists":{"field":"uid","boost":1.0}},{"exists":{"field":"name","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"uid":{"terms":{"field":"uid","missing_bucket":true,"missing_order":"first","order":"asc"}}}]},"aggregations":{"count(name)":{"value_count":{"field":"name"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_scalar_correlated_subquery_in_where.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_scalar_correlated_subquery_in_where.yaml index 3f4cb15d194..42e5e196667 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_scalar_correlated_subquery_in_where.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_scalar_correlated_subquery_in_where.yaml @@ -15,4 +15,4 @@ calcite: EnumerableHashJoin(condition=[=($1, $2)], joinType=[semi]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_worker]], PushDownContext=[[PROJECT->[name, id]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["name","id"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) EnumerableCalc(expr#0..1=[{inputs}], expr#2=[=($t0, $t1)], proj#0..1=[{exprs}], $condition=[$t2]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_work_information]], PushDownContext=[[PROJECT->[uid], FILTER->IS NOT NULL($0), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},max(uid)=MAX($1))], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"exists":{"field":"uid","boost":1.0}},"_source":{"includes":["uid"],"excludes":[]},"aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"uid1":{"terms":{"field":"uid","missing_bucket":true,"missing_order":"first","order":"asc"}}}]},"aggregations":{"max(uid)":{"max":{"field":"uid"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_work_information]], PushDownContext=[[FILTER->IS NOT NULL($0), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},max(uid)=MAX($1))], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"exists":{"field":"uid","boost":1.0}},"aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"uid1":{"terms":{"field":"uid","missing_bucket":true,"missing_order":"first","order":"asc"}}}]},"aggregations":{"max(uid)":{"max":{"field":"uid"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_script_push_on_text.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_script_push_on_text.yaml index d8e9a1727f7..3799dbe1da4 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_script_push_on_text.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_script_push_on_text.yaml @@ -7,4 +7,4 @@ calcite: LogicalFilter(condition=[>(CHAR_LENGTH($2), 0)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[address], SCRIPT->>(CHAR_LENGTH($0), 0), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},count()=COUNT()), PROJECT->[count(), address_length], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"script":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQCHHsKICAib3AiOiB7CiAgICAibmFtZSI6ICI8IiwKICAgICJraW5kIjogIkxFU1NfVEhBTiIsCiAgICAic3ludGF4IjogIkJJTkFSWSIKICB9LAogICJvcGVyYW5kcyI6IFsKICAgIHsKICAgICAgImR5bmFtaWNQYXJhbSI6IDAsCiAgICAgICJ0eXBlIjogewogICAgICAgICJ0eXBlIjogIkJJR0lOVCIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZQogICAgICB9CiAgICB9LAogICAgewogICAgICAib3AiOiB7CiAgICAgICAgIm5hbWUiOiAiQ0hBUl9MRU5HVEgiLAogICAgICAgICJraW5kIjogIkNIQVJfTEVOR1RIIiwKICAgICAgICAic3ludGF4IjogIkZVTkNUSU9OIgogICAgICB9LAogICAgICAib3BlcmFuZHMiOiBbCiAgICAgICAgewogICAgICAgICAgImR5bmFtaWNQYXJhbSI6IDEsCiAgICAgICAgICAidHlwZSI6IHsKICAgICAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgICAgICJudWxsYWJsZSI6IHRydWUsCiAgICAgICAgICAgICJwcmVjaXNpb24iOiAtMQogICAgICAgICAgfQogICAgICAgIH0KICAgICAgXQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[2,1],"DIGESTS":[0,"address"]}},"boost":1.0}},"_source":{"includes":["address"],"excludes":[]},"aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"address_length":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/3sKICAib3AiOiB7CiAgICAibmFtZSI6ICJDSEFSX0xFTkdUSCIsCiAgICAia2luZCI6ICJDSEFSX0xFTkdUSCIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[1],"DIGESTS":["address"]}},"missing_bucket":true,"value_type":"long","missing_order":"first","order":"asc"}}}]}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[SCRIPT->>(CHAR_LENGTH($0), 0), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},count()=COUNT()), PROJECT->[count(), address_length], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"script":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQCHHsKICAib3AiOiB7CiAgICAibmFtZSI6ICI8IiwKICAgICJraW5kIjogIkxFU1NfVEhBTiIsCiAgICAic3ludGF4IjogIkJJTkFSWSIKICB9LAogICJvcGVyYW5kcyI6IFsKICAgIHsKICAgICAgImR5bmFtaWNQYXJhbSI6IDAsCiAgICAgICJ0eXBlIjogewogICAgICAgICJ0eXBlIjogIkJJR0lOVCIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZQogICAgICB9CiAgICB9LAogICAgewogICAgICAib3AiOiB7CiAgICAgICAgIm5hbWUiOiAiQ0hBUl9MRU5HVEgiLAogICAgICAgICJraW5kIjogIkNIQVJfTEVOR1RIIiwKICAgICAgICAic3ludGF4IjogIkZVTkNUSU9OIgogICAgICB9LAogICAgICAib3BlcmFuZHMiOiBbCiAgICAgICAgewogICAgICAgICAgImR5bmFtaWNQYXJhbSI6IDEsCiAgICAgICAgICAidHlwZSI6IHsKICAgICAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgICAgICJudWxsYWJsZSI6IHRydWUsCiAgICAgICAgICAgICJwcmVjaXNpb24iOiAtMQogICAgICAgICAgfQogICAgICAgIH0KICAgICAgXQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[2,1],"DIGESTS":[0,"address"]}},"boost":1.0}},"aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"address_length":{"terms":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/3sKICAib3AiOiB7CiAgICAibmFtZSI6ICJDSEFSX0xFTkdUSCIsCiAgICAia2luZCI6ICJDSEFSX0xFTkdUSCIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[1],"DIGESTS":["address"]}},"missing_bucket":true,"value_type":"long","missing_order":"first","order":"asc"}}}]}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_sort_then_agg_push.json b/integ-test/src/test/resources/expectedOutput/calcite/explain_sort_then_agg_push.json index 75fea09dc13..0d6b8c4b9cf 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_sort_then_agg_push.json +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_sort_then_agg_push.json @@ -1,6 +1,6 @@ { "calcite": { "logical": "LogicalSystemLimit(fetch=[10000], type=[QUERY_SIZE_LIMIT])\n LogicalProject(avg(balance)=[$1], state=[$0])\n LogicalAggregate(group=[{0}], avg(balance)=[AVG($1)])\n LogicalProject(state=[$7], balance=[$3])\n LogicalSort(sort0=[$3], sort1=[$8], dir0=[ASC-nulls-first], dir1=[ASC-nulls-first])\n CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]])\n", - "physical": "CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[balance, state, age], AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},avg(balance)=AVG($1)), PROJECT->[avg(balance), state], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={\"from\":0,\"size\":0,\"timeout\":\"1m\",\"_source\":{\"includes\":[\"balance\",\"state\",\"age\"],\"excludes\":[]},\"aggregations\":{\"composite_buckets\":{\"composite\":{\"size\":1000,\"sources\":[{\"state\":{\"terms\":{\"field\":\"state.keyword\",\"missing_bucket\":true,\"missing_order\":\"first\",\"order\":\"asc\"}}}]},\"aggregations\":{\"avg(balance)\":{\"avg\":{\"field\":\"balance\"}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)])\n" + "physical": "CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={1},avg(balance)=AVG($0)), PROJECT->[avg(balance), state], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={\"from\":0,\"size\":0,\"timeout\":\"1m\",\"aggregations\":{\"composite_buckets\":{\"composite\":{\"size\":1000,\"sources\":[{\"state\":{\"terms\":{\"field\":\"state.keyword\",\"missing_bucket\":true,\"missing_order\":\"first\",\"order\":\"asc\"}}}]},\"aggregations\":{\"avg(balance)\":{\"avg\":{\"field\":\"balance\"}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)])\n" } } 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 bd9187b4908..9d16a625c0a 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_timechart.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_timechart.yaml @@ -20,8 +20,8 @@ calcite: CalciteLogicalIndexScan(table=[[OpenSearch, events]]) physical: | EnumerableLimit(fetch=[10000]) - EnumerableSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC]) - 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)], proj#0..1=[{exprs}], avg(cpu_usage)=[$t8]) + 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)], proj#0..1=[{exprs}], avg(cpu_usage)=[$t8]) + EnumerableSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC]) EnumerableAggregate(group=[{0, 1}], agg#0=[$SUM0($2)], agg#1=[COUNT($2)]) EnumerableCalc(expr#0..4=[{inputs}], expr#5=[IS NULL($t1)], expr#6=['NULL'], expr#7=[10], expr#8=[<=($t4, $t7)], expr#9=['OTHER'], expr#10=[CASE($t5, $t6, $t8, $t1, $t9)], @timestamp=[$t0], host=[$t10], avg(cpu_usage)=[$t2]) EnumerableMergeJoin(condition=[=($1, $3)], joinType=[left]) @@ -36,5 +36,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($t0, $t3, $t4)], host=[$t1], cpu_usage=[$t2], @timestamp0=[$t5]) - CalciteEnumerableIndexScan(table=[[OpenSearch, events]], PushDownContext=[[PROJECT->[@timestamp, host, cpu_usage], FILTER->AND(IS NOT NULL($0), IS NOT NULL($2)), FILTER->IS NOT NULL($1)], 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":["@timestamp","host","cpu_usage"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + 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)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_timechart_count.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_timechart_count.yaml index 17d8b0f917b..65f5d4f26b7 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_timechart_count.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_timechart_count.yaml @@ -32,5 +32,5 @@ calcite: EnumerableSort(sort0=[$0], dir0=[ASC]) EnumerableCalc(expr#0..2=[{inputs}], host=[$t0], $1=[$t2]) EnumerableWindow(window#0=[window(order by [1 DESC-nulls-last] rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) - EnumerableAggregate(group=[{1}], __grand_total__=[COUNT()]) - CalciteEnumerableIndexScan(table=[[OpenSearch, events]], PushDownContext=[[PROJECT->[@timestamp, host], FILTER->IS NOT NULL($0), FILTER->IS NOT NULL($1)], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","query":{"bool":{"filter":[{"exists":{"field":"@timestamp","boost":1.0}},{"exists":{"field":"host","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"_source":{"includes":["@timestamp","host"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + EnumerableAggregate(group=[{0}], __grand_total__=[COUNT()]) + CalciteEnumerableIndexScan(table=[[OpenSearch, events]], PushDownContext=[[PROJECT->[@timestamp, host], FILTER->IS NOT NULL($0), PROJECT->[host, @timestamp], FILTER->IS NOT NULL($0)], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","query":{"bool":{"filter":[{"exists":{"field":"@timestamp","boost":1.0}},{"exists":{"field":"host","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"_source":{"includes":["host","@timestamp"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) 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 5c919a0e0cf..4154a10ce51 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 @@ -5,34 +5,26 @@ package org.opensearch.sql.opensearch.planner.rules; -import static org.opensearch.sql.calcite.utils.OpenSearchTypeFactory.isTimeBasedType; +import static org.opensearch.sql.calcite.utils.PlanUtils.aggIgnoreNullBucket; +import static org.opensearch.sql.calcite.utils.PlanUtils.maybeTimeSpanAgg; import static org.opensearch.sql.expression.function.PPLBuiltinOperators.WIDTH_BUCKET; import java.util.List; -import java.util.function.Function; import java.util.function.Predicate; +import javax.annotation.Nullable; import org.apache.calcite.plan.RelOptRuleCall; import org.apache.calcite.plan.RelOptUtil; import org.apache.calcite.rel.AbstractRelNode; import org.apache.calcite.rel.RelNode; -import org.apache.calcite.rel.core.Aggregate; 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.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.RexSlot; -import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.fun.SqlStdOperatorTable; -import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.tools.RelBuilder; import org.immutables.value.Value; -import org.opensearch.sql.ast.expression.Argument; import org.opensearch.sql.calcite.plan.OpenSearchRuleConfig; import org.opensearch.sql.calcite.utils.PlanUtils; -import org.opensearch.sql.expression.function.BuiltinFunctionName; import org.opensearch.sql.expression.function.udf.binning.WidthBucketFunction; import org.opensearch.sql.opensearch.storage.scan.AbstractCalciteIndexScan; import org.opensearch.sql.opensearch.storage.scan.CalciteLogicalIndexScan; @@ -54,15 +46,7 @@ protected void onMatchImpl(RelOptRuleCall call) { final LogicalFilter filter = call.rel(2); final LogicalProject bottomProject = call.rel(3); final CalciteLogicalIndexScan scan = call.rel(4); - boolean ignoreNullBucket = Config.aggIgnoreNullBucket.test(aggregate); - List groupRefList = - aggregate.getGroupSet().asList().stream() - .map(topProject.getProjects()::get) - .filter(rex -> ignoreNullBucket || isTimeSpan(rex)) - .flatMap(expr -> PlanUtils.getInputRefs(expr).stream()) - .map(RexSlot::getIndex) - .toList(); - if (isNotNullDerivedFromAgg(filter, groupRefList)) { + if (PlanUtils.isNotNullDerivedFromAgg(filter.getCondition(), aggregate, topProject, null)) { final List newProjects = RelOptUtil.pushPastProjectUnlessBloat( topProject.getProjects(), bottomProject, RelOptUtil.DEFAULT_BLOAT); @@ -86,8 +70,7 @@ protected void onMatchImpl(RelOptRuleCall call) { final LogicalFilter filter = call.rel(1); final LogicalProject project = call.rel(2); final CalciteLogicalIndexScan scan = call.rel(3); - List groupList = aggregate.getGroupSet().asList(); - if (isNotNullDerivedFromAgg(filter, groupList)) { + if (PlanUtils.isNotNullDerivedFromAgg(filter.getCondition(), aggregate, project, null)) { // Try to do the aggregate push down and ignore the filter if the filter sources from the // aggregate's hint. See{@link CalciteRelNodeVisitor::visitAggregation} apply(call, aggregate, project, scan); @@ -98,7 +81,6 @@ protected void onMatchImpl(RelOptRuleCall call) { final CalciteLogicalIndexScan scan = call.rel(2); apply(call, aggregate, project, scan); } else if (call.rels.length == 2) { - // case of count() without group-by final LogicalAggregate aggregate = call.rel(0); final CalciteLogicalIndexScan scan = call.rel(1); apply(call, aggregate, null, scan); @@ -110,38 +92,15 @@ protected void onMatchImpl(RelOptRuleCall call) { } } - private boolean isTimeSpan(RexNode rex) { - return rex instanceof RexCall rexCall - && rexCall.getKind() == SqlKind.OTHER_FUNCTION - && rexCall.getOperator().getName().equalsIgnoreCase(BuiltinFunctionName.SPAN.name()) - && rexCall.getOperands().size() == 3 - && rexCall.getOperands().get(2) instanceof RexLiteral unitLiteral - && unitLiteral.getTypeName() != SqlTypeName.NULL; - } - - private boolean isNotNullDerivedFromAgg(LogicalFilter filter, List groupRefList) { - RexNode condition = filter.getCondition(); - Function isNotNullFromAgg = - rex -> - rex instanceof RexCall rexCall - && rexCall.isA(SqlKind.IS_NOT_NULL) - && rexCall.getOperands().get(0) instanceof RexInputRef ref - && groupRefList.contains(ref.getIndex()); - - return isNotNullFromAgg.apply(condition) - || (condition instanceof RexCall rexCall - && rexCall.getOperator() == SqlStdOperatorTable.AND - && rexCall.getOperands().stream().allMatch(isNotNullFromAgg::apply)); - } - protected void apply( RelOptRuleCall call, LogicalAggregate aggregate, - LogicalProject project, + @Nullable LogicalProject project, CalciteLogicalIndexScan scan) { AbstractRelNode newRelNode = scan.pushDownAggregate(aggregate, project); if (newRelNode != null) { call.transformTo(newRelNode); + PlanUtils.tryPruneRelNodes(call); } } @@ -176,21 +135,13 @@ public interface Config extends OpenSearchRuleConfig { AbstractCalciteIndexScan ::noAggregatePushed)) .noInputs()))); - Config COUNT_STAR = + Config AGGREGATE_SCAN = ImmutableAggregateIndexScanRule.Config.builder() .build() - .withDescription("Agg[count()]-TableScan") + .withDescription("Agg-TableScan") .withOperandSupplier( b0 -> b0.operand(LogicalAggregate.class) - .predicate( - agg -> - agg.getGroupSet().isEmpty() - && agg.getAggCallList().stream() - .allMatch( - call -> - call.getAggregation().kind == SqlKind.COUNT - && call.getArgList().isEmpty())) .oneInput( b1 -> b1.operand(CalciteLogicalIndexScan.class) @@ -198,20 +149,6 @@ public interface Config extends OpenSearchRuleConfig { Predicate.not(AbstractCalciteIndexScan::isLimitPushed) .and(AbstractCalciteIndexScan::noAggregatePushed)) .noInputs())); - Predicate aggIgnoreNullBucket = - agg -> - agg.getHints().stream() - .anyMatch( - hint -> - hint.hintName.equals("stats_args") - && hint.kvOptions.get(Argument.BUCKET_NULLABLE).equals("false")); - Predicate maybeTimeSpanAgg = - agg -> - agg.getGroupSet().stream() - .allMatch( - group -> - isTimeBasedType( - agg.getInput().getRowType().getFieldList().get(group).getType())); Config BUCKET_NON_NULL_AGG = ImmutableAggregateIndexScanRule.Config.builder() 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 47479bd7632..38059099662 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 @@ -5,16 +5,15 @@ package org.opensearch.sql.opensearch.planner.rules; -import com.google.common.collect.Streams; +import java.util.ArrayList; import java.util.List; -import java.util.Set; import java.util.function.Predicate; import java.util.stream.IntStream; -import javax.annotation.Nullable; 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; @@ -33,7 +32,8 @@ import org.opensearch.sql.opensearch.storage.scan.CalciteLogicalIndexScan; @Value.Enclosing -public class DedupPushdownRule extends InterruptibleRelRule { +public class DedupPushdownRule extends InterruptibleRelRule + implements SubstitutionRule { private static final Logger LOG = LogManager.getLogger(); protected DedupPushdownRule(Config config) { @@ -46,19 +46,9 @@ protected void onMatchImpl(RelOptRuleCall call) { // TODO Used when number of duplication is more than 1 final LogicalFilter numOfDedupFilter = call.rel(1); final LogicalProject projectWithWindow = call.rel(2); - if (call.rels.length == 5) { - final CalciteLogicalIndexScan scan = call.rel(4); - apply(call, finalProject, numOfDedupFilter, projectWithWindow, null, scan); - } else if (call.rels.length == 6) { - final LogicalProject projectWithExpr = call.rel(4); - final CalciteLogicalIndexScan scan = call.rel(5); - apply(call, finalProject, numOfDedupFilter, projectWithWindow, projectWithExpr, scan); - } else { - throw new AssertionError( - String.format( - "The length of rels should be %s but got %s", - this.operands.size(), call.rels.length)); - } + final LogicalProject projectWithExpr = call.rel(4); + final CalciteLogicalIndexScan scan = call.rel(5); + apply(call, finalProject, numOfDedupFilter, projectWithWindow, projectWithExpr, scan); } protected void apply( @@ -66,7 +56,7 @@ protected void apply( LogicalProject finalProject, LogicalFilter numOfDedupFilter, LogicalProject projectWithWindow, - @Nullable LogicalProject projectWithExpr, + LogicalProject bottomProject, CalciteLogicalIndexScan scan) { List windows = PlanUtils.getRexWindowFromProject(projectWithWindow); if (windows.size() != 1) { @@ -81,59 +71,45 @@ protected void apply( // TODO https://github.com/opensearch-project/sql/issues/4564 return; } - List dedupColumnIndices = getInputRefIndices(dedupColumns); - List dedupColumnNames = - dedupColumnIndices.stream() - .map( - i -> - projectWithWindow.getNamedProjects().stream() - .filter(pair -> pair.getKey().isA(SqlKind.INPUT_REF)) - .filter(pair -> ((RexInputRef) pair.getKey()).getIndex() == i) - .map(Pair::getValue) - .findFirst() - .orElse(projectWithWindow.getInput().getRowType().getFieldNames().get(i))) - .toList(); - if (dedupColumnIndices.size() != dedupColumnNames.size()) { - 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); - // We convert the dedup pushdown to composite aggregate + top_hits: - // Aggregate(literalAgg(dedupNumer), groups) - // +- Project(groups, remaining) RelBuilder relBuilder = call.builder(); - // 1 Initial a RelBuilder by pushing Scan and Project - if (projectWithExpr == null) { - // 1.1 if projectWithExpr not existed, push a scan then create a new project - relBuilder.push(scan); - List columnsFromScan = relBuilder.fields(); - List colNamesFromScan = relBuilder.peek().getRowType().getFieldNames(); - List> namedPairFromScan = - Streams.zip(columnsFromScan.stream(), colNamesFromScan.stream(), Pair::new).toList(); - List> reordered = - advanceDedupColumns(namedPairFromScan, dedupColumnIndices, dedupColumnNames); - relBuilder.project( - reordered.stream().map(Pair::getKey).toList(), - reordered.stream().map(Pair::getValue).toList(), - true); - } else { - // 1.2 if projectWithExpr existed, push a reordered projectWithExpr - List> reordered = - advanceDedupColumns( - projectWithExpr.getNamedProjects(), dedupColumnIndices, dedupColumnNames); - LogicalProject reorderedProject = - LogicalProject.create( - projectWithExpr.getInput(), - List.of(), - reordered.stream().map(Pair::getKey).toList(), - reordered.stream().map(Pair::getValue).toList(), - Set.of()); - relBuilder.push(reorderedProject); + 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<>(); + 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()))); + } else { + LOG.warn("The dedup column {} is illegal.", dedupColumn); + 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 @@ -141,109 +117,35 @@ protected void apply( // (1) Pass the dedupNumer to AggregateAnalyzer.processAggregateCalls() // (2) Distinguish it from an optimization operator and user defined aggregator. // (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(dedupColumnNames)), - relBuilder.literalAgg(dedupNumer)); + relBuilder.groupKey(relBuilder.fields(newGroupByList)), relBuilder.literalAgg(dedupNumer)); + // add bucket_nullable = false hint PlanUtils.addIgnoreNullBucketHintToAggregate(relBuilder); // peek the aggregate after hint being added LogicalAggregate aggregate = (LogicalAggregate) relBuilder.build(); + assert aggregate.getGroupSet().asList().equals(newGroupByList) + : "The group set of aggregate should be exactly the same as the generated group list"; CalciteLogicalIndexScan newScan = (CalciteLogicalIndexScan) scan.pushDownAggregate(aggregate, targetChildProject); if (newScan != null) { // Back to original project order call.transformTo(newScan.copyWithNewSchema(finalProject.getRowType())); + PlanUtils.tryPruneRelNodes(call); } } - private static List getInputRefIndices(List columns) { - return columns.stream() - .filter(rex -> rex.isA(SqlKind.INPUT_REF)) - .map(r -> ((RexInputRef) r).getIndex()) - .toList(); - } - - /** - * Move the dedup columns to the front of the original column list. - * - * @param originalColumnList The original column pair list - * @param dedupColumnIndices The indices of dedup columns - * @param dedupColumnNames The names of dedup columns - * @return The reordered column pair list - */ - private static List> advanceDedupColumns( - List> originalColumnList, - List dedupColumnIndices, - List dedupColumnNames) { - List> reordered = - IntStream.range(0, originalColumnList.size()) - .boxed() - .sorted( - (i1, i2) -> { - boolean in1 = dedupColumnIndices.contains(i1); - boolean in2 = dedupColumnIndices.contains(i2); - return in1 == in2 ? i1 - i2 : in2 ? 1 : -1; - }) - .map( - i -> { - Pair original = originalColumnList.get(i); - if (dedupColumnIndices.contains(i)) { - int dedupIndex = dedupColumnIndices.indexOf(i); - return Pair.of(original.getKey(), dedupColumnNames.get(dedupIndex)); - } - return original; - }) - .toList(); - return reordered; - } - @Value.Immutable public interface Config extends OpenSearchRuleConfig { - // Can only push the case with KEEPEMPTY=false: - // +- LogicalProject(no _row_number_dedup_) - // +- LogicalFilter(condition contains _row_number_dedup_) - // +- LogicalProject(contains _row_number_dedup_) - // +- LogicalFilter(condition=IS NOT NULL(dedupColumn))" - // +- CalciteLogicalIndexScan - Config DEFAULT = - ImmutableDedupPushdownRule.Config.builder() - .build() - .withDescription("Dedup-to-Aggregate") - .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) - .oneInput( - b4 -> - b4.operand( - CalciteLogicalIndexScan - .class) - .predicate( - Config - ::tableScanChecker) - .noInputs()))))); // +- 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) + // +- LogicalProject(dedupColumn is call or ref) // +- CalciteLogicalIndexScan - Config DEDUP_EXPR = + Config DEFAULT = ImmutableDedupPushdownRule.Config.builder() .build() .withDescription("DedupWithExpression-to-Aggregate") @@ -268,9 +170,6 @@ public interface Config extends OpenSearchRuleConfig { .oneInput( b4 -> b4.operand(LogicalProject.class) - .predicate( - PlanUtils - ::containsRexCall) .oneInput( b5 -> b5.operand( @@ -288,7 +187,6 @@ public interface Config extends OpenSearchRuleConfig { private static boolean tableScanChecker(AbstractCalciteIndexScan scan) { return Predicate.not(AbstractCalciteIndexScan::isLimitPushed) .and(AbstractCalciteIndexScan::noAggregatePushed) - .and(Predicate.not(AbstractCalciteIndexScan::isProjectPushed)) .test(scan); } 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 0a8b3ae5f33..cc7890ddc78 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 @@ -5,6 +5,8 @@ package org.opensearch.sql.opensearch.planner.rules; +import static org.opensearch.sql.calcite.utils.PlanUtils.tryPruneRelNodes; + import java.util.HashMap; import java.util.Map; import java.util.Optional; @@ -154,6 +156,7 @@ private void handleSimpleExpressionFieldSorts( Project newProject = project.copy(toTraits, project.getInput(), project.getProjects(), project.getRowType()); call.transformTo(newProject); + tryPruneRelNodes(call); } } 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 b35c74ac829..9cf6d297ada 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 @@ -12,6 +12,7 @@ import org.apache.calcite.rel.logical.LogicalFilter; import org.immutables.value.Value; import org.opensearch.sql.calcite.plan.OpenSearchRuleConfig; +import org.opensearch.sql.calcite.utils.PlanUtils; import org.opensearch.sql.opensearch.storage.scan.AbstractCalciteIndexScan; import org.opensearch.sql.opensearch.storage.scan.CalciteLogicalIndexScan; @@ -43,6 +44,7 @@ protected void apply(RelOptRuleCall call, Filter filter, CalciteLogicalIndexScan AbstractRelNode newRel = scan.pushDownFilter(filter); if (newRel != null) { call.transformTo(newRel); + PlanUtils.tryPruneRelNodes(call); } } 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 e9f7b725852..be84c8e5c4e 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 @@ -45,6 +45,7 @@ protected void onMatchImpl(RelOptRuleCall call) { AbstractRelNode newOperator = scan.pushDownLimit(sort, limitValue, offsetValue); if (newOperator != null) { call.transformTo(newOperator); + PlanUtils.tryPruneRelNodes(call); } } } diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/OpenSearchIndexRules.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/OpenSearchIndexRules.java index 37f45d4942c..32d4f3d1433 100644 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/OpenSearchIndexRules.java +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/OpenSearchIndexRules.java @@ -14,10 +14,10 @@ public class OpenSearchIndexRules { ProjectIndexScanRule.Config.DEFAULT.toRule(); private static final FilterIndexScanRule FILTER_INDEX_SCAN = FilterIndexScanRule.Config.DEFAULT.toRule(); - private static final AggregateIndexScanRule AGGREGATE_INDEX_SCAN = + private static final AggregateIndexScanRule AGGREGATE_PROJECT_INDEX_SCAN = AggregateIndexScanRule.Config.DEFAULT.toRule(); - private static final AggregateIndexScanRule COUNT_STAR_INDEX_SCAN = - AggregateIndexScanRule.Config.COUNT_STAR.toRule(); + private static final AggregateIndexScanRule AGGREGATE_INDEX_SCAN = + AggregateIndexScanRule.Config.AGGREGATE_SCAN.toRule(); // TODO: No need this rule once https://github.com/opensearch-project/sql/issues/4403 is addressed private static final AggregateIndexScanRule BUCKET_NON_NULL_AGG_INDEX_SCAN = AggregateIndexScanRule.Config.BUCKET_NON_NULL_AGG.toRule(); @@ -29,8 +29,6 @@ public class OpenSearchIndexRules { SortIndexScanRule.Config.DEFAULT.toRule(); private static final DedupPushdownRule DEDUP_PUSH_DOWN = DedupPushdownRule.Config.DEFAULT.toRule(); - private static final DedupPushdownRule DEDUP_EXPR_PUSH_DOWN = - DedupPushdownRule.Config.DEDUP_EXPR.toRule(); private static final SortProjectExprTransposeRule SORT_PROJECT_EXPR_TRANSPOSE = SortProjectExprTransposeRule.Config.DEFAULT.toRule(); private static final ExpandCollationOnProjectExprRule EXPAND_COLLATION_ON_PROJECT_EXPR = @@ -50,14 +48,13 @@ public class OpenSearchIndexRules { ImmutableList.of( PROJECT_INDEX_SCAN, FILTER_INDEX_SCAN, + AGGREGATE_PROJECT_INDEX_SCAN, AGGREGATE_INDEX_SCAN, - COUNT_STAR_INDEX_SCAN, BUCKET_NON_NULL_AGG_INDEX_SCAN, BUCKET_NON_NULL_AGG_WITH_UDF_INDEX_SCAN, LIMIT_INDEX_SCAN, SORT_INDEX_SCAN, DEDUP_PUSH_DOWN, - DEDUP_EXPR_PUSH_DOWN, SORT_PROJECT_EXPR_TRANSPOSE, SORT_AGGREGATION_METRICS_RULE, RARE_TOP_PUSH_DOWN, 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 70f467ee8d0..4561691ed9e 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 @@ -21,6 +21,7 @@ import org.apache.calcite.util.mapping.Mappings; import org.immutables.value.Value; import org.opensearch.sql.calcite.plan.OpenSearchRuleConfig; +import org.opensearch.sql.calcite.utils.PlanUtils; import org.opensearch.sql.opensearch.storage.OpenSearchIndex; import org.opensearch.sql.opensearch.storage.scan.CalciteLogicalIndexScan; @@ -79,6 +80,7 @@ public Void visitInputRef(RexInputRef inputRef) { } else { call.transformTo(call.builder().push(newScan).project(newProjectRexNodes).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 4403cdf9975..370b4131ed7 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 @@ -10,6 +10,7 @@ import org.apache.calcite.plan.RelOptRuleCall; 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.RexFieldCollation; import org.apache.calcite.rex.RexLiteral; @@ -23,7 +24,8 @@ import org.opensearch.sql.opensearch.storage.scan.context.RareTopDigest; @Value.Enclosing -public class RareTopPushdownRule extends InterruptibleRelRule { +public class RareTopPushdownRule extends InterruptibleRelRule + implements SubstitutionRule { protected RareTopPushdownRule(Config config) { super(config); @@ -69,6 +71,7 @@ protected void onMatchImpl(RelOptRuleCall call) { CalciteLogicalIndexScan newScan = scan.pushDownRareTop(project, digest); if (newScan != null) { call.transformTo(newScan); + PlanUtils.tryPruneRelNodes(call); } } 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 9c96ed0c103..2e27e0e871b 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 @@ -12,12 +12,14 @@ import org.apache.calcite.rel.AbstractRelNode; import org.apache.calcite.rel.core.Filter; import org.apache.calcite.rel.logical.LogicalFilter; +import org.apache.calcite.rel.rules.SubstitutionRule; import org.apache.calcite.rex.RexCall; import org.apache.calcite.rex.RexNode; 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.utils.PlanUtils; import org.opensearch.sql.opensearch.storage.scan.CalciteLogicalIndexScan; /** @@ -27,7 +29,7 @@ */ @Value.Enclosing public class RelevanceFunctionPushdownRule - extends InterruptibleRelRule { + extends InterruptibleRelRule implements SubstitutionRule { /** Creates an RelevanceFunctionPushdownRule. */ protected RelevanceFunctionPushdownRule(Config config) { @@ -55,6 +57,7 @@ protected void apply(RelOptRuleCall call, Filter filter, CalciteLogicalIndexScan AbstractRelNode newRel = scan.pushDownFilter(filter); if (newRel != null) { call.transformTo(newRel); + PlanUtils.tryPruneRelNodes(call); } } 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 db96f4550fe..b05db3203a3 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 @@ -30,6 +30,7 @@ protected void onMatchImpl(RelOptRuleCall call) { CalciteLogicalIndexScan newScan = scan.pushDownSortAggregateMeasure(sort); if (newScan != null) { call.transformTo(newScan); + PlanUtils.tryPruneRelNodes(call); } } 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 16d2da365ae..5c49dd6b7d2 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 @@ -116,6 +116,7 @@ protected void onMatchImpl(RelOptRuleCall call) { Project newProject = project.copy(sort.getTraitSet(), newScan, project.getProjects(), project.getRowType()); call.transformTo(newProject); + PlanUtils.tryPruneRelNodes(call); } } 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 44f98b315bd..e7e056738f6 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 @@ -32,6 +32,7 @@ protected void onMatchImpl(RelOptRuleCall call) { AbstractCalciteIndexScan newScan = scan.pushDownSort(collations); if (newScan != null) { call.transformTo(newScan); + PlanUtils.tryPruneRelNodes(call); } } diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/request/AggregateAnalyzer.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/request/AggregateAnalyzer.java index 6766c9f5f51..924b2ad1918 100644 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/request/AggregateAnalyzer.java +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/request/AggregateAnalyzer.java @@ -43,6 +43,8 @@ import java.util.Optional; import java.util.Set; import java.util.function.Function; +import java.util.stream.IntStream; +import javax.annotation.Nullable; import lombok.RequiredArgsConstructor; import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.rel.core.Aggregate; @@ -180,18 +182,28 @@ T inferValue(RexNode node, Class clazz) { } throw new IllegalStateException(String.format("Cannot infer value from RexNode %s", node)); } + + RexNode inferRexNodeFromIndex(int index, Project project) { + return project == null ? RexInputRef.of(index, rowType) : project.getProjects().get(index); + } + + String inferFieldNameFromIndex(int index, Project project) { + return project == null + ? rowType.getFieldNames().get(index) + : project.getRowType().getFieldNames().get(index); + } } // TODO: should we support filter aggregation? For PPL, we don't have filter in stats command public static Pair, OpenSearchAggregationResponseParser> analyze( Aggregate aggregate, - Project project, - List outputFields, + @Nullable Project project, + final List outputFields, AggregateBuilderHelper helper) throws ExpressionNotAnalyzableException { requireNonNull(aggregate, "aggregate"); try { - List groupList = aggregate.getGroupSet().asList(); + final List groupList = aggregate.getGroupSet().asList(); List aggFieldNames = outputFields.subList(groupList.size(), outputFields.size()); // Process all aggregate calls Pair> builderAndParser = @@ -225,12 +237,19 @@ public static Pair, OpenSearchAggregationResponseParser // Used to track the current sub-builder as analysis progresses Builder subBuilder = newMetricBuilder; // Push auto date span & case in group-by list into nested aggregations + List> groupNameAndIndexList = + IntStream.range(0, groupList.size()) + .mapToObj(i -> Pair.of(outputFields.get(i), groupList.get(i))) + .toList(); Pair, AggregationBuilder> aggPushedAndAggBuilder = - createNestedAggregation(groupList, project, subBuilder, helper); + createNestedAggregation(groupNameAndIndexList, project, subBuilder, helper); Set aggPushed = aggPushedAndAggBuilder.getLeft(); AggregationBuilder pushedAggBuilder = aggPushedAndAggBuilder.getRight(); // The group-by list after removing pushed aggregations - groupList = groupList.stream().filter(i -> !aggPushed.contains(i)).toList(); + groupNameAndIndexList = + groupNameAndIndexList.stream() + .filter(pair -> !aggPushed.contains(pair.getRight())) + .toList(); if (pushedAggBuilder != null) { subBuilder = new Builder().addAggregator(pushedAggBuilder); } @@ -242,7 +261,7 @@ public static Pair, OpenSearchAggregationResponseParser // - stats count() by ...auto_date_spans, ...range_fields // [AutoDateHistogram | RangeAgg]+ // Metric - if (groupList.isEmpty()) { + if (groupNameAndIndexList.isEmpty()) { return Pair.of( ImmutableList.copyOf(subBuilder.getAggregatorFactories()), new BucketAggregationParser(metricParsers, countAggNames)); @@ -257,8 +276,8 @@ public static Pair, OpenSearchAggregationResponseParser // Metric else { List> buckets = - createCompositeBuckets(groupList, project, helper); - if (buckets.size() != groupList.size()) { + createCompositeBuckets(groupNameAndIndexList, project, helper); + if (buckets.size() != groupNameAndIndexList.size()) { throw new UnsupportedOperationException( "Not all the left aggregations can be converted to value sources of composite" + " aggregation"); @@ -337,7 +356,7 @@ private static Pair> processAggregateCalls( for (int i = 0; i < aggCalls.size(); i++) { AggregateCall aggCall = aggCalls.get(i); - List> args = convertAggArgThroughProject(aggCall, project); + List> args = convertAggArgThroughProject(aggCall, project, helper); String aggFieldName = aggFieldNames.get(i); Pair builderAndParser = @@ -356,12 +375,19 @@ private static Pair> processAggregateCalls( * * @param aggCall the aggregate call * @param project the project + * @param helper the AggregateBuilderHelper * @return the converted Pair list */ private static List> convertAggArgThroughProject( - AggregateCall aggCall, Project project) { + AggregateCall aggCall, Project project, AggregateAnalyzer.AggregateBuilderHelper helper) { return project == null - ? List.of() + ? aggCall.getArgList().stream() + .map( + i -> + Pair.of( + (RexNode) RexInputRef.of(i, helper.rowType), + helper.rowType.getFieldNames().get(i))) + .toList() : PlanUtils.getObjectFromLiteralAgg(aggCall) != null ? project.getNamedProjects().stream() .filter(rex -> !rex.getKey().isA(SqlKind.ROW_NUMBER)) @@ -617,10 +643,15 @@ private static boolean supportsMaxMinAggregation(ExprType fieldType) { } private static List> createCompositeBuckets( - List groupList, Project project, AggregateAnalyzer.AggregateBuilderHelper helper) { + List> groupNameAndIndexList, + @Nullable Project project, + AggregateAnalyzer.AggregateBuilderHelper helper) { ImmutableList.Builder> resultBuilder = ImmutableList.builder(); - groupList.forEach( - groupIndex -> resultBuilder.add(createCompositeBucket(groupIndex, project, helper))); + groupNameAndIndexList.forEach( + nameAndIndex -> + resultBuilder.add( + createCompositeBucket( + nameAndIndex.getLeft(), nameAndIndex.getRight(), project, helper))); return resultBuilder.build(); } @@ -641,7 +672,7 @@ private static List> createCompositeBuckets( * └── Metric Aggregation (at the bottom) * * - * @param groupList the list of group by field indices from the query + * @param groupNameAndIndexList the list of group by field names and indices from the query * @param project the projection containing the expressions to analyze * @param metricBuilder the metric aggregation builder to be placed at the bottom of the hierarchy * @param helper the aggregation builder helper containing row type and utility methods @@ -653,20 +684,20 @@ private static List> createCompositeBuckets( * */ private static Pair, AggregationBuilder> createNestedAggregation( - List groupList, - Project project, + List> groupNameAndIndexList, + @Nullable Project project, Builder metricBuilder, AggregateAnalyzer.AggregateBuilderHelper helper) { AggregationBuilder rootAggBuilder = null; AggregationBuilder tailAggBuilder = null; Set aggPushed = new HashSet<>(); - for (Integer i : groupList) { - RexNode agg = project.getProjects().get(i); - String name = project.getNamedProjects().get(i).getValue(); + for (Pair nameAndIndex : groupNameAndIndexList) { + RexNode agg = helper.inferRexNodeFromIndex(nameAndIndex.getRight(), project); + String name = nameAndIndex.getLeft(); AggregationBuilder aggBuilder = createCompositeIncompatibleAggregation(agg, name, helper); if (aggBuilder != null) { - aggPushed.add(i); + aggPushed.add(nameAndIndex.getRight()); if (rootAggBuilder == null) { rootAggBuilder = aggBuilder; } else { @@ -737,9 +768,11 @@ private static boolean isCase(RexNode rex) { } private static CompositeValuesSourceBuilder createCompositeBucket( - Integer groupIndex, Project project, AggregateAnalyzer.AggregateBuilderHelper helper) { - RexNode rex = project.getProjects().get(groupIndex); - String bucketName = project.getRowType().getFieldNames().get(groupIndex); + String bucketName, + Integer groupIndex, + @Nullable Project project, + AggregateAnalyzer.AggregateBuilderHelper helper) { + RexNode rex = helper.inferRexNodeFromIndex(groupIndex, project); if (rex instanceof RexCall rexCall && rexCall.getKind() == SqlKind.OTHER_FUNCTION && rexCall.getOperator().getName().equalsIgnoreCase(BuiltinFunctionName.SPAN.name()) diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/storage/scan/CalciteLogicalIndexScan.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/storage/scan/CalciteLogicalIndexScan.java index 46b23b3600d..2a606c00ad6 100644 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/storage/scan/CalciteLogicalIndexScan.java +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/storage/scan/CalciteLogicalIndexScan.java @@ -10,6 +10,7 @@ import java.util.Map; import java.util.Objects; import java.util.stream.Collectors; +import javax.annotation.Nullable; import lombok.Getter; import org.apache.calcite.plan.Convention; import org.apache.calcite.plan.RelOptCluster; @@ -58,6 +59,7 @@ import org.opensearch.sql.opensearch.storage.scan.context.FilterDigest; import org.opensearch.sql.opensearch.storage.scan.context.LimitDigest; import org.opensearch.sql.opensearch.storage.scan.context.OSRequestBuilderAction; +import org.opensearch.sql.opensearch.storage.scan.context.ProjectDigest; import org.opensearch.sql.opensearch.storage.scan.context.PushDownContext; import org.opensearch.sql.opensearch.storage.scan.context.PushDownType; import org.opensearch.sql.opensearch.storage.scan.context.RareTopDigest; @@ -264,7 +266,10 @@ public CalciteLogicalIndexScan pushDownProject(List selectedColumns) { requestBuilder -> requestBuilder.pushDownProjectStream(newSchema.getFieldNames().stream()); } - newScan.pushDownContext.add(PushDownType.PROJECT, newSchema.getFieldNames(), action); + newScan.pushDownContext.add( + PushDownType.PROJECT, + new ProjectDigest(newSchema.getFieldNames(), selectedColumns), + action); return newScan; } @@ -336,7 +341,7 @@ public CalciteLogicalIndexScan pushDownRareTop(Project project, RareTopDigest di } } - public AbstractRelNode pushDownAggregate(Aggregate aggregate, Project project) { + public AbstractRelNode pushDownAggregate(Aggregate aggregate, @Nullable Project project) { try { CalciteLogicalIndexScan newScan = new CalciteLogicalIndexScan( @@ -346,8 +351,7 @@ public AbstractRelNode pushDownAggregate(Aggregate aggregate, Project project) { table, osIndex, aggregate.getRowType(), - // Aggregation will eliminate all collations. - pushDownContext.cloneWithoutSort()); + pushDownContext.cloneForAggregate(aggregate, project)); List schema = this.getRowType().getFieldNames(); Map fieldTypes = this.osIndex.getAllFieldTypes().entrySet().stream() @@ -394,10 +398,25 @@ public AbstractRelNode pushDownAggregate(Aggregate aggregate, Project project) { public AbstractRelNode pushDownLimit(LogicalSort sort, Integer limit, Integer offset) { try { if (pushDownContext.isAggregatePushed()) { + int totalSize = limit + offset; + // Since the AggPushDownAction is shared among different PushDownContext, its size() may be + // inaccurate(<= the actual size). + // So take the previous limit into account to decide whether it can update the context. + boolean canReduceEstimatedRowsCount = + !pushDownContext.isLimitPushed() + || pushDownContext.getQueue().reversed().stream() + .takeWhile(op -> op.type() != PushDownType.AGGREGATION) + .filter(op -> op.type() == PushDownType.LIMIT) + .findFirst() + .map(op -> (LimitDigest) op.digest()) + .map(d -> totalSize < d.offset() + d.limit()) + .orElse(true); + // Push down the limit into the aggregation bucket in advance to detect whether the limit // can update the aggregation builder boolean canUpdate = - pushDownContext.getAggPushDownAction().pushDownLimitIntoBucketSize(limit + offset); + canReduceEstimatedRowsCount + || pushDownContext.getAggPushDownAction().pushDownLimitIntoBucketSize(totalSize); if (!canUpdate && offset > 0) return null; CalciteLogicalIndexScan newScan = this.copyWithNewSchema(getRowType()); if (canUpdate) { diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/storage/scan/context/ProjectDigest.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/storage/scan/context/ProjectDigest.java new file mode 100644 index 00000000000..1d2c7e9f9ef --- /dev/null +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/storage/scan/context/ProjectDigest.java @@ -0,0 +1,15 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.opensearch.storage.scan.context; + +import java.util.List; + +public record ProjectDigest(List names, List selectedColumns) { + @Override + public String toString() { + return names.toString(); + } +} diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/storage/scan/context/PushDownContext.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/storage/scan/context/PushDownContext.java index 4f84746ade4..a3aacec842b 100644 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/storage/scan/context/PushDownContext.java +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/storage/scan/context/PushDownContext.java @@ -8,8 +8,13 @@ import java.util.AbstractCollection; import java.util.ArrayDeque; import java.util.Iterator; +import java.util.List; +import javax.annotation.Nullable; import lombok.Getter; +import org.apache.calcite.rel.core.Aggregate; +import org.apache.calcite.rel.core.Project; import org.jetbrains.annotations.NotNull; +import org.opensearch.sql.calcite.utils.PlanUtils; import org.opensearch.sql.opensearch.request.OpenSearchRequestBuilder; import org.opensearch.sql.opensearch.request.OpenSearchRequestBuilder.PushDownUnSupportedException; import org.opensearch.sql.opensearch.storage.OpenSearchIndex; @@ -63,6 +68,51 @@ public PushDownContext cloneWithoutSort() { return newContext; } + /** + * Aggregation will eliminate all collations, thus we need to remove sort. Will also remove + * project as sources is useless for aggregation, and remove filter if it's derived from the + * aggregate. + */ + public PushDownContext cloneForAggregate(Aggregate aggregate, @Nullable Project project) { + PushDownContext newContext = new PushDownContext(osIndex); + ArrayDeque tempQueue = new ArrayDeque<>(this.queue); + while (!tempQueue.isEmpty()) { + PushDownOperation operation = tempQueue.pollFirst(); + if (operation.type() == PushDownType.SORT + || operation.type() == PushDownType.SORT_EXPR + || operation.type() == PushDownType.PROJECT) { + continue; + } + if (operation.type() == PushDownType.FILTER) { + List currentColumns = null; + // If project push down happens between this aggregate push down and previous filter push + // down, + // there is a single project left in the queue. That project will affect the mapping between + // them. + while (!tempQueue.isEmpty() && tempQueue.peekFirst().type() == PushDownType.PROJECT) { + ProjectDigest projectDigest = (ProjectDigest) tempQueue.pollFirst().digest(); + List selectedColumns = projectDigest.selectedColumns(); + currentColumns = + currentColumns == null + ? selectedColumns + : selectedColumns.stream().map(currentColumns::get).toList(); + } + // Check if filter is derived from aggregate. Ensure there is no other push down operations + // left between them. + if (tempQueue.isEmpty() + && PlanUtils.isNotNullDerivedFromAgg( + ((FilterDigest) operation.digest()).condition(), + aggregate, + project, + currentColumns)) { + continue; + } + } + newContext.add(operation); + } + return newContext; + } + @NotNull @Override public Iterator iterator() { diff --git a/opensearch/src/test/java/org/opensearch/sql/opensearch/request/AggregateAnalyzerTest.java b/opensearch/src/test/java/org/opensearch/sql/opensearch/request/AggregateAnalyzerTest.java index 660744c8bb7..2e79da953b1 100644 --- a/opensearch/src/test/java/org/opensearch/sql/opensearch/request/AggregateAnalyzerTest.java +++ b/opensearch/src/test/java/org/opensearch/sql/opensearch/request/AggregateAnalyzerTest.java @@ -358,7 +358,7 @@ void analyze_firstAggregation() throws ExpressionNotAnalyzableException { buildAggregation("first_a") .withAggCall(b -> b.aggregateCall(PPLBuiltinOperators.FIRST, b.field("a")).as("first_a")) .expectDslQuery( - "[{\"first_a\":{\"top_hits\":{\"from\":0,\"size\":1,\"version\":false,\"seq_no_primary_term\":false,\"explain\":false}}}]") + "[{\"first_a\":{\"top_hits\":{\"from\":0,\"size\":1,\"version\":false,\"seq_no_primary_term\":false,\"explain\":false,\"fields\":[{\"field\":\"a\"}]}}}]") .expectResponseParser( new MetricParserHelper(List.of(new TopHitsParser("first_a", true, false)))) .verify(); @@ -369,7 +369,7 @@ void analyze_lastAggregation() throws ExpressionNotAnalyzableException { buildAggregation("last_b") .withAggCall(b -> b.aggregateCall(PPLBuiltinOperators.LAST, b.field("b")).as("last_b")) .expectDslQuery( - "[{\"last_b\":{\"top_hits\":{\"from\":0,\"size\":1,\"version\":false,\"seq_no_primary_term\":false,\"explain\":false,\"sort\":[{\"_doc\":{\"order\":\"desc\"}}]}}}]") + "[{\"last_b\":{\"top_hits\":{\"from\":0,\"size\":1,\"version\":false,\"seq_no_primary_term\":false,\"explain\":false,\"fields\":[{\"field\":\"b.keyword\"}],\"sort\":[{\"_doc\":{\"order\":\"desc\"}}]}}}]") .expectResponseParser( new MetricParserHelper(List.of(new TopHitsParser("last_b", true, false)))) .verify();