From 2a88ae61e0c3348961378d1b2e5d51fa0ac60238 Mon Sep 17 00:00:00 2001 From: Lantao Jin Date: Tue, 23 Dec 2025 20:12:28 +0800 Subject: [PATCH 1/4] Support enumerable TopK Signed-off-by: Lantao Jin --- .../calcite/clickbench/q28.yaml | 7 +- .../calcite/clickbench/q29.yaml | 7 +- .../explain_exists_correlated_subquery.yaml | 4 +- .../calcite/explain_limit_agg_pushdown5.json | 2 +- .../explain_multisearch_timestamp.yaml | 8 +-- .../explain_streamstats_distinct_count.yaml | 9 ++- .../explain_streamstats_earliest_latest.yaml | 9 ++- ...reamstats_earliest_latest_custom_time.yaml | 9 ++- ...xplain_streamstats_global_null_bucket.yaml | 25 +++---- .../explain_streamstats_null_bucket.yaml | 11 ++-- ...explain_streamstats_reset_null_bucket.yaml | 33 +++++----- .../physical/CalciteEnumerableTopK.java | 54 +++++++++++++++ .../planner/rules/EnumerableTopKRule.java | 65 +++++++++++++++++++ .../planner/rules/OpenSearchIndexRules.java | 3 + .../storage/scan/CalciteLogicalIndexScan.java | 7 +- .../storage/scan/context/PushDownContext.java | 5 ++ 16 files changed, 193 insertions(+), 65 deletions(-) create mode 100644 opensearch/src/main/java/org/opensearch/sql/opensearch/planner/physical/CalciteEnumerableTopK.java create mode 100644 opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/EnumerableTopKRule.java 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..6f4d2a1e6fd 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q28.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q28.yaml @@ -10,7 +10,6 @@ calcite: CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | EnumerableLimit(fetch=[10000]) - 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 + EnumerableLimitSort(sort0=[$0], dir0=[DESC-nulls-last], fetch=[25]) + EnumerableCalc(expr#0..2=[{inputs}], expr#3=[100000], expr#4=[>($t1, $t3)], proj#0..2=[{exprs}], $condition=[$t4]) + CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[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 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..b9b7fc67ec6 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q29.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q29.yaml @@ -12,7 +12,6 @@ calcite: CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | EnumerableLimit(fetch=[10000]) - 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)]) + EnumerableLimitSort(sort0=[$0], dir0=[DESC-nulls-last], fetch=[25]) + 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)]) \ No newline at end of file diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_exists_correlated_subquery.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_exists_correlated_subquery.yaml index 920149399c0..1ac2e585dd3 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_exists_correlated_subquery.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_exists_correlated_subquery.yaml @@ -12,8 +12,8 @@ calcite: })], variablesSet=[[$cor0]]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_worker]]) physical: | - EnumerableCalc(expr#0..3=[{inputs}], id=[$t1], name=[$t0], salary=[$t2]) - EnumerableLimit(fetch=[10000]) + EnumerableLimit(fetch=[10000]) + EnumerableCalc(expr#0..3=[{inputs}], id=[$t1], name=[$t0], salary=[$t2]) EnumerableSort(sort0=[$2], dir0=[DESC-nulls-last]) EnumerableCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_worker]], PushDownContext=[[PROJECT->[name, id, salary]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["name","id","salary"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_limit_agg_pushdown5.json b/integ-test/src/test/resources/expectedOutput/calcite/explain_limit_agg_pushdown5.json index e9756e74db6..7895625cde5 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_limit_agg_pushdown5.json +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_limit_agg_pushdown5.json @@ -1,6 +1,6 @@ { "calcite": { "logical": "LogicalSystemLimit(fetch=[10000], type=[QUERY_SIZE_LIMIT])\n LogicalSort(offset=[10], fetch=[10])\n LogicalSort(sort0=[$0], dir0=[ASC-nulls-first], 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 EnumerableLimit(offset=[10], fetch=[10])\n EnumerableLimit(fetch=[100])\n EnumerableSort(sort0=[$0], dir0=[ASC-nulls-first])\n CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},count()=COUNT()), PROJECT->[count(), state]], 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\"}}}]}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)])\n" + "physical": "EnumerableLimit(fetch=[10000])\n EnumerableLimit(offset=[10], fetch=[10])\n EnumerableLimitSort(sort0=[$0], dir0=[ASC-nulls-first], fetch=[100])\n CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},count()=COUNT()), PROJECT->[count(), state]], 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\"}}}]}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)])\n" } } diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_multisearch_timestamp.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_multisearch_timestamp.yaml index b38889b0323..4a3c5e0d325 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_multisearch_timestamp.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_multisearch_timestamp.yaml @@ -13,15 +13,15 @@ calcite: EnumerableLimit(fetch=[10000]) EnumerableLimit(fetch=[5]) EnumerableMergeUnion(all=[true]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]], PushDownContext=[[PROJECT->[@timestamp, category, value, timestamp], FILTER->SEARCH($1, Sarg['A':VARCHAR, 'B':VARCHAR]:VARCHAR), SORT->[{ + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]], PushDownContext=[[FILTER->SEARCH($1, Sarg['A':VARCHAR, 'B':VARCHAR]:VARCHAR), SORT->[{ "@timestamp" : { "order" : "desc", "missing" : "_first" } - }], LIMIT->5, LIMIT->5], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":5,"timeout":"1m","query":{"terms":{"category":["A","B"],"boost":1.0}},"_source":{"includes":["@timestamp","category","value","timestamp"],"excludes":[]},"sort":[{"@timestamp":{"order":"desc","missing":"_first"}}]}, requestedTotalSize=5, pageSize=null, startFrom=0)]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data2]], PushDownContext=[[PROJECT->[@timestamp, category, value, timestamp], FILTER->SEARCH($1, Sarg['E':VARCHAR, 'F':VARCHAR]:VARCHAR), SORT->[{ + }], LIMIT->5, PROJECT->[@timestamp, category, value, timestamp], LIMIT->5], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":5,"timeout":"1m","query":{"terms":{"category":["A","B"],"boost":1.0}},"_source":{"includes":["@timestamp","category","value","timestamp"],"excludes":[]},"sort":[{"@timestamp":{"order":"desc","missing":"_first"}}]}, requestedTotalSize=5, pageSize=null, startFrom=0)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data2]], PushDownContext=[[FILTER->SEARCH($1, Sarg['E':VARCHAR, 'F':VARCHAR]:VARCHAR), SORT->[{ "@timestamp" : { "order" : "desc", "missing" : "_first" } - }], LIMIT->5, LIMIT->5], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":5,"timeout":"1m","query":{"terms":{"category":["E","F"],"boost":1.0}},"_source":{"includes":["@timestamp","category","value","timestamp"],"excludes":[]},"sort":[{"@timestamp":{"order":"desc","missing":"_first"}}]}, requestedTotalSize=5, pageSize=null, startFrom=0)]) \ No newline at end of file + }], LIMIT->5, PROJECT->[@timestamp, category, value, timestamp], LIMIT->5], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":5,"timeout":"1m","query":{"terms":{"category":["E","F"],"boost":1.0}},"_source":{"includes":["@timestamp","category","value","timestamp"],"excludes":[]},"sort":[{"@timestamp":{"order":"desc","missing":"_first"}}]}, requestedTotalSize=5, pageSize=null, startFrom=0)]) \ No newline at end of file diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_distinct_count.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_distinct_count.yaml index 32538ab17df..e47cc2a3680 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_distinct_count.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_distinct_count.yaml @@ -8,8 +8,7 @@ calcite: CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | EnumerableCalc(expr#0..12=[{inputs}], proj#0..10=[{exprs}], distinct_states=[$t12]) - EnumerableLimit(fetch=[10000]) - EnumerableSort(sort0=[$11], dir0=[ASC]) - EnumerableWindow(window#0=[window(partition {4} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [DISTINCT_COUNT_APPROX($7)])]) - EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[account_number, firstname, address, balance, gender, city, employer, state, age, email, lastname]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["account_number","firstname","address","balance","gender","city","employer","state","age","email","lastname"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file + EnumerableLimitSort(sort0=[$11], dir0=[ASC], fetch=[10000]) + EnumerableWindow(window#0=[window(partition {4} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [DISTINCT_COUNT_APPROX($7)])]) + EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[account_number, firstname, address, balance, gender, city, employer, state, age, email, lastname]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["account_number","firstname","address","balance","gender","city","employer","state","age","email","lastname"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_earliest_latest.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_earliest_latest.yaml index cac21b929ee..e7b7e1ca264 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_earliest_latest.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_earliest_latest.yaml @@ -8,8 +8,7 @@ calcite: CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_logs]]) physical: | EnumerableCalc(expr#0..7=[{inputs}], proj#0..4=[{exprs}], earliest_message=[$t6], latest_message=[$t7]) - EnumerableLimit(fetch=[10000]) - EnumerableSort(sort0=[$5], dir0=[ASC]) - EnumerableWindow(window#0=[window(partition {1} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ARG_MIN($3, $2), ARG_MAX($3, $2)])]) - EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_logs]], PushDownContext=[[PROJECT->[created_at, server, @timestamp, message, level]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["created_at","server","@timestamp","message","level"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file + EnumerableLimitSort(sort0=[$5], dir0=[ASC], fetch=[10000]) + EnumerableWindow(window#0=[window(partition {1} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ARG_MIN($3, $2), ARG_MAX($3, $2)])]) + EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_logs]], PushDownContext=[[PROJECT->[created_at, server, @timestamp, message, level]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["created_at","server","@timestamp","message","level"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_earliest_latest_custom_time.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_earliest_latest_custom_time.yaml index f19625d85e5..30b0bbb289b 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_earliest_latest_custom_time.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_earliest_latest_custom_time.yaml @@ -8,8 +8,7 @@ calcite: CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_logs]]) physical: | EnumerableCalc(expr#0..7=[{inputs}], proj#0..4=[{exprs}], earliest_message=[$t6], latest_message=[$t7]) - EnumerableLimit(fetch=[10000]) - EnumerableSort(sort0=[$5], dir0=[ASC]) - EnumerableWindow(window#0=[window(partition {4} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ARG_MIN($3, $0), ARG_MAX($3, $0)])]) - EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_logs]], PushDownContext=[[PROJECT->[created_at, server, @timestamp, message, level]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["created_at","server","@timestamp","message","level"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file + EnumerableLimitSort(sort0=[$5], dir0=[ASC], fetch=[10000]) + EnumerableWindow(window#0=[window(partition {4} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ARG_MIN($3, $0), ARG_MAX($3, $0)])]) + EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_logs]], PushDownContext=[[PROJECT->[created_at, server, @timestamp, message, level]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["created_at","server","@timestamp","message","level"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_global_null_bucket.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_global_null_bucket.yaml index f615f0633b2..67cb26a297b 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_global_null_bucket.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_global_null_bucket.yaml @@ -13,18 +13,19 @@ calcite: CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | EnumerableCalc(expr#0..16=[{inputs}], proj#0..10=[{exprs}], avg_age=[$t16]) - EnumerableLimit(fetch=[10000]) - EnumerableHashJoin(condition=[AND(=($4, $13), =($11, $14), =($12, $15))], joinType=[left]) - EnumerableSort(sort0=[$11], dir0=[ASC]) + EnumerableLimitSort(sort0=[$11], dir0=[ASC], fetch=[10000]) + EnumerableMergeJoin(condition=[AND(=($4, $13), =($11, $14), =($12, $15))], joinType=[left]) + EnumerableSort(sort0=[$4], sort1=[$11], sort2=[$12], dir0=[ASC], dir1=[ASC], dir2=[ASC]) EnumerableCalc(expr#0..11=[{inputs}], expr#12=[1], expr#13=[-($t11, $t12)], proj#0..11=[{exprs}], $f12=[$t13]) EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[account_number, firstname, address, balance, gender, city, employer, state, age, email, lastname]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["account_number","firstname","address","balance","gender","city","employer","state","age","email","lastname"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) - EnumerableCalc(expr#0..4=[{inputs}], expr#5=[0], expr#6=[=($t4, $t5)], expr#7=[null:BIGINT], expr#8=[CASE($t6, $t7, $t3)], expr#9=[CAST($t8):DOUBLE], expr#10=[/($t9, $t4)], proj#0..2=[{exprs}], avg_age=[$t10]) - EnumerableAggregate(group=[{0, 1, 2}], agg#0=[$SUM0($4)], agg#1=[COUNT($4)]) - EnumerableHashJoin(condition=[AND(=($0, $3), >=($5, $2), <=($5, $1))], joinType=[inner]) - EnumerableAggregate(group=[{0, 1, 2}]) - EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], expr#3=[-($t1, $t2)], proj#0..1=[{exprs}], $f2=[$t3]) - EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[gender]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["gender"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) - EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[gender, age]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["gender","age"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file + EnumerableSort(sort0=[$0], sort1=[$1], sort2=[$2], dir0=[ASC], dir1=[ASC], dir2=[ASC]) + EnumerableCalc(expr#0..4=[{inputs}], expr#5=[0], expr#6=[=($t4, $t5)], expr#7=[null:BIGINT], expr#8=[CASE($t6, $t7, $t3)], expr#9=[CAST($t8):DOUBLE], expr#10=[/($t9, $t4)], proj#0..2=[{exprs}], avg_age=[$t10]) + EnumerableAggregate(group=[{0, 1, 2}], agg#0=[$SUM0($4)], agg#1=[COUNT($4)]) + EnumerableHashJoin(condition=[AND(=($0, $3), >=($5, $2), <=($5, $1))], joinType=[inner]) + EnumerableAggregate(group=[{0, 1, 2}]) + EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], expr#3=[-($t1, $t2)], proj#0..1=[{exprs}], $f2=[$t3]) + EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[gender]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["gender"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[gender, age]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["gender","age"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_null_bucket.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_null_bucket.yaml index fe79eb90faa..be9a5f3d391 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_null_bucket.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_null_bucket.yaml @@ -8,9 +8,8 @@ calcite: CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | EnumerableCalc(expr#0..14=[{inputs}], expr#15=[CAST($t14):DOUBLE NOT NULL], expr#16=[/($t13, $t15)], expr#17=[null:DOUBLE], expr#18=[CASE($t12, $t16, $t17)], proj#0..10=[{exprs}], avg_age=[$t18]) - EnumerableLimit(fetch=[10000]) - EnumerableSort(sort0=[$11], dir0=[ASC]) - EnumerableWindow(window#0=[window(partition {4} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [$SUM0($8), COUNT($8)])]) - EnumerableCalc(expr#0..11=[{inputs}], expr#12=[IS NOT NULL($t4)], proj#0..12=[{exprs}]) - EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[account_number, firstname, address, balance, gender, city, employer, state, age, email, lastname]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["account_number","firstname","address","balance","gender","city","employer","state","age","email","lastname"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file + EnumerableLimitSort(sort0=[$11], dir0=[ASC], fetch=[10000]) + EnumerableWindow(window#0=[window(partition {4} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [$SUM0($8), COUNT($8)])]) + EnumerableCalc(expr#0..11=[{inputs}], expr#12=[IS NOT NULL($t4)], proj#0..12=[{exprs}]) + EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[account_number, firstname, address, balance, gender, city, employer, state, age, email, lastname]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["account_number","firstname","address","balance","gender","city","employer","state","age","email","lastname"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_reset_null_bucket.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_reset_null_bucket.yaml index d2f0db97af9..48d059bdd85 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_reset_null_bucket.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_reset_null_bucket.yaml @@ -15,25 +15,26 @@ calcite: CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | EnumerableCalc(expr#0..16=[{inputs}], proj#0..10=[{exprs}], avg_age=[$t16]) - EnumerableLimit(fetch=[10000]) - EnumerableHashJoin(condition=[AND(=($4, $13), =($11, $14), =($12, $15))], joinType=[left]) - EnumerableSort(sort0=[$11], dir0=[ASC]) + EnumerableLimitSort(sort0=[$11], dir0=[ASC], fetch=[10000]) + EnumerableMergeJoin(condition=[AND(=($4, $13), =($11, $14), =($12, $15))], joinType=[left]) + EnumerableSort(sort0=[$4], sort1=[$11], sort2=[$12], dir0=[ASC], dir1=[ASC], dir2=[ASC]) EnumerableCalc(expr#0..15=[{inputs}], expr#16=[0], expr#17=[COALESCE($t15, $t16)], expr#18=[+($t14, $t17)], proj#0..11=[{exprs}], __seg_id__=[$t18]) EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [$SUM0($12)])], window#1=[window(rows between UNBOUNDED PRECEDING and $14 PRECEDING aggs [$SUM0($13)])], constants=[[1]]) EnumerableCalc(expr#0..11=[{inputs}], expr#12=[34], expr#13=[>($t8, $t12)], expr#14=[1], expr#15=[0], expr#16=[CASE($t13, $t14, $t15)], expr#17=[25], expr#18=[<($t8, $t17)], expr#19=[CASE($t18, $t14, $t15)], proj#0..11=[{exprs}], __reset_before_flag__=[$t16], __reset_after_flag__=[$t19]) EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[account_number, firstname, address, balance, gender, city, employer, state, age, email, lastname]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["account_number","firstname","address","balance","gender","city","employer","state","age","email","lastname"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) - EnumerableCalc(expr#0..4=[{inputs}], expr#5=[0], expr#6=[=($t4, $t5)], expr#7=[null:BIGINT], expr#8=[CASE($t6, $t7, $t3)], expr#9=[CAST($t8):DOUBLE], expr#10=[/($t9, $t4)], proj#0..2=[{exprs}], avg_age=[$t10]) - EnumerableAggregate(group=[{0, 1, 2}], agg#0=[$SUM0($4)], agg#1=[COUNT($4)]) - EnumerableHashJoin(condition=[AND(=($2, $6), =($0, $3), <($5, $1))], joinType=[inner]) - EnumerableAggregate(group=[{0, 1, 2}]) - EnumerableCalc(expr#0..5=[{inputs}], expr#6=[0], expr#7=[COALESCE($t5, $t6)], expr#8=[+($t4, $t7)], proj#0..1=[{exprs}], __seg_id__=[$t8]) - EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [$SUM0($2)])], window#1=[window(rows between UNBOUNDED PRECEDING and $4 PRECEDING aggs [$SUM0($3)])], constants=[[1]]) - EnumerableCalc(expr#0..2=[{inputs}], expr#3=[34], expr#4=[>($t1, $t3)], expr#5=[1], expr#6=[0], expr#7=[CASE($t4, $t5, $t6)], expr#8=[25], expr#9=[<($t1, $t8)], expr#10=[CASE($t9, $t5, $t6)], gender=[$t0], __stream_seq__=[$t2], __reset_before_flag__=[$t7], __reset_after_flag__=[$t10]) + EnumerableSort(sort0=[$0], sort1=[$1], sort2=[$2], dir0=[ASC], dir1=[ASC], dir2=[ASC]) + EnumerableCalc(expr#0..4=[{inputs}], expr#5=[0], expr#6=[=($t4, $t5)], expr#7=[null:BIGINT], expr#8=[CASE($t6, $t7, $t3)], expr#9=[CAST($t8):DOUBLE], expr#10=[/($t9, $t4)], proj#0..2=[{exprs}], avg_age=[$t10]) + EnumerableAggregate(group=[{0, 1, 2}], agg#0=[$SUM0($4)], agg#1=[COUNT($4)]) + EnumerableHashJoin(condition=[AND(=($2, $6), =($0, $3), <($5, $1))], joinType=[inner]) + EnumerableAggregate(group=[{0, 1, 2}]) + EnumerableCalc(expr#0..5=[{inputs}], expr#6=[0], expr#7=[COALESCE($t5, $t6)], expr#8=[+($t4, $t7)], proj#0..1=[{exprs}], __seg_id__=[$t8]) + EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [$SUM0($2)])], window#1=[window(rows between UNBOUNDED PRECEDING and $4 PRECEDING aggs [$SUM0($3)])], constants=[[1]]) + EnumerableCalc(expr#0..2=[{inputs}], expr#3=[34], expr#4=[>($t1, $t3)], expr#5=[1], expr#6=[0], expr#7=[CASE($t4, $t5, $t6)], expr#8=[25], expr#9=[<($t1, $t8)], expr#10=[CASE($t9, $t5, $t6)], gender=[$t0], __stream_seq__=[$t2], __reset_before_flag__=[$t7], __reset_after_flag__=[$t10]) + EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[gender, age]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["gender","age"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + EnumerableCalc(expr#0..6=[{inputs}], expr#7=[0], expr#8=[COALESCE($t6, $t7)], expr#9=[+($t5, $t8)], proj#0..2=[{exprs}], __seg_id__=[$t9]) + EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [$SUM0($3)])], window#1=[window(rows between UNBOUNDED PRECEDING and $5 PRECEDING aggs [$SUM0($4)])], constants=[[1]]) + EnumerableCalc(expr#0..2=[{inputs}], expr#3=[34], expr#4=[>($t1, $t3)], expr#5=[1], expr#6=[0], expr#7=[CASE($t4, $t5, $t6)], expr#8=[25], expr#9=[<($t1, $t8)], expr#10=[CASE($t9, $t5, $t6)], proj#0..2=[{exprs}], __reset_before_flag__=[$t7], __reset_after_flag__=[$t10]) EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[gender, age]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["gender","age"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) - EnumerableCalc(expr#0..6=[{inputs}], expr#7=[0], expr#8=[COALESCE($t6, $t7)], expr#9=[+($t5, $t8)], proj#0..2=[{exprs}], __seg_id__=[$t9]) - EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [$SUM0($3)])], window#1=[window(rows between UNBOUNDED PRECEDING and $5 PRECEDING aggs [$SUM0($4)])], constants=[[1]]) - EnumerableCalc(expr#0..2=[{inputs}], expr#3=[34], expr#4=[>($t1, $t3)], expr#5=[1], expr#6=[0], expr#7=[CASE($t4, $t5, $t6)], expr#8=[25], expr#9=[<($t1, $t8)], expr#10=[CASE($t9, $t5, $t6)], proj#0..2=[{exprs}], __reset_before_flag__=[$t7], __reset_after_flag__=[$t10]) - EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[gender, age]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["gender","age"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[gender, age]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["gender","age"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/physical/CalciteEnumerableTopK.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/physical/CalciteEnumerableTopK.java new file mode 100644 index 00000000000..b9a88eff5b9 --- /dev/null +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/physical/CalciteEnumerableTopK.java @@ -0,0 +1,54 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.opensearch.planner.physical; + +import org.apache.calcite.adapter.enumerable.EnumerableConvention; +import org.apache.calcite.adapter.enumerable.EnumerableLimitSort; +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelOptCost; +import org.apache.calcite.plan.RelOptPlanner; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.RelCollation; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.metadata.RelMetadataQuery; +import org.apache.calcite.rex.RexNode; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** The different between this and {@link EnumerableLimitSort} is the cost. */ +public class CalciteEnumerableTopK extends EnumerableLimitSort { + + public CalciteEnumerableTopK( + RelOptCluster cluster, + RelTraitSet traitSet, + RelNode input, + RelCollation collation, + @Nullable RexNode offset, + @Nullable RexNode fetch) { + super(cluster, traitSet, input, collation, offset, fetch); + } + + @Override + public @Nullable RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) { + RelOptCost original = super.computeSelfCost(planner, mq); + if (original == null) { + return null; + } + // CalciteEnumerableTopK is converted by merging EnumerableLimit + EnumerableSort. + // The cost should be less than cost(EnumerableLimit) + cost(CalciteEnumerableTopK) + // which equals to getRows() * 2 * 0.99 + return planner + .getCostFactory() + .makeCost(original.getRows() * 2 * 0.99, original.getCpu(), original.getIo()); + } + + public static CalciteEnumerableTopK create( + RelNode input, RelCollation collation, @Nullable RexNode offset, @Nullable RexNode fetch) { + final RelOptCluster cluster = input.getCluster(); + final RelTraitSet traitSet = + cluster.traitSetOf(EnumerableConvention.INSTANCE).replace(collation); + return new CalciteEnumerableTopK(cluster, traitSet, input, collation, offset, fetch); + } +} diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/EnumerableTopKRule.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/EnumerableTopKRule.java new file mode 100644 index 00000000000..6edeb0660a7 --- /dev/null +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/EnumerableTopKRule.java @@ -0,0 +1,65 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.opensearch.planner.rules; + +import org.apache.calcite.adapter.enumerable.EnumerableConvention; +import org.apache.calcite.plan.RelOptRuleCall; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.Sort; +import org.apache.calcite.rel.logical.LogicalSort; +import org.immutables.value.Value; +import org.opensearch.sql.calcite.plan.OpenSearchRuleConfig; +import org.opensearch.sql.opensearch.planner.physical.CalciteEnumerableTopK; + +/** + * Rule to convert an {@link LogicalSort} with ({@link LogicalSort#fetch} or {@link + * LogicalSort#offset}) and {@link LogicalSort#collation}(Order By) into an {@link + * CalciteEnumerableTopK}. + */ +@Value.Enclosing +public class EnumerableTopKRule extends InterruptibleRelRule { + protected EnumerableTopKRule(EnumerableTopKRule.Config config) { + super(config); + } + + @Override + protected void onMatchImpl(RelOptRuleCall call) { + final Sort sort = call.rel(0); + RelNode input = sort.getInput(); + final Sort topK = + CalciteEnumerableTopK.create( + convert( + call.getPlanner(), + input, + input.getTraitSet().replace(EnumerableConvention.INSTANCE)), + sort.getCollation(), + sort.offset, + sort.fetch); + call.transformTo(topK); + } + + /** Rule configuration. */ + @Value.Immutable + public interface Config extends OpenSearchRuleConfig { + EnumerableTopKRule.Config DEFAULT = + ImmutableEnumerableTopKRule.Config.builder() + .build() + .withDescription("EnumerableTopK") + .withOperandSupplier( + b0 -> + b0.operand(LogicalSort.class) + .predicate( + sort -> + (sort.fetch != null || sort.offset != null) + && !sort.collation.getFieldCollations().isEmpty()) + .anyInputs()); + + @Override + default EnumerableTopKRule toRule() { + return new EnumerableTopKRule(this); + } + } +} 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..660731c7b26 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 @@ -41,6 +41,8 @@ public class OpenSearchIndexRules { RareTopPushdownRule.Config.DEFAULT.toRule(); private static final SortExprIndexScanRule SORT_EXPR_INDEX_SCAN = SortExprIndexScanRule.Config.DEFAULT.toRule(); + private static final EnumerableTopKRule ENUMERABLE_TOP_K_RULE = + EnumerableTopKRule.Config.DEFAULT.toRule(); // Rule that always pushes down relevance functions regardless of pushdown settings public static final RelevanceFunctionPushdownRule RELEVANCE_FUNCTION_PUSHDOWN = @@ -61,6 +63,7 @@ public class OpenSearchIndexRules { SORT_PROJECT_EXPR_TRANSPOSE, SORT_AGGREGATION_METRICS_RULE, RARE_TOP_PUSH_DOWN, + ENUMERABLE_TOP_K_RULE, EXPAND_COLLATION_ON_PROJECT_EXPR, SORT_EXPR_INDEX_SCAN); 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..adba568bbcd 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 @@ -417,10 +417,15 @@ public AbstractRelNode pushDownLimit(LogicalSort sort, Integer limit, Integer of newScan.pushDownContext.add(PushDownType.LIMIT, new LimitDigest(limit, offset), action); return offset > 0 ? sort.copy(sort.getTraitSet(), List.of(newScan)) : newScan; } else { + LimitDigest digest = new LimitDigest(limit, offset); + if (pushDownContext.containsDigestOnTop(digest)) { + // avoid stack overflow + return null; + } CalciteLogicalIndexScan newScan = this.copyWithNewSchema(getRowType()); newScan.pushDownContext.add( PushDownType.LIMIT, - new LimitDigest(limit, offset), + digest, (OSRequestBuilderAction) requestBuilder -> requestBuilder.pushDownLimit(limit, offset)); return newScan; } 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..a5972228512 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 @@ -139,6 +139,11 @@ public boolean containsDigest(Object digest) { return this.stream().anyMatch(action -> action.digest().equals(digest)); } + // TODO check on adding + public boolean containsDigestOnTop(Object digest) { + return this.queue.peekLast() != null && this.queue.peekLast().digest().equals(digest); + } + /** * Get the digest of the first operation of a specific type. * From ba8d9dfa371d993d630651812732a535e7bcc99d Mon Sep 17 00:00:00 2001 From: Lantao Jin Date: Fri, 26 Dec 2025 16:43:29 +0800 Subject: [PATCH 2/4] Support merging ELimit+ESort to ETopK Signed-off-by: Lantao Jin --- ...ange_metric_sort_agg_measure_not_push.yaml | 5 +- ...te_autodate_sort_agg_measure_not_push.yaml | 5 +- ...ms_autodate_sort_agg_measure_not_push.yaml | 7 +-- ...osite_range_sort_agg_measure_not_push.yaml | 5 +- .../big5/dedup_metrics_size_field.yaml | 5 +- .../calcite/chart_multiple_group_keys.yaml | 29 +++++----- .../calcite/chart_null_str.yaml | 37 ++++++------ .../chart_timestamp_span_and_category.yaml | 21 ++++--- .../calcite/chart_use_other.yaml | 21 ++++--- ...t_on_measure_multi_buckets_not_pushed.yaml | 5 +- .../explain_exists_correlated_subquery.yaml | 15 +++-- .../explain_exists_uncorrelated_subquery.yaml | 15 +++-- .../explain_in_correlated_subquery.yaml | 15 +++-- .../explain_in_uncorrelated_subquery.yaml | 11 ++-- ...gg_with_sort_on_one_measure_not_push1.yaml | 5 +- ...gg_with_sort_on_one_measure_not_push2.yaml | 5 +- .../calcite/explain_sort_agg_push.json | 2 +- ..._sort_pass_through_join_then_pushdown.yaml | 13 +++-- .../calcite/explain_timechart.yaml | 35 ++++++------ .../calcite/explain_timechart_count.yaml | 29 +++++----- .../calcite/explain_top_k_then_sort_push.yaml | 5 +- ....java => EnumerableTopKConverterRule.java} | 13 +++-- .../rules/EnumerableTopKMergeRule.java | 56 +++++++++++++++++++ .../planner/rules/OpenSearchIndexRules.java | 7 ++- 24 files changed, 206 insertions(+), 160 deletions(-) rename opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/{EnumerableTopKRule.java => EnumerableTopKConverterRule.java} (81%) create mode 100644 opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/EnumerableTopKMergeRule.java diff --git a/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_autodate_range_metric_sort_agg_measure_not_push.yaml b/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_autodate_range_metric_sort_agg_measure_not_push.yaml index ade5ec642b6..137687d1015 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_autodate_range_metric_sort_agg_measure_not_push.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_autodate_range_metric_sort_agg_measure_not_push.yaml @@ -9,6 +9,5 @@ calcite: LogicalProject(@timestamp=[$0], category=[$1], value=[$2], _id=[$4], _index=[$5], _score=[$6], _maxscore=[$7], _sort=[$8], _routing=[$9], timestamp=[WIDTH_BUCKET($3, 3, -(MAX($3) OVER (), MIN($3) OVER ()), MAX($3) OVER ())], value_range=[CASE(<($2, 7000), 'small':VARCHAR, 'great':VARCHAR)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]]) physical: | - EnumerableLimit(fetch=[10000]) - EnumerableSort(sort0=[$1], dir0=[ASC-nulls-first]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1, 2},avg(value)=AVG($3),cnt=COUNT()), PROJECT->[avg(value), cnt, timestamp, value_range, category]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"category":{"terms":{"field":"category","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"timestamp":{"auto_date_histogram":{"field":"timestamp","buckets":3,"minimum_interval":null},"aggregations":{"value_range":{"range":{"field":"value","ranges":[{"key":"small","to":7000.0},{"key":"great","from":7000.0}],"keyed":true},"aggregations":{"avg(value)":{"avg":{"field":"value"}}}}}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + EnumerableLimitSort(sort0=[$1], dir0=[ASC-nulls-first], fetch=[10000]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1, 2},avg(value)=AVG($3),cnt=COUNT()), PROJECT->[avg(value), cnt, timestamp, value_range, category]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"category":{"terms":{"field":"category","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"timestamp":{"auto_date_histogram":{"field":"timestamp","buckets":3,"minimum_interval":null},"aggregations":{"value_range":{"range":{"field":"value","ranges":[{"key":"small","to":7000.0},{"key":"great","from":7000.0}],"keyed":true},"aggregations":{"avg(value)":{"avg":{"field":"value"}}}}}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file diff --git a/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_autodate_sort_agg_measure_not_push.yaml b/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_autodate_sort_agg_measure_not_push.yaml index 48689ff1503..792e689f4f5 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_autodate_sort_agg_measure_not_push.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_autodate_sort_agg_measure_not_push.yaml @@ -9,6 +9,5 @@ calcite: LogicalProject(@timestamp=[$0], category=[$1], value=[$2], _id=[$4], _index=[$5], _score=[$6], _maxscore=[$7], _sort=[$8], _routing=[$9], timestamp=[WIDTH_BUCKET($3, 3, -(MAX($3) OVER (), MIN($3) OVER ()), MAX($3) OVER ())]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]]) physical: | - EnumerableLimit(fetch=[10000]) - EnumerableSort(sort0=[$1], dir0=[ASC-nulls-first]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},avg(value)=AVG($2),cnt=COUNT()), PROJECT->[avg(value), cnt, timestamp, category]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"category":{"terms":{"field":"category","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"timestamp":{"auto_date_histogram":{"field":"timestamp","buckets":3,"minimum_interval":null},"aggregations":{"avg(value)":{"avg":{"field":"value"}}}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + EnumerableLimitSort(sort0=[$1], dir0=[ASC-nulls-first], fetch=[10000]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},avg(value)=AVG($2),cnt=COUNT()), PROJECT->[avg(value), cnt, timestamp, category]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"category":{"terms":{"field":"category","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"timestamp":{"auto_date_histogram":{"field":"timestamp","buckets":3,"minimum_interval":null},"aggregations":{"avg(value)":{"avg":{"field":"value"}}}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file diff --git a/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_multi_terms_autodate_sort_agg_measure_not_push.yaml b/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_multi_terms_autodate_sort_agg_measure_not_push.yaml index 6995097f878..d3c4f0a1ee2 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_multi_terms_autodate_sort_agg_measure_not_push.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_multi_terms_autodate_sort_agg_measure_not_push.yaml @@ -9,7 +9,6 @@ calcite: LogicalProject(@timestamp=[$0], category=[$1], value=[$2], _id=[$4], _index=[$5], _score=[$6], _maxscore=[$7], _sort=[$8], _routing=[$9], timestamp=[WIDTH_BUCKET($3, 3, -(MAX($3) OVER (), MIN($3) OVER ()), MAX($3) OVER ())]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]]) physical: | - EnumerableLimit(fetch=[10000]) - EnumerableSort(sort0=[$1], dir0=[ASC-nulls-first]) - EnumerableCalc(expr#0..3=[{inputs}], expr#4=[CAST($t1):DOUBLE], avg(value)=[$t4], cnt=[$t3], category=[$t0], value=[$t1], timestamp=[$t2]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1, 2},cnt=COUNT())], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"category":{"terms":{"field":"category","missing_bucket":false,"order":"asc"}}},{"value":{"terms":{"field":"value","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"timestamp":{"auto_date_histogram":{"field":"timestamp","buckets":3,"minimum_interval":null}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file + EnumerableLimitSort(sort0=[$1], dir0=[ASC-nulls-first], fetch=[10000]) + EnumerableCalc(expr#0..3=[{inputs}], expr#4=[CAST($t1):DOUBLE], avg(value)=[$t4], cnt=[$t3], category=[$t0], value=[$t1], timestamp=[$t2]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1, 2},cnt=COUNT())], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"category":{"terms":{"field":"category","missing_bucket":false,"order":"asc"}}},{"value":{"terms":{"field":"value","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"timestamp":{"auto_date_histogram":{"field":"timestamp","buckets":3,"minimum_interval":null}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file diff --git a/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_range_sort_agg_measure_not_push.yaml b/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_range_sort_agg_measure_not_push.yaml index 10e839ce22c..89e4081d598 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_range_sort_agg_measure_not_push.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_range_sort_agg_measure_not_push.yaml @@ -9,6 +9,5 @@ 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, 'great':VARCHAR)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]]) physical: | - EnumerableLimit(fetch=[10000]) - EnumerableSort(sort0=[$1], dir0=[ASC-nulls-first]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},avg(value)=AVG($2),cnt=COUNT()), PROJECT->[avg(value), cnt, value_range, category]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"category":{"terms":{"field":"category","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"value_range":{"range":{"field":"value","ranges":[{"key":"small","to":7000.0},{"key":"great","from":7000.0}],"keyed":true},"aggregations":{"avg(value)":{"avg":{"field":"value"}}}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + EnumerableLimitSort(sort0=[$1], dir0=[ASC-nulls-first], fetch=[10000]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},avg(value)=AVG($2),cnt=COUNT()), PROJECT->[avg(value), cnt, value_range, category]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"category":{"terms":{"field":"category","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"value_range":{"range":{"field":"value","ranges":[{"key":"small","to":7000.0},{"key":"great","from":7000.0}],"keyed":true},"aggregations":{"avg(value)":{"avg":{"field":"value"}}}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file 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 7550a82cca4..776c52e4da3 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/big5/dedup_metrics_size_field.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/big5/dedup_metrics_size_field.yaml @@ -9,6 +9,5 @@ calcite: LogicalFilter(condition=[IS NOT NULL($28)]) CalciteLogicalIndexScan(table=[[OpenSearch, big5]]) 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","process","log","message","tags","cloud","input","@timestamp","ecs","data_stream","meta","host","metrics","aws","event"],"excludes":[]},"script_fields":{}}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + EnumerableLimitSort(sort0=[$7], dir0=[DESC-nulls-last], fetch=[10000]) + CalciteEnumerableIndexScan(table=[[OpenSearch, big5]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0},agg#0=LITERAL_AGG(1))], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":10000,"sources":[{"metrics.size":{"terms":{"field":"metrics.size","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f1":{"top_hits":{"from":0,"size":1,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["metrics.size","agent","process","log","message","tags","cloud","input","@timestamp","ecs","data_stream","meta","host","metrics","aws","event"],"excludes":[]},"script_fields":{}}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file 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 42f0d27e392..a52d34f2f5c 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 @@ -19,18 +19,17 @@ calcite: LogicalFilter(condition=[AND(IS NOT NULL($4), IS NOT NULL($7))]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]]) 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(balance)=[$t8]) - 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)], gender=[$t0], age=[$t10], avg(balance)=[$t2]) - 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=[[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=[[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)]) + EnumerableLimitSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[10000]) + 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(balance)=[$t8]) + 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)], gender=[$t0], age=[$t10], avg(balance)=[$t2]) + 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=[[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=[[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)]) \ No newline at end of file diff --git a/integ-test/src/test/resources/expectedOutput/calcite/chart_null_str.yaml b/integ-test/src/test/resources/expectedOutput/calcite/chart_null_str.yaml index 70ca93b7ddf..27b54179208 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/chart_null_str.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/chart_null_str.yaml @@ -19,22 +19,21 @@ calcite: LogicalFilter(condition=[AND(IS NOT NULL($4), IS NOT NULL($3))]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank_with_null_values]]) 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(balance)=[$t8]) - EnumerableAggregate(group=[{0, 1}], agg#0=[$SUM0($2)], agg#1=[COUNT($2)]) - EnumerableCalc(expr#0..4=[{inputs}], expr#5=[IS NULL($t1)], expr#6=['nil'], expr#7=[10], expr#8=[<=($t4, $t7)], expr#9=['OTHER'], expr#10=[CASE($t5, $t6, $t8, $t1, $t9)], gender=[$t0], age=[$t10], avg(balance)=[$t2]) - EnumerableMergeJoin(condition=[=($1, $3)], joinType=[left]) - EnumerableSort(sort0=[$1], dir0=[ASC]) - EnumerableCalc(expr#0..3=[{inputs}], expr#4=[SAFE_CAST($t1)], expr#5=[0], expr#6=[=($t3, $t5)], expr#7=[null:BIGINT], expr#8=[CASE($t6, $t7, $t2)], expr#9=[CAST($t8):DOUBLE], expr#10=[/($t9, $t3)], gender=[$t0], age=[$t4], avg(balance)=[$t10]) - EnumerableAggregate(group=[{0, 2}], agg#0=[$SUM0($1)], agg#1=[COUNT($1)]) - EnumerableCalc(expr#0..2=[{inputs}], expr#3=[10], expr#4=[null:NULL], expr#5=[SPAN($t2, $t3, $t4)], gender=[$t1], balance=[$t0], age0=[$t5]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank_with_null_values]], PushDownContext=[[PROJECT->[balance, gender, age], FILTER->AND(IS NOT NULL($1), IS NOT NULL($0))], OpenSearchRequestBuilder(sourceBuilder={"from":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":["balance","gender","age"],"excludes":[]}}, 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..3=[{inputs}], expr#4=[SAFE_CAST($t1)], expr#5=[0], expr#6=[=($t3, $t5)], expr#7=[null:BIGINT], expr#8=[CASE($t6, $t7, $t2)], expr#9=[CAST($t8):DOUBLE], expr#10=[/($t9, $t3)], expr#11=[IS NOT NULL($t4)], age=[$t4], avg(balance)=[$t10], $condition=[$t11]) - EnumerableAggregate(group=[{0, 2}], agg#0=[$SUM0($1)], agg#1=[COUNT($1)]) - EnumerableCalc(expr#0..2=[{inputs}], expr#3=[10], expr#4=[null:NULL], expr#5=[SPAN($t2, $t3, $t4)], gender=[$t1], balance=[$t0], age0=[$t5]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank_with_null_values]], PushDownContext=[[PROJECT->[balance, gender, age], FILTER->AND(IS NOT NULL($1), IS NOT NULL($0))], OpenSearchRequestBuilder(sourceBuilder={"from":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":["balance","gender","age"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + EnumerableLimitSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[10000]) + 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(balance)=[$t8]) + EnumerableAggregate(group=[{0, 1}], agg#0=[$SUM0($2)], agg#1=[COUNT($2)]) + EnumerableCalc(expr#0..4=[{inputs}], expr#5=[IS NULL($t1)], expr#6=['nil'], expr#7=[10], expr#8=[<=($t4, $t7)], expr#9=['OTHER'], expr#10=[CASE($t5, $t6, $t8, $t1, $t9)], gender=[$t0], age=[$t10], avg(balance)=[$t2]) + EnumerableMergeJoin(condition=[=($1, $3)], joinType=[left]) + EnumerableSort(sort0=[$1], dir0=[ASC]) + EnumerableCalc(expr#0..3=[{inputs}], expr#4=[SAFE_CAST($t1)], expr#5=[0], expr#6=[=($t3, $t5)], expr#7=[null:BIGINT], expr#8=[CASE($t6, $t7, $t2)], expr#9=[CAST($t8):DOUBLE], expr#10=[/($t9, $t3)], gender=[$t0], age=[$t4], avg(balance)=[$t10]) + EnumerableAggregate(group=[{0, 2}], agg#0=[$SUM0($1)], agg#1=[COUNT($1)]) + EnumerableCalc(expr#0..2=[{inputs}], expr#3=[10], expr#4=[null:NULL], expr#5=[SPAN($t2, $t3, $t4)], gender=[$t1], balance=[$t0], age0=[$t5]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank_with_null_values]], PushDownContext=[[PROJECT->[balance, gender, age], FILTER->AND(IS NOT NULL($1), IS NOT NULL($0))], OpenSearchRequestBuilder(sourceBuilder={"from":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":["balance","gender","age"],"excludes":[]}}, 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..3=[{inputs}], expr#4=[SAFE_CAST($t1)], expr#5=[0], expr#6=[=($t3, $t5)], expr#7=[null:BIGINT], expr#8=[CASE($t6, $t7, $t2)], expr#9=[CAST($t8):DOUBLE], expr#10=[/($t9, $t3)], expr#11=[IS NOT NULL($t4)], age=[$t4], avg(balance)=[$t10], $condition=[$t11]) + EnumerableAggregate(group=[{0, 2}], agg#0=[$SUM0($1)], agg#1=[COUNT($1)]) + EnumerableCalc(expr#0..2=[{inputs}], expr#3=[10], expr#4=[null:NULL], expr#5=[SPAN($t2, $t3, $t4)], gender=[$t1], balance=[$t0], age0=[$t5]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank_with_null_values]], PushDownContext=[[PROJECT->[balance, gender, age], FILTER->AND(IS NOT NULL($1), IS NOT NULL($0))], OpenSearchRequestBuilder(sourceBuilder={"from":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":["balance","gender","age"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file 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 f3e8cae5ba3..1920d5620cb 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 @@ -19,14 +19,13 @@ calcite: LogicalFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($2))]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]]) physical: | - EnumerableLimit(fetch=[10000]) - EnumerableSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC]) - 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=[[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=[[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)]) + EnumerableLimitSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[10000]) + 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=[[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=[[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)]) \ No newline at end of file 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 953aab7ef47..16eccb1b191 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 @@ -18,14 +18,13 @@ calcite: LogicalFilter(condition=[AND(IS NOT NULL($23), IS NOT NULL($163))]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_otel_logs]]) physical: | - EnumerableLimit(fetch=[10000]) - EnumerableSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC]) - 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=[[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=[[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)]) + EnumerableLimitSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[10000]) + 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=[[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=[[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)]) \ No newline at end of file diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_agg_sort_on_measure_multi_buckets_not_pushed.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_agg_sort_on_measure_multi_buckets_not_pushed.yaml index 7532d318bb2..6707bb4c084 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_agg_sort_on_measure_multi_buckets_not_pushed.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_agg_sort_on_measure_multi_buckets_not_pushed.yaml @@ -8,6 +8,5 @@ calcite: LogicalFilter(condition=[AND(IS NOT NULL($8), IS NOT NULL($7))]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - EnumerableLimit(fetch=[10000]) - EnumerableSort(sort0=[$0], dir0=[ASC-nulls-first]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 2},c=COUNT(),s=SUM($1)), PROJECT->[c, s, span(age,5), 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"}}},{"span(age,5)":{"histogram":{"field":"age","missing_bucket":false,"order":"asc","interval":5.0}}}]},"aggregations":{"s":{"sum":{"field":"balance"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + EnumerableLimitSort(sort0=[$0], dir0=[ASC-nulls-first], fetch=[10000]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 2},c=COUNT(),s=SUM($1)), PROJECT->[c, s, span(age,5), 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"}}},{"span(age,5)":{"histogram":{"field":"age","missing_bucket":false,"order":"asc","interval":5.0}}}]},"aggregations":{"s":{"sum":{"field":"balance"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_exists_correlated_subquery.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_exists_correlated_subquery.yaml index 1ac2e585dd3..d28c948b6ad 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_exists_correlated_subquery.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_exists_correlated_subquery.yaml @@ -12,11 +12,10 @@ calcite: })], variablesSet=[[$cor0]]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_worker]]) physical: | - EnumerableLimit(fetch=[10000]) - EnumerableCalc(expr#0..3=[{inputs}], id=[$t1], name=[$t0], salary=[$t2]) - EnumerableSort(sort0=[$2], dir0=[DESC-nulls-last]) - EnumerableCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_worker]], PushDownContext=[[PROJECT->[name, id, salary]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["name","id","salary"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) - EnumerableAggregate(group=[{0}]) - EnumerableCalc(expr#0=[{inputs}], expr#1=[true], expr#2=[$cor0], expr#3=[$t2.id], expr#4=[=($t3, $t0)], i=[$t1], $condition=[$t4]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_work_information]], PushDownContext=[[PROJECT->[name, uid], FILTER->=($0, 'Tom'), LIMIT->10000, PROJECT->[uid]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10000,"timeout":"1m","query":{"term":{"name":{"value":"Tom","boost":1.0}}},"_source":{"includes":["uid"],"excludes":[]}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) \ No newline at end of file + EnumerableCalc(expr#0..3=[{inputs}], id=[$t1], name=[$t0], salary=[$t2]) + EnumerableLimitSort(sort0=[$2], dir0=[DESC-nulls-last], fetch=[10000]) + EnumerableCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_worker]], PushDownContext=[[PROJECT->[name, id, salary]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["name","id","salary"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + EnumerableAggregate(group=[{0}]) + EnumerableCalc(expr#0=[{inputs}], expr#1=[true], expr#2=[$cor0], expr#3=[$t2.id], expr#4=[=($t3, $t0)], i=[$t1], $condition=[$t4]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_work_information]], PushDownContext=[[PROJECT->[name, uid], FILTER->=($0, 'Tom'), LIMIT->10000, PROJECT->[uid]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10000,"timeout":"1m","query":{"term":{"name":{"value":"Tom","boost":1.0}}},"_source":{"includes":["uid"],"excludes":[]}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) \ No newline at end of file diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_exists_uncorrelated_subquery.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_exists_uncorrelated_subquery.yaml index c8c58c090b8..e6b6774dd6b 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_exists_uncorrelated_subquery.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_exists_uncorrelated_subquery.yaml @@ -11,11 +11,10 @@ calcite: })], variablesSet=[[$cor0]]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_worker]]) physical: | - EnumerableLimit(fetch=[10000]) - EnumerableCalc(expr#0..3=[{inputs}], id=[$t1], name=[$t0], salary=[$t2]) - EnumerableSort(sort0=[$2], dir0=[DESC-nulls-last]) - EnumerableNestedLoopJoin(condition=[true], joinType=[inner]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_worker]], PushDownContext=[[PROJECT->[name, id, salary]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["name","id","salary"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) - EnumerableAggregate(group=[{0}]) - EnumerableCalc(expr#0=[{inputs}], expr#1=[true], i=[$t1]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_work_information]], PushDownContext=[[PROJECT->[name], FILTER->=($0, 'Tom'), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10000,"timeout":"1m","query":{"term":{"name":{"value":"Tom","boost":1.0}}},"_source":{"includes":["name"],"excludes":[]}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) \ No newline at end of file + EnumerableCalc(expr#0..3=[{inputs}], id=[$t1], name=[$t0], salary=[$t2]) + EnumerableLimitSort(sort0=[$2], dir0=[DESC-nulls-last], fetch=[10000]) + EnumerableNestedLoopJoin(condition=[true], joinType=[inner]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_worker]], PushDownContext=[[PROJECT->[name, id, salary]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["name","id","salary"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + EnumerableAggregate(group=[{0}]) + EnumerableCalc(expr#0=[{inputs}], expr#1=[true], i=[$t1]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_work_information]], PushDownContext=[[PROJECT->[name], FILTER->=($0, 'Tom'), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10000,"timeout":"1m","query":{"term":{"name":{"value":"Tom","boost":1.0}}},"_source":{"includes":["name"],"excludes":[]}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) \ No newline at end of file diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_in_correlated_subquery.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_in_correlated_subquery.yaml index 7c9b83b75f5..eca1199768e 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_in_correlated_subquery.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_in_correlated_subquery.yaml @@ -13,11 +13,10 @@ calcite: CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_worker]]) physical: | EnumerableCalc(expr#0..3=[{inputs}], id=[$t1], name=[$t0], salary=[$t2]) - EnumerableLimit(fetch=[10000]) - EnumerableSort(sort0=[$2], dir0=[DESC-nulls-last]) - EnumerableCalc(expr#0..3=[{inputs}], expr#4=[=($t0, $t3)], proj#0..3=[{exprs}], $condition=[$t4]) - EnumerableCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_worker]], PushDownContext=[[PROJECT->[name, id, salary]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["name","id","salary"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) - EnumerableAggregate(group=[{0}]) - EnumerableCalc(expr#0..1=[{inputs}], expr#2=[$cor0], expr#3=[$t2.id], expr#4=[=($t3, $t1)], proj#0..1=[{exprs}], $condition=[$t4]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_work_information]], PushDownContext=[[PROJECT->[name, uid], FILTER->=($0, 'Tom'), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10000,"timeout":"1m","query":{"term":{"name":{"value":"Tom","boost":1.0}}},"_source":{"includes":["name","uid"],"excludes":[]}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) \ No newline at end of file + EnumerableLimitSort(sort0=[$2], dir0=[DESC-nulls-last], fetch=[10000]) + EnumerableCalc(expr#0..3=[{inputs}], expr#4=[=($t0, $t3)], proj#0..3=[{exprs}], $condition=[$t4]) + EnumerableCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_worker]], PushDownContext=[[PROJECT->[name, id, salary]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["name","id","salary"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + EnumerableAggregate(group=[{0}]) + EnumerableCalc(expr#0..1=[{inputs}], expr#2=[$cor0], expr#3=[$t2.id], expr#4=[=($t3, $t1)], proj#0..1=[{exprs}], $condition=[$t4]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_work_information]], PushDownContext=[[PROJECT->[name, uid], FILTER->=($0, 'Tom'), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10000,"timeout":"1m","query":{"term":{"name":{"value":"Tom","boost":1.0}}},"_source":{"includes":["name","uid"],"excludes":[]}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) \ No newline at end of file diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_in_uncorrelated_subquery.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_in_uncorrelated_subquery.yaml index 42de0ae2f46..a8baf16ac46 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_in_uncorrelated_subquery.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_in_uncorrelated_subquery.yaml @@ -10,9 +10,8 @@ calcite: })], variablesSet=[[$cor0]]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_worker]]) physical: | - EnumerableLimit(fetch=[10000]) - EnumerableCalc(expr#0..2=[{inputs}], id=[$t1], name=[$t0], salary=[$t2]) - EnumerableSort(sort0=[$2], dir0=[DESC-nulls-last]) - EnumerableHashJoin(condition=[=($1, $3)], joinType=[semi]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_worker]], PushDownContext=[[PROJECT->[name, id, salary]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["name","id","salary"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_work_information]], PushDownContext=[[PROJECT->[uid], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10000,"timeout":"1m","_source":{"includes":["uid"],"excludes":[]}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) \ No newline at end of file + EnumerableCalc(expr#0..2=[{inputs}], id=[$t1], name=[$t0], salary=[$t2]) + EnumerableLimitSort(sort0=[$2], dir0=[DESC-nulls-last], fetch=[10000]) + EnumerableHashJoin(condition=[=($1, $3)], joinType=[semi]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_worker]], PushDownContext=[[PROJECT->[name, id, salary]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["name","id","salary"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_work_information]], PushDownContext=[[PROJECT->[uid], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10000,"timeout":"1m","_source":{"includes":["uid"],"excludes":[]}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) \ No newline at end of file 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 4d460b3b361..dc918429eba 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 @@ -8,6 +8,5 @@ calcite: LogicalFilter(condition=[IS NOT NULL($7)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) 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={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)]) + EnumerableLimitSort(sort0=[$0], sort1=[$2], dir0=[ASC-nulls-first], dir1=[ASC-nulls-first], fetch=[10000]) + 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)]) \ No newline at end of file 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 de74d7f7f78..8c29e32bab1 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 @@ -8,6 +8,5 @@ calcite: LogicalFilter(condition=[IS NOT NULL($7)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) 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={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)]) + EnumerableLimitSort(sort0=[$0], sort1=[$1], dir0=[ASC-nulls-first], dir1=[ASC-nulls-first], fetch=[10000]) + 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)]) \ No newline at end of file diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_sort_agg_push.json b/integ-test/src/test/resources/expectedOutput/calcite/explain_sort_agg_push.json index 8822d13bd70..d54c254594d 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_sort_agg_push.json +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_sort_agg_push.json @@ -1,6 +1,6 @@ { "calcite": { "logical": "LogicalSystemLimit(sort0=[$0], dir0=[ASC-nulls-first], fetch=[10000], type=[QUERY_SIZE_LIMIT])\n LogicalSort(sort0=[$0], dir0=[ASC-nulls-first])\n LogicalProject(avg_age=[$2], state=[$0], city=[$1])\n LogicalAggregate(group=[{0, 1}], avg_age=[AVG($2)])\n LogicalProject(state=[$7], city=[$5], age=[$8])\n CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]])\n", - "physical": "EnumerableLimit(fetch=[10000])\n EnumerableSort(sort0=[$0], dir0=[ASC-nulls-first])\n CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},avg_age=AVG($2)), PROJECT->[avg_age, state, city]], 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\"}}},{\"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)])\n" + "physical": "EnumerableLimitSort(sort0=[$0], dir0=[ASC-nulls-first], fetch=[10000])\n CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},avg_age=AVG($2)), PROJECT->[avg_age, state, city]], 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\"}}},{\"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)])\n" } } diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_sort_pass_through_join_then_pushdown.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_sort_pass_through_join_then_pushdown.yaml index ae40a6a2de4..15d2907173a 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_sort_pass_through_join_then_pushdown.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_sort_pass_through_join_then_pushdown.yaml @@ -10,8 +10,13 @@ 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]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]]) physical: | - EnumerableLimit(fetch=[10000]) - EnumerableHashJoin(condition=[=($13, $15)], joinType=[left]) + EnumerableLimitSort(sort0=[$13], dir0=[ASC-nulls-first], fetch=[10000]) + EnumerableMergeJoin(condition=[=($13, $15)], joinType=[left]) EnumerableCalc(expr#0..12=[{inputs}], expr#13=['(?^[A-Z])'], expr#14=['initial'], expr#15=[REX_EXTRACT($t6, $t13, $t14)], proj#0..12=[{exprs}], $f13=[$t15]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[PROJECT->[account_number, firstname, address, birthdate, gender, city, lastname, balance, employer, state, age, email, male], SORT_EXPR->[REX_EXTRACT($6, '(?^[A-Z])', 'initial') ASCENDING NULLS_FIRST]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["account_number","firstname","address","birthdate","gender","city","lastname","balance","employer","state","age","email","male"],"excludes":[]},"sort":[{"_script":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQC63sKICAib3AiOiB7CiAgICAibmFtZSI6ICJSRVhfRVhUUkFDVCIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfSwKICAgIHsKICAgICAgImR5bmFtaWNQYXJhbSI6IDEsCiAgICAgICJ0eXBlIjogewogICAgICAgICJ0eXBlIjogIlZBUkNIQVIiLAogICAgICAgICJudWxsYWJsZSI6IHRydWUsCiAgICAgICAgInByZWNpc2lvbiI6IC0xCiAgICAgIH0KICAgIH0sCiAgICB7CiAgICAgICJkeW5hbWljUGFyYW0iOiAyLAogICAgICAidHlwZSI6IHsKICAgICAgICAidHlwZSI6ICJWQVJDSEFSIiwKICAgICAgICAibnVsbGFibGUiOiB0cnVlLAogICAgICAgICJwcmVjaXNpb24iOiAtMQogICAgICB9CiAgICB9CiAgXSwKICAiY2xhc3MiOiAib3JnLm9wZW5zZWFyY2guc3FsLmV4cHJlc3Npb24uZnVuY3Rpb24uVXNlckRlZmluZWRGdW5jdGlvbkJ1aWxkZXIkMSIsCiAgInR5cGUiOiB7CiAgICAidHlwZSI6ICJWQVJDSEFSIiwKICAgICJudWxsYWJsZSI6IHRydWUsCiAgICAicHJlY2lzaW9uIjogMjAwMAogIH0sCiAgImRldGVybWluaXN0aWMiOiB0cnVlLAogICJkeW5hbWljIjogZmFsc2UKfQ==\"}","lang":"opensearch_compounded_script","params":{"MISSING_MAX":false,"utcTimestamp": 0,"SOURCES":[0,2,2],"DIGESTS":["lastname","(?^[A-Z])","initial"]}},"type":"string","order":"asc"}}]}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[PROJECT->[account_number, firstname, address, birthdate, gender, city, lastname, balance, employer, state, age, email, male], LIMIT->50000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":50000,"timeout":"1m","_source":{"includes":["account_number","firstname","address","birthdate","gender","city","lastname","balance","employer","state","age","email","male"],"excludes":[]}}, requestedTotalSize=50000, pageSize=null, startFrom=0)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[PROJECT->[account_number, firstname, address, birthdate, gender, city, lastname, balance, employer, state, age, email, male], SORT_EXPR->[REX_EXTRACT($6, '(?^[A-Z])', 'initial') ASCENDING NULLS_LAST]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["account_number","firstname","address","birthdate","gender","city","lastname","balance","employer","state","age","email","male"],"excludes":[]},"sort":[{"_script":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQC63sKICAib3AiOiB7CiAgICAibmFtZSI6ICJSRVhfRVhUUkFDVCIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfSwKICAgIHsKICAgICAgImR5bmFtaWNQYXJhbSI6IDEsCiAgICAgICJ0eXBlIjogewogICAgICAgICJ0eXBlIjogIlZBUkNIQVIiLAogICAgICAgICJudWxsYWJsZSI6IHRydWUsCiAgICAgICAgInByZWNpc2lvbiI6IC0xCiAgICAgIH0KICAgIH0sCiAgICB7CiAgICAgICJkeW5hbWljUGFyYW0iOiAyLAogICAgICAidHlwZSI6IHsKICAgICAgICAidHlwZSI6ICJWQVJDSEFSIiwKICAgICAgICAibnVsbGFibGUiOiB0cnVlLAogICAgICAgICJwcmVjaXNpb24iOiAtMQogICAgICB9CiAgICB9CiAgXSwKICAiY2xhc3MiOiAib3JnLm9wZW5zZWFyY2guc3FsLmV4cHJlc3Npb24uZnVuY3Rpb24uVXNlckRlZmluZWRGdW5jdGlvbkJ1aWxkZXIkMSIsCiAgInR5cGUiOiB7CiAgICAidHlwZSI6ICJWQVJDSEFSIiwKICAgICJudWxsYWJsZSI6IHRydWUsCiAgICAicHJlY2lzaW9uIjogMjAwMAogIH0sCiAgImRldGVybWluaXN0aWMiOiB0cnVlLAogICJkeW5hbWljIjogZmFsc2UKfQ==\"}","lang":"opensearch_compounded_script","params":{"MISSING_MAX":true,"utcTimestamp": 0,"SOURCES":[0,2,2],"DIGESTS":["lastname","(?^[A-Z])","initial"]}},"type":"string","order":"asc"}}]}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[PROJECT->[account_number, firstname, address, birthdate, gender, city, lastname, balance, employer, state, age, email, male], LIMIT->50000, SORT->[{ + "firstname" : { + "order" : "asc", + "missing" : "_last" + } + }]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":50000,"timeout":"1m","_source":{"includes":["account_number","firstname","address","birthdate","gender","city","lastname","balance","employer","state","age","email","male"],"excludes":[]},"sort":[{"firstname":{"order":"asc","missing":"_last"}}]}, requestedTotalSize=50000, pageSize=null, startFrom=0)]) \ No newline at end of file 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 9d16a625c0a..ec04bb50c28 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_timechart.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_timechart.yaml @@ -19,22 +19,21 @@ calcite: LogicalFilter(condition=[AND(IS NOT NULL($1), IS NOT NULL($7))]) CalciteLogicalIndexScan(table=[[OpenSearch, events]]) physical: | - EnumerableLimit(fetch=[10000]) + EnumerableLimitSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[10000]) 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]) - EnumerableSort(sort0=[$1], dir0=[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)], @timestamp=[$t1], 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))], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","query":{"bool":{"must":[{"exists":{"field":"@timestamp","boost":1.0}},{"exists":{"field":"cpu_usage","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"_source":{"includes":["@timestamp","host","cpu_usage"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) - 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=[{0}], __grand_total__=[SUM($1)]) - EnumerableCalc(expr#0..3=[{inputs}], expr#4=[0], expr#5=[=($t3, $t4)], expr#6=[null:DOUBLE], expr#7=[CASE($t5, $t6, $t2)], expr#8=[/($t7, $t3)], host=[$t0], avg(cpu_usage)=[$t8]) - EnumerableAggregate(group=[{0, 2}], agg#0=[$SUM0($1)], agg#1=[COUNT($1)]) - EnumerableCalc(expr#0..2=[{inputs}], expr#3=[1], expr#4=['m'], expr#5=[SPAN($t2, $t3, $t4)], proj#0..1=[{exprs}], $f2=[$t5]) - CalciteEnumerableIndexScan(table=[[OpenSearch, events]], PushDownContext=[[PROJECT->[@timestamp, host, cpu_usage], FILTER->AND(IS NOT NULL($0), IS NOT NULL($2)), PROJECT->[host, cpu_usage, @timestamp], FILTER->IS NOT NULL($0)], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","query":{"bool":{"filter":[{"bool":{"must":[{"exists":{"field":"@timestamp","boost":1.0}},{"exists":{"field":"cpu_usage","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},{"exists":{"field":"host","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"_source":{"includes":["host","cpu_usage","@timestamp"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + 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]) + EnumerableSort(sort0=[$1], dir0=[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)], @timestamp=[$t1], 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))], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","query":{"bool":{"must":[{"exists":{"field":"@timestamp","boost":1.0}},{"exists":{"field":"cpu_usage","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"_source":{"includes":["@timestamp","host","cpu_usage"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + 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=[{0}], __grand_total__=[SUM($1)]) + EnumerableCalc(expr#0..3=[{inputs}], expr#4=[0], expr#5=[=($t3, $t4)], expr#6=[null:DOUBLE], expr#7=[CASE($t5, $t6, $t2)], expr#8=[/($t7, $t3)], host=[$t0], avg(cpu_usage)=[$t8]) + EnumerableAggregate(group=[{0, 2}], agg#0=[$SUM0($1)], agg#1=[COUNT($1)]) + EnumerableCalc(expr#0..2=[{inputs}], expr#3=[1], expr#4=['m'], expr#5=[SPAN($t2, $t3, $t4)], proj#0..1=[{exprs}], $f2=[$t5]) + CalciteEnumerableIndexScan(table=[[OpenSearch, events]], PushDownContext=[[PROJECT->[@timestamp, host, cpu_usage], FILTER->AND(IS NOT NULL($0), IS NOT NULL($2)), PROJECT->[host, cpu_usage, @timestamp], FILTER->IS NOT NULL($0)], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","query":{"bool":{"filter":[{"bool":{"must":[{"exists":{"field":"@timestamp","boost":1.0}},{"exists":{"field":"cpu_usage","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},{"exists":{"field":"host","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"_source":{"includes":["host","cpu_usage","@timestamp"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file 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 65f5d4f26b7..4f8267f3117 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 @@ -19,18 +19,17 @@ calcite: LogicalFilter(condition=[IS NOT NULL($1)]) CalciteLogicalIndexScan(table=[[OpenSearch, events]]) physical: | - EnumerableLimit(fetch=[10000]) - EnumerableSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC]) - EnumerableAggregate(group=[{0, 1}], count()=[$SUM0($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], count()=[$t2]) - EnumerableMergeJoin(condition=[=($1, $3)], joinType=[left]) - EnumerableSort(sort0=[$1], dir0=[ASC]) - EnumerableCalc(expr#0..2=[{inputs}], @timestamp=[$t1], host=[$t0], count()=[$t2]) - EnumerableAggregate(group=[{0, 1}], count()=[COUNT()]) - EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], expr#3=['m'], expr#4=[SPAN($t0, $t2, $t3)], host=[$t1], @timestamp0=[$t4]) - CalciteEnumerableIndexScan(table=[[OpenSearch, events]], PushDownContext=[[PROJECT->[@timestamp, host], FILTER->IS NOT NULL($0)], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","query":{"exists":{"field":"@timestamp","boost":1.0}},"_source":{"includes":["@timestamp","host"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) - 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=[{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)]) + EnumerableLimitSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[10000]) + EnumerableAggregate(group=[{0, 1}], count()=[$SUM0($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], count()=[$t2]) + EnumerableMergeJoin(condition=[=($1, $3)], joinType=[left]) + EnumerableSort(sort0=[$1], dir0=[ASC]) + EnumerableCalc(expr#0..2=[{inputs}], @timestamp=[$t1], host=[$t0], count()=[$t2]) + EnumerableAggregate(group=[{0, 1}], count()=[COUNT()]) + EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], expr#3=['m'], expr#4=[SPAN($t0, $t2, $t3)], host=[$t1], @timestamp0=[$t4]) + CalciteEnumerableIndexScan(table=[[OpenSearch, events]], PushDownContext=[[PROJECT->[@timestamp, host], FILTER->IS NOT NULL($0)], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","query":{"exists":{"field":"@timestamp","boost":1.0}},"_source":{"includes":["@timestamp","host"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) + 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=[{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)]) \ No newline at end of file diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_top_k_then_sort_push.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_top_k_then_sort_push.yaml index 2b48af8c89a..9c0a714771b 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_top_k_then_sort_push.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_top_k_then_sort_push.yaml @@ -7,9 +7,8 @@ calcite: CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | EnumerableCalc(expr#0..1=[{inputs}], age=[$t1]) - EnumerableLimit(fetch=[10000]) - EnumerableSort(sort0=[$1], dir0=[ASC-nulls-first]) - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[balance, age], SORT->[{ + EnumerableLimitSort(sort0=[$1], dir0=[ASC-nulls-first], fetch=[10000]) + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[balance, age], SORT->[{ "balance" : { "order" : "asc", "missing" : "_first" diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/EnumerableTopKRule.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/EnumerableTopKConverterRule.java similarity index 81% rename from opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/EnumerableTopKRule.java rename to opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/EnumerableTopKConverterRule.java index 6edeb0660a7..4fe2ff50c27 100644 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/EnumerableTopKRule.java +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/EnumerableTopKConverterRule.java @@ -20,8 +20,9 @@ * CalciteEnumerableTopK}. */ @Value.Enclosing -public class EnumerableTopKRule extends InterruptibleRelRule { - protected EnumerableTopKRule(EnumerableTopKRule.Config config) { +public class EnumerableTopKConverterRule + extends InterruptibleRelRule { + protected EnumerableTopKConverterRule(EnumerableTopKConverterRule.Config config) { super(config); } @@ -44,8 +45,8 @@ protected void onMatchImpl(RelOptRuleCall call) { /** Rule configuration. */ @Value.Immutable public interface Config extends OpenSearchRuleConfig { - EnumerableTopKRule.Config DEFAULT = - ImmutableEnumerableTopKRule.Config.builder() + EnumerableTopKConverterRule.Config DEFAULT = + ImmutableEnumerableTopKConverterRule.Config.builder() .build() .withDescription("EnumerableTopK") .withOperandSupplier( @@ -58,8 +59,8 @@ public interface Config extends OpenSearchRuleConfig { .anyInputs()); @Override - default EnumerableTopKRule toRule() { - return new EnumerableTopKRule(this); + default EnumerableTopKConverterRule toRule() { + return new EnumerableTopKConverterRule(this); } } } diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/EnumerableTopKMergeRule.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/EnumerableTopKMergeRule.java new file mode 100644 index 00000000000..e0d034fb5ae --- /dev/null +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/EnumerableTopKMergeRule.java @@ -0,0 +1,56 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.opensearch.planner.rules; + +import org.apache.calcite.adapter.enumerable.EnumerableLimit; +import org.apache.calcite.adapter.enumerable.EnumerableSort; +import org.apache.calcite.plan.RelOptRuleCall; +import org.immutables.value.Value; +import org.opensearch.sql.calcite.plan.OpenSearchRuleConfig; +import org.opensearch.sql.opensearch.planner.physical.CalciteEnumerableTopK; + +/** + * Rule to merge an {@link EnumerableLimit} and an ({@link EnumerableSort} into an {@link + * CalciteEnumerableTopK}. + */ +@Value.Enclosing +public class EnumerableTopKMergeRule extends InterruptibleRelRule { + protected EnumerableTopKMergeRule(EnumerableTopKMergeRule.Config config) { + super(config); + } + + @Override + protected void onMatchImpl(RelOptRuleCall call) { + final EnumerableLimit limit = call.rel(0); + final EnumerableSort sort = call.rel(1); + final CalciteEnumerableTopK topK = + CalciteEnumerableTopK.create( + sort.getInput(), sort.getCollation(), limit.offset, limit.fetch); + call.transformTo(topK); + } + + /** Rule configuration. */ + @Value.Immutable + public interface Config extends OpenSearchRuleConfig { + EnumerableTopKMergeRule.Config DEFAULT = + ImmutableEnumerableTopKMergeRule.Config.builder() + .build() + .withDescription("EnumerableTopKMerge") + .withOperandSupplier( + b0 -> + b0.operand(EnumerableLimit.class) + .oneInput( + b1 -> + b1.operand(EnumerableSort.class) + .predicate(sort -> sort.fetch == null) + .anyInputs())); + + @Override + default EnumerableTopKMergeRule toRule() { + return new EnumerableTopKMergeRule(this); + } + } +} 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 a8a46ed8378..5f4fecc6b65 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 @@ -39,8 +39,10 @@ public class OpenSearchIndexRules { RareTopPushdownRule.Config.DEFAULT.toRule(); private static final SortExprIndexScanRule SORT_EXPR_INDEX_SCAN = SortExprIndexScanRule.Config.DEFAULT.toRule(); - private static final EnumerableTopKRule ENUMERABLE_TOP_K_RULE = - EnumerableTopKRule.Config.DEFAULT.toRule(); + private static final EnumerableTopKConverterRule ENUMERABLE_TOP_K_RULE = + EnumerableTopKConverterRule.Config.DEFAULT.toRule(); + private static final EnumerableTopKMergeRule ENUMERABLE_TOP_K_MERGE_RULE = + EnumerableTopKMergeRule.Config.DEFAULT.toRule(); // Rule that always pushes down relevance functions regardless of pushdown settings public static final RelevanceFunctionPushdownRule RELEVANCE_FUNCTION_PUSHDOWN = @@ -61,6 +63,7 @@ public class OpenSearchIndexRules { SORT_AGGREGATION_METRICS_RULE, RARE_TOP_PUSH_DOWN, ENUMERABLE_TOP_K_RULE, + ENUMERABLE_TOP_K_MERGE_RULE, EXPAND_COLLATION_ON_PROJECT_EXPR, SORT_EXPR_INDEX_SCAN); From 940fc4d69d3eb6d5d79324fd5b22681f65be23e8 Mon Sep 17 00:00:00 2001 From: Lantao Jin Date: Fri, 26 Dec 2025 17:29:31 +0800 Subject: [PATCH 3/4] remove the converter rule Signed-off-by: Lantao Jin --- .../rules/EnumerableTopKConverterRule.java | 66 ------------------- .../planner/rules/OpenSearchIndexRules.java | 3 - 2 files changed, 69 deletions(-) delete mode 100644 opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/EnumerableTopKConverterRule.java diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/EnumerableTopKConverterRule.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/EnumerableTopKConverterRule.java deleted file mode 100644 index 4fe2ff50c27..00000000000 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/rules/EnumerableTopKConverterRule.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ - -package org.opensearch.sql.opensearch.planner.rules; - -import org.apache.calcite.adapter.enumerable.EnumerableConvention; -import org.apache.calcite.plan.RelOptRuleCall; -import org.apache.calcite.rel.RelNode; -import org.apache.calcite.rel.core.Sort; -import org.apache.calcite.rel.logical.LogicalSort; -import org.immutables.value.Value; -import org.opensearch.sql.calcite.plan.OpenSearchRuleConfig; -import org.opensearch.sql.opensearch.planner.physical.CalciteEnumerableTopK; - -/** - * Rule to convert an {@link LogicalSort} with ({@link LogicalSort#fetch} or {@link - * LogicalSort#offset}) and {@link LogicalSort#collation}(Order By) into an {@link - * CalciteEnumerableTopK}. - */ -@Value.Enclosing -public class EnumerableTopKConverterRule - extends InterruptibleRelRule { - protected EnumerableTopKConverterRule(EnumerableTopKConverterRule.Config config) { - super(config); - } - - @Override - protected void onMatchImpl(RelOptRuleCall call) { - final Sort sort = call.rel(0); - RelNode input = sort.getInput(); - final Sort topK = - CalciteEnumerableTopK.create( - convert( - call.getPlanner(), - input, - input.getTraitSet().replace(EnumerableConvention.INSTANCE)), - sort.getCollation(), - sort.offset, - sort.fetch); - call.transformTo(topK); - } - - /** Rule configuration. */ - @Value.Immutable - public interface Config extends OpenSearchRuleConfig { - EnumerableTopKConverterRule.Config DEFAULT = - ImmutableEnumerableTopKConverterRule.Config.builder() - .build() - .withDescription("EnumerableTopK") - .withOperandSupplier( - b0 -> - b0.operand(LogicalSort.class) - .predicate( - sort -> - (sort.fetch != null || sort.offset != null) - && !sort.collation.getFieldCollations().isEmpty()) - .anyInputs()); - - @Override - default EnumerableTopKConverterRule toRule() { - return new EnumerableTopKConverterRule(this); - } - } -} 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 5f4fecc6b65..0fa726072f0 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 @@ -39,8 +39,6 @@ public class OpenSearchIndexRules { RareTopPushdownRule.Config.DEFAULT.toRule(); private static final SortExprIndexScanRule SORT_EXPR_INDEX_SCAN = SortExprIndexScanRule.Config.DEFAULT.toRule(); - private static final EnumerableTopKConverterRule ENUMERABLE_TOP_K_RULE = - EnumerableTopKConverterRule.Config.DEFAULT.toRule(); private static final EnumerableTopKMergeRule ENUMERABLE_TOP_K_MERGE_RULE = EnumerableTopKMergeRule.Config.DEFAULT.toRule(); @@ -62,7 +60,6 @@ public class OpenSearchIndexRules { SORT_PROJECT_EXPR_TRANSPOSE, SORT_AGGREGATION_METRICS_RULE, RARE_TOP_PUSH_DOWN, - ENUMERABLE_TOP_K_RULE, ENUMERABLE_TOP_K_MERGE_RULE, EXPAND_COLLATION_ON_PROJECT_EXPR, SORT_EXPR_INDEX_SCAN); From 65d062ab3720c8b581ce0ff9566c650fab8c7d97 Mon Sep 17 00:00:00 2001 From: Lantao Jin Date: Mon, 29 Dec 2025 13:29:08 +0800 Subject: [PATCH 4/4] address commments Signed-off-by: Lantao Jin --- ...ange_metric_sort_agg_measure_not_push.yaml | 2 +- ...te_autodate_sort_agg_measure_not_push.yaml | 2 +- ...ms_autodate_sort_agg_measure_not_push.yaml | 2 +- ...osite_range_sort_agg_measure_not_push.yaml | 2 +- .../big5/dedup_metrics_size_field.yaml | 2 +- .../calcite/chart_multiple_group_keys.yaml | 2 +- .../calcite/chart_null_str.yaml | 2 +- .../chart_timestamp_span_and_category.yaml | 2 +- .../calcite/chart_use_other.yaml | 2 +- .../calcite/clickbench/q28.yaml | 2 +- .../calcite/clickbench/q29.yaml | 2 +- ...t_on_measure_multi_buckets_not_pushed.yaml | 2 +- .../calcite/explain_dedup_with_expr4.yaml | 2 +- .../explain_dedup_with_expr4_alternative.yaml | 2 +- .../explain_exists_correlated_subquery.yaml | 2 +- .../explain_exists_uncorrelated_subquery.yaml | 2 +- .../explain_in_correlated_subquery.yaml | 2 +- .../explain_in_uncorrelated_subquery.yaml | 2 +- .../calcite/explain_limit_agg_pushdown5.json | 2 +- ...gg_with_sort_on_one_measure_not_push1.yaml | 2 +- ...gg_with_sort_on_one_measure_not_push2.yaml | 2 +- .../calcite/explain_sort_agg_push.json | 2 +- ..._sort_pass_through_join_then_pushdown.yaml | 2 +- .../explain_streamstats_distinct_count.yaml | 2 +- .../explain_streamstats_earliest_latest.yaml | 2 +- ...reamstats_earliest_latest_custom_time.yaml | 2 +- ...xplain_streamstats_global_null_bucket.yaml | 2 +- .../explain_streamstats_null_bucket.yaml | 2 +- ...explain_streamstats_reset_null_bucket.yaml | 2 +- .../calcite/explain_timechart.yaml | 2 +- .../calcite/explain_timechart_count.yaml | 2 +- .../calcite/explain_top_k_then_sort_push.yaml | 2 +- .../physical/CalciteEnumerableTopK.java | 20 ++++++++++++++----- .../planner/rules/SortIndexScanRule.java | 4 +++- 34 files changed, 50 insertions(+), 38 deletions(-) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_autodate_range_metric_sort_agg_measure_not_push.yaml b/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_autodate_range_metric_sort_agg_measure_not_push.yaml index 137687d1015..c3095707831 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_autodate_range_metric_sort_agg_measure_not_push.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_autodate_range_metric_sort_agg_measure_not_push.yaml @@ -9,5 +9,5 @@ calcite: LogicalProject(@timestamp=[$0], category=[$1], value=[$2], _id=[$4], _index=[$5], _score=[$6], _maxscore=[$7], _sort=[$8], _routing=[$9], timestamp=[WIDTH_BUCKET($3, 3, -(MAX($3) OVER (), MIN($3) OVER ()), MAX($3) OVER ())], value_range=[CASE(<($2, 7000), 'small':VARCHAR, 'great':VARCHAR)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]]) physical: | - EnumerableLimitSort(sort0=[$1], dir0=[ASC-nulls-first], fetch=[10000]) + CalciteEnumerableTopK(sort0=[$1], dir0=[ASC-nulls-first], fetch=[10000]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1, 2},avg(value)=AVG($3),cnt=COUNT()), PROJECT->[avg(value), cnt, timestamp, value_range, category]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"category":{"terms":{"field":"category","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"timestamp":{"auto_date_histogram":{"field":"timestamp","buckets":3,"minimum_interval":null},"aggregations":{"value_range":{"range":{"field":"value","ranges":[{"key":"small","to":7000.0},{"key":"great","from":7000.0}],"keyed":true},"aggregations":{"avg(value)":{"avg":{"field":"value"}}}}}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file diff --git a/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_autodate_sort_agg_measure_not_push.yaml b/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_autodate_sort_agg_measure_not_push.yaml index 792e689f4f5..594fdbdde56 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_autodate_sort_agg_measure_not_push.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_autodate_sort_agg_measure_not_push.yaml @@ -9,5 +9,5 @@ calcite: LogicalProject(@timestamp=[$0], category=[$1], value=[$2], _id=[$4], _index=[$5], _score=[$6], _maxscore=[$7], _sort=[$8], _routing=[$9], timestamp=[WIDTH_BUCKET($3, 3, -(MAX($3) OVER (), MIN($3) OVER ()), MAX($3) OVER ())]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]]) physical: | - EnumerableLimitSort(sort0=[$1], dir0=[ASC-nulls-first], fetch=[10000]) + CalciteEnumerableTopK(sort0=[$1], dir0=[ASC-nulls-first], fetch=[10000]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},avg(value)=AVG($2),cnt=COUNT()), PROJECT->[avg(value), cnt, timestamp, category]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"category":{"terms":{"field":"category","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"timestamp":{"auto_date_histogram":{"field":"timestamp","buckets":3,"minimum_interval":null},"aggregations":{"avg(value)":{"avg":{"field":"value"}}}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file diff --git a/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_multi_terms_autodate_sort_agg_measure_not_push.yaml b/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_multi_terms_autodate_sort_agg_measure_not_push.yaml index d3c4f0a1ee2..3716b17dd01 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_multi_terms_autodate_sort_agg_measure_not_push.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_multi_terms_autodate_sort_agg_measure_not_push.yaml @@ -9,6 +9,6 @@ calcite: LogicalProject(@timestamp=[$0], category=[$1], value=[$2], _id=[$4], _index=[$5], _score=[$6], _maxscore=[$7], _sort=[$8], _routing=[$9], timestamp=[WIDTH_BUCKET($3, 3, -(MAX($3) OVER (), MIN($3) OVER ()), MAX($3) OVER ())]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]]) physical: | - EnumerableLimitSort(sort0=[$1], dir0=[ASC-nulls-first], fetch=[10000]) + CalciteEnumerableTopK(sort0=[$1], dir0=[ASC-nulls-first], fetch=[10000]) EnumerableCalc(expr#0..3=[{inputs}], expr#4=[CAST($t1):DOUBLE], avg(value)=[$t4], cnt=[$t3], category=[$t0], value=[$t1], timestamp=[$t2]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1, 2},cnt=COUNT())], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"category":{"terms":{"field":"category","missing_bucket":false,"order":"asc"}}},{"value":{"terms":{"field":"value","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"timestamp":{"auto_date_histogram":{"field":"timestamp","buckets":3,"minimum_interval":null}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file diff --git a/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_range_sort_agg_measure_not_push.yaml b/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_range_sort_agg_measure_not_push.yaml index 89e4081d598..a9eae9a2c4a 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_range_sort_agg_measure_not_push.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/agg_composite_range_sort_agg_measure_not_push.yaml @@ -9,5 +9,5 @@ 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, 'great':VARCHAR)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]]) physical: | - EnumerableLimitSort(sort0=[$1], dir0=[ASC-nulls-first], fetch=[10000]) + CalciteEnumerableTopK(sort0=[$1], dir0=[ASC-nulls-first], fetch=[10000]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},avg(value)=AVG($2),cnt=COUNT()), PROJECT->[avg(value), cnt, value_range, category]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"category":{"terms":{"field":"category","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"value_range":{"range":{"field":"value","ranges":[{"key":"small","to":7000.0},{"key":"great","from":7000.0}],"keyed":true},"aggregations":{"avg(value)":{"avg":{"field":"value"}}}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file 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 776c52e4da3..4387e72fd96 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/big5/dedup_metrics_size_field.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/big5/dedup_metrics_size_field.yaml @@ -9,5 +9,5 @@ calcite: LogicalFilter(condition=[IS NOT NULL($28)]) CalciteLogicalIndexScan(table=[[OpenSearch, big5]]) physical: | - EnumerableLimitSort(sort0=[$7], dir0=[DESC-nulls-last], fetch=[10000]) + CalciteEnumerableTopK(sort0=[$7], dir0=[DESC-nulls-last], fetch=[10000]) CalciteEnumerableIndexScan(table=[[OpenSearch, big5]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0},agg#0=LITERAL_AGG(1))], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":10000,"sources":[{"metrics.size":{"terms":{"field":"metrics.size","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f1":{"top_hits":{"from":0,"size":1,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["metrics.size","agent","process","log","message","tags","cloud","input","@timestamp","ecs","data_stream","meta","host","metrics","aws","event"],"excludes":[]},"script_fields":{}}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file 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 a52d34f2f5c..2f539057b66 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 @@ -19,7 +19,7 @@ calcite: LogicalFilter(condition=[AND(IS NOT NULL($4), IS NOT NULL($7))]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]]) physical: | - EnumerableLimitSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[10000]) + CalciteEnumerableTopK(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[10000]) 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(balance)=[$t8]) 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)], gender=[$t0], age=[$t10], avg(balance)=[$t2]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/chart_null_str.yaml b/integ-test/src/test/resources/expectedOutput/calcite/chart_null_str.yaml index 27b54179208..df5a461ab85 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/chart_null_str.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/chart_null_str.yaml @@ -19,7 +19,7 @@ calcite: LogicalFilter(condition=[AND(IS NOT NULL($4), IS NOT NULL($3))]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank_with_null_values]]) physical: | - EnumerableLimitSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[10000]) + CalciteEnumerableTopK(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[10000]) 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(balance)=[$t8]) EnumerableAggregate(group=[{0, 1}], agg#0=[$SUM0($2)], agg#1=[COUNT($2)]) EnumerableCalc(expr#0..4=[{inputs}], expr#5=[IS NULL($t1)], expr#6=['nil'], expr#7=[10], expr#8=[<=($t4, $t7)], expr#9=['OTHER'], expr#10=[CASE($t5, $t6, $t8, $t1, $t9)], gender=[$t0], age=[$t10], avg(balance)=[$t2]) 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 1920d5620cb..9267e6faab1 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 @@ -19,7 +19,7 @@ calcite: LogicalFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($2))]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_time_data]]) physical: | - EnumerableLimitSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[10000]) + CalciteEnumerableTopK(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[10000]) 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]) 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 16eccb1b191..8b9f72596b1 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 @@ -18,7 +18,7 @@ calcite: LogicalFilter(condition=[AND(IS NOT NULL($23), IS NOT NULL($163))]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_otel_logs]]) physical: | - EnumerableLimitSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[10000]) + CalciteEnumerableTopK(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[10000]) 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]) 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 f1876e40a2d..3b02510db8b 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q28.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q28.yaml @@ -10,6 +10,6 @@ calcite: CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | EnumerableLimit(fetch=[10000]) - EnumerableLimitSort(sort0=[$0], dir0=[DESC-nulls-last], fetch=[25]) + CalciteEnumerableTopK(sort0=[$0], dir0=[DESC-nulls-last], fetch=[25]) EnumerableCalc(expr#0..2=[{inputs}], expr#3=[100000], expr#4=[>($t1, $t3)], proj#0..2=[{exprs}], $condition=[$t4]) CalciteEnumerableIndexScan(table=[[OpenSearch, hits]], PushDownContext=[[FILTER->AND(<>($0, ''), IS NOT NULL($1)), AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},l=AVG($1),c=COUNT()), PROJECT->[l, c, CounterID]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","query":{"bool":{"must":[{"bool":{"must":[{"exists":{"field":"URL","boost":1.0}}],"must_not":[{"term":{"URL":{"value":"","boost":1.0}}}],"adjust_pure_negative":true,"boost":1.0}},{"exists":{"field":"CounterID","boost":1.0}}],"adjust_pure_negative":true,"boost":1.0}},"aggregations":{"composite_buckets":{"composite":{"size":10000,"sources":[{"CounterID":{"terms":{"field":"CounterID","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"l":{"avg":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/3sKICAib3AiOiB7CiAgICAibmFtZSI6ICJDSEFSX0xFTkdUSCIsCiAgICAia2luZCI6ICJDSEFSX0xFTkdUSCIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["URL"]}}}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file 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 99ee71e7ccc..c5d5092d7cf 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q29.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/clickbench/q29.yaml @@ -12,6 +12,6 @@ calcite: CalciteLogicalIndexScan(table=[[OpenSearch, hits]]) physical: | EnumerableLimit(fetch=[10000]) - EnumerableLimitSort(sort0=[$0], dir0=[DESC-nulls-last], fetch=[25]) + CalciteEnumerableTopK(sort0=[$0], dir0=[DESC-nulls-last], fetch=[25]) EnumerableCalc(expr#0..3=[{inputs}], expr#4=[100000], expr#5=[>($t1, $t4)], proj#0..3=[{exprs}], $condition=[$t5]) 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)]) \ No newline at end of file diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_agg_sort_on_measure_multi_buckets_not_pushed.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_agg_sort_on_measure_multi_buckets_not_pushed.yaml index 6707bb4c084..68cb12a49dd 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_agg_sort_on_measure_multi_buckets_not_pushed.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_agg_sort_on_measure_multi_buckets_not_pushed.yaml @@ -8,5 +8,5 @@ calcite: LogicalFilter(condition=[AND(IS NOT NULL($8), IS NOT NULL($7))]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - EnumerableLimitSort(sort0=[$0], dir0=[ASC-nulls-first], fetch=[10000]) + CalciteEnumerableTopK(sort0=[$0], dir0=[ASC-nulls-first], fetch=[10000]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 2},c=COUNT(),s=SUM($1)), PROJECT->[c, s, span(age,5), 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"}}},{"span(age,5)":{"histogram":{"field":"age","missing_bucket":false,"order":"asc","interval":5.0}}}]},"aggregations":{"s":{"sum":{"field":"balance"}}}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file 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 cb796a4dd38..0767a8357e1 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_expr4.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_expr4.yaml @@ -9,4 +9,4 @@ calcite: LogicalProject(account_number=[$0], gender=[$4], age=[$8], state=[$7], new_gender=[LOWER($4)], new_state=[LOWER($7)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(2)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"gender":{"terms":{"field":"gender.keyword","missing_bucket":false,"order":"asc"}}},{"state":{"terms":{"field":"state.keyword","missing_bucket":false,"order":"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=10000, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(2)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"gender":{"terms":{"field":"gender.keyword","missing_bucket":false,"order":"asc"}}},{"state":{"terms":{"field":"state.keyword","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":2,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["gender","state","account_number","age"],"excludes":[]},"script_fields":{"new_state":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"ignore_failure":false},"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) \ No newline at end of file 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 8f39b025c9a..4527da4f140 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_expr4_alternative.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_dedup_with_expr4_alternative.yaml @@ -9,4 +9,4 @@ calcite: LogicalProject(account_number=[$0], gender=[$4], age=[$8], state=[$7], new_gender=[LOWER($4)], new_state=[LOWER($7)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(2)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"gender":{"terms":{"field":"gender.keyword","missing_bucket":false,"order":"asc"}}},{"state":{"terms":{"field":"state.keyword","missing_bucket":false,"order":"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=10000, pageSize=null, startFrom=0)]) \ No newline at end of file + CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=LogicalProject#,group={0, 1},agg#0=LITERAL_AGG(2)), LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":0,"timeout":"1m","aggregations":{"composite_buckets":{"composite":{"size":1000,"sources":[{"gender":{"terms":{"field":"gender.keyword","missing_bucket":false,"order":"asc"}}},{"state":{"terms":{"field":"state.keyword","missing_bucket":false,"order":"asc"}}}]},"aggregations":{"$f2":{"top_hits":{"from":0,"size":2,"version":false,"seq_no_primary_term":false,"explain":false,"_source":{"includes":["gender","state","account_number","age"],"excludes":[]},"script_fields":{"new_gender":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["gender.keyword"]}},"ignore_failure":false},"new_state":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQA/HsKICAib3AiOiB7CiAgICAibmFtZSI6ICJMT1dFUiIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfQogIF0KfQ==\"}","lang":"opensearch_compounded_script","params":{"utcTimestamp": 0,"SOURCES":[0],"DIGESTS":["state.keyword"]}},"ignore_failure":false}}}}}}}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) \ No newline at end of file diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_exists_correlated_subquery.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_exists_correlated_subquery.yaml index d28c948b6ad..05dd3778b8f 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_exists_correlated_subquery.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_exists_correlated_subquery.yaml @@ -13,7 +13,7 @@ calcite: CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_worker]]) physical: | EnumerableCalc(expr#0..3=[{inputs}], id=[$t1], name=[$t0], salary=[$t2]) - EnumerableLimitSort(sort0=[$2], dir0=[DESC-nulls-last], fetch=[10000]) + CalciteEnumerableTopK(sort0=[$2], dir0=[DESC-nulls-last], fetch=[10000]) EnumerableCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_worker]], PushDownContext=[[PROJECT->[name, id, salary]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["name","id","salary"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) EnumerableAggregate(group=[{0}]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_exists_uncorrelated_subquery.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_exists_uncorrelated_subquery.yaml index e6b6774dd6b..f54be57fefd 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_exists_uncorrelated_subquery.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_exists_uncorrelated_subquery.yaml @@ -12,7 +12,7 @@ calcite: CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_worker]]) physical: | EnumerableCalc(expr#0..3=[{inputs}], id=[$t1], name=[$t0], salary=[$t2]) - EnumerableLimitSort(sort0=[$2], dir0=[DESC-nulls-last], fetch=[10000]) + CalciteEnumerableTopK(sort0=[$2], dir0=[DESC-nulls-last], fetch=[10000]) EnumerableNestedLoopJoin(condition=[true], joinType=[inner]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_worker]], PushDownContext=[[PROJECT->[name, id, salary]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["name","id","salary"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) EnumerableAggregate(group=[{0}]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_in_correlated_subquery.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_in_correlated_subquery.yaml index eca1199768e..18cd0c5147a 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_in_correlated_subquery.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_in_correlated_subquery.yaml @@ -13,7 +13,7 @@ calcite: CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_worker]]) physical: | EnumerableCalc(expr#0..3=[{inputs}], id=[$t1], name=[$t0], salary=[$t2]) - EnumerableLimitSort(sort0=[$2], dir0=[DESC-nulls-last], fetch=[10000]) + CalciteEnumerableTopK(sort0=[$2], dir0=[DESC-nulls-last], fetch=[10000]) EnumerableCalc(expr#0..3=[{inputs}], expr#4=[=($t0, $t3)], proj#0..3=[{exprs}], $condition=[$t4]) EnumerableCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_worker]], PushDownContext=[[PROJECT->[name, id, salary]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["name","id","salary"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_in_uncorrelated_subquery.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_in_uncorrelated_subquery.yaml index a8baf16ac46..ae5e866620b 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_in_uncorrelated_subquery.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_in_uncorrelated_subquery.yaml @@ -11,7 +11,7 @@ calcite: CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_worker]]) physical: | EnumerableCalc(expr#0..2=[{inputs}], id=[$t1], name=[$t0], salary=[$t2]) - EnumerableLimitSort(sort0=[$2], dir0=[DESC-nulls-last], fetch=[10000]) + CalciteEnumerableTopK(sort0=[$2], dir0=[DESC-nulls-last], fetch=[10000]) EnumerableHashJoin(condition=[=($1, $3)], joinType=[semi]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_worker]], PushDownContext=[[PROJECT->[name, id, salary]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["name","id","salary"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_work_information]], PushDownContext=[[PROJECT->[uid], LIMIT->10000], OpenSearchRequestBuilder(sourceBuilder={"from":0,"size":10000,"timeout":"1m","_source":{"includes":["uid"],"excludes":[]}}, requestedTotalSize=10000, pageSize=null, startFrom=0)]) \ No newline at end of file diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_limit_agg_pushdown5.json b/integ-test/src/test/resources/expectedOutput/calcite/explain_limit_agg_pushdown5.json index 7895625cde5..3b0fede88bf 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_limit_agg_pushdown5.json +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_limit_agg_pushdown5.json @@ -1,6 +1,6 @@ { "calcite": { "logical": "LogicalSystemLimit(fetch=[10000], type=[QUERY_SIZE_LIMIT])\n LogicalSort(offset=[10], fetch=[10])\n LogicalSort(sort0=[$0], dir0=[ASC-nulls-first], 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 EnumerableLimit(offset=[10], fetch=[10])\n EnumerableLimitSort(sort0=[$0], dir0=[ASC-nulls-first], fetch=[100])\n CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},count()=COUNT()), PROJECT->[count(), state]], 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\"}}}]}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)])\n" + "physical": "EnumerableLimit(fetch=[10000])\n EnumerableLimit(offset=[10], fetch=[10])\n CalciteEnumerableTopK(sort0=[$0], dir0=[ASC-nulls-first], fetch=[100])\n CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0},count()=COUNT()), PROJECT->[count(), state]], 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\"}}}]}}}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)])\n" } } 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 dc918429eba..3767a38b3a9 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 @@ -8,5 +8,5 @@ calcite: LogicalFilter(condition=[IS NOT NULL($7)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - EnumerableLimitSort(sort0=[$0], sort1=[$2], dir0=[ASC-nulls-first], dir1=[ASC-nulls-first], fetch=[10000]) + CalciteEnumerableTopK(sort0=[$0], sort1=[$2], dir0=[ASC-nulls-first], dir1=[ASC-nulls-first], fetch=[10000]) 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)]) \ No newline at end of file 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 8c29e32bab1..520f729b7f9 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 @@ -8,5 +8,5 @@ calcite: LogicalFilter(condition=[IS NOT NULL($7)]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | - EnumerableLimitSort(sort0=[$0], sort1=[$1], dir0=[ASC-nulls-first], dir1=[ASC-nulls-first], fetch=[10000]) + CalciteEnumerableTopK(sort0=[$0], sort1=[$1], dir0=[ASC-nulls-first], dir1=[ASC-nulls-first], fetch=[10000]) 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)]) \ No newline at end of file diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_sort_agg_push.json b/integ-test/src/test/resources/expectedOutput/calcite/explain_sort_agg_push.json index d54c254594d..9eea9f67a53 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_sort_agg_push.json +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_sort_agg_push.json @@ -1,6 +1,6 @@ { "calcite": { "logical": "LogicalSystemLimit(sort0=[$0], dir0=[ASC-nulls-first], fetch=[10000], type=[QUERY_SIZE_LIMIT])\n LogicalSort(sort0=[$0], dir0=[ASC-nulls-first])\n LogicalProject(avg_age=[$2], state=[$0], city=[$1])\n LogicalAggregate(group=[{0, 1}], avg_age=[AVG($2)])\n LogicalProject(state=[$7], city=[$5], age=[$8])\n CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]])\n", - "physical": "EnumerableLimitSort(sort0=[$0], dir0=[ASC-nulls-first], fetch=[10000])\n CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},avg_age=AVG($2)), PROJECT->[avg_age, state, city]], 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\"}}},{\"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)])\n" + "physical": "CalciteEnumerableTopK(sort0=[$0], dir0=[ASC-nulls-first], fetch=[10000])\n CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[AGGREGATION->rel#:LogicalAggregate.NONE.[](input=RelSubset#,group={0, 1},avg_age=AVG($2)), PROJECT->[avg_age, state, city]], 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\"}}},{\"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)])\n" } } diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_sort_pass_through_join_then_pushdown.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_sort_pass_through_join_then_pushdown.yaml index 15d2907173a..36ae1037150 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_sort_pass_through_join_then_pushdown.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_sort_pass_through_join_then_pushdown.yaml @@ -10,7 +10,7 @@ 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]) CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]]) physical: | - EnumerableLimitSort(sort0=[$13], dir0=[ASC-nulls-first], fetch=[10000]) + CalciteEnumerableTopK(sort0=[$13], dir0=[ASC-nulls-first], fetch=[10000]) EnumerableMergeJoin(condition=[=($13, $15)], joinType=[left]) EnumerableCalc(expr#0..12=[{inputs}], expr#13=['(?^[A-Z])'], expr#14=['initial'], expr#15=[REX_EXTRACT($t6, $t13, $t14)], proj#0..12=[{exprs}], $f13=[$t15]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_bank]], PushDownContext=[[PROJECT->[account_number, firstname, address, birthdate, gender, city, lastname, balance, employer, state, age, email, male], SORT_EXPR->[REX_EXTRACT($6, '(?^[A-Z])', 'initial') ASCENDING NULLS_LAST]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["account_number","firstname","address","birthdate","gender","city","lastname","balance","employer","state","age","email","male"],"excludes":[]},"sort":[{"_script":{"script":{"source":"{\"langType\":\"calcite\",\"script\":\"rO0ABXQC63sKICAib3AiOiB7CiAgICAibmFtZSI6ICJSRVhfRVhUUkFDVCIsCiAgICAia2luZCI6ICJPVEhFUl9GVU5DVElPTiIsCiAgICAic3ludGF4IjogIkZVTkNUSU9OIgogIH0sCiAgIm9wZXJhbmRzIjogWwogICAgewogICAgICAiZHluYW1pY1BhcmFtIjogMCwKICAgICAgInR5cGUiOiB7CiAgICAgICAgInR5cGUiOiAiVkFSQ0hBUiIsCiAgICAgICAgIm51bGxhYmxlIjogdHJ1ZSwKICAgICAgICAicHJlY2lzaW9uIjogLTEKICAgICAgfQogICAgfSwKICAgIHsKICAgICAgImR5bmFtaWNQYXJhbSI6IDEsCiAgICAgICJ0eXBlIjogewogICAgICAgICJ0eXBlIjogIlZBUkNIQVIiLAogICAgICAgICJudWxsYWJsZSI6IHRydWUsCiAgICAgICAgInByZWNpc2lvbiI6IC0xCiAgICAgIH0KICAgIH0sCiAgICB7CiAgICAgICJkeW5hbWljUGFyYW0iOiAyLAogICAgICAidHlwZSI6IHsKICAgICAgICAidHlwZSI6ICJWQVJDSEFSIiwKICAgICAgICAibnVsbGFibGUiOiB0cnVlLAogICAgICAgICJwcmVjaXNpb24iOiAtMQogICAgICB9CiAgICB9CiAgXSwKICAiY2xhc3MiOiAib3JnLm9wZW5zZWFyY2guc3FsLmV4cHJlc3Npb24uZnVuY3Rpb24uVXNlckRlZmluZWRGdW5jdGlvbkJ1aWxkZXIkMSIsCiAgInR5cGUiOiB7CiAgICAidHlwZSI6ICJWQVJDSEFSIiwKICAgICJudWxsYWJsZSI6IHRydWUsCiAgICAicHJlY2lzaW9uIjogMjAwMAogIH0sCiAgImRldGVybWluaXN0aWMiOiB0cnVlLAogICJkeW5hbWljIjogZmFsc2UKfQ==\"}","lang":"opensearch_compounded_script","params":{"MISSING_MAX":true,"utcTimestamp": 0,"SOURCES":[0,2,2],"DIGESTS":["lastname","(?^[A-Z])","initial"]}},"type":"string","order":"asc"}}]}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_distinct_count.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_distinct_count.yaml index e47cc2a3680..d464ea5b6d8 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_distinct_count.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_distinct_count.yaml @@ -8,7 +8,7 @@ calcite: CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | EnumerableCalc(expr#0..12=[{inputs}], proj#0..10=[{exprs}], distinct_states=[$t12]) - EnumerableLimitSort(sort0=[$11], dir0=[ASC], fetch=[10000]) + CalciteEnumerableTopK(sort0=[$11], dir0=[ASC], fetch=[10000]) EnumerableWindow(window#0=[window(partition {4} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [DISTINCT_COUNT_APPROX($7)])]) EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[account_number, firstname, address, balance, gender, city, employer, state, age, email, lastname]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["account_number","firstname","address","balance","gender","city","employer","state","age","email","lastname"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_earliest_latest.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_earliest_latest.yaml index e7b7e1ca264..b8ddae83a2e 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_earliest_latest.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_earliest_latest.yaml @@ -8,7 +8,7 @@ calcite: CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_logs]]) physical: | EnumerableCalc(expr#0..7=[{inputs}], proj#0..4=[{exprs}], earliest_message=[$t6], latest_message=[$t7]) - EnumerableLimitSort(sort0=[$5], dir0=[ASC], fetch=[10000]) + CalciteEnumerableTopK(sort0=[$5], dir0=[ASC], fetch=[10000]) EnumerableWindow(window#0=[window(partition {1} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ARG_MIN($3, $2), ARG_MAX($3, $2)])]) EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_logs]], PushDownContext=[[PROJECT->[created_at, server, @timestamp, message, level]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["created_at","server","@timestamp","message","level"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_earliest_latest_custom_time.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_earliest_latest_custom_time.yaml index 30b0bbb289b..86666bdaf8a 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_earliest_latest_custom_time.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_earliest_latest_custom_time.yaml @@ -8,7 +8,7 @@ calcite: CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_logs]]) physical: | EnumerableCalc(expr#0..7=[{inputs}], proj#0..4=[{exprs}], earliest_message=[$t6], latest_message=[$t7]) - EnumerableLimitSort(sort0=[$5], dir0=[ASC], fetch=[10000]) + CalciteEnumerableTopK(sort0=[$5], dir0=[ASC], fetch=[10000]) EnumerableWindow(window#0=[window(partition {4} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ARG_MIN($3, $0), ARG_MAX($3, $0)])]) EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_logs]], PushDownContext=[[PROJECT->[created_at, server, @timestamp, message, level]], OpenSearchRequestBuilder(sourceBuilder={"from":0,"timeout":"1m","_source":{"includes":["created_at","server","@timestamp","message","level"],"excludes":[]}}, requestedTotalSize=2147483647, pageSize=null, startFrom=0)]) \ No newline at end of file diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_global_null_bucket.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_global_null_bucket.yaml index 67cb26a297b..3959777f707 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_global_null_bucket.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_global_null_bucket.yaml @@ -13,7 +13,7 @@ calcite: CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | EnumerableCalc(expr#0..16=[{inputs}], proj#0..10=[{exprs}], avg_age=[$t16]) - EnumerableLimitSort(sort0=[$11], dir0=[ASC], fetch=[10000]) + CalciteEnumerableTopK(sort0=[$11], dir0=[ASC], fetch=[10000]) EnumerableMergeJoin(condition=[AND(=($4, $13), =($11, $14), =($12, $15))], joinType=[left]) EnumerableSort(sort0=[$4], sort1=[$11], sort2=[$12], dir0=[ASC], dir1=[ASC], dir2=[ASC]) EnumerableCalc(expr#0..11=[{inputs}], expr#12=[1], expr#13=[-($t11, $t12)], proj#0..11=[{exprs}], $f12=[$t13]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_null_bucket.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_null_bucket.yaml index be9a5f3d391..c4b49653614 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_null_bucket.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_null_bucket.yaml @@ -8,7 +8,7 @@ calcite: CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | EnumerableCalc(expr#0..14=[{inputs}], expr#15=[CAST($t14):DOUBLE NOT NULL], expr#16=[/($t13, $t15)], expr#17=[null:DOUBLE], expr#18=[CASE($t12, $t16, $t17)], proj#0..10=[{exprs}], avg_age=[$t18]) - EnumerableLimitSort(sort0=[$11], dir0=[ASC], fetch=[10000]) + CalciteEnumerableTopK(sort0=[$11], dir0=[ASC], fetch=[10000]) EnumerableWindow(window#0=[window(partition {4} rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [$SUM0($8), COUNT($8)])]) EnumerableCalc(expr#0..11=[{inputs}], expr#12=[IS NOT NULL($t4)], proj#0..12=[{exprs}]) EnumerableWindow(window#0=[window(rows between UNBOUNDED PRECEDING and CURRENT ROW aggs [ROW_NUMBER()])]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_reset_null_bucket.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_reset_null_bucket.yaml index 48d059bdd85..7be62745372 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_reset_null_bucket.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_streamstats_reset_null_bucket.yaml @@ -15,7 +15,7 @@ calcite: CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | EnumerableCalc(expr#0..16=[{inputs}], proj#0..10=[{exprs}], avg_age=[$t16]) - EnumerableLimitSort(sort0=[$11], dir0=[ASC], fetch=[10000]) + CalciteEnumerableTopK(sort0=[$11], dir0=[ASC], fetch=[10000]) EnumerableMergeJoin(condition=[AND(=($4, $13), =($11, $14), =($12, $15))], joinType=[left]) EnumerableSort(sort0=[$4], sort1=[$11], sort2=[$12], dir0=[ASC], dir1=[ASC], dir2=[ASC]) EnumerableCalc(expr#0..15=[{inputs}], expr#16=[0], expr#17=[COALESCE($t15, $t16)], expr#18=[+($t14, $t17)], proj#0..11=[{exprs}], __seg_id__=[$t18]) 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 ec04bb50c28..5dd080d68b5 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_timechart.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_timechart.yaml @@ -19,7 +19,7 @@ calcite: LogicalFilter(condition=[AND(IS NOT NULL($1), IS NOT NULL($7))]) CalciteLogicalIndexScan(table=[[OpenSearch, events]]) physical: | - EnumerableLimitSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[10000]) + CalciteEnumerableTopK(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[10000]) 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]) 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]) 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 4f8267f3117..9e462061605 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 @@ -19,7 +19,7 @@ calcite: LogicalFilter(condition=[IS NOT NULL($1)]) CalciteLogicalIndexScan(table=[[OpenSearch, events]]) physical: | - EnumerableLimitSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[10000]) + CalciteEnumerableTopK(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[10000]) EnumerableAggregate(group=[{0, 1}], count()=[$SUM0($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], count()=[$t2]) EnumerableMergeJoin(condition=[=($1, $3)], joinType=[left]) diff --git a/integ-test/src/test/resources/expectedOutput/calcite/explain_top_k_then_sort_push.yaml b/integ-test/src/test/resources/expectedOutput/calcite/explain_top_k_then_sort_push.yaml index 9c0a714771b..cba5870494c 100644 --- a/integ-test/src/test/resources/expectedOutput/calcite/explain_top_k_then_sort_push.yaml +++ b/integ-test/src/test/resources/expectedOutput/calcite/explain_top_k_then_sort_push.yaml @@ -7,7 +7,7 @@ calcite: CalciteLogicalIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]]) physical: | EnumerableCalc(expr#0..1=[{inputs}], age=[$t1]) - EnumerableLimitSort(sort0=[$1], dir0=[ASC-nulls-first], fetch=[10000]) + CalciteEnumerableTopK(sort0=[$1], dir0=[ASC-nulls-first], fetch=[10000]) CalciteEnumerableIndexScan(table=[[OpenSearch, opensearch-sql_test_index_account]], PushDownContext=[[PROJECT->[balance, age], SORT->[{ "balance" : { "order" : "asc", diff --git a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/physical/CalciteEnumerableTopK.java b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/physical/CalciteEnumerableTopK.java index b9a88eff5b9..28d49ebc684 100644 --- a/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/physical/CalciteEnumerableTopK.java +++ b/opensearch/src/main/java/org/opensearch/sql/opensearch/planner/physical/CalciteEnumerableTopK.java @@ -15,7 +15,6 @@ import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.metadata.RelMetadataQuery; import org.apache.calcite.rex.RexNode; -import org.checkerframework.checker.nullness.qual.Nullable; /** The different between this and {@link EnumerableLimitSort} is the cost. */ public class CalciteEnumerableTopK extends EnumerableLimitSort { @@ -25,13 +24,13 @@ public CalciteEnumerableTopK( RelTraitSet traitSet, RelNode input, RelCollation collation, - @Nullable RexNode offset, - @Nullable RexNode fetch) { + RexNode offset, + RexNode fetch) { super(cluster, traitSet, input, collation, offset, fetch); } @Override - public @Nullable RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) { + public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) { RelOptCost original = super.computeSelfCost(planner, mq); if (original == null) { return null; @@ -44,8 +43,19 @@ public CalciteEnumerableTopK( .makeCost(original.getRows() * 2 * 0.99, original.getCpu(), original.getIo()); } + @Override + public CalciteEnumerableTopK copy( + RelTraitSet traitSet, + RelNode newInput, + RelCollation newCollation, + RexNode offset, + RexNode fetch) { + return new CalciteEnumerableTopK( + this.getCluster(), traitSet, newInput, newCollation, offset, fetch); + } + public static CalciteEnumerableTopK create( - RelNode input, RelCollation collation, @Nullable RexNode offset, @Nullable RexNode fetch) { + RelNode input, RelCollation collation, RexNode offset, RexNode fetch) { final RelOptCluster cluster = input.getCluster(); final RelTraitSet traitSet = cluster.traitSetOf(EnumerableConvention.INSTANCE).replace(collation); 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 e7e056738f6..ca1d5384f59 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 @@ -11,6 +11,7 @@ import org.immutables.value.Value; import org.opensearch.sql.calcite.plan.OpenSearchRuleConfig; import org.opensearch.sql.calcite.utils.PlanUtils; +import org.opensearch.sql.opensearch.planner.physical.CalciteEnumerableTopK; import org.opensearch.sql.opensearch.storage.scan.AbstractCalciteIndexScan; @Value.Enclosing @@ -39,13 +40,14 @@ protected void onMatchImpl(RelOptRuleCall call) { /** Rule configuration. */ @Value.Immutable public interface Config extends OpenSearchRuleConfig { + Predicate isTopK = CalciteEnumerableTopK.class::isInstance; SortIndexScanRule.Config DEFAULT = ImmutableSortIndexScanRule.Config.builder() .build() .withOperandSupplier( b0 -> b0.operand(Sort.class) - .predicate(PlanUtils::sortByFieldsOnly) + .predicate(Predicate.not(isTopK).and(PlanUtils::sortByFieldsOnly)) .oneInput( b1 -> b1.operand(AbstractCalciteIndexScan.class)