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 *&frasl;,
+     * with <strong>/*+ MODEL_PRIORITY(m1,m2) *&frasl;</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 =>