You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by xx...@apache.org on 2023/05/06 06:59:05 UTC
[kylin] 04/38: KYLIN-5523 computed column as join key & partition column
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 520e7c67eaf1c070ea0de1cddac99a5dc69263e7
Author: Pengfei Zhan <pe...@kyligence.io>
AuthorDate: Tue Feb 7 22:15:40 2023 +0800
KYLIN-5523 computed column as join key & partition column
---
.../org/apache/kylin/common/KylinConfigBase.java | 6 +-
.../org/apache/kylin/common/msg/CnMessage.java | 2 +-
.../java/org/apache/kylin/common/msg/Message.java | 2 +-
.../util/{StringUtil.java => StringHelper.java} | 85 ++-
.../common/util/NLocalFileMetadataTestCase.java | 2 +-
.../apache/kylin/common/util/StringHelperTest.java | 104 ++++
.../kylin/job/execution/AbstractExecutable.java | 16 +-
.../job/execution/DefaultExecutableOnModel.java | 8 +-
.../job/execution/EmailNotificationContent.java | 18 +-
.../apache/kylin/dimension/DimensionEncoding.java | 6 +-
.../apache/kylin/metadata/model/ColumnDesc.java | 97 ++--
.../apache/kylin/metadata/model/JoinsGraph.java | 610 +++++++++++++++++++++
.../kylin/metadata/model/ModelDimensionDesc.java | 4 +-
.../kylin/metadata/model/MultiPartitionDesc.java | 2 +-
.../apache/kylin/metadata/model/NDataModel.java | 25 +-
.../apache/kylin/metadata/model/PartitionDesc.java | 10 +-
.../org/apache/kylin/metadata/model/TblColRef.java | 32 +-
.../metadata/model/util/ComputedColumnUtil.java | 74 +--
.../kylin/metadata/project/ProjectInstance.java | 6 +-
.../kylin/source/adhocquery/IPushDownRunner.java | 18 +-
.../cube/planner/algorithm/AlgorithmTestBase.java | 6 +-
.../kylin/metadata/model/JoinsGraphTest.java | 4 +-
.../model/util/ComputedColumnUtilTest.java | 4 +-
.../kylin/rest/controller/SampleController.java | 6 +-
.../rest/controller/open/OpenSampleController.java | 7 +-
.../controller/open/OpenSampleControllerTest.java | 10 +-
.../org/apache/kylin/rest/service/JobService.java | 5 +-
.../apache/kylin/rest/service/SnapshotService.java | 86 +--
.../kylin/rest/service/ModelServiceBuildTest.java | 7 +-
.../sdk/datasource/PushDownRunnerSDKImpl.java | 4 +-
.../kylin/rest/service/SparkSourceService.java | 22 +-
.../apache/kylin/rest/service/TableService.java | 53 +-
.../kylin/metadata/model/AntiFlatCheckerTest.java | 4 +-
.../metadata/model/ColExcludedCheckerTest.java | 4 +-
.../kylin/newten/NBadQueryAndPushDownTest.java | 6 +-
.../java/org/apache/kylin/query/KylinTestBase.java | 16 +-
.../java/org/apache/kylin/util/ExecAndComp.java | 3 +-
.../kylin/rest/controller/NTableController.java | 20 +-
.../rest/controller/NTableControllerTest.java | 10 +-
.../kylin/rest/service/ModelSemanticHelper.java | 21 +-
.../apache/kylin/rest/service/ModelService.java | 57 +-
.../kylin/rest/service/ModelServiceTest.java | 57 +-
.../kylin/rest/service/TableServiceTest.java | 132 +++--
.../org/apache/kylin/query/util/PushDownUtil.java | 353 ++++++++----
.../apache/kylin/query/util/QueryAliasMatcher.java | 23 +-
.../org/apache/kylin/query/util/QueryUtil.java | 416 +++-----------
.../kylin/rest/service/ModelServiceQueryTest.java | 3 +-
src/query/pom.xml | 9 +-
.../optrule/AbstractAggCaseWhenFunctionRule.java | 18 +-
.../optrule/CountDistinctCaseWhenFunctionRule.java | 17 +-
.../query/optrule/KapAggFilterTransposeRule.java | 4 +-
.../kap/query/optrule/KapAggJoinTransposeRule.java | 4 +-
.../kap/query/optrule/KapAggProjectMergeRule.java | 4 +-
.../query/optrule/KapAggProjectTransposeRule.java | 4 +-
.../query/optrule/KapCountDistinctJoinRule.java | 4 +-
.../kap/query/optrule/KapSumCastTransposeRule.java | 19 +-
.../main/java/org/apache/kylin/query/QueryCli.java | 78 ---
.../kylin/query/engine/QueryRoutingEngine.java | 16 +-
.../org/apache/kylin/query/util/RuleUtils.java | 150 +++++
.../apache/kylin/query/util/PushDownUtilTest.java | 80 ++-
.../org/apache/kylin/query/util/QueryUtilTest.java | 336 +++++-------
.../kap/secondstorage/tdvt/TDVTHiveTest.java | 4 +-
.../service/ModelServiceWithSecondStorageTest.java | 3 +-
.../engine/spark/application/SparkApplication.java | 27 +-
.../kylin/engine/spark/job/NSparkExecutable.java | 4 +-
.../engine/spark/utils/ComputedColumnEvalUtil.java | 52 +-
.../engine/spark/builder/CreateFlatTable.scala | 12 +-
.../engine/spark/builder/DFBuilderHelper.scala | 12 +-
.../engine/spark/builder/SegmentFlatTable.scala | 23 +-
.../kylin/engine/spark/job/FlatTableHelper.scala | 18 +-
.../job/stage/build/FlatTableAndDictBase.scala | 15 +-
.../spark/smarter/IndexDependencyParser.scala | 23 +-
.../query/pushdown/PushDownRunnerJdbcImpl.java | 4 +-
.../query/pushdown/PushDownRunnerSparkImpl.java | 3 +-
.../pushdown/PushDownRunnerSparkImplTest.java | 15 +-
.../kylin/engine/spark/job/NSparkCubingUtil.java | 11 -
.../scala/io/kyligence/kap/common/SSSource.scala | 89 +++
.../kylin/streaming/CreateStreamingFlatTable.scala | 24 +-
78 files changed, 2150 insertions(+), 1398 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 f1770e9fb2..0d4796e6e3 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
@@ -72,7 +72,7 @@ import org.apache.kylin.common.util.FileUtils;
import org.apache.kylin.common.util.HadoopUtil;
import org.apache.kylin.common.util.RandomUtil;
import org.apache.kylin.common.util.SizeConvertUtil;
-import org.apache.kylin.common.util.StringUtil;
+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;
@@ -2123,7 +2123,7 @@ public abstract class KylinConfigBase implements Serializable {
return getOptional("kylin.query.pushdown.partition-check.runner-class-name", "");
}
- public String getPartitionCheckRunnerClassNameWithDefaultValue() {
+ public String getDefaultPartitionCheckerClassName() {
String partitionCheckRunner = getPartitionCheckRunnerClassName();
if (StringUtils.isEmpty(partitionCheckRunner)) {
partitionCheckRunner = "org.apache.kylin.query.pushdown.PushDownRunnerSparkImpl";
@@ -2790,7 +2790,7 @@ public abstract class KylinConfigBase implements Serializable {
public String getEngineWriteFs() {
String engineWriteFs = getOptional("kylin.env.engine-write-fs", "");
- return StringUtil.dropSuffix(engineWriteFs, File.separator);
+ return StringHelper.dropSuffix(engineWriteFs, File.separator);
}
public boolean isAllowedProjectAdminGrantAcl() {
diff --git a/src/core-common/src/main/java/org/apache/kylin/common/msg/CnMessage.java b/src/core-common/src/main/java/org/apache/kylin/common/msg/CnMessage.java
index 679e15176f..d36e2c2c41 100644
--- a/src/core-common/src/main/java/org/apache/kylin/common/msg/CnMessage.java
+++ b/src/core-common/src/main/java/org/apache/kylin/common/msg/CnMessage.java
@@ -997,7 +997,7 @@ public class CnMessage extends Message {
@Override
public String getBadSqlColumnNotFoundReason() {
- return "无法找到列 \"%s\"。请检查此列是否在源表中存在。若存在,可尝试重载表;若不存在,请联系管理员添加。";
+ return "无法找到列 \"%s\"。请检查此列是否在源表或可计算列中存在。若存在,可尝试重载表;若不存在,请联系管理员添加。";
}
@Override
diff --git a/src/core-common/src/main/java/org/apache/kylin/common/msg/Message.java b/src/core-common/src/main/java/org/apache/kylin/common/msg/Message.java
index 32ef6010b5..ade226726c 100644
--- a/src/core-common/src/main/java/org/apache/kylin/common/msg/Message.java
+++ b/src/core-common/src/main/java/org/apache/kylin/common/msg/Message.java
@@ -957,7 +957,7 @@ public class Message {
}
public String getBadSqlColumnNotFoundReason() {
- return "Can’t find column \"%s\". Please check if it exists in the source table. If exists, please try reloading the table; if not exist, please contact admin to add it.";
+ return "Can’t find column \"%s\". Please check if it exists in the source table or computed columns. If exists, please try reloading the table; if not exist, please contact admin to add it.";
}
public String getBadSqlColumnNotFoundSuggest() {
diff --git a/src/core-common/src/main/java/org/apache/kylin/common/util/StringUtil.java b/src/core-common/src/main/java/org/apache/kylin/common/util/StringHelper.java
similarity index 65%
rename from src/core-common/src/main/java/org/apache/kylin/common/util/StringUtil.java
rename to src/core-common/src/main/java/org/apache/kylin/common/util/StringHelper.java
index 5776a5895b..7be0bc9e07 100644
--- a/src/core-common/src/main/java/org/apache/kylin/common/util/StringUtil.java
+++ b/src/core-common/src/main/java/org/apache/kylin/common/util/StringHelper.java
@@ -21,18 +21,21 @@ package org.apache.kylin.common.util;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
-import java.util.Locale;
import java.util.regex.Pattern;
import org.apache.commons.lang3.StringUtils;
+import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
-/**
- */
-public class StringUtil {
+public class StringHelper {
+
+ public static final char QUOTE = '\'';
+ public static final char DOUBLE_QUOTE = '"';
+ public static final char BACKTICK = '`';
- public static String[] EMPTY_ARRAY = new String[0];
+ private StringHelper() {
+ }
public static String[] filterSystemArgs(String[] args) {
List<String> whatsLeft = Lists.newArrayList();
@@ -86,7 +89,7 @@ public class StringUtil {
if (source != null) {
for (int i = 0; i < source.length; i++) {
if (source[i] != null) {
- target[i] = source[i].toUpperCase(Locale.ROOT);
+ target[i] = StringUtils.upperCase(source[i]);
}
}
}
@@ -175,7 +178,7 @@ public class StringUtil {
if (!s.isEmpty())
r.add(s);
}
- return r.toArray(new String[r.size()]);
+ return r.toArray(new String[0]);
}
// calculating length in UTF-8 of Java String without actually encoding it
@@ -213,4 +216,72 @@ public class StringUtil {
return str.split(splitBy);
}
+ public static String backtickToDoubleQuote(String expression) {
+ return convert(expression, StringHelper.BACKTICK, StringHelper.DOUBLE_QUOTE);
+ }
+
+ public static String doubleQuoteToBacktick(String expression) {
+ return convert(expression, StringHelper.DOUBLE_QUOTE, StringHelper.BACKTICK);
+ }
+
+ private static String convert(String expression, char srcQuote, char targetQuote) {
+ char[] chars = expression.toCharArray();
+ List<Integer> indexList = StringHelper.findQuoteIndexes(srcQuote, expression);
+ for (Integer integer : indexList) {
+ chars[integer] = targetQuote;
+ }
+ return new String(chars);
+ }
+
+ public static String backtickQuote(String identifier) {
+ String str = StringUtils.remove(identifier, StringHelper.BACKTICK);
+ return StringHelper.BACKTICK + str + StringHelper.BACKTICK;
+ }
+
+ public static String doubleQuote(String identifier) {
+ String str = StringUtils.remove(identifier, StringHelper.DOUBLE_QUOTE);
+ return StringHelper.DOUBLE_QUOTE + str + StringHelper.DOUBLE_QUOTE;
+ }
+
+ /**
+ * Search identifier quotes in the sql string.
+ * @param key the char to search
+ * @param str the input string
+ * @return index list of {@code key}
+ */
+ public static List<Integer> findQuoteIndexes(char key, String str) {
+ Preconditions.checkState(key == BACKTICK || key == DOUBLE_QUOTE);
+ char[] chars = str.toCharArray();
+ List<Integer> indexList = Lists.newArrayList();
+ List<Pair<Integer, Character>> toMatchTokens = Lists.newArrayList();
+ for (int i = 0; i < chars.length; i++) {
+ char ch = chars[i];
+ if (toMatchTokens.isEmpty()) {
+ if (ch == key || ch == QUOTE) {
+ toMatchTokens.add(new Pair<>(i, ch));
+ }
+ continue;
+ }
+
+ // The toMatchTokens is not empty, try to collect
+ Character ex = toMatchTokens.get(toMatchTokens.size() - 1).getSecond();
+ if (ch == ex && ch == key) {
+ toMatchTokens.add(new Pair<>(i, ex));
+ Preconditions.checkState(toMatchTokens.size() == 2);
+ indexList.add(toMatchTokens.get(0).getFirst());
+ indexList.add(toMatchTokens.get(1).getFirst());
+ toMatchTokens.clear();
+ } else if (ch == ex && ch == QUOTE) {
+ // There are two kind of single quote in the char array.
+ // One kind has two successive single quote '', we need to clear the toMatchTokens.
+ // Another kind has a form of \', just ignore it and go on match the next char.
+ Preconditions.checkState(toMatchTokens.size() == 1);
+ if (chars[i - 1] != '\\') {
+ toMatchTokens.clear();
+ }
+ }
+ }
+ Preconditions.checkState(indexList.size() % 2 == 0);
+ return indexList;
+ }
}
diff --git a/src/core-common/src/test/java/org/apache/kylin/common/util/NLocalFileMetadataTestCase.java b/src/core-common/src/test/java/org/apache/kylin/common/util/NLocalFileMetadataTestCase.java
index 212dea3c0c..bcab2b07d6 100644
--- a/src/core-common/src/test/java/org/apache/kylin/common/util/NLocalFileMetadataTestCase.java
+++ b/src/core-common/src/test/java/org/apache/kylin/common/util/NLocalFileMetadataTestCase.java
@@ -147,7 +147,7 @@ public class NLocalFileMetadataTestCase extends AbstractTestCase {
val kylinHomePath = new File(getTestConfig().getMetadataUrl().toString()).getParentFile().getAbsolutePath();
overwriteSystemProp("KYLIN_HOME", kylinHomePath);
val jobJar = org.apache.kylin.common.util.FileUtils.findFile(
- new File(kylinHomePath, "../../../assembly/target/").getAbsolutePath(), "kap-assembly(.?)\\.jar");
+ new File(kylinHomePath, "../../../assembly/target/").getAbsolutePath(), "ke-assembly(.*?)\\.jar");
getTestConfig().setProperty("kylin.engine.spark.job-jar", jobJar == null ? "" : jobJar.getAbsolutePath());
getTestConfig().setProperty("kylin.query.security.acl-tcr-enabled", "false");
getTestConfig().setProperty("kylin.streaming.enabled", "true");
diff --git a/src/core-common/src/test/java/org/apache/kylin/common/util/StringHelperTest.java b/src/core-common/src/test/java/org/apache/kylin/common/util/StringHelperTest.java
new file mode 100644
index 0000000000..dcadc2a013
--- /dev/null
+++ b/src/core-common/src/test/java/org/apache/kylin/common/util/StringHelperTest.java
@@ -0,0 +1,104 @@
+/*
+ * 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.common.util;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+class StringHelperTest {
+
+ @Test
+ void testMin() {
+ Assertions.assertEquals("", StringHelper.min(null, ""));
+ Assertions.assertEquals("", StringHelper.min("", null));
+ Assertions.assertEquals("1", StringHelper.min("1", "2"));
+ }
+
+ @Test
+ void testMax() {
+ Assertions.assertEquals("", StringHelper.max(null, ""));
+ Assertions.assertEquals("", StringHelper.max("", null));
+ Assertions.assertEquals("2", StringHelper.max("1", "2"));
+ }
+
+ @Test
+ void testValidateBoolean() {
+ Assertions.assertTrue(StringHelper.validateBoolean("true"));
+ Assertions.assertTrue(StringHelper.validateBoolean("false"));
+ }
+
+ @Test
+ void testBacktickToDoubleQuote() {
+ Assertions.assertEquals("\"a\".\"b\" + 1", StringHelper.backtickToDoubleQuote("`a`.`b` + 1"));
+ }
+
+ @Test
+ void testDoubleQuoteToBackTick() {
+ Assertions.assertEquals("`a`.`b` + '''1'", StringHelper.doubleQuoteToBacktick("\"a\".\"b\" + '''1'"));
+ }
+
+ @Test
+ void testBacktickQuote() {
+ Assertions.assertEquals("`aa`", StringHelper.backtickQuote("aa"));
+ }
+
+ @Test
+ void testDoubleQuote() {
+ Assertions.assertEquals("\"aa\"", StringHelper.doubleQuote("aa"));
+ }
+
+ @Test
+ void testSubArray() {
+ String[] arr = { "a", "b", "c" };
+ try {
+ StringHelper.subArray(arr, -1, 1);
+ Assertions.fail();
+ } catch (Exception e) {
+ Assertions.assertTrue(e instanceof IllegalArgumentException);
+ }
+
+ try {
+ StringHelper.subArray(arr, 2, 1);
+ Assertions.fail();
+ } catch (Exception e) {
+ Assertions.assertTrue(e instanceof IllegalArgumentException);
+ }
+
+ try {
+ StringHelper.subArray(arr, 1, 5);
+ Assertions.fail();
+ } catch (Exception e) {
+ Assertions.assertTrue(e instanceof IllegalArgumentException);
+ }
+
+ String[] arrNew = StringHelper.subArray(arr, 0, 2);
+ Assertions.assertEquals(2, arrNew.length);
+ Assertions.assertEquals("a", arrNew[0]);
+ Assertions.assertEquals("b", arrNew[1]);
+ }
+
+ @Test
+ void testSplitAndTrim() {
+ String[] arr = StringHelper.splitAndTrim("a, ,b, c", ",");
+ Assertions.assertEquals(3, arr.length);
+ Assertions.assertEquals("a", arr[0]);
+ Assertions.assertEquals("b", arr[1]);
+ Assertions.assertEquals("c", arr[2]);
+ }
+}
diff --git a/src/core-job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java b/src/core-job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java
index 2805ee0b31..bfe6f1fb6a 100644
--- a/src/core-job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java
+++ b/src/core-job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java
@@ -50,7 +50,6 @@ import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Consumer;
import java.util.stream.Collectors;
-import io.kyligence.kap.guava20.shaded.common.base.Throwables;
import org.apache.commons.collections.CollectionUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.lang3.ArrayUtils;
@@ -63,7 +62,7 @@ import org.apache.kylin.common.metrics.MetricsName;
import org.apache.kylin.common.persistence.transaction.UnitOfWork;
import org.apache.kylin.common.util.MailHelper;
import org.apache.kylin.common.util.RandomUtil;
-import org.apache.kylin.common.util.StringUtil;
+import org.apache.kylin.common.util.StringHelper;
import org.apache.kylin.common.util.ThrowableUtils;
import org.apache.kylin.job.constant.JobIssueEnum;
import org.apache.kylin.job.dao.ExecutableOutputPO;
@@ -82,13 +81,14 @@ import org.apache.kylin.metadata.project.ProjectInstance;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
import com.google.common.base.Preconditions;
+import com.google.common.base.Throwables;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
-import io.kyligence.kap.guava20.shaded.common.annotations.VisibleForTesting;
-import io.kyligence.kap.guava20.shaded.common.base.MoreObjects;
import lombok.Getter;
import lombok.Setter;
import lombok.val;
@@ -917,7 +917,7 @@ public abstract class AbstractExecutable implements Executable {
String layouts = getParam(NBatchConstants.P_LAYOUT_IDS);
if (layouts != null) {
- return computeDriverMemory(StringUtil.splitAndTrim(layouts, ",").length);
+ return computeDriverMemory(StringHelper.splitAndTrim(layouts, ",").length);
}
return 0;
}
@@ -925,8 +925,8 @@ public abstract class AbstractExecutable implements Executable {
public static Integer computeDriverMemory(Integer cuboidNum) {
KylinConfig config = KylinConfig.getInstanceFromEnv();
int[] driverMemoryStrategy = config.getSparkEngineDriverMemoryStrategy();
- List strategy = Lists.newArrayList(cuboidNum);
- Arrays.stream(driverMemoryStrategy).forEach(x -> strategy.add(Integer.valueOf(x)));
+ List<Integer> strategy = Lists.newArrayList(cuboidNum);
+ Arrays.stream(driverMemoryStrategy).forEach(strategy::add);
Collections.sort(strategy);
int index = strategy.indexOf(cuboidNum);
int driverMemoryMaximum = config.getSparkEngineDriverMemoryMaximum();
@@ -938,7 +938,7 @@ public abstract class AbstractExecutable implements Executable {
@Override
public String toString() {
- return MoreObjects.toStringHelper(this).add("id", getId()).add("name", getName()).add("state", getStatus())
+ return Objects.toStringHelper(this).add("id", getId()).add("name", getName()).add("state", getStatus())
.toString();
}
diff --git a/src/core-job/src/main/java/org/apache/kylin/job/execution/DefaultExecutableOnModel.java b/src/core-job/src/main/java/org/apache/kylin/job/execution/DefaultExecutableOnModel.java
index 339efeef8d..77d31bec84 100644
--- a/src/core-job/src/main/java/org/apache/kylin/job/execution/DefaultExecutableOnModel.java
+++ b/src/core-job/src/main/java/org/apache/kylin/job/execution/DefaultExecutableOnModel.java
@@ -23,12 +23,10 @@ import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.Stream;
-import com.google.common.collect.Sets;
import org.apache.commons.lang3.StringUtils;
import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.StringUtil;
+import org.apache.kylin.common.util.StringHelper;
import org.apache.kylin.job.model.JobParam;
-import org.apache.kylin.metadata.realization.RealizationStatusEnum;
import org.apache.kylin.metadata.cube.model.NBatchConstants;
import org.apache.kylin.metadata.cube.model.NDataSegment;
import org.apache.kylin.metadata.cube.model.NDataflow;
@@ -36,8 +34,10 @@ import org.apache.kylin.metadata.cube.model.NDataflowManager;
import org.apache.kylin.metadata.cube.model.NIndexPlanManager;
import org.apache.kylin.metadata.cube.model.SegmentPartition;
import org.apache.kylin.metadata.model.ManagementType;
+import org.apache.kylin.metadata.realization.RealizationStatusEnum;
import com.google.common.base.Preconditions;
+import com.google.common.collect.Sets;
import lombok.Getter;
import lombok.Setter;
@@ -108,7 +108,7 @@ public class DefaultExecutableOnModel extends DefaultExecutable {
val indexPlanManager = NIndexPlanManager.getInstance(getConfig(), getProject());
val indexPlan = indexPlanManager.getIndexPlan(getTargetModel());
val allLayoutIds = indexPlan.getAllLayouts().stream().map(l -> l.getId() + "").collect(Collectors.toSet());
- return Stream.of(StringUtil.splitAndTrim(layouts, ",")).anyMatch(allLayoutIds::contains);
+ return Stream.of(StringHelper.splitAndTrim(layouts, ",")).anyMatch(allLayoutIds::contains);
}
private boolean checkTargetSegmentAndPartitionExists(String segmentId) {
diff --git a/src/core-job/src/main/java/org/apache/kylin/job/execution/EmailNotificationContent.java b/src/core-job/src/main/java/org/apache/kylin/job/execution/EmailNotificationContent.java
index 5106cc16cc..4d9074cdd3 100644
--- a/src/core-job/src/main/java/org/apache/kylin/job/execution/EmailNotificationContent.java
+++ b/src/core-job/src/main/java/org/apache/kylin/job/execution/EmailNotificationContent.java
@@ -24,7 +24,7 @@ import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.constant.NonCustomProjectLevelConfig;
import org.apache.kylin.common.util.DateFormat;
import org.apache.kylin.common.util.Pair;
-import org.apache.kylin.common.util.StringUtil;
+import org.apache.kylin.common.util.StringHelper;
import org.apache.kylin.job.constant.ExecutableConstants;
import org.apache.kylin.job.constant.JobIssueEnum;
import org.apache.kylin.job.util.MailNotificationUtil;
@@ -89,7 +89,7 @@ public class EmailNotificationContent {
if (state == ExecutableState.ERROR) {
checkErrorTask(executable, dataMap, tasks);
dataMap.put("error_log",
- Matcher.quoteReplacement(StringUtil.noBlank(output.getShortErrMsg(), "no error message")));
+ Matcher.quoteReplacement(StringHelper.noBlank(output.getShortErrMsg(), "no error message")));
}
return Pair.newPair(getMailTitle(state, executable), getMailContent(state, dataMap));
@@ -116,7 +116,7 @@ public class EmailNotificationContent {
dataMap.put("error_step", errorTask.getName());
if (errorTask.getOutput().getExtra().containsKey(ExecutableConstants.MR_JOB_ID)) {
final String mrJobId = errorOutput.getExtra().get(ExecutableConstants.MR_JOB_ID);
- dataMap.put(ExecutableConstants.MR_JOB_ID, StringUtil.noBlank(mrJobId, "Not initialized"));
+ dataMap.put(ExecutableConstants.MR_JOB_ID, StringHelper.noBlank(mrJobId, "Not initialized"));
} else {
dataMap.put(ExecutableConstants.MR_JOB_ID, MailNotificationUtil.NA);
}
@@ -127,26 +127,26 @@ public class EmailNotificationContent {
AbstractExecutable executable) {
logger.info("notify on metadata persist exception: {}", exception.getMessage());
Map<String, Object> dataMap = getDataMap(executable);
- dataMap.put("error_log", Matcher.quoteReplacement(StringUtil.noBlank(
+ dataMap.put("error_log", Matcher.quoteReplacement(StringHelper.noBlank(
exception.getMessage(), "no error message")));
String content = MailNotificationUtil.getMailContent(MailNotificationUtil.METADATA_PERSIST_FAIL, dataMap);
String title = MailNotificationUtil.getMailTitle("METADATA_PERSIST",
"FAIL",
- StringUtil.noBlank(executable.getConfig().getDeployEnv(), MailNotificationUtil.NA),
+ StringHelper.noBlank(executable.getConfig().getDeployEnv(), MailNotificationUtil.NA),
executable.getProject(),
- StringUtil.noBlank(executable.getTargetSubjectAlias(), MailNotificationUtil.NA));
+ StringHelper.noBlank(executable.getTargetSubjectAlias(), MailNotificationUtil.NA));
return Pair.newPair(title, content);
}
private static Map<String, Object> getDataMap(AbstractExecutable executable) {
Map<String, Object> dataMap = Maps.newHashMap();
- dataMap.put("job_name", StringUtil.noBlank(executable.getName(), "missing job_name"));
+ dataMap.put("job_name", StringHelper.noBlank(executable.getName(), "missing job_name"));
dataMap.put("env_name", executable.getConfig().getDeployEnv());
- dataMap.put("submitter", StringUtil.noBlank(executable.getSubmitter(), "missing submitter"));
+ dataMap.put("submitter", StringHelper.noBlank(executable.getSubmitter(), "missing submitter"));
dataMap.put("job_engine", MailNotificationUtil.getLocalHostName());
dataMap.put("project_name", executable.getProject());
- dataMap.put("model_name", StringUtil.noBlank(executable.getTargetModelAlias(), "missing model_name"));
+ dataMap.put("model_name", StringHelper.noBlank(executable.getTargetModelAlias(), "missing model_name"));
return dataMap;
}
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/dimension/DimensionEncoding.java b/src/core-metadata/src/main/java/org/apache/kylin/dimension/DimensionEncoding.java
index 1fa4e8e5e8..08465a7397 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/dimension/DimensionEncoding.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/dimension/DimensionEncoding.java
@@ -20,7 +20,7 @@ package org.apache.kylin.dimension;
import java.io.Externalizable;
-import org.apache.kylin.common.util.StringUtil;
+import org.apache.kylin.common.util.StringHelper;
import org.apache.kylin.metadata.datatype.DataTypeSerializer;
/**
@@ -58,8 +58,8 @@ public abstract class DimensionEncoding implements Externalizable {
final String encodingName = parts[0];
final String[] encodingArgs = parts[parts.length - 1].isEmpty() //
- ? StringUtil.subArray(parts, 1, parts.length - 1)
- : StringUtil.subArray(parts, 1, parts.length);
+ ? StringHelper.subArray(parts, 1, parts.length - 1)
+ : StringHelper.subArray(parts, 1, parts.length);
return new Object[] { encodingName, encodingArgs };
}
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/ColumnDesc.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/ColumnDesc.java
index e179e2bb63..49094bf6a3 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/ColumnDesc.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/ColumnDesc.java
@@ -19,9 +19,10 @@
package org.apache.kylin.metadata.model;
import java.io.Serializable;
-import java.util.Locale;
import org.apache.calcite.avatica.util.Quoting;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.kylin.common.util.StringHelper;
import org.apache.kylin.metadata.datatype.DataType;
import com.fasterxml.jackson.annotation.JsonAutoDetect;
@@ -31,6 +32,9 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonSetter;
import com.google.common.base.Preconditions;
+import lombok.Getter;
+import lombok.Setter;
+
/**
* Column Metadata from Source. All name should be uppercase.
* <p/>
@@ -40,27 +44,34 @@ public class ColumnDesc implements Serializable {
private static final String BACK_TICK = Quoting.BACK_TICK.string;
+ @Getter
@JsonProperty("id")
private String id;
@JsonProperty("name")
private String name;
+ @Getter
@JsonProperty("datatype")
private String datatype;
@JsonProperty("comment")
@JsonInclude(JsonInclude.Include.NON_NULL)
+ @Getter
+ @Setter
private String comment;
@JsonProperty("data_gen")
@JsonInclude(JsonInclude.Include.NON_NULL)
+ @Getter
private String dataGen;
@JsonProperty("index")
@JsonInclude(JsonInclude.Include.NON_NULL)
+ @Getter
private String index;
+ @Setter
@JsonProperty("cc_expr")
@JsonInclude(JsonInclude.Include.NON_NULL)
private String computedColumnExpr = null;//if null, it's not a computed column
@@ -73,11 +84,19 @@ public class ColumnDesc implements Serializable {
public boolean isPartitioned = false;
// parsed from data type
+ @Getter
private DataType type;
+ @Setter
private DataType upgradedType;
+ @Getter
+ @Setter
private TableDesc table;
+ @Getter
private int zeroBasedIndex = -1;
+
+ @Getter
+ @Setter
private boolean isNullable = true;
public ColumnDesc() { // default constructor for Jackson
@@ -107,27 +126,11 @@ public class ColumnDesc implements Serializable {
this.computedColumnExpr = computedColumnExpr;
}
- public void setComputedColumn(String exp) {
- computedColumnExpr = exp;
- }
-
- public int getZeroBasedIndex() {
- return zeroBasedIndex;
- }
-
- public String getDatatype() {
- return datatype;
- }
-
public void setDatatype(String datatype) {
this.datatype = datatype;
type = DataType.getType(datatype);
}
- public void setUpgradedType(DataType upgradedType) {
- this.upgradedType = upgradedType;
- }
-
public DataType getUpgradedType() {
if (this.upgradedType == null) {
return this.type;
@@ -136,10 +139,6 @@ public class ColumnDesc implements Serializable {
}
}
- public String getId() {
- return id;
- }
-
public void setId(String id) {
this.id = id;
if (id != null)
@@ -152,7 +151,7 @@ public class ColumnDesc implements Serializable {
}
public String getName() {
- return (name == null) ? null : name.toUpperCase(Locale.ROOT);
+ return StringUtils.upperCase(name);
}
public String getIdentity() {
@@ -177,26 +176,6 @@ public class ColumnDesc implements Serializable {
this.caseSensitiveName = caseSensitiveName;
}
- public TableDesc getTable() {
- return table;
- }
-
- public void setTable(TableDesc table) {
- this.table = table;
- }
-
- public String getComment() {
- return comment;
- }
-
- public void setComment(String comment) {
- this.comment = comment;
- }
-
- public DataType getType() {
- return type;
- }
-
public String getTypeName() {
return type.getName();
}
@@ -209,28 +188,20 @@ public class ColumnDesc implements Serializable {
return type.getScale();
}
- public boolean isNullable() {
- return this.isNullable;
- }
-
- public void setNullable(boolean nullable) {
- this.isNullable = nullable;
- }
-
- public String getDataGen() {
- return dataGen;
- }
-
- public String getIndex() {
- return index;
- }
-
public String getComputedColumnExpr() {
Preconditions.checkState(computedColumnExpr != null);
-
return computedColumnExpr;
}
+ public String getDoubleQuoteInnerExpr() {
+ Preconditions.checkState(computedColumnExpr != null);
+ int quoteCnt = StringUtils.countMatches(computedColumnExpr, StringHelper.QUOTE);
+ if (quoteCnt == 0 || quoteCnt == 1) {
+ return computedColumnExpr.replace(StringHelper.BACKTICK, StringHelper.DOUBLE_QUOTE);
+ }
+ return StringHelper.backtickToDoubleQuote(computedColumnExpr);
+ }
+
public boolean isComputedColumn() {
return computedColumnExpr != null;
}
@@ -306,8 +277,12 @@ public class ColumnDesc implements Serializable {
@Override
public String toString() {
- return "ColumnDesc{" + "id='" + id + '\'' + ", name='" + name + '\'' + ", datatype='" + datatype + '\''
- + ", comment='" + comment + '\'' + '}';
+ return "ColumnDesc{" //
+ + "id='" + id //
+ + "', name='" + name //
+ + "', datatype='" + datatype //
+ + "', comment='" + comment //
+ + "'}";
}
public ColumnDesc copy() {
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinsGraph.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinsGraph.java
new file mode 100644
index 0000000000..acd810fbec
--- /dev/null
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinsGraph.java
@@ -0,0 +1,610 @@
+/*
+ * 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.metadata.model;
+
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.Pair;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import lombok.Getter;
+import lombok.NonNull;
+import lombok.Setter;
+
+public class JoinsGraph implements Serializable {
+
+ public class Edge implements Serializable {
+
+ @Getter
+ private final JoinDesc join;
+ private final ColumnDesc[] leftCols;
+ private final ColumnDesc[] rightCols;
+ private final NonEquiJoinCondition nonEquiJoinCondition;
+
+ private Edge(JoinDesc join) {
+ this.join = join;
+
+ leftCols = new ColumnDesc[join.getForeignKeyColumns().length];
+ int i = 0;
+ for (TblColRef colRef : join.getForeignKeyColumns()) {
+ leftCols[i++] = colRef.getColumnDesc();
+ }
+
+ rightCols = new ColumnDesc[join.getPrimaryKeyColumns().length];
+ i = 0;
+ for (TblColRef colRef : join.getPrimaryKeyColumns()) {
+ rightCols[i++] = colRef.getColumnDesc();
+ }
+
+ nonEquiJoinCondition = join.getNonEquiJoinCondition();
+ }
+
+ public boolean isJoinMatched(JoinDesc other) {
+ return join.equals(other);
+ }
+
+ public boolean isNonEquiJoin() {
+ return nonEquiJoinCondition != null;
+ }
+
+ public boolean isLeftJoin() {
+ return !join.isLeftOrInnerJoin() && join.isLeftJoin();
+ }
+
+ public boolean isLeftOrInnerJoin() {
+ return join.isLeftOrInnerJoin();
+ }
+
+ public boolean isInnerJoin() {
+ return !join.isLeftOrInnerJoin() && join.isInnerJoin();
+ }
+
+ private TableRef left() {
+ return join.getFKSide();
+ }
+
+ private TableRef right() {
+ return join.getPKSide();
+ }
+
+ private boolean isFkSide(TableRef tableRef) {
+ return join.getFKSide().equals(tableRef);
+ }
+
+ private boolean isPkSide(TableRef tableRef) {
+ return join.getPKSide().equals(tableRef);
+ }
+
+ private TableRef other(TableRef tableRef) {
+ if (left().equals(tableRef)) {
+ return right();
+ } else if (right().equals(tableRef)) {
+ return left();
+ }
+ throw new IllegalArgumentException("table " + tableRef + " is not on the edge " + this);
+ }
+
+ @Override
+ public boolean equals(Object that) {
+ if (that == null)
+ return false;
+
+ if (this.getClass() != that.getClass())
+ return false;
+
+ return joinEdgeMatcher.matches(this, (Edge) that);
+ }
+
+ @Override
+ public int hashCode() {
+ if (this.isLeftJoin()) {
+ return Objects.hash(isLeftJoin(), leftCols, rightCols);
+ } else {
+ if (Arrays.hashCode(leftCols) < Arrays.hashCode(rightCols)) {
+ return Objects.hash(isLeftJoin(), leftCols, rightCols);
+ } else {
+ return Objects.hash(isLeftJoin(), rightCols, leftCols);
+ }
+ }
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder("Edge: ");
+ sb.append(left())
+ .append(isLeftJoin() ? " LEFT JOIN "
+ : isLeftOrInnerJoin() ? " LEFT OR INNER JOIN " : " INNER JOIN ")
+ .append(right()).append(" ON ")
+ .append(Arrays.toString(Arrays.stream(leftCols).map(ColumnDesc::getName).toArray())).append(" = ")
+ .append(Arrays.toString(Arrays.stream(rightCols).map(ColumnDesc::getName).toArray()));
+ return sb.toString();
+ }
+ }
+
+ private Edge edgeOf(JoinDesc join) {
+ return new Edge(join);
+ }
+
+ private static final IJoinEdgeMatcher DEFAULT_JOIN_EDGE_MATCHER = new DefaultJoinEdgeMatcher();
+ @Setter
+ private IJoinEdgeMatcher joinEdgeMatcher = DEFAULT_JOIN_EDGE_MATCHER;
+
+ /**
+ * compare:
+ * 1. JoinType
+ * 2. Columns on both sides
+ */
+ public interface IJoinEdgeMatcher extends Serializable {
+ boolean matches(@NonNull Edge join1, @NonNull Edge join2);
+ }
+
+ public static class DefaultJoinEdgeMatcher implements IJoinEdgeMatcher {
+ @Override
+ public boolean matches(@NonNull Edge join1, @NonNull Edge join2) {
+ if (join1.isLeftJoin() != join2.isLeftJoin() && !join1.isLeftOrInnerJoin() && !join2.isLeftOrInnerJoin()) {
+ return false;
+ }
+
+ if (!Objects.equals(join1.nonEquiJoinCondition, join2.nonEquiJoinCondition)) {
+ return false;
+ }
+
+ if (join1.isLeftJoin()) {
+ return columnDescEquals(join1.leftCols, join2.leftCols)
+ && columnDescEquals(join1.rightCols, join2.rightCols);
+ } else {
+ return (columnDescEquals(join1.leftCols, join2.leftCols)
+ && columnDescEquals(join1.rightCols, join2.rightCols))
+ || (columnDescEquals(join1.leftCols, join2.rightCols)
+ && columnDescEquals(join1.rightCols, join2.leftCols));
+ }
+ }
+
+ private boolean columnDescEquals(ColumnDesc[] a, ColumnDesc[] b) {
+ if (a.length != b.length) {
+ return false;
+ }
+
+ List<ColumnDesc> oneList = Lists.newArrayList(a);
+ List<ColumnDesc> anotherList = Lists.newArrayList(b);
+
+ for (ColumnDesc obj : oneList) {
+ anotherList.removeIf(dual -> columnDescEquals(obj, dual));
+ }
+ return anotherList.isEmpty();
+ }
+
+ protected boolean columnDescEquals(ColumnDesc a, ColumnDesc b) {
+ return Objects.equals(a, b);
+ }
+ }
+
+ @Getter
+ private final TableRef center;
+ private final Map<String, TableRef> nodes = new HashMap<>();
+ private final Set<Edge> edges = new HashSet<>();
+ private final Map<TableRef, List<Edge>> edgesFromNode = new HashMap<>();
+ private final Map<TableRef, List<Edge>> edgesToNode = new HashMap<>();
+
+ /**
+ * For model there's always a center, if there's only one tableScan it's the center.
+ * Otherwise the center is not determined, it's a linked graph, hard to tell the center.
+ */
+ public JoinsGraph(TableRef root, List<JoinDesc> joins) {
+ this.center = root;
+ addNode(root);
+
+ for (JoinDesc join : joins) {
+ Preconditions.checkState(Arrays.stream(join.getForeignKeyColumns()).allMatch(TblColRef::isQualified));
+ Preconditions.checkState(Arrays.stream(join.getPrimaryKeyColumns()).allMatch(TblColRef::isQualified));
+ addAsEdge(join);
+ }
+
+ validate(joins);
+ }
+
+ private void addNode(TableRef table) {
+ Preconditions.checkNotNull(table);
+ String alias = table.getAlias();
+ TableRef node = nodes.get(alias);
+ if (node != null) {
+ Preconditions.checkArgument(node.equals(table), "[%s]'s Alias \"%s\" has conflict with [%s].", table, alias,
+ node);
+ } else {
+ nodes.put(alias, table);
+ }
+ }
+
+ private void addAsEdge(JoinDesc join) {
+ TableRef fkTable = join.getFKSide();
+ TableRef pkTable = join.getPKSide();
+ addNode(pkTable);
+
+ Edge edge = edgeOf(join);
+ edgesFromNode.computeIfAbsent(fkTable, fk -> Lists.newArrayList());
+ edgesFromNode.get(fkTable).add(edge);
+ edgesToNode.computeIfAbsent(pkTable, pk -> Lists.newArrayList());
+ edgesToNode.get(pkTable).add(edge);
+ if (!edge.isLeftJoin()) {
+ // inner join is reversible
+ edgesFromNode.computeIfAbsent(pkTable, pk -> Lists.newArrayList());
+ edgesFromNode.get(pkTable).add(edge);
+ edgesToNode.computeIfAbsent(fkTable, fk -> Lists.newArrayList());
+ edgesToNode.get(fkTable).add(edge);
+ }
+ edges.add(edge);
+ }
+
+ public void setJoinToLeftOrInner(JoinDesc join) {
+ if (!join.isLeftJoin()) {
+ join.setLeftOrInner(true);
+ return;
+ }
+
+ join.setLeftOrInner(true);
+ TableRef fkTable = join.getFKSide();
+ TableRef pkTable = join.getPKSide();
+ Edge edge = edges.stream().filter(e -> e.isJoinMatched(join)).findFirst().orElse(null);
+ if (edge == null) {
+ return;
+ }
+ edgesFromNode.computeIfAbsent(pkTable, pk -> Lists.newArrayList());
+ edgesFromNode.get(pkTable).add(edge);
+ edgesToNode.computeIfAbsent(fkTable, fk -> Lists.newArrayList());
+ edgesToNode.get(fkTable).add(edge);
+ }
+
+ private void validate(List<JoinDesc> joins) {
+ for (JoinDesc join : joins) {
+ TableRef fkTable = join.getFKSide();
+ Preconditions.checkNotNull(nodes.get(fkTable.getAlias()));
+ Preconditions.checkState(nodes.get(fkTable.getAlias()).equals(fkTable));
+ }
+ Preconditions.checkState(nodes.size() == joins.size() + 1);
+ }
+
+ public boolean match(JoinsGraph pattern, Map<String, String> matchAlias) {
+ return match(pattern, matchAlias, false);
+ }
+
+ public boolean match(JoinsGraph pattern, Map<String, String> matchAlias, boolean matchPatial) {
+ return match(pattern, matchAlias, matchPatial, false);
+ }
+
+ public boolean match(JoinsGraph pattern, Map<String, String> matchAlias, boolean matchPatial,
+ boolean matchPartialNonEquiJoin) {
+ if (pattern == null || pattern.center == null) {
+ throw new IllegalArgumentException("pattern(model) should have a center: " + pattern);
+ }
+
+ List<TableRef> candidatesOfQCenter = searchCenterByIdentity(pattern.center);
+ if (CollectionUtils.isEmpty(candidatesOfQCenter)) {
+ return false;
+ }
+
+ for (TableRef queryCenter : candidatesOfQCenter) {
+ // query <-> pattern
+ Map<TableRef, TableRef> trialMatch = Maps.newHashMap();
+ trialMatch.put(queryCenter, pattern.center);
+
+ if (!checkInnerJoinNum(pattern, queryCenter, pattern.center, matchPatial)) {
+ continue;
+ }
+
+ AtomicReference<Map<TableRef, TableRef>> finalMatchRef = new AtomicReference<>();
+ innerMatch(pattern, trialMatch, matchPatial, finalMatchRef);
+ if (finalMatchRef.get() != null
+ && (matchPartialNonEquiJoin || checkNonEquiJoinMatches(finalMatchRef.get(), pattern))) {
+ matchAlias.clear();
+ matchAlias.putAll(finalMatchRef.get().entrySet().stream()
+ .collect(Collectors.toMap(e -> e.getKey().getAlias(), e -> e.getValue().getAlias())));
+ return true;
+ }
+ }
+ return false;
+ }
+
+ public static JoinsGraph normalizeJoinGraph(JoinsGraph joinsGraph) {
+ for (Edge edge : joinsGraph.edges) {
+ if (!edge.isLeftJoin() || edge.isLeftOrInnerJoin()) {
+ TableRef leftTable = edge.left();
+ List<Edge> edgeList = joinsGraph.edgesToNode.get(leftTable);
+ if (CollectionUtils.isEmpty(edgeList)) {
+ continue;
+ }
+ for (Edge targetEdge : edgeList) {
+ if (!edge.equals(targetEdge) && leftTable.equals(targetEdge.right())
+ && !targetEdge.isLeftOrInnerJoin()) {
+ joinsGraph.setJoinToLeftOrInner(targetEdge.join);
+ normalizeJoinGraph(joinsGraph);
+ }
+ }
+ }
+ }
+ return joinsGraph;
+ }
+
+ public List<TableRef> getAllTblRefNodes() {
+ return Lists.newArrayList(nodes.values());
+ }
+
+ /**
+ * check if any non-equi join is missed in the pattern
+ * if so, we cannot match the current graph with the the pattern graph.
+ * set `kylin.query.match-partial-non-equi-join-model` to skip this checking
+ * @param matches
+ * @return
+ */
+ private boolean checkNonEquiJoinMatches(Map<TableRef, TableRef> matches, JoinsGraph pattern) {
+ HashSet<TableRef> patternGraphTables = new HashSet<>(pattern.nodes.values());
+
+ for (TableRef patternTable : patternGraphTables) {
+ List<Edge> outgoingEdges = pattern.getEdgesByFKSide(patternTable);
+ // for all outgoing non-equi join edges
+ // if there is no match found for the right side table in the current graph
+ // return false
+ for (Edge outgoingEdge : outgoingEdges) {
+ if (outgoingEdge.isNonEquiJoin()) {
+ if (!matches.containsValue(patternTable) || !matches.containsValue(outgoingEdge.right())) {
+ return false;
+ }
+ }
+ }
+ }
+ return true;
+ }
+
+ private boolean isAllJoinInner(JoinsGraph joinsGraph, TableRef tableRef) {
+ List<Edge> edgesFromNode = joinsGraph.edgesFromNode.get(tableRef);
+ List<Edge> edgesToNode = joinsGraph.edgesToNode.get(tableRef);
+
+ if (edgesFromNode == null) {
+ return false;
+ }
+
+ if (edgesToNode == null) {
+ return false;
+ }
+
+ if (edgesToNode.size() != edgesFromNode.size()) {
+ return false;
+ }
+
+ for (Edge edge : edgesFromNode) {
+ if (edge.join.isLeftJoin()) {
+ return false;
+ }
+ }
+
+ return true;
+ }
+
+ private boolean checkInnerJoinNum(JoinsGraph pattern, TableRef queryTableRef, TableRef patternTableRef,
+ boolean matchPartial) {
+ if (matchPartial) {
+ return true;
+ }
+ // fully match: unmatched if extra inner join edge on either graph
+ // matched if: query graph join count: model graph join count:
+ // 1) inner join <= inner join
+ // 2) inner join + left or inner join >= inner join
+ List<Edge> innerQueryEdges = this.edgesFrom(queryTableRef).stream().filter(Edge::isInnerJoin)
+ .collect(Collectors.toList());
+ List<Edge> notLeftQueryEdges = this.edgesFrom(queryTableRef).stream().filter(e -> !e.isLeftJoin())
+ .collect(Collectors.toList());
+ List<Edge> innerPatternEdges = pattern.edgesFrom(patternTableRef).stream().filter(Edge::isInnerJoin)
+ .collect(Collectors.toList());
+
+ // if all joins are inner joins, compare sum of both sides
+ if (isAllJoinInner(this, queryTableRef) && isAllJoinInner(pattern, patternTableRef)) {
+ int cntInnerQueryEdges = innerQueryEdges.size();
+ int cntNotLeftQueryEdges = notLeftQueryEdges.size();
+ int cntInnerPatternEdges = innerPatternEdges.size();
+ return cntInnerQueryEdges <= cntInnerPatternEdges && cntNotLeftQueryEdges >= cntInnerPatternEdges;
+ }
+
+ // if not all joins are inner, compare left side and right side separately
+ // Calculate join count in query graph
+ int cntLeftSideInnerQueryEdges = (int) innerQueryEdges.stream()
+ .filter(edge -> edge.right().equals(queryTableRef)).count();
+ int cntRightSideInnerQueryEdges = (int) innerQueryEdges.stream()
+ .filter(edge -> edge.left().equals(queryTableRef)).count();
+ int cntLeftSideNotLeftQueryEdges = (int) notLeftQueryEdges.stream()
+ .filter(edge -> edge.right().equals(queryTableRef)).count();
+ int cntRightSideNotLeftQueryEdges = (int) notLeftQueryEdges.stream()
+ .filter(edge -> edge.left().equals(queryTableRef)).count();
+ // Calculate join count in model graph
+ int cntLeftSideInnerPatternEdges = (int) innerPatternEdges.stream()
+ .filter(edge -> edge.right().equals(patternTableRef)).count();
+ int cntRightSideInnerPatternEdges = (int) innerPatternEdges.stream()
+ .filter(edge -> edge.left().equals(patternTableRef)).count();
+
+ boolean isLeftEqual = cntLeftSideInnerQueryEdges <= cntLeftSideInnerPatternEdges
+ && cntLeftSideNotLeftQueryEdges >= cntLeftSideInnerPatternEdges;
+ boolean isRightEqual = cntRightSideInnerQueryEdges <= cntRightSideInnerPatternEdges
+ && cntRightSideNotLeftQueryEdges >= cntRightSideInnerPatternEdges;
+ return isLeftEqual && isRightEqual;
+ }
+
+ private void innerMatch(JoinsGraph pattern, Map<TableRef, TableRef> trialMatches, boolean matchPartial,
+ AtomicReference<Map<TableRef, TableRef>> finalMatch) {
+ if (trialMatches.size() == nodes.size()) {
+ //match is found
+ finalMatch.set(trialMatches);
+ return;
+ }
+
+ Preconditions.checkState(nodes.size() > trialMatches.size());
+ Optional<Pair<Edge, TableRef>> toMatch = trialMatches.keySet().stream()
+ .map(t -> edgesFrom(t).stream().filter(e -> !trialMatches.containsKey(e.other(t))).findFirst()
+ .map(edge -> new Pair<>(edge, edge.other(t))).orElse(null))
+ .filter(Objects::nonNull).findFirst();
+
+ Preconditions.checkState(toMatch.isPresent());
+ Edge toMatchQueryEdge = toMatch.get().getFirst();
+ TableRef toMatchQueryNode = toMatch.get().getSecond();
+ TableRef matchedQueryNode = toMatchQueryEdge.other(toMatchQueryNode);
+ TableRef matchedPatternNode = trialMatches.get(matchedQueryNode);
+
+ List<TableRef> toMatchPatternNodeCandidates = Lists.newArrayList();
+ for (Edge patternEdge : pattern.edgesFrom(matchedPatternNode)) {
+ TableRef toMatchPatternNode = patternEdge.other(matchedPatternNode);
+ if (!toMatchQueryNode.getTableIdentity().equals(toMatchPatternNode.getTableIdentity())
+ || !toMatchQueryEdge.equals(patternEdge) || trialMatches.containsValue(toMatchPatternNode)
+ || !checkInnerJoinNum(pattern, toMatchQueryNode, toMatchPatternNode, matchPartial)) {
+ continue;
+ }
+ toMatchPatternNodeCandidates.add(toMatchPatternNode);
+ }
+
+ for (TableRef toMatchPatternNode : toMatchPatternNodeCandidates) {
+ Map<TableRef, TableRef> newTrialMatches = Maps.newHashMap();
+ newTrialMatches.putAll(trialMatches);
+ newTrialMatches.put(toMatchQueryNode, toMatchPatternNode);
+ innerMatch(pattern, newTrialMatches, matchPartial, finalMatch);
+ if (finalMatch.get() != null) {
+ //get out of recursive invoke chain straightly
+ return;
+ }
+ }
+ }
+
+ public List<Edge> unmatched(JoinsGraph pattern) {
+ List<Edge> unmatched = Lists.newArrayList();
+ Set<Edge> all = edgesFromNode.values().stream().flatMap(List::stream).collect(Collectors.toSet());
+ for (Edge edge : all) {
+ List<JoinDesc> joins = getJoinsPathByPKSide(edge.right());
+ JoinsGraph subGraph = new JoinsGraph(center, joins);
+ if (subGraph.match(pattern, Maps.newHashMap())) {
+ continue;
+ }
+ unmatched.add(edge);
+ }
+ return unmatched;
+ }
+
+ private List<TableRef> searchCenterByIdentity(final TableRef table) {
+ // special case: several same nodes in a JoinGraph
+ return nodes.values().stream().filter(node -> node.getTableIdentity().equals(table.getTableIdentity()))
+ .filter(node -> {
+ List<JoinDesc> path2Center = getJoinsPathByPKSide(node);
+ return path2Center.stream().noneMatch(JoinDesc::isLeftJoin);
+ }).collect(Collectors.toList());
+ }
+
+ private List<Edge> edgesFrom(TableRef thisSide) {
+ return edgesFromNode.getOrDefault(thisSide, Lists.newArrayList());
+ }
+
+ public Map<String, String> matchAlias(JoinsGraph joinsGraph, KylinConfig kylinConfig) {
+ Map<String, String> matchAlias = Maps.newHashMap();
+ match(joinsGraph, matchAlias, kylinConfig.isQueryMatchPartialInnerJoinModel(),
+ kylinConfig.partialMatchNonEquiJoins());
+ return matchAlias;
+ }
+
+ public Map<String, String> matchAlias(JoinsGraph joinsGraph, boolean matchPartial) {
+ Map<String, String> matchAlias = Maps.newHashMap();
+ match(joinsGraph, matchAlias, matchPartial);
+ return matchAlias;
+ }
+
+ public List<Edge> getEdgesByFKSide(TableRef table) {
+ if (!edgesFromNode.containsKey(table)) {
+ return Lists.newArrayList();
+ }
+ return edgesFromNode.get(table).stream().filter(e -> e.isFkSide(table)).collect(Collectors.toList());
+ }
+
+ private Edge getEdgeByPKSide(TableRef table) {
+ if (!edgesToNode.containsKey(table)) {
+ return null;
+ }
+ List<Edge> edgesByPkSide = edgesToNode.get(table).stream().filter(e -> e.isPkSide(table))
+ .collect(Collectors.toList());
+ if (edgesByPkSide.isEmpty()) {
+ return null;
+ }
+ Preconditions.checkState(edgesByPkSide.size() == 1, "%s is allowed to be Join PK side once", table);
+ return edgesByPkSide.get(0);
+ }
+
+ public JoinDesc getJoinByPKSide(TableRef table) {
+ Edge edge = getEdgeByPKSide(table);
+ return edge != null ? edge.join : null;
+ }
+
+ private List<JoinDesc> getJoinsPathByPKSide(TableRef table) {
+ List<JoinDesc> pathToRoot = Lists.newArrayList();
+ TableRef pkSide = table; // start from leaf
+ while (pkSide != null) {
+ JoinDesc subJoin = getJoinByPKSide(pkSide);
+ if (subJoin != null) {
+ pathToRoot.add(subJoin);
+ pkSide = subJoin.getFKSide();
+ } else {
+ pkSide = null;
+ }
+ }
+ return Lists.reverse(pathToRoot);
+ }
+
+ public JoinsGraph getSubgraphByAlias(Set<String> aliasSets) {
+ TableRef subGraphRoot = this.center;
+ Set<JoinDesc> subGraphJoin = Sets.newHashSet();
+ for (String alias : aliasSets) {
+ subGraphJoin.addAll(getJoinsPathByPKSide(nodes.get(alias)));
+ }
+ return new JoinsGraph(subGraphRoot, Lists.newArrayList(subGraphJoin));
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder graphStrBuilder = new StringBuilder();
+ graphStrBuilder.append("Root: ").append(center);
+ List<Edge> nextEdges = getEdgesByFKSide(center);
+ nextEdges.forEach(e -> buildGraphStr(graphStrBuilder, e, 1));
+ return graphStrBuilder.toString();
+ }
+
+ private void buildGraphStr(StringBuilder sb, @NonNull Edge edge, int indent) {
+ sb.append('\n');
+ for (int i = 0; i < indent; i++) {
+ sb.append(" ");
+ }
+ sb.append(edge);
+ List<Edge> nextEdges = getEdgesByFKSide(edge.right());
+ nextEdges.forEach(e -> buildGraphStr(sb, e, indent + 1));
+ }
+}
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/ModelDimensionDesc.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/ModelDimensionDesc.java
index dd3ff1e6c5..cc7f6f4a32 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/ModelDimensionDesc.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/ModelDimensionDesc.java
@@ -22,7 +22,7 @@ import java.io.Serializable;
import java.util.List;
import java.util.Locale;
-import org.apache.kylin.common.util.StringUtil;
+import org.apache.kylin.common.util.StringHelper;
import com.fasterxml.jackson.annotation.JsonAutoDetect;
import com.fasterxml.jackson.annotation.JsonProperty;
@@ -60,7 +60,7 @@ public class ModelDimensionDesc implements Serializable {
public void init(NDataModel model) {
table = table.toUpperCase(Locale.ROOT);
if (columns != null) {
- StringUtil.toUpperCaseArray(columns, columns);
+ StringHelper.toUpperCaseArray(columns, columns);
}
if (model != null) {
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/MultiPartitionDesc.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/MultiPartitionDesc.java
index cf8c3916fa..859a2bc230 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/MultiPartitionDesc.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/MultiPartitionDesc.java
@@ -192,7 +192,7 @@ public class MultiPartitionDesc implements Serializable {
List<String> conditions = Lists.newArrayList();
for (int i = 0; i < columnRefs.size(); i++) {
final int x = i;
- String item = columnRefs.get(x).getBackTickExpressionInSourceDB() + " in (" + //
+ String item = columnRefs.get(x).getBackTickExp() + " in (" + //
values.stream().map(a -> generateFormattedValue(columnRefs.get(x).getType(), a[x]))
.collect(Collectors.joining(", "))
+ ")";
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/NDataModel.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/NDataModel.java
index 3d7cffbd17..c51b0cd5ff 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/NDataModel.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/NDataModel.java
@@ -53,7 +53,7 @@ import org.apache.kylin.common.persistence.ResourceStore;
import org.apache.kylin.common.persistence.RootPersistentEntity;
import org.apache.kylin.common.scheduler.SchedulerEventNotifier;
import org.apache.kylin.common.util.Pair;
-import org.apache.kylin.common.util.StringUtil;
+import org.apache.kylin.common.util.StringHelper;
import org.apache.kylin.metadata.MetadataConstants;
import org.apache.kylin.metadata.model.graph.JoinsGraph;
import org.apache.kylin.metadata.model.tool.CalciteParser;
@@ -588,7 +588,7 @@ public class NDataModel extends RootPersistentEntity {
for (TableRef t : allTableRefs) {
if (t.getTableIdentity().equals(table.getIdentity())
- && StringUtil.equals(t.getTableDesc().getProject(), table.getProject()))
+ && StringUtils.equals(t.getTableDesc().getProject(), table.getProject()))
return true;
}
return false;
@@ -822,23 +822,17 @@ public class NDataModel extends RootPersistentEntity {
continue;
}
} else {
- if (0 == quotationType) {
- quotationType = 1;
- continue;
- }
+ quotationType = 1;
+ continue;
}
}
if ('"' == this.filterCondition.charAt(i)) {
if (quotationType > 0) {
if (2 == quotationType) {
quotationType = 0;
- continue;
}
} else {
- if (0 == quotationType) {
- quotationType = 2;
- continue;
- }
+ quotationType = 2;
}
}
}
@@ -849,11 +843,12 @@ public class NDataModel extends RootPersistentEntity {
for (JoinTableDesc joinTable : joinTables) {
TableRef dimTable = joinTable.getTableRef();
JoinDesc join = joinTable.getJoin();
- if (join == null)
+ if (join == null) {
throw new IllegalStateException("Missing join conditions on table " + dimTable);
+ }
- StringUtil.toUpperCaseArray(join.getForeignKey(), join.getForeignKey());
- StringUtil.toUpperCaseArray(join.getPrimaryKey(), join.getPrimaryKey());
+ StringHelper.toUpperCaseArray(join.getForeignKey(), join.getForeignKey());
+ StringHelper.toUpperCaseArray(join.getPrimaryKey(), join.getPrimaryKey());
// primary key
String[] pks = join.getPrimaryKey();
@@ -1082,7 +1077,6 @@ public class NDataModel extends RootPersistentEntity {
}
private ImmutableBiMap<Integer, TblColRef> initAllNamedColumns(Predicate<NamedColumn> filter) {
- List<TblColRef> all = new ArrayList<>(allNamedColumns.size());
ImmutableBiMap.Builder<Integer, TblColRef> mapBuilder = ImmutableBiMap.builder();
for (NamedColumn d : allNamedColumns) {
if (!d.isExist()) {
@@ -1090,7 +1084,6 @@ public class NDataModel extends RootPersistentEntity {
}
TblColRef col = this.findColumn(d.aliasDotColumn);
d.aliasDotColumn = col.getIdentity();
- all.add(col);
if (filter.test(d)) {
mapBuilder.put(d.id, col);
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/PartitionDesc.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/PartitionDesc.java
index 47b0adc805..d7717b1771 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/PartitionDesc.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/PartitionDesc.java
@@ -307,7 +307,7 @@ public class PartitionDesc implements Serializable {
private static void buildSingleColumnRangeCondAsDate(StringBuilder builder, TblColRef partitionColumn,
long startInclusive, long endExclusive, String partitionColumnDateFormat) {
- String partitionColumnName = partitionColumn.getBackTickExpressionInSourceDB();
+ String partitionColumnName = partitionColumn.getBackTickExp();
builder.append(partitionColumnName).append(" >= ").append(String.format(Locale.ROOT, "to_date('%s', '%s')",
DateFormat.formatToDateStr(startInclusive, partitionColumnDateFormat), partitionColumnDateFormat));
builder.append(and);
@@ -317,7 +317,7 @@ public class PartitionDesc implements Serializable {
private static void buildSingleColumnRangeCondAsTimestamp(StringBuilder builder, TblColRef partitionColumn,
long startInclusive, long endExclusive) {
- String partitionColumnName = partitionColumn.getBackTickExpressionInSourceDB();
+ String partitionColumnName = partitionColumn.getBackTickExp();
builder.append(partitionColumnName).append(" >= ").append(startInclusive);
builder.append(and);
builder.append(partitionColumnName).append(" < ").append(endExclusive);
@@ -325,7 +325,7 @@ public class PartitionDesc implements Serializable {
private static void buildSingleColumnRangeCondAsYmInt(StringBuilder builder, TblColRef partitionColumn,
long startInclusive, long endExclusive) {
- String partitionColumnName = partitionColumn.getBackTickExpressionInSourceDB();
+ String partitionColumnName = partitionColumn.getBackTickExp();
builder.append(partitionColumnName).append(" >= ")
.append(DateFormat.formatToDateStr(startInclusive, DateFormat.COMPACT_MONTH_PATTERN));
builder.append(and);
@@ -335,7 +335,7 @@ public class PartitionDesc implements Serializable {
private static void buildSingleColumnRangeCondAsYmdInt(StringBuilder builder, TblColRef partitionColumn,
long startInclusive, long endExclusive) {
- String partitionColumnName = partitionColumn.getBackTickExpressionInSourceDB();
+ String partitionColumnName = partitionColumn.getBackTickExp();
builder.append(partitionColumnName).append(" >= ")
.append(DateFormat.formatToDateStr(startInclusive, DateFormat.COMPACT_DATE_PATTERN));
builder.append(and);
@@ -345,7 +345,7 @@ public class PartitionDesc implements Serializable {
private static void buildSingleColumnRangeCondition(StringBuilder builder, TblColRef partitionColumn,
long startInclusive, long endExclusive, String partitionColumnDateFormat) {
- String partitionColumnName = partitionColumn.getBackTickExpressionInSourceDB();
+ String partitionColumnName = partitionColumn.getBackTickExp();
if (endExclusive <= startInclusive) {
builder.append("1=1");
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/TblColRef.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/TblColRef.java
index b6cd7ebd2a..1c2b132897 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/TblColRef.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/TblColRef.java
@@ -18,12 +18,9 @@
package org.apache.kylin.metadata.model;
-import static com.google.common.base.Preconditions.checkArgument;
-
import java.io.Serializable;
import java.util.HashSet;
import java.util.List;
-import java.util.Locale;
import java.util.Set;
import org.apache.calcite.avatica.util.Quoting;
@@ -32,12 +29,13 @@ import org.apache.calcite.sql.SqlOperator;
import org.apache.commons.lang3.StringUtils;
import org.apache.kylin.metadata.datatype.DataType;
+import com.google.common.base.Preconditions;
+
import lombok.Getter;
import lombok.Setter;
/**
*/
-@SuppressWarnings({ "serial" })
public class TblColRef implements Serializable {
private static final String INNER_TABLE_NAME = "_kylin_table";
@@ -128,8 +126,7 @@ public class TblColRef implements Serializable {
column.setDatatype(DYNAMIC_DATA_TYPE);
TableDesc table = new TableDesc();
column.setTable(table);
- TblColRef colRef = new TblColRef(column);
- return colRef;
+ return new TblColRef(column);
}
private static final NDataModel UNKNOWN_MODEL = new NDataModel();
@@ -142,14 +139,14 @@ public class TblColRef implements Serializable {
}
public static TblColRef columnForUnknownModel(TableRef table, ColumnDesc colDesc) {
- checkArgument(table.getModel() == UNKNOWN_MODEL);
+ Preconditions.checkArgument(table.getModel() == UNKNOWN_MODEL);
return new TblColRef(table, colDesc);
}
public static void fixUnknownModel(NDataModel model, String alias, TblColRef col) {
- checkArgument(col.table.getModel() == UNKNOWN_MODEL || col.table.getModel() == model);
+ Preconditions.checkArgument(col.table.getModel() == UNKNOWN_MODEL || col.table.getModel() == model);
TableRef tableRef = model.findTable(alias);
- checkArgument(tableRef.getTableDesc().getIdentity().equals(col.column.getTable().getIdentity()));
+ Preconditions.checkArgument(tableRef.getTableDesc().getIdentity().equals(col.column.getTable().getIdentity()));
col.fixTableRef(tableRef);
}
@@ -199,7 +196,7 @@ public class TblColRef implements Serializable {
}
public TblColRef(TableRef table, ColumnDesc column) {
- checkArgument(table.getTableDesc().getIdentity().equals(column.getTable().getIdentity()));
+ Preconditions.checkArgument(table.getTableDesc().getIdentity().equals(column.getTable().getIdentity()));
this.table = table;
this.column = column;
}
@@ -251,18 +248,15 @@ public class TblColRef implements Serializable {
}
}
- public String getDoubleQuoteExpressionInSourceDB() {
+ public String getDoubleQuoteExp() {
if (column.isComputedColumn())
- return column.getComputedColumnExpr();
+ return column.getDoubleQuoteInnerExpr();
return wrapIdentity(DOUBLE_QUOTE);
}
- public String getBackTickExpressionInSourceDB() {
- if (column.isComputedColumn())
- return column.getComputedColumnExpr();
-
- return wrapIdentity(BACK_TICK);
+ public String getBackTickExp() {
+ return column.isComputedColumn() ? column.getComputedColumnExpr() : wrapIdentity(BACK_TICK);
}
public String getTable() {
@@ -377,13 +371,13 @@ public class TblColRef implements Serializable {
if (column.getTable() == null) {
return "NULL";
} else {
- return column.getTable().getIdentity().toUpperCase(Locale.ROOT);
+ return StringUtils.upperCase(column.getTable().getIdentity());
}
}
// return DB.TABLE.COLUMN
public String getColumnWithTableAndSchema() {
- return (getTableWithSchema() + "." + column.getName()).toUpperCase(Locale.ROOT);
+ return StringUtils.upperCase(getTableWithSchema() + "." + column.getName());
}
public boolean isCastInnerColumn() {
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/util/ComputedColumnUtil.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/util/ComputedColumnUtil.java
index 773afd993d..09435140f8 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/util/ComputedColumnUtil.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/util/ComputedColumnUtil.java
@@ -45,7 +45,6 @@ import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.exception.KylinException;
import org.apache.kylin.common.msg.MsgPicker;
import org.apache.kylin.common.util.Pair;
-import org.apache.kylin.common.util.StringUtil;
import org.apache.kylin.metadata.cube.model.NDataflowManager;
import org.apache.kylin.metadata.model.BadModelException;
import org.apache.kylin.metadata.model.BadModelException.CauseType;
@@ -89,7 +88,7 @@ public class ComputedColumnUtil {
JoinsGraph newCCGraph = getCCExprRelatedSubgraph(newCC, newModel);
for (NDataModel existingModel : otherModels) {
for (ComputedColumnDesc existingCC : existingModel.getComputedColumnDescs()) {
- if (!StringUtil.equals(newCC.getTableIdentity(), existingCC.getTableIdentity())) {
+ if (!StringUtils.equals(newCC.getTableIdentity(), existingCC.getTableIdentity())) {
continue;
}
JoinsGraph existCCGraph = getCCExprRelatedSubgraph(existingCC, existingModel);
@@ -107,16 +106,6 @@ public class ComputedColumnUtil {
return null;
}
- public static BiMap<String, String> getAllCCNameAndExp(List<NDataModel> allModels) {
- BiMap<String, String> allCCNameAndExp = HashBiMap.create();
- for (NDataModel otherModel : allModels) {
- for (ComputedColumnDesc cc : otherModel.getComputedColumnDescs()) {
- allCCNameAndExp.put(cc.getColumnName(), cc.getExpression());
- }
- }
- return allCCNameAndExp;
- }
-
public static class ExprIdentifierFinder extends SqlBasicVisitor<SqlNode> {
List<Pair<String, String>> columnWithTableAlias;
@@ -147,7 +136,6 @@ public class ComputedColumnUtil {
@Override
public SqlNode visit(SqlIdentifier id) {
- //Preconditions.checkState(id.names.size() == 2, "error when get identifier in cc's expr");
if (id.names.size() == 2) {
columnWithTableAlias.add(Pair.newPair(id.names.get(0), id.names.get(1)));
}
@@ -155,10 +143,6 @@ public class ComputedColumnUtil {
}
}
- public static Map<String, Set<String>> getCCUsedColsMapWithProject(String project, ColumnDesc columnDesc) {
- return getCCUsedColsMapWithModel(getModel(project, columnDesc.getName()), columnDesc);
- }
-
public static Set<String> getCCUsedColsWithProject(String project, ColumnDesc columnDesc) {
NDataModel model = getModel(project, columnDesc.getName());
return getCCUsedColsWithModel(model, columnDesc);
@@ -201,21 +185,20 @@ public class ComputedColumnUtil {
public static Map<String, Set<String>> getCCUsedColsMap(NDataModel model, String colName) {
Map<String, Set<String>> usedCols = Maps.newHashMap();
Map<String, String> aliasTableMap = getAliasTableMap(model);
- Preconditions.checkState(aliasTableMap.size() > 0, "can not find cc:" + colName + "'s table alias");
+ Preconditions.checkState(aliasTableMap.size() > 0, "can not find cc:%s's table alias", colName);
ComputedColumnDesc targetCC = model.getComputedColumnDescs().stream()
.filter(cc -> cc.getColumnName().equalsIgnoreCase(colName)) //
.findFirst().orElse(null);
if (targetCC == null) {
- throw new RuntimeException("ComputedColumn(name: " + colName + ") is not on model: " + model.getUuid());
+ throw new IllegalStateException(
+ "ComputedColumn(name: " + colName + ") is not on model: " + model.getUuid());
}
List<Pair<String, String>> colsWithAlias = ExprIdentifierFinder.getExprIdentifiers(targetCC.getExpression());
for (Pair<String, String> cols : colsWithAlias) {
String tableIdentifier = aliasTableMap.get(cols.getFirst());
- if (!usedCols.containsKey(tableIdentifier)) {
- usedCols.put(tableIdentifier, Sets.newHashSet());
- }
+ usedCols.putIfAbsent(tableIdentifier, Sets.newHashSet());
usedCols.get(tableIdentifier).add(cols.getSecond());
}
return usedCols;
@@ -224,13 +207,12 @@ public class ComputedColumnUtil {
private static Set<String> getCCUsedCols(NDataModel model, String colName, String ccExpr) {
Set<String> usedCols = new HashSet<>();
Map<String, String> aliasTableMap = getAliasTableMap(model);
- Preconditions.checkState(aliasTableMap.size() > 0, "can not find cc:" + colName + "'s table alias");
+ Preconditions.checkState(aliasTableMap.size() > 0, "can not find cc:%s's table alias", colName);
List<Pair<String, String>> colsWithAlias = ExprIdentifierFinder.getExprIdentifiers(ccExpr);
for (Pair<String, String> cols : colsWithAlias) {
String tableIdentifier = aliasTableMap.get(cols.getFirst());
usedCols.add(tableIdentifier + "." + cols.getSecond());
}
- //Preconditions.checkState(usedCols.size() > 0, "can not find cc:" + columnDesc.getUuid() + "'s used cols");
return usedCols;
}
@@ -246,8 +228,7 @@ public class ComputedColumnUtil {
private static NDataModel getModel(String project, String ccName) {
List<NDataModel> models = NDataflowManager.getInstance(KylinConfig.getInstanceFromEnv(), project)
.listUnderliningDataModels();
- for (NDataModel modelDesc : models) {
- NDataModel model = modelDesc;
+ for (NDataModel model : models) {
Set<String> computedColumnNames = model.getComputedColumnNames();
if (computedColumnNames.contains(ccName)) {
return model;
@@ -337,25 +318,6 @@ public class ComputedColumnUtil {
col2.getTableIdentity() + "." + col2.getColumnName());
}
- public static boolean isLiteralSameCCExpr(ComputedColumnDesc existingCC, ComputedColumnDesc newCC) {
- String definition0 = existingCC.getExpression();
- String definition1 = newCC.getExpression();
-
- if (definition0 == null) {
- return definition1 == null;
- } else if (definition1 == null) {
- return false;
- }
-
- return isLiteralSameCCExprString(definition0, definition1);
- }
-
- public static boolean isLiteralSameCCExprString(String definition0, String definition1) {
- definition0 = StringUtils.replaceAll(definition0, "\\s*", "");
- definition1 = StringUtils.replaceAll(definition1, "\\s*", "");
- return definition0.equalsIgnoreCase(definition1);
- }
-
private static boolean isSameCCExpr(ComputedColumnDesc existingCC, ComputedColumnDesc newCC,
AliasMapping aliasMapping) {
if (existingCC.getExpression() == null) {
@@ -547,12 +509,13 @@ public class ComputedColumnUtil {
@Override
public void handleOnSingleModelSameExpr(NDataModel existingModel, ComputedColumnDesc existingCC,
ComputedColumnDesc newCC) {
- logger.error(
- String.format(Locale.ROOT, "In model %s, computed columns %s and %s have equivalent expressions.",
- existingModel.getAlias(), existingCC.getFullName(), newCC.getFullName()));
+ String ccFullName = newCC.getFullName();
+ String errorMsg = "In model " + existingModel.getAlias() + ", computed columns " + existingCC.getFullName()
+ + " and " + ccFullName + " have equivalent expressions.";
+ logger.error(errorMsg);
String msg = MsgPicker.getMsg().getComputedColumnExpressionDuplicatedSingleModel();
throw new BadModelException(DUPLICATE_COMPUTED_COLUMN_EXPRESSION, msg,
- BadModelException.CauseType.SELF_CONFLICT_WITH_SAME_EXPRESSION, null, null, newCC.getFullName());
+ BadModelException.CauseType.SELF_CONFLICT_WITH_SAME_EXPRESSION, null, null, ccFullName);
}
}
@@ -678,19 +641,6 @@ public class ComputedColumnUtil {
}
}
- public static List<Pair<ComputedColumnDesc, NDataModel>> getExistingCCs(String modelId,
- List<NDataModel> otherModels) {
- List<Pair<ComputedColumnDesc, NDataModel>> existingCCs = Lists.newArrayList();
- for (NDataModel otherModel : otherModels) {
- if (!StringUtils.equals(otherModel.getUuid(), modelId)) {
- for (ComputedColumnDesc cc : otherModel.getComputedColumnDescs()) {
- existingCCs.add(Pair.newPair(cc, otherModel));
- }
- }
- }
- return existingCCs;
- }
-
public static List<ComputedColumnDesc> getAuthorizedCC(List<NDataModel> modelList,
Predicate<Set<String>> isColumnAuthorizedFunc) {
val authorizedCC = Lists.<ComputedColumnDesc> newArrayList();
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectInstance.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectInstance.java
index ed9f738cd3..1e3ef22468 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectInstance.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectInstance.java
@@ -32,7 +32,7 @@ import org.apache.kylin.common.KylinConfigExt;
import org.apache.kylin.common.constant.NonCustomProjectLevelConfig;
import org.apache.kylin.common.persistence.ResourceStore;
import org.apache.kylin.common.persistence.RootPersistentEntity;
-import org.apache.kylin.common.util.StringUtil;
+import org.apache.kylin.common.util.StringHelper;
import org.apache.kylin.metadata.MetadataConstants;
import org.apache.kylin.metadata.model.AutoMergeTimeEnum;
import org.apache.kylin.metadata.model.ISourceAware;
@@ -356,7 +356,7 @@ public class ProjectInstance extends RootPersistentEntity implements ISourceAwar
for (String resource : modelResource) {
String[] path = resource.split("/");
resource = path[path.length - 1];
- resource = StringUtil.dropSuffix(resource, MetadataConstants.FILE_SURFIX);
+ resource = StringHelper.dropSuffix(resource, MetadataConstants.FILE_SURFIX);
nameList.add(resource);
}
return nameList;
@@ -369,7 +369,7 @@ public class ProjectInstance extends RootPersistentEntity implements ISourceAwar
public List<String> getEmailUsers() {
String users = this.getOverrideKylinProps().get(NonCustomProjectLevelConfig.NOTIFICATION_USER_EMAILS.getValue());
if(users != null) {
- return Arrays.asList(StringUtil.split(users, ","));
+ return Arrays.asList(StringHelper.split(users, ","));
}
return new ArrayList<>();
}
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/source/adhocquery/IPushDownRunner.java b/src/core-metadata/src/main/java/org/apache/kylin/source/adhocquery/IPushDownRunner.java
index d442600a19..b2f9fc0a0e 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/source/adhocquery/IPushDownRunner.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/source/adhocquery/IPushDownRunner.java
@@ -18,6 +18,7 @@
package org.apache.kylin.source.adhocquery;
+import java.sql.SQLException;
import java.util.List;
import org.apache.kylin.common.KylinConfig;
@@ -37,18 +38,18 @@ public interface IPushDownRunner {
void init(KylinConfig config);
/**
- * Run an pushdown query in the source database in case Kylin cannot serve using cube.
+ * Run a push-down query in the source database in case Kylin cannot serve using cube.
*
* @param query the query statement
* @param returnRows an empty list to collect returning rows
* @param returnColumnMeta an empty list to collect metadata of returning columns
* @param project the project name
- * @throws Exception if running pushdown query fails
+ * @throws SQLException if running pushdown query fails
*/
void executeQuery(String query, List<List<String>> returnRows, List<SelectedColumnMeta> returnColumnMeta,
- String project) throws Exception;
+ String project) throws SQLException;
- default PushdownResult executeQueryToIterator(String query, String project) throws Exception {
+ default PushdownResult executeQueryToIterator(String query, String project) throws SQLException {
List<List<String>> returnRows = Lists.newArrayList();
List<SelectedColumnMeta> returnColumnMeta = Lists.newArrayList();
executeQuery(query, returnRows, returnColumnMeta, project);
@@ -58,13 +59,12 @@ public interface IPushDownRunner {
/**
* Run an pushdown non-query sql
*
- * @param sql the sql statement
+ * @param sql the sql statement
+ * @param project the project
*
- * @return whether the SQL is executed successfully
- *
- * @throws Exception if running pushdown fails
+ * @throws SQLException if running pushdown fails
*/
- void executeUpdate(String sql, String project) throws Exception;
+ void executeUpdate(String sql, String project) throws SQLException;
String getName();
diff --git a/src/core-metadata/src/test/java/org/apache/kylin/metadata/cube/planner/algorithm/AlgorithmTestBase.java b/src/core-metadata/src/test/java/org/apache/kylin/metadata/cube/planner/algorithm/AlgorithmTestBase.java
index b1e4356db7..a4efd3523c 100644
--- a/src/core-metadata/src/test/java/org/apache/kylin/metadata/cube/planner/algorithm/AlgorithmTestBase.java
+++ b/src/core-metadata/src/test/java/org/apache/kylin/metadata/cube/planner/algorithm/AlgorithmTestBase.java
@@ -35,7 +35,7 @@ import java.util.Objects;
import java.util.Set;
import org.apache.commons.lang.StringUtils;
-import org.apache.kylin.common.util.StringUtil;
+import org.apache.kylin.common.util.StringHelper;
import org.junit.After;
import org.junit.Before;
@@ -357,7 +357,7 @@ public class AlgorithmTestBase {
StandardCharsets.UTF_8));
while ((sCurrentLine = br.readLine()) != null) {
- String[] statPair = StringUtil.split(sCurrentLine, " ");
+ String[] statPair = StringHelper.split(sCurrentLine, " ");
countMap.put(BigInteger.valueOf(Long.valueOf(statPair[0])), Long.valueOf(statPair[1]));
}
@@ -564,4 +564,4 @@ public class AlgorithmTestBase {
return scanCountMap;
}
-}
\ No newline at end of file
+}
diff --git a/src/core-metadata/src/test/java/org/apache/kylin/metadata/model/JoinsGraphTest.java b/src/core-metadata/src/test/java/org/apache/kylin/metadata/model/JoinsGraphTest.java
index 65a359e63f..65ad1e25da 100644
--- a/src/core-metadata/src/test/java/org/apache/kylin/metadata/model/JoinsGraphTest.java
+++ b/src/core-metadata/src/test/java/org/apache/kylin/metadata/model/JoinsGraphTest.java
@@ -29,8 +29,6 @@ import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.exception.KylinException;
import org.apache.kylin.common.util.NLocalFileMetadataTestCase;
import org.apache.kylin.common.util.Unsafe;
-import org.apache.kylin.metadata.model.graph.DefaultJoinEdgeMatcher;
-import org.apache.kylin.metadata.model.graph.JoinsGraph;
import org.apache.kylin.metadata.project.NProjectManager;
import org.junit.Assert;
import org.junit.Before;
@@ -351,7 +349,7 @@ public class JoinsGraphTest extends NLocalFileMetadataTestCase {
public void testColumnDescEquals() throws NoSuchMethodException, InvocationTargetException, IllegalAccessException {
NTableMetadataManager manager = NTableMetadataManager.getInstance(getTestConfig(), "default");
TableDesc tableDesc = manager.getTableDesc("DEFAULT.TEST_KYLIN_FACT");
- DefaultJoinEdgeMatcher matcher = new DefaultJoinEdgeMatcher();
+ JoinsGraph.DefaultJoinEdgeMatcher matcher = new JoinsGraph.DefaultJoinEdgeMatcher();
ColumnDesc one = new ColumnDesc();
one.setTable(tableDesc);
one.setName("one");
diff --git a/src/core-metadata/src/test/java/org/apache/kylin/metadata/model/util/ComputedColumnUtilTest.java b/src/core-metadata/src/test/java/org/apache/kylin/metadata/model/util/ComputedColumnUtilTest.java
index 1cdcca85fa..55f0bf990d 100644
--- a/src/core-metadata/src/test/java/org/apache/kylin/metadata/model/util/ComputedColumnUtilTest.java
+++ b/src/core-metadata/src/test/java/org/apache/kylin/metadata/model/util/ComputedColumnUtilTest.java
@@ -103,7 +103,7 @@ public class ComputedColumnUtilTest extends NLocalFileMetadataTestCase {
NDataModel modelNew = modelManager.getDataModelDesc(model.getUuid());
ColumnDesc column = new ColumnDesc();
column.setName("DEAL_AMOUNT");
- column.setComputedColumn("`TEST_KYLIN_FACT`.`PRICE` * `TEST_KYLIN_FACT`.`ITEM_COUNT`");
+ column.setComputedColumnExpr("`TEST_KYLIN_FACT`.`PRICE` * `TEST_KYLIN_FACT`.`ITEM_COUNT`");
Map<String, Set<String>> colsMapWithModel = ComputedColumnUtil.getCCUsedColsMapWithModel(modelNew, column);
Assert.assertEquals(1, colsMapWithModel.size());
Assert.assertTrue(colsMapWithModel.containsKey("DEFAULT.TEST_KYLIN_FACT"));
@@ -112,7 +112,7 @@ public class ComputedColumnUtilTest extends NLocalFileMetadataTestCase {
ColumnDesc notExistColumn = new ColumnDesc();
notExistColumn.setName("CC_NOT_EXIST");
- notExistColumn.setComputedColumn("`TEST_KYLIN_FACT`.`PRICE` * 0.95");
+ notExistColumn.setComputedColumnExpr("`TEST_KYLIN_FACT`.`PRICE` * 0.95");
try {
ComputedColumnUtil.getCCUsedColsMapWithModel(modelNew, notExistColumn);
Assert.fail();
diff --git a/src/data-loading-server/src/main/java/org/apache/kylin/rest/controller/SampleController.java b/src/data-loading-server/src/main/java/org/apache/kylin/rest/controller/SampleController.java
index 60f672d993..f13bc7f40b 100644
--- a/src/data-loading-server/src/main/java/org/apache/kylin/rest/controller/SampleController.java
+++ b/src/data-loading-server/src/main/java/org/apache/kylin/rest/controller/SampleController.java
@@ -88,12 +88,14 @@ public class SampleController extends BaseController {
@ResponseBody
public EnvelopeResponse<String> getPartitionColumnFormat(@RequestParam(value = "project") String project,
@RequestParam(value = "table") String table,
- @RequestParam(value = "partition_column") String partitionColumn) throws Exception {
+ @RequestParam(value = "partition_column") String partitionColumn,
+ @RequestParam(value = "partition_expression", required = false) String partitionExpression)
+ throws Exception {
checkProjectName(project);
checkRequiredArg(TABLE, table);
checkRequiredArg("partitionColumn", partitionColumn);
return new EnvelopeResponse<>(KylinException.CODE_SUCCESS,
- tableService.getPartitionColumnFormat(project, table, partitionColumn), "");
+ tableService.getPartitionColumnFormat(project, table, partitionColumn, partitionExpression), "");
}
@ApiOperation(value = "samplingJobs", tags = { "AI" })
diff --git a/src/data-loading-server/src/main/java/org/apache/kylin/rest/controller/open/OpenSampleController.java b/src/data-loading-server/src/main/java/org/apache/kylin/rest/controller/open/OpenSampleController.java
index de0bbdfd6b..4932580b7d 100644
--- a/src/data-loading-server/src/main/java/org/apache/kylin/rest/controller/open/OpenSampleController.java
+++ b/src/data-loading-server/src/main/java/org/apache/kylin/rest/controller/open/OpenSampleController.java
@@ -96,13 +96,14 @@ public class OpenSampleController extends BaseController {
@ResponseBody
public EnvelopeResponse<OpenPartitionColumnFormatResponse> getPartitionColumnFormat(
@RequestParam(value = "project") String project, @RequestParam(value = "table") String table,
- @RequestParam(value = "column_name") String columnName) throws Exception {
+ @RequestParam(value = "column_name") String columnName,
+ @RequestParam(value = "expression", required = false) String expression) throws Exception {
String projectName = checkProjectName(project);
checkRequiredArg(TABLE, table);
checkRequiredArg("column_name", columnName);
- String columnFormat = tableService.getPartitionColumnFormat(projectName,
- StringUtils.upperCase(table, Locale.ROOT), columnName);
+ String columnFormat = tableService.getPartitionColumnFormat(projectName, StringUtils.upperCase(table),
+ columnName, expression);
OpenPartitionColumnFormatResponse columnFormatResponse = new OpenPartitionColumnFormatResponse();
columnFormatResponse.setColumnName(columnName);
columnFormatResponse.setColumnFormat(columnFormat);
diff --git a/src/data-loading-server/src/test/java/org/apache/kylin/rest/controller/open/OpenSampleControllerTest.java b/src/data-loading-server/src/test/java/org/apache/kylin/rest/controller/open/OpenSampleControllerTest.java
index 0b494a9c30..442ed3c584 100644
--- a/src/data-loading-server/src/test/java/org/apache/kylin/rest/controller/open/OpenSampleControllerTest.java
+++ b/src/data-loading-server/src/test/java/org/apache/kylin/rest/controller/open/OpenSampleControllerTest.java
@@ -131,7 +131,7 @@ public class OpenSampleControllerTest extends NLocalFileMetadataTestCase {
.andExpect(MockMvcResultMatchers.status().isOk());
Mockito.verify(openSampleController).refreshSegments(Mockito.any(RefreshSegmentsRequest.class));
}
-
+
@Test
public void testSubmitSamplingCaseInsensitive() throws Exception {
String tableMixture = "dEFault.teST_kylIN_fact";
@@ -199,7 +199,7 @@ public class OpenSampleControllerTest extends NLocalFileMetadataTestCase {
.param("project", project).param("table", tableName).param("column_name", columnName)
.accept(MediaType.parseMediaType(HTTP_VND_APACHE_KYLIN_V4_PUBLIC_JSON))) //
.andExpect(MockMvcResultMatchers.status().isOk());
- Mockito.verify(openSampleController).getPartitionColumnFormat(project, tableName, columnName);
+ Mockito.verify(openSampleController).getPartitionColumnFormat(project, tableName, columnName, null);
}
{
@@ -216,7 +216,7 @@ public class OpenSampleControllerTest extends NLocalFileMetadataTestCase {
.accept(MediaType.parseMediaType(HTTP_VND_APACHE_KYLIN_V4_PUBLIC_JSON))) //
.andExpect(MockMvcResultMatchers.status().isOk());
Mockito.verify(tableService, Mockito.times(1)).getPartitionColumnFormat(project, tableNameUppercase,
- columnName);
+ columnName, null);
mockMvc.perform(MockMvcRequestBuilders.get("/api/tables/column_format") //
.contentType(MediaType.APPLICATION_JSON) //
@@ -224,7 +224,7 @@ public class OpenSampleControllerTest extends NLocalFileMetadataTestCase {
.accept(MediaType.parseMediaType(HTTP_VND_APACHE_KYLIN_V4_PUBLIC_JSON))) //
.andExpect(MockMvcResultMatchers.status().isOk());
Mockito.verify(tableService, Mockito.times(2)).getPartitionColumnFormat(project, tableNameUppercase,
- columnName);
+ columnName, null);
mockMvc.perform(MockMvcRequestBuilders.get("/api/tables/column_format") //
.contentType(MediaType.APPLICATION_JSON) //
@@ -232,7 +232,7 @@ public class OpenSampleControllerTest extends NLocalFileMetadataTestCase {
.accept(MediaType.parseMediaType(HTTP_VND_APACHE_KYLIN_V4_PUBLIC_JSON))) //
.andExpect(MockMvcResultMatchers.status().isOk());
Mockito.verify(tableService, Mockito.times(3)).getPartitionColumnFormat(project, tableNameUppercase,
- columnName);
+ columnName, null);
}
}
diff --git a/src/data-loading-service/src/main/java/org/apache/kylin/rest/service/JobService.java b/src/data-loading-service/src/main/java/org/apache/kylin/rest/service/JobService.java
index dc35003be5..0e0c4a6e9f 100644
--- a/src/data-loading-service/src/main/java/org/apache/kylin/rest/service/JobService.java
+++ b/src/data-loading-service/src/main/java/org/apache/kylin/rest/service/JobService.java
@@ -76,7 +76,7 @@ import org.apache.kylin.common.scheduler.JobDiscardNotifier;
import org.apache.kylin.common.scheduler.JobReadyNotifier;
import org.apache.kylin.common.util.JsonUtil;
import org.apache.kylin.common.util.Pair;
-import org.apache.kylin.common.util.StringUtil;
+import org.apache.kylin.common.util.StringHelper;
import org.apache.kylin.job.common.JobUtil;
import org.apache.kylin.job.common.ShellExecutable;
import org.apache.kylin.job.constant.ExecutableConstants;
@@ -140,6 +140,7 @@ import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
+import io.kyligence.kap.metadata.epoch.EpochManager;
import io.kyligence.kap.secondstorage.SecondStorageUtil;
import lombok.Getter;
import lombok.Setter;
@@ -1366,7 +1367,7 @@ public class JobService extends BasicService implements JobSupporter, ISmartAppl
return;
}
// The user's browser may contain multiple language preferences, such as xx,xx;ss,ss
- String language = StringUtil.dropFirstSuffix(StringUtil.dropFirstSuffix(languageToHandle, ";"), ",");
+ String language = StringHelper.dropFirstSuffix(StringHelper.dropFirstSuffix(languageToHandle, ";"), ",");
if (CHINESE_LANGUAGE.equals(language) || CHINESE_SIMPLE_LANGUAGE.equals(language)
|| CHINESE_HK_LANGUAGE.equals(language) || CHINESE_TW_LANGUAGE.equals(language)) {
ErrorCode.setMsg("cn");
diff --git a/src/data-loading-service/src/main/java/org/apache/kylin/rest/service/SnapshotService.java b/src/data-loading-service/src/main/java/org/apache/kylin/rest/service/SnapshotService.java
index 176c6281a2..1a40feb5ec 100644
--- a/src/data-loading-service/src/main/java/org/apache/kylin/rest/service/SnapshotService.java
+++ b/src/data-loading-service/src/main/java/org/apache/kylin/rest/service/SnapshotService.java
@@ -24,12 +24,38 @@ import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import org.apache.kylin.engine.spark.job.NSparkSnapshotJob;
import lombok.val;
+import static org.apache.kylin.common.exception.ServerErrorCode.COLUMN_NOT_EXIST;
+import static org.apache.kylin.common.exception.ServerErrorCode.DATABASE_NOT_EXIST;
+import static org.apache.kylin.common.exception.ServerErrorCode.INVALID_PARAMETER;
+import static org.apache.kylin.common.exception.ServerErrorCode.PERMISSION_DENIED;
+import static org.apache.kylin.common.exception.ServerErrorCode.SNAPSHOT_MANAGEMENT_NOT_ENABLED;
+import static org.apache.kylin.common.exception.ServerErrorCode.SNAPSHOT_NOT_EXIST;
+import static org.apache.kylin.common.exception.ServerErrorCode.TABLE_NOT_EXIST;
+import static org.apache.kylin.common.exception.code.ErrorCodeServer.JOB_CREATE_CHECK_FAIL;
+import static org.apache.kylin.common.exception.code.ErrorCodeServer.REQUEST_PARAMETER_EMPTY_OR_VALUE_EMPTY;
+import static org.apache.kylin.job.execution.JobTypeEnum.SNAPSHOT_BUILD;
+import static org.apache.kylin.job.execution.JobTypeEnum.SNAPSHOT_REFRESH;
+import static org.apache.kylin.rest.constant.SnapshotStatus.BROKEN;
+import static org.apache.kylin.rest.util.TableUtils.calculateTableSize;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
import org.apache.commons.lang3.StringUtils;
import org.apache.kylin.common.exception.KylinException;
import org.apache.kylin.common.exception.ServerErrorCode;
import org.apache.kylin.common.msg.MsgPicker;
import org.apache.kylin.common.util.Pair;
-import org.apache.kylin.common.util.StringUtil;
import org.apache.kylin.common.util.TimeUtil;
import org.apache.kylin.job.dao.ExecutablePO;
import org.apache.kylin.job.dao.JobStatisticsManager;
@@ -83,20 +109,6 @@ import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors;
import java.util.stream.Stream;
-import static org.apache.kylin.common.exception.ServerErrorCode.COLUMN_NOT_EXIST;
-import static org.apache.kylin.common.exception.ServerErrorCode.DATABASE_NOT_EXIST;
-import static org.apache.kylin.common.exception.ServerErrorCode.INVALID_PARAMETER;
-import static org.apache.kylin.common.exception.ServerErrorCode.PERMISSION_DENIED;
-import static org.apache.kylin.common.exception.ServerErrorCode.SNAPSHOT_MANAGEMENT_NOT_ENABLED;
-import static org.apache.kylin.common.exception.ServerErrorCode.SNAPSHOT_NOT_EXIST;
-import static org.apache.kylin.common.exception.ServerErrorCode.TABLE_NOT_EXIST;
-import static org.apache.kylin.common.exception.code.ErrorCodeServer.JOB_CREATE_CHECK_FAIL;
-import static org.apache.kylin.common.exception.code.ErrorCodeServer.REQUEST_PARAMETER_EMPTY_OR_VALUE_EMPTY;
-import static org.apache.kylin.job.execution.JobTypeEnum.SNAPSHOT_BUILD;
-import static org.apache.kylin.job.execution.JobTypeEnum.SNAPSHOT_REFRESH;
-import static org.apache.kylin.rest.constant.SnapshotStatus.BROKEN;
-import static org.apache.kylin.rest.util.TableUtils.calculateTableSize;
-
@Component("snapshotService")
public class SnapshotService extends BasicService implements SnapshotSupporter {
@@ -158,7 +170,7 @@ public class SnapshotService extends BasicService implements SnapshotSupporter {
}
public JobInfoResponse buildSnapshotsInner(SnapshotRequest snapshotsRequest, boolean isRefresh,
- Set<String> needBuildSnapshotTables, Set<TableDesc> tables) {
+ Set<String> needBuildSnapshotTables, Set<TableDesc> tables) {
val project = snapshotsRequest.getProject();
val options = snapshotsRequest.getOptions();
List<String> invalidSnapshotsToBuild = new ArrayList<>();
@@ -212,15 +224,15 @@ public class SnapshotService extends BasicService implements SnapshotSupporter {
}
private void updateTableDesc(String project, Set<TableDesc> tables,
- Map<String, SnapshotRequest.TableOption> finalOptions) {
+ Map<String, SnapshotRequest.TableOption> finalOptions) {
NTableMetadataManager tableManager = getManager(NTableMetadataManager.class, project);
for (TableDesc tableDesc : tables) {
SnapshotRequest.TableOption option = finalOptions.get(tableDesc.getIdentity());
if (tableDesc.isSnapshotHasBroken()
- || !StringUtil.equals(option.getPartitionCol(), tableDesc.getSelectedSnapshotPartitionCol())) {
+ || !StringUtils.equals(option.getPartitionCol(), tableDesc.getSelectedSnapshotPartitionCol())) {
TableDesc newTable = tableManager.copyForWrite(tableDesc);
newTable.setSnapshotHasBroken(false);
- if (!StringUtil.equals(option.getPartitionCol(), tableDesc.getSelectedSnapshotPartitionCol())) {
+ if (!StringUtils.equals(option.getPartitionCol(), tableDesc.getSelectedSnapshotPartitionCol())) {
newTable.setSelectedSnapshotPartitionCol(option.getPartitionCol());
}
tableManager.updateTableDesc(newTable);
@@ -229,7 +241,7 @@ public class SnapshotService extends BasicService implements SnapshotSupporter {
}
private static void invalidSnapshotsToBuild(Map<String, SnapshotRequest.TableOption> options,
- List<String> invalidSnapshotsToBuild) {
+ List<String> invalidSnapshotsToBuild) {
for (Map.Entry<String, SnapshotRequest.TableOption> entry : options.entrySet()) {
Set<String> partitionToBuild = entry.getValue().getPartitionsToBuild();
if (partitionToBuild != null && partitionToBuild.isEmpty()) {
@@ -243,7 +255,7 @@ public class SnapshotService extends BasicService implements SnapshotSupporter {
}
public JobInfoResponse buildSnapshots(SnapshotRequest snapshotsRequest, boolean isRefresh,
- Set<String> needBuildSnapshotTables) {
+ Set<String> needBuildSnapshotTables) {
val project = snapshotsRequest.getProject();
checkSnapshotManualManagement(project);
Set<TableDesc> tables = checkAndGetTable(project, needBuildSnapshotTables);
@@ -440,8 +452,8 @@ public class SnapshotService extends BasicService implements SnapshotSupporter {
@Override
public Pair<List<SnapshotInfoResponse>, Integer> getProjectSnapshots(String project, String table,
- Set<SnapshotStatus> statusFilter, Set<Boolean> partitionFilter, String sortBy, boolean isReversed,
- Pair<Integer, Integer> offsetAndLimit) {
+ Set<SnapshotStatus> statusFilter, Set<Boolean> partitionFilter, String sortBy, boolean isReversed,
+ Pair<Integer, Integer> offsetAndLimit) {
checkSnapshotManualManagement(project);
aclEvaluate.checkProjectReadPermission(project);
NTableMetadataManager nTableMetadataManager = getManager(NTableMetadataManager.class, project);
@@ -472,8 +484,8 @@ public class SnapshotService extends BasicService implements SnapshotSupporter {
}
TableExtDesc tableExtDesc = nTableMetadataManager.getOrCreateTableExt(tableDesc);
Pair<Integer, Integer> countPair = getModelCount(tableDesc);
- response.add(new SnapshotInfoResponse(tableDesc, tableExtDesc, tableDesc.getSnapshotTotalRows(), countPair.getFirst(),
- countPair.getSecond(), getSnapshotJobStatus(tableDesc, executables),
+ response.add(new SnapshotInfoResponse(tableDesc, tableExtDesc, tableDesc.getSnapshotTotalRows(),
+ countPair.getFirst(), countPair.getSecond(), getSnapshotJobStatus(tableDesc, executables),
getForbiddenColumns(tableDesc)));
satisfiedTableSize.getAndIncrement();
});
@@ -487,7 +499,8 @@ public class SnapshotService extends BasicService implements SnapshotSupporter {
// Here the positive order needs to be cut from the offset position backwards
Comparator<SnapshotInfoResponse> comparator = BasicService.propertyComparator(sortBy, !isReversed);
response.sort(comparator);
- return Pair.newPair(PagingUtil.cutPage(response, offsetAndLimit.getFirst(), offsetAndLimit.getSecond()), actualTableSize);
+ return Pair.newPair(PagingUtil.cutPage(response, offsetAndLimit.getFirst(), offsetAndLimit.getSecond()),
+ actualTableSize);
}
}
@@ -500,8 +513,8 @@ public class SnapshotService extends BasicService implements SnapshotSupporter {
}
public List<TableDesc> getFilteredTables(NTableMetadataManager nTableMetadataManager,
- Pair<String, String> databaseAndTable, boolean canUseACLGreenChannel, Set<String> finalAuthorizedTables,
- List<AbstractExecutable> executables, Set<SnapshotStatus> statusFilter, Set<Boolean> partitionFilter) {
+ Pair<String, String> databaseAndTable, boolean canUseACLGreenChannel, Set<String> finalAuthorizedTables,
+ List<AbstractExecutable> executables, Set<SnapshotStatus> statusFilter, Set<Boolean> partitionFilter) {
String finalDatabase = databaseAndTable.getFirst();
String finalTable = databaseAndTable.getSecond();
return nTableMetadataManager.listAllTables().stream().filter(tableDesc -> {
@@ -523,15 +536,14 @@ public class SnapshotService extends BasicService implements SnapshotSupporter {
return true;
}
return finalAuthorizedTables.contains(tableDesc.getIdentity());
- }).filter(tableDesc -> hasLoadedSnapshot(tableDesc, executables)
- ).filter(tableDesc -> statusFilter.isEmpty() || statusFilter.contains(getSnapshotJobStatus(tableDesc, executables))
- ).filter(tableDesc -> {
- if (partitionFilter.size() != 1) {
- return true;
- }
- boolean isPartition = partitionFilter.iterator().next();
- return isPartition != (tableDesc.getSelectedSnapshotPartitionCol() == null);
- }).collect(Collectors.toList());
+ }).filter(tableDesc -> hasLoadedSnapshot(tableDesc, executables)).filter(tableDesc -> statusFilter.isEmpty()
+ || statusFilter.contains(getSnapshotJobStatus(tableDesc, executables))).filter(tableDesc -> {
+ if (partitionFilter.size() != 1) {
+ return true;
+ }
+ boolean isPartition = partitionFilter.iterator().next();
+ return isPartition != (tableDesc.getSelectedSnapshotPartitionCol() == null);
+ }).collect(Collectors.toList());
}
private Pair<Integer, Integer> getModelCount(TableDesc tableDesc) {
diff --git a/src/data-loading-service/src/test/java/org/apache/kylin/rest/service/ModelServiceBuildTest.java b/src/data-loading-service/src/test/java/org/apache/kylin/rest/service/ModelServiceBuildTest.java
index ed87e6c628..cb2896da8f 100644
--- a/src/data-loading-service/src/test/java/org/apache/kylin/rest/service/ModelServiceBuildTest.java
+++ b/src/data-loading-service/src/test/java/org/apache/kylin/rest/service/ModelServiceBuildTest.java
@@ -95,7 +95,6 @@ import org.apache.kylin.metadata.query.QueryTimesResponse;
import org.apache.kylin.metadata.realization.RealizationStatusEnum;
import org.apache.kylin.metadata.recommendation.candidate.JdbcRawRecStore;
import org.apache.kylin.query.util.PushDownUtil;
-import org.apache.kylin.query.util.QueryUtil;
import org.apache.kylin.rest.config.initialize.ModelBrokenListener;
import org.apache.kylin.rest.request.ModelRequest;
import org.apache.kylin.rest.request.PartitionsRefreshRequest;
@@ -199,7 +198,7 @@ public class ModelServiceBuildTest extends SourceTestCase {
ReflectionTestUtils.setField(modelBuildService, "userGroupService", userGroupService);
ReflectionTestUtils.setField(semanticService, "expandableMeasureUtil",
new ExpandableMeasureUtil((model, ccDesc) -> {
- String ccExpression = QueryUtil.massageComputedColumn(model, model.getProject(), ccDesc,
+ String ccExpression = PushDownUtil.massageComputedColumn(model, model.getProject(), ccDesc,
AclPermissionUtil.createAclInfo(model.getProject(),
semanticService.getCurrentUserGroups()));
ccDesc.setInnerExpression(ccExpression);
@@ -787,7 +786,7 @@ public class ModelServiceBuildTest extends SourceTestCase {
NDataflowUpdate dataflowUpdate = new NDataflowUpdate(dataflow.getUuid());
dataflowUpdate.setToRemoveSegs(dataflow.getSegments().toArray(new NDataSegment[dataflow.getSegments().size()]));
dataflowManager.updateDataflow(dataflowUpdate);
- val minAndMaxTime = PushDownUtil.getMaxAndMinTime(modelUpdate.getPartitionDesc().getPartitionDateColumn(),
+ val minAndMaxTime = PushDownUtil.probeMinMaxTs(modelUpdate.getPartitionDesc().getPartitionDateColumn(),
modelUpdate.getRootFactTableName(), "default");
val dateFormat = DateFormat.proposeDateFormat(minAndMaxTime.getFirst());
modelBuildService.buildSegmentsManually("default", "89af4ee2-2cdb-4b07-b39e-4c29856309aa",
@@ -807,7 +806,7 @@ public class ModelServiceBuildTest extends SourceTestCase {
Assert.assertEquals(t1, dataflow.getSegments().get(0).getSegRange().getStart());
Assert.assertEquals(t2, dataflow.getSegments().get(0).getSegRange().getEnd());
- val result = PushDownUtil.getMaxAndMinTimeWithTimeOut(modelUpdate.getPartitionDesc().getPartitionDateColumn(),
+ val result = PushDownUtil.probeMinMaxTsWithTimeout(modelUpdate.getPartitionDesc().getPartitionDateColumn(),
modelUpdate.getRootFactTableName(), "default");
Assert.assertNotNull(result);
}
diff --git a/src/datasource-sdk/src/main/java/org/apache/kylin/sdk/datasource/PushDownRunnerSDKImpl.java b/src/datasource-sdk/src/main/java/org/apache/kylin/sdk/datasource/PushDownRunnerSDKImpl.java
index 5660e92ec9..1839d98c6b 100644
--- a/src/datasource-sdk/src/main/java/org/apache/kylin/sdk/datasource/PushDownRunnerSDKImpl.java
+++ b/src/datasource-sdk/src/main/java/org/apache/kylin/sdk/datasource/PushDownRunnerSDKImpl.java
@@ -53,7 +53,7 @@ public class PushDownRunnerSDKImpl implements IPushDownRunner {
@Override
public void executeQuery(String query, List<List<String>> returnRows, List<SelectedColumnMeta> returnColumnMeta,
- String project) throws Exception {
+ String project) throws SQLException {
query = dataSource.convertSql(query);
//extract column metadata
@@ -90,7 +90,7 @@ public class PushDownRunnerSDKImpl implements IPushDownRunner {
}
@Override
- public void executeUpdate(String sql, String project) throws Exception {
+ public void executeUpdate(String sql, String project) throws SQLException {
dataSource.executeUpdate(sql);
}
diff --git a/src/datasource-service/src/main/java/org/apache/kylin/rest/service/SparkSourceService.java b/src/datasource-service/src/main/java/org/apache/kylin/rest/service/SparkSourceService.java
index b26fcf61ac..4acf6676ae 100644
--- a/src/datasource-service/src/main/java/org/apache/kylin/rest/service/SparkSourceService.java
+++ b/src/datasource-service/src/main/java/org/apache/kylin/rest/service/SparkSourceService.java
@@ -34,15 +34,15 @@ import org.apache.kylin.common.exception.KylinException;
import org.apache.kylin.common.exception.ServerErrorCode;
import org.apache.kylin.common.msg.MsgPicker;
import org.apache.kylin.common.util.HadoopUtil;
-import org.apache.kylin.common.util.StringUtil;
-import org.apache.kylin.source.ISourceMetadataExplorer;
-import org.apache.kylin.source.SourceFactory;
+import org.apache.kylin.common.util.StringHelper;
import org.apache.kylin.metadata.model.NTableMetadataManager;
import org.apache.kylin.metadata.project.NProjectManager;
import org.apache.kylin.rest.request.DDLRequest;
import org.apache.kylin.rest.response.DDLResponse;
import org.apache.kylin.rest.response.ExportTablesResponse;
import org.apache.kylin.rest.response.TableNameResponse;
+import org.apache.kylin.source.ISourceMetadataExplorer;
+import org.apache.kylin.source.SourceFactory;
import org.apache.spark.sql.AnalysisException;
import org.apache.spark.sql.DDLDesc;
import org.apache.spark.sql.Dataset;
@@ -272,15 +272,15 @@ public class SparkSourceService extends BasicService {
ss.sql(CREATE_VIEW_P_LINEORDER);
createdTables.add(VIEW_P_LINEORDER);
}
- log.info("Load samples {} successfully", StringUtil.join(createdTables, ","));
+ log.info("Load samples {} successfully", StringHelper.join(createdTables, ","));
} finally {
lock.unlock();
}
return createdTables;
}
- private void loadSamples(SparkSession ss, SaveMode mode, String table, String tableName,
- String db, File file, String fileName) throws IOException {
+ private void loadSamples(SparkSession ss, SaveMode mode, String table, String tableName, String db, File file,
+ String fileName) throws IOException {
String filePath = file.getAbsolutePath();
FileSystem fileSystem = HadoopUtil.getWorkingFileSystem();
String hdfsPath = String.format(Locale.ROOT, "/tmp/%s", fileName);
@@ -289,13 +289,12 @@ public class SparkSourceService extends BasicService {
File[] parquetFiles = file.listFiles();
if (parquetFiles != null) {
for (File parquetFile : parquetFiles) {
- fileSystem.copyFromLocalFile(new Path(parquetFile.getAbsolutePath()),
- new Path(hdfsPath));
+ fileSystem.copyFromLocalFile(new Path(parquetFile.getAbsolutePath()), new Path(hdfsPath));
}
}
// KC-6666, check and delete location
- String tbLocation = String.format(Locale.ROOT, "%s/%s",
- ss.catalog().getDatabase(db).locationUri(), tableName);
+ String tbLocation = String.format(Locale.ROOT, "%s/%s", ss.catalog().getDatabase(db).locationUri(),
+ tableName);
FileSystem fs = FileSystem.get(ss.sparkContext().hadoopConfiguration());
Path path = new Path(tbLocation);
if (fs.exists(path)) {
@@ -305,8 +304,7 @@ public class SparkSourceService extends BasicService {
ss.read().parquet(hdfsPath).write().mode(mode).saveAsTable(table);
} catch (Exception e) {
log.error("Load sample {} failed.", fileName, e);
- throw new IllegalStateException(String.format(Locale.ROOT, "Load sample %s failed", fileName),
- e);
+ throw new IllegalStateException(String.format(Locale.ROOT, "Load sample %s failed", fileName), e);
} finally {
fileSystem.delete(new Path(hdfsPath), false);
}
diff --git a/src/datasource-service/src/main/java/org/apache/kylin/rest/service/TableService.java b/src/datasource-service/src/main/java/org/apache/kylin/rest/service/TableService.java
index efa49e11b1..fbde97f9ec 100644
--- a/src/datasource-service/src/main/java/org/apache/kylin/rest/service/TableService.java
+++ b/src/datasource-service/src/main/java/org/apache/kylin/rest/service/TableService.java
@@ -228,20 +228,24 @@ public class TableService extends BasicService {
@Autowired
private ClusterManager clusterManager;
- public Pair<List<TableDesc>, Integer> getTableDesc(String project, boolean withExt, final String table, final String database,
- boolean isFuzzy, List<Integer> sourceType, int returnTableSize) throws IOException {
- TableDescRequest internalTableDescRequest = new TableDescRequest(project, withExt, table, database, isFuzzy, sourceType);
+ public Pair<List<TableDesc>, Integer> getTableDesc(String project, boolean withExt, final String table,
+ final String database, boolean isFuzzy, List<Integer> sourceType, int returnTableSize) throws IOException {
+ TableDescRequest internalTableDescRequest = new TableDescRequest(project, withExt, table, database, isFuzzy,
+ sourceType);
return getTableDesc(internalTableDescRequest, returnTableSize);
}
- public Pair<List<TableDesc>, Integer> getTableDesc(TableDescRequest tableDescRequest, int returnTableSize) throws IOException {
+ public Pair<List<TableDesc>, Integer> getTableDesc(TableDescRequest tableDescRequest, int returnTableSize)
+ throws IOException {
aclEvaluate.checkProjectReadPermission(tableDescRequest.getProject());
boolean streamingEnabled = getConfig().streamingEnabled();
- NTableMetadataManager nTableMetadataManager = getManager(NTableMetadataManager.class, tableDescRequest.getProject());
+ NTableMetadataManager nTableMetadataManager = getManager(NTableMetadataManager.class,
+ tableDescRequest.getProject());
List<TableDesc> tables = Lists.newArrayList();
//get table not fuzzy,can use getTableDesc(tableName)
if (StringUtils.isNotEmpty(tableDescRequest.getTable()) && !tableDescRequest.isFuzzy()) {
- val tableDesc = nTableMetadataManager.getTableDesc(tableDescRequest.getDatabase() + "." + tableDescRequest.getTable());
+ val tableDesc = nTableMetadataManager
+ .getTableDesc(tableDescRequest.getDatabase() + "." + tableDescRequest.getTable());
if (tableDesc != null && tableDesc.isAccessible(streamingEnabled))
tables.add(tableDesc);
} else {
@@ -254,7 +258,8 @@ public class TableService extends BasicService {
if (StringUtils.isEmpty(tableDescRequest.getTable())) {
return true;
}
- return tableDesc.getName().toLowerCase(Locale.ROOT).contains(tableDescRequest.getTable().toLowerCase(Locale.ROOT));
+ return tableDesc.getName().toLowerCase(Locale.ROOT)
+ .contains(tableDescRequest.getTable().toLowerCase(Locale.ROOT));
}).filter(tableDesc -> {
// Advance the logic of filtering the table by sourceType to here
if (!tableDescRequest.getSourceType().isEmpty()) {
@@ -451,7 +456,8 @@ public class TableService extends BasicService {
return tableDescResponse;
}
- private Pair<List<TableDesc>, Integer> getTablesResponse(List<TableDesc> tables, String project, boolean withExt, int returnTableSize) {
+ private Pair<List<TableDesc>, Integer> getTablesResponse(List<TableDesc> tables, String project, boolean withExt,
+ int returnTableSize) {
List<TableDesc> descs = new ArrayList<>();
val projectManager = getManager(NProjectManager.class);
val groups = getCurrentUserGroups();
@@ -515,8 +521,7 @@ public class TableService extends BasicService {
AclTCRManager manager = getManager(AclTCRManager.class, project);
Map<Integer, AclTCR.ColumnRealRows> columnRows = Arrays.stream(rtableDesc.getExtColumns()).map(cdr -> {
int id = Integer.parseInt(cdr.getId());
- val columnRealRows = manager.getAuthorizedRows(dbTblName, cdr.getName(),
- aclTCRS);
+ val columnRealRows = manager.getAuthorizedRows(dbTblName, cdr.getName(), aclTCRS);
return new AbstractMap.SimpleEntry<>(id, columnRealRows);
}).collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
for (String[] row : rtableDesc.getSamplingRows()) {
@@ -711,7 +716,8 @@ public class TableService extends BasicService {
() -> jobManager.addSegmentJob(new JobParam(newSegment, model, getUsername())));
}
- public String getPartitionColumnFormat(String project, String table, String partitionColumn) throws Exception {
+ public String getPartitionColumnFormat(String project, String table, String partitionColumn,
+ String partitionExpression) throws Exception {
aclEvaluate.checkProjectOperationPermission(project);
NTableMetadataManager tableManager = getManager(NTableMetadataManager.class, project);
@@ -719,7 +725,7 @@ public class TableService extends BasicService {
Preconditions.checkNotNull(tableDesc, String.format(Locale.ROOT, MsgPicker.getMsg().getTableNotFound(), table));
Set<String> columnSet = Stream.of(tableDesc.getColumns()).map(ColumnDesc::getName)
.map(str -> str.toUpperCase(Locale.ROOT)).collect(Collectors.toSet());
- if (!columnSet.contains(partitionColumn.toUpperCase(Locale.ROOT))) {
+ if (!columnSet.contains(StringUtils.upperCase(partitionColumn))) {
throw new KylinException(COLUMN_NOT_EXIST, String.format(Locale.ROOT,
"Can not find the column:%s in table:%s, project:%s", partitionColumn, table, project));
}
@@ -735,11 +741,14 @@ public class TableService extends BasicService {
mapping.forEach((key, value) -> mappingAllCaps.put(key.toUpperCase(Locale.ROOT), value));
String cell = (String) mappingAllCaps.get(partitionColumn);
return DateFormat.proposeDateFormat(cell);
+ } else if (partitionExpression == null) {
+ List<String> list = PushDownUtil.backtickQuote(partitionColumn.split("\\."));
+ String cell = PushDownUtil.probeColFormat(table, String.join(".", list), project);
+ return DateFormat.proposeDateFormat(cell);
} else {
- String cell = PushDownUtil.getFormatIfNotExist(table, partitionColumn, project);
+ String cell = PushDownUtil.probeExpFormat(table, partitionExpression, project);
return DateFormat.proposeDateFormat(cell);
}
-
} catch (KylinException e) {
throw e;
} catch (Exception e) {
@@ -1786,12 +1795,14 @@ public class TableService extends BasicService {
}
public NInitTablesResponse getProjectTables(String project, String table, int offset, int limit,
- boolean withExcluded, boolean useHiveDatabase, List<Integer> sourceType) throws Exception {
- TableDescRequest internalTableDescRequest = new TableDescRequest(project, table, offset, limit, withExcluded, sourceType);
+ boolean withExcluded, boolean useHiveDatabase, List<Integer> sourceType) throws Exception {
+ TableDescRequest internalTableDescRequest = new TableDescRequest(project, table, offset, limit, withExcluded,
+ sourceType);
return getProjectTables(internalTableDescRequest, useHiveDatabase);
}
- public NInitTablesResponse getProjectTables(TableDescRequest tableDescRequest, boolean useHiveDatabase) throws Exception {
+ public NInitTablesResponse getProjectTables(TableDescRequest tableDescRequest, boolean useHiveDatabase)
+ throws Exception {
String project = tableDescRequest.getProject();
aclEvaluate.checkProjectReadPermission(project);
NInitTablesResponse response = new NInitTablesResponse();
@@ -1825,12 +1836,14 @@ public class TableService extends BasicService {
objWithActualSize.setSecond(hiveTableNameResponses.size());
} else {
int returnTableSize = calculateTableSize(tableDescRequest.getOffset(), tableDescRequest.getLimit());
- Pair<List<TableDesc>, Integer> tableDescWithActualSize = getTableDesc(tableDescRequest, returnTableSize);
+ Pair<List<TableDesc>, Integer> tableDescWithActualSize = getTableDesc(tableDescRequest,
+ returnTableSize);
objWithActualSize.setFirst(tableDescWithActualSize.getFirst());
objWithActualSize.setSecond(tableDescWithActualSize.getSecond());
}
table = notAllowedModifyTableName;
- List<?> tablePage = PagingUtil.cutPage(objWithActualSize.getFirst(), tableDescRequest.getOffset(), tableDescRequest.getLimit());
+ List<?> tablePage = PagingUtil.cutPage(objWithActualSize.getFirst(), tableDescRequest.getOffset(),
+ tableDescRequest.getLimit());
if (!tablePage.isEmpty()) {
response.putDatabase(database, objWithActualSize.getSecond(), tablePage);
}
@@ -2013,7 +2026,7 @@ public class TableService extends BasicService {
public void refreshTable(String table, List<String> refreshed, List<String> failed) {
try {
- PushDownUtil.trySimplePushDownExecute("REFRESH TABLE " + table, null);
+ PushDownUtil.trySimplyExecute("REFRESH TABLE " + table, null);
refreshed.add(table);
} catch (Exception e) {
failed.add(table);
diff --git a/src/kylin-it/src/test/java/org/apache/kylin/metadata/model/AntiFlatCheckerTest.java b/src/kylin-it/src/test/java/org/apache/kylin/metadata/model/AntiFlatCheckerTest.java
index 02d0f0f185..bb7a59013c 100644
--- a/src/kylin-it/src/test/java/org/apache/kylin/metadata/model/AntiFlatCheckerTest.java
+++ b/src/kylin-it/src/test/java/org/apache/kylin/metadata/model/AntiFlatCheckerTest.java
@@ -23,7 +23,7 @@ import java.util.List;
import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.util.JsonUtil;
import org.apache.kylin.junit.annotation.MetadataInfo;
-import org.apache.kylin.query.util.QueryUtil;
+import org.apache.kylin.query.util.PushDownUtil;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
@@ -42,7 +42,7 @@ class AntiFlatCheckerTest {
NDataModelManager modelManager = NDataModelManager.getInstance(kylinConfig, getProject());
NDataModel model = modelManager.getDataModelDesc(modelId);
model.getComputedColumnDescs().forEach(cc -> {
- String innerExp = QueryUtil.massageComputedColumn(model, getProject(), cc, null);
+ String innerExp = PushDownUtil.massageComputedColumn(model, getProject(), cc, null);
cc.setInnerExpression(innerExp);
});
model.init(kylinConfig, getProject(), Lists.newArrayList());
diff --git a/src/kylin-it/src/test/java/org/apache/kylin/metadata/model/ColExcludedCheckerTest.java b/src/kylin-it/src/test/java/org/apache/kylin/metadata/model/ColExcludedCheckerTest.java
index 82c7bac737..996bd70094 100644
--- a/src/kylin-it/src/test/java/org/apache/kylin/metadata/model/ColExcludedCheckerTest.java
+++ b/src/kylin-it/src/test/java/org/apache/kylin/metadata/model/ColExcludedCheckerTest.java
@@ -23,7 +23,7 @@ import java.util.Set;
import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.junit.annotation.MetadataInfo;
-import org.apache.kylin.query.util.QueryUtil;
+import org.apache.kylin.query.util.PushDownUtil;
import org.apache.kylin.util.MetadataTestUtils;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
@@ -44,7 +44,7 @@ class ColExcludedCheckerTest {
NDataModelManager modelManager = NDataModelManager.getInstance(kylinConfig, getProject());
NDataModel model = modelManager.getDataModelDesc(modelId);
model.getComputedColumnDescs().forEach(cc -> {
- String innerExp = QueryUtil.massageComputedColumn(model, getProject(), cc, null);
+ String innerExp = PushDownUtil.massageComputedColumn(model, getProject(), cc, null);
cc.setInnerExpression(innerExp);
});
model.init(kylinConfig, getProject(), Lists.newArrayList());
diff --git a/src/kylin-it/src/test/java/org/apache/kylin/newten/NBadQueryAndPushDownTest.java b/src/kylin-it/src/test/java/org/apache/kylin/newten/NBadQueryAndPushDownTest.java
index 22072e7d93..268045a66a 100644
--- a/src/kylin-it/src/test/java/org/apache/kylin/newten/NBadQueryAndPushDownTest.java
+++ b/src/kylin-it/src/test/java/org/apache/kylin/newten/NBadQueryAndPushDownTest.java
@@ -36,7 +36,7 @@ import org.apache.kylin.engine.spark.NLocalWithSparkSessionTest;
import org.apache.kylin.job.impl.threadpool.NDefaultScheduler;
import org.apache.kylin.metadata.querymeta.SelectedColumnMeta;
import org.apache.kylin.metadata.realization.NoRealizationFoundException;
-import org.apache.kylin.query.util.PushDownUtil;
+import org.apache.kylin.query.KylinTestBase;
import org.apache.kylin.query.util.QueryParams;
import org.apache.kylin.query.util.QueryUtil;
import org.apache.kylin.util.ExecAndComp;
@@ -253,13 +253,13 @@ public class NBadQueryAndPushDownTest extends NLocalWithSparkSessionTest {
int offset, SQLException sqlException, boolean isForced) throws Exception {
populateSSWithCSVData(KylinConfig.getInstanceFromEnv(), prjName, SparderEnv.getSparkSession());
String pushdownSql = ExecAndComp.removeDataBaseInSql(sql);
- String massagedSql = QueryUtil.normalMassageSql(KylinConfig.getInstanceFromEnv(), pushdownSql, limit, offset);
+ String massagedSql = QueryUtil.appendLimitOffset(prjName, pushdownSql, limit, offset);
QueryParams queryParams = new QueryParams(prjName, massagedSql, "DEFAULT", BackdoorToggles.getPrepareOnly(),
sqlException, isForced);
queryParams.setSelect(true);
queryParams.setLimit(limit);
queryParams.setOffset(offset);
- Pair<List<List<String>>, List<SelectedColumnMeta>> result = PushDownUtil.tryPushDownQuery(queryParams);
+ Pair<List<List<String>>, List<SelectedColumnMeta>> result = KylinTestBase.tryPushDownQuery(queryParams);
if (result == null) {
throw sqlException;
}
diff --git a/src/kylin-it/src/test/java/org/apache/kylin/query/KylinTestBase.java b/src/kylin-it/src/test/java/org/apache/kylin/query/KylinTestBase.java
index c538c0dd1e..be3d732e46 100644
--- a/src/kylin-it/src/test/java/org/apache/kylin/query/KylinTestBase.java
+++ b/src/kylin-it/src/test/java/org/apache/kylin/query/KylinTestBase.java
@@ -51,8 +51,11 @@ import org.apache.kylin.util.ExecAndComp;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import com.google.common.collect.ImmutableList;
import com.google.common.io.Files;
+import lombok.val;
+
/**
*/
public class KylinTestBase extends NLocalFileMetadataTestCase {
@@ -134,13 +137,22 @@ public class KylinTestBase extends NLocalFileMetadataTestCase {
public Pair<List<List<String>>, List<SelectedColumnMeta>> tryPushDownSelectQuery(String project, String sql,
String defaultSchema, SQLException sqlException, boolean isPrepare, boolean isForced) throws Exception {
- String massagedSql = QueryUtil.normalMassageSql(KylinConfig.getInstanceFromEnv(), sql, 0, 0);
+ String massagedSql = QueryUtil.appendLimitOffset(project, sql, 0, 0);
QueryParams queryParams = new QueryParams(project, massagedSql, defaultSchema, isPrepare, sqlException,
isForced);
queryParams.setSelect(true);
queryParams.setLimit(0);
queryParams.setOffset(0);
- return PushDownUtil.tryPushDownQuery(queryParams);
+ return tryPushDownQuery(queryParams);
+ }
+
+ public static Pair<List<List<String>>, List<SelectedColumnMeta>> tryPushDownQuery(QueryParams queryParams)
+ throws Exception {
+ val results = PushDownUtil.tryIterQuery(queryParams);
+ if (results == null) {
+ return null;
+ }
+ return new Pair<>(ImmutableList.copyOf(results.getRows()), results.getColumnMetas());
}
// end of execute
diff --git a/src/kylin-it/src/test/java/org/apache/kylin/util/ExecAndComp.java b/src/kylin-it/src/test/java/org/apache/kylin/util/ExecAndComp.java
index c3fd6c4df3..801ae07b65 100644
--- a/src/kylin-it/src/test/java/org/apache/kylin/util/ExecAndComp.java
+++ b/src/kylin-it/src/test/java/org/apache/kylin/util/ExecAndComp.java
@@ -45,6 +45,7 @@ import org.apache.kylin.metadata.query.StructField;
import org.apache.kylin.query.engine.QueryExec;
import org.apache.kylin.query.engine.data.QueryResult;
import org.apache.kylin.query.relnode.OLAPContext;
+import org.apache.kylin.query.util.PushDownUtil;
import org.apache.kylin.query.util.QueryParams;
import org.apache.kylin.query.util.QueryUtil;
import org.apache.spark.sql.Dataset;
@@ -175,7 +176,7 @@ public class ExecAndComp {
QueryParams queryParams = new QueryParams(prj, compareSql, "default", false);
queryParams.setKylinConfig(NProjectManager.getProjectConfig(prj));
- String afterConvert = QueryUtil.massagePushDownSql(queryParams);
+ String afterConvert = PushDownUtil.massagePushDownSql(queryParams);
// Table schema comes from csv and DATABASE.TABLE is not supported.
String sqlForSpark = removeDataBaseInSql(afterConvert);
val ds = querySparkSql(sqlForSpark);
diff --git a/src/metadata-server/src/main/java/org/apache/kylin/rest/controller/NTableController.java b/src/metadata-server/src/main/java/org/apache/kylin/rest/controller/NTableController.java
index 1dfd998028..b685328a42 100644
--- a/src/metadata-server/src/main/java/org/apache/kylin/rest/controller/NTableController.java
+++ b/src/metadata-server/src/main/java/org/apache/kylin/rest/controller/NTableController.java
@@ -42,7 +42,7 @@ import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.exception.KylinException;
import org.apache.kylin.common.msg.MsgPicker;
import org.apache.kylin.common.util.Pair;
-import org.apache.kylin.common.util.StringUtil;
+import org.apache.kylin.common.util.StringHelper;
import org.apache.kylin.metadata.model.TableDesc;
import org.apache.kylin.metadata.project.NProjectManager;
import org.apache.kylin.rest.request.AWSTableLoadRequest;
@@ -117,7 +117,8 @@ public class NTableController extends NBasicController {
"AI" }, notes = "Update Param: is_fuzzy, page_offset, page_size; Update Response: no format!")
@GetMapping(value = "", produces = { HTTP_VND_APACHE_KYLIN_JSON })
@ResponseBody
- public EnvelopeResponse<Map<String, Object>> getTableDesc(@RequestParam(value = "ext", required = false) boolean withExt,
+ public EnvelopeResponse<Map<String, Object>> getTableDesc(
+ @RequestParam(value = "ext", required = false) boolean withExt,
@RequestParam(value = "project") String project,
@RequestParam(value = "table", required = false) String table,
@RequestParam(value = "database", required = false) String database,
@@ -133,7 +134,8 @@ public class NTableController extends NBasicController {
int returnTableSize = calculateTableSize(offset, limit);
TableDescRequest tableDescRequest = new TableDescRequest(project, table, database, withExt, isFuzzy,
Pair.newPair(offset, limit), Collections.singletonList(sourceType));
- Pair<List<TableDesc>, Integer> tableDescWithActualSize = tableService.getTableDesc(tableDescRequest, returnTableSize);
+ Pair<List<TableDesc>, Integer> tableDescWithActualSize = tableService.getTableDesc(tableDescRequest,
+ returnTableSize);
// Finally, the results are processed based on the paging parameters and returned to the front-end UI,
// where the results table to be processed each time is getting longer as the number of paging increases
Map<String, Object> mockDataResponse = setCustomDataResponse("tables", tableDescWithActualSize, offset, limit);
@@ -155,8 +157,8 @@ public class NTableController extends NBasicController {
@RequestParam(value = "source_type", required = false, defaultValue = "9") List<Integer> sourceType)
throws Exception {
checkProjectName(project);
- TableDescRequest tableDescRequest = new TableDescRequest(project, table, "", withExt, isFuzzy,
- offset, limit, sourceType, withExcluded);
+ TableDescRequest tableDescRequest = new TableDescRequest(project, table, "", withExt, isFuzzy, offset, limit,
+ sourceType, withExcluded);
NInitTablesResponse projectTables = tableService.getProjectTables(tableDescRequest, false);
return new EnvelopeResponse<>(KylinException.CODE_SUCCESS, projectTables, "");
}
@@ -230,7 +232,7 @@ public class NTableController extends NBasicController {
LoadTableResponse loadTableResponse = new LoadTableResponse();
if (ArrayUtils.isNotEmpty(tableLoadRequest.getTables())) {
- StringUtil.toUpperCaseArray(tableLoadRequest.getTables(), tableLoadRequest.getTables());
+ StringHelper.toUpperCaseArray(tableLoadRequest.getTables(), tableLoadRequest.getTables());
LoadTableResponse loadByTable = tableExtService.loadDbTables(tableLoadRequest.getTables(),
tableLoadRequest.getProject(), false);
loadTableResponse.getFailed().addAll(loadByTable.getFailed());
@@ -238,7 +240,7 @@ public class NTableController extends NBasicController {
}
if (ArrayUtils.isNotEmpty(tableLoadRequest.getDatabases())) {
- StringUtil.toUpperCaseArray(tableLoadRequest.getDatabases(), tableLoadRequest.getDatabases());
+ StringHelper.toUpperCaseArray(tableLoadRequest.getDatabases(), tableLoadRequest.getDatabases());
LoadTableResponse loadByDb = tableExtService.loadDbTables(tableLoadRequest.getDatabases(),
tableLoadRequest.getProject(), true);
loadTableResponse.getFailed().addAll(loadByDb.getFailed());
@@ -351,8 +353,8 @@ public class NTableController extends NBasicController {
@RequestParam(value = "page_offset", required = false, defaultValue = "0") Integer offset,
@RequestParam(value = "page_size", required = false, defaultValue = "10") Integer limit) throws Exception {
String projectName = checkProjectName(project);
- NInitTablesResponse data = tableService.getProjectTables(projectName, table, offset, limit, true,
- true, Collections.emptyList());
+ NInitTablesResponse data = tableService.getProjectTables(projectName, table, offset, limit, true, true,
+ Collections.emptyList());
return new EnvelopeResponse<>(KylinException.CODE_SUCCESS, data, "");
}
diff --git a/src/metadata-server/src/test/java/org/apache/kylin/rest/controller/NTableControllerTest.java b/src/metadata-server/src/test/java/org/apache/kylin/rest/controller/NTableControllerTest.java
index 1f0e6aa271..a1f41efe75 100644
--- a/src/metadata-server/src/test/java/org/apache/kylin/rest/controller/NTableControllerTest.java
+++ b/src/metadata-server/src/test/java/org/apache/kylin/rest/controller/NTableControllerTest.java
@@ -34,7 +34,7 @@ import org.apache.kylin.common.exception.KylinException;
import org.apache.kylin.common.util.JsonUtil;
import org.apache.kylin.common.util.NLocalFileMetadataTestCase;
import org.apache.kylin.common.util.Pair;
-import org.apache.kylin.common.util.StringUtil;
+import org.apache.kylin.common.util.StringHelper;
import org.apache.kylin.metadata.model.TableDesc;
import org.apache.kylin.rest.constant.Constant;
import org.apache.kylin.rest.request.AWSTableLoadRequest;
@@ -152,8 +152,8 @@ public class NTableControllerTest extends NLocalFileMetadataTestCase {
@Test
public void testGetTableDesc() throws Exception {
- TableDescRequest mockTableDescRequest = new TableDescRequest("default", "", "DEFAULT", false,
- true, Pair.newPair(0, 10), Collections.singletonList(9));
+ TableDescRequest mockTableDescRequest = new TableDescRequest("default", "", "DEFAULT", false, true,
+ Pair.newPair(0, 10), Collections.singletonList(9));
Mockito.when(tableService.getTableDesc(mockTableDescRequest, 10)).thenReturn(Pair.newPair(mockTables(), 10));
@@ -294,8 +294,8 @@ public class NTableControllerTest extends NLocalFileMetadataTestCase {
}
private void initMockito(LoadTableResponse loadTableResponse, TableLoadRequest tableLoadRequest) throws Exception {
- StringUtil.toUpperCaseArray(tableLoadRequest.getTables(), tableLoadRequest.getTables());
- StringUtil.toUpperCaseArray(tableLoadRequest.getDatabases(), tableLoadRequest.getDatabases());
+ StringHelper.toUpperCaseArray(tableLoadRequest.getTables(), tableLoadRequest.getTables());
+ StringHelper.toUpperCaseArray(tableLoadRequest.getDatabases(), tableLoadRequest.getDatabases());
Mockito.when(tableExtService.loadDbTables(tableLoadRequest.getTables(), "default", false))
.thenReturn(loadTableResponse);
Mockito.when(tableExtService.loadDbTables(tableLoadRequest.getDatabases(), "default", true))
diff --git a/src/modeling-service/src/main/java/org/apache/kylin/rest/service/ModelSemanticHelper.java b/src/modeling-service/src/main/java/org/apache/kylin/rest/service/ModelSemanticHelper.java
index b7a21eb955..147b413710 100644
--- a/src/modeling-service/src/main/java/org/apache/kylin/rest/service/ModelSemanticHelper.java
+++ b/src/modeling-service/src/main/java/org/apache/kylin/rest/service/ModelSemanticHelper.java
@@ -38,6 +38,7 @@ import java.util.function.Function;
import java.util.stream.Collectors;
import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.dialect.CalciteSqlDialect;
import org.apache.calcite.sql.dialect.HiveSqlDialect;
import org.apache.calcite.sql.util.SqlVisitor;
import org.apache.commons.collections.CollectionUtils;
@@ -96,6 +97,7 @@ import org.apache.kylin.metadata.model.util.scd2.SimplifiedJoinTableDesc;
import org.apache.kylin.metadata.project.NProjectManager;
import org.apache.kylin.metadata.recommendation.ref.OptRecManagerV2;
import org.apache.kylin.query.util.QueryUtil;
+import org.apache.kylin.query.util.PushDownUtil;
import org.apache.kylin.rest.request.ModelRequest;
import org.apache.kylin.rest.response.BuildIndexResponse;
import org.apache.kylin.rest.response.SimplifiedMeasure;
@@ -131,7 +133,7 @@ public class ModelSemanticHelper extends BasicService {
private static final Logger logger = LoggerFactory.getLogger(ModelSemanticHelper.class);
private final ExpandableMeasureUtil expandableMeasureUtil = new ExpandableMeasureUtil((model, ccDesc) -> {
- String ccExpression = QueryUtil.massageComputedColumn(model, model.getProject(), ccDesc,
+ String ccExpression = PushDownUtil.massageComputedColumn(model, model.getProject(), ccDesc,
AclPermissionUtil.createAclInfo(model.getProject(), getCurrentUserGroups()));
ccDesc.setInnerExpression(ccExpression);
ComputedColumnEvalUtil.evaluateExprAndType(model, ccDesc);
@@ -158,13 +160,22 @@ public class ModelSemanticHelper extends BasicService {
logger.error("Parse json failed...", e);
throw new KylinException(CommonErrorCode.FAILED_PARSE_JSON, e);
}
+
+ Map<String, TableDesc> allTablesMap = getManager(NTableMetadataManager.class, modelRequest.getProject())
+ .getAllTablesMap();
+ List<ComputedColumnDesc> ccList = dataModel.getComputedColumnDescs();
+ if (!ccList.isEmpty()) {
+ String factTableIdentity = dataModel.getRootFactTableName();
+ TableDesc tableDesc = allTablesMap.get(factTableIdentity);
+ TableDesc extendTable = tableDesc.appendColumns(ComputedColumnUtil.createComputedColumns(ccList, tableDesc),
+ true);
+ allTablesMap.put(factTableIdentity, extendTable);
+ }
dataModel.setUuid(modelRequest.getUuid() != null ? modelRequest.getUuid() : RandomUtil.randomUUIDStr());
dataModel.setProject(modelRequest.getProject());
dataModel.setAllMeasures(convertMeasure(simplifiedMeasures));
dataModel.setAllNamedColumns(convertNamedColumns(modelRequest.getProject(), dataModel, modelRequest));
-
- dataModel.initJoinDesc(KylinConfig.getInstanceFromEnv(),
- getManager(NTableMetadataManager.class, modelRequest.getProject()).getAllTablesMap());
+ dataModel.initJoinDesc(KylinConfig.getInstanceFromEnv(), allTablesMap);
convertNonEquiJoinCond(dataModel, modelRequest);
dataModel.setModelType(dataModel.getModelTypeFromTable());
return dataModel;
@@ -431,7 +442,7 @@ public class ModelSemanticHelper extends BasicService {
SqlVisitor<Object> modifyAlias = new ModifyTableNameSqlVisitor(oldAliasName, newAliasName);
SqlNode sqlNode = CalciteParser.getExpNode(filterCondition);
sqlNode.accept(modifyAlias);
- String newFilterCondition = sqlNode.toSqlString(HiveSqlDialect.DEFAULT).toString();
+ String newFilterCondition = sqlNode.toSqlString(CalciteSqlDialect.DEFAULT, true).toString();
model.setFilterCondition(newFilterCondition);
}
}
diff --git a/src/modeling-service/src/main/java/org/apache/kylin/rest/service/ModelService.java b/src/modeling-service/src/main/java/org/apache/kylin/rest/service/ModelService.java
index ba966e6d28..a7dd882b84 100644
--- a/src/modeling-service/src/main/java/org/apache/kylin/rest/service/ModelService.java
+++ b/src/modeling-service/src/main/java/org/apache/kylin/rest/service/ModelService.java
@@ -73,6 +73,7 @@ import static org.apache.kylin.metadata.model.FunctionDesc.PARAMETER_TYPE_COLUMN
import java.io.IOException;
import java.math.BigDecimal;
+import java.sql.SQLException;
import java.text.MessageFormat;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
@@ -94,7 +95,6 @@ import java.util.regex.Pattern;
import java.util.stream.Collectors;
import java.util.stream.Stream;
-import org.apache.calcite.sql.SqlDialect;
import org.apache.calcite.sql.SqlIdentifier;
import org.apache.calcite.sql.SqlNode;
import org.apache.calcite.sql.dialect.CalciteSqlDialect;
@@ -105,7 +105,6 @@ import org.apache.commons.collections.CollectionUtils;
import org.apache.commons.collections.MapUtils;
import org.apache.commons.lang3.ArrayUtils;
import org.apache.commons.lang3.StringUtils;
-import org.apache.commons.lang.exception.ExceptionUtils;
import org.apache.kylin.common.KapConfig;
import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.QueryContext;
@@ -128,7 +127,7 @@ import org.apache.kylin.common.util.DateFormat;
import org.apache.kylin.common.util.JsonUtil;
import org.apache.kylin.common.util.Pair;
import org.apache.kylin.common.util.RandomUtil;
-import org.apache.kylin.common.util.StringUtil;
+import org.apache.kylin.common.util.StringHelper;
import org.apache.kylin.engine.spark.utils.ComputedColumnEvalUtil;
import org.apache.kylin.job.SecondStorageJobParamUtil;
import org.apache.kylin.job.common.SegmentUtil;
@@ -200,7 +199,6 @@ import org.apache.kylin.metadata.realization.RealizationStatusEnum;
import org.apache.kylin.metadata.streaming.KafkaConfig;
import org.apache.kylin.query.util.PushDownUtil;
import org.apache.kylin.query.util.QueryParams;
-import org.apache.kylin.query.util.QueryUtil;
import org.apache.kylin.rest.aspect.Transaction;
import org.apache.kylin.rest.constant.ModelAttributeEnum;
import org.apache.kylin.rest.constant.ModelStatusToDisplayEnum;
@@ -1738,7 +1736,7 @@ public class ModelService extends AbstractModelService implements TableModelSupp
QueryParams queryParams = new QueryParams(project, flatTableSql, "default", false);
queryParams.setKylinConfig(prjInstance.getConfig());
queryParams.setAclInfo(AclPermissionUtil.createAclInfo(project, getCurrentUserGroups()));
- String pushdownSql = QueryUtil.massagePushDownSql(queryParams);
+ String pushdownSql = PushDownUtil.massagePushDownSql(queryParams);
ss.sql(pushdownSql);
}
} catch (Exception e) {
@@ -2264,8 +2262,8 @@ public class ModelService extends AbstractModelService implements TableModelSupp
}
}
- public String probeDateFormatIfNotExist(String project, NDataModel modelDesc) throws Exception {
- val partitionDesc = modelDesc.getPartitionDesc();
+ public String probeDateFormatIfNotExist(String project, NDataModel model) throws SQLException {
+ PartitionDesc partitionDesc = model.getPartitionDesc();
if (PartitionDesc.isEmptyPartitionDesc(partitionDesc)
|| StringUtils.isNotEmpty(partitionDesc.getPartitionDateFormat()))
return "";
@@ -2275,9 +2273,8 @@ public class ModelService extends AbstractModelService implements TableModelSupp
return partitionDesc.getPartitionDateColumn();
}
- String partitionColumn = modelDesc.getPartitionDesc().getPartitionDateColumnRef().getExpressionInSourceDB();
-
- val date = PushDownUtil.getFormatIfNotExist(modelDesc.getRootFactTableName(), partitionColumn, project);
+ String partitionColumn = model.getPartitionDesc().getPartitionDateColumnRef().getBackTickExp();
+ String date = PushDownUtil.probeColFormat(model.getRootFactTableName(), partitionColumn, project);
return DateFormat.proposeDateFormat(date);
}
@@ -2297,7 +2294,7 @@ public class ModelService extends AbstractModelService implements TableModelSupp
String dateFormat = desc.getPartitionDateFormat();
Preconditions.checkArgument(StringUtils.isNotEmpty(dateFormat) && StringUtils.isNotEmpty(partitionColumn));
- val minAndMaxTime = PushDownUtil.getMaxAndMinTimeWithTimeOut(partitionColumn, table, project);
+ val minAndMaxTime = PushDownUtil.probeMinMaxTsWithTimeout(partitionColumn, table, project);
return new Pair<>(DateFormat.getFormattedDate(minAndMaxTime.getFirst(), dateFormat),
DateFormat.getFormattedDate(minAndMaxTime.getSecond(), dateFormat));
@@ -2628,7 +2625,7 @@ public class ModelService extends AbstractModelService implements TableModelSupp
model.init(getConfig(), project, getManager(NDataModelManager.class, project).getCCRelatedModels(model));
model.getComputedColumnDescs().forEach(cc -> {
- String innerExp = QueryUtil.massageComputedColumn(model, project, cc, null);
+ String innerExp = PushDownUtil.massageComputedColumn(model, project, cc, null);
cc.setInnerExpression(innerExp);
});
@@ -2655,7 +2652,7 @@ public class ModelService extends AbstractModelService implements TableModelSupp
MsgPicker.getMsg().getccOnAntiFlattenLookup(), antiFlattenLookup));
}
ComputedColumnDesc.simpleParserCheck(cc.getExpression(), model.getAliasMap().keySet());
- String innerExpression = QueryUtil.massageComputedColumn(model, project, cc, aclInfo);
+ String innerExpression = PushDownUtil.massageComputedColumn(model, project, cc, aclInfo);
cc.setInnerExpression(innerExpression);
//check by data source, this could be slow
@@ -2757,10 +2754,10 @@ public class ModelService extends AbstractModelService implements TableModelSupp
// Update CC expression from query transformers
QueryContext.AclInfo aclInfo = AclPermissionUtil.createAclInfo(project, getCurrentUserGroups());
for (ComputedColumnDesc ccDesc : model.getComputedColumnDescs()) {
- String ccExpression = QueryUtil.massageComputedColumn(model, project, ccDesc, aclInfo);
+ String ccExpression = PushDownUtil.massageComputedColumn(model, project, ccDesc, aclInfo);
ccDesc.setInnerExpression(ccExpression);
TblColRef tblColRef = model.findColumn(ccDesc.getTableAlias(), ccDesc.getColumnName());
- tblColRef.getColumnDesc().setComputedColumn(ccExpression);
+ tblColRef.getColumnDesc().setComputedColumnExpr(ccExpression);
}
ComputedColumnEvalUtil.evalDataTypeOfCCInBatch(model, model.getComputedColumnDescs());
@@ -3045,7 +3042,7 @@ public class ModelService extends AbstractModelService implements TableModelSupp
return baseIndexResponse;
}, project);
} catch (TransactionException te) {
- Throwable root = ExceptionUtils.getCause(te);
+ Throwable root = te.getCause();
if (root instanceof RuntimeException) {
throw (RuntimeException) root;
}
@@ -3439,24 +3436,24 @@ public class ModelService extends AbstractModelService implements TableModelSupp
String pattitions = props.get("kylin.engine.spark-conf.spark.sql.shuffle.partitions");
String memory = props.get("kylin.engine.spark-conf.spark.executor.memory");
String baseCuboidAllowed = props.get("kylin.cube.aggrgroup.is-base-cuboid-always-valid");
- if (null != cores && !StringUtil.validateNumber(cores)) {
+ if (null != cores && !StringHelper.validateNumber(cores)) {
throw new KylinException(INVALID_PARAMETER,
String.format(Locale.ROOT, MsgPicker.getMsg().getInvalidIntegerFormat(), "spark.executor.cores"));
}
- if (null != instances && !StringUtil.validateNumber(instances)) {
+ if (null != instances && !StringHelper.validateNumber(instances)) {
throw new KylinException(INVALID_PARAMETER, String.format(Locale.ROOT,
MsgPicker.getMsg().getInvalidIntegerFormat(), "spark.executor.instances"));
}
- if (null != pattitions && !StringUtil.validateNumber(pattitions)) {
+ if (null != pattitions && !StringHelper.validateNumber(pattitions)) {
throw new KylinException(INVALID_PARAMETER, String.format(Locale.ROOT,
MsgPicker.getMsg().getInvalidIntegerFormat(), "spark.sql.shuffle.partitions"));
}
if (null != memory
- && (!memory.endsWith("g") || !StringUtil.validateNumber(memory.substring(0, memory.length() - 1)))) {
+ && (!memory.endsWith("g") || !StringHelper.validateNumber(memory.substring(0, memory.length() - 1)))) {
throw new KylinException(INVALID_PARAMETER,
String.format(Locale.ROOT, MsgPicker.getMsg().getInvalidMemorySize(), "spark.executor.memory"));
}
- if (null != baseCuboidAllowed && !StringUtil.validateBoolean(baseCuboidAllowed)) {
+ if (null != baseCuboidAllowed && !StringHelper.validateBoolean(baseCuboidAllowed)) {
throw new KylinException(INVALID_PARAMETER, String.format(Locale.ROOT,
MsgPicker.getMsg().getInvalidBooleanFormat(), "is-base-cuboid-always-valid"));
}
@@ -3600,17 +3597,15 @@ public class ModelService extends AbstractModelService implements TableModelSupp
*
* @param model
*/
- @VisibleForTesting
- public void massageModelFilterCondition(final NDataModel model) {
+ void massageModelFilterCondition(final NDataModel model) {
if (StringUtils.isEmpty(model.getFilterCondition())) {
return;
}
- String massagedFilterCond = QueryUtil.massageExpression(model, model.getProject(), model.getFilterCondition(),
- AclPermissionUtil.createAclInfo(model.getProject(), getCurrentUserGroups()), false);
-
- String filterConditionWithTableName = addTableNameIfNotExist(massagedFilterCond, model);
-
+ String filterConditionWithTableName = addTableNameIfNotExist(model.getFilterCondition(), model);
+ QueryContext.AclInfo aclInfo = AclPermissionUtil.createAclInfo(model.getProject(), getCurrentUserGroups());
+ // validate as soon as possible
+ PushDownUtil.massageExpression(model, model.getProject(), model.getFilterCondition(), aclInfo);
model.setFilterCondition(filterConditionWithTableName);
}
@@ -3648,9 +3643,7 @@ public class ModelService extends AbstractModelService implements TableModelSupp
MsgPicker.getMsg().getFilterConditionOnAntiFlattenLookup(), antiFlatLookup));
}
}
- String exp = sqlNode
- .toSqlString(new CalciteSqlDialect(SqlDialect.EMPTY_CONTEXT.withIdentifierQuoteString("`")), true)
- .toString();
+ String exp = sqlNode.toSqlString(CalciteSqlDialect.DEFAULT, true).toString();
return CalciteParser.normalize(exp);
}
@@ -4171,7 +4164,7 @@ public class ModelService extends AbstractModelService implements TableModelSupp
for (ComputedColumnDesc cc : model.getComputedColumnDescs()) {
String innerExp = cc.getInnerExpression();
if (cc.getExpression().equalsIgnoreCase(innerExp)) {
- innerExp = QueryUtil.massageComputedColumn(model, project, cc, null);
+ innerExp = PushDownUtil.massageComputedColumn(model, project, cc, null);
}
cc.setInnerExpression(innerExp);
}
diff --git a/src/modeling-service/src/test/java/org/apache/kylin/rest/service/ModelServiceTest.java b/src/modeling-service/src/test/java/org/apache/kylin/rest/service/ModelServiceTest.java
index 9c8293e8b9..cb82fc5921 100644
--- a/src/modeling-service/src/test/java/org/apache/kylin/rest/service/ModelServiceTest.java
+++ b/src/modeling-service/src/test/java/org/apache/kylin/rest/service/ModelServiceTest.java
@@ -159,6 +159,7 @@ import org.apache.kylin.metadata.realization.RealizationStatusEnum;
import org.apache.kylin.metadata.recommendation.candidate.JdbcRawRecStore;
import org.apache.kylin.metadata.user.ManagedUser;
import org.apache.kylin.query.util.QueryUtil;
+import org.apache.kylin.query.util.PushDownUtil;
import org.apache.kylin.rest.config.initialize.ModelBrokenListener;
import org.apache.kylin.rest.constant.Constant;
import org.apache.kylin.rest.constant.ModelStatusToDisplayEnum;
@@ -285,7 +286,7 @@ public class ModelServiceTest extends SourceTestCase {
ReflectionTestUtils.setField(semanticService, "userGroupService", userGroupService);
ReflectionTestUtils.setField(semanticService, "expandableMeasureUtil",
new ExpandableMeasureUtil((model, ccDesc) -> {
- String ccExpression = QueryUtil.massageComputedColumn(model, model.getProject(), ccDesc,
+ String ccExpression = PushDownUtil.massageComputedColumn(model, model.getProject(), ccDesc,
AclPermissionUtil.createAclInfo(model.getProject(),
semanticService.getCurrentUserGroups()));
ccDesc.setInnerExpression(ccExpression);
@@ -3267,8 +3268,8 @@ public class ModelServiceTest extends SourceTestCase {
modelRequest.getPartitionDesc().setPartitionDateFormat("yyyy-MM-dd");
String filterCond = "trans_id = 0 and TEST_KYLIN_FACT.order_id < 100 and DEAL_AMOUNT > 123";
- String expectedFilterCond = "(((`TEST_KYLIN_FACT`.`TRANS_ID` = 0) AND (`TEST_KYLIN_FACT`.`ORDER_ID` <"
- + " 100)) AND ((`TEST_KYLIN_FACT`.`PRICE` * `TEST_KYLIN_FACT`.`ITEM_COUNT`) > 123))";
+ String expectedFilterCond = "(((\"TEST_KYLIN_FACT\".\"TRANS_ID\" = 0) "
+ + "AND (\"TEST_KYLIN_FACT\".\"ORDER_ID\" < 100)) AND (\"TEST_KYLIN_FACT\".\"DEAL_AMOUNT\" > 123))";
modelRequest.setFilterCondition(filterCond);
val newModel = modelService.createModel(modelRequest.getProject(), modelRequest);
@@ -3292,8 +3293,7 @@ public class ModelServiceTest extends SourceTestCase {
modelRequest.setUuid(null);
String filterCond = "\"day\" = 0 and \"123TABLE\".\"day#\" = 1 and \"中文列\" = 1";
- String expectedFilterCond = "(((`123TABLE`.`DAY` = 0) AND (`123TABLE`.`day#` = 1)) AND (`123TABLE`"
- + ".`中文列` = 1))";
+ String expectedFilterCond = "(((\"123TABLE\".\"DAY\" = 0) AND (\"123TABLE\".\"day#\" = 1)) AND (\"123TABLE\".\"中文列\" = 1))";
modelRequest.setFilterCondition(filterCond);
val newModel = modelService.createModel(modelRequest.getProject(), modelRequest);
@@ -3542,10 +3542,8 @@ public class ModelServiceTest extends SourceTestCase {
String originSql = "trans_id = 0 and TEST_KYLIN_FACT.order_id < 100 and DEAL_AMOUNT > 123";
model.setFilterCondition(originSql);
modelService.massageModelFilterCondition(model);
- Assert.assertEquals(
- "(((`TEST_KYLIN_FACT`.`TRANS_ID` = 0) AND " + "(`TEST_KYLIN_FACT`.`ORDER_ID` < 100)) AND "
- + "((`TEST_KYLIN_FACT`.`PRICE` * `TEST_KYLIN_FACT`.`ITEM_COUNT`) > 123))",
- model.getFilterCondition());
+ Assert.assertEquals("(((\"TEST_KYLIN_FACT\".\"TRANS_ID\" = 0) AND (\"TEST_KYLIN_FACT\".\"ORDER_ID\" < 100)) "
+ + "AND (\"TEST_KYLIN_FACT\".\"DEAL_AMOUNT\" > 123))", model.getFilterCondition());
}
@Test
@@ -3577,7 +3575,7 @@ public class ModelServiceTest extends SourceTestCase {
final NDataModel model1 = modelManager.getDataModelDesc(modelId);
model1.setFilterCondition("TIMESTAMPDIFF(DAY, CURRENT_DATE, TEST_KYLIN_FACT.\"CURRENT_DATE\") >= 0");
modelService.massageModelFilterCondition(model1);
- Assert.assertEquals("(TIMESTAMPDIFF(DAY, CURRENT_DATE(), `TEST_KYLIN_FACT`.`CURRENT_DATE`) >= 0)",
+ Assert.assertEquals("(TIMESTAMPDIFF(DAY, CURRENT_DATE(), \"TEST_KYLIN_FACT\".\"CURRENT_DATE\") >= 0)",
model1.getFilterCondition());
}
@@ -3593,10 +3591,8 @@ public class ModelServiceTest extends SourceTestCase {
String originSql = "trans_id = 0 and TEST_ORDER.order_id < 100 and DEAL_AMOUNT > 123";
model.setFilterCondition(originSql);
modelService.massageModelFilterCondition(model);
- Assert.assertEquals(
- "(((`TEST_KYLIN_FACT`.`TRANS_ID` = 0) " + "AND (`TEST_ORDER`.`ORDER_ID` < 100)) "
- + "AND ((`TEST_KYLIN_FACT`.`PRICE` * `TEST_KYLIN_FACT`.`ITEM_COUNT`) > 123))",
- model.getFilterCondition());
+ Assert.assertEquals("(((\"TEST_KYLIN_FACT\".\"TRANS_ID\" = 0) AND (\"TEST_ORDER\".\"ORDER_ID\" < 100)) "
+ + "AND (\"TEST_KYLIN_FACT\".\"DEAL_AMOUNT\" > 123))", model.getFilterCondition());
}
@Test
@@ -3624,10 +3620,11 @@ public class ModelServiceTest extends SourceTestCase {
NDataModel model = modelManager.getDataModelDesc("89af4ee2-2cdb-4b07-b39e-4c29856309aa");
String originSql = "trans_id = 0 and TEST_KYLIN_FACT.order_id < 100";
String newSql = modelService.addTableNameIfNotExist(originSql, model);
- Assert.assertEquals("((`TEST_KYLIN_FACT`.`TRANS_ID` = 0) AND (`TEST_KYLIN_FACT`.`ORDER_ID` < 100))", newSql);
+ Assert.assertEquals("((\"TEST_KYLIN_FACT\".\"TRANS_ID\" = 0) AND (\"TEST_KYLIN_FACT\".\"ORDER_ID\" < 100))",
+ newSql);
originSql = "trans_id between 1 and 10";
newSql = modelService.addTableNameIfNotExist(originSql, model);
- Assert.assertEquals("(`TEST_KYLIN_FACT`.`TRANS_ID` BETWEEN 1 AND 10)", newSql);
+ Assert.assertEquals("(\"TEST_KYLIN_FACT\".\"TRANS_ID\" BETWEEN 1 AND 10)", newSql);
modelManager.updateDataModel(model.getUuid(), copyForWrite -> {
List<JoinTableDesc> joinTables = copyForWrite.getJoinTables();
@@ -3685,11 +3682,11 @@ public class ModelServiceTest extends SourceTestCase {
}
@Test
- public void testComputedColumnNameCheck_PreProcessBeforeModelSave_ExceptionWhenCCNameIsSameWithColumnInLookupTable() {
+ public void testCheckingCcNameIsSameWithLookupColNameBeforeModelSaveThenThrowException() {
expectedEx.expect(KylinException.class);
- expectedEx.expectMessage("Can’t validate the expression \"TEST_KYLIN_FACT.NEST2\" (computed column: "
- + "TEST_KYLIN_FACT.NEST1 * 12). Please check the expression, or try again later.");
+ expectedEx.expectMessage("Can’t validate the expression \"TEST_KYLIN_FACT.SITE_ID\" (computed column: "
+ + "nvl(TEST_SITES.SITE_ID)). Please check the expression, or try again later.");
String tableIdentity = "DEFAULT.TEST_KYLIN_FACT";
String columnName = "SITE_ID";
String expression = "nvl(TEST_SITES.SITE_ID)";
@@ -3703,13 +3700,17 @@ public class ModelServiceTest extends SourceTestCase {
String project = "default";
NDataModelManager dataModelManager = modelService.getManager(NDataModelManager.class, "default");
NDataModel model = dataModelManager.getDataModelDesc("741ca86a-1f13-46da-a59f-95fb68615e3a");
- model.getComputedColumnDescs().add(ccDesc);
+ model.getComputedColumnDescs().add(0, ccDesc);
modelService.preProcessBeforeModelSave(model, project);
}
@Test
- public void testCheckCCNameAmbiguity() {
+ public void testCheckingCcNameIsSameWithLookupColNameWhenCheckingCCThenThrowException() {
+
+ expectedEx.expect(KylinException.class);
+ expectedEx.expectMessage("Can’t validate the expression \"TEST_KYLIN_FACT.SITE_ID\" (computed column: "
+ + "nvl(TEST_SITES.SITE_ID)). Please check the expression, or try again later.");
String tableIdentity = "DEFAULT.TEST_KYLIN_FACT";
String columnName = "SITE_ID";
String expression = "nvl(TEST_SITES.SITE_ID)";
@@ -3720,19 +3721,16 @@ public class ModelServiceTest extends SourceTestCase {
ccDesc.setExpression(expression);
ccDesc.setDatatype(dataType);
+ String project = "default";
NDataModelManager dataModelManager = modelService.getManager(NDataModelManager.class, "default");
NDataModel model = dataModelManager.getDataModelDesc("741ca86a-1f13-46da-a59f-95fb68615e3a");
- model.getComputedColumnDescs().add(ccDesc);
+ model.getComputedColumnDescs().add(0, ccDesc);
- modelService.checkCCNameAmbiguity(model);
+ modelService.checkComputedColumn(model, project, null);
}
@Test
- public void testComputedColumnNameCheck_CheckCC_ExceptionWhenCCNameIsSameWithColumnInLookupTable() {
-
- expectedEx.expect(KylinException.class);
- expectedEx.expectMessage("Can’t validate the expression \"TEST_KYLIN_FACT.NEST2\" (computed column: "
- + "TEST_KYLIN_FACT.NEST1 * 12). Please check the expression, or try again later.");
+ public void testCheckCCNameAmbiguity() {
String tableIdentity = "DEFAULT.TEST_KYLIN_FACT";
String columnName = "SITE_ID";
String expression = "nvl(TEST_SITES.SITE_ID)";
@@ -3743,12 +3741,11 @@ public class ModelServiceTest extends SourceTestCase {
ccDesc.setExpression(expression);
ccDesc.setDatatype(dataType);
- String project = "default";
NDataModelManager dataModelManager = modelService.getManager(NDataModelManager.class, "default");
NDataModel model = dataModelManager.getDataModelDesc("741ca86a-1f13-46da-a59f-95fb68615e3a");
model.getComputedColumnDescs().add(ccDesc);
- modelService.checkComputedColumn(model, project, null);
+ modelService.checkCCNameAmbiguity(model);
}
private NDataSegment mockSegment() {
diff --git a/src/modeling-service/src/test/java/org/apache/kylin/rest/service/TableServiceTest.java b/src/modeling-service/src/test/java/org/apache/kylin/rest/service/TableServiceTest.java
index 5864a7e5de..b9d9b71857 100644
--- a/src/modeling-service/src/test/java/org/apache/kylin/rest/service/TableServiceTest.java
+++ b/src/modeling-service/src/test/java/org/apache/kylin/rest/service/TableServiceTest.java
@@ -208,23 +208,22 @@ public class TableServiceTest extends CSVSourceTestCase {
List<Integer> sourceType = new ArrayList<>();
sourceType.add(1); // Kafka table
sourceType.add(9); // Hive table
- List<TableDesc> tableDesc = tableService.getTableDesc("default", true, "", "DEFAULT", true,
- sourceType, 12).getFirst();
+ List<TableDesc> tableDesc = tableService.getTableDesc("default", true, "", "DEFAULT", true, sourceType, 12)
+ .getFirst();
Assert.assertEquals(12, tableDesc.size());
- List<TableDesc> tableDesc2 = tableService.getTableDesc("default", true, "TEST_COUNTRY", "DEFAULT", false,
- sourceType, 10).getFirst();
+ List<TableDesc> tableDesc2 = tableService
+ .getTableDesc("default", true, "TEST_COUNTRY", "DEFAULT", false, sourceType, 10).getFirst();
Assert.assertEquals(1, tableDesc2.size());
- List<TableDesc> tables3 = tableService.getTableDesc("default", true, "", "", true,
- sourceType, 100).getFirst();
+ List<TableDesc> tables3 = tableService.getTableDesc("default", true, "", "", true, sourceType, 100).getFirst();
Assert.assertEquals(21, tables3.size());
- List<TableDesc> tables = tableService.getTableDesc("default", true, "TEST_KYLIN_FACT", "DEFAULT", true,
- sourceType, 10).getFirst();
+ List<TableDesc> tables = tableService
+ .getTableDesc("default", true, "TEST_KYLIN_FACT", "DEFAULT", true, sourceType, 10).getFirst();
Assert.assertEquals("TEST_KYLIN_FACT", tables.get(0).getName());
Assert.assertEquals(5633024, ((TableDescResponse) tables.get(0)).getStorageSize());
Assert.assertEquals(0, ((TableDescResponse) tables.get(0)).getTotalRecords());
- List<TableDesc> table2 = tableService.getTableDesc("default", true, "country", "DEFAULT", true,
- sourceType, 10).getFirst();
+ List<TableDesc> table2 = tableService.getTableDesc("default", true, "country", "DEFAULT", true, sourceType, 10)
+ .getFirst();
Assert.assertEquals("TEST_COUNTRY", table2.get(0).getName());
Assert.assertEquals(0L, ((TableDescResponse) table2.get(0)).getStorageSize());
@@ -233,25 +232,22 @@ public class TableServiceTest extends CSVSourceTestCase {
countryTable.setLastSnapshotPath("cannot/find/it");
manager.updateTableDesc(countryTable);
- table2 = tableService.getTableDesc("default", true, "country", "DEFAULT", true,
- sourceType, 10).getFirst();
+ table2 = tableService.getTableDesc("default", true, "country", "DEFAULT", true, sourceType, 10).getFirst();
Assert.assertEquals("TEST_COUNTRY", table2.get(0).getName());
Assert.assertEquals(0L, ((TableDescResponse) table2.get(0)).getStorageSize());
// get a not existing table desc
- tableDesc = tableService.getTableDesc("default", true, "not_exist_table", "DEFAULT", false,
- sourceType, 10).getFirst();
+ tableDesc = tableService.getTableDesc("default", true, "not_exist_table", "DEFAULT", false, sourceType, 10)
+ .getFirst();
Assert.assertEquals(0, tableDesc.size());
- tableDesc = tableService.getTableDesc("streaming_test", true, "", "DEFAULT", true,
- sourceType, 10).getFirst();
+ tableDesc = tableService.getTableDesc("streaming_test", true, "", "DEFAULT", true, sourceType, 10).getFirst();
Assert.assertEquals(2, tableDesc.size());
val tableMetadataManager = getInstance(getTestConfig(), "streaming_test");
var tableDesc1 = tableMetadataManager.getTableDesc("DEFAULT.SSB_TOPIC");
Assert.assertTrue(tableDesc1.isAccessible(getTestConfig().streamingEnabled()));
getTestConfig().setProperty("kylin.streaming.enabled", "false");
- tableDesc = tableService.getTableDesc("streaming_test", true, "", "DEFAULT", true,
- sourceType, 10).getFirst();
+ tableDesc = tableService.getTableDesc("streaming_test", true, "", "DEFAULT", true, sourceType, 10).getFirst();
Assert.assertEquals(0, tableDesc.size());
// check kafka table
Assert.assertFalse(tableDesc1.isAccessible(getTestConfig().streamingEnabled()));
@@ -289,8 +285,8 @@ public class TableServiceTest extends CSVSourceTestCase {
Assert.assertEquals(1, newTableExt.getAllColumnStats().size());
// call api to check tableDescResponse has the correct value
- final List<TableDesc> tables = tableService.getTableDesc("newten", true, "TEST_COUNTRY", "DEFAULT", true,
- Collections.emptyList(), 10).getFirst();
+ final List<TableDesc> tables = tableService
+ .getTableDesc("newten", true, "TEST_COUNTRY", "DEFAULT", true, Collections.emptyList(), 10).getFirst();
Assert.assertEquals(1, tables.size());
Assert.assertTrue(tables.get(0) instanceof TableDescResponse);
TableDescResponse t = (TableDescResponse) tables.get(0);
@@ -333,8 +329,8 @@ public class TableServiceTest extends CSVSourceTestCase {
Mockito.when(userAclService.hasUserAclPermissionInProject(Mockito.anyString(), Mockito.anyString()))
.thenReturn(false);
- List<TableDesc> tableExtList = tableService.getTableDesc("newten", true, "TEST_COUNTRY", "DEFAULT",
- true, Collections.emptyList(), 10).getFirst();
+ List<TableDesc> tableExtList = tableService
+ .getTableDesc("newten", true, "TEST_COUNTRY", "DEFAULT", true, Collections.emptyList(), 10).getFirst();
Assert.assertEquals(0, ((TableDescResponse) tableExtList.get(0)).getSamplingRows().size());
SecurityContextHolder.getContext()
.setAuthentication(new TestingAuthenticationToken("ADMIN", "ADMIN", Constant.ROLE_ADMIN));
@@ -406,8 +402,8 @@ public class TableServiceTest extends CSVSourceTestCase {
Assert.assertEquals("float", confirmedTableDesc.getColumns()[2].getDatatype());
// call api to check tableDescResponse has the correct value
- final List<TableDesc> tables = tableService.getTableDesc("newten", true, "TEST_COUNTRY", "DEFAULT", true,
- Collections.emptyList(), 10).getFirst();
+ final List<TableDesc> tables = tableService
+ .getTableDesc("newten", true, "TEST_COUNTRY", "DEFAULT", true, Collections.emptyList(), 10).getFirst();
Assert.assertEquals(1, tables.size());
Assert.assertTrue(tables.get(0) instanceof TableDescResponse);
TableDescResponse t = (TableDescResponse) tables.get(0);
@@ -495,8 +491,8 @@ public class TableServiceTest extends CSVSourceTestCase {
@Test
public void testLoadTableToProject() throws IOException {
- List<TableDesc> tables = tableService.getTableDesc("default", true, "TEST_COUNTRY", "DEFAULT", true,
- Collections.emptyList(), 10).getFirst();
+ List<TableDesc> tables = tableService
+ .getTableDesc("default", true, "TEST_COUNTRY", "DEFAULT", true, Collections.emptyList(), 10).getFirst();
TableDesc nTableDesc = new TableDesc(tables.get(0));
TableExtDesc tableExt = new TableExtDesc();
tableExt.setIdentity("DEFAULT.TEST_COUNTRY");
@@ -509,8 +505,8 @@ public class TableServiceTest extends CSVSourceTestCase {
public void testLoadTableToProjectWithS3Role() throws IOException {
getTestConfig().setProperty("kylin.env.use-dynamic-S3-role-credential-in-table", "true");
assert !SparderEnv.getSparkSession().conf().contains(String.format(S3AUtil.ROLE_ARN_KEY_FORMAT, "testbucket"));
- List<TableDesc> tables = tableService.getTableDesc("default", true, "TEST_COUNTRY", "DEFAULT", true,
- Collections.emptyList(), 10).getFirst();
+ List<TableDesc> tables = tableService
+ .getTableDesc("default", true, "TEST_COUNTRY", "DEFAULT", true, Collections.emptyList(), 10).getFirst();
TableDesc nTableDesc = new TableDesc(tables.get(0));
TableExtDesc tableExt = new TableExtDesc();
tableExt.setIdentity("DEFAULT.TEST_COUNTRY");
@@ -738,7 +734,7 @@ public class TableServiceTest extends CSVSourceTestCase {
tableDesc.setTableType(TableDesc.TABLE_TYPE_VIEW);
tableMgr.updateTableDesc(tableDesc);
try {
- tableService.getPartitionColumnFormat("default", table, "CAL_DT");
+ tableService.getPartitionColumnFormat("default", table, "CAL_DT", null);
Assert.fail();
} catch (Exception e) {
Assert.assertEquals(MsgPicker.getMsg().getViewDateFormatDetectionError(), e.getMessage());
@@ -752,7 +748,7 @@ public class TableServiceTest extends CSVSourceTestCase {
testGetBatchLoadTablesBefore();
final String table = "DEFAULT.TEST_KYLIN_FACT";
try {
- tableService.getPartitionColumnFormat("default", table, "CAL_DT");
+ tableService.getPartitionColumnFormat("default", table, "CAL_DT", null);
Assert.fail();
} catch (Exception e) {
Assert.assertEquals(MsgPicker.getMsg().getPushdownPartitionFormatError(), e.getMessage());
@@ -774,8 +770,8 @@ public class TableServiceTest extends CSVSourceTestCase {
private void testSetPartitionKeyWithoutException() throws Exception {
tableService.setPartitionKey("DEFAULT.TEST_KYLIN_FACT", "default", "CAL_DT", "yyyy-MM-dd");
- List<TableDesc> tables = tableService.getTableDesc("default", false, "", "DEFAULT", true,
- Collections.emptyList(), 10).getFirst();
+ List<TableDesc> tables = tableService
+ .getTableDesc("default", false, "", "DEFAULT", true, Collections.emptyList(), 10).getFirst();
//test set fact and table list order by fact
Assert.assertTrue(tables.get(0).getName().equals("TEST_KYLIN_FACT") && tables.get(0).isIncrementLoading());
}
@@ -869,7 +865,7 @@ public class TableServiceTest extends CSVSourceTestCase {
private void testgetPartitionColumnFormat() throws Exception {
// Test on batch table
- String format = tableService.getPartitionColumnFormat("default", "DEFAULT.TEST_KYLIN_FACT", "CAL_DT");
+ String format = tableService.getPartitionColumnFormat("default", "DEFAULT.TEST_KYLIN_FACT", "CAL_DT", null);
Assert.assertEquals("yyyy-MM-dd", format);
// Test on streaming table
@@ -897,7 +893,8 @@ public class TableServiceTest extends CSVSourceTestCase {
when(kafkaServiceMock.decodeMessage(any())).thenReturn(mockResp);
when(kafkaServiceMock.parserMessage(any(String.class), any(), any(String.class))).thenReturn(parseMap);
- String format2 = tableService.getPartitionColumnFormat("default", "DEFAULT.STREAMING_TABLE", "MINUTE_START");
+ String format2 = tableService.getPartitionColumnFormat("default", "DEFAULT.STREAMING_TABLE", "MINUTE_START",
+ null);
Assert.assertEquals("yyyy-MM-dd HH:mm:ss", format2);
when(kafkaServiceMock.getMessages(any(), any(String.class))).thenCallRealMethod();
@@ -922,8 +919,8 @@ public class TableServiceTest extends CSVSourceTestCase {
public void testSetTop() throws IOException {
TopTableRequest topTableRequest = mockTopTableRequest();
tableService.setTop(topTableRequest.getTable(), topTableRequest.getProject(), topTableRequest.isTop());
- List<TableDesc> tables = tableService.getTableDesc("default", false, "", "DEFAULT", true,
- Collections.emptyList(), 10).getFirst();
+ List<TableDesc> tables = tableService
+ .getTableDesc("default", false, "", "DEFAULT", true, Collections.emptyList(), 10).getFirst();
Assert.assertTrue(tables.get(0).isTop());
}
@@ -1139,35 +1136,29 @@ public class TableServiceTest extends CSVSourceTestCase {
NInitTablesResponse response;
overwriteSystemProp("kylin.source.load-hive-tablename-enabled", "false");
- response = tableService.getProjectTables("default", "SSB.SS", 0, 14, true, true,
- Collections.emptyList());
+ response = tableService.getProjectTables("default", "SSB.SS", 0, 14, true, true, Collections.emptyList());
Assert.assertEquals(0, response.getDatabases().size());
- response = tableService.getProjectTables("default", "SSB.CU", 0, 14, true, true,
- Collections.emptyList());
+ response = tableService.getProjectTables("default", "SSB.CU", 0, 14, true, true, Collections.emptyList());
Assert.assertEquals(1, response.getDatabases().size());
Assert.assertEquals(2, response.getDatabases().get(0).getTables().size());
- response = tableService.getProjectTables("default", "", 0, 14, true, true,
- Collections.emptyList());
+ response = tableService.getProjectTables("default", "", 0, 14, true, true, Collections.emptyList());
Assert.assertEquals(3, response.getDatabases().size());
Assert.assertEquals(21,
response.getDatabases().get(0).getTables().size() + response.getDatabases().get(1).getTables().size()
+ response.getDatabases().get(2).getTables().size());
- response = tableService.getProjectTables("default", "TEST", 0, 14, true, true,
- Collections.emptyList());
+ response = tableService.getProjectTables("default", "TEST", 0, 14, true, true, Collections.emptyList());
Assert.assertEquals(2, response.getDatabases().size());
Assert.assertEquals(13,
response.getDatabases().get(0).getTables().size() + response.getDatabases().get(1).getTables().size());
- response = tableService.getProjectTables("default", "EDW.", 0, 14, true, true,
- Collections.emptyList());
+ response = tableService.getProjectTables("default", "EDW.", 0, 14, true, true, Collections.emptyList());
Assert.assertEquals(1, response.getDatabases().size());
Assert.assertEquals(3, response.getDatabases().get(0).getTables().size());
- response = tableService.getProjectTables("default", "EDW.", 0, 14, true, false,
- Collections.emptyList());
+ response = tableService.getProjectTables("default", "EDW.", 0, 14, true, false, Collections.emptyList());
Assert.assertEquals(1, response.getDatabases().size());
Assert.assertEquals(3, response.getDatabases().get(0).getTables().size());
@@ -1176,12 +1167,10 @@ public class TableServiceTest extends CSVSourceTestCase {
Assert.assertEquals(1, response.getDatabases().size());
Assert.assertEquals(1, response.getDatabases().get(0).getTables().size());
- response = tableService.getProjectTables("default", ".TEST_ORDER", 0, 14, true, false,
- Collections.emptyList());
+ response = tableService.getProjectTables("default", ".TEST_ORDER", 0, 14, true, false, Collections.emptyList());
Assert.assertEquals(0, response.getDatabases().size());
- response = tableService.getProjectTables("default", "", 0, 14, true, true,
- Collections.singletonList(9));
+ response = tableService.getProjectTables("default", "", 0, 14, true, true, Collections.singletonList(9));
Assert.assertEquals(3, response.getDatabases().size());
}
@@ -1323,18 +1312,18 @@ public class TableServiceTest extends CSVSourceTestCase {
String warehousePath = getTestConfig().exportToProperties()
.getProperty("kylin.storage.columnar.spark-conf.spark.sql.warehouse.dir").substring(5)
+ "/test_kylin_refresh/";
- PushDownUtil.trySimplePushDownExecute("drop table if exists test_kylin_refresh", null);
- PushDownUtil.trySimplePushDownExecute("create table test_kylin_refresh (word string) STORED AS PARQUET", null);
- PushDownUtil.trySimplePushDownExecute("insert into test_kylin_refresh values ('a')", null);
- PushDownUtil.trySimplePushDownExecute("insert into test_kylin_refresh values ('c')", null);
- PushDownUtil.trySimplePushDownExecute("select * from test_kylin_refresh", null);
+ PushDownUtil.trySimplyExecute("drop table if exists test_kylin_refresh", null);
+ PushDownUtil.trySimplyExecute("create table test_kylin_refresh (word string) STORED AS PARQUET", null);
+ PushDownUtil.trySimplyExecute("insert into test_kylin_refresh values ('a')", null);
+ PushDownUtil.trySimplyExecute("insert into test_kylin_refresh values ('c')", null);
+ PushDownUtil.trySimplyExecute("select * from test_kylin_refresh", null);
CliCommandExecutor.CliCmdExecResult res = command.execute("ls " + warehousePath, null, null);
val files = Arrays.stream(res.getCmd().split("\n")).filter(file -> file.endsWith("parquet"))
.collect(Collectors.toList());
command.execute("rm " + warehousePath + files.get(0), null, null);
try {
- PushDownUtil.trySimplePushDownExecute("select * from test_kylin_refresh", null);
+ PushDownUtil.trySimplyExecute("select * from test_kylin_refresh", null);
Assert.fail();
} catch (Exception e) {
Assert.assertTrue(e.getMessage().contains("REFRESH TABLE tableName"));
@@ -1343,7 +1332,7 @@ public class TableServiceTest extends CSVSourceTestCase {
HashMap<String, Object> request = Maps.newHashMap();
request.put("tables", Collections.singletonList("test_kylin_refresh"));
TableRefresh refreshRes = tableService.refreshSingleCatalogCache(request);
- PushDownUtil.trySimplePushDownExecute("select * from test_kylin_refresh", null);
+ PushDownUtil.trySimplyExecute("select * from test_kylin_refresh", null);
Assert.assertEquals(1, refreshRes.getRefreshed().size());
Assert.assertEquals("test_kylin_refresh", refreshRes.getRefreshed().get(0));
SparderEnv.getSparkSession().stop();
@@ -1407,8 +1396,8 @@ public class TableServiceTest extends CSVSourceTestCase {
tableExt.setJodID("949afe5d-0221-420f-92db-cdd91cb31ac8");
tableMgr.mergeAndUpdateTableExt(oldExtDesc, tableExt);
- List<TableDesc> tables = tableService.getTableDesc("newten", true, "TEST_COUNTRY", "DEFAULT", true,
- Collections.emptyList(), 10).getFirst();
+ List<TableDesc> tables = tableService
+ .getTableDesc("newten", true, "TEST_COUNTRY", "DEFAULT", true, Collections.emptyList(), 10).getFirst();
Assert.assertEquals(1, tables.size());
Assert.assertEquals("949afe5d-0221-420f-92db-cdd91cb31ac8", ((TableDescResponse) tables.get(0)).getJodID());
@@ -1437,18 +1426,20 @@ public class TableServiceTest extends CSVSourceTestCase {
public void testGetTableDescByType() {
String project = "streaming_test";
try {
- val tableDescs = tableService.getTableDesc(project, true, "", "default", true,
- Collections.singletonList(1), 10).getFirst();
+ val tableDescs = tableService
+ .getTableDesc(project, true, "", "default", true, Collections.singletonList(1), 10).getFirst();
Assert.assertNotNull(tableDescs);
- val tableDescs1 = tableService.getTableDesc(project, true, "P_LINEORDER_STREAMING", "ssb", true,
- Collections.singletonList(1), 10).getFirst();
+ val tableDescs1 = tableService
+ .getTableDesc(project, true, "P_LINEORDER_STREAMING", "ssb", true, Collections.singletonList(1), 10)
+ .getFirst();
Assert.assertEquals(1, tableDescs1.size());
val tableDesc1 = tableDescs1.get(0);
Assert.assertEquals(tableDesc1.getTableAlias(), tableDesc1.getKafkaConfig().getBatchTable());
- val tableDescs2 = tableService.getTableDesc(project, true, "LINEORDER_HIVE", "SSB", false,
- Collections.singletonList(9), 10).getFirst();
+ val tableDescs2 = tableService
+ .getTableDesc(project, true, "LINEORDER_HIVE", "SSB", false, Collections.singletonList(9), 10)
+ .getFirst();
Assert.assertEquals(1, tableDescs2.size());
val tableDesc2 = tableDescs2.get(0);
Assert.assertEquals(tableDesc2.getTableAlias(), tableDesc2.getIdentity());
@@ -1463,8 +1454,7 @@ public class TableServiceTest extends CSVSourceTestCase {
String project = "streaming_test";
try {
List<Integer> sourceTypes = Arrays.asList(1, 9);
- val tableDescs2 = tableService.getTableDesc(project, true, "", "SSB", false,
- sourceTypes, 10).getFirst();
+ val tableDescs2 = tableService.getTableDesc(project, true, "", "SSB", false, sourceTypes, 10).getFirst();
assert tableDescs2.stream().anyMatch(tableDesc -> tableDesc.getSourceType() == 1);
assert tableDescs2.stream().anyMatch(tableDesc -> tableDesc.getSourceType() == 9);
} catch (Exception e) {
@@ -1549,8 +1539,8 @@ public class TableServiceTest extends CSVSourceTestCase {
tableExt.setColumnStats(Lists.newArrayList(col1));
tableMgr.mergeAndUpdateTableExt(oldExtDesc, tableExt);
- final List<TableDesc> tables = tableService.getTableDesc("newten", true, "TEST_COUNTRY", "DEFAULT", true,
- Collections.emptyList(), 10).getFirst();
+ final List<TableDesc> tables = tableService
+ .getTableDesc("newten", true, "TEST_COUNTRY", "DEFAULT", true, Collections.emptyList(), 10).getFirst();
Assert.assertEquals(1, tables.size());
Assert.assertTrue(tables.get(0) instanceof TableDescResponse);
TableDescResponse t = (TableDescResponse) tables.get(0);
diff --git a/src/query-common/src/main/java/org/apache/kylin/query/util/PushDownUtil.java b/src/query-common/src/main/java/org/apache/kylin/query/util/PushDownUtil.java
index ddbca4afbe..9f3be8de77 100644
--- a/src/query-common/src/main/java/org/apache/kylin/query/util/PushDownUtil.java
+++ b/src/query-common/src/main/java/org/apache/kylin/query/util/PushDownUtil.java
@@ -25,11 +25,16 @@ import java.sql.SQLException;
import java.util.Arrays;
import java.util.List;
import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.StringJoiner;
+import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
+import java.util.regex.Pattern;
import java.util.stream.Collectors;
import javax.ws.rs.BadRequestException;
@@ -48,15 +53,23 @@ import org.apache.kylin.common.exception.QueryErrorCode;
import org.apache.kylin.common.msg.MsgPicker;
import org.apache.kylin.common.util.ClassUtil;
import org.apache.kylin.common.util.Pair;
+import org.apache.kylin.common.util.StringHelper;
+import org.apache.kylin.metadata.model.ComputedColumnDesc;
import org.apache.kylin.metadata.model.ISourceAware;
+import org.apache.kylin.metadata.model.JoinDesc;
+import org.apache.kylin.metadata.model.JoinTableDesc;
+import org.apache.kylin.metadata.model.NDataModel;
import org.apache.kylin.metadata.model.NTableMetadataManager;
import org.apache.kylin.metadata.model.SegmentRange;
import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.metadata.model.TableRef;
+import org.apache.kylin.metadata.model.TblColRef;
import org.apache.kylin.metadata.project.NProjectManager;
import org.apache.kylin.metadata.querymeta.SelectedColumnMeta;
import org.apache.kylin.metadata.realization.RoutingIndicatorException;
import org.apache.kylin.query.exception.NoAuthorizedColsError;
import org.apache.kylin.query.security.AccessDeniedException;
+import org.apache.kylin.source.adhocquery.IPushDownConverter;
import org.apache.kylin.source.adhocquery.IPushDownRunner;
import org.apache.kylin.source.adhocquery.PushdownResult;
import org.codehaus.commons.compiler.CompileException;
@@ -64,34 +77,45 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
-
-import lombok.val;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
public class PushDownUtil {
- private static final Logger logger = LoggerFactory.getLogger(PushDownUtil.class);
+ private static final Logger logger = LoggerFactory.getLogger("query");
+
+ // sql hint "/*+ MODEL_PRIORITY({cube_name}) */"
+ private static final Pattern SQL_HINT_PATTERN = Pattern
+ .compile("/\\*\\s*\\+\\s*(?i)MODEL_PRIORITY\\s*\\([\\s\\S]*\\)\\s*\\*/");
+ public static final String DEFAULT_SCHEMA = "DEFAULT";
+ private static final String CC_SPLITTER = "'##CC_PUSH_DOWN_TOKEN##'";
private static final ExecutorService asyncExecutor = Executors.newCachedThreadPool();
+ private static final Map<String, IPushDownConverter> PUSH_DOWN_CONVERTER_MAP = Maps.newConcurrentMap();
+
+ static {
+ String[] classNames = KylinConfig.getInstanceFromEnv().getPushDownConverterClassNames();
+ for (String clz : classNames) {
+ try {
+ IPushDownConverter converter = (IPushDownConverter) ClassUtil.newInstance(clz);
+ PUSH_DOWN_CONVERTER_MAP.put(clz, converter);
+ } catch (Exception e) {
+ logger.error("Failed to init push-down converter of the sys-config: {}", clz);
+ }
+ }
+ }
private PushDownUtil() {
}
- public static PushdownResult tryPushDownQueryToIterator(QueryParams queryParams) throws Exception {
+ public static PushdownResult tryIterQuery(QueryParams queryParams) throws SQLException {
- KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
- val prjManager = NProjectManager.getInstance(kylinConfig);
- val prj = prjManager.getProject(queryParams.getProject());
String sql = queryParams.getSql();
String project = queryParams.getProject();
- kylinConfig = prj.getConfig();
- if (!kylinConfig.isPushDownEnabled()) {
- SQLException sqlException = queryParams.getSqlException();
- if (queryParams.isForcedToPushDown() || (sqlException != null
- && sqlException.getMessage().contains(QueryContext.ROUTE_USE_FORCEDTOTIEREDSTORAGE))) {
- throw new KylinException(QueryErrorCode.INVALID_PARAMETER_PUSH_DOWN,
- MsgPicker.getMsg().getDisablePushDownPrompt());
- }
+
+ KylinConfig projectConfig = NProjectManager.getProjectConfig(queryParams.getProject());
+ if (!projectConfig.isPushDownEnabled()) {
+ checkPushDownIncapable(queryParams);
return null;
}
if (queryParams.isSelect()) {
@@ -106,26 +130,23 @@ public class PushDownUtil {
logger.info("Kylin cannot support non-select queries, routing to other engines");
}
- IPushDownRunner runner = (IPushDownRunner) ClassUtil.newInstance(kylinConfig.getPushDownRunnerClassName());
- runner.init(kylinConfig, project);
+ IPushDownRunner runner = (IPushDownRunner) ClassUtil.newInstance(projectConfig.getPushDownRunnerClassName());
+ runner.init(projectConfig, project);
logger.debug("Query Pushdown runner {}", runner);
// set pushdown engine in query context
- String pushdownEngine;
+
// for file source
- int sourceType = kylinConfig.getManager(NProjectManager.class).getProject(queryParams.getProject())
- .getSourceType();
- if (sourceType == ISourceAware.ID_SPARK && KapConfig.getInstanceFromEnv().isCloud()) {
- pushdownEngine = QueryContext.PUSHDOWN_OBJECT_STORAGE;
- } else {
- pushdownEngine = runner.getName();
- }
- QueryContext.current().setPushdownEngine(pushdownEngine);
+ int sourceType = projectConfig.getDefaultSource();
+ String engine = sourceType == ISourceAware.ID_SPARK && KapConfig.getInstanceFromEnv().isCloud()
+ ? QueryContext.PUSHDOWN_OBJECT_STORAGE
+ : runner.getName();
+ QueryContext.current().setPushdownEngine(engine);
- queryParams.setKylinConfig(kylinConfig);
+ queryParams.setKylinConfig(projectConfig);
queryParams.setSql(sql);
try {
- sql = QueryUtil.massagePushDownSql(queryParams);
+ sql = massagePushDownSql(queryParams);
} catch (NoAuthorizedColsError e) {
// on no authorized cols found, return empty result
return PushdownResult.emptyResult();
@@ -143,12 +164,171 @@ public class PushDownUtil {
return PushdownResult.emptyResult();
}
- public static Pair<String, String> getMaxAndMinTimeWithTimeOut(String partitionColumn, String table, String project)
- throws Exception {
+ private static void checkPushDownIncapable(QueryParams queryParams) {
+ SQLException sqlException = queryParams.getSqlException();
+ if (queryParams.isForcedToPushDown() || (sqlException != null
+ && sqlException.getMessage().contains(QueryContext.ROUTE_USE_FORCEDTOTIEREDSTORAGE))) {
+ throw new KylinException(QueryErrorCode.INVALID_PARAMETER_PUSH_DOWN,
+ MsgPicker.getMsg().getDisablePushDownPrompt());
+ }
+ }
+
+ public static String massageComputedColumn(NDataModel model, String project, ComputedColumnDesc cc,
+ QueryContext.AclInfo aclInfo) {
+ return massageExpression(model, project, cc.getExpression(), aclInfo);
+ }
+
+ public static String massageExpression(NDataModel model, String project, String expression,
+ QueryContext.AclInfo aclInfo) {
+ String ccSql = expandComputedColumnExp(model, project, expression);
+ QueryParams queryParams = new QueryParams(project, ccSql, PushDownUtil.DEFAULT_SCHEMA, false);
+ queryParams.setKylinConfig(NProjectManager.getProjectConfig(project));
+ queryParams.setAclInfo(aclInfo);
+ String s = massagePushDownSql(queryParams);
+ return s.substring("select ".length(), s.indexOf(CC_SPLITTER) - 2).trim();
+ }
+
+ public static String massagePushDownSql(QueryParams queryParams) {
+ String sql = queryParams.getSql();
+ sql = QueryUtil.trimRightSemiColon(sql);
+ sql = SQL_HINT_PATTERN.matcher(sql).replaceAll("");
+
+ List<IPushDownConverter> pushDownConverters = fetchConverters(queryParams.getKylinConfig());
+ if (logger.isDebugEnabled()) {
+ logger.debug("All used push-down converters are: {}", pushDownConverters.stream()
+ .map(c -> c.getClass().getCanonicalName()).collect(Collectors.joining(",")));
+ }
+ for (IPushDownConverter converter : pushDownConverters) {
+ QueryUtil.checkThreadInterrupted("Interrupted sql transformation at the stage of " + converter.getClass(),
+ "Current step: Massage push-down sql. ");
+ sql = converter.convert(sql, queryParams.getProject(), queryParams.getDefaultSchema());
+ }
+ sql = replaceEscapedQuote(sql);
+ return sql;
+ }
+
+ static List<IPushDownConverter> fetchConverters(KylinConfig kylinConfig) {
+ List<IPushDownConverter> converters = Lists.newArrayList();
+ for (String clz : kylinConfig.getPushDownConverterClassNames()) {
+ if (PUSH_DOWN_CONVERTER_MAP.containsKey(clz)) {
+ converters.add(PUSH_DOWN_CONVERTER_MAP.get(clz));
+ } else {
+ try {
+ IPushDownConverter converter = (IPushDownConverter) ClassUtil.newInstance(clz);
+ PUSH_DOWN_CONVERTER_MAP.put(clz, converter);
+ converters.add(PUSH_DOWN_CONVERTER_MAP.get(clz));
+ } catch (Exception e) {
+ throw new IllegalStateException("Failed to init pushdown converter", e);
+ }
+ }
+ }
+ return converters;
+ }
+
+ public static String expandComputedColumnExp(NDataModel model, String project, String expression) {
+ StringBuilder forCC = new StringBuilder();
+ forCC.append("select ").append(expression).append(" ,").append(CC_SPLITTER) //
+ .append(" FROM ") //
+ .append(model.getRootFactTable().getTableDesc().getDoubleQuoteIdentity());
+ appendJoinStatement(model, forCC, false);
+
+ String ccSql = KeywordDefaultDirtyHack.transform(forCC.toString());
+ try {
+ // massage nested CC for drafted model
+ Map<String, NDataModel> modelMap = Maps.newHashMap();
+ modelMap.put(model.getUuid(), model);
+ ccSql = RestoreFromComputedColumn.convertWithGivenModels(ccSql, project, DEFAULT_SCHEMA, modelMap);
+ } catch (Exception e) {
+ logger.warn("Failed to massage SQL expression [{}] with input model {}", ccSql, model.getUuid(), e);
+ }
+ return ccSql;
+ }
+
+ public static void appendJoinStatement(NDataModel model, StringBuilder sql, boolean singleLine) {
+ final String sep = singleLine ? " " : "\n";
+ Set<TableRef> dimTableCache = Sets.newHashSet();
+ sql.append(sep);
+ for (JoinTableDesc lookupDesc : model.getJoinTables()) {
+ JoinDesc join = lookupDesc.getJoin();
+ TableRef dimTable = lookupDesc.getTableRef();
+ if (join == null || org.apache.commons.lang3.StringUtils.isEmpty(join.getType())
+ || dimTableCache.contains(dimTable)) {
+ continue;
+ }
+
+ TblColRef[] pk = join.getPrimaryKeyColumns();
+ TblColRef[] fk = join.getForeignKeyColumns();
+ if (pk.length != fk.length) {
+ throw new IllegalStateException("Invalid join condition of lookup table: " + lookupDesc);
+ }
+ String joinType = join.getType().toUpperCase(Locale.ROOT);
+
+ sql.append(joinType).append(" JOIN ").append(doubleQuote(dimTable)) //
+ .append(" as ").append(StringHelper.doubleQuote(dimTable.getAlias())) //
+ .append(sep).append("ON ");
+
+ if (pk.length == 0 && join.getNonEquiJoinCondition() != null) {
+ sql.append(join.getNonEquiJoinCondition().getExpr());
+ dimTableCache.add(dimTable);
+ } else {
+ sql.append(concatEqualJoinCondition(pk, fk, sep));
+ dimTableCache.add(dimTable);
+ }
+ }
+ }
+
+ private static String doubleQuote(TableRef tableRef) {
+ TableDesc table = tableRef.getTableDesc();
+ return StringHelper.doubleQuote(table.getDatabase()) + "." + StringHelper.doubleQuote(table.getName());
+ }
+
+ private static String concatEqualJoinCondition(TblColRef[] pk, TblColRef[] fk, String sep) {
+ StringJoiner joiner = new StringJoiner(" AND ", "", sep);
+ for (int i = 0; i < pk.length; i++) {
+ String s = fk[i].getDoubleQuoteExp() + " = " + pk[i].getDoubleQuoteExp();
+ joiner.add(s);
+ }
+ return joiner.toString();
+ }
+
+ /**
+ * Replace the escaped quote {@code ''} with {@code \'}. <br/>
+ * For example: <br/>
+ * the origin sql is {@code select 'a''b' from t} <br/>
+ * the replaced sql is {@code select 'a\'b' from t}
+ * @param sql the input sql
+ * @return replaced sql
+ */
+ static String replaceEscapedQuote(String sql) {
+ boolean inStrVal = false;
+ boolean needTransfer = false;
+ char[] res = sql.toCharArray();
+ for (int i = 0; i < res.length; i++) {
+ if (res[i] == '\'') {
+ if (inStrVal) {
+ if (needTransfer) {
+ res[i - 1] = '\\';
+ needTransfer = false;
+ } else {
+ needTransfer = true;
+ }
+ } else {
+ inStrVal = true;
+ }
+ } else if (needTransfer) {
+ inStrVal = false;
+ needTransfer = false;
+ }
+ }
+ return new String(res);
+ }
+
+ public static Pair<String, String> probeMinMaxTsWithTimeout(String partitionColumn, String table, String project)
+ throws ExecutionException, InterruptedException {
Future<Pair<String, String>> pushDownTask = asyncExecutor.submit(() -> {
try {
- return getMaxAndMinTime(partitionColumn, table, project);
+ return probeMinMaxTs(partitionColumn, table, project);
} catch (Exception e) {
logger.error("Failed to get partition column latest data range by push down!", e);
if (e instanceof KylinException) {
@@ -158,9 +338,9 @@ public class PushDownUtil {
return null;
});
- Pair<String, String> pushdownResult;
+ Pair<String, String> result;
try {
- pushdownResult = pushDownTask.get(30, TimeUnit.SECONDS);
+ result = pushDownTask.get(30, TimeUnit.SECONDS);
} catch (TimeoutException e) {
pushDownTask.cancel(true);
throw new KylinTimeoutException("The query exceeds the set time limit of "
@@ -168,18 +348,17 @@ public class PushDownUtil {
+ "s. Current step: Getting latest data range by push down. ");
}
- return pushdownResult;
+ return result;
}
- public static Pair<String, String> getMaxAndMinTime(String partitionColumn, String table, String project)
- throws Exception {
- Pair<String, String> pair = addBackTickForIdentity(table, partitionColumn);
- String sql = String.format(Locale.ROOT, "select min(%s), max(%s) from %s", pair.getSecond(), pair.getSecond(),
- pair.getFirst());
- Pair<String, String> result = new Pair<>();
- // pushdown
- List<List<String>> returnRows = PushDownUtil.selectPartitionColumn(sql, table, project).getFirst();
+ public static Pair<String, String> probeMinMaxTs(String partitionColumn, String table, String project)
+ throws SQLException {
+ String t = String.join(".", backtickQuote(table.split("\\.")));
+ String pc = String.join(".", backtickQuote(partitionColumn.split("\\.")));
+ String sql = String.format(Locale.ROOT, "select min(%s), max(%s) from %s", pc, pc, t);
+ Pair<String, String> result = new Pair<>();
+ List<List<String>> returnRows = probePartitionColInfo(sql, table, project).getFirst();
if (returnRows.isEmpty() || returnRows.get(0).get(0) == null || returnRows.get(0).get(1) == null)
throw new BadRequestException(String.format(Locale.ROOT, MsgPicker.getMsg().getNoDataInTable(), table));
@@ -194,75 +373,72 @@ public class PushDownUtil {
}
/**
- * Use push down engine to select partition column
+ * Use push-down engine to detect partition column info.
*/
- public static Pair<List<List<String>>, List<SelectedColumnMeta>> selectPartitionColumn(String sql, String table,
- String project) throws Exception {
+ public static Pair<List<List<String>>, List<SelectedColumnMeta>> probePartitionColInfo(String sql, String table,
+ String project) throws SQLException {
NTableMetadataManager tableMgr = NTableMetadataManager.getInstance(KylinConfig.getInstanceFromEnv(), project);
final TableDesc tableDesc = tableMgr.getTableDesc(table);
if (tableDesc.isView()) {
throw new KylinException(VIEW_PARTITION_DATE_FORMAT_DETECTION_FORBIDDEN,
MsgPicker.getMsg().getViewDateFormatDetectionError());
}
- val prjManager = NProjectManager.getInstance(KylinConfig.getInstanceFromEnv());
- val prj = prjManager.getProject(project);
- val kylinConfig = prj.getConfig();
+
List<List<String>> returnRows = Lists.newArrayList();
List<SelectedColumnMeta> returnColumnMeta = Lists.newArrayList();
- // pushdown
+ KylinConfig projectConfig = NProjectManager.getProjectConfig(project);
IPushDownRunner runner = (IPushDownRunner) ClassUtil
- .newInstance(kylinConfig.getPartitionCheckRunnerClassNameWithDefaultValue());
- runner.init(kylinConfig, project);
+ .newInstance(projectConfig.getDefaultPartitionCheckerClassName());
+ runner.init(projectConfig, project);
runner.executeQuery(sql, returnRows, returnColumnMeta, project);
return Pair.newPair(returnRows, returnColumnMeta);
}
- public static void trySimplePushDownExecute(String sql, String project) throws Exception {
+ public static void trySimplyExecute(String sql, String project) throws SQLException {
KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
IPushDownRunner runner = (IPushDownRunner) ClassUtil.newInstance(kylinConfig.getPushDownRunnerClassName());
runner.init(kylinConfig, project);
runner.executeUpdate(sql, project);
}
- public static String getFormatIfNotExist(String table, String partitionColumn, String project) throws Exception {
- Pair<String, String> pair = addBackTickForIdentity(table, partitionColumn);
- String sql = String.format(Locale.ROOT, "select %s from %s where %s is not null limit 1", pair.getSecond(),
- pair.getFirst(), pair.getSecond());
+ public static String probeColFormat(String table, String partitionColumn, String project) throws SQLException {
+ String t = String.join(".", backtickQuote(table.split("\\.")));
+ String sql = String.format(Locale.ROOT, "select %s from %s where %s is not null limit 1", partitionColumn, t,
+ partitionColumn);
+ return probe(sql, table, project);
+ }
+
+ public static String probeExpFormat(String project, String table, String expression) throws SQLException {
+ String t = String.join(".", backtickQuote(table.split("\\.")));
+ String sql = String.format(Locale.ROOT, "select %s from %s limit 1", expression, t);
+ return probe(sql, table, project);
+ }
- // push down
- List<List<String>> returnRows = PushDownUtil.selectPartitionColumn(sql, table, project).getFirst();
- if (CollectionUtils.isEmpty(returnRows) || CollectionUtils.isEmpty(returnRows.get(0)))
+ private static String probe(String sql, String table, String project) throws SQLException {
+ List<List<String>> returnRows = probePartitionColInfo(sql, table, project).getFirst();
+ if (CollectionUtils.isEmpty(returnRows) || CollectionUtils.isEmpty(returnRows.get(0))) {
throw new KylinException(EMPTY_TABLE,
String.format(Locale.ROOT, MsgPicker.getMsg().getNoDataInTable(), table));
-
+ }
return returnRows.get(0).get(0);
}
+ public static List<String> backtickQuote(String[] arr) {
+ return Arrays.stream(arr).map(StringHelper::backtickQuote).collect(Collectors.toList());
+ }
+
private static boolean isExpectedCause(SQLException sqlException) {
Preconditions.checkArgument(sqlException != null);
Throwable rootCause = ExceptionUtils.getRootCause(sqlException);
//SqlValidatorException is not an excepted exception in the origin design.But in the multi pass scene,
//query pushdown may create tables, and the tables are not in the model, so will throw SqlValidatorException.
- if (rootCause instanceof KylinTimeoutException) {
- return false;
- }
-
- if (rootCause instanceof AccessDeniedException) {
+ if (rootCause instanceof KylinTimeoutException || rootCause instanceof AccessDeniedException) {
return false;
- }
-
- if (rootCause instanceof RoutingIndicatorException) {
- return true;
- }
-
- if (rootCause instanceof CalciteNotSupportException) {
- return true;
- }
-
- if (rootCause instanceof CompileException) {
+ } else if (rootCause instanceof RoutingIndicatorException || rootCause instanceof CalciteNotSupportException
+ || rootCause instanceof CompileException) {
return true;
}
@@ -280,27 +456,4 @@ public class PushDownUtil {
}
return start;
}
-
- /**
- *
- * @param queryParams
- * @return
- * @throws Exception
- */
- public static Pair<List<List<String>>, List<SelectedColumnMeta>> tryPushDownQuery(QueryParams queryParams)
- throws Exception {
- val results = tryPushDownQueryToIterator(queryParams);
- if (results == null) {
- return null;
- }
- return new Pair<>(ImmutableList.copyOf(results.getRows()), results.getColumnMetas());
- }
-
- protected static Pair<String, String> addBackTickForIdentity(String table, String partitionColumn) {
- String tableName = Arrays.stream(table.split("\\.")).map(s -> "`" + s + "`").collect(Collectors.joining("."));
- String partitionColumnName = Arrays.stream(partitionColumn.split("\\.")).map(s -> "`" + s + "`")
- .collect(Collectors.joining("."));
- return Pair.newPair(tableName, partitionColumnName);
- }
-
}
diff --git a/src/query-common/src/main/java/org/apache/kylin/query/util/QueryAliasMatcher.java b/src/query-common/src/main/java/org/apache/kylin/query/util/QueryAliasMatcher.java
index f5940b6dfb..ab4fb87d2e 100644
--- a/src/query-common/src/main/java/org/apache/kylin/query/util/QueryAliasMatcher.java
+++ b/src/query-common/src/main/java/org/apache/kylin/query/util/QueryAliasMatcher.java
@@ -105,14 +105,14 @@ public class QueryAliasMatcher {
String tableAlias = namesOfIdentifier.get(1);
String colName = namesOfIdentifier.get(2);
ColumnRowType columnRowType = alias2CRT.get(tableAlias);
- Preconditions.checkState(columnRowType != null, "Alias {} is not defined", tableAlias);
+ Preconditions.checkState(columnRowType != null, "Alias %s is not defined", tableAlias);
return columnRowType == QueryAliasMatcher.SUBQUERY_TAG ? null : columnRowType.getColumnByName(colName);
} else if (namesOfIdentifier.size() == 2) {
// tableAlias.colName
String tableAlias = namesOfIdentifier.get(0);
String colName = namesOfIdentifier.get(1);
ColumnRowType columnRowType = alias2CRT.get(tableAlias);
- Preconditions.checkState(columnRowType != null, "Alias {} is not defined", tableAlias);
+ Preconditions.checkState(columnRowType != null, "Alias %s is not defined", tableAlias);
return columnRowType == QueryAliasMatcher.SUBQUERY_TAG ? null : columnRowType.getColumnByName(colName);
} else if (namesOfIdentifier.size() == 1) {
// only colName
@@ -500,9 +500,9 @@ public class QueryAliasMatcher {
return null;
}
- private TblColRef resolveComputedColumnRef(SqlCall call, String... tableCandidates) {
+ private TblColRef resolveComputedColumnRef(SqlCall call) {
foundCC = true;
- String table = findComputedColumnTable(call, tableCandidates);
+ String table = findComputedColumnTable(call);
ColumnDesc columnDesc = new ColumnDesc("-1", RandomUtil.randomUUIDStr(), "string", "", null, null,
call.toSqlString(CalciteSqlDialect.DEFAULT).getSql());
TableRef tableRef = alias2CRT.get(table).getColumnByIndex(0).getTableRef();
@@ -510,16 +510,16 @@ public class QueryAliasMatcher {
return TblColRef.columnForUnknownModel(tableRef, columnDesc);
}
- private String findComputedColumnTable(SqlCall call, final String... tableCandidates) {
+ private String findComputedColumnTable(SqlCall call) {
final String[] result = new String[1];
SqlBasicVisitor<SqlNode> visitor = new SqlBasicVisitor<SqlNode>() {
@Override
public SqlNode visit(SqlIdentifier sqlIdentifier) {
TblColRef colRef = resolveTblColRef(sqlIdentifier, alias2CRT);
- for (String table : tableCandidates) {
- if (alias2CRT.get(table).getAllColumns().contains(colRef)) {
- result[0] = table;
+ for (Map.Entry<String, ColumnRowType> aliasMap : alias2CRT.entrySet()) {
+ if (aliasMap.getValue().getAllColumns().contains(colRef)) {
+ result[0] = aliasMap.getKey();
return sqlIdentifier;
}
}
@@ -548,18 +548,15 @@ public class QueryAliasMatcher {
if ((operand0 instanceof SqlIdentifier || operand0 instanceof SqlCall)
&& (operand1 instanceof SqlIdentifier || operand1 instanceof SqlCall)) {
- int numOfAlias = alias2CRT.size();
String pkAlias = Iterables.getLast(alias2CRT.keySet());
- String fkAlias = Iterables.get(alias2CRT.keySet(), numOfAlias - 2);
-
// sqlCall maybe used as join condition, which need to
// translate as CC
TblColRef tblColRef0 = operand0 instanceof SqlIdentifier
? resolveTblColRef((SqlIdentifier) operand0, alias2CRT)
- : resolveComputedColumnRef((SqlCall) operand0, pkAlias, fkAlias);
+ : resolveComputedColumnRef((SqlCall) operand0);
TblColRef tblColRef1 = operand1 instanceof SqlIdentifier
? resolveTblColRef((SqlIdentifier) operand1, alias2CRT)
- : resolveComputedColumnRef((SqlCall) operand1, pkAlias, fkAlias);
+ : resolveComputedColumnRef((SqlCall) operand1);
if (tblColRef0 == null || tblColRef1 == null) {
return null;
diff --git a/src/query-common/src/main/java/org/apache/kylin/query/util/QueryUtil.java b/src/query-common/src/main/java/org/apache/kylin/query/util/QueryUtil.java
index c85c447fe0..6459d29a10 100644
--- a/src/query-common/src/main/java/org/apache/kylin/query/util/QueryUtil.java
+++ b/src/query-common/src/main/java/org/apache/kylin/query/util/QueryUtil.java
@@ -18,73 +18,39 @@
package org.apache.kylin.query.util;
-import java.util.Arrays;
-import java.util.Collections;
import java.util.List;
import java.util.Locale;
import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
-import java.util.stream.IntStream;
-
-import org.apache.calcite.plan.hep.HepRelVertex;
-import org.apache.calcite.plan.volcano.RelSubset;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.RelVisitor;
-import org.apache.calcite.rel.core.Aggregate;
-import org.apache.calcite.rel.core.Filter;
-import org.apache.calcite.rel.core.Join;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.core.TableScan;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexNode;
+
import org.apache.calcite.sql.SqlCall;
-import org.apache.calcite.sql.SqlKind;
import org.apache.calcite.sql.SqlOrderBy;
import org.apache.calcite.sql.SqlSelect;
-import org.apache.calcite.util.Util;
import org.apache.commons.lang3.StringUtils;
import org.apache.kylin.common.KapConfig;
import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.QueryContext;
import org.apache.kylin.common.exception.KylinTimeoutException;
import org.apache.kylin.common.util.ClassUtil;
-import org.apache.kylin.common.util.RandomUtil;
-import org.apache.kylin.metadata.model.ComputedColumnDesc;
-import org.apache.kylin.metadata.model.JoinDesc;
-import org.apache.kylin.metadata.model.JoinTableDesc;
-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.metadata.project.NProjectManager;
import org.apache.kylin.metadata.query.BigQueryThresholdUpdater;
import org.apache.kylin.query.IQueryTransformer;
import org.apache.kylin.query.SlowQueryDetector;
import org.apache.kylin.query.exception.UserStopQueryException;
-import org.apache.kylin.query.relnode.KapJoinRel;
import org.apache.kylin.query.security.AccessDeniedException;
-import org.apache.kylin.source.adhocquery.IPushDownConverter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
public class QueryUtil {
- private QueryUtil() {
- }
-
private static final Logger log = LoggerFactory.getLogger("query");
- public static final String DEFAULT_SCHEMA = "DEFAULT";
public static final ImmutableSet<String> REMOVED_TRANSFORMERS = ImmutableSet.of("ReplaceStringWithVarchar");
private static final Pattern SELECT_PATTERN = Pattern.compile("^select", Pattern.CASE_INSENSITIVE);
private static final Pattern SELECT_STAR_PTN = Pattern.compile("^select\\s+\\*\\p{all}*", Pattern.CASE_INSENSITIVE);
@@ -93,9 +59,22 @@ public class QueryUtil {
private static final String COLON = ":";
private static final String SEMI_COLON = ";";
public static final String JDBC = "jdbc";
+ private static final Map<String, IQueryTransformer> QUERY_TRANSFORMER_MAP = Maps.newConcurrentMap();
+
+ static {
+ String[] classNames = KylinConfig.getInstanceFromEnv().getQueryTransformers();
+ for (String clz : classNames) {
+ try {
+ IQueryTransformer transformer = (IQueryTransformer) ClassUtil.newInstance(clz);
+ QUERY_TRANSFORMER_MAP.put(clz, transformer);
+ } catch (Exception e) {
+ log.error("Failed to init query transformer of the sys-config: {}", clz);
+ }
+ }
+ }
- public static List<IQueryTransformer> queryTransformers = Collections.emptyList();
- public static List<IPushDownConverter> pushDownConverters = Collections.emptyList();
+ private QueryUtil() {
+ }
public static boolean isSelectStatement(String sql) {
String sql1 = sql.toLowerCase(Locale.ROOT);
@@ -113,8 +92,17 @@ public class QueryUtil {
return SELECT_STAR_PTN.matcher(sql).find();
}
+ /**
+ * Remove comment from query statement. There are two kind of comment patterns:<br/>
+ * 1. single line comment begins with "--",
+ * with <strong>-- CubePriority(m1,m2)</strong> excluded.<br/>
+ * 2. block comment like "/* comment content *⁄,
+ * with <strong>/*+ MODEL_PRIORITY(m1,m2) *⁄</strong> excluded.<br/>
+ *
+ * @param sql the sql to handle
+ * @return sql without comment
+ */
public static String removeCommentInSql(String sql) {
- // match two patterns, one is "-- comment", the other is "/* comment */"
try {
return new RawSqlParser(sql).parse().getStatementString();
} catch (Exception ex) {
@@ -190,207 +178,9 @@ public class QueryUtil {
return limitMatcher.find() ? originString : replacedString.concat(" limit 1");
}
- public static String massageExpression(NDataModel model, String project, String expression,
- QueryContext.AclInfo aclInfo, boolean isForPushDown) {
- String tempConst = "'" + RandomUtil.randomUUIDStr() + "'";
- StringBuilder forCC = new StringBuilder();
- forCC.append("select ").append(expression).append(" ,").append(tempConst) //
- .append(" FROM ") //
- .append(model.getRootFactTable().getTableDesc().getDoubleQuoteIdentity());
- appendJoinStatement(model, forCC, false);
-
- String ccSql = KeywordDefaultDirtyHack.transform(forCC.toString());
- try {
- // massage nested CC for drafted model
- Map<String, NDataModel> modelMap = Maps.newHashMap();
- modelMap.put(model.getUuid(), model);
- ccSql = RestoreFromComputedColumn.convertWithGivenModels(ccSql, project, DEFAULT_SCHEMA, modelMap);
- QueryParams queryParams = new QueryParams(project, ccSql, DEFAULT_SCHEMA, false);
- queryParams.setKylinConfig(NProjectManager.getProjectConfig(project));
- queryParams.setAclInfo(aclInfo);
-
- if (isForPushDown) {
- ccSql = massagePushDownSql(queryParams);
- }
- } catch (Exception e) {
- log.warn("Failed to massage SQL expression [{}] with input model {}", ccSql, model.getUuid(), e);
- }
-
- return ccSql.substring("select ".length(), ccSql.indexOf(tempConst) - 2).trim();
- }
-
- public static String massageExpression(NDataModel model, String project, String expression,
- QueryContext.AclInfo aclInfo) {
- return massageExpression(model, project, expression, aclInfo, true);
- }
-
- public static String massageComputedColumn(NDataModel model, String project, ComputedColumnDesc cc,
- QueryContext.AclInfo aclInfo) {
- return massageExpression(model, project, cc.getExpression(), aclInfo);
- }
-
- public static void appendJoinStatement(NDataModel model, StringBuilder sql, boolean singleLine) {
- final String sep = singleLine ? " " : "\n";
- Set<TableRef> dimTableCache = Sets.newHashSet();
- sql.append(sep);
- for (JoinTableDesc lookupDesc : model.getJoinTables()) {
- JoinDesc join = lookupDesc.getJoin();
- TableRef dimTable = lookupDesc.getTableRef();
- if (join == null || StringUtils.isEmpty(join.getType()) || dimTableCache.contains(dimTable)) {
- continue;
- }
-
- TblColRef[] pk = join.getPrimaryKeyColumns();
- TblColRef[] fk = join.getForeignKeyColumns();
- if (pk.length != fk.length) {
- throw new IllegalArgumentException("Invalid join condition of lookup table:" + lookupDesc);
- }
- String joinType = join.getType().toUpperCase(Locale.ROOT);
-
- sql.append(String.format(Locale.ROOT, "%s JOIN \"%s\".\"%s\" as \"%s\"", //
- joinType, dimTable.getTableDesc().getDatabase(), dimTable.getTableDesc().getName(),
- dimTable.getAlias()));
- sql.append(sep);
- sql.append("ON ");
-
- if (pk.length == 0 && join.getNonEquiJoinCondition() != null) {
- sql.append(join.getNonEquiJoinCondition().getExpr());
- dimTableCache.add(dimTable);
- } else {
- String collect = IntStream.range(0, pk.length) //
- .mapToObj(i -> fk[i].getDoubleQuoteExpressionInSourceDB() + " = "
- + pk[i].getDoubleQuoteExpressionInSourceDB())
- .collect(Collectors.joining(" AND ", "", sep));
- sql.append(collect);
- dimTableCache.add(dimTable);
- }
- }
- }
-
- public static SqlSelect extractSqlSelect(SqlCall selectOrOrderby) {
- SqlSelect sqlSelect = null;
-
- if (selectOrOrderby instanceof SqlSelect) {
- sqlSelect = (SqlSelect) selectOrOrderby;
- } else if (selectOrOrderby instanceof SqlOrderBy) {
- SqlOrderBy sqlOrderBy = ((SqlOrderBy) selectOrOrderby);
- if (sqlOrderBy.query instanceof SqlSelect) {
- sqlSelect = (SqlSelect) sqlOrderBy.query;
- }
- }
-
- return sqlSelect;
- }
-
- public static boolean isJoinOnlyOneAggChild(KapJoinRel joinRel) {
- RelNode joinLeftChild;
- RelNode joinRightChild;
- final RelNode joinLeft = joinRel.getLeft();
- final RelNode joinRight = joinRel.getRight();
- if (joinLeft instanceof RelSubset && joinRight instanceof RelSubset) {
- final RelSubset joinLeftChildSub = (RelSubset) joinLeft;
- final RelSubset joinRightChildSub = (RelSubset) joinRight;
- joinLeftChild = Util.first(joinLeftChildSub.getBest(), joinLeftChildSub.getOriginal());
- joinRightChild = Util.first(joinRightChildSub.getBest(), joinRightChildSub.getOriginal());
-
- } else if (joinLeft instanceof HepRelVertex && joinRight instanceof HepRelVertex) {
- joinLeftChild = ((HepRelVertex) joinLeft).getCurrentRel();
- joinRightChild = ((HepRelVertex) joinRight).getCurrentRel();
- } else {
- return false;
- }
-
- String project = QueryContext.current().getProject();
- if (project != null && NProjectManager.getProjectConfig(project).isEnhancedAggPushDownEnabled()
- && RelAggPushDownUtil.canRelAnsweredBySnapshot(project, joinRight)
- && RelAggPushDownUtil.isUnmatchedJoinRel(joinRel)) {
- QueryContext.current().setEnhancedAggPushDown(true);
- return true;
- }
-
- return isContainAggregate(joinLeftChild) ^ isContainAggregate(joinRightChild);
- }
-
- private static boolean isContainAggregate(RelNode node) {
- boolean[] isContainAggregate = new boolean[] { false };
- new RelVisitor() {
- @Override
- public void visit(RelNode node, int ordinal, RelNode parent) {
- if (isContainAggregate[0]) {
- // pruning
- return;
- }
- RelNode relNode = node;
- if (node instanceof RelSubset) {
- relNode = Util.first(((RelSubset) node).getBest(), ((RelSubset) node).getOriginal());
- } else if (node instanceof HepRelVertex) {
- relNode = ((HepRelVertex) node).getCurrentRel();
- }
- if (relNode instanceof Aggregate) {
- isContainAggregate[0] = true;
- }
- super.visit(relNode, ordinal, parent);
- }
- }.go(node);
- return isContainAggregate[0];
- }
-
- public static boolean isCast(RexNode rexNode) {
- if (!(rexNode instanceof RexCall)) {
- return false;
- }
- return SqlKind.CAST == rexNode.getKind();
- }
-
- public static boolean isPlainTableColumn(int colIdx, RelNode relNode) {
- if (relNode instanceof HepRelVertex) {
- relNode = ((HepRelVertex) relNode).getCurrentRel();
- }
- if (relNode instanceof TableScan) {
- return true;
- } else if (relNode instanceof Join) {
- Join join = (Join) relNode;
- int offset = 0;
- for (RelNode input : join.getInputs()) {
- if (colIdx >= offset && colIdx < offset + input.getRowType().getFieldCount()) {
- return isPlainTableColumn(colIdx - offset, input);
- }
- offset += input.getRowType().getFieldCount();
- }
- } else if (relNode instanceof Project) {
- RexNode inputRex = ((Project) relNode).getProjects().get(colIdx);
- if (inputRex instanceof RexInputRef) {
- return isPlainTableColumn(((RexInputRef) inputRex).getIndex(), ((Project) relNode).getInput());
- }
- } else if (relNode instanceof Filter) {
- return isPlainTableColumn(colIdx, relNode.getInput(0));
- }
- return false;
- }
-
- public static boolean containCast(RexNode rexNode) {
- if (!(rexNode instanceof RexCall)) {
- return false;
- }
- if (SqlKind.CAST == rexNode.getKind()) {
- RexNode operand = ((RexCall) rexNode).getOperands().get(0);
- return !(operand instanceof RexCall) || operand.getKind() == SqlKind.CASE;
- }
-
- return false;
- }
-
- public static boolean isNotNullLiteral(RexNode node) {
- return !isNullLiteral(node);
- }
-
- public static boolean isNullLiteral(RexNode node) {
- return node instanceof RexLiteral && ((RexLiteral) node).isNull();
- }
-
public static String massageSql(QueryParams queryParams) {
- String massagedSql = normalMassageSql(queryParams.getKylinConfig(), queryParams.getSql(),
- queryParams.getLimit(), queryParams.getOffset());
+ String massagedSql = appendLimitOffset(queryParams.getProject(), queryParams.getSql(), queryParams.getLimit(),
+ queryParams.getOffset());
queryParams.setSql(massagedSql);
massagedSql = transformSql(queryParams);
QueryContext.current().record("massage");
@@ -404,10 +194,15 @@ public class QueryUtil {
}
private static String transformSql(QueryParams queryParams) {
- // customizable SQL transformation
- initQueryTransformersIfNeeded(queryParams.getKylinConfig(), queryParams.isCCNeeded());
String sql = queryParams.getSql();
- for (IQueryTransformer t : queryTransformers) {
+ String[] classes = queryParams.getKylinConfig().getQueryTransformers();
+ List<IQueryTransformer> transformers = fetchTransformers(queryParams.isCCNeeded(), classes);
+ if (log.isDebugEnabled()) {
+ log.debug("All used query transformers are: {}", transformers.stream()
+ .map(clz -> clz.getClass().getCanonicalName()).collect(Collectors.joining(",")));
+ }
+
+ for (IQueryTransformer t : transformers) {
QueryUtil.checkThreadInterrupted("Interrupted sql transformation at the stage of " + t.getClass(),
"Current step: SQL transformation.");
sql = t.transform(sql, queryParams.getProject(), queryParams.getDefaultSchema());
@@ -415,14 +210,14 @@ public class QueryUtil {
return sql;
}
- private static String trimRightSemiColon(String sql) {
+ public static String trimRightSemiColon(String sql) {
while (sql.endsWith(SEMI_COLON)) {
sql = sql.substring(0, sql.length() - 1).trim();
}
return sql;
}
- public static String normalMassageSql(KylinConfig kylinConfig, String sql, int limit, int offset) {
+ public static String appendLimitOffset(String project, String sql, int limit, int offset) {
sql = sql.trim();
sql = sql.replace("\r", StringUtils.SPACE).replace("\n", System.getProperty("line.separator"));
sql = trimRightSemiColon(sql);
@@ -430,18 +225,19 @@ public class QueryUtil {
//Split keywords and variables from sql by punctuation and whitespace character
List<String> sqlElements = Lists.newArrayList(sql.toLowerCase(Locale.ROOT).split("(?![._'\"`])\\p{P}|\\s+"));
- Integer maxRows = kylinConfig.getMaxResultRows();
+ KylinConfig projectConfig = NProjectManager.getProjectConfig(project);
+ Integer maxRows = projectConfig.getMaxResultRows();
if (maxRows != null && maxRows > 0 && (maxRows < limit || limit <= 0)) {
limit = maxRows;
}
// https://issues.apache.org/jira/browse/KYLIN-2649
- if (kylinConfig.getForceLimit() > 0 && limit <= 0 && !sql.toLowerCase(Locale.ROOT).contains("limit")
+ if (projectConfig.getForceLimit() > 0 && limit <= 0 && !sql.toLowerCase(Locale.ROOT).contains("limit")
&& isSelectStarStatement(sql)) {
- limit = kylinConfig.getForceLimit();
+ limit = projectConfig.getForceLimit();
}
- if (checkBigQueryPushDown(kylinConfig)) {
+ if (isBigQueryPushDownCapable(projectConfig)) {
long bigQueryThreshold = BigQueryThresholdUpdater.getBigQueryThreshold();
if (limit <= 0 && bigQueryThreshold > 0) {
log.info("Big query route to pushdown, Add limit {} to sql.", bigQueryThreshold);
@@ -460,117 +256,51 @@ public class QueryUtil {
return sql;
}
- public static boolean checkBigQueryPushDown(KylinConfig kylinConfig) {
+ public static boolean isBigQueryPushDownCapable(KylinConfig kylinConfig) {
return kylinConfig.isBigQueryPushDown()
&& JDBC.equals(KapConfig.getInstanceFromEnv().getShareStateSwitchImplement());
}
- public static void initQueryTransformersIfNeeded(KylinConfig kylinConfig, boolean isCCNeeded) {
- String[] currentTransformers = queryTransformers.stream().map(Object::getClass).map(Class::getCanonicalName)
- .toArray(String[]::new);
- String[] configTransformers = kylinConfig.getQueryTransformers();
- boolean containsCCTransformer = Arrays.asList(configTransformers)
- .contains("org.apache.kylin.query.util.ConvertToComputedColumn");
- boolean transformersEqual = Objects.deepEquals(currentTransformers, configTransformers);
- if (transformersEqual && (isCCNeeded || !containsCCTransformer)) {
- return;
- }
-
- List<IQueryTransformer> transformers = initTransformers(isCCNeeded, configTransformers);
- queryTransformers = Collections.unmodifiableList(transformers);
- log.debug("SQL transformer: {}", queryTransformers);
- }
-
- public static List<IQueryTransformer> initTransformers(boolean isCCNeeded, String[] configTransformers) {
+ public static List<IQueryTransformer> fetchTransformers(boolean isCCNeeded, String[] configTransformers) {
List<IQueryTransformer> transformers = Lists.newArrayList();
- List<String> classList = Lists.newArrayList(configTransformers);
- classList.removeIf(clazz -> {
- String name = clazz.substring(clazz.lastIndexOf(".") + 1);
- return REMOVED_TRANSFORMERS.contains(name);
- });
-
- for (String clz : classList) {
- if (!isCCNeeded && clz.equals("org.apache.kylin.query.util.ConvertToComputedColumn"))
+ for (String clz : configTransformers) {
+ String name = clz.substring(clz.lastIndexOf('.') + 1);
+ if (REMOVED_TRANSFORMERS.contains(name)) {
continue;
-
- try {
- IQueryTransformer t = (IQueryTransformer) ClassUtil.newInstance(clz);
-
- transformers.add(t);
- } catch (Exception e) {
- throw new IllegalStateException("Failed to init query transformer", e);
}
- }
- return transformers;
- }
-
- // fix KE-34379,filter "/*+ MODEL_PRIORITY({cube_name}) */" hint
- private static final Pattern SQL_HINT_ERASER = Pattern
- .compile("/\\*\\s*\\+\\s*(?i)MODEL_PRIORITY\\s*\\([\\s\\S]*\\)\\s*\\*/");
-
- public static String massagePushDownSql(QueryParams queryParams) {
- String sql = queryParams.getSql();
- sql = trimRightSemiColon(sql);
-
- sql = SQL_HINT_ERASER.matcher(sql).replaceAll("");
- initPushDownConvertersIfNeeded(queryParams.getKylinConfig());
- for (IPushDownConverter converter : pushDownConverters) {
- QueryUtil.checkThreadInterrupted("Interrupted sql transformation at the stage of " + converter.getClass(),
- "Current step: Massage push-down sql. ");
- sql = converter.convert(sql, queryParams.getProject(), queryParams.getDefaultSchema());
- }
-
- sql = replaceDoubleQuoteToSingle(sql);
- return sql;
- }
-
- // To keep the results of cube and pushDown are same, we need to replace here
- public static String replaceDoubleQuoteToSingle(String originSql) {
- boolean inStrVal = false;
- boolean needTransfer = false;
- char[] res = originSql.toCharArray();
- for (int i = 0; i < res.length; i++) {
- if (res[i] == '\'') {
- if (inStrVal) {
- if (needTransfer) {
- res[i - 1] = '\\';
- needTransfer = false;
- } else {
- needTransfer = true;
- }
- } else {
- inStrVal = true;
- }
+ IQueryTransformer transformer;
+ if (QUERY_TRANSFORMER_MAP.containsKey(clz)) {
+ transformer = QUERY_TRANSFORMER_MAP.get(clz);
} else {
- if (needTransfer) {
- inStrVal = false;
- needTransfer = false;
+ try {
+ transformer = (IQueryTransformer) ClassUtil.newInstance(clz);
+ QUERY_TRANSFORMER_MAP.putIfAbsent(clz, transformer);
+ } catch (Exception e) {
+ throw new IllegalStateException("Failed to init query transformer", e);
}
}
+ if (transformer instanceof ConvertToComputedColumn) {
+ if (isCCNeeded) {
+ transformers.add(transformer);
+ }
+ } else {
+ transformers.add(transformer);
+ }
}
- return new String(res);
+ return transformers;
}
- static void initPushDownConvertersIfNeeded(KylinConfig kylinConfig) {
- String[] currentConverters = pushDownConverters.stream().map(Object::getClass).map(Class::getCanonicalName)
- .toArray(String[]::new);
- String[] configConverters = kylinConfig.getPushDownConverterClassNames();
- boolean skipInit = Objects.deepEquals(currentConverters, configConverters);
-
- if (skipInit) {
- return;
- }
-
- List<IPushDownConverter> converters = Lists.newArrayList();
- for (String clz : configConverters) {
- try {
- IPushDownConverter converter = (IPushDownConverter) ClassUtil.newInstance(clz);
- converters.add(converter);
- } catch (Exception e) {
- throw new IllegalStateException("Failed to init pushdown converter", e);
+ public static SqlSelect extractSqlSelect(SqlCall selectOrOrderby) {
+ SqlSelect sqlSelect = null;
+ if (selectOrOrderby instanceof SqlSelect) {
+ sqlSelect = (SqlSelect) selectOrOrderby;
+ } else if (selectOrOrderby instanceof SqlOrderBy) {
+ SqlOrderBy sqlOrderBy = ((SqlOrderBy) selectOrOrderby);
+ if (sqlOrderBy.query instanceof SqlSelect) {
+ sqlSelect = (SqlSelect) sqlOrderBy.query;
}
}
- pushDownConverters = Collections.unmodifiableList(converters);
+ return sqlSelect;
}
public static void checkThreadInterrupted(String errorMsgLog, String stepInfo) {
diff --git a/src/query-service/src/test/java/org/apache/kylin/rest/service/ModelServiceQueryTest.java b/src/query-service/src/test/java/org/apache/kylin/rest/service/ModelServiceQueryTest.java
index 2d9e81b01b..2b0eeb31d8 100644
--- a/src/query-service/src/test/java/org/apache/kylin/rest/service/ModelServiceQueryTest.java
+++ b/src/query-service/src/test/java/org/apache/kylin/rest/service/ModelServiceQueryTest.java
@@ -35,6 +35,7 @@ import org.apache.kylin.metadata.cube.model.NDataflowManager;
import org.apache.kylin.metadata.model.NDataModel;
import org.apache.kylin.metadata.model.util.ExpandableMeasureUtil;
import org.apache.kylin.metadata.query.QueryTimesResponse;
+import org.apache.kylin.query.util.PushDownUtil;
import org.apache.kylin.metadata.recommendation.candidate.JdbcRawRecStore;
import org.apache.kylin.query.util.QueryUtil;
import org.apache.kylin.rest.config.initialize.ModelBrokenListener;
@@ -114,7 +115,7 @@ public class ModelServiceQueryTest extends SourceTestCase {
ReflectionTestUtils.setField(semanticService, "userGroupService", userGroupService);
ReflectionTestUtils.setField(semanticService, "expandableMeasureUtil",
new ExpandableMeasureUtil((model, ccDesc) -> {
- String ccExpression = QueryUtil.massageComputedColumn(model, model.getProject(), ccDesc,
+ String ccExpression = PushDownUtil.massageComputedColumn(model, model.getProject(), ccDesc,
AclPermissionUtil.createAclInfo(model.getProject(),
semanticService.getCurrentUserGroups()));
ccDesc.setInnerExpression(ccExpression);
diff --git a/src/query/pom.xml b/src/query/pom.xml
index ed43542bf9..2c5b3a9a54 100644
--- a/src/query/pom.xml
+++ b/src/query/pom.xml
@@ -20,7 +20,7 @@
<packaging>jar</packaging>
<name>Kylin - Query</name>
<url>http://kylin.apache.org</url>
- <description>Kylin Query Engine</description>
+ <description>Kylin Query Engine</description>
<parent>
<groupId>org.apache.kylin</groupId>
@@ -116,6 +116,13 @@
<scope>provided</scope>
</dependency>
+ <!--test jars-->
+ <dependency>
+ <groupId>org.apache.kylin</groupId>
+ <artifactId>kylin-core-metadata</artifactId>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
<dependency>
<groupId>org.awaitility</groupId>
<artifactId>awaitility</artifactId>
diff --git a/src/query/src/main/java/io/kyligence/kap/query/optrule/AbstractAggCaseWhenFunctionRule.java b/src/query/src/main/java/io/kyligence/kap/query/optrule/AbstractAggCaseWhenFunctionRule.java
index 6a015f6ada..a2bfb3a03b 100644
--- a/src/query/src/main/java/io/kyligence/kap/query/optrule/AbstractAggCaseWhenFunctionRule.java
+++ b/src/query/src/main/java/io/kyligence/kap/query/optrule/AbstractAggCaseWhenFunctionRule.java
@@ -18,9 +18,6 @@
package io.kyligence.kap.query.optrule;
-import static org.apache.kylin.query.util.QueryUtil.isCast;
-import static org.apache.kylin.query.util.QueryUtil.isNotNullLiteral;
-
import java.math.BigDecimal;
import java.util.ArrayList;
import java.util.Arrays;
@@ -55,6 +52,7 @@ import org.apache.kylin.query.relnode.ContextUtil;
import org.apache.kylin.query.util.AggExpressionUtil;
import org.apache.kylin.query.util.AggExpressionUtil.AggExpression;
import org.apache.kylin.query.util.AggExpressionUtil.GroupExpression;
+import org.apache.kylin.query.util.RuleUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -69,7 +67,7 @@ public abstract class AbstractAggCaseWhenFunctionRule extends RelOptRule {
private static final Logger logger = LoggerFactory.getLogger(AbstractAggCaseWhenFunctionRule.class);
- public AbstractAggCaseWhenFunctionRule(RelOptRuleOperand operand, RelBuilderFactory relBuilderFactory,
+ protected AbstractAggCaseWhenFunctionRule(RelOptRuleOperand operand, RelBuilderFactory relBuilderFactory,
String description) {
super(operand, relBuilderFactory, description);
}
@@ -319,7 +317,7 @@ public abstract class AbstractAggCaseWhenFunctionRule extends RelOptRule {
List<RexNode> values = aggExpression.getValuesList();
for (int i = 0; i < values.size(); i++) {
aggExpression.getBottomAggValuesInput()[i] = bottomProjectList.size();
- if (isCast(values.get(i))) {
+ if (RuleUtils.isCast(values.get(i))) {
RexNode rexNode = ((RexCall) (values.get(i))).operands.get(0);
DataType dataType = DataType.getType(rexNode.getType().getSqlTypeName().getName());
if (!AggExpressionUtil.isSum(aggExpression.getAggCall().getAggregation().kind)
@@ -328,7 +326,7 @@ public abstract class AbstractAggCaseWhenFunctionRule extends RelOptRule {
} else {
bottomProjectList.add(values.get(i));
}
- } else if (isNotNullLiteral(values.get(i))) {
+ } else if (RuleUtils.isNotNullLiteral(values.get(i))) {
bottomProjectList.add(values.get(i));
} else {
bottomProjectList.add(rexBuilder.makeBigintLiteral(BigDecimal.ZERO));
@@ -405,7 +403,8 @@ public abstract class AbstractAggCaseWhenFunctionRule extends RelOptRule {
thenNode = relBuilder.getRexBuilder().makeCast(((RexCall) thenNode).type,
relBuilder.getRexBuilder().makeInputRef(relBuilder.peek(),
aggExpression.getTopProjValuesInput()[whenIndex]));
- } else if (isNotNullLiteral(thenNode)) {// TODO? keep null or sum(null)
+ } else if (RuleUtils.isNotNullLiteral(thenNode)) {
+ // keep null or sum(null)?
thenNode = relBuilder.getRexBuilder().makeInputRef(relBuilder.peek(),
aggExpression.getTopProjValuesInput()[whenIndex]);
}
@@ -415,7 +414,8 @@ public abstract class AbstractAggCaseWhenFunctionRule extends RelOptRule {
if (isNeedTackCast(elseNode)) {
elseNode = relBuilder.getRexBuilder().makeCast(((RexCall) elseNode).type, relBuilder.getRexBuilder()
.makeInputRef(relBuilder.peek(), aggExpression.getTopProjValuesInput()[whenIndex]));
- } else if (isNotNullLiteral(elseNode)) {// TODO? keep null or sum(null)
+ } else if (RuleUtils.isNotNullLiteral(elseNode)) {
+ // keep null or sum(null)?
elseNode = relBuilder.getRexBuilder().makeInputRef(relBuilder.peek(),
aggExpression.getTopProjValuesInput()[whenIndex]);
}
@@ -467,7 +467,7 @@ public abstract class AbstractAggCaseWhenFunctionRule extends RelOptRule {
}
protected boolean isNeedTackCast(RexNode rexNode) {
- return isCast(rexNode);
+ return RuleUtils.isCast(rexNode);
}
private static final String BOTTOM_AGG_PREFIX = "SUB_AGG$";
diff --git a/src/query/src/main/java/io/kyligence/kap/query/optrule/CountDistinctCaseWhenFunctionRule.java b/src/query/src/main/java/io/kyligence/kap/query/optrule/CountDistinctCaseWhenFunctionRule.java
index ddf0e6d303..46ee6b609e 100644
--- a/src/query/src/main/java/io/kyligence/kap/query/optrule/CountDistinctCaseWhenFunctionRule.java
+++ b/src/query/src/main/java/io/kyligence/kap/query/optrule/CountDistinctCaseWhenFunctionRule.java
@@ -18,10 +18,6 @@
package io.kyligence.kap.query.optrule;
-import static org.apache.kylin.query.util.QueryUtil.isCast;
-import static org.apache.kylin.query.util.QueryUtil.isNullLiteral;
-import static org.apache.kylin.query.util.QueryUtil.isPlainTableColumn;
-
import java.util.ArrayList;
import java.util.List;
@@ -62,6 +58,7 @@ import org.apache.kylin.query.relnode.KapAggregateRel;
import org.apache.kylin.query.relnode.KapProjectRel;
import org.apache.kylin.query.util.AggExpressionUtil;
import org.apache.kylin.query.util.AggExpressionUtil.AggExpression;
+import org.apache.kylin.query.util.RuleUtils;
/**
* COUNT(DISTINCT (CASE WHEN ... THEN COLUMN ELSE NULL))
@@ -119,11 +116,11 @@ public class CountDistinctCaseWhenFunctionRule extends AbstractAggCaseWhenFuncti
}
private boolean isSimpleCaseWhen(Project inputProject, RexNode n1, RexNode n2) {
- if (isNullLiteral(n1)) {
+ if (RuleUtils.isNullLiteral(n1)) {
if (n2 instanceof RexInputRef) {
- return isPlainTableColumn(((RexInputRef) n2).getIndex(), inputProject.getInput(0));
- } else if (isCast(n2) && ((RexCall) n2).getOperands().get(0) instanceof RexInputRef) {
- return isPlainTableColumn(((RexInputRef) ((RexCall) n2).getOperands().get(0)).getIndex(),
+ return RuleUtils.isPlainTableColumn(((RexInputRef) n2).getIndex(), inputProject.getInput(0));
+ } else if (RuleUtils.isCast(n2) && ((RexCall) n2).getOperands().get(0) instanceof RexInputRef) {
+ return RuleUtils.isPlainTableColumn(((RexInputRef) ((RexCall) n2).getOperands().get(0)).getIndex(),
inputProject.getInput(0)) && !isNeedTackCast(n2);
}
}
@@ -209,7 +206,7 @@ public class CountDistinctCaseWhenFunctionRule extends AbstractAggCaseWhenFuncti
@Override
protected boolean isValidAggColumnExpr(RexNode rexNode) {
- return !isNullLiteral(rexNode);
+ return !RuleUtils.isNullLiteral(rexNode);
}
/**
@@ -217,7 +214,7 @@ public class CountDistinctCaseWhenFunctionRule extends AbstractAggCaseWhenFuncti
*/
@Override
protected boolean isNeedTackCast(RexNode rexNode) {
- if (!isCast(rexNode)) {
+ if (!RuleUtils.isCast(rexNode)) {
return false;
}
return !SqlTypeUtil.canCastFrom(rexNode.getType(), ((RexCall) rexNode).getOperands().get(0).getType(), false);
diff --git a/src/query/src/main/java/io/kyligence/kap/query/optrule/KapAggFilterTransposeRule.java b/src/query/src/main/java/io/kyligence/kap/query/optrule/KapAggFilterTransposeRule.java
index e6d8ca19cd..984ec00f4e 100644
--- a/src/query/src/main/java/io/kyligence/kap/query/optrule/KapAggFilterTransposeRule.java
+++ b/src/query/src/main/java/io/kyligence/kap/query/optrule/KapAggFilterTransposeRule.java
@@ -40,7 +40,7 @@ import org.apache.calcite.util.mapping.Mappings;
import org.apache.kylin.query.relnode.KapAggregateRel;
import org.apache.kylin.query.relnode.KapFilterRel;
import org.apache.kylin.query.relnode.KapJoinRel;
-import org.apache.kylin.query.util.QueryUtil;
+import org.apache.kylin.query.util.RuleUtils;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
@@ -71,7 +71,7 @@ public class KapAggFilterTransposeRule extends RelOptRule {
final KapJoinRel joinRel = call.rel(2);
//Only one agg child of join is accepted
- return QueryUtil.isJoinOnlyOneAggChild(joinRel);
+ return RuleUtils.isJoinOnlyOneAggChild(joinRel);
}
@Override
diff --git a/src/query/src/main/java/io/kyligence/kap/query/optrule/KapAggJoinTransposeRule.java b/src/query/src/main/java/io/kyligence/kap/query/optrule/KapAggJoinTransposeRule.java
index 71bd01daa3..b1a26b4b76 100644
--- a/src/query/src/main/java/io/kyligence/kap/query/optrule/KapAggJoinTransposeRule.java
+++ b/src/query/src/main/java/io/kyligence/kap/query/optrule/KapAggJoinTransposeRule.java
@@ -57,7 +57,7 @@ import org.apache.calcite.util.mapping.Mapping;
import org.apache.calcite.util.mapping.Mappings;
import org.apache.kylin.query.relnode.KapAggregateRel;
import org.apache.kylin.query.relnode.KapJoinRel;
-import org.apache.kylin.query.util.QueryUtil;
+import org.apache.kylin.query.util.RuleUtils;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
@@ -89,7 +89,7 @@ public class KapAggJoinTransposeRule extends RelOptRule {
final KapAggregateRel aggregate = call.rel(0);
final KapJoinRel joinRel = call.rel(1);
//Only one agg child of join is accepted
- return !aggregate.isContainCountDistinct() && QueryUtil.isJoinOnlyOneAggChild(joinRel);
+ return !aggregate.isContainCountDistinct() && RuleUtils.isJoinOnlyOneAggChild(joinRel);
}
@Override
diff --git a/src/query/src/main/java/io/kyligence/kap/query/optrule/KapAggProjectMergeRule.java b/src/query/src/main/java/io/kyligence/kap/query/optrule/KapAggProjectMergeRule.java
index d4ba23a7b7..3b532d778b 100644
--- a/src/query/src/main/java/io/kyligence/kap/query/optrule/KapAggProjectMergeRule.java
+++ b/src/query/src/main/java/io/kyligence/kap/query/optrule/KapAggProjectMergeRule.java
@@ -39,7 +39,7 @@ import org.apache.kylin.query.relnode.KapAggregateRel;
import org.apache.kylin.query.relnode.KapFilterRel;
import org.apache.kylin.query.relnode.KapJoinRel;
import org.apache.kylin.query.relnode.KapProjectRel;
-import org.apache.kylin.query.util.QueryUtil;
+import org.apache.kylin.query.util.RuleUtils;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
@@ -72,7 +72,7 @@ public class KapAggProjectMergeRule extends RelOptRule {
? call.rel(3)
: call.rel(2);
//Only one agg child of join is accepted
- if (!QueryUtil.isJoinOnlyOneAggChild(joinRel)) {
+ if (!RuleUtils.isJoinOnlyOneAggChild(joinRel)) {
return false;
}
diff --git a/src/query/src/main/java/io/kyligence/kap/query/optrule/KapAggProjectTransposeRule.java b/src/query/src/main/java/io/kyligence/kap/query/optrule/KapAggProjectTransposeRule.java
index c35e7b2ff5..6c2d7f107f 100644
--- a/src/query/src/main/java/io/kyligence/kap/query/optrule/KapAggProjectTransposeRule.java
+++ b/src/query/src/main/java/io/kyligence/kap/query/optrule/KapAggProjectTransposeRule.java
@@ -48,7 +48,7 @@ import org.apache.kylin.query.relnode.KapAggregateRel;
import org.apache.kylin.query.relnode.KapFilterRel;
import org.apache.kylin.query.relnode.KapJoinRel;
import org.apache.kylin.query.relnode.KapProjectRel;
-import org.apache.kylin.query.util.QueryUtil;
+import org.apache.kylin.query.util.RuleUtils;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
@@ -93,7 +93,7 @@ public class KapAggProjectTransposeRule extends RelOptRule {
}
//Only one agg child of join is accepted
- if (!QueryUtil.isJoinOnlyOneAggChild(joinRel)) {
+ if (!RuleUtils.isJoinOnlyOneAggChild(joinRel)) {
return false;
}
diff --git a/src/query/src/main/java/io/kyligence/kap/query/optrule/KapCountDistinctJoinRule.java b/src/query/src/main/java/io/kyligence/kap/query/optrule/KapCountDistinctJoinRule.java
index 01addae49b..16f62d12d0 100644
--- a/src/query/src/main/java/io/kyligence/kap/query/optrule/KapCountDistinctJoinRule.java
+++ b/src/query/src/main/java/io/kyligence/kap/query/optrule/KapCountDistinctJoinRule.java
@@ -35,7 +35,7 @@ import org.apache.kylin.query.relnode.KapAggregateRel;
import org.apache.kylin.query.relnode.KapJoinRel;
import org.apache.kylin.query.relnode.KapProjectRel;
import org.apache.kylin.query.relnode.KapRel;
-import org.apache.kylin.query.util.QueryUtil;
+import org.apache.kylin.query.util.RuleUtils;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
@@ -68,7 +68,7 @@ public class KapCountDistinctJoinRule extends RelOptRule {
} else {
join = call.rel(2);
}
- return aggregate.isContainCountDistinct() && QueryUtil.isJoinOnlyOneAggChild(join);
+ return aggregate.isContainCountDistinct() && RuleUtils.isJoinOnlyOneAggChild(join);
}
@Override
diff --git a/src/query/src/main/java/io/kyligence/kap/query/optrule/KapSumCastTransposeRule.java b/src/query/src/main/java/io/kyligence/kap/query/optrule/KapSumCastTransposeRule.java
index f9608854d7..53e568de1c 100644
--- a/src/query/src/main/java/io/kyligence/kap/query/optrule/KapSumCastTransposeRule.java
+++ b/src/query/src/main/java/io/kyligence/kap/query/optrule/KapSumCastTransposeRule.java
@@ -18,9 +18,6 @@
package io.kyligence.kap.query.optrule;
-import static org.apache.kylin.query.util.QueryUtil.containCast;
-import static org.apache.kylin.query.util.QueryUtil.isNotNullLiteral;
-
import java.math.BigDecimal;
import java.util.List;
import java.util.stream.Collectors;
@@ -53,6 +50,7 @@ import org.apache.kylin.query.relnode.ContextUtil;
import org.apache.kylin.query.relnode.KapAggregateRel;
import org.apache.kylin.query.relnode.KapProjectRel;
import org.apache.kylin.query.util.AggExpressionUtil;
+import org.apache.kylin.query.util.RuleUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -77,7 +75,7 @@ public class KapSumCastTransposeRule extends RelOptRule {
}
List<RexNode> childExps = project.getChildExps();
for (RexNode rexNode : childExps) {
- if (containCast(rexNode)) {
+ if (RuleUtils.containCast(rexNode)) {
return true;
}
}
@@ -93,7 +91,7 @@ public class KapSumCastTransposeRule extends RelOptRule {
if (AggExpressionUtil.isSum(aggCall.getAggregation().kind)) {
int index = aggCall.getArgList().get(0);
RexNode value = originalProject.getProjects().get(index);
- if (containCast(value)) {
+ if (RuleUtils.containCast(value)) {
RexNode rexNode = ((RexCall) value).getOperands().get(0);
DataType dataType = DataType.getType(rexNode.getType().getSqlTypeName().getName());
return dataType.isNumberFamily() || dataType.isIntegerFamily();
@@ -123,7 +121,7 @@ public class KapSumCastTransposeRule extends RelOptRule {
relBuilder.push(oldProject.getInput());
List<AggExpressionUtil.AggExpression> aggExpressions = oldAgg.getAggCallList().stream()
- .map(call -> new AggExpressionUtil.AggExpression(call)).collect(Collectors.toList());
+ .map(AggExpressionUtil.AggExpression::new).collect(Collectors.toList());
// #1 Build bottom project
List<RexNode> bottomProjectList = buildBottomProject(oldProject, aggExpressions);
@@ -140,8 +138,7 @@ public class KapSumCastTransposeRule extends RelOptRule {
List<RexNode> caseProjList = buildTopProject(relBuilder, oldProject, oldAgg, aggExpressions);
relBuilder.project(caseProjList);
- RelNode relNode = relBuilder.build();
- return relNode;
+ return relBuilder.build();
}
private List<RexNode> buildBottomProject(Project oldProject, List<AggExpressionUtil.AggExpression> aggExpressions) {
@@ -156,7 +153,7 @@ public class KapSumCastTransposeRule extends RelOptRule {
if (AggExpressionUtil.isSum(aggCall.getAggregation().kind)) {
int index = aggCall.getArgList().get(0);
RexNode value = oldProject.getProjects().get(index);
- if (containCast(value)) {
+ if (RuleUtils.containCast(value)) {
bottomProjectList.set(index, ((RexCall) (value)).operands.get(0));
RelDataType type = ((RexCall) (value)).operands.get(0).getType();
if (type instanceof BasicSqlType && SqlTypeName.INTEGER == type.getSqlTypeName()) {
@@ -205,7 +202,7 @@ public class KapSumCastTransposeRule extends RelOptRule {
if (AggExpressionUtil.isSum(aggCall.getAggregation().kind)) {
int index = aggCall.getArgList().get(0);
RexNode value = oldProject.getProjects().get(index);
- if (containCast(value)) {
+ if (RuleUtils.containCast(value)) {
RelDataType type = ((RexCall) value).type;
if (type instanceof BasicSqlType && type.getPrecision() < aggCall.getType().getPrecision()) {
type = aggCall.getType();
@@ -213,7 +210,7 @@ public class KapSumCastTransposeRule extends RelOptRule {
value = relBuilder.getRexBuilder().makeCast(type,
relBuilder.getRexBuilder().makeInputRef(relBuilder.peek(), i));
topProjectList.add(value);
- } else if (isNotNullLiteral(value)) {
+ } else if (RuleUtils.isNotNullLiteral(value)) {
value = relBuilder.getRexBuilder().makeInputRef(relBuilder.peek(), i);
topProjectList.add(value);
} else {
diff --git a/src/query/src/main/java/org/apache/kylin/query/QueryCli.java b/src/query/src/main/java/org/apache/kylin/query/QueryCli.java
deleted file mode 100644
index b042875653..0000000000
--- a/src/query/src/main/java/org/apache/kylin/query/QueryCli.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kylin.query;
-
-import java.sql.Connection;
-import java.sql.ResultSet;
-import java.sql.Statement;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.CommandLineParser;
-import org.apache.commons.cli.GnuParser;
-import org.apache.commons.cli.Option;
-import org.apache.commons.cli.OptionBuilder;
-import org.apache.commons.cli.Options;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.DBUtils;
-
-public class QueryCli {
-
- @SuppressWarnings("static-access")
- private static final Option OPTION_METADATA = OptionBuilder.withArgName("metadata url").hasArg().isRequired()
- .withDescription("Metadata URL").create("metadata");
-
- @SuppressWarnings("static-access")
- private static final Option OPTION_SQL = OptionBuilder.withArgName("input sql").hasArg().isRequired()
- .withDescription("SQL").create("sql");
-
- public static void main(String[] args) throws Exception {
-
- Options options = new Options();
- options.addOption(OPTION_METADATA);
- options.addOption(OPTION_SQL);
-
- CommandLineParser parser = new GnuParser();
- CommandLine commandLine = parser.parse(options, args);
- KylinConfig config = KylinConfig.createInstanceFromUri(commandLine.getOptionValue(OPTION_METADATA.getOpt()));
- String sql = commandLine.getOptionValue(OPTION_SQL.getOpt());
-
- Connection conn = null;
- Statement stmt = null;
- ResultSet rs = null;
- try {
-
- // remove since this class looks deprecated
- // stmt = conn.createStatement();
- // rs = stmt.executeQuery(sql);
- // int n = 0;
- // ResultSetMetaData meta = rs.getMetaData();
- // while (rs.next()) {
- // n++;
- // for (int i = 1; i <= meta.getColumnCount(); i++) {
- // System.out.println(n + " - " + meta.getColumnLabel(i) + ":\t" + rs.getObject(i));
- // }
- // }
- } finally {
- DBUtils.closeQuietly(rs);
- DBUtils.closeQuietly(stmt);
- DBUtils.closeQuietly(conn);
- }
-
- }
-}
diff --git a/src/query/src/main/java/org/apache/kylin/query/engine/QueryRoutingEngine.java b/src/query/src/main/java/org/apache/kylin/query/engine/QueryRoutingEngine.java
index 81439609e9..cad6d87c86 100644
--- a/src/query/src/main/java/org/apache/kylin/query/engine/QueryRoutingEngine.java
+++ b/src/query/src/main/java/org/apache/kylin/query/engine/QueryRoutingEngine.java
@@ -76,8 +76,7 @@ public class QueryRoutingEngine {
public QueryResult queryWithSqlMassage(QueryParams queryParams) throws Exception {
QueryContext.current().setAclInfo(queryParams.getAclInfo());
- KylinConfig projectKylinConfig = NProjectManager.getInstance(KylinConfig.getInstanceFromEnv())
- .getProject(queryParams.getProject()).getConfig();
+ KylinConfig projectKylinConfig = NProjectManager.getProjectConfig(queryParams.getProject());
QueryExec queryExec = new QueryExec(queryParams.getProject(), projectKylinConfig, true);
queryParams.setDefaultSchema(queryExec.getDefaultSchemaName());
@@ -230,7 +229,7 @@ public class QueryRoutingEngine {
private boolean checkBigQueryPushDown(QueryParams queryParams) {
KylinConfig kylinConfig = NProjectManager.getInstance(KylinConfig.getInstanceFromEnv())
.getProject(queryParams.getProject()).getConfig();
- boolean isPush = QueryUtil.checkBigQueryPushDown(kylinConfig);
+ boolean isPush = QueryUtil.isBigQueryPushDownCapable(kylinConfig);
if (isPush) {
logger.info("Big query route to pushdown.");
}
@@ -271,22 +270,21 @@ public class QueryRoutingEngine {
sqlString = QueryUtil.addLimit(sqlString);
}
- String massagedSql = QueryUtil.normalMassageSql(KylinConfig.getInstanceFromEnv(), sqlString,
- queryParams.getLimit(), queryParams.getOffset());
+ String massagedSql = QueryUtil.appendLimitOffset(queryParams.getProject(), sqlString, queryParams.getLimit(),
+ queryParams.getOffset());
if (isPrepareStatementWithParams(queryParams)) {
QueryContext.current().getMetrics().setCorrectedSql(massagedSql);
}
queryParams.setSql(massagedSql);
queryParams.setSqlException(sqlException);
queryParams.setPrepare(isPrepare);
- return PushDownUtil.tryPushDownQueryToIterator(queryParams);
+ return PushDownUtil.tryIterQuery(queryParams);
}
private boolean isPrepareStatementWithParams(QueryParams queryParams) {
- KylinConfig kylinConfig = NProjectManager.getInstance(KylinConfig.getInstanceFromEnv())
- .getProject(queryParams.getProject()).getConfig();
+ KylinConfig projectConfig = NProjectManager.getProjectConfig(queryParams.getProject());
return (KapConfig.getInstanceFromEnv().enablePushdownPrepareStatementWithParams()
- || kylinConfig.enableReplaceDynamicParams()) && queryParams.isPrepareStatementWithParams();
+ || projectConfig.enableReplaceDynamicParams()) && queryParams.isPrepareStatementWithParams();
}
private QueryResult prepareOnly(String correctedSql, QueryExec queryExec, List<List<String>> results,
diff --git a/src/query/src/main/java/org/apache/kylin/query/util/RuleUtils.java b/src/query/src/main/java/org/apache/kylin/query/util/RuleUtils.java
new file mode 100644
index 0000000000..61e9607a85
--- /dev/null
+++ b/src/query/src/main/java/org/apache/kylin/query/util/RuleUtils.java
@@ -0,0 +1,150 @@
+/*
+ * 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.util;
+
+import org.apache.calcite.plan.hep.HepRelVertex;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelVisitor;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.Util;
+import org.apache.kylin.common.QueryContext;
+import org.apache.kylin.metadata.project.NProjectManager;
+import org.apache.kylin.query.relnode.KapJoinRel;
+
+public class RuleUtils {
+
+ private RuleUtils() {
+ }
+
+ public static boolean isJoinOnlyOneAggChild(KapJoinRel joinRel) {
+ RelNode joinLeftChild;
+ RelNode joinRightChild;
+ final RelNode joinLeft = joinRel.getLeft();
+ final RelNode joinRight = joinRel.getRight();
+ if (joinLeft instanceof RelSubset && joinRight instanceof RelSubset) {
+ final RelSubset joinLeftChildSub = (RelSubset) joinLeft;
+ final RelSubset joinRightChildSub = (RelSubset) joinRight;
+ joinLeftChild = Util.first(joinLeftChildSub.getBest(), joinLeftChildSub.getOriginal());
+ joinRightChild = Util.first(joinRightChildSub.getBest(), joinRightChildSub.getOriginal());
+
+ } else if (joinLeft instanceof HepRelVertex && joinRight instanceof HepRelVertex) {
+ joinLeftChild = ((HepRelVertex) joinLeft).getCurrentRel();
+ joinRightChild = ((HepRelVertex) joinRight).getCurrentRel();
+ } else {
+ return false;
+ }
+
+ String project = QueryContext.current().getProject();
+ if (project != null && NProjectManager.getProjectConfig(project).isEnhancedAggPushDownEnabled()
+ && RelAggPushDownUtil.canRelAnsweredBySnapshot(project, joinRight)
+ && RelAggPushDownUtil.isUnmatchedJoinRel(joinRel)) {
+ QueryContext.current().setEnhancedAggPushDown(true);
+ return true;
+ }
+
+ return isContainAggregate(joinLeftChild) ^ isContainAggregate(joinRightChild);
+ }
+
+ private static boolean isContainAggregate(RelNode node) {
+ boolean[] isContainAggregate = new boolean[] { false };
+ new RelVisitor() {
+ @Override
+ public void visit(RelNode node, int ordinal, RelNode parent) {
+ if (isContainAggregate[0]) {
+ // pruning
+ return;
+ }
+ RelNode relNode = node;
+ if (node instanceof RelSubset) {
+ relNode = Util.first(((RelSubset) node).getBest(), ((RelSubset) node).getOriginal());
+ } else if (node instanceof HepRelVertex) {
+ relNode = ((HepRelVertex) node).getCurrentRel();
+ }
+ if (relNode instanceof Aggregate) {
+ isContainAggregate[0] = true;
+ }
+ super.visit(relNode, ordinal, parent);
+ }
+ }.go(node);
+ return isContainAggregate[0];
+ }
+
+ public static boolean isCast(RexNode rexNode) {
+ if (!(rexNode instanceof RexCall)) {
+ return false;
+ }
+ return SqlKind.CAST == rexNode.getKind();
+ }
+
+ public static boolean isPlainTableColumn(int colIdx, RelNode relNode) {
+ if (relNode instanceof HepRelVertex) {
+ relNode = ((HepRelVertex) relNode).getCurrentRel();
+ }
+ if (relNode instanceof TableScan) {
+ return true;
+ } else if (relNode instanceof Join) {
+ Join join = (Join) relNode;
+ int offset = 0;
+ for (RelNode input : join.getInputs()) {
+ if (colIdx >= offset && colIdx < offset + input.getRowType().getFieldCount()) {
+ return isPlainTableColumn(colIdx - offset, input);
+ }
+ offset += input.getRowType().getFieldCount();
+ }
+ } else if (relNode instanceof Project) {
+ RexNode inputRex = ((Project) relNode).getProjects().get(colIdx);
+ if (inputRex instanceof RexInputRef) {
+ return isPlainTableColumn(((RexInputRef) inputRex).getIndex(), ((Project) relNode).getInput());
+ }
+ } else if (relNode instanceof Filter) {
+ return isPlainTableColumn(colIdx, relNode.getInput(0));
+ }
+ return false;
+ }
+
+ public static boolean containCast(RexNode rexNode) {
+ if (!(rexNode instanceof RexCall)) {
+ return false;
+ }
+ if (SqlKind.CAST == rexNode.getKind()) {
+ RexNode operand = ((RexCall) rexNode).getOperands().get(0);
+ return !(operand instanceof RexCall) || operand.getKind() == SqlKind.CASE;
+ }
+
+ return false;
+ }
+
+ public static boolean isNotNullLiteral(RexNode node) {
+ return !isNullLiteral(node);
+ }
+
+ public static boolean isNullLiteral(RexNode node) {
+ return node instanceof RexLiteral && ((RexLiteral) node).isNull();
+ }
+}
diff --git a/src/query/src/test/java/org/apache/kylin/query/util/PushDownUtilTest.java b/src/query/src/test/java/org/apache/kylin/query/util/PushDownUtilTest.java
index 798da7d042..0a57502dcf 100644
--- a/src/query/src/test/java/org/apache/kylin/query/util/PushDownUtilTest.java
+++ b/src/query/src/test/java/org/apache/kylin/query/util/PushDownUtilTest.java
@@ -18,8 +18,10 @@
package org.apache.kylin.query.util;
+import java.util.Properties;
+
+import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.util.NLocalFileMetadataTestCase;
-import org.apache.kylin.common.util.Pair;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@@ -38,13 +40,13 @@ public class PushDownUtilTest extends NLocalFileMetadataTestCase {
}
@Test
- public void testTryPushDownQuery_ForcedToPushDownBasic() throws Exception {
+ public void testTryForcePushDown() {
try {
QueryParams queryParams = new QueryParams();
queryParams.setProject("default");
queryParams.setSelect(true);
queryParams.setForcedToPushDown(true);
- PushDownUtil.tryPushDownQuery(queryParams);
+ PushDownUtil.tryIterQuery(queryParams);
Assert.fail();
} catch (Exception e) {
Assert.assertFalse(e instanceof IllegalArgumentException);
@@ -52,14 +54,14 @@ public class PushDownUtilTest extends NLocalFileMetadataTestCase {
}
@Test
- public void testTryPushDownQuery_PushDownDisable() throws Exception {
+ public void testTryWithPushDownDisable() {
try {
overwriteSystemProp("kylin.query.pushdown-enabled", "false");
QueryParams queryParams = new QueryParams();
queryParams.setProject("default");
queryParams.setSelect(true);
queryParams.setForcedToPushDown(true);
- PushDownUtil.tryPushDownQuery(queryParams);
+ PushDownUtil.tryIterQuery(queryParams);
Assert.fail();
} catch (Exception e) {
Assert.assertFalse(e instanceof IllegalArgumentException);
@@ -67,12 +69,68 @@ public class PushDownUtilTest extends NLocalFileMetadataTestCase {
}
@Test
- public void testAddBackticksForIdentity() {
+ public void testBacktickQuote() {
String table = "db.table";
- String partitionColumn = "table.column";
- Pair<String, String> backticksIdentityPair = PushDownUtil.addBackTickForIdentity(table, partitionColumn);
- Assert.assertEquals("`db`.`table`", backticksIdentityPair.getFirst());
- Assert.assertEquals("`table`.`column`", backticksIdentityPair.getSecond());
+ Assert.assertEquals("`db`.`table`", String.join(".", PushDownUtil.backtickQuote(table.split("\\."))));
+ }
+
+ @Test
+ public void testMassagePushDownSql() {
+ KylinConfig config = KylinConfig.createKylinConfig(new Properties());
+ try (KylinConfig.SetAndUnsetThreadLocalConfig autoUnset = KylinConfig.setAndUnsetThreadLocalConfig(config)) {
+ config.setProperty("kylin.query.pushdown.converter-class-names",
+ SparkSQLFunctionConverter.class.getCanonicalName());
+ String sql = "SELECT \"Z_PROVDASH_UM_ED\".\"GENDER\" AS \"GENDER\",\n"
+ + "SUM({fn CONVERT(0, SQL_BIGINT)}) AS \"sum_Calculation_336925569152049156_ok\"\n"
+ + "FROM \"POPHEALTH_ANALYTICS\".\"Z_PROVDASH_UM_ED\" \"Z_PROVDASH_UM_ED\"";
+
+ QueryParams queryParams = new QueryParams("", sql, "default", false);
+ queryParams.setKylinConfig(config);
+ String massagedSql = PushDownUtil.massagePushDownSql(queryParams);
+ String expectedSql = "SELECT `Z_PROVDASH_UM_ED`.`GENDER` AS `GENDER`,\n"
+ + "SUM(CAST(0 AS BIGINT)) AS `sum_Calculation_336925569152049156_ok`\n"
+ + "FROM `POPHEALTH_ANALYTICS`.`Z_PROVDASH_UM_ED` `Z_PROVDASH_UM_ED`";
+ Assert.assertEquals(expectedSql, massagedSql);
+ }
+ }
+
+ @Test
+ public void testMassagePushDownSqlWithDoubleQuote() {
+ KylinConfig config = KylinConfig.createKylinConfig(new Properties());
+ String sql = "select '''',trans_id from test_kylin_fact where LSTG_FORMAT_NAME like '%''%' group by trans_id limit 2;";
+ QueryParams queryParams = new QueryParams("", sql, "default", false);
+ queryParams.setKylinConfig(config);
+ String massagedSql = PushDownUtil.massagePushDownSql(queryParams);
+ String expectedSql = "select '\\'', `TRANS_ID` from `TEST_KYLIN_FACT` where `LSTG_FORMAT_NAME` like '%\\'%' group by `TRANS_ID` limit 2";
+ Assert.assertEquals(expectedSql, massagedSql);
+ }
+
+ @Test
+ public void testMassagePushDownSqlWithDialectConverter() {
+ KylinConfig config = KylinConfig.createKylinConfig(new Properties());
+ try (KylinConfig.SetAndUnsetThreadLocalConfig autoUnset = KylinConfig.setAndUnsetThreadLocalConfig(config)) {
+ config.setProperty("kylin.query.pushdown.converter-class-names",
+ "org.apache.kylin.query.util.DialectConverter,org.apache.kylin.source.adhocquery.DoubleQuotePushDownConverter,"
+ + SparkSQLFunctionConverter.class.getCanonicalName());
+ String sql = "SELECT \"Z_PROVDASH_UM_ED\".\"GENDER\" AS \"GENDER\",\n"
+ + "SUM({fn CONVERT(0, SQL_BIGINT)}) AS \"sum_Calculation_336925569152049156_ok\"\n"
+ + "FROM \"POPHEALTH_ANALYTICS\".\"Z_PROVDASH_UM_ED\" \"Z_PROVDASH_UM_ED\""
+ + " fetch first 1 rows only";
+
+ QueryParams queryParams = new QueryParams("", sql, "default", false);
+ queryParams.setKylinConfig(config);
+ String massagedSql = PushDownUtil.massagePushDownSql(queryParams);
+ String expectedSql = "SELECT `Z_PROVDASH_UM_ED`.`GENDER` AS `GENDER`, "
+ + "SUM(CAST(0 AS BIGINT)) AS `sum_Calculation_336925569152049156_ok`\n"
+ + "FROM `POPHEALTH_ANALYTICS`.`Z_PROVDASH_UM_ED` AS `Z_PROVDASH_UM_ED`\n" + "LIMIT 1";
+ Assert.assertEquals(expectedSql, massagedSql);
+ }
}
-}
\ No newline at end of file
+ @Test
+ public void testReplaceDoubleQuoteToSingle() {
+ String sql = "select ab from table where aa = '' and bb = '''as''n'''";
+ String resSql = "select ab from table where aa = '' and bb = '\\'as\\'n\\''";
+ Assert.assertEquals(resSql, PushDownUtil.replaceEscapedQuote(sql));
+ }
+}
diff --git a/src/query/src/test/java/org/apache/kylin/query/util/QueryUtilTest.java b/src/query/src/test/java/org/apache/kylin/query/util/QueryUtilTest.java
index 3d0ac0f1c5..0b2120263f 100644
--- a/src/query/src/test/java/org/apache/kylin/query/util/QueryUtilTest.java
+++ b/src/query/src/test/java/org/apache/kylin/query/util/QueryUtilTest.java
@@ -19,33 +19,31 @@
package org.apache.kylin.query.util;
import static org.junit.Assert.assertEquals;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.mockStatic;
-import static org.mockito.Mockito.when;
import java.sql.SQLException;
-import java.util.Collections;
+import java.util.List;
+import java.util.Map;
import java.util.Properties;
import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.KylinConfig.SetAndUnsetThreadLocalConfig;
import org.apache.kylin.common.util.NLocalFileMetadataTestCase;
import org.apache.kylin.metadata.model.ComputedColumnDesc;
import org.apache.kylin.metadata.model.NDataModelManager;
+import org.apache.kylin.query.IQueryTransformer;
import org.apache.kylin.query.security.AccessDeniedException;
+import org.apache.kylin.util.MetadataTestUtils;
import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
-import org.mockito.MockedStatic;
+
+import com.google.common.collect.Maps;
public class QueryUtilTest extends NLocalFileMetadataTestCase {
@Before
public void setUp() throws Exception {
- QueryUtil.queryTransformers = Collections.emptyList();
- QueryUtil.pushDownConverters = Collections.emptyList();
this.createTestMetadata();
}
@@ -54,48 +52,40 @@ public class QueryUtilTest extends NLocalFileMetadataTestCase {
this.cleanupTestMetadata();
}
- public static final String SQL = "select * from table1";
-
@Test
public void testMaxResultRowsEnabled() {
- try (MockedStatic<KylinConfig> kylinConfigMockedStatic = mockStatic(KylinConfig.class)) {
- KylinConfig kylinConfig = mock(KylinConfig.class);
- kylinConfigMockedStatic.when(KylinConfig::getInstanceFromEnv).thenReturn(kylinConfig);
- when(kylinConfig.getMaxResultRows()).thenReturn(15);
- when(kylinConfig.getForceLimit()).thenReturn(14);
- String result = QueryUtil.normalMassageSql(kylinConfig, SQL, 16, 0);
- assertEquals("select * from table1" + "\n" + "LIMIT 15", result);
- }
+ Map<String, String> map = Maps.newHashMap();
+ map.put("kylin.query.max-result-rows", "15");
+ map.put("kylin.query.force-limit", "14");
+ MetadataTestUtils.updateProjectConfig("default", map);
+ String result = QueryUtil.appendLimitOffset("default", "select * from table1", 16, 0);
+ assertEquals("select * from table1" + "\n" + "LIMIT 15", result);
}
@Test
public void testCompareMaxResultRowsAndLimit() {
- try (MockedStatic<KylinConfig> kylinConfigMockedStatic = mockStatic(KylinConfig.class)) {
- KylinConfig kylinConfig = mock(KylinConfig.class);
- kylinConfigMockedStatic.when(KylinConfig::getInstanceFromEnv).thenReturn(kylinConfig);
- when(kylinConfig.getMaxResultRows()).thenReturn(15);
- when(kylinConfig.getForceLimit()).thenReturn(14);
- String result = QueryUtil.normalMassageSql(kylinConfig, SQL, 13, 0);
- assertEquals("select * from table1" + "\n" + "LIMIT 13", result);
- }
+ Map<String, String> map = Maps.newHashMap();
+ map.put("kylin.query.max-result-rows", "15");
+ map.put("kylin.query.force-limit", "14");
+ MetadataTestUtils.updateProjectConfig("default", map);
+ String result = QueryUtil.appendLimitOffset("default", "select * from table1", 13, 0);
+ assertEquals("select * from table1" + "\n" + "LIMIT 13", result);
}
@Test
public void testMassageSql() {
- KylinConfig config = KylinConfig.createKylinConfig(new Properties());
- try (SetAndUnsetThreadLocalConfig autoUnset = KylinConfig.setAndUnsetThreadLocalConfig(config)) {
- config.setProperty("kylin.query.transformers", DefaultQueryTransformer.class.getCanonicalName());
-
- String sql = "SELECT * FROM TABLE";
- QueryParams queryParams1 = new QueryParams(config, sql, "", 100, 20, "", true);
- String newSql = QueryUtil.massageSql(queryParams1);
- Assert.assertEquals("SELECT * FROM TABLE\nLIMIT 100\nOFFSET 20", newSql);
-
- String sql2 = "SELECT SUM({fn convert(0, INT)}) from TABLE";
- QueryParams queryParams2 = new QueryParams(config, sql2, "", 0, 0, "", true);
- String newSql2 = QueryUtil.massageSql(queryParams2);
- Assert.assertEquals("SELECT SUM({fn convert(0, INT)}) from TABLE", newSql2);
- }
+ KylinConfig config = KylinConfig.getInstanceFromEnv();
+ config.setProperty("kylin.query.transformers", DefaultQueryTransformer.class.getCanonicalName());
+
+ String sql = "SELECT * FROM TABLE1";
+ QueryParams queryParams1 = new QueryParams(config, sql, "default", 100, 20, "", true);
+ String newSql = QueryUtil.massageSql(queryParams1);
+ Assert.assertEquals("SELECT * FROM TABLE1\nLIMIT 100\nOFFSET 20", newSql);
+
+ String sql2 = "SELECT SUM({fn convert(0, INT)}) from TABLE1";
+ QueryParams queryParams2 = new QueryParams(config, sql2, "default", 0, 0, "", true);
+ String newSql2 = QueryUtil.massageSql(queryParams2);
+ Assert.assertEquals("SELECT SUM({fn convert(0, INT)}) from TABLE1", newSql2);
}
@Test
@@ -121,29 +111,27 @@ public class QueryUtilTest extends NLocalFileMetadataTestCase {
public void testMassageWithoutConvertToComputedColumn() {
KylinConfig config = KylinConfig.getInstanceFromEnv();
- try (SetAndUnsetThreadLocalConfig autoUnset = KylinConfig.setAndUnsetThreadLocalConfig(config)) {
- // enable ConvertToComputedColumn
- config.setProperty("kylin.query.transformers", "org.apache.kylin.query.util.ConvertToComputedColumn");
- QueryParams queryParams1 = new QueryParams(config, "SELECT price * item_count FROM test_kylin_fact",
- "default", 0, 0, "DEFAULT", true);
- String newSql1 = QueryUtil.massageSql(queryParams1);
- Assert.assertEquals("SELECT TEST_KYLIN_FACT.DEAL_AMOUNT FROM test_kylin_fact", newSql1);
- QueryParams queryParams2 = new QueryParams(config,
- "SELECT price * item_count,DEAL_AMOUNT FROM test_kylin_fact", "default", 0, 0, "DEFAULT", true);
- newSql1 = QueryUtil.massageSql(queryParams2);
- Assert.assertEquals("SELECT TEST_KYLIN_FACT.DEAL_AMOUNT,DEAL_AMOUNT FROM test_kylin_fact", newSql1);
-
- // disable ConvertToComputedColumn
- config.setProperty("kylin.query.transformers", "");
- QueryParams queryParams3 = new QueryParams(config, "SELECT price * item_count FROM test_kylin_fact",
- "default", 0, 0, "DEFAULT", true);
- String newSql2 = QueryUtil.massageSql(queryParams3);
- Assert.assertEquals("SELECT price * item_count FROM test_kylin_fact", newSql2);
- QueryParams queryParams4 = new QueryParams(config,
- "SELECT price * item_count,DEAL_AMOUNT FROM test_kylin_fact", "default", 0, 0, "DEFAULT", false);
- newSql2 = QueryUtil.massageSql(queryParams4);
- Assert.assertEquals("SELECT price * item_count,DEAL_AMOUNT FROM test_kylin_fact", newSql2);
- }
+ // enable ConvertToComputedColumn
+ config.setProperty("kylin.query.transformers", "org.apache.kylin.query.util.ConvertToComputedColumn");
+ QueryParams queryParams1 = new QueryParams(config, "SELECT price * item_count FROM test_kylin_fact", "default",
+ 0, 0, "DEFAULT", true);
+ String newSql1 = QueryUtil.massageSql(queryParams1);
+ Assert.assertEquals("SELECT TEST_KYLIN_FACT.DEAL_AMOUNT FROM test_kylin_fact", newSql1);
+ QueryParams queryParams2 = new QueryParams(config, "SELECT price * item_count,DEAL_AMOUNT FROM test_kylin_fact",
+ "default", 0, 0, "DEFAULT", true);
+ newSql1 = QueryUtil.massageSql(queryParams2);
+ Assert.assertEquals("SELECT TEST_KYLIN_FACT.DEAL_AMOUNT,DEAL_AMOUNT FROM test_kylin_fact", newSql1);
+
+ // disable ConvertToComputedColumn
+ config.setProperty("kylin.query.transformers", "");
+ QueryParams queryParams3 = new QueryParams(config, "SELECT price * item_count FROM test_kylin_fact", "default",
+ 0, 0, "DEFAULT", true);
+ String newSql2 = QueryUtil.massageSql(queryParams3);
+ Assert.assertEquals("SELECT price * item_count FROM test_kylin_fact", newSql2);
+ QueryParams queryParams4 = new QueryParams(config, "SELECT price * item_count,DEAL_AMOUNT FROM test_kylin_fact",
+ "default", 0, 0, "DEFAULT", false);
+ newSql2 = QueryUtil.massageSql(queryParams4);
+ Assert.assertEquals("SELECT price * item_count,DEAL_AMOUNT FROM test_kylin_fact", newSql2);
}
@Test
@@ -196,64 +184,31 @@ public class QueryUtilTest extends NLocalFileMetadataTestCase {
}
@Test
- public void testMassagePushDownSql() {
+ public void testInit() {
KylinConfig config = KylinConfig.createKylinConfig(new Properties());
- try (SetAndUnsetThreadLocalConfig autoUnset = KylinConfig.setAndUnsetThreadLocalConfig(config)) {
- config.setProperty("kylin.query.pushdown.converter-class-names",
- SparkSQLFunctionConverter.class.getCanonicalName());
- String sql = "SELECT \"Z_PROVDASH_UM_ED\".\"GENDER\" AS \"GENDER\",\n"
- + "SUM({fn CONVERT(0, SQL_BIGINT)}) AS \"sum_Calculation_336925569152049156_ok\"\n"
- + "FROM \"POPHEALTH_ANALYTICS\".\"Z_PROVDASH_UM_ED\" \"Z_PROVDASH_UM_ED\"";
-
- QueryParams queryParams = new QueryParams("", sql, "default", false);
- queryParams.setKylinConfig(config);
- String massagedSql = QueryUtil.massagePushDownSql(queryParams);
- String expectedSql = "SELECT `Z_PROVDASH_UM_ED`.`GENDER` AS `GENDER`,\n"
- + "SUM(CAST(0 AS BIGINT)) AS `sum_Calculation_336925569152049156_ok`\n"
- + "FROM `POPHEALTH_ANALYTICS`.`Z_PROVDASH_UM_ED` `Z_PROVDASH_UM_ED`";
- Assert.assertEquals(expectedSql, massagedSql);
- }
- }
+ config.setProperty("kylin.query.transformers", DefaultQueryTransformer.class.getCanonicalName());
+ List<IQueryTransformer> transformers = QueryUtil.fetchTransformers(true, config.getQueryTransformers());
+ Assert.assertEquals(1, transformers.size());
+ Assert.assertTrue(transformers.get(0) instanceof DefaultQueryTransformer);
- @Test
- public void testMassagePushDownSqlWithDoubleQuote() {
- KylinConfig config = KylinConfig.createKylinConfig(new Properties());
- String sql = "select '''',trans_id from test_kylin_fact where LSTG_FORMAT_NAME like '%''%' group by trans_id limit 2;";
- QueryParams queryParams = new QueryParams("", sql, "default", false);
- queryParams.setKylinConfig(config);
- String massagedSql = QueryUtil.massagePushDownSql(queryParams);
- String expectedSql = "select '\\'', `TRANS_ID` from `TEST_KYLIN_FACT` where `LSTG_FORMAT_NAME` like '%\\'%' group by `TRANS_ID` limit 2";
- Assert.assertEquals(expectedSql, massagedSql);
- }
+ config.setProperty("kylin.query.transformers", KeywordDefaultDirtyHack.class.getCanonicalName());
+ transformers = QueryUtil.fetchTransformers(true, config.getQueryTransformers());
+ Assert.assertEquals(1, transformers.size());
+ Assert.assertTrue(transformers.get(0) instanceof KeywordDefaultDirtyHack);
+
+ transformers = QueryUtil.fetchTransformers(false, config.getQueryTransformers());
+ Assert.assertEquals(1, transformers.size());
+ Assert.assertTrue(transformers.get(0) instanceof KeywordDefaultDirtyHack);
+
+ config.setProperty("kylin.query.transformers", DefaultQueryTransformer.class.getCanonicalName() + ","
+ + ConvertToComputedColumn.class.getCanonicalName());
+ transformers = QueryUtil.fetchTransformers(true, config.getQueryTransformers());
+ Assert.assertEquals(2, transformers.size());
+
+ transformers = QueryUtil.fetchTransformers(false, config.getQueryTransformers());
+ Assert.assertEquals(1, transformers.size());
+ Assert.assertTrue(transformers.get(0) instanceof DefaultQueryTransformer);
- @Test
- public void testInit() {
- KylinConfig config = KylinConfig.createKylinConfig(new Properties());
- try (SetAndUnsetThreadLocalConfig autoUnset = KylinConfig.setAndUnsetThreadLocalConfig(config)) {
-
- config.setProperty("kylin.query.transformers", DefaultQueryTransformer.class.getCanonicalName());
- Assert.assertEquals(0, QueryUtil.queryTransformers.size());
- QueryUtil.initQueryTransformersIfNeeded(config, true);
- Assert.assertEquals(1, QueryUtil.queryTransformers.size());
- Assert.assertTrue(QueryUtil.queryTransformers.get(0) instanceof DefaultQueryTransformer);
-
- config.setProperty("kylin.query.transformers", KeywordDefaultDirtyHack.class.getCanonicalName());
- QueryUtil.initQueryTransformersIfNeeded(config, true);
- Assert.assertEquals(1, QueryUtil.queryTransformers.size());
- Assert.assertTrue(QueryUtil.queryTransformers.get(0) instanceof KeywordDefaultDirtyHack);
-
- QueryUtil.initQueryTransformersIfNeeded(config, false);
- Assert.assertEquals(1, QueryUtil.queryTransformers.size());
- Assert.assertTrue(QueryUtil.queryTransformers.get(0) instanceof KeywordDefaultDirtyHack);
-
- config.setProperty("kylin.query.transformers", DefaultQueryTransformer.class.getCanonicalName() + ","
- + ConvertToComputedColumn.class.getCanonicalName());
- QueryUtil.initQueryTransformersIfNeeded(config, true);
- Assert.assertEquals(2, QueryUtil.queryTransformers.size());
- QueryUtil.initQueryTransformersIfNeeded(config, false);
- Assert.assertEquals(1, QueryUtil.queryTransformers.size());
- Assert.assertTrue(QueryUtil.queryTransformers.get(0) instanceof DefaultQueryTransformer);
- }
}
@Test
@@ -466,7 +421,6 @@ public class QueryUtilTest extends NLocalFileMetadataTestCase {
Assert.assertEquals(
"select LINEORDER.CC_CAST_LO_ORDERKEY from lineorder inner join customer on lineorder.lo_custkey = customer.c_custkey",
newSql2);
-
}
@Test
@@ -518,28 +472,6 @@ public class QueryUtilTest extends NLocalFileMetadataTestCase {
newSql6);
}
- @Test
- public void testMassagePushDownSqlWithDialectConverter() {
- KylinConfig config = KylinConfig.createKylinConfig(new Properties());
- try (SetAndUnsetThreadLocalConfig autoUnset = KylinConfig.setAndUnsetThreadLocalConfig(config)) {
- config.setProperty("kylin.query.pushdown.converter-class-names",
- "org.apache.kylin.query.util.DialectConverter,org.apache.kylin.source.adhocquery.DoubleQuotePushDownConverter,"
- + SparkSQLFunctionConverter.class.getCanonicalName());
- String sql = "SELECT \"Z_PROVDASH_UM_ED\".\"GENDER\" AS \"GENDER\",\n"
- + "SUM({fn CONVERT(0, SQL_BIGINT)}) AS \"sum_Calculation_336925569152049156_ok\"\n"
- + "FROM \"POPHEALTH_ANALYTICS\".\"Z_PROVDASH_UM_ED\" \"Z_PROVDASH_UM_ED\""
- + " fetch first 1 rows only";
-
- QueryParams queryParams = new QueryParams("", sql, "default", false);
- queryParams.setKylinConfig(config);
- String massagedSql = QueryUtil.massagePushDownSql(queryParams);
- String expectedSql = "SELECT `Z_PROVDASH_UM_ED`.`GENDER` AS `GENDER`, "
- + "SUM(CAST(0 AS BIGINT)) AS `sum_Calculation_336925569152049156_ok`\n"
- + "FROM `POPHEALTH_ANALYTICS`.`Z_PROVDASH_UM_ED` AS `Z_PROVDASH_UM_ED`\n" + "LIMIT 1";
- Assert.assertEquals(expectedSql, massagedSql);
- }
- }
-
@Test
public void testBigQueryPushDown() {
KylinConfig config = KylinConfig.getInstanceFromEnv();
@@ -557,6 +489,7 @@ public class QueryUtilTest extends NLocalFileMetadataTestCase {
@Test
public void testBigQueryPushDownByParams() {
+ // no limit offset from backend and front-end
KylinConfig config = KylinConfig.createKylinConfig(new Properties());
String sql1 = "select TRANS_ID as test_limit, ORDER_ID as test_offset from TEST_KYLIN_FACT group by TRANS_ID, ORDER_ID";
QueryParams queryParams1 = new QueryParams(config, sql1, "default", 0, 0, "DEFAULT", true);
@@ -564,77 +497,104 @@ public class QueryUtilTest extends NLocalFileMetadataTestCase {
Assert.assertEquals(
"select TRANS_ID as test_limit, ORDER_ID as test_offset from TEST_KYLIN_FACT group by TRANS_ID, ORDER_ID",
newSql1);
+
+ // both limit and offset are 0
String sql = "select TRANS_ID as test_limit, ORDER_ID as test_offset from TEST_KYLIN_FACT group by TRANS_ID, ORDER_ID";
QueryParams queryParams = new QueryParams(config, sql, "default", 0, 0, "DEFAULT", true);
String targetSQL = QueryUtil.massageSql(queryParams);
Assert.assertEquals(
"select TRANS_ID as test_limit, ORDER_ID as test_offset from TEST_KYLIN_FACT group by TRANS_ID, ORDER_ID",
targetSQL);
+
+ // limit 1 from front-end
queryParams = new QueryParams(config, sql, "default", 1, 0, "DEFAULT", true);
targetSQL = QueryUtil.massageSql(queryParams);
Assert.assertEquals(
"select TRANS_ID as test_limit, ORDER_ID as test_offset from TEST_KYLIN_FACT group by TRANS_ID, ORDER_ID\n"
+ "LIMIT 1",
targetSQL);
- config.setProperty("kylin.query.max-result-rows", "2");
- queryParams = new QueryParams(config, sql, "default", 0, 0, "DEFAULT", true);
- targetSQL = QueryUtil.massageSql(queryParams);
+ }
+
+ @Test
+ public void testAddLimitOffsetBetweenBigQueryPushDownByParamsAndMaxResultRows() {
+ KylinConfig config = KylinConfig.createKylinConfig(new Properties());
+ // read project config of `kylin.query.max-result-rows`
+ String sql = "select TRANS_ID as test_limit, ORDER_ID as test_offset from TEST_KYLIN_FACT group by TRANS_ID, ORDER_ID";
+ MetadataTestUtils.updateProjectConfig("default", "kylin.query.max-result-rows", "2");
+ QueryParams queryParams = new QueryParams(config, sql, "default", 0, 0, "DEFAULT", true);
+ String targetSQL = QueryUtil.massageSql(queryParams);
Assert.assertEquals(
"select TRANS_ID as test_limit, ORDER_ID as test_offset from TEST_KYLIN_FACT group by TRANS_ID, ORDER_ID\n"
+ "LIMIT 2",
targetSQL);
+
+ // read project config of `kylin.query.max-result-rows=2` but front-end limit has a high priority
queryParams = new QueryParams(config, sql, "default", 1, 0, "DEFAULT", true);
targetSQL = QueryUtil.massageSql(queryParams);
Assert.assertEquals(
"select TRANS_ID as test_limit, ORDER_ID as test_offset from TEST_KYLIN_FACT group by TRANS_ID, ORDER_ID\n"
+ "LIMIT 1",
targetSQL);
- config.setProperty("kylin.query.max-result-rows", "-1");
- config.setProperty("kylin.query.force-limit", "3");
- queryParams = new QueryParams(config, sql, "default", 0, 0, "DEFAULT", true);
- targetSQL = QueryUtil.massageSql(queryParams);
- Assert.assertEquals(
- "select TRANS_ID as test_limit, ORDER_ID as test_offset from TEST_KYLIN_FACT group by TRANS_ID, ORDER_ID",
- targetSQL);
- queryParams = new QueryParams(config, sql, "default", 1, 0, "DEFAULT", true);
- targetSQL = QueryUtil.massageSql(queryParams);
- Assert.assertEquals(
- "select TRANS_ID as test_limit, ORDER_ID as test_offset from TEST_KYLIN_FACT group by TRANS_ID, ORDER_ID\n"
- + "LIMIT 1",
- targetSQL);
- sql1 = "select * from table1";
- queryParams = new QueryParams(config, sql1, "default", 0, 0, "DEFAULT", true);
- targetSQL = QueryUtil.massageSql(queryParams);
- Assert.assertEquals("select * from table1" + "\n" + "LIMIT 3", targetSQL);
- queryParams = new QueryParams(config, sql1, "default", 2, 0, "DEFAULT", true);
- targetSQL = QueryUtil.massageSql(queryParams);
- Assert.assertEquals("select * from table1" + "\n" + "LIMIT 2", targetSQL);
- sql1 = "select * from table1 limit 4";
- queryParams = new QueryParams(config, sql1, "default", 0, 0, "DEFAULT", true);
- targetSQL = QueryUtil.massageSql(queryParams);
- Assert.assertEquals("select * from table1 limit 4", targetSQL);
- queryParams = new QueryParams(config, sql1, "default", 2, 0, "DEFAULT", true);
- targetSQL = QueryUtil.massageSql(queryParams);
- Assert.assertEquals("select * from table1 limit 4", targetSQL);
- config.setProperty("kylin.query.force-limit", "-1");
- config.setProperty("kylin.query.share-state-switch-implement", "jdbc");
- queryParams = new QueryParams(config, sql, "default", 0, 0, "DEFAULT", true);
- targetSQL = QueryUtil.massageSql(queryParams);
- Assert.assertEquals(
- "select TRANS_ID as test_limit, ORDER_ID as test_offset from TEST_KYLIN_FACT group by TRANS_ID, ORDER_ID",
- targetSQL);
- config.setProperty("kylin.query.big-query-pushdown", "true");
- queryParams = new QueryParams(config, sql, "default", 0, 0, "DEFAULT", true);
- targetSQL = QueryUtil.massageSql(queryParams);
- Assert.assertEquals(
- "select TRANS_ID as test_limit, ORDER_ID as test_offset from TEST_KYLIN_FACT group by TRANS_ID, ORDER_ID",
- targetSQL);
}
@Test
- public void testReplaceDoubleQuoteToSingle() {
- String sql = "select ab from table where aa = '' and bb = '''as''n'''";
- String resSql = "select ab from table where aa = '' and bb = '\\'as\\'n\\''";
- Assert.assertEquals(resSql, QueryUtil.replaceDoubleQuoteToSingle(sql));
+ public void testAddLimitOffsetBetweenBigQueryPushDownWithForceLimit() {
+ KylinConfig config = KylinConfig.createKylinConfig(new Properties());
+ String sql = "select TRANS_ID as test_limit, ORDER_ID as test_offset from TEST_KYLIN_FACT group by TRANS_ID, ORDER_ID";
+ // compare the priority of two properties, the `kylin.query.max-result-rows` has higher priority if it is bigger than 0
+ {
+ Map<String, String> map = Maps.newHashMap();
+ map.put("kylin.query.max-result-rows", "-1");
+ map.put("kylin.query.force-limit", "3");
+ MetadataTestUtils.updateProjectConfig("default", map);
+ QueryParams queryParams = new QueryParams(config, sql, "default", 0, 0, "DEFAULT", true);
+ String targetSQL = QueryUtil.massageSql(queryParams);
+ Assert.assertEquals(
+ "select TRANS_ID as test_limit, ORDER_ID as test_offset from TEST_KYLIN_FACT group by TRANS_ID, ORDER_ID",
+ targetSQL);
+
+ // the front-end param has a higher priority
+ queryParams = new QueryParams(config, sql, "default", 1, 0, "DEFAULT", true);
+ targetSQL = QueryUtil.massageSql(queryParams);
+ Assert.assertEquals(
+ "select TRANS_ID as test_limit, ORDER_ID as test_offset from TEST_KYLIN_FACT group by TRANS_ID, ORDER_ID\n"
+ + "LIMIT 1",
+ targetSQL);
+
+ String sql1 = "select * from table1";
+ queryParams = new QueryParams(config, sql1, "default", 0, 0, "DEFAULT", true);
+ targetSQL = QueryUtil.massageSql(queryParams);
+ Assert.assertEquals("select * from table1" + "\n" + "LIMIT 3", targetSQL);
+ queryParams = new QueryParams(config, sql1, "default", 2, 0, "DEFAULT", true);
+ targetSQL = QueryUtil.massageSql(queryParams);
+ Assert.assertEquals("select * from table1" + "\n" + "LIMIT 2", targetSQL);
+ sql1 = "select * from table1 limit 4";
+ queryParams = new QueryParams(config, sql1, "default", 0, 0, "DEFAULT", true);
+ targetSQL = QueryUtil.massageSql(queryParams);
+ Assert.assertEquals("select * from table1 limit 4", targetSQL);
+ queryParams = new QueryParams(config, sql1, "default", 2, 0, "DEFAULT", true);
+ targetSQL = QueryUtil.massageSql(queryParams);
+ Assert.assertEquals("select * from table1 limit 4", targetSQL);
+ }
+
+ {
+ Map<String, String> map = Maps.newHashMap();
+ map.put("kylin.query.force-limit", "-1");
+ map.put("kylin.query.share-state-switch-implement", "jdbc");
+ MetadataTestUtils.updateProjectConfig("default", map);
+ QueryParams queryParams = new QueryParams(config, sql, "default", 0, 0, "DEFAULT", true);
+ String targetSQL = QueryUtil.massageSql(queryParams);
+ Assert.assertEquals(
+ "select TRANS_ID as test_limit, ORDER_ID as test_offset from TEST_KYLIN_FACT group by TRANS_ID, ORDER_ID",
+ targetSQL);
+
+ // add one more property `kylin.query.big-query-pushdown`
+ MetadataTestUtils.updateProjectConfig("default", "kylin.query.big-query-pushdown", "true");
+ queryParams = new QueryParams(config, sql, "default", 0, 0, "DEFAULT", true);
+ targetSQL = QueryUtil.massageSql(queryParams);
+ Assert.assertEquals(
+ "select TRANS_ID as test_limit, ORDER_ID as test_offset from TEST_KYLIN_FACT group by TRANS_ID, ORDER_ID",
+ targetSQL);
+ }
}
}
diff --git a/src/second-storage/clickhouse-it/src/test/java/io/kyligence/kap/secondstorage/tdvt/TDVTHiveTest.java b/src/second-storage/clickhouse-it/src/test/java/io/kyligence/kap/secondstorage/tdvt/TDVTHiveTest.java
index 124c76b4fc..a80d39ed5f 100644
--- a/src/second-storage/clickhouse-it/src/test/java/io/kyligence/kap/secondstorage/tdvt/TDVTHiveTest.java
+++ b/src/second-storage/clickhouse-it/src/test/java/io/kyligence/kap/secondstorage/tdvt/TDVTHiveTest.java
@@ -38,8 +38,8 @@ import org.apache.kylin.common.util.Unsafe;
import org.apache.kylin.engine.spark.IndexDataConstructor;
import org.apache.kylin.metadata.model.NDataModelManager;
import org.apache.kylin.metadata.project.NProjectManager;
+import org.apache.kylin.query.util.PushDownUtil;
import org.apache.kylin.query.util.QueryParams;
-import org.apache.kylin.query.util.QueryUtil;
import org.apache.kylin.util.ExecAndComp;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
@@ -196,7 +196,7 @@ public class TDVTHiveTest {
private String runWithHive(String sqlStatement) {
QueryParams queryParams = new QueryParams(project, sqlStatement, "default", false);
queryParams.setKylinConfig(NProjectManager.getProjectConfig(project));
- String afterConvert = QueryUtil.massagePushDownSql(queryParams);
+ String afterConvert = PushDownUtil.massagePushDownSql(queryParams);
// Table schema comes from csv and DATABASE.TABLE is not supported.
String sqlForSpark = ExecAndComp.removeDataBaseInSql(afterConvert);
Dataset<Row> plan = ExecAndComp.querySparkSql(sqlForSpark);
diff --git a/src/second-storage/core-ui/src/test/java/org/apache/kylin/rest/service/ModelServiceWithSecondStorageTest.java b/src/second-storage/core-ui/src/test/java/org/apache/kylin/rest/service/ModelServiceWithSecondStorageTest.java
index 9ef91ef98d..afe9add1cd 100644
--- a/src/second-storage/core-ui/src/test/java/org/apache/kylin/rest/service/ModelServiceWithSecondStorageTest.java
+++ b/src/second-storage/core-ui/src/test/java/org/apache/kylin/rest/service/ModelServiceWithSecondStorageTest.java
@@ -39,6 +39,7 @@ import org.apache.kylin.metadata.project.NProjectManager;
import org.apache.kylin.metadata.query.QueryTimesResponse;
import org.apache.kylin.metadata.recommendation.candidate.JdbcRawRecStore;
import org.apache.kylin.query.util.QueryUtil;
+import org.apache.kylin.query.util.PushDownUtil;
import org.apache.kylin.rest.config.initialize.ModelBrokenListener;
import org.apache.kylin.rest.constant.Constant;
import org.apache.kylin.rest.request.ModelRequest;
@@ -141,7 +142,7 @@ public class ModelServiceWithSecondStorageTest extends NLocalFileMetadataTestCas
ReflectionTestUtils.setField(semanticService, "userGroupService", userGroupService);
ReflectionTestUtils.setField(semanticService, "expandableMeasureUtil",
new ExpandableMeasureUtil((model, ccDesc) -> {
- String ccExpression = QueryUtil.massageComputedColumn(model, model.getProject(), ccDesc,
+ String ccExpression = PushDownUtil.massageComputedColumn(model, model.getProject(), ccDesc,
AclPermissionUtil.createAclInfo(model.getProject(),
semanticService.getCurrentUserGroups()));
ccDesc.setInnerExpression(ccExpression);
diff --git a/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/application/SparkApplication.java b/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/application/SparkApplication.java
index db9c31c9b0..cb395e88ba 100644
--- a/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/application/SparkApplication.java
+++ b/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/application/SparkApplication.java
@@ -80,7 +80,6 @@ import org.apache.kylin.metadata.view.LogicalView;
import org.apache.kylin.metadata.view.LogicalViewManager;
import org.apache.kylin.query.pushdown.SparkSubmitter;
import org.apache.kylin.query.util.PushDownUtil;
-
import org.apache.spark.SparkConf;
import org.apache.spark.SparkException;
import org.apache.spark.application.NoRetryException;
@@ -97,7 +96,6 @@ import org.apache.spark.sql.catalyst.rules.Rule;
import org.apache.spark.sql.execution.datasource.AlignmentTableStats;
import org.apache.spark.sql.hive.utils.ResourceDetectUtils;
import org.apache.spark.util.Utils;
-
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -106,7 +104,6 @@ import com.google.common.collect.Maps;
import org.apache.kylin.engine.spark.job.SegmentBuildJob;
import lombok.val;
-
import scala.runtime.AbstractFunction1;
import scala.runtime.BoxedUnit;
@@ -360,7 +357,7 @@ public abstract class SparkApplication implements Application {
}
// Permission exception will not be retried. Simply let the job fail.
- protected void interceptAccessControlException(Throwable e) throws NoRetryException{
+ protected void interceptAccessControlException(Throwable e) throws NoRetryException {
logger.error("Permission denied.", e);
throw new NoRetryException("Permission denied.");
}
@@ -534,28 +531,27 @@ public abstract class SparkApplication implements Application {
return;
}
val modelManager = NDataModelManager.getInstance(config, project);
- NDataModel modelDesc = modelManager.getDataModelDesc(modelId);
- if (checkRangePartitionTableIsExist(modelDesc)) {
+ NDataModel model = modelManager.getDataModelDesc(modelId);
+ if (checkRangePartitionTableIsExist(model)) {
logger.info("Range partitioned tables do not support pushdown, so do not need to perform subsequent logic");
return;
}
- val partitionDesc = modelDesc.getPartitionDesc();
+ val partitionDesc = model.getPartitionDesc();
if (PartitionDesc.isEmptyPartitionDesc(partitionDesc)
- || org.apache.commons.lang.StringUtils.isEmpty(partitionDesc.getPartitionDateFormat()))
+ || StringUtils.isEmpty(partitionDesc.getPartitionDateFormat()))
return;
- if (CatalogTableType.VIEW().name().equals(modelDesc.getRootFactTable().getTableDesc().getTableType()))
+ if (CatalogTableType.VIEW().name().equals(model.getRootFactTable().getTableDesc().getTableType()))
return;
- String partitionColumn = modelDesc.getPartitionDesc().getPartitionDateColumnRef().getExpressionInSourceDB();
+ String partitionColumn = model.getPartitionDesc().getPartitionDateColumnRef().getBackTickExp();
SparkSession sparkSession = atomicSparkSession.get();
try (SparkSubmitter.OverriddenSparkSession ignored = SparkSubmitter.getInstance()
.overrideSparkSession(sparkSession)) {
- String dateString = PushDownUtil.getFormatIfNotExist(modelDesc.getRootFactTableName(), partitionColumn,
- project);
- val sdf = new SimpleDateFormat(modelDesc.getPartitionDesc().getPartitionDateFormat(),
+ String dateString = PushDownUtil.probeColFormat(model.getRootFactTableName(), partitionColumn, project);
+ val sdf = new SimpleDateFormat(model.getPartitionDesc().getPartitionDateFormat(),
Locale.getDefault(Locale.Category.FORMAT));
val date = sdf.parse(dateString);
if (date == null || !dateString.equals(sdf.format(date))) {
@@ -665,9 +661,8 @@ public abstract class SparkApplication implements Application {
LogicalViewManager viewManager = LogicalViewManager.getInstance(config);
if (StringUtils.isNotBlank(dataflowId)) {
- viewManager
- .findLogicalViewsInModel(project, dataflowId)
- .forEach(view -> LogicalViewLoader.loadView(view.getTableName(), true, ss));
+ viewManager.findLogicalViewsInModel(project, dataflowId)
+ .forEach(view -> LogicalViewLoader.loadView(view.getTableName(), true, ss));
}
if (StringUtils.isNotBlank(tableName)) {
LogicalView view = viewManager.findLogicalViewInProject(getProject(), tableName);
diff --git a/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/job/NSparkExecutable.java b/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/job/NSparkExecutable.java
index 4370252206..a72b7c070d 100644
--- a/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/job/NSparkExecutable.java
+++ b/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/job/NSparkExecutable.java
@@ -53,7 +53,7 @@ import org.apache.kylin.common.persistence.transaction.UnitOfWorkParams;
import org.apache.kylin.common.util.ClassUtil;
import org.apache.kylin.common.util.HadoopUtil;
import org.apache.kylin.common.util.JsonUtil;
-import org.apache.kylin.common.util.StringUtil;
+import org.apache.kylin.common.util.StringHelper;
import org.apache.kylin.engine.spark.merger.MetadataMerger;
import org.apache.kylin.job.exception.ExecuteException;
import org.apache.kylin.job.exception.JobStoppedException;
@@ -660,7 +660,7 @@ public class NSparkExecutable extends AbstractExecutable implements ChainedStage
}
// when layout ids not null, set index count
if (StringUtils.isNotBlank(getParam(NBatchConstants.P_LAYOUT_IDS))) {
- int indexCount = StringUtil.splitAndTrim(getParam(NBatchConstants.P_LAYOUT_IDS), ",").length;
+ int indexCount = StringHelper.splitAndTrim(getParam(NBatchConstants.P_LAYOUT_IDS), ",").length;
setParam(NBatchConstants.P_INDEX_COUNT, String.valueOf(indexCount));
}
}
diff --git a/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/utils/ComputedColumnEvalUtil.java b/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/utils/ComputedColumnEvalUtil.java
index ecb0852c4d..03ef1ba57d 100644
--- a/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/utils/ComputedColumnEvalUtil.java
+++ b/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/utils/ComputedColumnEvalUtil.java
@@ -24,31 +24,22 @@ import java.util.Locale;
import org.apache.kylin.common.exception.QueryErrorCode;
import org.apache.kylin.common.msg.MsgPicker;
-import org.apache.kylin.metadata.model.ColumnDesc;
-import org.apache.kylin.metadata.model.JoinTableDesc;
-import org.apache.kylin.metadata.model.TableRef;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.engine.spark.builder.CreateFlatTable;
import org.apache.kylin.engine.spark.job.NSparkCubingUtil;
+import org.apache.kylin.engine.spark.smarter.IndexDependencyParser;
import org.apache.kylin.metadata.model.BadModelException;
import org.apache.kylin.metadata.model.ComputedColumnDesc;
import org.apache.kylin.metadata.model.NDataModel;
import org.apache.kylin.metadata.model.exception.IllegalCCExpressionException;
import org.apache.kylin.metadata.model.util.ComputedColumnUtil;
-import org.apache.spark.sql.AnalysisException;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SparderEnv;
import org.apache.spark.sql.SparkSession;
-import org.apache.spark.sql.execution.utils.SchemaProcessor;
import org.apache.spark.sql.util.SparderTypeUtil;
import org.springframework.util.CollectionUtils;
import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import lombok.val;
import lombok.extern.slf4j.Slf4j;
@Slf4j
@@ -90,7 +81,7 @@ public class ComputedColumnEvalUtil {
}
try {
evalDataTypeOfCC(computedColumns, SparderEnv.getSparkSession(), nDataModel, 0, computedColumns.size());
- } catch (AnalysisException e) {
+ } catch (Exception e) {
evalDataTypeOfCCInManual(computedColumns, nDataModel, 0, computedColumns.size());
}
}
@@ -100,7 +91,7 @@ public class ComputedColumnEvalUtil {
for (int i = start; i < end; i++) {
try {
evalDataTypeOfCC(computedColumns, SparderEnv.getSparkSession(), nDataModel, i, i + 1);
- } catch (AnalysisException e) {
+ } catch (Exception e) {
Preconditions.checkNotNull(computedColumns.get(i));
throw new IllegalCCExpressionException(QueryErrorCode.CC_EXPRESSION_ILLEGAL,
String.format(Locale.ROOT, MsgPicker.getMsg().getCheckCCExpression(),
@@ -111,42 +102,23 @@ public class ComputedColumnEvalUtil {
}
private static void evalDataTypeOfCC(List<ComputedColumnDesc> computedColumns, SparkSession ss,
- NDataModel nDataModel, int start, int end) throws AnalysisException {
- val originDf = generateFullFlatTableDF(ss, nDataModel);
+ NDataModel nDataModel, int start, int end) {
+ IndexDependencyParser parser = new IndexDependencyParser(nDataModel);
+ Dataset<Row> originDf = parser.generateFullFlatTableDF(ss, nDataModel);
originDf.persist();
- Dataset<Row> ds = originDf
- .selectExpr(computedColumns.subList(start, end).stream().map(ComputedColumnDesc::getInnerExpression)
- .map(NSparkCubingUtil::convertFromDotWithBackTick).toArray(String[]::new));
+ String[] ccExprArray = computedColumns.subList(start, end).stream() //
+ .map(ComputedColumnDesc::getInnerExpression) //
+ .map(NSparkCubingUtil::convertFromDotWithBackTick).toArray(String[]::new);
+ Dataset<Row> ds = originDf.selectExpr(ccExprArray);
for (int i = start; i < end; i++) {
String dataType = SparderTypeUtil.convertSparkTypeToSqlType(ds.schema().fields()[i - start].dataType());
computedColumns.get(i).setDatatype(dataType);
}
}
- private static Dataset<Row> generateFullFlatTableDF(SparkSession ss, NDataModel model) {
- // root fact table
- val rootDF = generateDatasetOnTable(ss, model.getRootFactTable());
-
- // look up tables
- val joinTableDFMap = Maps.<JoinTableDesc, Dataset<Row>> newLinkedHashMap();
- model.getJoinTables().forEach(
- joinTable -> joinTableDFMap.put(joinTable, generateDatasetOnTable(ss, joinTable.getTableRef())));
-
- return CreateFlatTable.joinFactTableWithLookupTables(rootDF, joinTableDFMap, model, ss);
- }
-
- private static Dataset<Row> generateDatasetOnTable(SparkSession ss, TableRef tableRef) {
- val tableCols = tableRef.getColumns().stream().map(TblColRef::getColumnDesc)
- .filter(col -> !col.isComputedColumn()).toArray(ColumnDesc[]::new);
- val structType = SchemaProcessor.buildSchemaWithRawTable(tableCols);
- val alias = tableRef.getAlias();
- val dataset = ss.createDataFrame(Lists.newArrayList(), structType).alias(alias);
- return CreateFlatTable.changeSchemaToAliasDotName(dataset, alias);
- }
-
public static boolean resolveCCName(ComputedColumnDesc ccDesc, NDataModel dataModel, List<NDataModel> otherModels) {
- // Resolve CC name, Limit MAX_RENAME_CC_TIME retries to avoid infinite loop
- // TODO: what if the dataModel has more than MAX_RENAME_CC_TIME computed columns?
+ // Resolve CC name, Limit MAX_RENAME_CC_TIME retries to avoid infinite loop.
+ // What if the dataModel has more than MAX_RENAME_CC_TIME computed columns?
int retryCount = 0;
while (retryCount < MAX_RENAME_CC_TIME) {
retryCount++;
diff --git a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/builder/CreateFlatTable.scala b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/builder/CreateFlatTable.scala
index fb91ce7933..fdb4d87cd2 100644
--- a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/builder/CreateFlatTable.scala
+++ b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/builder/CreateFlatTable.scala
@@ -112,9 +112,9 @@ class CreateFlatTable(val flatTable: IJoinedFlatTableDesc,
}
protected def encodeWithCols(ds: Dataset[Row],
- ccCols: Set[TblColRef],
- dictCols: Set[TblColRef],
- encodeCols: Set[TblColRef]): Dataset[Row] = {
+ ccCols: Set[TblColRef],
+ dictCols: Set[TblColRef],
+ encodeCols: Set[TblColRef]): Dataset[Row] = {
val ccDataset = withColumn(ds, ccCols)
if (seg.isDictReady) {
logInfo(s"Skip already built dict, segment: ${seg.getId} of dataflow: ${seg.getDataflow.getId}")
@@ -128,7 +128,7 @@ class CreateFlatTable(val flatTable: IJoinedFlatTableDesc,
val matchedCols = selectColumnsInTable(ds, withCols)
var withDs = ds
matchedCols.foreach(m => withDs = withDs.withColumn(convertFromDot(m.getBackTickIdentity),
- expr(convertFromDot(m.getBackTickExpressionInSourceDB))))
+ expr(convertFromDot(m.getBackTickExp))))
withDs
}
@@ -189,8 +189,8 @@ object CreateFlatTable extends LogEx {
}
def generateLookupTableDataset(model: NDataModel,
- cols: Seq[TblColRef],
- ss: SparkSession): mutable.LinkedHashMap[JoinTableDesc, Dataset[Row]] = {
+ cols: Seq[TblColRef],
+ ss: SparkSession): mutable.LinkedHashMap[JoinTableDesc, Dataset[Row]] = {
val lookupTables = mutable.LinkedHashMap[JoinTableDesc, Dataset[Row]]()
model.getJoinTables.asScala.map(
joinDesc => {
diff --git a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/builder/DFBuilderHelper.scala b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/builder/DFBuilderHelper.scala
index 209361f089..e58a803c62 100644
--- a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/builder/DFBuilderHelper.scala
+++ b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/builder/DFBuilderHelper.scala
@@ -18,10 +18,10 @@
package org.apache.kylin.engine.spark.builder
+import org.apache.kylin.common.KylinConfig
import org.apache.kylin.common.persistence.transaction.UnitOfWork
-import org.apache.kylin.engine.spark.job.NSparkCubingUtil._
+import org.apache.kylin.engine.spark.job.NSparkCubingUtil
import org.apache.kylin.metadata.cube.model.{NDataSegment, NDataflowManager, NDataflowUpdate}
-import org.apache.kylin.common.KylinConfig
import org.apache.kylin.metadata.model.TblColRef
import org.apache.spark.internal.Logging
import org.apache.spark.sql.functions.expr
@@ -51,7 +51,7 @@ object DFBuilderHelper extends Logging {
def selectColumnsInTable(table: Dataset[Row], columns: Set[TblColRef]): Set[TblColRef] = {
columns.filter(col =>
- isColumnInTable(convertFromDot(col.getBackTickExpressionInSourceDB), table))
+ isColumnInTable(NSparkCubingUtil.convertFromDot(col.getBackTickExp), table))
}
// ============================= Used by {@link DFBuildJob}.Functions are deprecated. ========================= //
@@ -63,7 +63,7 @@ object DFBuilderHelper extends Logging {
@deprecated
def filterOutIntegerFamilyType(table: Dataset[Row], columns: Set[TblColRef]): Set[TblColRef] = {
columns.filterNot(_.getType.isIntegerFamily).filter(cc =>
- isColumnInTable(convertFromDot(cc.getBackTickExpressionInSourceDB), table))
+ isColumnInTable(NSparkCubingUtil.convertFromDot(cc.getBackTickExp), table))
}
def isColumnInTable(colExpr: String, table: Dataset[Row]): Boolean = {
@@ -78,8 +78,8 @@ object DFBuilderHelper extends Logging {
def chooseSuitableCols(ds: Dataset[Row], needCheckCols: Iterable[TblColRef]): Seq[Column] = {
needCheckCols
.filter(ref => isColumnInTable(ref.getExpressionInSourceDB, ds))
- .map(ref => expr(convertFromDotWithBackTick(ref.getBackTickExpressionInSourceDB))
- .alias(convertFromDot(ref.getBackTickIdentity)))
+ .map(ref => expr(NSparkCubingUtil.convertFromDotWithBackTick(ref.getBackTickExp))
+ .alias(NSparkCubingUtil.convertFromDot(ref.getBackTickIdentity)))
.toSeq
}
diff --git a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/builder/SegmentFlatTable.scala b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/builder/SegmentFlatTable.scala
index 931570c6c9..7449aebdcd 100644
--- a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/builder/SegmentFlatTable.scala
+++ b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/builder/SegmentFlatTable.scala
@@ -18,8 +18,12 @@
package org.apache.kylin.engine.spark.builder
-import com.google.common.collect.Sets
+import java.util.concurrent.{CountDownLatch, TimeUnit}
+import java.util.{Locale, Objects, Timer, TimerTask}
+
import org.apache.commons.lang3.StringUtils
+import org.apache.kylin.common.constant.LogConstant
+import org.apache.kylin.common.logging.SetLogCategory
import org.apache.kylin.common.util.HadoopUtil
import org.apache.kylin.common.{CustomUtils, KapConfig, KylinConfig}
import org.apache.kylin.engine.spark.builder.DFBuilderHelper._
@@ -30,18 +34,12 @@ import org.apache.kylin.engine.spark.utils.LogEx
import org.apache.kylin.engine.spark.utils.SparkDataSource._
import org.apache.kylin.metadata.cube.model.NDataSegment
import org.apache.kylin.metadata.model._
-import org.apache.kylin.query.util.QueryUtil
+import org.apache.kylin.query.util.PushDownUtil
import org.apache.spark.sql._
import org.apache.spark.sql.functions.{col, expr}
import org.apache.spark.sql.types.StructField
import org.apache.spark.sql.util.SparderTypeUtil
import org.apache.spark.utils.ProxyThreadUtils
-import java.util.concurrent.{CountDownLatch, TimeUnit}
-import java.util.{Locale, Objects, Timer, TimerTask}
-
-import org.apache.kylin.common.constant.LogConstant
-import org.apache.kylin.common.logging.SetLogCategory
-import org.apache.spark.util.Utils
import scala.collection.JavaConverters._
import scala.collection.mutable
@@ -50,6 +48,8 @@ import scala.concurrent.duration.{Duration, MILLISECONDS}
import scala.concurrent.forkjoin.ForkJoinPool
import scala.util.{Failure, Success, Try}
+import com.google.common.collect.Sets
+
class SegmentFlatTable(private val sparkSession: SparkSession, //
private val tableDesc: SegmentFlatTableDesc) extends LogEx {
@@ -128,7 +128,7 @@ class SegmentFlatTable(private val sparkSession: SparkSession, //
val jointDS = joinFactTableWithLookupTables(fastFactTableDS, lookupTableDSMap, dataModel, sparkSession)
concatCCs(jointDS, factTableCCs)
} else {
- fastFactTableDS
+ concatCCs(fastFactTableDS, factTableCCs)
}
flatTableDS = applyFilterCondition(flatTableDS)
changeSchemeToColumnId(flatTableDS, tableDesc)
@@ -258,8 +258,7 @@ class SegmentFlatTable(private val sparkSession: SparkSession, //
logInfo(s"No available FILTER-CONDITION segment $segmentId")
return originDS
}
- val expression = QueryUtil.massageExpression(dataModel, project, //
- dataModel.getFilterCondition, null)
+ val expression = PushDownUtil.massageExpression(dataModel, project, dataModel.getFilterCondition, null)
val converted = replaceDot(expression, dataModel)
val condition = s" (1=1) AND ($converted)"
logInfo(s"Apply FILTER-CONDITION: $condition segment $segmentId")
@@ -481,7 +480,7 @@ class SegmentFlatTable(private val sparkSession: SparkSession, //
var tableWithCcs = table
matchedCols.foreach(m =>
tableWithCcs = tableWithCcs.withColumn(convertFromDot(m.getBackTickIdentity),
- expr(convertFromDot(m.getBackTickExpressionInSourceDB))))
+ expr(convertFromDot(m.getBackTickExp))))
tableWithCcs
}
diff --git a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/FlatTableHelper.scala b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/FlatTableHelper.scala
index d9af449653..5964c661a6 100644
--- a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/FlatTableHelper.scala
+++ b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/FlatTableHelper.scala
@@ -21,16 +21,15 @@ package org.apache.kylin.engine.spark.job
import org.apache.commons.lang3.StringUtils
import org.apache.kylin.engine.spark.builder.CreateFlatTable.replaceDot
import org.apache.kylin.metadata.model.IJoinedFlatTableDesc
-import org.apache.kylin.query.util.QueryUtil
+import org.apache.kylin.query.util.PushDownUtil
import org.apache.spark.internal.Logging
import org.apache.spark.sql.{Dataset, Row}
object FlatTableHelper extends Logging {
- def applyPartitionDesc(
- flatTable: IJoinedFlatTableDesc,
- ds: Dataset[Row],
- needReplaceDot: Boolean): Dataset[Row] = {
+ def applyPartitionDesc(flatTable: IJoinedFlatTableDesc,
+ ds: Dataset[Row],
+ needReplaceDot: Boolean): Dataset[Row] = {
var afterFilter = ds
val model = flatTable.getDataModel
@@ -49,16 +48,15 @@ object FlatTableHelper extends Logging {
afterFilter
}
- def applyFilterCondition(
- flatTable: IJoinedFlatTableDesc,
- ds: Dataset[Row],
- needReplaceDot: Boolean): Dataset[Row] = {
+ def applyFilterCondition(flatTable: IJoinedFlatTableDesc,
+ ds: Dataset[Row],
+ needReplaceDot: Boolean): Dataset[Row] = {
var afterFilter = ds
val model = flatTable.getDataModel
if (StringUtils.isNotBlank(model.getFilterCondition)) {
var filterCond = model.getFilterCondition
- filterCond = QueryUtil.massageExpression(model, model.getProject, filterCond, null);
+ filterCond = PushDownUtil.massageExpression(model, model.getProject, filterCond, null);
if (needReplaceDot) filterCond = replaceDot(filterCond, model)
filterCond = s" (1=1) AND (" + filterCond + s")"
logInfo(s"Filter condition is $filterCond")
diff --git a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/FlatTableAndDictBase.scala b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/FlatTableAndDictBase.scala
index 24391d28a7..1409ff8228 100644
--- a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/FlatTableAndDictBase.scala
+++ b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/stage/build/FlatTableAndDictBase.scala
@@ -18,7 +18,9 @@
package org.apache.kylin.engine.spark.job.stage.build
-import com.google.common.collect.Sets
+import java.util.concurrent.{CountDownLatch, TimeUnit}
+import java.util.{Locale, Objects, Timer, TimerTask}
+
import org.apache.commons.lang3.StringUtils
import org.apache.hadoop.fs.Path
import org.apache.kylin.common.util.HadoopUtil
@@ -37,7 +39,7 @@ import org.apache.kylin.engine.spark.utils.SparkDataSource._
import org.apache.kylin.metadata.cube.model.NDataSegment
import org.apache.kylin.metadata.cube.planner.CostBasePlannerUtils
import org.apache.kylin.metadata.model._
-import org.apache.kylin.query.util.QueryUtil
+import org.apache.kylin.query.util.PushDownUtil
import org.apache.spark.sql.KapFunctions.dict_encode_v3
import org.apache.spark.sql._
import org.apache.spark.sql.functions.{col, expr}
@@ -59,6 +61,8 @@ import scala.concurrent.duration.{Duration, MILLISECONDS}
import scala.concurrent.forkjoin.ForkJoinPool
import scala.util.{Failure, Success, Try}
+import com.google.common.collect.Sets
+
abstract class FlatTableAndDictBase(private val jobContext: SegmentJob,
private val dataSegment: NDataSegment,
private val buildParam: BuildParam)
@@ -144,7 +148,7 @@ abstract class FlatTableAndDictBase(private val jobContext: SegmentJob,
val jointDS = joinFactTableWithLookupTables(fastFactTableDS, lookupTableDSMap, dataModel, sparkSession)
concatCCs(jointDS, factTableCCs)
} else {
- fastFactTableDS
+ concatCCs(fastFactTableDS, factTableCCs)
}
flatTableDS = applyFilterCondition(flatTableDS)
changeSchemeToColumnId(flatTableDS, tableDesc)
@@ -317,8 +321,7 @@ abstract class FlatTableAndDictBase(private val jobContext: SegmentJob,
logInfo(s"No available FILTER-CONDITION segment $segmentId")
return originDS
}
- val expression = QueryUtil.massageExpression(dataModel, project, //
- dataModel.getFilterCondition, null)
+ val expression = PushDownUtil.massageExpression(dataModel, project, dataModel.getFilterCondition, null)
val converted = replaceDot(expression, dataModel)
val condition = s" (1=1) AND ($converted)"
logInfo(s"Apply FILTER-CONDITION: $condition segment $segmentId")
@@ -605,7 +608,7 @@ abstract class FlatTableAndDictBase(private val jobContext: SegmentJob,
var tableWithCcs = table
matchedCols.foreach(m =>
tableWithCcs = tableWithCcs.withColumn(convertFromDot(m.getBackTickIdentity),
- expr(convertFromDot(m.getBackTickExpressionInSourceDB))))
+ expr(convertFromDot(m.getBackTickExp))))
tableWithCcs
}
diff --git a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/smarter/IndexDependencyParser.scala b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/smarter/IndexDependencyParser.scala
index b4f57ac47f..e1335978fc 100644
--- a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/smarter/IndexDependencyParser.scala
+++ b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/smarter/IndexDependencyParser.scala
@@ -20,14 +20,13 @@ package org.apache.kylin.engine.spark.smarter
import java.util
import java.util.Collections
-import com.google.common.collect.{Lists, Maps, Sets}
+import org.apache.commons.collections.CollectionUtils
+import org.apache.commons.lang3.StringUtils
import org.apache.kylin.engine.spark.builder.SegmentFlatTable
import org.apache.kylin.engine.spark.job.NSparkCubingUtil
import org.apache.kylin.metadata.cube.model.LayoutEntity
-import org.apache.kylin.metadata.model.NDataModel
-import org.apache.commons.collections.CollectionUtils
-import org.apache.commons.lang3.StringUtils
-import org.apache.kylin.metadata.model.{FunctionDesc, JoinTableDesc, TableRef, TblColRef}
+import org.apache.kylin.metadata.model.{FunctionDesc, JoinTableDesc, NDataModel, TableRef, TblColRef}
+import org.apache.kylin.query.util.PushDownUtil
import org.apache.spark.sql.execution.utils.SchemaProcessor
import org.apache.spark.sql.types.StructField
import org.apache.spark.sql.{Dataset, Row, SparderEnv, SparkSession}
@@ -35,6 +34,8 @@ import org.apache.spark.sql.{Dataset, Row, SparderEnv, SparkSession}
import scala.collection.JavaConverters._
import scala.collection.mutable
+import com.google.common.collect.{Lists, Maps, Sets}
+
class IndexDependencyParser(val model: NDataModel) {
private val ccTableNameAliasMap = Maps.newHashMap[String, util.Set[String]]
@@ -105,7 +106,7 @@ class IndexDependencyParser(val model: NDataModel) {
}
}
- private def generateFullFlatTableDF(ss: SparkSession, model: NDataModel): Dataset[Row] = {
+ def generateFullFlatTableDF(ss: SparkSession, model: NDataModel): Dataset[Row] = {
val rootDF = generateDatasetOnTable(ss, model.getRootFactTable)
// look up tables
val joinTableDFMap = mutable.LinkedHashMap[JoinTableDesc, Dataset[Row]]()
@@ -113,14 +114,16 @@ class IndexDependencyParser(val model: NDataModel) {
joinTableDFMap.put(joinTable, generateDatasetOnTable(ss, joinTable.getTableRef))
})
val df = SegmentFlatTable.joinFactTableWithLookupTables(rootDF, joinTableDFMap, model, ss)
- if (StringUtils.isNotEmpty(model.getFilterCondition)) {
- df.where(NSparkCubingUtil.convertFromDotWithBackTick(model.getFilterCondition))
+ val filterCondition = model.getFilterCondition
+ if (StringUtils.isNotEmpty(filterCondition)) {
+ val massagedCondition = PushDownUtil.massageExpression(model, model.getProject, filterCondition, null)
+ df.where(NSparkCubingUtil.convertFromDotWithBackTick(massagedCondition))
}
df
}
private def generateDatasetOnTable(ss: SparkSession, tableRef: TableRef): Dataset[Row] = {
- val tableCols = tableRef.getColumns.asScala.map(_.getColumnDesc).filter(!_.isComputedColumn).toArray
+ val tableCols = tableRef.getColumns.asScala.map(_.getColumnDesc).toArray
val structType = SchemaProcessor.buildSchemaWithRawTable(tableCols)
val alias = tableRef.getAlias
val dataset = ss.createDataFrame(Lists.newArrayList[Row], structType).alias(alias)
@@ -190,7 +193,7 @@ class IndexDependencyParser(val model: NDataModel) {
}
}
- private def initJoinTableName() {
+ private def initJoinTableName(): Unit = {
if (CollectionUtils.isEmpty(model.getJoinTables)) {
return
}
diff --git a/src/spark-project/sparder/src/main/java/org/apache/kylin/query/pushdown/PushDownRunnerJdbcImpl.java b/src/spark-project/sparder/src/main/java/org/apache/kylin/query/pushdown/PushDownRunnerJdbcImpl.java
index 4b53b720c0..de3a0d8f4d 100644
--- a/src/spark-project/sparder/src/main/java/org/apache/kylin/query/pushdown/PushDownRunnerJdbcImpl.java
+++ b/src/spark-project/sparder/src/main/java/org/apache/kylin/query/pushdown/PushDownRunnerJdbcImpl.java
@@ -52,7 +52,7 @@ public class PushDownRunnerJdbcImpl implements IPushDownRunner {
@Override
public void executeQuery(String query, List<List<String>> results, List<SelectedColumnMeta> columnMetas,
- String project) throws Exception {
+ String project) throws SQLException {
Statement statement = null;
Connection connection = manager.getConnection();
ResultSet resultSet = null;
@@ -83,7 +83,7 @@ public class PushDownRunnerJdbcImpl implements IPushDownRunner {
}
@Override
- public void executeUpdate(String sql, String project) throws Exception {
+ public void executeUpdate(String sql, String project) throws SQLException {
Statement statement = null;
Connection connection = manager.getConnection();
diff --git a/src/spark-project/sparder/src/main/java/org/apache/kylin/query/pushdown/PushDownRunnerSparkImpl.java b/src/spark-project/sparder/src/main/java/org/apache/kylin/query/pushdown/PushDownRunnerSparkImpl.java
index 81887d9c8f..4cf2543e7e 100644
--- a/src/spark-project/sparder/src/main/java/org/apache/kylin/query/pushdown/PushDownRunnerSparkImpl.java
+++ b/src/spark-project/sparder/src/main/java/org/apache/kylin/query/pushdown/PushDownRunnerSparkImpl.java
@@ -18,6 +18,7 @@
package org.apache.kylin.query.pushdown;
+import java.sql.SQLException;
import java.util.LinkedList;
import java.util.List;
@@ -45,7 +46,7 @@ public class PushDownRunnerSparkImpl implements IPushDownRunner {
@Override
public void executeQuery(String query, List<List<String>> results, List<SelectedColumnMeta> columnMetas,
- String project) {
+ String project) throws SQLException {
PushdownResult response = executeQueryToIterator(query, project);
response.getRows().forEach(results::add);
columnMetas.addAll(response.getColumnMetas());
diff --git a/src/spark-project/sparder/src/test/java/org/apache/kylin/query/pushdown/PushDownRunnerSparkImplTest.java b/src/spark-project/sparder/src/test/java/org/apache/kylin/query/pushdown/PushDownRunnerSparkImplTest.java
index 5694f11483..f8a24e3077 100644
--- a/src/spark-project/sparder/src/test/java/org/apache/kylin/query/pushdown/PushDownRunnerSparkImplTest.java
+++ b/src/spark-project/sparder/src/test/java/org/apache/kylin/query/pushdown/PushDownRunnerSparkImplTest.java
@@ -18,6 +18,7 @@
package org.apache.kylin.query.pushdown;
+import java.sql.SQLException;
import java.util.ArrayList;
import java.util.List;
import java.util.UUID;
@@ -99,14 +100,18 @@ public class PushDownRunnerSparkImplTest extends NLocalFileMetadataTestCase {
queries.forEach(q -> {
returnRows.clear();
- pushDownRunnerSpark.executeQuery(q, returnRows, returnColumnMeta, "tpch");
+ try {
+ pushDownRunnerSpark.executeQuery(q, returnRows, returnColumnMeta, "tpch");
+ } catch (SQLException e) {
+ //
+ }
Assert.assertEquals(10, returnRows.size());
});
}
@Test
- public void testPushDownRunnerSpark() {
+ public void testPushDownRunnerSpark() throws SQLException {
PushDownRunnerSparkImpl pushDownRunnerSpark = new PushDownRunnerSparkImpl();
pushDownRunnerSpark.init(null, "tpch");
@@ -122,7 +127,7 @@ public class PushDownRunnerSparkImplTest extends NLocalFileMetadataTestCase {
}
@Test
- public void testPushDownRunnerSparkWithDotColumn() {
+ public void testPushDownRunnerSparkWithDotColumn() throws SQLException {
PushDownRunnerSparkImpl pushDownRunnerSpark = new PushDownRunnerSparkImpl();
pushDownRunnerSpark.init(null, "tpch");
@@ -138,7 +143,7 @@ public class PushDownRunnerSparkImplTest extends NLocalFileMetadataTestCase {
}
@Test
- public void testSelectTwoSameExpr() {
+ public void testSelectTwoSameExpr() throws SQLException {
PushDownRunnerSparkImpl pushDownRunnerSpark = new PushDownRunnerSparkImpl();
pushDownRunnerSpark.init(null, "tpch");
@@ -154,7 +159,7 @@ public class PushDownRunnerSparkImplTest extends NLocalFileMetadataTestCase {
}
@Test
- public void testCaseSensitiveOnAlias() {
+ public void testCaseSensitiveOnAlias() throws SQLException {
PushDownRunnerSparkImpl pushDownRunnerSpark = new PushDownRunnerSparkImpl();
pushDownRunnerSpark.init(null, "tpch");
diff --git a/src/spark-project/spark-common/src/main/scala/org/apache/kylin/engine/spark/job/NSparkCubingUtil.java b/src/spark-project/spark-common/src/main/scala/org/apache/kylin/engine/spark/job/NSparkCubingUtil.java
index c22178e9d7..89a0f316a3 100644
--- a/src/spark-project/spark-common/src/main/scala/org/apache/kylin/engine/spark/job/NSparkCubingUtil.java
+++ b/src/spark-project/spark-common/src/main/scala/org/apache/kylin/engine/spark/job/NSparkCubingUtil.java
@@ -20,7 +20,6 @@ package org.apache.kylin.engine.spark.job;
import java.util.Arrays;
import java.util.LinkedHashSet;
-import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.regex.Matcher;
@@ -38,10 +37,6 @@ import org.apache.kylin.metadata.model.Segments;
import org.apache.spark.sql.Column;
import org.sparkproject.guava.collect.Sets;
-import com.google.common.collect.Maps;
-
-import lombok.val;
-
public class NSparkCubingUtil {
public static final String SEPARATOR = "_0_DOT_0_";
@@ -285,10 +280,4 @@ public class NSparkCubingUtil {
return withoutDot.replace(SEPARATOR, ".");
}
- public static Map<Long, LayoutEntity> toLayoutMap(IndexPlan indexPlan, Set<Long> layoutIds) {
- val layouts = toLayouts(indexPlan, layoutIds).stream().filter(Objects::nonNull).collect(Collectors.toSet());
- Map<Long, LayoutEntity> map = Maps.newHashMap();
- layouts.forEach(layout -> map.put(layout.getId(), layout));
- return map;
- }
}
diff --git a/src/spark-project/spark-it/src/test/scala/io/kyligence/kap/common/SSSource.scala b/src/spark-project/spark-it/src/test/scala/io/kyligence/kap/common/SSSource.scala
new file mode 100644
index 0000000000..f071e36acc
--- /dev/null
+++ b/src/spark-project/spark-it/src/test/scala/io/kyligence/kap/common/SSSource.scala
@@ -0,0 +1,89 @@
+/*
+ * Copyright (C) 2016 Kyligence Inc. All rights reserved.
+ * http://kyligence.io
+ * This software is the confidential and proprietary information of
+ * Kyligence Inc. ("Confidential Information"). You shall not disclose
+ * such Confidential Information and shall use it only in accordance
+ * with the terms of the license agreement you entered into with
+ * Kyligence Inc.
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ *
+ */
+
+package io.kyligence.kap.common
+
+import java.util.Locale
+
+import org.apache.kylin.common.KylinConfig
+import org.apache.kylin.common.util.TempMetadataBuilder
+import org.apache.kylin.metadata.model.NTableMetadataManager
+import org.apache.kylin.metadata.project.NProjectManager
+import org.apache.kylin.query.util.{PushDownUtil, QueryParams}
+import org.apache.spark.sql.common.{LocalMetadata, SharedSparkSession}
+import org.apache.spark.sql.execution.utils.SchemaProcessor
+import org.scalatest.Suite
+
+import com.google.common.base.Preconditions
+
+trait SSSource extends SharedSparkSession with LocalMetadata {
+ self: Suite =>
+
+ val CSV_TABLE_DIR: String = "../" + TempMetadataBuilder.TEMP_TEST_METADATA + "/data/%s.csv"
+
+ override def beforeAll(): Unit = {
+ super.beforeAll()
+ val project = getProject
+ import org.apache.kylin.metadata.project.NProjectManager
+ val kylinConf = KylinConfig.getInstanceFromEnv
+ val projectInstance =
+ NProjectManager.getInstance(kylinConf).getProject(project)
+ Preconditions.checkArgument(projectInstance != null)
+ import scala.collection.JavaConverters._
+ projectInstance.getTables.asScala
+ .filter(!_.equals("DEFAULT.STREAMING_TABLE"))
+ .foreach { table =>
+ val tableDesc = NTableMetadataManager
+ .getInstance(kylinConf, project)
+ .getTableDesc(table)
+ val columns = tableDesc.getColumns
+ val schema = SchemaProcessor.buildSchemaWithRawTable(columns)
+ var tableN = tableDesc.getName
+ if (table.equals("DEFAULT.TEST_KYLIN_FACT")) {
+ tableN = tableDesc.getName + "_table"
+ }
+ spark.catalog.createTable(
+ tableName = tableN,
+ source = "csv",
+ schema = schema,
+ options = Map("path" -> String.format(Locale.ROOT, CSV_TABLE_DIR, table)))
+ if (table.equals("DEFAULT.TEST_KYLIN_FACT")) {
+ spark.sql("create view " + tableDesc.getName + " as select * from " + tableN)
+ }
+ }
+ }
+
+ protected def getProject: String = "default"
+
+ def cleanSql(originSql: String): String = {
+ val sqlForSpark = originSql
+ .replaceAll("edw\\.", "")
+ .replaceAll("\"EDW\"\\.", "")
+ .replaceAll("EDW\\.", "")
+ .replaceAll("default\\.", "")
+ .replaceAll("DEFAULT\\.", "")
+ .replaceAll("\"DEFAULT\"\\.", "")
+ val queryParams = new QueryParams("default", sqlForSpark, "DEFAULT", false)
+ queryParams.setKylinConfig(NProjectManager.getProjectConfig("default"))
+ PushDownUtil.massagePushDownSql(queryParams)
+ }
+}
diff --git a/src/streaming/src/main/scala/org/apache/kylin/streaming/CreateStreamingFlatTable.scala b/src/streaming/src/main/scala/org/apache/kylin/streaming/CreateStreamingFlatTable.scala
index 13fb54a288..909f39957e 100644
--- a/src/streaming/src/main/scala/org/apache/kylin/streaming/CreateStreamingFlatTable.scala
+++ b/src/streaming/src/main/scala/org/apache/kylin/streaming/CreateStreamingFlatTable.scala
@@ -18,9 +18,9 @@
package org.apache.kylin.streaming
-import com.google.common.base.Preconditions
+import java.nio.ByteBuffer
+
import org.apache.commons.lang3.StringUtils
-import org.apache.kafka.common.config.SaslConfigs
import org.apache.kylin.common.KylinConfig
import org.apache.kylin.engine.spark.NSparkCubingEngine
import org.apache.kylin.engine.spark.builder.CreateFlatTable
@@ -31,17 +31,17 @@ import org.apache.kylin.metadata.model._
import org.apache.kylin.parser.AbstractDataParser
import org.apache.kylin.source.SourceFactory
import org.apache.kylin.streaming.common.CreateFlatTableEntry
-import org.apache.kylin.streaming.jobs.StreamingJobUtils
import org.apache.spark.sql.catalyst.encoders.RowEncoder
import org.apache.spark.sql.types._
import org.apache.spark.sql.util.SparderTypeUtil
import org.apache.spark.sql.{DataFrame, Dataset, Row}
import org.apache.spark.storage.StorageLevel
-import java.nio.ByteBuffer
import scala.collection.JavaConverters._
import scala.collection.mutable
+import com.google.common.base.Preconditions
+
class CreateStreamingFlatTable(entry: CreateFlatTableEntry) extends
CreateFlatTable(entry.flatTable, entry.seg, entry.toBuildTree, entry.ss, entry.sourceInfo) {
@@ -63,12 +63,12 @@ class CreateStreamingFlatTable(entry: CreateFlatTableEntry) extends
val kafkaJobParams = config.getStreamingKafkaConfigOverride.asScala
val securityProtocol = kafkaJobParams.get(SECURITY_PROTOCOL)
if (securityProtocol.isDefined) {
- kafkaJobParams.remove(SECURITY_PROTOCOL);
+ kafkaJobParams.remove(SECURITY_PROTOCOL)
kafkaJobParams.put("kafka." + SECURITY_PROTOCOL, securityProtocol.get)
}
val saslMechanism = kafkaJobParams.get(SASL_MECHANISM)
if (saslMechanism.isDefined) {
- kafkaJobParams.remove(SASL_MECHANISM);
+ kafkaJobParams.remove(SASL_MECHANISM)
kafkaJobParams.put("kafka." + SASL_MECHANISM, saslMechanism.get)
}
kafkaJobParams.foreach { param =>
@@ -91,7 +91,7 @@ class CreateStreamingFlatTable(entry: CreateFlatTableEntry) extends
val schema =
StructType(
tableDesc.getColumns.map { columnDescs =>
- StructField(columnDescs.getName, SparderTypeUtil.toSparkType(columnDescs.getType, false))
+ StructField(columnDescs.getName, SparderTypeUtil.toSparkType(columnDescs.getType))
}
)
val rootFactTable = changeSchemaToAliasDotName(
@@ -104,11 +104,11 @@ class CreateStreamingFlatTable(entry: CreateFlatTableEntry) extends
val cols = model.getRootFactTable.getColumns.asScala.map(item => {
col(NSparkCubingUtil.convertFromDot(item.getBackTickIdentity))
}).toList
- rootFactTable.withWatermark(partitionColumn, entry.watermark).groupBy(cols: _*).count()
+ rootFactTable.withWatermark(partitionColumn(), entry.watermark).groupBy(cols: _*).count()
} else {
rootFactTable
}
- tableRefreshInterval = StreamingUtils.parseTableRefreshInterval(config.getStreamingTableRefreshInterval())
+ tableRefreshInterval = StreamingUtils.parseTableRefreshInterval(config.getStreamingTableRefreshInterval)
loadLookupTables()
joinFactTableWithLookupTables(factTableDataset, lookupTablesGlobal, model, ss)
}
@@ -116,7 +116,7 @@ class CreateStreamingFlatTable(entry: CreateFlatTableEntry) extends
def loadLookupTables(): Unit = {
val ccCols = model().getRootFactTable.getColumns.asScala.filter(_.getColumnDesc.isComputedColumn).toSet
val cleanLookupCC = cleanComputColumn(ccCols.toSeq, factTableDataset.columns.toSet)
- lookupTablesGlobal = generateLookupTableDataset(model, cleanLookupCC, ss)
+ lookupTablesGlobal = generateLookupTableDataset(model(), cleanLookupCC, ss)
lookupTablesGlobal.foreach { case (_, df) =>
df.persist(StorageLevel.MEMORY_AND_DISK)
}
@@ -138,10 +138,10 @@ class CreateStreamingFlatTable(entry: CreateFlatTableEntry) extends
}
def encodeStreamingDataset(seg: NDataSegment, model: NDataModel, batchDataset: Dataset[Row]): Dataset[Row] = {
- val ccCols = model.getRootFactTable.getColumns.asScala.filter(_.getColumnDesc.isComputedColumn).toSet
+ val ccCols = model.getRootFactTable.getColumns.asScala.toSet
val (dictCols, encodeCols): GlobalDictType = assemblyGlobalDictTuple(seg, toBuildTree)
val encodedDataset = encodeWithCols(batchDataset, ccCols, dictCols, encodeCols)
- val filterEncodedDataset = FlatTableHelper.applyFilterCondition(flatTable, encodedDataset, true)
+ val filterEncodedDataset = FlatTableHelper.applyFilterCondition(flatTable, encodedDataset, needReplaceDot = true)
flatTable match {
case joined: NCubeJoinedFlatTableDesc =>