This is an automated email from the ASF dual-hosted git repository. xxyu pushed a commit to branch kylin5 in repository https://gitbox.apache.org/repos/asf/kylin.git
commit b548c90edc1d43d62126f4eb2b6df5c8305cb21a Author: Pengfei Zhan <dethr...@gmail.com> AuthorDate: Thu Feb 23 14:34:40 2023 +0800 KYLIN-5578 Support min/max aggregation queries using metadata response queries --- .../org/apache/kylin/common/KylinConfigBase.java | 9 +- .../kylin-backward-compatibility.properties | 1 + .../org/apache/kylin/metadata/model/DataType.java | 2 +- .../kylin/query/engine/QueryExecWithMetaTest.java | 230 +++++++++++++++++++ .../test/resources/query/sql_min_max/query01.sql | 20 ++ .../test/resources/query/sql_min_max/query02.sql | 27 +++ .../test/resources/query/sql_min_max/query03-2.sql | 22 ++ .../test/resources/query/sql_min_max/query03.sql | 22 ++ .../test/resources/query/sql_min_max/query04.sql | 25 ++ .../test/resources/query/sql_min_max/query05.sql | 21 ++ .../test/resources/query/sql_min_max/query06.sql | 32 +++ .../test/resources/query/sql_min_max/query07.sql | 20 ++ .../metadata/_global/project/min_max.json | 35 +++ .../39e41387-c17e-1014-53cc-3d4c7b78e286.json | 185 +++++++++++++++ .../4623e6d3-2ca2-319e-9a3f-e26bd819734f.json | 101 ++++++++ .../8a21c1db-f6b3-36cc-e454-84135de78352.json | 85 +++++++ .../c5eecd91-0b8a-f8d7-89c8-c0afdaa01800.json | 73 ++++++ .../7b16f7c7-7648-b41c-6a7a-7188660370b5.json | 36 +++ .../7b8cf7c7-7648-b41c-623a-7188660370b5.json | 36 +++ .../c183b315-eb0e-3eac-8607-73c5b1552bce.json | 36 +++ .../d5768bf0-978f-9a46-e295-2026484744ff.json | 36 +++ .../d36576c3-8ca1-a567-e54d-5d4062d3b742.json | 50 ++++ .../39e41387-c17e-1014-53cc-3d4c7b78e286.json | 63 +++++ .../4623e6d3-2ca2-319e-9a3f-e26bd819734f.json | 63 +++++ .../8a21c1db-f6b3-36cc-e454-84135de78352.json | 63 +++++ .../c5eecd91-0b8a-f8d7-89c8-c0afdaa01800.json | 84 +++++++ .../39e41387-c17e-1014-53cc-3d4c7b78e286.json | 227 ++++++++++++++++++ .../4623e6d3-2ca2-319e-9a3f-e26bd819734f.json | 222 ++++++++++++++++++ .../8a21c1db-f6b3-36cc-e454-84135de78352.json | 254 +++++++++++++++++++++ .../c5eecd91-0b8a-f8d7-89c8-c0afdaa01800.json | 179 +++++++++++++++ .../metadata/min_max/table/SSB.CUSTOMER.json | 78 +++++++ .../metadata/min_max/table/SSB.DATES.json | 131 +++++++++++ .../metadata/min_max/table/SSB.LINEORDER.json | 131 +++++++++++ .../kylin/query/enumerator/MetadataEnumerator.java | 73 ++++++ .../apache/kylin/query/enumerator/OLAPQuery.java | 8 +- .../apache/kylin/query/relnode/OLAPContext.java | 134 ++++++++++- .../apache/kylin/query/relnode/OLAPJoinRel.java | 11 +- .../apache/kylin/query/relnode/OLAPProjectRel.java | 2 +- .../apache/kylin/query/relnode/OLAPTableScan.java | 22 +- .../org/apache/kylin/query/schema/OLAPTable.java | 4 + .../kylin/query/runtime/CalciteToSparkPlaner.scala | 93 +++++--- .../kylin/query/runtime/plan/TableScanPlan.scala | 79 +++++-- .../org/apache/spark/sql/SparderTypeUtil.scala | 26 +++ 43 files changed, 2968 insertions(+), 83 deletions(-) diff --git a/src/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java b/src/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java index 1fc95ac5cd..7fcb827ebc 100644 --- a/src/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java +++ b/src/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java @@ -75,13 +75,12 @@ import org.apache.kylin.common.util.SizeConvertUtil; import org.apache.kylin.common.util.StringHelper; import org.apache.kylin.common.util.TimeUtil; import org.apache.kylin.common.util.Unsafe; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import org.apache.kylin.guava30.shaded.common.collect.ImmutableSet; import org.apache.kylin.guava30.shaded.common.collect.Lists; import org.apache.kylin.guava30.shaded.common.collect.Maps; import org.apache.kylin.guava30.shaded.common.collect.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import io.kyligence.config.core.loader.IExternalConfigLoader; import lombok.val; @@ -1728,6 +1727,10 @@ public abstract class KylinConfigBase implements Serializable { // QUERY // ============================================================================ + public boolean isRouteToMetadataEnabled() { + return Boolean.parseBoolean(this.getOptional("kylin.query.using-metadata-answer-minmax-of-dimension", FALSE)); + } + public boolean partialMatchNonEquiJoins() { return Boolean.parseBoolean(getOptional("kylin.query.match-partial-non-equi-join-model", FALSE)); } diff --git a/src/core-common/src/main/resources/kylin-backward-compatibility.properties b/src/core-common/src/main/resources/kylin-backward-compatibility.properties index fc239ae6a8..7d4a79751e 100644 --- a/src/core-common/src/main/resources/kylin-backward-compatibility.properties +++ b/src/core-common/src/main/resources/kylin-backward-compatibility.properties @@ -147,6 +147,7 @@ kylin.query.dim.distinct.max=kylin.query.max-dimension-count-distinct kylin.query.security.enabled=kylin.query.security-enabled kylin.query.access.org.apache.kylin.controller=kylin.query.access-org.apache.kylin.controller kylin.query.udf.=kylin.query.udf. +kylin.query.try-route-to-metadata-enabled=kylin.query.using-metadata-answer-minmax-of-dimension #deprecated kylin.query.pushdown.limit.max=kylin.query.max-limit-pushdown diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataType.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataType.java index 4c6e24797b..271bd09b31 100644 --- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataType.java +++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataType.java @@ -58,7 +58,7 @@ public class DataType implements Serializable { @Override public boolean equals(Object obj) { - if (obj == null || !(obj instanceof DataType)) { + if (!(obj instanceof DataType)) { return false; } DataType other = (DataType) obj; diff --git a/src/kylin-it/src/test/java/org/apache/kylin/query/engine/QueryExecWithMetaTest.java b/src/kylin-it/src/test/java/org/apache/kylin/query/engine/QueryExecWithMetaTest.java new file mode 100644 index 0000000000..906128c97d --- /dev/null +++ b/src/kylin-it/src/test/java/org/apache/kylin/query/engine/QueryExecWithMetaTest.java @@ -0,0 +1,230 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kylin.query.engine; + +import java.io.IOException; +import java.io.InputStreamReader; +import java.sql.SQLException; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; + +import org.apache.kylin.engine.spark.NLocalWithSparkSessionTest; +import org.apache.kylin.guava30.shaded.common.base.Charsets; +import org.apache.kylin.guava30.shaded.common.io.CharStreams; +import org.apache.kylin.metadata.model.NDataModelManager; +import org.apache.kylin.query.engine.data.QueryResult; +import org.apache.kylin.query.util.QueryUtil; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class QueryExecWithMetaTest extends NLocalWithSparkSessionTest { + String project = "min_max"; + QueryExec queryExec; + + @Before + public void setup() { + createTestMetadata("src/test/resources/ut_meta/query_min_max_with_meta"); + overwriteSystemProp("kylin.query.using-metadata-answer-minmax-of-dimension", "true"); + queryExec = new QueryExec(project, getTestConfig()); + + } + + @After + public void teardown() { + cleanupTestMetadata(); + } + + @Test + public void testSingleTableModel() throws SQLException, IOException { + NDataModelManager modelManager = NDataModelManager.getInstance(getTestConfig(), project); + modelManager.dropModel("4623e6d3-2ca2-319e-9a3f-e26bd819734f"); + String sql = getSql("/query/sql_min_max/query01.sql"); + QueryResult queryResult = queryExec.executeQuery(sql); + Assert.assertEquals(1, queryResult.getSize()); + List<String> next = queryResult.getRowsIterable().iterator().next(); + Assert.assertEquals("1", next.get(0)); + Assert.assertEquals("60000", next.get(1)); + Assert.assertEquals("1.001", next.get(2)); + } + + @Test + public void testSingleTableModelWithUnion() throws SQLException, IOException { + String sql = getSql("/query/sql_min_max/query02.sql"); + QueryResult queryResult = queryExec.executeQuery(sql); + Assert.assertEquals(2, queryResult.getSize()); + Iterator<List<String>> iterator = queryResult.getRowsIterable().iterator(); + List<String> firstRow = iterator.next(); + Assert.assertEquals("1", firstRow.get(0)); + Assert.assertEquals("60000", firstRow.get(1)); + Assert.assertEquals("1.001", firstRow.get(2)); + List<String> secondRow = iterator.next(); + Assert.assertEquals("1", secondRow.get(0)); + Assert.assertEquals("60000", secondRow.get(1)); + Assert.assertEquals("1.001", secondRow.get(2)); + } + + @Test + public void testSingleTableModelDoNotUseMetadata() throws IOException { + overwriteSystemProp("kylin.query.using-metadata-answer-minmax-of-dimension", "false"); + NDataModelManager modelManager = NDataModelManager.getInstance(getTestConfig(), project); + modelManager.dropModel("4623e6d3-2ca2-319e-9a3f-e26bd819734f"); + String sql = getSql("/query/sql_min_max/query01.sql"); + try { + queryExec.executeQuery(sql); + Assert.fail(); + } catch (Exception e) { + // because the index was not built when running this case + Assert.assertTrue(e.getMessage().endsWith( + "parquet/c5eecd91-0b8a-f8d7-89c8-c0afdaa01800/d36576c3-8ca1-a567-e54d-5d4062d3b742/1 does not exist")); + } + } + + @Test + public void testCanNotUseMetadataDueToGroupByNonConst() { + NDataModelManager modelManager = NDataModelManager.getInstance(getTestConfig(), project); + modelManager.dropModel("4623e6d3-2ca2-319e-9a3f-e26bd819734f"); + String sql = "select LO_ORDERKEY, min(LO_ORDERKEY) from ssb.lineorder group by LO_ORDERKEY"; + try { + queryExec.executeQuery(sql); + Assert.fail(); + } catch (Exception e) { + // because the index was not built when running this case + Assert.assertTrue(e.getMessage().endsWith( + "parquet/c5eecd91-0b8a-f8d7-89c8-c0afdaa01800/d36576c3-8ca1-a567-e54d-5d4062d3b742/1 does not exist")); + } + } + + @Test + public void testSingleTableModelDoNotUseMetadataDueToExpressionAsParameter() throws IOException { + NDataModelManager modelManager = NDataModelManager.getInstance(getTestConfig(), project); + modelManager.dropModel("4623e6d3-2ca2-319e-9a3f-e26bd819734f"); + String sql = getSql("/query/sql_min_max/query07.sql"); + try { + queryExec.executeQuery(sql); + Assert.fail(); + } catch (Exception e) { + // because the index was not built when running this case + Assert.assertTrue(e.getMessage().endsWith( + "parquet/c5eecd91-0b8a-f8d7-89c8-c0afdaa01800/d36576c3-8ca1-a567-e54d-5d4062d3b742/1 does not exist")); + } + } + + @Test + public void testCanNotUseMetadataDueToMinMaxColumnIsNotDimension() { + NDataModelManager modelManager = NDataModelManager.getInstance(getTestConfig(), project); + modelManager.dropModel("4623e6d3-2ca2-319e-9a3f-e26bd819734f"); + String sql = "select min(LO_ORDERPRIOTITY) from ssb.lineorder"; + try { + queryExec.executeQuery(sql); + Assert.fail(); + } catch (Exception e) { + // because the index was not built when running this case + Assert.assertTrue(e.getMessage().endsWith( + "parquet/c5eecd91-0b8a-f8d7-89c8-c0afdaa01800/d36576c3-8ca1-a567-e54d-5d4062d3b742/10001 does not exist")); + } + } + + @Test + public void testCanNotUseMetadataDueToAggIsNotMinMax() { + String sql = "SELECT SUM(LO_QUANTITY) FROM SSB.LINEORDER"; + try { + queryExec.executeQuery(sql); + Assert.fail(); + } catch (Exception e) { + // because the index was not built when running this case + Assert.assertTrue(e.getMessage().endsWith( + "parquet/c5eecd91-0b8a-f8d7-89c8-c0afdaa01800/d36576c3-8ca1-a567-e54d-5d4062d3b742/1 does not exist")); + } + } + + // has partition column + @Test + public void testInnerJoinModel() throws IOException, SQLException { + String sql = getSql("/query/sql_min_max/query03.sql"); + QueryResult queryResult = queryExec.executeQuery(sql); + Assert.assertEquals(1, queryResult.getSize()); + List<String> next = queryResult.getRowsIterable().iterator().next(); + Assert.assertEquals("1", next.get(0)); + Assert.assertEquals("60000", next.get(1)); + Assert.assertEquals("1.001", next.get(2)); + } + + @Test + public void testInnerJoinModelWithIncrementalBuiltSegments() throws IOException, SQLException { + String sql = getSql("/query/sql_min_max/query03-2.sql"); + QueryResult queryResult = queryExec.executeQuery(sql); + Assert.assertEquals(1, queryResult.getSize()); + List<String> next = queryResult.getRowsIterable().iterator().next(); + Assert.assertEquals("1", next.get(0)); + Assert.assertEquals("20000", next.get(1)); + Assert.assertEquals("1.001", next.get(2)); + } + + @Test + public void testInnerJoinModelWithSingleTableModel() throws IOException, SQLException { + String sql = getSql("/query/sql_min_max/query04.sql"); + QueryResult queryResult = queryExec.executeQuery(sql); + Assert.assertEquals(1, queryResult.getSize()); + List<String> next = queryResult.getRowsIterable().iterator().next(); + Assert.assertEquals("1", next.get(0)); + Assert.assertEquals("60000", next.get(1)); + Assert.assertEquals("1.001", next.get(2)); + } + + @Test + public void testLeftJoinModel() throws IOException, SQLException { + String sql = getSql("/query/sql_min_max/query05.sql"); + QueryResult queryResult = queryExec.executeQuery(sql); + Assert.assertEquals(1, queryResult.getSize()); + List<String> next = queryResult.getRowsIterable().iterator().next(); + Assert.assertEquals("0", next.get(0)); + Assert.assertEquals("20", next.get(1)); + Assert.assertEquals("2023", next.get(2)); + } + + @Test + public void testLeftJoinModelWithUnion() throws IOException, SQLException { + String sql = getSql("/query/sql_min_max/query06.sql"); + QueryResult queryResult = queryExec.executeQuery(sql); + Assert.assertEquals(3, queryResult.getSize()); + Iterator<List<String>> iterator = queryResult.getRowsIterable().iterator(); + List<String> firstRow = iterator.next(); + Assert.assertEquals("0", firstRow.get(0)); + Assert.assertEquals("20", firstRow.get(1)); + Assert.assertEquals("2023", firstRow.get(2)); + List<String> secondRow = iterator.next(); + Assert.assertEquals("0", secondRow.get(0)); + Assert.assertEquals("20", secondRow.get(1)); + Assert.assertEquals("2023", secondRow.get(2)); + List<String> thirdRow = iterator.next(); + Assert.assertEquals("1", thirdRow.get(0)); + Assert.assertEquals("60000", thirdRow.get(1)); + Assert.assertEquals("1001", thirdRow.get(2)); + } + + private String getSql(String path) throws IOException { + String sql = CharStreams.toString( + new InputStreamReader(Objects.requireNonNull(getClass().getResourceAsStream(path)), Charsets.UTF_8)); + return QueryUtil.removeCommentInSql(sql); + } + +} diff --git a/src/kylin-it/src/test/resources/query/sql_min_max/query01.sql b/src/kylin-it/src/test/resources/query/sql_min_max/query01.sql new file mode 100644 index 0000000000..213cec0af7 --- /dev/null +++ b/src/kylin-it/src/test/resources/query/sql_min_max/query01.sql @@ -0,0 +1,20 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one +-- or more contributor license agreements. See the NOTICE file +-- distributed with this work for additional information +-- regarding copyright ownership. The ASF licenses this file +-- to you under the Apache License, Version 2.0 (the +-- "License"); you may not use this file except in compliance +-- with the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. + +select min(LO_ORDERKEY) as min_key, max(LO_ORDERKEY) as max_key, 1.001 as const +from ssb.lineorder +group by 1.001 diff --git a/src/kylin-it/src/test/resources/query/sql_min_max/query02.sql b/src/kylin-it/src/test/resources/query/sql_min_max/query02.sql new file mode 100644 index 0000000000..9bf30b699a --- /dev/null +++ b/src/kylin-it/src/test/resources/query/sql_min_max/query02.sql @@ -0,0 +1,27 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one +-- or more contributor license agreements. See the NOTICE file +-- distributed with this work for additional information +-- regarding copyright ownership. The ASF licenses this file +-- to you under the Apache License, Version 2.0 (the +-- "License"); you may not use this file except in compliance +-- with the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. + +select min(LO_ORDERKEY) as min_key, max(LO_ORDERKEY) as max_key, 1.001 as const +from ssb.lineorder +group by 1.001 +union all +select min(LO_ORDERKEY), max(LO_ORDERKEY), 1.001 +from ssb.lineorder +union all +select 1, 2, 3 +from ssb.lineorder +where 1 = 2 diff --git a/src/kylin-it/src/test/resources/query/sql_min_max/query03-2.sql b/src/kylin-it/src/test/resources/query/sql_min_max/query03-2.sql new file mode 100644 index 0000000000..5aa928965b --- /dev/null +++ b/src/kylin-it/src/test/resources/query/sql_min_max/query03-2.sql @@ -0,0 +1,22 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one +-- or more contributor license agreements. See the NOTICE file +-- distributed with this work for additional information +-- regarding copyright ownership. The ASF licenses this file +-- to you under the Apache License, Version 2.0 (the +-- "License"); you may not use this file except in compliance +-- with the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. + +SELECT MIN(LO_PARTKEY) AS MIN_KEY, MAX(LO_PARTKEY) AS MAX_KEY, 1.001 AS CONST +FROM SSB.LINEORDER + INNER JOIN SSB.CUSTOMER + ON LINEORDER.LO_CUSTKEY = CUSTOMER.C_CUSTKEY +GROUP BY 1.001 diff --git a/src/kylin-it/src/test/resources/query/sql_min_max/query03.sql b/src/kylin-it/src/test/resources/query/sql_min_max/query03.sql new file mode 100644 index 0000000000..68faf6e5d4 --- /dev/null +++ b/src/kylin-it/src/test/resources/query/sql_min_max/query03.sql @@ -0,0 +1,22 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one +-- or more contributor license agreements. See the NOTICE file +-- distributed with this work for additional information +-- regarding copyright ownership. The ASF licenses this file +-- to you under the Apache License, Version 2.0 (the +-- "License"); you may not use this file except in compliance +-- with the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. + +SELECT MIN(LO_ORDERKEY) AS MIN_KEY, MAX(LO_ORDERKEY) AS MAX_KEY, 1.001 AS CONST +FROM SSB.LINEORDER + INNER JOIN SSB.DATES + ON LINEORDER.LO_ORDERDATE = DATES.D_DATEKEY +GROUP BY 1.001 diff --git a/src/kylin-it/src/test/resources/query/sql_min_max/query04.sql b/src/kylin-it/src/test/resources/query/sql_min_max/query04.sql new file mode 100644 index 0000000000..16cad10c46 --- /dev/null +++ b/src/kylin-it/src/test/resources/query/sql_min_max/query04.sql @@ -0,0 +1,25 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one +-- or more contributor license agreements. See the NOTICE file +-- distributed with this work for additional information +-- regarding copyright ownership. The ASF licenses this file +-- to you under the Apache License, Version 2.0 (the +-- "License"); you may not use this file except in compliance +-- with the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. + +select * +from (SELECT MIN(LO_ORDERKEY) AS MIN_KEY, MAX(LO_ORDERKEY) AS MAX_KEY, 1.001 AS CONST + FROM SSB.LINEORDER + INNER JOIN SSB.DATES + ON LINEORDER.LO_ORDERDATE = DATES.D_DATEKEY) t1 + LEFT JOIN (SELECT MIN(LO_ORDERKEY) AS MIN_KEY, MAX(LO_ORDERKEY) AS MAX_KEY, 1.0001 AS CONST + FROM SSB.LINEORDER + GROUP BY 1.0001) T ON T.MIN_KEY = t1.MIN_KEY diff --git a/src/kylin-it/src/test/resources/query/sql_min_max/query05.sql b/src/kylin-it/src/test/resources/query/sql_min_max/query05.sql new file mode 100644 index 0000000000..1c65aced3b --- /dev/null +++ b/src/kylin-it/src/test/resources/query/sql_min_max/query05.sql @@ -0,0 +1,21 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one +-- or more contributor license agreements. See the NOTICE file +-- distributed with this work for additional information +-- regarding copyright ownership. The ASF licenses this file +-- to you under the Apache License, Version 2.0 (the +-- "License"); you may not use this file except in compliance +-- with the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. + +SELECT MIN(LO_SHIPPRIOTITY), MAX(LO_SUPPKEY), 2023 AS CONST +FROM SSB.LINEORDER AS LINEORDER + LEFT JOIN SSB.CUSTOMER AS CUSTOMER + ON LINEORDER.LO_CUSTKEY = CUSTOMER.C_CUSTKEY diff --git a/src/kylin-it/src/test/resources/query/sql_min_max/query06.sql b/src/kylin-it/src/test/resources/query/sql_min_max/query06.sql new file mode 100644 index 0000000000..3eb1286f0c --- /dev/null +++ b/src/kylin-it/src/test/resources/query/sql_min_max/query06.sql @@ -0,0 +1,32 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one +-- or more contributor license agreements. See the NOTICE file +-- distributed with this work for additional information +-- regarding copyright ownership. The ASF licenses this file +-- to you under the Apache License, Version 2.0 (the +-- "License"); you may not use this file except in compliance +-- with the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. + +select * +from (SELECT MIN(LO_SHIPPRIOTITY), MAX(LO_SUPPKEY), 2023 AS CONST + FROM SSB.LINEORDER AS LINEORDER + LEFT JOIN SSB.CUSTOMER AS CUSTOMER + ON LINEORDER.LO_CUSTKEY = CUSTOMER.C_CUSTKEY + UNION ALL + SELECT MIN(LO_SHIPPRIOTITY), MAX(LO_SUPPKEY), 2023 AS CONST + FROM SSB.LINEORDER AS LINEORDER + LEFT JOIN SSB.CUSTOMER AS CUSTOMER + ON LINEORDER.LO_CUSTKEY = CUSTOMER.C_CUSTKEY + UNION ALL + SELECT MIN(LO_ORDERKEY) AS MIN_KEY, MAX(LO_ORDERKEY) AS MAX_KEY, 1001 AS CONST + FROM SSB.LINEORDER + GROUP BY 1001) +order by 1 asc diff --git a/src/kylin-it/src/test/resources/query/sql_min_max/query07.sql b/src/kylin-it/src/test/resources/query/sql_min_max/query07.sql new file mode 100644 index 0000000000..49b33bccff --- /dev/null +++ b/src/kylin-it/src/test/resources/query/sql_min_max/query07.sql @@ -0,0 +1,20 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one +-- or more contributor license agreements. See the NOTICE file +-- distributed with this work for additional information +-- regarding copyright ownership. The ASF licenses this file +-- to you under the Apache License, Version 2.0 (the +-- "License"); you may not use this file except in compliance +-- with the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. + +select min(LO_ORDERKEY + 1) as min_key, max(LO_ORDERKEY) as max_key, 1.001 as const +from ssb.lineorder +group by 1.001 diff --git a/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/_global/project/min_max.json b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/_global/project/min_max.json new file mode 100644 index 0000000000..8d5a20a565 --- /dev/null +++ b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/_global/project/min_max.json @@ -0,0 +1,35 @@ +{ + "uuid" : "cd5b9780-fdf9-9f0c-8703-03538d3c451a", + "last_modified" : 1678676893524, + "create_time" : 1678676893495, + "version" : "4.0.0.0", + "name" : "min_max", + "owner" : "ADMIN", + "status" : "ENABLED", + "create_time_utc" : 1678676893495, + "default_database" : "DEFAULT", + "description" : "", + "principal" : null, + "keytab" : null, + "maintain_model_type" : "MANUAL_MAINTAIN", + "override_kylin_properties" : { + "kylin.metadata.semi-automatic-mode" : "false", + "kylin.query.metadata.expose-computed-column" : "true", + "kylin.source.default" : "9" + }, + "segment_config" : { + "auto_merge_enabled" : false, + "auto_merge_time_ranges" : [ "WEEK", "MONTH", "QUARTER", "YEAR" ], + "volatile_range" : { + "volatile_range_number" : 0, + "volatile_range_enabled" : false, + "volatile_range_type" : "DAY" + }, + "retention_range" : { + "retention_range_number" : 1, + "retention_range_enabled" : false, + "retention_range_type" : "MONTH" + }, + "create_empty_segment_enabled" : false + } +} diff --git a/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/dataflow/39e41387-c17e-1014-53cc-3d4c7b78e286.json b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/dataflow/39e41387-c17e-1014-53cc-3d4c7b78e286.json new file mode 100644 index 0000000000..526915c2dd --- /dev/null +++ b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/dataflow/39e41387-c17e-1014-53cc-3d4c7b78e286.json @@ -0,0 +1,185 @@ +{ + "uuid": "39e41387-c17e-1014-53cc-3d4c7b78e286", + "last_modified": 1678694546834, + "create_time": 1678693554349, + "version": "4.0.0.0", + "status": "ONLINE", + "last_status": null, + "cost": 50, + "query_hit_count": 0, + "last_query_time": 0, + "layout_query_hit_count": {}, + "segments": [ + { + "id": "7b16f7c7-7648-b41c-6a7a-7188660370b5", + "name": "19920101000000_19930802000000", + "create_time_utc": 1678694113558, + "status": "READY", + "segRange": { + "@class": "org.apache.kylin.metadata.model.SegmentRange$TimePartitionedSegmentRange", + "date_range_start": 694195200000, + "date_range_end": 744220800000 + }, + "timeRange": null, + "dimension_range_info_map": { + "0": { + "min": "6", + "max": "59969" + }, + "11": { + "min": "1-URGENT", + "max": "5-LOW" + }, + "22": { + "min": "Customer#000000001", + "max": "Customer#000000299" + }, + "1": { + "min": "1", + "max": "200" + }, + "12": { + "min": "0", + "max": "0" + }, + "3": { + "min": "54060", + "max": "114059" + }, + "15": { + "min": "1", + "max": "7" + }, + "16": { + "min": "1", + "max": "299" + }, + "6": { + "min": "0", + "max": "8" + }, + "17": { + "min": " 6I1TTaoG7bbiogC", + "max": "zwrDoaY2gxCk" + }, + "10": { + "min": "1992-01-01", + "max": "1993-08-01" + } + }, + "parameters": null, + "dictionaries": null, + "snapshots": null, + "last_build_time": 1679587339025, + "source_count": 14293, + "source_bytes_size": 5687314, + "column_source_bytes": { + "SSB.LINEORDER.LO_LINENUMBER": 14293, + "SSB.LINEORDER.LO_SHIPPRIOTITY": 14293, + "SSB.CUSTOMER.C_NAME": 5400, + "SSB.LINEORDER.LO_ORDERKEY": 53970, + "SSB.LINEORDER.LO_ORDERPRIOTITY": 108326, + "SSB.LINEORDER.LO_ORDERDATE": 142930, + "SSB.LINEORDER.LO_TAX": 14293, + "SSB.LINEORDER.LO_PARTKEY": 49368, + "SSB.LINEORDER.LO_CUSTKEY": 37862, + "SSB.CUSTOMER.C_ADDRESS": 4604, + "SSB.LINEORDER.LO_SUPPLYCOST": 74780 + }, + "ori_snapshot_size": {}, + "additionalInfo": {}, + "is_realtime_segment": false, + "is_snapshot_ready": false, + "is_dict_ready": false, + "is_flat_table_ready": true, + "is_fact_view_ready": false, + "multi_partitions": [], + "max_bucket_id": -1 + }, { + "id": "7b8cf7c7-7648-b41c-623a-7188660370b5", + "name": "19930802000000_19940802000000", + "create_time_utc": 1678694113558, + "status": "READY", + "segRange": { + "@class": "org.apache.kylin.metadata.model.SegmentRange$TimePartitionedSegmentRange", + "date_range_start": 744220800000, + "date_range_end": 775756800000 + }, + "timeRange": null, + "dimension_range_info_map": { + "0": { + "min": "6", + "max": "59969" + }, + "11": { + "min": "1-URGENT", + "max": "5-LOW" + }, + "22": { + "min": "Customer#000000001", + "max": "Customer#000000299" + }, + "1": { + "min": "5", + "max": "20000" + }, + "12": { + "min": "0", + "max": "0" + }, + "3": { + "min": "54060", + "max": "114059" + }, + "15": { + "min": "1", + "max": "7" + }, + "16": { + "min": "1", + "max": "299" + }, + "6": { + "min": "0", + "max": "8" + }, + "17": { + "min": " 6I1TTaoG7bbiogC", + "max": "zwrDoaY2gxCk" + }, + "10": { + "min": "1992-01-01", + "max": "1993-08-01" + } + }, + "parameters": null, + "dictionaries": null, + "snapshots": null, + "last_build_time": 1679587339025, + "source_count": 14293, + "source_bytes_size": 5687314, + "column_source_bytes": { + "SSB.LINEORDER.LO_LINENUMBER": 14293, + "SSB.LINEORDER.LO_SHIPPRIOTITY": 14293, + "SSB.CUSTOMER.C_NAME": 5400, + "SSB.LINEORDER.LO_ORDERKEY": 53970, + "SSB.LINEORDER.LO_ORDERPRIOTITY": 108326, + "SSB.LINEORDER.LO_ORDERDATE": 142930, + "SSB.LINEORDER.LO_TAX": 14293, + "SSB.LINEORDER.LO_PARTKEY": 49368, + "SSB.LINEORDER.LO_CUSTKEY": 37862, + "SSB.CUSTOMER.C_ADDRESS": 4604, + "SSB.LINEORDER.LO_SUPPLYCOST": 74780 + }, + "ori_snapshot_size": {}, + "additionalInfo": {}, + "is_realtime_segment": false, + "is_snapshot_ready": false, + "is_dict_ready": false, + "is_flat_table_ready": true, + "is_fact_view_ready": false, + "multi_partitions": [], + "max_bucket_id": -1 + } + ] +} diff --git a/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/dataflow/4623e6d3-2ca2-319e-9a3f-e26bd819734f.json b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/dataflow/4623e6d3-2ca2-319e-9a3f-e26bd819734f.json new file mode 100644 index 0000000000..e922cbbed3 --- /dev/null +++ b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/dataflow/4623e6d3-2ca2-319e-9a3f-e26bd819734f.json @@ -0,0 +1,101 @@ +{ + "uuid": "4623e6d3-2ca2-319e-9a3f-e26bd819734f", + "last_modified": 1678694363841, + "create_time": 1678693815468, + "version": "4.0.0.0", + "status": "ONLINE", + "last_status": null, + "cost": 50, + "query_hit_count": 0, + "last_query_time": 0, + "layout_query_hit_count": {}, + "segments": [ + { + "id": "c183b315-eb0e-3eac-8607-73c5b1552bce", + "name": "FULL_BUILD", + "create_time_utc": 1678693815468, + "status": "READY", + "segRange": { + "@class": "org.apache.kylin.metadata.model.SegmentRange$TimePartitionedSegmentRange", + "date_range_start": 0, + "date_range_end": 9223372036854775807 + }, + "timeRange": null, + "dimension_range_info_map": { + "0": { + "min": "1", + "max": "60000" + }, + "1": { + "min": "1", + "max": "2000" + }, + "12": { + "min": "0", + "max": "0" + }, + "15": { + "min": "1", + "max": "7" + }, + "16": { + "min": "1", + "max": "299" + }, + "7": { + "min": "1", + "max": "20" + }, + "18": { + "min": "ALGERIA", + "max": "VIETNAM" + }, + "8": { + "min": "87489", + "max": "46600128" + }, + "9": { + "min": "82530", + "max": "9479950" + }, + "10": { + "min": "1992-01-01", + "max": "1998-08-02" + }, + "21": { + "min": "AFRICA", + "max": "MIDDLE EAST" + } + }, + "parameters": null, + "dictionaries": null, + "snapshots": null, + "last_build_time": 1679587178031, + "source_count": 60175, + "source_bytes_size": 5687314, + "column_source_bytes": { + "SSB.CUSTOMER.C_REGION": 2107, + "SSB.LINEORDER.LO_LINENUMBER": 60175, + "SSB.LINEORDER.LO_SHIPPRIOTITY": 60175, + "SSB.LINEORDER.LO_ORDERKEY": 172702, + "SSB.LINEORDER.LO_ORDERDATE": 601750, + "SSB.LINEORDER.LO_REVENUE": 412258, + "SSB.LINEORDER.LO_SUPPKEY": 94173, + "SSB.LINEORDER.LO_PARTKEY": 207663, + "SSB.LINEORDER.LO_ORDTOTALPRICE": 469425, + "SSB.LINEORDER.LO_CUSTKEY": 155432, + "SSB.CUSTOMER.C_NATION": 2073 + }, + "ori_snapshot_size": {}, + "additionalInfo": {}, + "is_realtime_segment": false, + "is_snapshot_ready": false, + "is_dict_ready": false, + "is_flat_table_ready": true, + "is_fact_view_ready": false, + "multi_partitions": [], + "max_bucket_id": -1 + } + ] +} + diff --git a/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/dataflow/8a21c1db-f6b3-36cc-e454-84135de78352.json b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/dataflow/8a21c1db-f6b3-36cc-e454-84135de78352.json new file mode 100644 index 0000000000..c8e67bb18c --- /dev/null +++ b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/dataflow/8a21c1db-f6b3-36cc-e454-84135de78352.json @@ -0,0 +1,85 @@ +{ + "uuid": "8a21c1db-f6b3-36cc-e454-84135de78352", + "last_modified": 1678693584579, + "create_time": 1678693307906, + "version": "4.0.0.0", + "status": "ONLINE", + "last_status": null, + "cost": 50, + "query_hit_count": 0, + "last_query_time": 0, + "layout_query_hit_count": {}, + "segments": [ + { + "id": "d5768bf0-978f-9a46-e295-2026484744ff", + "name": "FULL_BUILD", + "create_time_utc": 1678693307908, + "status": "READY", + "segRange": { + "@class": "org.apache.kylin.metadata.model.SegmentRange$TimePartitionedSegmentRange", + "date_range_start": 0, + "date_range_end": 9223372036854775807 + }, + "timeRange": null, + "dimension_range_info_map": { + "0": { + "min": "1", + "max": "60000" + }, + "1": { + "min": "1", + "max": "2000" + }, + "3": { + "min": "54060", + "max": "114059" + }, + "15": { + "min": "1", + "max": "7" + }, + "16": { + "min": "1", + "max": "299" + }, + "7": { + "min": "1", + "max": "20" + }, + "9": { + "min": "82530", + "max": "9479950" + }, + "10": { + "min": "1992-01-01", + "max": "1998-08-02" + } + }, + "parameters": null, + "dictionaries": null, + "snapshots": null, + "last_build_time": 1679586425865, + "source_count": 60175, + "source_bytes_size": 5897747, + "column_source_bytes": { + "SSB.LINEORDER.LO_LINENUMBER": 60175, + "SSB.LINEORDER.LO_ORDERKEY": 172702, + "SSB.LINEORDER.LO_ORDERDATE": 601750, + "SSB.LINEORDER.LO_REVENUE": 412258, + "SSB.LINEORDER.LO_SUPPKEY": 94173, + "SSB.LINEORDER.LO_PARTKEY": 207663, + "SSB.LINEORDER.LO_CUSTKEY": 155432, + "SSB.LINEORDER.LO_SUPPLYCOST": 316159 + }, + "ori_snapshot_size": {}, + "additionalInfo": {}, + "is_realtime_segment": false, + "is_snapshot_ready": false, + "is_dict_ready": false, + "is_flat_table_ready": true, + "is_fact_view_ready": false, + "multi_partitions": [], + "max_bucket_id": -1 + } + ] +} diff --git a/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/dataflow/c5eecd91-0b8a-f8d7-89c8-c0afdaa01800.json b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/dataflow/c5eecd91-0b8a-f8d7-89c8-c0afdaa01800.json new file mode 100644 index 0000000000..c60c21d255 --- /dev/null +++ b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/dataflow/c5eecd91-0b8a-f8d7-89c8-c0afdaa01800.json @@ -0,0 +1,73 @@ +{ + "uuid" : "c5eecd91-0b8a-f8d7-89c8-c0afdaa01800", + "last_modified" : 1678679278043, + "create_time" : 1678677245818, + "version" : "4.0.0.0", + "status" : "ONLINE", + "last_status" : null, + "cost" : 50, + "query_hit_count" : 0, + "last_query_time" : 0, + "layout_query_hit_count" : { }, + "segments" : [ { + "id" : "d36576c3-8ca1-a567-e54d-5d4062d3b742", + "name" : "FULL_BUILD", + "create_time_utc" : 1678678578344, + "status" : "READY", + "segRange" : { + "@class" : "org.apache.kylin.metadata.model.SegmentRange$TimePartitionedSegmentRange", + "date_range_start" : 0, + "date_range_end" : 9223372036854775807 + }, + "timeRange" : null, + "dimension_range_info_map" : { + "0" : { + "min" : "1", + "max" : "60000" + }, + "1" : { + "min" : "1", + "max" : "2000" + }, + "13" : { + "min" : "1", + "max" : "50" + }, + "15" : { + "min" : "1", + "max" : "7" + }, + "16" : { + "min" : "1", + "max" : "299" + }, + "8" : { + "min" : "87489", + "max" : "46600128" + } + }, + "parameters" : null, + "dictionaries" : null, + "snapshots" : null, + "last_build_time" : 1679571652315, + "source_count" : 60175, + "source_bytes_size" : 5659955, + "column_source_bytes" : { + "SSB.LINEORDER.LO_LINENUMBER" : 60175, + "SSB.LINEORDER.LO_QUANTITY" : 109578, + "SSB.LINEORDER.LO_ORDERKEY" : 172702, + "SSB.LINEORDER.LO_PARTKEY" : 207663, + "SSB.LINEORDER.LO_ORDTOTALPRICE" : 469425, + "SSB.LINEORDER.LO_CUSTKEY" : 155432 + }, + "ori_snapshot_size" : { }, + "additionalInfo" : { }, + "is_realtime_segment" : false, + "is_snapshot_ready" : false, + "is_dict_ready" : true, + "is_flat_table_ready" : true, + "is_fact_view_ready" : false, + "multi_partitions" : [ ], + "max_bucket_id" : -1 + } ] +} diff --git a/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/dataflow_details/39e41387-c17e-1014-53cc-3d4c7b78e286/7b16f7c7-7648-b41c-6a7a-7188660370b5.json b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/dataflow_details/39e41387-c17e-1014-53cc-3d4c7b78e286/7b16f7c7-7648-b41c-6a7a-7188660370b5.json new file mode 100644 index 0000000000..77e21dd04f --- /dev/null +++ b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/dataflow_details/39e41387-c17e-1014-53cc-3d4c7b78e286/7b16f7c7-7648-b41c-6a7a-7188660370b5.json @@ -0,0 +1,36 @@ +{ + "uuid" : "7b16f7c7-7648-b41c-6a7a-7188660370b5", + "last_modified" : 1678694113559, + "create_time" : 1678694113559, + "version" : "4.0.0.0", + "dataflow" : "39e41387-c17e-1014-53cc-3d4c7b78e286", + "layout_instances" : [ { + "layout_id" : 1, + "build_job_id" : "5660d5d6-5cb0-4cf9-19e2-7ad9f034dd0c-39e41387-c17e-1014-53cc-3d4c7b78e286", + "rows" : 14293, + "byte_size" : 156478, + "file_count" : 1, + "source_rows" : 14293, + "source_byte_size" : 0, + "partition_num" : 1, + "partition_values" : [ ], + "is_ready" : false, + "create_time" : 1678694458193, + "multi_partition" : [ ], + "abnormal_type" : null + }, { + "layout_id" : 20000000001, + "build_job_id" : "5660d5d6-5cb0-4cf9-19e2-7ad9f034dd0c-39e41387-c17e-1014-53cc-3d4c7b78e286", + "rows" : 14293, + "byte_size" : 156154, + "file_count" : 1, + "source_rows" : 14293, + "source_byte_size" : 0, + "partition_num" : 1, + "partition_values" : [ ], + "is_ready" : false, + "create_time" : 1678694458193, + "multi_partition" : [ ], + "abnormal_type" : null + } ] +} diff --git a/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/dataflow_details/39e41387-c17e-1014-53cc-3d4c7b78e286/7b8cf7c7-7648-b41c-623a-7188660370b5.json b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/dataflow_details/39e41387-c17e-1014-53cc-3d4c7b78e286/7b8cf7c7-7648-b41c-623a-7188660370b5.json new file mode 100644 index 0000000000..fa9400e2b2 --- /dev/null +++ b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/dataflow_details/39e41387-c17e-1014-53cc-3d4c7b78e286/7b8cf7c7-7648-b41c-623a-7188660370b5.json @@ -0,0 +1,36 @@ +{ + "uuid" : "7b16f7c7-7648-b41c-6a7a-7188660370b5", + "last_modified" : 1678694113559, + "create_time" : 1678694113559, + "version" : "4.0.0.0", + "dataflow" : "39e41387-c17e-1014-53cc-3d4c7b78e286", + "layout_instances" : [ { + "layout_id" : 1, + "build_job_id" : "5607d5d6-5cb0-4cf9-19e2-7ad9f034dd0c-39e41387-c17e-1014-53cc-3d4c7b78e286", + "rows" : 14293, + "byte_size" : 156478, + "file_count" : 1, + "source_rows" : 14293, + "source_byte_size" : 0, + "partition_num" : 1, + "partition_values" : [ ], + "is_ready" : false, + "create_time" : 1678694458193, + "multi_partition" : [ ], + "abnormal_type" : null + }, { + "layout_id" : 20000000001, + "build_job_id" : "5607d5d6-5cb0-4cf9-19e2-7ad9f034dd0c-39e41387-c17e-1014-53cc-3d4c7b78e286", + "rows" : 14293, + "byte_size" : 156154, + "file_count" : 1, + "source_rows" : 14293, + "source_byte_size" : 0, + "partition_num" : 1, + "partition_values" : [ ], + "is_ready" : false, + "create_time" : 1678694458193, + "multi_partition" : [ ], + "abnormal_type" : null + } ] +} diff --git a/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/dataflow_details/4623e6d3-2ca2-319e-9a3f-e26bd819734f/c183b315-eb0e-3eac-8607-73c5b1552bce.json b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/dataflow_details/4623e6d3-2ca2-319e-9a3f-e26bd819734f/c183b315-eb0e-3eac-8607-73c5b1552bce.json new file mode 100644 index 0000000000..dfaa40fee8 --- /dev/null +++ b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/dataflow_details/4623e6d3-2ca2-319e-9a3f-e26bd819734f/c183b315-eb0e-3eac-8607-73c5b1552bce.json @@ -0,0 +1,36 @@ +{ + "uuid" : "c183b315-eb0e-3eac-8607-73c5b1552bce", + "last_modified" : 1678693815468, + "create_time" : 1678693815468, + "version" : "4.0.0.0", + "dataflow" : "4623e6d3-2ca2-319e-9a3f-e26bd819734f", + "layout_instances" : [ { + "layout_id" : 1, + "build_job_id" : "baf39b32-703e-adf1-4215-d4752172a60f-4623e6d3-2ca2-319e-9a3f-e26bd819734f", + "rows" : 60175, + "byte_size" : 1139420, + "file_count" : 1, + "source_rows" : 60175, + "source_byte_size" : 0, + "partition_num" : 1, + "partition_values" : [ ], + "is_ready" : false, + "create_time" : 1678694301324, + "multi_partition" : [ ], + "abnormal_type" : null + }, { + "layout_id" : 20000000001, + "build_job_id" : "baf39b32-703e-adf1-4215-d4752172a60f-4623e6d3-2ca2-319e-9a3f-e26bd819734f", + "rows" : 60175, + "byte_size" : 961903, + "file_count" : 1, + "source_rows" : 60175, + "source_byte_size" : 0, + "partition_num" : 1, + "partition_values" : [ ], + "is_ready" : false, + "create_time" : 1678694301324, + "multi_partition" : [ ], + "abnormal_type" : null + } ] +} diff --git a/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/dataflow_details/8a21c1db-f6b3-36cc-e454-84135de78352/d5768bf0-978f-9a46-e295-2026484744ff.json b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/dataflow_details/8a21c1db-f6b3-36cc-e454-84135de78352/d5768bf0-978f-9a46-e295-2026484744ff.json new file mode 100644 index 0000000000..8c675921f7 --- /dev/null +++ b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/dataflow_details/8a21c1db-f6b3-36cc-e454-84135de78352/d5768bf0-978f-9a46-e295-2026484744ff.json @@ -0,0 +1,36 @@ +{ + "uuid" : "d5768bf0-978f-9a46-e295-2026484744ff", + "last_modified" : 1678693307910, + "create_time" : 1678693307910, + "version" : "4.0.0.0", + "dataflow" : "8a21c1db-f6b3-36cc-e454-84135de78352", + "layout_instances" : [ { + "layout_id" : 1, + "build_job_id" : "f476d270-b049-9256-8d67-999a67e8b879-8a21c1db-f6b3-36cc-e454-84135de78352", + "rows" : 60175, + "byte_size" : 819206, + "file_count" : 1, + "source_rows" : 60175, + "source_byte_size" : 0, + "partition_num" : 1, + "partition_values" : [ ], + "is_ready" : false, + "create_time" : 1678693557277, + "multi_partition" : [ ], + "abnormal_type" : null + }, { + "layout_id" : 20000000001, + "build_job_id" : "f476d270-b049-9256-8d67-999a67e8b879-8a21c1db-f6b3-36cc-e454-84135de78352", + "rows" : 60175, + "byte_size" : 818694, + "file_count" : 1, + "source_rows" : 60175, + "source_byte_size" : 0, + "partition_num" : 1, + "partition_values" : [ ], + "is_ready" : false, + "create_time" : 1678693557276, + "multi_partition" : [ ], + "abnormal_type" : null + } ] +} diff --git a/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/dataflow_details/c5eecd91-0b8a-f8d7-89c8-c0afdaa01800/d36576c3-8ca1-a567-e54d-5d4062d3b742.json b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/dataflow_details/c5eecd91-0b8a-f8d7-89c8-c0afdaa01800/d36576c3-8ca1-a567-e54d-5d4062d3b742.json new file mode 100644 index 0000000000..d410dcf86f --- /dev/null +++ b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/dataflow_details/c5eecd91-0b8a-f8d7-89c8-c0afdaa01800/d36576c3-8ca1-a567-e54d-5d4062d3b742.json @@ -0,0 +1,50 @@ +{ + "uuid" : "d36576c3-8ca1-a567-e54d-5d4062d3b742", + "last_modified" : 1678678578368, + "create_time" : 1678678578367, + "version" : "4.0.0.0", + "dataflow" : "c5eecd91-0b8a-f8d7-89c8-c0afdaa01800", + "layout_instances" : [ { + "layout_id" : 1, + "build_job_id" : "dffcf8f3-982f-40a6-504b-289dd47fe4ca-c5eecd91-0b8a-f8d7-89c8-c0afdaa01800", + "rows" : 60175, + "byte_size" : 526029, + "file_count" : 1, + "source_rows" : 60175, + "source_byte_size" : 0, + "partition_num" : 1, + "partition_values" : [ ], + "is_ready" : false, + "create_time" : 1678678774644, + "multi_partition" : [ ], + "abnormal_type" : null + },{ + "layout_id" : 10001, + "build_job_id" : "dffcf8f3-982f-40a6-504b-289dd47fe4ca-c5eecd91-0b8a-f8d7-89c8-c0afdaa01800", + "rows" : 60175, + "byte_size" : 526029, + "file_count" : 1, + "source_rows" : 60175, + "source_byte_size" : 0, + "partition_num" : 1, + "partition_values" : [ ], + "is_ready" : false, + "create_time" : 1678678774644, + "multi_partition" : [ ], + "abnormal_type" : null + }, { + "layout_id" : 20000000001, + "build_job_id" : "dffcf8f3-982f-40a6-504b-289dd47fe4ca-c5eecd91-0b8a-f8d7-89c8-c0afdaa01800", + "rows" : 60175, + "byte_size" : 525521, + "file_count" : 1, + "source_rows" : 60175, + "source_byte_size" : 0, + "partition_num" : 1, + "partition_values" : [ ], + "is_ready" : false, + "create_time" : 1678678774644, + "multi_partition" : [ ], + "abnormal_type" : null + } ] +} diff --git a/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/index_plan/39e41387-c17e-1014-53cc-3d4c7b78e286.json b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/index_plan/39e41387-c17e-1014-53cc-3d4c7b78e286.json new file mode 100644 index 0000000000..33342818eb --- /dev/null +++ b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/index_plan/39e41387-c17e-1014-53cc-3d4c7b78e286.json @@ -0,0 +1,63 @@ +{ + "uuid" : "39e41387-c17e-1014-53cc-3d4c7b78e286", + "last_modified" : 1678693554349, + "create_time" : 1678693554346, + "version" : "4.0.0.0", + "description" : null, + "rule_based_index" : null, + "indexes" : [ { + "id" : 0, + "dimensions" : [ 0, 1, 10, 11, 12, 15, 16, 17, 22 ], + "measures" : [ 100000, 100001, 100002 ], + "layouts" : [ { + "id" : 1, + "name" : null, + "owner" : null, + "col_order" : [ 0, 1, 10, 11, 12, 15, 16, 17, 22, 100000, 100001, 100002 ], + "shard_by_columns" : [ ], + "partition_by_columns" : [ ], + "sort_by_columns" : [ ], + "storage_type" : 20, + "update_time" : 1678693554346, + "manual" : false, + "auto" : false, + "base" : true, + "draft_version" : null, + "index_range" : null + } ], + "next_layout_offset" : 2 + }, { + "id" : 20000000000, + "dimensions" : [ 0, 1, 3, 6, 10, 11, 12, 15, 16, 17, 22 ], + "measures" : [ ], + "layouts" : [ { + "id" : 20000000001, + "name" : null, + "owner" : null, + "col_order" : [ 0, 1, 3, 6, 10, 11, 12, 15, 16, 17, 22 ], + "shard_by_columns" : [ ], + "partition_by_columns" : [ ], + "sort_by_columns" : [ ], + "storage_type" : 20, + "update_time" : 1678693554347, + "manual" : false, + "auto" : false, + "base" : true, + "draft_version" : null, + "index_range" : null + } ], + "next_layout_offset" : 2 + } ], + "override_properties" : { }, + "to_be_deleted_indexes" : [ ], + "auto_merge_time_ranges" : null, + "retention_range" : 0, + "engine_type" : 80, + "next_aggregation_index_id" : 10000, + "next_table_index_id" : 20000010000, + "agg_shard_by_columns" : [ ], + "extend_partition_columns" : [ ], + "layout_bucket_num" : { }, + "approved_additional_recs" : 0, + "approved_removal_recs" : 0 +} diff --git a/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/index_plan/4623e6d3-2ca2-319e-9a3f-e26bd819734f.json b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/index_plan/4623e6d3-2ca2-319e-9a3f-e26bd819734f.json new file mode 100644 index 0000000000..b3526c844e --- /dev/null +++ b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/index_plan/4623e6d3-2ca2-319e-9a3f-e26bd819734f.json @@ -0,0 +1,63 @@ +{ + "uuid" : "4623e6d3-2ca2-319e-9a3f-e26bd819734f", + "last_modified" : 1678693815467, + "create_time" : 1678693815466, + "version" : "4.0.0.0", + "description" : null, + "rule_based_index" : null, + "indexes" : [ { + "id" : 0, + "dimensions" : [ 0, 1, 7, 8, 10, 12, 15, 16, 18, 21 ], + "measures" : [ 100000, 100001, 100002 ], + "layouts" : [ { + "id" : 1, + "name" : null, + "owner" : null, + "col_order" : [ 0, 1, 7, 8, 10, 12, 15, 16, 18, 21, 100000, 100001, 100002 ], + "shard_by_columns" : [ ], + "partition_by_columns" : [ ], + "sort_by_columns" : [ ], + "storage_type" : 20, + "update_time" : 1678693815466, + "manual" : false, + "auto" : false, + "base" : true, + "draft_version" : null, + "index_range" : null + } ], + "next_layout_offset" : 2 + }, { + "id" : 20000000000, + "dimensions" : [ 0, 1, 7, 8, 9, 10, 12, 15, 16, 18, 21 ], + "measures" : [ ], + "layouts" : [ { + "id" : 20000000001, + "name" : null, + "owner" : null, + "col_order" : [ 0, 1, 7, 8, 9, 10, 12, 15, 16, 18, 21 ], + "shard_by_columns" : [ ], + "partition_by_columns" : [ ], + "sort_by_columns" : [ ], + "storage_type" : 20, + "update_time" : 1678693815466, + "manual" : false, + "auto" : false, + "base" : true, + "draft_version" : null, + "index_range" : null + } ], + "next_layout_offset" : 2 + } ], + "override_properties" : { }, + "to_be_deleted_indexes" : [ ], + "auto_merge_time_ranges" : null, + "retention_range" : 0, + "engine_type" : 80, + "next_aggregation_index_id" : 10000, + "next_table_index_id" : 20000010000, + "agg_shard_by_columns" : [ ], + "extend_partition_columns" : [ ], + "layout_bucket_num" : { }, + "approved_additional_recs" : 0, + "approved_removal_recs" : 0 +} diff --git a/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/index_plan/8a21c1db-f6b3-36cc-e454-84135de78352.json b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/index_plan/8a21c1db-f6b3-36cc-e454-84135de78352.json new file mode 100644 index 0000000000..0cfcfb1a5b --- /dev/null +++ b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/index_plan/8a21c1db-f6b3-36cc-e454-84135de78352.json @@ -0,0 +1,63 @@ +{ + "uuid" : "8a21c1db-f6b3-36cc-e454-84135de78352", + "last_modified" : 1678693307904, + "create_time" : 1678693307838, + "version" : "4.0.0.0", + "description" : null, + "rule_based_index" : null, + "indexes" : [ { + "id" : 0, + "dimensions" : [ 0, 1, 7, 10, 15, 16 ], + "measures" : [ 100000, 100001, 100002 ], + "layouts" : [ { + "id" : 1, + "name" : null, + "owner" : null, + "col_order" : [ 0, 1, 7, 10, 15, 16, 100000, 100001, 100002 ], + "shard_by_columns" : [ ], + "partition_by_columns" : [ ], + "sort_by_columns" : [ ], + "storage_type" : 20, + "update_time" : 1678693307856, + "manual" : false, + "auto" : false, + "base" : true, + "draft_version" : null, + "index_range" : null + } ], + "next_layout_offset" : 2 + }, { + "id" : 20000000000, + "dimensions" : [ 0, 1, 3, 7, 9, 10, 15, 16 ], + "measures" : [ ], + "layouts" : [ { + "id" : 20000000001, + "name" : null, + "owner" : null, + "col_order" : [ 0, 1, 3, 7, 9, 10, 15, 16 ], + "shard_by_columns" : [ ], + "partition_by_columns" : [ ], + "sort_by_columns" : [ ], + "storage_type" : 20, + "update_time" : 1678693307861, + "manual" : false, + "auto" : false, + "base" : true, + "draft_version" : null, + "index_range" : null + } ], + "next_layout_offset" : 2 + } ], + "override_properties" : { }, + "to_be_deleted_indexes" : [ ], + "auto_merge_time_ranges" : null, + "retention_range" : 0, + "engine_type" : 80, + "next_aggregation_index_id" : 10000, + "next_table_index_id" : 20000010000, + "agg_shard_by_columns" : [ ], + "extend_partition_columns" : [ ], + "layout_bucket_num" : { }, + "approved_additional_recs" : 0, + "approved_removal_recs" : 0 +} diff --git a/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/index_plan/c5eecd91-0b8a-f8d7-89c8-c0afdaa01800.json b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/index_plan/c5eecd91-0b8a-f8d7-89c8-c0afdaa01800.json new file mode 100644 index 0000000000..71d5ccc626 --- /dev/null +++ b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/index_plan/c5eecd91-0b8a-f8d7-89c8-c0afdaa01800.json @@ -0,0 +1,84 @@ +{ + "uuid" : "c5eecd91-0b8a-f8d7-89c8-c0afdaa01800", + "last_modified" : 1678677245813, + "create_time" : 1678677245536, + "version" : "4.0.0.0", + "description" : null, + "rule_based_index" : null, + "indexes" : [ { + "id" : 0, + "dimensions" : [ 0, 1, 15, 16 ], + "measures" : [ 100000, 100001, 100002 ], + "layouts" : [ { + "id" : 1, + "name" : null, + "owner" : null, + "col_order" : [ 0, 1, 15, 16, 100000, 100001, 100002 ], + "shard_by_columns" : [ ], + "partition_by_columns" : [ ], + "sort_by_columns" : [ ], + "storage_type" : 20, + "update_time" : 1678677245565, + "manual" : false, + "auto" : false, + "base" : true, + "draft_version" : null, + "index_range" : null + } ], + "next_layout_offset" : 2 + },{ + "id" : 10000, + "dimensions" : [ 1], + "measures" : [ 100000, 100003 ], + "layouts" : [ { + "id" : 10001, + "name" : null, + "owner" : null, + "col_order" : [ 1, 100000, 100003 ], + "shard_by_columns" : [ ], + "partition_by_columns" : [ ], + "sort_by_columns" : [ ], + "storage_type" : 20, + "update_time" : 1678677245565, + "manual" : false, + "auto" : true, + "base" : false, + "draft_version" : null, + "index_range" : null + } ], + "next_layout_offset" : 2 + }, { + "id" : 20000000000, + "dimensions" : [ 0, 1, 8, 13, 15, 16 ], + "measures" : [ ], + "layouts" : [ { + "id" : 20000000001, + "name" : null, + "owner" : null, + "col_order" : [ 0, 1, 8, 13, 15, 16 ], + "shard_by_columns" : [ ], + "partition_by_columns" : [ ], + "sort_by_columns" : [ ], + "storage_type" : 20, + "update_time" : 1678677245570, + "manual" : true, + "auto" : false, + "base" : false, + "draft_version" : null, + "index_range" : null + } ], + "next_layout_offset" : 2 + } ], + "override_properties" : { }, + "to_be_deleted_indexes" : [ ], + "auto_merge_time_ranges" : null, + "retention_range" : 0, + "engine_type" : 80, + "next_aggregation_index_id" : 20000, + "next_table_index_id" : 20000010000, + "agg_shard_by_columns" : [ ], + "extend_partition_columns" : [ ], + "layout_bucket_num" : { }, + "approved_additional_recs" : 0, + "approved_removal_recs" : 0 +} diff --git a/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/model_desc/39e41387-c17e-1014-53cc-3d4c7b78e286.json b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/model_desc/39e41387-c17e-1014-53cc-3d4c7b78e286.json new file mode 100644 index 0000000000..a59c997cc2 --- /dev/null +++ b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/model_desc/39e41387-c17e-1014-53cc-3d4c7b78e286.json @@ -0,0 +1,227 @@ +{ + "uuid" : "39e41387-c17e-1014-53cc-3d4c7b78e286", + "last_modified" : 1678693554345, + "create_time" : 1678693553274, + "version" : "4.0.0.0", + "alias" : "lineorder_join_customer", + "owner" : "ADMIN", + "config_last_modifier" : null, + "config_last_modified" : 0, + "description" : "", + "fact_table" : "SSB.LINEORDER", + "fact_table_alias" : null, + "management_type" : "MODEL_BASED", + "join_tables" : [ { + "table" : "SSB.CUSTOMER", + "kind" : "LOOKUP", + "alias" : "CUSTOMER", + "join" : { + "type" : "INNER", + "primary_key" : [ "CUSTOMER.C_CUSTKEY" ], + "foreign_key" : [ "LINEORDER.LO_CUSTKEY" ], + "non_equi_join_condition" : null, + "primary_table" : null, + "foreign_table" : null + }, + "flattenable" : "flatten", + "join_relation_type" : "MANY_TO_ONE" + } ], + "filter_condition" : "", + "partition_desc" : { + "partition_date_column" : "LINEORDER.LO_ORDERDATE", + "partition_date_start" : 0, + "partition_date_format" : "yyyy-MM-dd", + "partition_type" : "APPEND", + "partition_condition_builder" : "org.apache.kylin.metadata.model.PartitionDesc$DefaultPartitionConditionBuilder" + }, + "capacity" : "MEDIUM", + "segment_config" : { + "auto_merge_enabled" : null, + "auto_merge_time_ranges" : null, + "volatile_range" : null, + "retention_range" : null, + "create_empty_segment_enabled" : false + }, + "data_check_desc" : null, + "semantic_version" : 0, + "storage_type" : 0, + "model_type" : "BATCH", + "all_named_columns" : [ { + "id" : 0, + "name" : "LO_ORDERKEY", + "column" : "LINEORDER.LO_ORDERKEY", + "status" : "DIMENSION" + }, { + "id" : 1, + "name" : "LO_PARTKEY", + "column" : "LINEORDER.LO_PARTKEY", + "status" : "DIMENSION" + }, { + "id" : 2, + "name" : "LO_DISCOUNT", + "column" : "LINEORDER.LO_DISCOUNT" + }, { + "id" : 3, + "name" : "LO_SUPPLYCOST", + "column" : "LINEORDER.LO_SUPPLYCOST" + }, { + "id" : 4, + "name" : "LO_COMMITDATE", + "column" : "LINEORDER.LO_COMMITDATE" + }, { + "id" : 5, + "name" : "LO_EXTENDEDPRICE", + "column" : "LINEORDER.LO_EXTENDEDPRICE" + }, { + "id" : 6, + "name" : "LO_TAX", + "column" : "LINEORDER.LO_TAX" + }, { + "id" : 7, + "name" : "LO_SUPPKEY", + "column" : "LINEORDER.LO_SUPPKEY" + }, { + "id" : 8, + "name" : "LO_ORDTOTALPRICE", + "column" : "LINEORDER.LO_ORDTOTALPRICE" + }, { + "id" : 9, + "name" : "LO_REVENUE", + "column" : "LINEORDER.LO_REVENUE" + }, { + "id" : 10, + "name" : "LO_ORDERDATE", + "column" : "LINEORDER.LO_ORDERDATE", + "status" : "DIMENSION" + }, { + "id" : 11, + "name" : "LO_ORDERPRIOTITY", + "column" : "LINEORDER.LO_ORDERPRIOTITY", + "status" : "DIMENSION" + }, { + "id" : 12, + "name" : "LO_SHIPPRIOTITY", + "column" : "LINEORDER.LO_SHIPPRIOTITY", + "status" : "DIMENSION" + }, { + "id" : 13, + "name" : "LO_QUANTITY", + "column" : "LINEORDER.LO_QUANTITY" + }, { + "id" : 14, + "name" : "LO_SHIPMODE", + "column" : "LINEORDER.LO_SHIPMODE" + }, { + "id" : 15, + "name" : "LO_LINENUMBER", + "column" : "LINEORDER.LO_LINENUMBER", + "status" : "DIMENSION" + }, { + "id" : 16, + "name" : "LO_CUSTKEY", + "column" : "LINEORDER.LO_CUSTKEY", + "status" : "DIMENSION" + }, { + "id" : 17, + "name" : "C_ADDRESS", + "column" : "CUSTOMER.C_ADDRESS", + "status" : "DIMENSION" + }, { + "id" : 18, + "name" : "C_NATION", + "column" : "CUSTOMER.C_NATION" + }, { + "id" : 19, + "name" : "C_CITY", + "column" : "CUSTOMER.C_CITY" + }, { + "id" : 20, + "name" : "C_PHONE", + "column" : "CUSTOMER.C_PHONE" + }, { + "id" : 21, + "name" : "C_REGION", + "column" : "CUSTOMER.C_REGION" + }, { + "id" : 22, + "name" : "C_NAME", + "column" : "CUSTOMER.C_NAME", + "status" : "DIMENSION" + }, { + "id" : 23, + "name" : "C_MKTSEGMENT", + "column" : "CUSTOMER.C_MKTSEGMENT" + }, { + "id" : 24, + "name" : "C_CUSTKEY", + "column" : "CUSTOMER.C_CUSTKEY" + } ], + "all_measures" : [ { + "name" : "COUNT_ALL", + "function" : { + "expression" : "COUNT", + "parameters" : [ { + "type" : "constant", + "value" : "1" + } ], + "returntype" : "bigint" + }, + "column" : null, + "comment" : null, + "id" : 100000, + "type" : "NORMAL", + "internal_ids" : [ ] + }, { + "name" : "LO_SUPPLYCOST_MIN", + "function" : { + "expression" : "MIN", + "parameters" : [ { + "type" : "column", + "value" : "LINEORDER.LO_SUPPLYCOST" + } ], + "returntype" : "bigint" + }, + "column" : null, + "comment" : "", + "id" : 100001, + "type" : "NORMAL", + "internal_ids" : [ ] + }, { + "name" : "LO_TAX_SUM", + "function" : { + "expression" : "SUM", + "parameters" : [ { + "type" : "column", + "value" : "LINEORDER.LO_TAX" + } ], + "returntype" : "bigint" + }, + "column" : null, + "comment" : "", + "id" : 100002, + "type" : "NORMAL", + "internal_ids" : [ ] + } ], + "recommendations_count" : 0, + "computed_columns" : [ ], + "canvas" : { + "coordinate" : { + "LINEORDER" : { + "x" : 347.33330620659723, + "y" : 123.38887532552084, + "width" : 200.0, + "height" : 230.0 + }, + "CUSTOMER" : { + "x" : 873.9999728732639, + "y" : 124.49998643663196, + "width" : 200.0, + "height" : 230.0 + } + }, + "zoom" : 9.0 + }, + "multi_partition_desc" : null, + "multi_partition_key_mapping" : null, + "fusion_id" : null +} diff --git a/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/model_desc/4623e6d3-2ca2-319e-9a3f-e26bd819734f.json b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/model_desc/4623e6d3-2ca2-319e-9a3f-e26bd819734f.json new file mode 100644 index 0000000000..7648410f73 --- /dev/null +++ b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/model_desc/4623e6d3-2ca2-319e-9a3f-e26bd819734f.json @@ -0,0 +1,222 @@ +{ + "uuid" : "4623e6d3-2ca2-319e-9a3f-e26bd819734f", + "last_modified" : 1678693815465, + "create_time" : 1678693814402, + "version" : "4.0.0.0", + "alias" : "lineorder_left_customer", + "owner" : "ADMIN", + "config_last_modifier" : null, + "config_last_modified" : 0, + "description" : "", + "fact_table" : "SSB.LINEORDER", + "fact_table_alias" : null, + "management_type" : "MODEL_BASED", + "join_tables" : [ { + "table" : "SSB.CUSTOMER", + "kind" : "LOOKUP", + "alias" : "CUSTOMER", + "join" : { + "type" : "LEFT", + "primary_key" : [ "CUSTOMER.C_CUSTKEY" ], + "foreign_key" : [ "LINEORDER.LO_CUSTKEY" ], + "non_equi_join_condition" : null, + "primary_table" : null, + "foreign_table" : null + }, + "flattenable" : "flatten", + "join_relation_type" : "MANY_TO_ONE" + } ], + "filter_condition" : "", + "partition_desc" : null, + "capacity" : "MEDIUM", + "segment_config" : { + "auto_merge_enabled" : null, + "auto_merge_time_ranges" : null, + "volatile_range" : null, + "retention_range" : null, + "create_empty_segment_enabled" : false + }, + "data_check_desc" : null, + "semantic_version" : 0, + "storage_type" : 0, + "model_type" : "BATCH", + "all_named_columns" : [ { + "id" : 0, + "name" : "LO_ORDERKEY", + "column" : "LINEORDER.LO_ORDERKEY", + "status" : "DIMENSION" + }, { + "id" : 1, + "name" : "LO_PARTKEY", + "column" : "LINEORDER.LO_PARTKEY", + "status" : "DIMENSION" + }, { + "id" : 2, + "name" : "LO_DISCOUNT", + "column" : "LINEORDER.LO_DISCOUNT" + }, { + "id" : 3, + "name" : "LO_SUPPLYCOST", + "column" : "LINEORDER.LO_SUPPLYCOST" + }, { + "id" : 4, + "name" : "LO_COMMITDATE", + "column" : "LINEORDER.LO_COMMITDATE" + }, { + "id" : 5, + "name" : "LO_EXTENDEDPRICE", + "column" : "LINEORDER.LO_EXTENDEDPRICE" + }, { + "id" : 6, + "name" : "LO_TAX", + "column" : "LINEORDER.LO_TAX" + }, { + "id" : 7, + "name" : "LO_SUPPKEY", + "column" : "LINEORDER.LO_SUPPKEY", + "status" : "DIMENSION" + }, { + "id" : 8, + "name" : "LO_ORDTOTALPRICE", + "column" : "LINEORDER.LO_ORDTOTALPRICE", + "status" : "DIMENSION" + }, { + "id" : 9, + "name" : "LO_REVENUE", + "column" : "LINEORDER.LO_REVENUE" + }, { + "id" : 10, + "name" : "LO_ORDERDATE", + "column" : "LINEORDER.LO_ORDERDATE", + "status" : "DIMENSION" + }, { + "id" : 11, + "name" : "LO_ORDERPRIOTITY", + "column" : "LINEORDER.LO_ORDERPRIOTITY" + }, { + "id" : 12, + "name" : "LO_SHIPPRIOTITY", + "column" : "LINEORDER.LO_SHIPPRIOTITY", + "status" : "DIMENSION" + }, { + "id" : 13, + "name" : "LO_QUANTITY", + "column" : "LINEORDER.LO_QUANTITY" + }, { + "id" : 14, + "name" : "LO_SHIPMODE", + "column" : "LINEORDER.LO_SHIPMODE" + }, { + "id" : 15, + "name" : "LO_LINENUMBER", + "column" : "LINEORDER.LO_LINENUMBER", + "status" : "DIMENSION" + }, { + "id" : 16, + "name" : "LO_CUSTKEY", + "column" : "LINEORDER.LO_CUSTKEY", + "status" : "DIMENSION" + }, { + "id" : 17, + "name" : "C_ADDRESS", + "column" : "CUSTOMER.C_ADDRESS" + }, { + "id" : 18, + "name" : "C_NATION", + "column" : "CUSTOMER.C_NATION", + "status" : "DIMENSION" + }, { + "id" : 19, + "name" : "C_CITY", + "column" : "CUSTOMER.C_CITY" + }, { + "id" : 20, + "name" : "C_PHONE", + "column" : "CUSTOMER.C_PHONE" + }, { + "id" : 21, + "name" : "C_REGION", + "column" : "CUSTOMER.C_REGION", + "status" : "DIMENSION" + }, { + "id" : 22, + "name" : "C_NAME", + "column" : "CUSTOMER.C_NAME" + }, { + "id" : 23, + "name" : "C_MKTSEGMENT", + "column" : "CUSTOMER.C_MKTSEGMENT" + }, { + "id" : 24, + "name" : "C_CUSTKEY", + "column" : "CUSTOMER.C_CUSTKEY" + } ], + "all_measures" : [ { + "name" : "COUNT_ALL", + "function" : { + "expression" : "COUNT", + "parameters" : [ { + "type" : "constant", + "value" : "1" + } ], + "returntype" : "bigint" + }, + "column" : null, + "comment" : null, + "id" : 100000, + "type" : "NORMAL", + "internal_ids" : [ ] + }, { + "name" : "LO_ORDTOTALPRICE_SUM", + "function" : { + "expression" : "SUM", + "parameters" : [ { + "type" : "column", + "value" : "LINEORDER.LO_ORDTOTALPRICE" + } ], + "returntype" : "bigint" + }, + "column" : null, + "comment" : "", + "id" : 100001, + "type" : "NORMAL", + "internal_ids" : [ ] + }, { + "name" : "LO_REVENUE_MAX", + "function" : { + "expression" : "MAX", + "parameters" : [ { + "type" : "column", + "value" : "LINEORDER.LO_REVENUE" + } ], + "returntype" : "bigint" + }, + "column" : null, + "comment" : "", + "id" : 100002, + "type" : "NORMAL", + "internal_ids" : [ ] + } ], + "recommendations_count" : 0, + "computed_columns" : [ ], + "canvas" : { + "coordinate" : { + "LINEORDER" : { + "x" : 466.22219509548614, + "y" : 196.72220865885419, + "width" : 200.0, + "height" : 230.0 + }, + "CUSTOMER" : { + "x" : 1005.111083984375, + "y" : 180.0555419921875, + "width" : 200.0, + "height" : 230.0 + } + }, + "zoom" : 9.0 + }, + "multi_partition_desc" : null, + "multi_partition_key_mapping" : null, + "fusion_id" : null +} diff --git a/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/model_desc/8a21c1db-f6b3-36cc-e454-84135de78352.json b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/model_desc/8a21c1db-f6b3-36cc-e454-84135de78352.json new file mode 100644 index 0000000000..94fd08267e --- /dev/null +++ b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/model_desc/8a21c1db-f6b3-36cc-e454-84135de78352.json @@ -0,0 +1,254 @@ +{ + "uuid" : "8a21c1db-f6b3-36cc-e454-84135de78352", + "last_modified" : 1678693307837, + "create_time" : 1678693303624, + "version" : "4.0.0.0", + "alias" : "lineorder_join_dates", + "owner" : "ADMIN", + "config_last_modifier" : null, + "config_last_modified" : 0, + "description" : "", + "fact_table" : "SSB.LINEORDER", + "fact_table_alias" : null, + "management_type" : "MODEL_BASED", + "join_tables" : [ { + "table" : "SSB.DATES", + "kind" : "LOOKUP", + "alias" : "DATES", + "join" : { + "type" : "INNER", + "primary_key" : [ "DATES.D_DATEKEY" ], + "foreign_key" : [ "LINEORDER.LO_ORDERDATE" ], + "non_equi_join_condition" : null, + "primary_table" : null, + "foreign_table" : null + }, + "flattenable" : "flatten", + "join_relation_type" : "MANY_TO_ONE" + } ], + "filter_condition" : "", + "partition_desc" : null, + "capacity" : "MEDIUM", + "segment_config" : { + "auto_merge_enabled" : null, + "auto_merge_time_ranges" : null, + "volatile_range" : null, + "retention_range" : null, + "create_empty_segment_enabled" : false + }, + "data_check_desc" : null, + "semantic_version" : 0, + "storage_type" : 0, + "model_type" : "BATCH", + "all_named_columns" : [ { + "id" : 0, + "name" : "LO_ORDERKEY", + "column" : "LINEORDER.LO_ORDERKEY", + "status" : "DIMENSION" + }, { + "id" : 1, + "name" : "LO_PARTKEY", + "column" : "LINEORDER.LO_PARTKEY", + "status" : "DIMENSION" + }, { + "id" : 2, + "name" : "LO_DISCOUNT", + "column" : "LINEORDER.LO_DISCOUNT" + }, { + "id" : 3, + "name" : "LO_SUPPLYCOST", + "column" : "LINEORDER.LO_SUPPLYCOST" + }, { + "id" : 4, + "name" : "LO_COMMITDATE", + "column" : "LINEORDER.LO_COMMITDATE" + }, { + "id" : 5, + "name" : "LO_EXTENDEDPRICE", + "column" : "LINEORDER.LO_EXTENDEDPRICE" + }, { + "id" : 6, + "name" : "LO_TAX", + "column" : "LINEORDER.LO_TAX" + }, { + "id" : 7, + "name" : "LO_SUPPKEY", + "column" : "LINEORDER.LO_SUPPKEY", + "status" : "DIMENSION" + }, { + "id" : 8, + "name" : "LO_ORDTOTALPRICE", + "column" : "LINEORDER.LO_ORDTOTALPRICE" + }, { + "id" : 9, + "name" : "LO_REVENUE", + "column" : "LINEORDER.LO_REVENUE" + }, { + "id" : 10, + "name" : "LO_ORDERDATE", + "column" : "LINEORDER.LO_ORDERDATE", + "status" : "DIMENSION" + }, { + "id" : 11, + "name" : "LO_ORDERPRIOTITY", + "column" : "LINEORDER.LO_ORDERPRIOTITY" + }, { + "id" : 12, + "name" : "LO_SHIPPRIOTITY", + "column" : "LINEORDER.LO_SHIPPRIOTITY" + }, { + "id" : 13, + "name" : "LO_QUANTITY", + "column" : "LINEORDER.LO_QUANTITY" + }, { + "id" : 14, + "name" : "LO_SHIPMODE", + "column" : "LINEORDER.LO_SHIPMODE" + }, { + "id" : 15, + "name" : "LO_LINENUMBER", + "column" : "LINEORDER.LO_LINENUMBER", + "status" : "DIMENSION" + }, { + "id" : 16, + "name" : "LO_CUSTKEY", + "column" : "LINEORDER.LO_CUSTKEY", + "status" : "DIMENSION" + }, { + "id" : 17, + "name" : "D_WEEKNUMINYEAR", + "column" : "DATES.D_WEEKNUMINYEAR" + }, { + "id" : 18, + "name" : "D_LASTDAYINWEEKFL", + "column" : "DATES.D_LASTDAYINWEEKFL" + }, { + "id" : 19, + "name" : "D_LASTDAYINMONTHFL", + "column" : "DATES.D_LASTDAYINMONTHFL" + }, { + "id" : 20, + "name" : "D_DAYOFWEEK", + "column" : "DATES.D_DAYOFWEEK" + }, { + "id" : 21, + "name" : "D_MONTHNUMINYEAR", + "column" : "DATES.D_MONTHNUMINYEAR" + }, { + "id" : 22, + "name" : "D_YEARMONTHNUM", + "column" : "DATES.D_YEARMONTHNUM" + }, { + "id" : 23, + "name" : "D_YEARMONTH", + "column" : "DATES.D_YEARMONTH" + }, { + "id" : 24, + "name" : "D_DAYNUMINMONTH", + "column" : "DATES.D_DAYNUMINMONTH" + }, { + "id" : 25, + "name" : "D_SELLINGSEASON", + "column" : "DATES.D_SELLINGSEASON" + }, { + "id" : 26, + "name" : "D_WEEKDAYFL", + "column" : "DATES.D_WEEKDAYFL" + }, { + "id" : 27, + "name" : "D_YEAR", + "column" : "DATES.D_YEAR" + }, { + "id" : 28, + "name" : "D_HOLIDAYFL", + "column" : "DATES.D_HOLIDAYFL" + }, { + "id" : 29, + "name" : "D_DAYNUMINWEEK", + "column" : "DATES.D_DAYNUMINWEEK" + }, { + "id" : 30, + "name" : "D_DAYNUMINYEAR", + "column" : "DATES.D_DAYNUMINYEAR" + }, { + "id" : 31, + "name" : "D_DATE", + "column" : "DATES.D_DATE" + }, { + "id" : 32, + "name" : "D_MONTH", + "column" : "DATES.D_MONTH" + }, { + "id" : 33, + "name" : "D_DATEKEY", + "column" : "DATES.D_DATEKEY" + } ], + "all_measures" : [ { + "name" : "COUNT_ALL", + "function" : { + "expression" : "COUNT", + "parameters" : [ { + "type" : "constant", + "value" : "1" + } ], + "returntype" : "bigint" + }, + "column" : null, + "comment" : null, + "id" : 100000, + "type" : "NORMAL", + "internal_ids" : [ ] + }, { + "name" : "LO_REVENUE_SUM", + "function" : { + "expression" : "SUM", + "parameters" : [ { + "type" : "column", + "value" : "LINEORDER.LO_REVENUE" + } ], + "returntype" : "bigint" + }, + "column" : null, + "comment" : "", + "id" : 100001, + "type" : "NORMAL", + "internal_ids" : [ ] + }, { + "name" : "LO_SUPPLYCOST_MIN", + "function" : { + "expression" : "MIN", + "parameters" : [ { + "type" : "column", + "value" : "LINEORDER.LO_SUPPLYCOST" + } ], + "returntype" : "bigint" + }, + "column" : null, + "comment" : "", + "id" : 100002, + "type" : "NORMAL", + "internal_ids" : [ ] + } ], + "recommendations_count" : 0, + "computed_columns" : [ ], + "canvas" : { + "coordinate" : { + "LINEORDER" : { + "x" : 376.22219509548614, + "y" : 305.61109754774304, + "width" : 200.0, + "height" : 230.0 + }, + "DATES" : { + "x" : 758.4444173177084, + "y" : 195.61109754774307, + "width" : 200.0, + "height" : 230.0 + } + }, + "zoom" : 9.0 + }, + "multi_partition_desc" : null, + "multi_partition_key_mapping" : null, + "fusion_id" : null +} diff --git a/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/model_desc/c5eecd91-0b8a-f8d7-89c8-c0afdaa01800.json b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/model_desc/c5eecd91-0b8a-f8d7-89c8-c0afdaa01800.json new file mode 100644 index 0000000000..a62ea66a8b --- /dev/null +++ b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/model_desc/c5eecd91-0b8a-f8d7-89c8-c0afdaa01800.json @@ -0,0 +1,179 @@ +{ + "uuid" : "c5eecd91-0b8a-f8d7-89c8-c0afdaa01800", + "last_modified" : 1678677245532, + "create_time" : 1678677235207, + "version" : "4.0.0.0", + "alias" : "lineorder", + "owner" : "ADMIN", + "config_last_modifier" : null, + "config_last_modified" : 0, + "description" : "", + "fact_table" : "SSB.LINEORDER", + "fact_table_alias" : null, + "management_type" : "MODEL_BASED", + "join_tables" : [ ], + "filter_condition" : "", + "partition_desc" : null, + "capacity" : "MEDIUM", + "segment_config" : { + "auto_merge_enabled" : null, + "auto_merge_time_ranges" : null, + "volatile_range" : null, + "retention_range" : null, + "create_empty_segment_enabled" : false + }, + "data_check_desc" : null, + "semantic_version" : 0, + "storage_type" : 0, + "model_type" : "BATCH", + "all_named_columns" : [ { + "id" : 0, + "name" : "LO_ORDERKEY", + "column" : "LINEORDER.LO_ORDERKEY", + "status" : "DIMENSION" + }, { + "id" : 1, + "name" : "LO_PARTKEY", + "column" : "LINEORDER.LO_PARTKEY", + "status" : "DIMENSION" + }, { + "id" : 2, + "name" : "LO_DISCOUNT", + "column" : "LINEORDER.LO_DISCOUNT" + }, { + "id" : 3, + "name" : "LO_SUPPLYCOST", + "column" : "LINEORDER.LO_SUPPLYCOST" + }, { + "id" : 4, + "name" : "LO_COMMITDATE", + "column" : "LINEORDER.LO_COMMITDATE" + }, { + "id" : 5, + "name" : "LO_EXTENDEDPRICE", + "column" : "LINEORDER.LO_EXTENDEDPRICE" + }, { + "id" : 6, + "name" : "LO_TAX", + "column" : "LINEORDER.LO_TAX" + }, { + "id" : 7, + "name" : "LO_SUPPKEY", + "column" : "LINEORDER.LO_SUPPKEY" + }, { + "id" : 8, + "name" : "LO_ORDTOTALPRICE", + "column" : "LINEORDER.LO_ORDTOTALPRICE" + }, { + "id" : 9, + "name" : "LO_REVENUE", + "column" : "LINEORDER.LO_REVENUE" + }, { + "id" : 10, + "name" : "LO_ORDERDATE", + "column" : "LINEORDER.LO_ORDERDATE" + }, { + "id" : 11, + "name" : "LO_ORDERPRIOTITY", + "column" : "LINEORDER.LO_ORDERPRIOTITY" + }, { + "id" : 12, + "name" : "LO_SHIPPRIOTITY", + "column" : "LINEORDER.LO_SHIPPRIOTITY" + }, { + "id" : 13, + "name" : "LO_QUANTITY", + "column" : "LINEORDER.LO_QUANTITY" + }, { + "id" : 14, + "name" : "LO_SHIPMODE", + "column" : "LINEORDER.LO_SHIPMODE" + }, { + "id" : 15, + "name" : "LO_LINENUMBER", + "column" : "LINEORDER.LO_LINENUMBER", + "status" : "DIMENSION" + }, { + "id" : 16, + "name" : "LO_CUSTKEY", + "column" : "LINEORDER.LO_CUSTKEY", + "status" : "DIMENSION" + } ], + "all_measures" : [ { + "name" : "COUNT_ALL", + "function" : { + "expression" : "COUNT", + "parameters" : [ { + "type" : "constant", + "value" : "1" + } ], + "returntype" : "bigint" + }, + "column" : null, + "comment" : null, + "id" : 100000, + "type" : "NORMAL", + "internal_ids" : [ ] + }, { + "name" : "LO_QUANTITY_SUM", + "function" : { + "expression" : "SUM", + "parameters" : [ { + "type" : "column", + "value" : "LINEORDER.LO_QUANTITY" + } ], + "returntype" : "bigint" + }, + "column" : null, + "comment" : "", + "id" : 100001, + "type" : "NORMAL", + "internal_ids" : [ ] + }, { + "name" : "LO_ORDTOTALPRICE_MIN", + "function" : { + "expression" : "MIN", + "parameters" : [ { + "type" : "column", + "value" : "LINEORDER.LO_ORDTOTALPRICE" + } ], + "returntype" : "bigint" + }, + "column" : null, + "comment" : "", + "id" : 100002, + "type" : "NORMAL", + "internal_ids" : [ ] + },{ + "name" : "LO_ORDERPRIOTITY_MIN", + "function" : { + "expression" : "MIN", + "parameters" : [ { + "type" : "column", + "value" : "LINEORDER.LO_ORDERPRIOTITY" + } ], + "returntype" : "bigint" + }, + "column" : null, + "comment" : "", + "id" : 100003, + "type" : "NORMAL", + "internal_ids" : [ ] + } ], + "recommendations_count" : 0, + "computed_columns" : [ ], + "canvas" : { + "coordinate" : { + "LINEORDER" : { + "x" : 392.88886176215277, + "y" : 150.16665988498264, + "width" : 200.0, + "height" : 230.0 + } + }, + "zoom" : 9.0 + }, + "multi_partition_desc" : null, + "multi_partition_key_mapping" : null, + "fusion_id" : null +} diff --git a/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/table/SSB.CUSTOMER.json b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/table/SSB.CUSTOMER.json new file mode 100644 index 0000000000..e604a3b378 --- /dev/null +++ b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/table/SSB.CUSTOMER.json @@ -0,0 +1,78 @@ +{ + "uuid": "f940ac1b-f41d-2189-0ff5-ca0a46d68b24", + "last_modified": 1678694363878, + "create_time": 1678676932844, + "version": "4.0.0.0", + "name": "CUSTOMER", + "columns": [ + { + "id": "1", + "name": "C_CUSTKEY", + "datatype": "integer", + "case_sensitive_name": "c_custkey" + }, + { + "id": "2", + "name": "C_NAME", + "datatype": "varchar(4096)", + "case_sensitive_name": "c_name" + }, + { + "id": "3", + "name": "C_ADDRESS", + "datatype": "varchar(4096)", + "case_sensitive_name": "c_address" + }, + { + "id": "4", + "name": "C_CITY", + "datatype": "varchar(4096)", + "case_sensitive_name": "c_city" + }, + { + "id": "5", + "name": "C_NATION", + "datatype": "varchar(4096)", + "case_sensitive_name": "c_nation" + }, + { + "id": "6", + "name": "C_REGION", + "datatype": "varchar(4096)", + "case_sensitive_name": "c_region" + }, + { + "id": "7", + "name": "C_PHONE", + "datatype": "varchar(4096)", + "case_sensitive_name": "c_phone" + }, + { + "id": "8", + "name": "C_MKTSEGMENT", + "datatype": "varchar(4096)", + "case_sensitive_name": "c_mktsegment" + } + ], + "source_type": 9, + "table_type": "EXTERNAL", + "top": false, + "increment_loading": false, + "last_snapshot_path": "min_max/table_snapshot/SSB.CUSTOMER/c734fe97-e1cf-46f9-8ba0-d22a12298138", + "last_snapshot_size": 16685, + "snapshot_last_modified": 1678694383206, + "query_hit_count": 0, + "partition_column": null, + "snapshot_partitions": {}, + "snapshot_partitions_info": {}, + "snapshot_total_rows": 0, + "snapshot_partition_col": null, + "selected_snapshot_partition_col": null, + "temp_snapshot_path": null, + "snapshot_has_broken": false, + "database": "SSB", + "transactional": false, + "rangePartition": false, + "partition_desc": null +} + diff --git a/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/table/SSB.DATES.json b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/table/SSB.DATES.json new file mode 100644 index 0000000000..ff78497e2e --- /dev/null +++ b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/table/SSB.DATES.json @@ -0,0 +1,131 @@ +{ + "uuid": "28d73613-a178-65eb-52fa-73c2077c58c7", + "last_modified": 1678676933560, + "create_time": 1678676932925, + "version": "4.0.0.0", + "name": "DATES", + "columns": [ + { + "id": "1", + "name": "D_DATEKEY", + "datatype": "date", + "case_sensitive_name": "d_datekey" + }, + { + "id": "2", + "name": "D_DATE", + "datatype": "varchar(4096)", + "case_sensitive_name": "d_date" + }, + { + "id": "3", + "name": "D_DAYOFWEEK", + "datatype": "varchar(4096)", + "case_sensitive_name": "d_dayofweek" + }, + { + "id": "4", + "name": "D_MONTH", + "datatype": "varchar(4096)", + "case_sensitive_name": "d_month" + }, + { + "id": "5", + "name": "D_YEAR", + "datatype": "integer", + "case_sensitive_name": "d_year" + }, + { + "id": "6", + "name": "D_YEARMONTHNUM", + "datatype": "integer", + "case_sensitive_name": "d_yearmonthnum" + }, + { + "id": "7", + "name": "D_YEARMONTH", + "datatype": "varchar(4096)", + "case_sensitive_name": "d_yearmonth" + }, + { + "id": "8", + "name": "D_DAYNUMINWEEK", + "datatype": "integer", + "case_sensitive_name": "d_daynuminweek" + }, + { + "id": "9", + "name": "D_DAYNUMINMONTH", + "datatype": "integer", + "case_sensitive_name": "d_daynuminmonth" + }, + { + "id": "10", + "name": "D_DAYNUMINYEAR", + "datatype": "integer", + "case_sensitive_name": "d_daynuminyear" + }, + { + "id": "11", + "name": "D_MONTHNUMINYEAR", + "datatype": "integer", + "case_sensitive_name": "d_monthnuminyear" + }, + { + "id": "12", + "name": "D_WEEKNUMINYEAR", + "datatype": "integer", + "case_sensitive_name": "d_weeknuminyear" + }, + { + "id": "13", + "name": "D_SELLINGSEASON", + "datatype": "varchar(4096)", + "case_sensitive_name": "d_sellingseason" + }, + { + "id": "14", + "name": "D_LASTDAYINWEEKFL", + "datatype": "integer", + "case_sensitive_name": "d_lastdayinweekfl" + }, + { + "id": "15", + "name": "D_LASTDAYINMONTHFL", + "datatype": "integer", + "case_sensitive_name": "d_lastdayinmonthfl" + }, + { + "id": "16", + "name": "D_HOLIDAYFL", + "datatype": "integer", + "case_sensitive_name": "d_holidayfl" + }, + { + "id": "17", + "name": "D_WEEKDAYFL", + "datatype": "integer", + "case_sensitive_name": "d_weekdayfl" + } + ], + "source_type": 9, + "table_type": "EXTERNAL", + "top": false, + "increment_loading": false, + "last_snapshot_path": "min_max/table_snapshot/SSB.DATES/3ccf9aee-773e-45fe-8076-bd4f52bbdfc1", + "last_snapshot_size": 54735, + "snapshot_last_modified": 1678693517971, + "query_hit_count": 0, + "partition_column": null, + "snapshot_partitions": {}, + "snapshot_partitions_info": {}, + "snapshot_total_rows": 0, + "snapshot_partition_col": null, + "selected_snapshot_partition_col": null, + "temp_snapshot_path": null, + "snapshot_has_broken": false, + "database": "SSB", + "transactional": false, + "rangePartition": false, + "partition_desc": null +} diff --git a/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/table/SSB.LINEORDER.json b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/table/SSB.LINEORDER.json new file mode 100644 index 0000000000..8e1aad95b8 --- /dev/null +++ b/src/kylin-it/src/test/resources/ut_meta/query_min_max_with_meta/metadata/min_max/table/SSB.LINEORDER.json @@ -0,0 +1,131 @@ +{ + "uuid": "08449e92-34d8-cbd3-260b-5d59790f652f", + "last_modified": 0, + "create_time": 1678676932824, + "version": "4.0.0.0", + "name": "LINEORDER", + "columns": [ + { + "id": "1", + "name": "LO_ORDERKEY", + "datatype": "bigint", + "case_sensitive_name": "lo_orderkey" + }, + { + "id": "2", + "name": "LO_LINENUMBER", + "datatype": "bigint", + "case_sensitive_name": "lo_linenumber" + }, + { + "id": "3", + "name": "LO_CUSTKEY", + "datatype": "integer", + "case_sensitive_name": "lo_custkey" + }, + { + "id": "4", + "name": "LO_PARTKEY", + "datatype": "integer", + "case_sensitive_name": "lo_partkey" + }, + { + "id": "5", + "name": "LO_SUPPKEY", + "datatype": "integer", + "case_sensitive_name": "lo_suppkey" + }, + { + "id": "6", + "name": "LO_ORDERDATE", + "datatype": "date", + "case_sensitive_name": "lo_orderdate" + }, + { + "id": "7", + "name": "LO_ORDERPRIOTITY", + "datatype": "varchar(4096)", + "case_sensitive_name": "lo_orderpriotity" + }, + { + "id": "8", + "name": "LO_SHIPPRIOTITY", + "datatype": "integer", + "case_sensitive_name": "lo_shippriotity" + }, + { + "id": "9", + "name": "LO_QUANTITY", + "datatype": "bigint", + "case_sensitive_name": "lo_quantity" + }, + { + "id": "10", + "name": "LO_EXTENDEDPRICE", + "datatype": "bigint", + "case_sensitive_name": "lo_extendedprice" + }, + { + "id": "11", + "name": "LO_ORDTOTALPRICE", + "datatype": "bigint", + "case_sensitive_name": "lo_ordtotalprice" + }, + { + "id": "12", + "name": "LO_DISCOUNT", + "datatype": "bigint", + "case_sensitive_name": "lo_discount" + }, + { + "id": "13", + "name": "LO_REVENUE", + "datatype": "bigint", + "case_sensitive_name": "lo_revenue" + }, + { + "id": "14", + "name": "LO_SUPPLYCOST", + "datatype": "bigint", + "case_sensitive_name": "lo_supplycost" + }, + { + "id": "15", + "name": "LO_TAX", + "datatype": "bigint", + "case_sensitive_name": "lo_tax" + }, + { + "id": "16", + "name": "LO_COMMITDATE", + "datatype": "date", + "case_sensitive_name": "lo_commitdate" + }, + { + "id": "17", + "name": "LO_SHIPMODE", + "datatype": "varchar(4096)", + "case_sensitive_name": "lo_shipmode" + } + ], + "source_type": 9, + "table_type": "EXTERNAL", + "top": false, + "increment_loading": false, + "last_snapshot_path": null, + "last_snapshot_size": 0, + "snapshot_last_modified": 0, + "query_hit_count": 0, + "partition_column": null, + "snapshot_partitions": {}, + "snapshot_partitions_info": {}, + "snapshot_total_rows": 0, + "snapshot_partition_col": null, + "selected_snapshot_partition_col": null, + "temp_snapshot_path": null, + "snapshot_has_broken": false, + "database": "SSB", + "transactional": false, + "rangePartition": false, + "partition_desc": null +} diff --git a/src/query-common/src/main/java/org/apache/kylin/query/enumerator/MetadataEnumerator.java b/src/query-common/src/main/java/org/apache/kylin/query/enumerator/MetadataEnumerator.java new file mode 100644 index 0000000000..1e089e33bf --- /dev/null +++ b/src/query-common/src/main/java/org/apache/kylin/query/enumerator/MetadataEnumerator.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kylin.query.enumerator; + +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; + +import org.apache.calcite.linq4j.Enumerator; +import org.apache.kylin.query.relnode.OLAPContext; +import org.apache.kylin.query.util.QueryInterruptChecker; + +/** + */ +public class MetadataEnumerator implements Enumerator<Object[]> { + + private Object[] current; + private final List<Object[]> result; + private Iterator<Object[]> iterator; + private int scanCount = 0; + + public MetadataEnumerator(OLAPContext olapContext) { + + this.result = olapContext.getColValuesRange(); + reset(); + } + + @Override + public boolean moveNext() { + if (scanCount++ % 1000 == 1) { + QueryInterruptChecker.checkThreadInterrupted("Interrupted at the stage of collecting result", + "MetadataEnumerator"); + } + + boolean hasNext = iterator.hasNext(); + if (hasNext) { + current = iterator.next(); + } + return hasNext; + } + + @Override + public Object[] current() { + // Note that array copy used to avoid messy result. + return Arrays.copyOf(current, current.length); + } + + @Override + public void reset() { + this.iterator = result.iterator(); + } + + @Override + public void close() { + // Nothing to do. + } +} diff --git a/src/query-common/src/main/java/org/apache/kylin/query/enumerator/OLAPQuery.java b/src/query-common/src/main/java/org/apache/kylin/query/enumerator/OLAPQuery.java index c52083dfb1..6df121adb3 100644 --- a/src/query-common/src/main/java/org/apache/kylin/query/enumerator/OLAPQuery.java +++ b/src/query-common/src/main/java/org/apache/kylin/query/enumerator/OLAPQuery.java @@ -35,6 +35,7 @@ public class OLAPQuery extends AbstractEnumerable<Object[]> implements Enumerabl private final DataContext optiqContext; private final EnumeratorTypeEnum type; private final int contextId; + public OLAPQuery(DataContext optiqContext, EnumeratorTypeEnum type, int ctxId) { this.optiqContext = optiqContext; this.type = type; @@ -56,6 +57,10 @@ public class OLAPQuery extends AbstractEnumerable<Object[]> implements Enumerabl return new OLAPEnumerator(olapContext, optiqContext); case HIVE: return new HiveEnumerator(olapContext); + case METADATA: + // Typically, this branch is considered to be unreachable + // for we use sparder query engine rather than Calcite query engine. + return new MetadataEnumerator(olapContext); default: throw new IllegalArgumentException("Wrong type " + type + "!"); } @@ -64,7 +69,8 @@ public class OLAPQuery extends AbstractEnumerable<Object[]> implements Enumerabl public enum EnumeratorTypeEnum { SIMPLE_AGGREGATION, //probing query like select min(2) from table OLAP, //finish query with Cube or II, or a combination of both - HIVE //using hive + HIVE, //using hive + METADATA // using metadata for min/max } public static class EmptyEnumerator implements Enumerator<Object[]> { diff --git a/src/query-common/src/main/java/org/apache/kylin/query/relnode/OLAPContext.java b/src/query-common/src/main/java/org/apache/kylin/query/relnode/OLAPContext.java index 23a1681565..991a576fa0 100644 --- a/src/query-common/src/main/java/org/apache/kylin/query/relnode/OLAPContext.java +++ b/src/query-common/src/main/java/org/apache/kylin/query/relnode/OLAPContext.java @@ -28,6 +28,7 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; import org.apache.calcite.rel.AbstractRelNode; import org.apache.calcite.rel.RelNode; @@ -39,12 +40,19 @@ import org.apache.calcite.rex.RexNode; import org.apache.commons.collections.CollectionUtils; import org.apache.kylin.common.KylinConfig; import org.apache.kylin.common.QueryContext; +import org.apache.kylin.guava30.shaded.common.base.Preconditions; +import org.apache.kylin.guava30.shaded.common.collect.Lists; +import org.apache.kylin.guava30.shaded.common.collect.Sets; import org.apache.kylin.metadata.cube.cuboid.NLayoutCandidate; +import org.apache.kylin.metadata.cube.model.DimensionRangeInfo; +import org.apache.kylin.metadata.cube.model.NDataSegment; +import org.apache.kylin.metadata.cube.model.NDataflow; import org.apache.kylin.metadata.cube.realization.HybridRealization; import org.apache.kylin.metadata.model.FunctionDesc; import org.apache.kylin.metadata.model.JoinDesc; import org.apache.kylin.metadata.model.MeasureDesc; import org.apache.kylin.metadata.model.NDataModel; +import org.apache.kylin.metadata.model.SegmentStatusEnum; import org.apache.kylin.metadata.model.TableRef; import org.apache.kylin.metadata.model.TblColRef; import org.apache.kylin.metadata.model.graph.JoinsGraph; @@ -52,6 +60,7 @@ import org.apache.kylin.metadata.query.NativeQueryRealization; import org.apache.kylin.metadata.query.QueryMetrics; import org.apache.kylin.metadata.realization.IRealization; import org.apache.kylin.metadata.realization.SQLDigest; +import org.apache.kylin.metadata.tuple.Tuple; import org.apache.kylin.metadata.tuple.TupleInfo; import org.apache.kylin.query.routing.RealizationCheck; import org.apache.kylin.query.schema.OLAPSchema; @@ -59,9 +68,6 @@ import org.apache.kylin.storage.StorageContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.kylin.guava30.shaded.common.collect.Lists; -import org.apache.kylin.guava30.shaded.common.collect.Sets; - import io.kyligence.kap.secondstorage.SecondStorageUtil; import lombok.Getter; import lombok.Setter; @@ -80,7 +86,7 @@ public class OLAPContext { // query info public OLAPSchema olapSchema = null; public OLAPTableScan firstTableScan = null; // to be fact table scan except "select * from lookupTable" - public Set<OLAPTableScan> allTableScans = new HashSet<>(); + public Set<OLAPTableScan> allTableScans = new LinkedHashSet<>(); public Set<OLAPJoinRel> allOlapJoins = new HashSet<>(); public Set<MeasureDesc> involvedMeasure = new HashSet<>(); public TupleInfo returnTupleInfo = null; @@ -542,6 +548,126 @@ public class OLAPContext { return createUniqueInputRefAmongTables(table, columnIdx, allTableScans); } + public String genExecFunc(OLAPRel rel, String tableName) { + setReturnTupleInfo(rel.getRowType(), rel.getColumnRowType()); + if (canMinMaxDimAnsweredByMetadata(rel)) { + return "executeMetadataQuery"; + } + + if (isConstantQueryWithAggregations()) { + return "executeSimpleAggregationQuery"; + } + + // If the table being scanned is not a fact table, then it is a lookup table. + if (realization.getModel().isLookupTable(tableName)) { + return "executeLookupTableQuery"; + } + + return "executeOLAPQuery"; + } + + private boolean canMinMaxDimAnsweredByMetadata(OLAPRel rel) { + if (!KylinConfig.getInstanceFromEnv().isRouteToMetadataEnabled()) { + return false; + } + + if (!(realization instanceof NDataflow) || !(rel instanceof OLAPJoinRel || rel instanceof OLAPTableScan)) { + logger.info("Can't route to metadata, the realization is {} and this OLAPRel is {}", realization, rel); + return false; + } + + /* + * Find the target pattern as shown below. + * (other rel) + * | + * Agg + * | + * Project + * | + * (TableScan or JoinRel) + */ + List<OLAPRel> relStack = new ArrayList<>(); + OLAPRel current = this.topNode; + while (current != rel && current.getInputs().size() == 1 && current.getInput(0) instanceof OLAPRel) { + relStack.add(current); + current = (OLAPRel) current.getInput(0); + } + if (current != rel || relStack.size() < 2 || !(relStack.get(relStack.size() - 1) instanceof OLAPProjectRel) + || !(relStack.get(relStack.size() - 2) instanceof OLAPAggregateRel)) { + logger.info("Can't route to query metadata, the rel stack is not matched"); + return false; + } + + OLAPAggregateRel aggregateRel = (OLAPAggregateRel) relStack.get(relStack.size() - 2); + if (aggregateRel.groups.size() > 1 || aggregateRel.groups.size() == 1 && !TblColRef.InnerDataTypeEnum.LITERAL + .getDataType().equals(aggregateRel.groups.get(0).getDatatype())) { + logger.info("Cannot route to query metadata, only group by constants are supported."); + return false; + } + + if (aggregations.isEmpty() || !aggregations.stream().allMatch(agg -> agg.isMin() || agg.isMax())) { + logger.info("Cannot route to query metadata, only min/max aggregate functions are supported."); + return false; + } + + if (aggregations.stream() + .anyMatch(agg -> TblColRef.InnerDataTypeEnum.contains(agg.getColRefs().get(0).getDatatype()))) { + logger.info("Cannot route to query metadata, not support min(expression), such as min(id+1)"); + return false; + } + + if (!Sets.newHashSet(realization.getAllDimensions()).containsAll(allColumns)) { + logger.info("Cannot route to query metadata, not all columns queried are treated as dimensions of index."); + return false; + } + + // reset rewriteAggCalls to aggCall, to avoid using measures. + aggregateRel.rewriteAggCalls.clear(); + aggregateRel.rewriteAggCalls.addAll(aggregateRel.getAggCallList()); + logger.info("Use kylin metadata to answer query with realization : {}", realization); + return true; + } + + public List<Object[]> getColValuesRange() { + Preconditions.checkState(realization instanceof NDataflow, "Only support dataflow"); + // As it is a min/max aggregate function, it only has one parameter. + List<TblColRef> cols = aggregations.stream() // + .map(FunctionDesc::getColRefs) // + .filter(tblColRefs -> tblColRefs.size() == 1) // + .map(tblColRefs -> tblColRefs.get(0)) // + .collect(Collectors.toList()); + List<TblColRef> allFields = new ArrayList<>(); + allTableScans.forEach(tableScan -> { + List<TblColRef> colRefs = tableScan.getColumnRowType().getAllColumns(); + allFields.addAll(colRefs); + }); + List<Object[]> result = new ArrayList<>(); + for (NDataSegment segment : ((NDataflow) realization).getSegments()) { + if (segment.getStatus() != SegmentStatusEnum.READY) { + continue; + } + Map<String, DimensionRangeInfo> infoMap = segment.getDimensionRangeInfoMap(); + Object[] minList = new Object[allFields.size()]; + Object[] maxList = new Object[allFields.size()]; + for (TblColRef col : cols) { + String dataType = col.getColumnDesc().getUpgradedType().getName(); + int colId = allFields.indexOf(col); + String tblColRefIndex = getTblColRefIndex(col, realization); + minList[colId] = Tuple.convertOptiqCellValue(infoMap.get(tblColRefIndex).getMin(), dataType); + maxList[colId] = Tuple.convertOptiqCellValue(infoMap.get(tblColRefIndex).getMax(), dataType); + } + + result.add(minList); + result.add(maxList); + } + return result; + } + + private String getTblColRefIndex(TblColRef colRef, IRealization df) { + NDataModel model = df.getModel(); + return String.valueOf(model.getColumnIdByColumnName(colRef.getAliasDotName())); + } + public interface IAccessController { void check(List<OLAPContext> contexts, OLAPRel tree, KylinConfig config); } diff --git a/src/query-common/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java b/src/query-common/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java index 3dce67eac6..c62ba3e4ab 100644 --- a/src/query-common/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java +++ b/src/query-common/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java @@ -221,7 +221,7 @@ public class OLAPJoinRel extends EnumerableJoin implements OLAPRel { "RowType=" + this.rowType.getFieldCount() + ", ColumnRowType=" + columns.size()); } - columns.stream().forEach(col -> sourceColumns.add(col.getSourceColumns())); + columns.forEach(col -> sourceColumns.add(col.getSourceColumns())); return new ColumnRowType(columns, sourceColumns); } @@ -244,9 +244,9 @@ public class OLAPJoinRel extends EnumerableJoin implements OLAPRel { JoinDesc join = new JoinDesc(); join.setForeignKey(fks.toArray(COLUMN_ARRAY_MARKER)); - join.setForeignKeyColumns(fkCols.toArray(new TblColRef[fkCols.size()])); + join.setForeignKeyColumns(fkCols.toArray(new TblColRef[0])); join.setPrimaryKey(pks.toArray(COLUMN_ARRAY_MARKER)); - join.setPrimaryKeyColumns(pkCols.toArray(new TblColRef[pkCols.size()])); + join.setPrimaryKeyColumns(pkCols.toArray(new TblColRef[0])); join.sortByFK(); return join; } @@ -299,12 +299,9 @@ public class OLAPJoinRel extends EnumerableJoin implements OLAPRel { */ @Override public Result implement(EnumerableRelImplementor implementor, Prefer pref) { - context.setReturnTupleInfo(rowType, columnRowType); - + String execFunc = context.genExecFunc(this, ""); PhysType physType = PhysTypeImpl.of(implementor.getTypeFactory(), getRowType(), pref.preferArray()); RelOptTable factTable = context.firstTableScan.getTable(); - // query result is error like select min(2+2), max(2) from EmptyTable - String execFunc = context.isConstantQueryWithAggregations() ? "executeSimpleAggregationQuery" : "executeOLAPQuery"; MethodCallExpression exprCall = Expressions.call(factTable.getExpression(OLAPTable.class), execFunc, implementor.getRootExpression(), Expressions.constant(context.id)); return implementor.result(physType, Blocks.toBlock(exprCall)); diff --git a/src/query-common/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java b/src/query-common/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java index 909f4e9b57..35e40017ad 100644 --- a/src/query-common/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java +++ b/src/query-common/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java @@ -157,7 +157,7 @@ public class OLAPProjectRel extends Project implements OLAPRel { TblColRef translateRexNode(RexNode rexNode, ColumnRowType inputColumnRowType, String fieldName, Set<TblColRef> sourceCollector, Map<RexNode, TblColRef> nodeAndTblColMap) { - if (!this.rewriting && !this.afterAggregate) { + if (!this.afterAggregate) { return RexToTblColRefTranslator.translateRexNode(rexNode, inputColumnRowType, fieldName, sourceCollector, nodeAndTblColMap); } else { diff --git a/src/query-common/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java b/src/query-common/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java index 573c7e3dd3..d09e6c2755 100644 --- a/src/query-common/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java +++ b/src/query-common/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java @@ -52,11 +52,11 @@ import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.sql.type.SqlTypeFactoryImpl; import org.apache.kylin.common.KylinConfig; import org.apache.kylin.metadata.model.ColumnDesc; +import org.apache.kylin.metadata.model.NDataModel; import org.apache.kylin.metadata.model.TableRef; import org.apache.kylin.metadata.model.TblColRef; import org.apache.kylin.query.schema.OLAPSchema; import org.apache.kylin.query.schema.OLAPTable; -import org.apache.kylin.metadata.model.NDataModel; import org.apache.kylin.guava30.shaded.common.base.Preconditions; @@ -176,7 +176,7 @@ public class OLAPTableScan extends TableScan implements OLAPRel, EnumerableRel { } /** - * There're 3 special RelNode in parents stack, OLAPProjectRel, OLAPToEnumerableConverter + * There are 3 special RelNode in parents stack, OLAPProjectRel, OLAPToEnumerableConverter * and OLAPUnionRel. OLAPProjectRel will helps collect required columns but the other two * don't. Go through the parent RelNodes from bottom to top, and the first-met special * RelNode determines the behavior. @@ -266,29 +266,13 @@ public class OLAPTableScan extends TableScan implements OLAPRel, EnumerableRel { */ @Override public Result implement(EnumerableRelImplementor implementor, Prefer pref) { - context.setReturnTupleInfo(rowType, columnRowType); - String execFunction = genExecFunc(); - + String execFunction = context.genExecFunc(this, tableName); PhysType physType = PhysTypeImpl.of(implementor.getTypeFactory(), getRowType(), JavaRowFormat.ARRAY, false); MethodCallExpression exprCall = Expressions.call(table.getExpression(OLAPTable.class), execFunction, implementor.getRootExpression(), Expressions.constant(context.id)); return implementor.result(physType, Blocks.toBlock(exprCall)); } - public String genExecFunc() { - context.setReturnTupleInfo(rowType, columnRowType); - if (context.isConstantQueryWithAggregations()) - return "executeSimpleAggregationQuery"; - // if the table to scan is not the fact table of cube, then it's a lookup table, - // TODO: this is not right! - if (context.realization.getModel().isLookupTable(tableName)) { - return "executeLookupTableQuery"; - } else { - return "executeOLAPQuery"; - } - - } - @Override public ColumnRowType getColumnRowType() { return columnRowType; diff --git a/src/query-common/src/main/java/org/apache/kylin/query/schema/OLAPTable.java b/src/query-common/src/main/java/org/apache/kylin/query/schema/OLAPTable.java index 5fdd6763b0..2641f9eab9 100644 --- a/src/query-common/src/main/java/org/apache/kylin/query/schema/OLAPTable.java +++ b/src/query-common/src/main/java/org/apache/kylin/query/schema/OLAPTable.java @@ -381,4 +381,8 @@ public class OLAPTable extends AbstractQueryableTable implements TranslatableTab public Enumerable<Object[]> executeSimpleAggregationQuery(DataContext optiqContext, int ctxSeq) { return new OLAPQuery(optiqContext, OLAPQuery.EnumeratorTypeEnum.SIMPLE_AGGREGATION, ctxSeq); } + + public Enumerable<Object[]> executeMetadataQuery(DataContext optiqContext, int ctxSeq) { + return new OLAPQuery(optiqContext, OLAPQuery.EnumeratorTypeEnum.METADATA, ctxSeq); + } } diff --git a/src/spark-project/sparder/src/main/scala/org/apache/kylin/query/runtime/CalciteToSparkPlaner.scala b/src/spark-project/sparder/src/main/scala/org/apache/kylin/query/runtime/CalciteToSparkPlaner.scala index 97053a3007..bab69339b4 100644 --- a/src/spark-project/sparder/src/main/scala/org/apache/kylin/query/runtime/CalciteToSparkPlaner.scala +++ b/src/spark-project/sparder/src/main/scala/org/apache/kylin/query/runtime/CalciteToSparkPlaner.scala @@ -19,18 +19,21 @@ package org.apache.kylin.query.runtime import java.util + import org.apache.kylin.guava30.shaded.common.collect.Lists import org.apache.kylin.engine.spark.utils.LogEx import org.apache.calcite.DataContext import org.apache.calcite.rel.{RelNode, RelVisitor} +import org.apache.kylin.common.KylinConfig +import org.apache.kylin.engine.spark.utils.LogEx import org.apache.kylin.query.relnode.{KapAggregateRel, KapFilterRel, KapJoinRel, KapLimitRel, KapMinusRel, KapModelViewRel, KapNonEquiJoinRel, KapProjectRel, KapRel, KapSortRel, KapTableScan, KapUnionRel, KapValuesRel, KapWindowRel} import org.apache.kylin.query.runtime.plan.{AggregatePlan, FilterPlan, LimitPlan, ProjectPlan, SortPlan, TableScanPlan, ValuesPlan, WindowPlan} import org.apache.kylin.query.util.KapRelUtil -import org.apache.kylin.common.KylinConfig import org.apache.spark.sql.DataFrame import scala.collection.JavaConverters._ + class CalciteToSparkPlaner(dataContext: DataContext) extends RelVisitor with LogEx { private val stack = new util.Stack[DataFrame]() private val setOpStack = new util.Stack[Int]() @@ -56,15 +59,7 @@ class CalciteToSparkPlaner(dataContext: DataContext) extends RelVisitor with Log node.childrenAccept(this) } stack.push(node match { - case rel: KapTableScan => - rel.genExecFunc() match { - case "executeLookupTableQuery" => - logTime("createLookupTable") { TableScanPlan.createLookupTable(rel) } - case "executeOLAPQuery" => - logTime("createOLAPTable") { TableScanPlan.createOLAPTable(rel) } - case "executeSimpleAggregationQuery" => - logTime("createSingleRow") { TableScanPlan.createSingleRow() } - } + case rel: KapTableScan => convertTableScan(rel) case rel: KapFilterRel => logTime("filter") { FilterPlan.filter(Lists.newArrayList(stack.pop()), rel, dataContext) } case rel: KapProjectRel => @@ -85,26 +80,8 @@ class CalciteToSparkPlaner(dataContext: DataContext) extends RelVisitor with Log AggregatePlan.agg(Lists.newArrayList(stack.pop()), rel) } } - case rel: KapJoinRel => - if (!rel.isRuntimeJoin) { - logTime("join with table scan") { TableScanPlan.createOLAPTable(rel) } - } else { - val right = stack.pop() - val left = stack.pop() - logTime("join") { plan.JoinPlan.join(Lists.newArrayList(left, right), rel) } - } - case rel: KapNonEquiJoinRel => - if (!rel.isRuntimeJoin) { - logTime("join with table scan") { - TableScanPlan.createOLAPTable(rel) - } - } else { - val right = stack.pop() - val left = stack.pop() - logTime("non-equi join") { - plan.JoinPlan.nonEquiJoin(Lists.newArrayList(left, right), rel, dataContext) - } - } + case rel: KapJoinRel => convertJoinRel(rel) + case rel: KapNonEquiJoinRel => convertNonEquiJoinRel(rel) case rel: KapUnionRel => val size = setOpStack.pop() val java = Range(0, stack.size() - size).map(a => stack.pop()).asJava @@ -122,6 +99,62 @@ class CalciteToSparkPlaner(dataContext: DataContext) extends RelVisitor with Log } } + private def convertTableScan(rel: KapTableScan): DataFrame = { + rel.getContext.genExecFunc(rel, rel.getTableName) match { + case "executeLookupTableQuery" => + logTime("createLookupTable") { + TableScanPlan.createLookupTable(rel) + } + case "executeOLAPQuery" => + logTime("createOLAPTable") { + TableScanPlan.createOLAPTable(rel) + } + case "executeSimpleAggregationQuery" => + logTime("createSingleRow") { + TableScanPlan.createSingleRow() + } + case "executeMetadataQuery" => + logTime("createMetadataTable") { + TableScanPlan.createMetadataTable(rel) + } + } + } + + private def convertJoinRel(rel: KapJoinRel): DataFrame = { + if (!rel.isRuntimeJoin) { + rel.getContext.genExecFunc(rel, "") match { + case "executeMetadataQuery" => + logTime("createMetadataTable") { + TableScanPlan.createMetadataTable(rel) + } + case _ => + logTime("join with table scan") { + TableScanPlan.createOLAPTable(rel) + } + } + } else { + val right = stack.pop() + val left = stack.pop() + logTime("join") { + plan.JoinPlan.join(Lists.newArrayList(left, right), rel) + } + } + } + + private def convertNonEquiJoinRel(rel: KapNonEquiJoinRel): DataFrame = { + if (!rel.isRuntimeJoin) { + logTime("join with table scan") { + TableScanPlan.createOLAPTable(rel) + } + } else { + val right = stack.pop() + val left = stack.pop() + logTime("non-equi join") { + plan.JoinPlan.nonEquiJoin(Lists.newArrayList(left, right), rel, dataContext) + } + } + } + def actionWith(rel: KapRel)(body: => DataFrame): DataFrame = { if (!dataframeCache) { body diff --git a/src/spark-project/sparder/src/main/scala/org/apache/kylin/query/runtime/plan/TableScanPlan.scala b/src/spark-project/sparder/src/main/scala/org/apache/kylin/query/runtime/plan/TableScanPlan.scala index d9b4065bda..6d30326e70 100644 --- a/src/spark-project/sparder/src/main/scala/org/apache/kylin/query/runtime/plan/TableScanPlan.scala +++ b/src/spark-project/sparder/src/main/scala/org/apache/kylin/query/runtime/plan/TableScanPlan.scala @@ -17,10 +17,10 @@ */ package org.apache.kylin.query.runtime.plan -import org.apache.kylin.guava30.shaded.common.base.Joiner -import org.apache.kylin.guava30.shaded.common.collect.{Lists, Sets} import org.apache.kylin.common.{KapConfig, KylinConfig, QueryContext} import org.apache.kylin.engine.spark.utils.{LogEx, LogUtils} +import org.apache.kylin.guava30.shaded.common.base.Joiner +import org.apache.kylin.guava30.shaded.common.collect.{Lists, Sets} import org.apache.kylin.metadata.cube.cuboid.NLayoutCandidate import org.apache.kylin.metadata.cube.gridtable.NLayoutToGridTableMapping import org.apache.kylin.metadata.cube.model.{LayoutEntity, NDataSegment, NDataflow} @@ -34,7 +34,7 @@ import org.apache.kylin.query.util.{RuntimeHelper, SparderDerivedUtil} import org.apache.spark.sql.execution.utils.SchemaProcessor import org.apache.spark.sql.functions.col import org.apache.spark.sql.manager.SparderLookupManager -import org.apache.spark.sql.types.{ArrayType, DoubleType, StructField, StructType} +import org.apache.spark.sql.types._ import org.apache.spark.sql.util.SparderTypeUtil import org.apache.spark.sql._ @@ -42,15 +42,13 @@ import java.util.concurrent.ConcurrentHashMap import java.{lang, util} import scala.collection.JavaConverters._ + // scalastyle:off object TableScanPlan extends LogEx { - def listSegmentsForQuery(cube: NDataflow): util.List[NDataSegment] = { + def listSegmentsForQuery(dataflow: NDataflow): util.List[NDataSegment] = { val r = new util.ArrayList[NDataSegment] - import scala.collection.JavaConversions._ - for (seg <- cube.getQueryableSegments) { - r.add(seg) - } + dataflow.getQueryableSegments.forEach(seg => r.add(seg)) r } @@ -80,14 +78,59 @@ object TableScanPlan extends LogEx { }).reduce(_.union(_)) } + def createMetadataTable(rel: KapRel): DataFrame = { + val session: SparkSession = SparderEnv.getSparkSession + val olapContext = rel.getContext + val allFields: util.List[TblColRef] = new util.ArrayList[TblColRef] + olapContext.allTableScans.forEach(tableScan => { + val columns = tableScan.getColumnRowType.getAllColumns + allFields.addAll(columns) + }) + // convert data + val dataSet = olapContext.getColValuesRange + val result = new util.ArrayList[Row] + dataSet.forEach(rowData => { + val sparderData = new Array[Any](rowData.length) + sparderData.indices.foreach(index => { + val dataType = allFields.get(index).getColumnDesc.getUpgradedType + sparderData(index) = SparderTypeUtil.convertStringToResultValueBasedOnKylinSQLType(rowData.apply(index), dataType) + }) + result.add(Row.fromSeq(sparderData)) + }) + + // create schema + val structTypes = new util.ArrayList[StructField]() + allFields.forEach(col => { + try { + val dataType = col.getColumnDesc.getUpgradedType + val spaType = SparderTypeUtil.kylinTypeToSparkResultType(dataType) + structTypes.add(StructField(col.getIdentity.replace(".", "_"), spaType)) + } catch { + // some dataTypes are not support in sparder, such as 'any', + // but these types can not be used in min/max, just return stringType + case e: IllegalArgumentException => { + structTypes.add(StructField(col.getIdentity.replace(".", "_"), StringType)) + logInfo(e.toString) + } + } + }) + val schema: StructType = StructType(structTypes) + + session.createDataFrame(result, schema) + } + // prunedSegments is null - def tableScanEmptySegment(rel: KapRel): DataFrame = { + private def tableScanEmptySegment(rel: KapRel): DataFrame = { logInfo("prunedSegments is null") val df = SparkOperation.createEmptyDataFrame( - StructType( - rel.getColumnRowType.getAllColumns.asScala - .map(column => - StructField(column.toString.replaceAll("\\.", "_"), SparderTypeUtil.toSparkType(column.getType))))) + StructType(rel.getColumnRowType + .getAllColumns.asScala + .map(column => StructField( + column.toString.replaceAll("\\.", "_"), + SparderTypeUtil.toSparkType(column.getType)) + ) + ) + ) val cols = df.schema.map(structField => { col(structField.name) }) @@ -101,7 +144,8 @@ object TableScanPlan extends LogEx { } def tableScan(rel: KapRel, dataflow: NDataflow, olapContext: OLAPContext, - session: SparkSession, prunedSegments: util.List[NDataSegment], candidate: NLayoutCandidate): DataFrame = { + session: SparkSession, prunedSegments: util.List[NDataSegment], + candidate: NLayoutCandidate): DataFrame = { val prunedPartitionMap = olapContext.storageContext.getPrunedPartitions olapContext.resetSQLDigest() //TODO: refactor @@ -122,7 +166,7 @@ object TableScanPlan extends LogEx { val fileList = prunedSegments.asScala.map( seg => toLayoutPath(dataflow, cuboidLayout.getId, basePath, seg, prunedPartitionMap) ) - val path = fileList.mkString(",") + olapContext.isExactlyFastBitmap() + val path = fileList.mkString(",") + olapContext.isExactlyFastBitmap printLogInfo(basePath, dataflow.getId, cuboidLayout.getId, prunedSegments, prunedPartitionMap) val cached = cacheDf.get().getOrDefault(path, null) @@ -139,7 +183,7 @@ object TableScanPlan extends LogEx { } }.mkString(",") val newDf = session.kylin - .isFastBitmapEnabled(olapContext.isExactlyFastBitmap()) + .isFastBitmapEnabled(olapContext.isExactlyFastBitmap) .bucketingEnabled(bucketEnabled(olapContext, cuboidLayout)) .cuboidTable(dataflow, cuboidLayout, pruningInfo) .toDF(columnNames: _*) @@ -238,7 +282,8 @@ object TableScanPlan extends LogEx { s"$basePath${dataflow.getUuid}/${seg.getId}/$cuboidId" } - def toLayoutPath(dataflow: NDataflow, layoutId: Long, basePath: String, seg: NDataSegment, partitionsMap: util.Map[String, util.List[lang.Long]]): List[String] = { + def toLayoutPath(dataflow: NDataflow, layoutId: Long, basePath: String, + seg: NDataSegment, partitionsMap: util.Map[String, util.List[lang.Long]]): List[String] = { if (partitionsMap == null) { List(toLayoutPath(dataflow, layoutId, basePath, seg)) } else { diff --git a/src/spark-project/spark-common/src/main/scala/org/apache/spark/sql/SparderTypeUtil.scala b/src/spark-project/spark-common/src/main/scala/org/apache/spark/sql/SparderTypeUtil.scala index 6791718885..31022d8a77 100644 --- a/src/spark-project/spark-common/src/main/scala/org/apache/spark/sql/SparderTypeUtil.scala +++ b/src/spark-project/spark-common/src/main/scala/org/apache/spark/sql/SparderTypeUtil.scala @@ -122,6 +122,32 @@ object SparderTypeUtil extends Logging { } } + def convertStringToResultValueBasedOnKylinSQLType(s: Any, dataTp: DataType): Any = { + if (s == null) { + null + } else { + dataTp.getName match { + case "decimal" => new java.math.BigDecimal(s.toString) + case "date" => new java.sql.Date(DateFormat.stringToMillis(s.toString)) + case "time" => new Timestamp(DateFormat.stringToMillis(s.toString)) + case "timestamp" => new Timestamp(DateFormat.stringToMillis(s.toString)) + case "datetime" => new java.sql.Date(DateFormat.stringToMillis(s.toString)) + case "tinyint" => s.toString.toByte + case "smallint" => s.toString.toShort + case "integer" => s.toString.toInt + case "int4" => s.toString.toInt + case "bigint" => s.toString.toLong + case "long8" => s.toString.toLong + case "float" => java.lang.Float.parseFloat(s.toString) + case "double" => java.lang.Double.parseDouble(s.toString) + case tp if tp.startsWith("varchar") => s.toString + case tp if tp.startsWith("char") => s.toString + case "boolean" => java.lang.Boolean.parseBoolean(s.toString) + case noSupport => throw new IllegalArgumentException(s"No supported data type: $noSupport") + } + } + } + def convertSqlTypeToSparkType(dt: RelDataType): org.apache.spark.sql.types.DataType = { dt.getSqlTypeName match { case SqlTypeName.DECIMAL => DecimalType(dt.getPrecision, dt.getScale)