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/01/29 09:31:44 UTC

[kylin] 02/06: KYLIN-5309 propose more flexible runtime join scenarios for Kylin

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 59a9b46a2cc33334b296361efb7f0e89dcba9b5e
Author: Pengfei Zhan <de...@gmail.com>
AuthorDate: Sat Jan 14 14:07:15 2023 +0800

    KYLIN-5309 propose more flexible runtime join scenarios for Kylin
---
 .../rest/request/ProjectExclusionRequest.java}     |  15 +-
 .../kylin/rest/request/TableExclusionRequest.java  |  57 ++++
 .../rest/response/ExcludedColumnResponse.java      |  52 ++++
 .../rest/response/ExcludedTableDetailResponse.java |  48 ++++
 .../rest/response/ExcludedTableResponse.java}      |  31 +-
 .../kylin/rest/response/ProjectConfigResponse.java |  25 +-
 .../apache/kylin/rest/service/ProjectService.java  |  37 ++-
 .../org/apache/kylin/common/KylinConfigBase.java   |  18 +-
 .../common/exception/code/ErrorCodeServer.java     |   4 +-
 .../common/persistence/RootPersistentEntity.java   |   2 +-
 .../resources/kylin_error_msg_conf_cn.properties   |   1 +
 .../resources/kylin_error_msg_conf_en.properties   |   1 +
 .../main/resources/kylin_errorcode_conf.properties |   1 +
 .../apache/kylin/metadata/MetadataExtension.java   |  87 ------
 .../metadata/cube/cuboid/AggIndexMatcher.java      |  60 ++--
 .../kylin/metadata/cube/cuboid/ChooserContext.java |  11 +-
 .../kylin/metadata/cube/cuboid/IndexMatcher.java   |  94 ++++--
 .../metadata/cube/cuboid/NQueryLayoutChooser.java  | 152 +++++-----
 .../metadata/cube/cuboid/TableIndexMatcher.java    |  60 ++--
 .../kylin/metadata/cube/model/NBatchConstants.java |   1 -
 .../kylin/metadata/cube/model/NDataSegment.java    |  14 -
 .../metadata/favorite/FavoriteRuleManager.java     |   9 +-
 ...udedLookupChecker.java => AntiFlatChecker.java} | 186 +++++-------
 .../kylin/metadata/model/ColExcludedChecker.java   | 177 ++++++++++++
 .../kylin/metadata/model/ComputedColumnDesc.java   |   4 +
 .../metadata/model/NTableMetadataManager.java      |  12 +
 .../apache/kylin/metadata/model/TableExtDesc.java  |  99 +++----
 .../kylin/metadata/model/alias/AliasMapping.java   |  18 +-
 .../metadata/model/alias/ExpressionComparator.java |  18 +-
 .../kylin/metadata/model/tool/CalciteParser.java   |   4 +-
 .../metadata/model/util/ComputedColumnUtil.java    |   8 +-
 .../metadata/recommendation/ref/OptRecV2.java      |  20 +-
 .../org/apache/kylin/model/TableExtDescTest.java   |  32 +--
 .../org/apache/kylin/util/MetadataTestUtils.java   | 214 ++++++++++++++
 src/datasource-service/pom.xml                     |   6 +
 .../kylin/rest/response/TableDescResponse.java     |   6 +-
 .../apache/kylin/rest/service/TableExtService.java | 131 ++++++++-
 .../apache/kylin/rest/service/TableService.java    | 131 +++++----
 .../kylin/rest/service/TableExtServiceTest.java    | 317 ++++++++++++++++++++-
 .../test_case_data/sandbox/kylin.properties        |   4 +-
 .../kylin/metadata/model/AntiFlatCheckerTest.java  | 167 +++++++++++
 .../metadata/model/ColExcludedCheckerTest.java     | 205 +++++++++++++
 .../java/org/apache/kylin/util/ExecAndComp.java    |   5 +-
 .../kylin/rest/controller/NProjectController.java  |  19 +-
 .../kylin/rest/controller/NTableController.java    |  86 +++++-
 .../rest/controller/NProjectControllerTest.java    |  15 +-
 .../rest/controller/NTableControllerTest.java      |  71 ++++-
 src/modeling-service/pom.xml                       |  21 ++
 .../kylin/rest/response/NDataModelResponse.java    |  65 ++---
 .../apache/kylin/rest/service/ModelService.java    |  27 +-
 .../rest/response/NDataModelResponseTest.java      | 105 +++++--
 .../kylin/rest/service/ModelServiceTest.java       |  19 +-
 .../kylin/rest/service/ProjectServiceTest.java     |  43 ++-
 .../kylin/rest/service/TableReloadServiceTest.java | 210 ++++++++++----
 .../kylin/rest/service/TableServiceTest.java       |  16 +-
 .../apache/kylin/query/relnode/OLAPContext.java    |  32 +--
 .../kylin/query/routing/RealizationChooser.java    |  28 +-
 .../kylin/query/routing/RealizationPruner.java     |  16 +-
 .../kylin/query/util/ComputedColumnRewriter.java   |  51 ++--
 .../kylin/query/util/ConvertToComputedColumn.java  |  80 +++---
 .../apache/kylin/query/util/QueryAliasMatcher.java |   8 +
 .../org/apache/kylin/query/util/QueryUtil.java     |   6 +-
 .../query/util/RestoreFromComputedColumn.java      |   2 +-
 .../kylin/rest/service/ModelServiceQueryTest.java  |   2 -
 .../org/apache/kylin/query/engine/QueryExec.java   |  11 +-
 .../kylin/query/util/ExpressionComparatorTest.java | 100 ++++---
 .../org/apache/kylin/query/util/QueryUtilTest.java |  26 +-
 src/spark-project/engine-spark/pom.xml             |   6 +
 .../kylin/engine/spark/job/NSparkCubingJob.java    | 148 +++++-----
 .../kylin/engine/spark/job/NSparkExecutable.java   |  12 -
 .../engine/spark/builder/SegmentFlatTable.scala    |  18 +-
 .../apache/kylin/engine/spark/job/SegmentJob.java  |  11 +-
 .../job/stage/build/FlatTableAndDictBase.scala     |  16 +-
 .../engine/spark/model/SegmentFlatTableDesc.java   |  15 +-
 .../engine/spark/NLocalWithSparkSessionTest.java   |   6 -
 .../engine/spark/job/NSparkCubingJobTest.java      |  56 ++--
 76 files changed, 2744 insertions(+), 1147 deletions(-)

diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/query/QueryExcludedTablesExtension.java b/src/common-service/src/main/java/org/apache/kylin/rest/request/ProjectExclusionRequest.java
similarity index 65%
rename from src/core-metadata/src/main/java/org/apache/kylin/metadata/query/QueryExcludedTablesExtension.java
rename to src/common-service/src/main/java/org/apache/kylin/rest/request/ProjectExclusionRequest.java
index bb0a51a511..2bdbfc7b1e 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/query/QueryExcludedTablesExtension.java
+++ b/src/common-service/src/main/java/org/apache/kylin/rest/request/ProjectExclusionRequest.java
@@ -16,15 +16,14 @@
  * limitations under the License.
  */
 
-package org.apache.kylin.metadata.query;
+package org.apache.kylin.rest.request;
 
-import java.util.Set;
+import com.fasterxml.jackson.annotation.JsonProperty;
 
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.extension.KylinExtension;
+import lombok.Data;
 
-public interface QueryExcludedTablesExtension extends KylinExtension {
-    Set<String> getExcludedTables(KylinConfig kylinConfig, String projectName);
-
-    void addExcludedTables(KylinConfig config, String projectName, String tableName, boolean isEnabled);
+@Data
+public class ProjectExclusionRequest {
+    @JsonProperty("table_exclusion_enabled")
+    private boolean isTableExclusionEnabled;
 }
diff --git a/src/common-service/src/main/java/org/apache/kylin/rest/request/TableExclusionRequest.java b/src/common-service/src/main/java/org/apache/kylin/rest/request/TableExclusionRequest.java
new file mode 100644
index 0000000000..bc92a13b31
--- /dev/null
+++ b/src/common-service/src/main/java/org/apache/kylin/rest/request/TableExclusionRequest.java
@@ -0,0 +1,57 @@
+/*
+ * 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.rest.request;
+
+import java.io.Serializable;
+import java.util.List;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.collect.Lists;
+
+import lombok.Data;
+import lombok.EqualsAndHashCode;
+import lombok.Getter;
+import lombok.Setter;
+
+@Getter
+@Setter
+@EqualsAndHashCode
+public class TableExclusionRequest implements Serializable {
+    @JsonProperty("project")
+    private String project;
+    @JsonProperty("canceled_tables")
+    private List<String> canceledTables = Lists.newArrayList();
+    @JsonProperty("excluded_tables")
+    private List<ExcludedTable> excludedTables = Lists.newArrayList();
+
+    @Data
+    @Getter
+    @Setter
+    @EqualsAndHashCode
+    public static class ExcludedTable implements Serializable {
+        @JsonProperty("table")
+        private String table;
+        @JsonProperty("excluded")
+        private boolean excluded;
+        @JsonProperty("removed_columns")
+        private List<String> removedColumns = Lists.newArrayList();
+        @JsonProperty("added_columns")
+        private List<String> addedColumns = Lists.newArrayList();
+    }
+}
diff --git a/src/common-service/src/main/java/org/apache/kylin/rest/response/ExcludedColumnResponse.java b/src/common-service/src/main/java/org/apache/kylin/rest/response/ExcludedColumnResponse.java
new file mode 100644
index 0000000000..60285fbddc
--- /dev/null
+++ b/src/common-service/src/main/java/org/apache/kylin/rest/response/ExcludedColumnResponse.java
@@ -0,0 +1,52 @@
+/*
+ * 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.rest.response;
+
+import java.io.Serializable;
+
+import org.apache.kylin.metadata.model.ColumnDesc;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import lombok.NoArgsConstructor;
+import lombok.Setter;
+
+@Getter
+@Setter
+@NoArgsConstructor
+@AllArgsConstructor
+public class ExcludedColumnResponse implements Serializable {
+    @JsonProperty("name")
+    private String name;
+    @JsonProperty("datatype")
+    private String datatype;
+    @JsonProperty("comment")
+    private String comment;
+    @JsonProperty("excluded")
+    private boolean excluded;
+
+    public ExcludedColumnResponse(ColumnDesc column, boolean excluded) {
+        this.name = column.getName();
+        this.datatype = column.getDatatype();
+        this.comment = column.getComment();
+        this.excluded = excluded;
+    }
+}
diff --git a/src/common-service/src/main/java/org/apache/kylin/rest/response/ExcludedTableDetailResponse.java b/src/common-service/src/main/java/org/apache/kylin/rest/response/ExcludedTableDetailResponse.java
new file mode 100644
index 0000000000..9731f32748
--- /dev/null
+++ b/src/common-service/src/main/java/org/apache/kylin/rest/response/ExcludedTableDetailResponse.java
@@ -0,0 +1,48 @@
+/*
+ * 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.rest.response;
+
+import java.util.List;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.collect.Lists;
+
+import lombok.Getter;
+import lombok.NoArgsConstructor;
+import lombok.Setter;
+
+@Getter
+@Setter
+@NoArgsConstructor
+public class ExcludedTableDetailResponse {
+    @JsonProperty("table")
+    private String table;
+    @JsonProperty("excluded")
+    private boolean excluded;
+    @JsonProperty("total_size")
+    private int totalSize;
+    @JsonProperty("offset")
+    private int offset;
+    @JsonProperty("limit")
+    private int limit;
+    @JsonProperty("excluded_columns")
+    private List<ExcludedColumnResponse> excludedColumns = Lists.newArrayList();
+    @JsonProperty("admitted_columns")
+    private List<ExcludedColumnResponse> admittedColumns = Lists.newArrayList();
+}
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/alias/AliasMapping.java b/src/common-service/src/main/java/org/apache/kylin/rest/response/ExcludedTableResponse.java
similarity index 55%
copy from src/core-metadata/src/main/java/org/apache/kylin/metadata/model/alias/AliasMapping.java
copy to src/common-service/src/main/java/org/apache/kylin/rest/response/ExcludedTableResponse.java
index b79c8198cc..e9ed02ed26 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/alias/AliasMapping.java
+++ b/src/common-service/src/main/java/org/apache/kylin/rest/response/ExcludedTableResponse.java
@@ -15,19 +15,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.kylin.metadata.model.alias;
 
-import com.google.common.collect.BiMap;
+package org.apache.kylin.rest.response;
 
-public class AliasMapping {
+import java.util.List;
 
-    private BiMap<String, String> aliasMapping;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.collect.Lists;
 
-    public AliasMapping(BiMap<String, String> aliasMapping) {
-        this.aliasMapping = aliasMapping;
-    }
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import lombok.NoArgsConstructor;
+import lombok.Setter;
 
-    public BiMap<String, String> getAliasMapping() {
-        return aliasMapping;
-    }
+@Getter
+@Setter
+@NoArgsConstructor
+@AllArgsConstructor
+public class ExcludedTableResponse {
+    @JsonProperty("table")
+    private String table;
+    @JsonProperty("excluded")
+    private boolean excluded;
+    @JsonProperty("excluded_col_size")
+    private int excludedColSize;
+    @JsonProperty("excluded_columns")
+    private List<String> excludedColumns = Lists.newArrayList();
 }
diff --git a/src/common-service/src/main/java/org/apache/kylin/rest/response/ProjectConfigResponse.java b/src/common-service/src/main/java/org/apache/kylin/rest/response/ProjectConfigResponse.java
index 784731ec9b..613915a7ad 100644
--- a/src/common-service/src/main/java/org/apache/kylin/rest/response/ProjectConfigResponse.java
+++ b/src/common-service/src/main/java/org/apache/kylin/rest/response/ProjectConfigResponse.java
@@ -43,7 +43,8 @@ public class ProjectConfigResponse {
     private String defaultDatabase;
     @JsonProperty("semi_automatic_mode")
     private boolean semiAutomaticMode;
-
+    @JsonProperty("table_exclusion_enabled")
+    private boolean tableExclusionEnabled;
     @JsonProperty("storage_quota_size")
     private long storageQuotaSize;
 
@@ -145,17 +146,17 @@ public class ProjectConfigResponse {
 
     public void setFrequencyTimeWindow(int frequencyTimeWindow) {
         switch (frequencyTimeWindow) {
-            case 1:
-                this.frequencyTimeWindow = "DAY";
-                break;
-            case 7:
-                this.frequencyTimeWindow = "WEEK";
-                break;
-            case 30:
-                this.frequencyTimeWindow = "MONTH";
-                break;
-            default:
-                break;
+        case 1:
+            this.frequencyTimeWindow = "DAY";
+            break;
+        case 7:
+            this.frequencyTimeWindow = "WEEK";
+            break;
+        case 30:
+            this.frequencyTimeWindow = "MONTH";
+            break;
+        default:
+            break;
         }
 
     }
diff --git a/src/common-service/src/main/java/org/apache/kylin/rest/service/ProjectService.java b/src/common-service/src/main/java/org/apache/kylin/rest/service/ProjectService.java
index b24de0d000..3658763858 100644
--- a/src/common-service/src/main/java/org/apache/kylin/rest/service/ProjectService.java
+++ b/src/common-service/src/main/java/org/apache/kylin/rest/service/ProjectService.java
@@ -81,6 +81,8 @@ import org.apache.kylin.job.execution.NExecutableManager;
 import org.apache.kylin.metadata.MetadataConstants;
 import org.apache.kylin.metadata.cube.storage.ProjectStorageInfoCollector;
 import org.apache.kylin.metadata.cube.storage.StorageInfoEnum;
+import org.apache.kylin.metadata.epoch.EpochManager;
+import org.apache.kylin.metadata.favorite.FavoriteRuleManager;
 import org.apache.kylin.metadata.model.ISourceAware;
 import org.apache.kylin.metadata.model.NDataModelManager;
 import org.apache.kylin.metadata.model.NTableMetadataManager;
@@ -88,6 +90,7 @@ import org.apache.kylin.metadata.project.EnhancedUnitOfWork;
 import org.apache.kylin.metadata.project.NProjectManager;
 import org.apache.kylin.metadata.project.ProjectInstance;
 import org.apache.kylin.metadata.realization.RealizationStatusEnum;
+import org.apache.kylin.metadata.recommendation.candidate.RawRecManager;
 import org.apache.kylin.rest.aspect.Transaction;
 import org.apache.kylin.rest.config.initialize.ProjectDropListener;
 import org.apache.kylin.rest.constant.Constant;
@@ -98,6 +101,7 @@ import org.apache.kylin.rest.request.JdbcSourceInfoRequest;
 import org.apache.kylin.rest.request.JobNotificationConfigRequest;
 import org.apache.kylin.rest.request.MultiPartitionConfigRequest;
 import org.apache.kylin.rest.request.OwnerChangeRequest;
+import org.apache.kylin.rest.request.ProjectExclusionRequest;
 import org.apache.kylin.rest.request.ProjectGeneralInfoRequest;
 import org.apache.kylin.rest.request.ProjectKerberosInfoRequest;
 import org.apache.kylin.rest.request.PushDownConfigRequest;
@@ -136,9 +140,6 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
-import org.apache.kylin.metadata.epoch.EpochManager;
-import org.apache.kylin.metadata.favorite.FavoriteRuleManager;
-import org.apache.kylin.metadata.recommendation.candidate.RawRecManager;
 import io.kyligence.kap.secondstorage.SecondStorageUtil;
 import lombok.SneakyThrows;
 import lombok.val;
@@ -556,6 +557,7 @@ public class ProjectService extends BasicService {
         response.setDescription(projectInstance.getDescription());
         response.setDefaultDatabase(projectInstance.getDefaultDatabase());
         response.setSemiAutomaticMode(config.isSemiAutoMode());
+        response.setTableExclusionEnabled(config.isTableExclusionEnabled());
 
         response.setStorageQuotaSize(config.getStorageQuotaSize());
 
@@ -1012,6 +1014,9 @@ public class ProjectService extends BasicService {
         case "favorite_rule_config":
             resetProjectRecommendationConfig(project);
             break;
+        case "table_exclusion_config":
+            resetTableExclusionConfig(project);
+            break;
         default:
             throw new KylinException(INVALID_PARAMETER,
                     "No valid value for 'reset_item'. Please enter a project setting "
@@ -1051,14 +1056,14 @@ public class ProjectService extends BasicService {
         toBeRemovedProps.add("kylin.job.notification-enable-states");
         toBeRemovedProps.add("kylin.job.notification-user-emails");
         toBeRemovedProps.add("kylin.job.notification-on-metadata-persist");
-        removeProjectOveridedProps(project, toBeRemovedProps);
+        removeProjectOverrideProps(project, toBeRemovedProps);
     }
 
     private void resetQueryAccelerateThreshold(String project) {
         Set<String> toBeRemovedProps = Sets.newHashSet();
         toBeRemovedProps.add("kylin.favorite.query-accelerate-threshold");
         toBeRemovedProps.add("kylin.favorite.query-accelerate-tips-enable");
-        removeProjectOveridedProps(project, toBeRemovedProps);
+        removeProjectOverrideProps(project, toBeRemovedProps);
     }
 
     private void resetProjectRecommendationConfig(String project) {
@@ -1071,7 +1076,13 @@ public class ProjectService extends BasicService {
         Set<String> toBeRemovedProps = Sets.newHashSet();
         toBeRemovedProps.add("kylin.cube.low-frequency-threshold");
         toBeRemovedProps.add("kylin.cube.frequency-time-window");
-        removeProjectOveridedProps(project, toBeRemovedProps);
+        removeProjectOverrideProps(project, toBeRemovedProps);
+    }
+
+    private void resetTableExclusionConfig(String project) {
+        Set<String> toBeRemovedProps = Sets.newHashSet();
+        toBeRemovedProps.add("kylin.metadata.table-exclusion-enabled");
+        removeProjectOverrideProps(project, toBeRemovedProps);
     }
 
     private void resetSegmentConfig(String project) {
@@ -1086,7 +1097,7 @@ public class ProjectService extends BasicService {
         });
     }
 
-    private void removeProjectOveridedProps(String project, Set<String> toBeRemovedProps) {
+    private void removeProjectOverrideProps(String project, Set<String> toBeRemovedProps) {
         val projectManager = getManager(NProjectManager.class);
         val projectInstance = projectManager.getProject(project);
         if (projectInstance == null) {
@@ -1111,7 +1122,7 @@ public class ProjectService extends BasicService {
     private void resetProjectStorageQuotaConfig(String project) {
         Set<String> toBeRemovedProps = Sets.newHashSet();
         toBeRemovedProps.add("kylin.storage.quota-in-giga-bytes");
-        removeProjectOveridedProps(project, toBeRemovedProps);
+        removeProjectOverrideProps(project, toBeRemovedProps);
     }
 
     private List<ProjectInstance> getProjectsWithFilter(Predicate<ProjectInstance> filter) {
@@ -1182,4 +1193,14 @@ public class ProjectService extends BasicService {
         overrideKylinProps.put("kylin.source.default", String.valueOf(ISourceAware.ID_JDBC));
         updateProjectOverrideKylinProps(project, overrideKylinProps);
     }
+
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#project, 'ADMINISTRATION')")
+    @Transaction(project = 0)
+    public void updateTableExclusionRule(String project, ProjectExclusionRequest request) {
+        getManager(NProjectManager.class).updateProject(project, copyForWrite -> {
+            boolean exclusionEnabled = request.isTableExclusionEnabled();
+            copyForWrite.putOverrideKylinProps("kylin.metadata.table-exclusion-enabled",
+                    String.valueOf(exclusionEnabled));
+        });
+    }
 }
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 e9d033b675..bebd3f403f 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
@@ -742,10 +742,23 @@ public abstract class KylinConfigBase implements Serializable {
         return Boolean.parseBoolean(getOptional("kylin.metadata.semi-automatic-mode", FALSE));
     }
 
+    public boolean isTableExclusionEnabled() {
+        return Boolean.parseBoolean(getOptional("kylin.metadata.table-exclusion-enabled", FALSE));
+    }
+
+    /**
+     * When table exclusion is enabled, this setting ensures the accuracy of query result.
+     */
+    public boolean isSnapshotPreferred() {
+        return Boolean.parseBoolean(getOptional("kylin.query.snapshot-preferred-for-table-exclusion", TRUE));
+    }
+
+    public boolean onlyReuseUserDefinedCC() {
+        return Boolean.parseBoolean(getOptional("kylin.metadata.only-reuse-user-defined-computed-column", FALSE));
+    }
+
     /**
      * expose computed column in the table metadata and select * queries
-     *
-     * @return
      */
     public boolean exposeComputedColumn() {
         return Boolean.parseBoolean(getOptional("kylin.query.metadata.expose-computed-column", FALSE));
@@ -1159,6 +1172,7 @@ public abstract class KylinConfigBase implements Serializable {
     public Boolean getJobMetadataPersistNotificationEnabled() {
         return Boolean.parseBoolean(this.getOptional("kylin.job.notification-on-metadata-persist", FALSE));
     }
+
     public int getJobRetry() {
         return Integer.parseInt(getOptional("kylin.job.retry", "0"));
     }
diff --git a/src/core-common/src/main/java/org/apache/kylin/common/exception/code/ErrorCodeServer.java b/src/core-common/src/main/java/org/apache/kylin/common/exception/code/ErrorCodeServer.java
index c8abaa1142..00bcf704a2 100644
--- a/src/core-common/src/main/java/org/apache/kylin/common/exception/code/ErrorCodeServer.java
+++ b/src/core-common/src/main/java/org/apache/kylin/common/exception/code/ErrorCodeServer.java
@@ -60,7 +60,9 @@ public enum ErrorCodeServer implements ErrorCodeProducer {
     SEGMENT_MERGE_CHECK_PARTITION_ILLEGAL("KE-010022219"),
 
     // 100072XX table
-    TABLE_RELOAD_MODEL_RETRY("KE-010007204"), TABLE_RELOAD_HAVING_NOT_FINAL_JOB("KE-010007208"),
+    TABLE_RELOAD_MODEL_RETRY("KE-010007204"),
+    TABLE_RELOAD_HAVING_NOT_FINAL_JOB("KE-010007208"),
+    EXCLUDED_TABLE_REQUEST_NOT_ALLOWED("KE-010007301"),
 
     // 10032XXX add job result
     JOB_CREATE_CHECK_MULTI_PARTITION_EMPTY("KE-010032201"),
diff --git a/src/core-common/src/main/java/org/apache/kylin/common/persistence/RootPersistentEntity.java b/src/core-common/src/main/java/org/apache/kylin/common/persistence/RootPersistentEntity.java
index c4038e0a57..53f6ace8cc 100644
--- a/src/core-common/src/main/java/org/apache/kylin/common/persistence/RootPersistentEntity.java
+++ b/src/core-common/src/main/java/org/apache/kylin/common/persistence/RootPersistentEntity.java
@@ -47,7 +47,7 @@ import lombok.extern.slf4j.Slf4j;
 @SuppressWarnings("serial")
 @JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
 @Slf4j
-abstract public class RootPersistentEntity implements AclEntity, Serializable {
+public abstract class RootPersistentEntity implements AclEntity, Serializable {
 
     // for spring session save serializable object(ManagerUser), do not modify
     private static final long serialVersionUID = 0L;
diff --git a/src/core-common/src/main/resources/kylin_error_msg_conf_cn.properties b/src/core-common/src/main/resources/kylin_error_msg_conf_cn.properties
index 7fab056ac7..3f1a080837 100644
--- a/src/core-common/src/main/resources/kylin_error_msg_conf_cn.properties
+++ b/src/core-common/src/main/resources/kylin_error_msg_conf_cn.properties
@@ -63,6 +63,7 @@ KE-010022219=当前 Segments 所包含的分区不一致,请先构建分区并
 ## 100072XX table
 KE-010007204=源表 %1$s 中列 %2$s 的数据类型发生变更。请从模型 %3$s 中删除该列,或修改该列的数据类型。
 KE-010007208=当前暂不可重载表。存在运行中的任务,任务对象为: %s。请等任务完成后再重载,或手动终止任务。
+KE-010007301=获取屏蔽列设置失败,项目 %s 没有打开屏蔽列开关。
 
 ## 100322XX job
 KE-010032201=无法添加任务,子分区值为空。请检查后重试。
diff --git a/src/core-common/src/main/resources/kylin_error_msg_conf_en.properties b/src/core-common/src/main/resources/kylin_error_msg_conf_en.properties
index 967a6c7d8c..6f37eec3c3 100644
--- a/src/core-common/src/main/resources/kylin_error_msg_conf_en.properties
+++ b/src/core-common/src/main/resources/kylin_error_msg_conf_en.properties
@@ -62,6 +62,7 @@ KE-010022219=The partitions included in the selected segments are not consistent
 ## 100072XX table
 KE-010007204=The data type of column %2$s from the source table %1$s has changed. Please remove the column from model %3$s, or modify the data type.
 KE-010007208=The table metadata can't be reloaded now. There are ongoing jobs with the following target subjects(s): %s. Please try reloading until all the jobs are completed, or manually discard the jobs.
+KE-010007301=Fail to fetch setting of excluded columns, because project %s is not enable excluded columns.
 
 ## 100322XX job
 KE-010032201=Can't add the job, as the subpartition value is empty. Please check and try again.
diff --git a/src/core-common/src/main/resources/kylin_errorcode_conf.properties b/src/core-common/src/main/resources/kylin_errorcode_conf.properties
index 7f07828a6a..ff675b94f1 100644
--- a/src/core-common/src/main/resources/kylin_errorcode_conf.properties
+++ b/src/core-common/src/main/resources/kylin_errorcode_conf.properties
@@ -61,6 +61,7 @@ KE-010022217
 ## 100072XX table
 KE-010007204
 KE-010007208
+KE-010007301
 
 ## 100322XX job
 KE-010032201
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/MetadataExtension.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/MetadataExtension.java
deleted file mode 100644
index ab87b6c9df..0000000000
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/MetadataExtension.java
+++ /dev/null
@@ -1,87 +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.metadata;
-
-import java.util.Set;
-
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.extension.ExtensionFactoryLoader;
-import org.apache.kylin.metadata.query.QueryExcludedTablesExtension;
-
-import com.google.common.collect.Sets;
-
-import io.kyligence.kap.guava20.shaded.common.annotations.VisibleForTesting;
-
-public class MetadataExtension {
-
-    private static Factory extensionFactory = null;
-    private static final ExtensionFactoryLoader<Factory> loader = new ExtensionFactoryLoader<>();
-
-    private MetadataExtension() {
-    }
-
-    public static Factory getFactory() {
-        if (extensionFactory == null) {
-            synchronized (MetadataExtension.class) {
-                extensionFactory = loader.loadFactory(Factory.class,
-                        KylinConfig.getInstanceFromEnv().getMetadataExtensionFactory());
-            }
-        }
-        return extensionFactory;
-    }
-
-    @VisibleForTesting
-    public static void setFactory(Factory newFactory) {
-        synchronized (MetadataExtension.class) {
-            extensionFactory = newFactory;
-        }
-    }
-
-    public static class Factory {
-
-        public Factory() {
-            // Do nothing
-        }
-
-        private QueryExcludedTablesExtension queryExcludedTablesExtension;
-
-        public final QueryExcludedTablesExtension getQueryExcludedTablesExtension() {
-            if (queryExcludedTablesExtension == null) {
-                queryExcludedTablesExtension = createQueryExcludedTablesExtension();
-            }
-            return queryExcludedTablesExtension;
-        }
-
-        protected QueryExcludedTablesExtension createQueryExcludedTablesExtension() {
-            return new QueryExcludedTablesExtensionDefault();
-        }
-    }
-
-    private static class QueryExcludedTablesExtensionDefault implements QueryExcludedTablesExtension {
-        @Override
-        public Set<String> getExcludedTables(KylinConfig kylinConfig, String projectName) {
-            return Sets.newHashSet();
-        }
-
-        @Override
-        public void addExcludedTables(KylinConfig config, String projectName, String tableName, boolean isEnabled) {
-            // do nothing
-        }
-    }
-}
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/cuboid/AggIndexMatcher.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/cuboid/AggIndexMatcher.java
index 3a2b521119..7854f7a8f1 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/cuboid/AggIndexMatcher.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/cuboid/AggIndexMatcher.java
@@ -31,15 +31,19 @@ import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.kylin.measure.MeasureType;
 import org.apache.kylin.measure.basic.BasicMeasureType;
+import org.apache.kylin.metadata.cube.model.IndexEntity;
+import org.apache.kylin.metadata.cube.model.LayoutEntity;
+import org.apache.kylin.metadata.cube.model.NDataflow;
+import org.apache.kylin.metadata.model.AntiFlatChecker;
+import org.apache.kylin.metadata.model.ColExcludedChecker;
 import org.apache.kylin.metadata.model.DeriveInfo;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.NDataModel;
 import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.metadata.project.NProjectManager;
 import org.apache.kylin.metadata.realization.CapabilityResult;
 import org.apache.kylin.metadata.realization.SQLDigest;
-import org.apache.kylin.metadata.cube.model.IndexEntity;
-import org.apache.kylin.metadata.cube.model.LayoutEntity;
-import org.apache.kylin.metadata.model.NDataModel;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
@@ -50,19 +54,17 @@ import lombok.extern.slf4j.Slf4j;
 @Slf4j
 public class AggIndexMatcher extends IndexMatcher {
 
-    private final boolean isReplaceCount;
-    private Set<Integer> sqlColumns;
-    private final Map<FunctionDesc, List<Integer>> functionCols = Maps.newHashMap();
-    private final boolean valid;
+    private final Map<FunctionDesc, List<Integer>> functionCols;
 
-    public AggIndexMatcher(SQLDigest sqlDigest, ChooserContext model, Set<String> excludedTables,
-            boolean isReplaceCount) {
-        super(sqlDigest, model, excludedTables);
-        this.isReplaceCount = isReplaceCount;
-        valid = init();
+    public AggIndexMatcher(SQLDigest sqlDigest, ChooserContext chooserContext, NDataflow dataflow,
+            ColExcludedChecker exColChecker, AntiFlatChecker antiFlatChecker) {
+        super(sqlDigest, chooserContext, dataflow, exColChecker, antiFlatChecker);
+        this.functionCols = Maps.newHashMap();
+        this.valid = fastValidCheckBeforeMatch();
     }
 
-    private boolean init() {
+    @Override
+    protected boolean fastValidCheckBeforeMatch() {
         // cols may have null values as the CC col in query may not present in the model
         sqlColumns = Stream.concat(sqlDigest.filterColumns.stream(), sqlDigest.groupbyColumns.stream())
                 .map(tblColMap::get).collect(Collectors.toSet());
@@ -82,28 +84,18 @@ public class AggIndexMatcher extends IndexMatcher {
         return true;
     }
 
-    public boolean valid() {
-        return valid;
-    }
-
     @Override
     MatchResult match(LayoutEntity layout) {
-        if (!needAggIndexMatch(layout.getIndex()) || !valid) {
-            return new MatchResult(false);
+        if (canSkipIndexMatch(layout.getIndex()) || !isValid()) {
+            return new MatchResult();
         }
         log.trace("Matching agg index");
-        Set<Integer> unmatchedCols = Sets.newHashSet();
-        unmatchedCols.addAll(sqlColumns);
         Set<FunctionDesc> unmatchedMetrics = Sets.newHashSet(sqlDigest.aggregations);
-
-        if (isBatchFusionModel) {
-            unmatchedCols.removeAll(layout.getStreamingColumns().keySet());
-        }
-        unmatchedCols.removeAll(layout.getOrderedDimensions().keySet());
+        Set<Integer> unmatchedCols = initUnmatchedColumnIds(layout);
         final Map<Integer, DeriveInfo> needDerive = Maps.newHashMap();
         goThruDerivedDims(layout.getIndex(), needDerive, unmatchedCols);
         unmatchedAggregations(unmatchedMetrics, layout);
-        if (isReplaceCount) {
+        if (NProjectManager.getProjectConfig(project).isReplaceColCountWithCountStar()) {
             unmatchedCountColumnIfExistCountStar(unmatchedMetrics);
         }
 
@@ -115,6 +107,13 @@ public class AggIndexMatcher extends IndexMatcher {
         }
 
         boolean matched = unmatchedCols.isEmpty() && unmatchedMetrics.isEmpty();
+        if (!matched) {
+            unmatchedCols.removeAll(filterExcludedDims(layout));
+            log.debug("After rolling back to AggIndex to match, the unmatched columns are: ({}), "
+                    + "the unmatched measures are: ({})", unmatchedCols, unmatchedMetrics);
+            matched = unmatchedMetrics.isEmpty() && unmatchedCols.isEmpty();
+        }
+
         if (!matched && log.isDebugEnabled()) {
             log.debug("Agg index {} with unmatched columns {}, unmatched metrics {}", //
                     layout, unmatchedCols, unmatchedMetrics);
@@ -123,8 +122,9 @@ public class AggIndexMatcher extends IndexMatcher {
         return new MatchResult(matched, needDerive, null, influences);
     }
 
-    private boolean needAggIndexMatch(IndexEntity indexEntity) {
-        return !indexEntity.isTableIndex() && !sqlDigest.isRawQuery;
+    @Override
+    protected boolean canSkipIndexMatch(IndexEntity indexEntity) {
+        return indexEntity.isTableIndex() || sqlDigest.isRawQuery;
     }
 
     private void removeUnmatchedGroupingAgg(Collection<FunctionDesc> unmatchedAggregations) {
@@ -210,7 +210,7 @@ public class AggIndexMatcher extends IndexMatcher {
             }
         }
         if (!influencingMeasures.isEmpty()) {
-            log.info("NDataflow {} CapabilityInfluences: {}", indexEntity.getIndexPlan().getUuid(),
+            log.info("NDataflow {} CapabilityInfluences: {}", dataflow.getUuid(),
                     StringUtils.join(influencingMeasures, ","));
         }
     }
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/cuboid/ChooserContext.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/cuboid/ChooserContext.java
index e8a4f69387..5e27635131 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/cuboid/ChooserContext.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/cuboid/ChooserContext.java
@@ -25,15 +25,16 @@ import java.util.stream.Collectors;
 
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.metadata.model.TableExtDesc;
-import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.model.NDataModel;
 import org.apache.kylin.metadata.model.NDataModelManager;
 import org.apache.kylin.metadata.model.NTableMetadataManager;
+import org.apache.kylin.metadata.model.TableExtDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+
+import com.google.common.collect.ImmutableMultimap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 
-import io.kyligence.kap.guava20.shaded.common.collect.ImmutableMultimap;
-import io.kyligence.kap.guava20.shaded.common.collect.Lists;
-import io.kyligence.kap.guava20.shaded.common.collect.Maps;
 import lombok.Getter;
 import lombok.val;
 
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/cuboid/IndexMatcher.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/cuboid/IndexMatcher.java
index f90114b457..8dad4df87e 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/cuboid/IndexMatcher.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/cuboid/IndexMatcher.java
@@ -27,36 +27,43 @@ import java.util.stream.Stream;
 
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang.StringUtils;
+import org.apache.kylin.metadata.cube.model.IndexEntity;
+import org.apache.kylin.metadata.cube.model.LayoutEntity;
+import org.apache.kylin.metadata.cube.model.NDataflow;
+import org.apache.kylin.metadata.model.AntiFlatChecker;
+import org.apache.kylin.metadata.model.ColExcludedChecker;
 import org.apache.kylin.metadata.model.DeriveInfo;
 import org.apache.kylin.metadata.model.JoinDesc;
+import org.apache.kylin.metadata.model.NDataModel;
 import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.metadata.model.util.scd2.SCD2NonEquiCondSimplification;
+import org.apache.kylin.metadata.project.NProjectManager;
 import org.apache.kylin.metadata.realization.CapabilityResult;
 import org.apache.kylin.metadata.realization.SQLDigest;
-import org.apache.kylin.metadata.cube.model.IndexEntity;
-import org.apache.kylin.metadata.cube.model.LayoutEntity;
-import org.apache.kylin.metadata.model.ExcludedLookupChecker;
-import org.apache.kylin.metadata.model.NDataModel;
-import org.apache.kylin.metadata.model.util.scd2.SCD2NonEquiCondSimplification;
 
-import io.kyligence.kap.guava20.shaded.common.base.Preconditions;
-import io.kyligence.kap.guava20.shaded.common.collect.ImmutableCollection;
-import io.kyligence.kap.guava20.shaded.common.collect.ImmutableMultimap;
-import io.kyligence.kap.guava20.shaded.common.collect.Iterables;
-import io.kyligence.kap.guava20.shaded.common.collect.Lists;
-import io.kyligence.kap.guava20.shaded.common.collect.Maps;
-import io.kyligence.kap.guava20.shaded.common.collect.Sets;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableCollection;
+import com.google.common.collect.ImmutableMultimap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
 import lombok.AllArgsConstructor;
 import lombok.Getter;
-import lombok.NonNull;
 import lombok.RequiredArgsConstructor;
-import lombok.val;
+import lombok.extern.slf4j.Slf4j;
 
+@Slf4j
 public abstract class IndexMatcher {
 
     final SQLDigest sqlDigest;
+    final String project;
+    final NDataflow dataflow;
     final NDataModel model;
-    final Set<String> excludedTables;
     final boolean isBatchFusionModel;
+    @Getter
+    boolean valid;
 
     final ChooserContext chooserContext;
 
@@ -64,14 +71,19 @@ public abstract class IndexMatcher {
     final Map<String, List<Integer>> primaryKeyColumnIds;
     final Map<String, List<Integer>> foreignKeyColumnIds;
     final ImmutableMultimap<Integer, Integer> fk2Pk;
+    Set<Integer> sqlColumns;
 
+    ColExcludedChecker excludedChecker;
+    AntiFlatChecker antiFlatChecker;
     final Map<Integer, DeriveInfo> toManyDerivedInfoMap = Maps.newHashMap();
 
-    IndexMatcher(SQLDigest sqlDigest, ChooserContext chooserContext, Set<String> excludedTables) {
+    IndexMatcher(SQLDigest sqlDigest, ChooserContext chooserContext, NDataflow dataflow,
+            ColExcludedChecker excludedChecker, AntiFlatChecker antiFlatChecker) {
         this.sqlDigest = sqlDigest;
+        this.dataflow = dataflow;
+        this.project = dataflow.getProject();
         this.model = chooserContext.getModel();
         this.chooserContext = chooserContext;
-        this.excludedTables = excludedTables;
         this.isBatchFusionModel = chooserContext.isBatchFusionModel();
 
         this.fk2Pk = chooserContext.getFk2Pk();
@@ -79,12 +91,14 @@ public abstract class IndexMatcher {
         this.primaryKeyColumnIds = chooserContext.getPrimaryKeyColumnIds();
         this.foreignKeyColumnIds = chooserContext.getForeignKeyColumnIds();
 
+        this.excludedChecker = excludedChecker;
+        this.antiFlatChecker = antiFlatChecker;
+
         // suppose: A join B && A join C, the relation of A->C is TO_MANY and C need to derive,
         // then the built index of this join relation only based on the flat table of A join B,
         // in order to get the correct result, the query result must join the snapshot of C.
-        ExcludedLookupChecker checker = new ExcludedLookupChecker(excludedTables, model.getJoinTables(), model);
         model.getJoinTables().forEach(joinTableDesc -> {
-            if (checker.getExcludedLookups().contains(joinTableDesc.getTable())) {
+            if (antiFlatChecker.getAntiFlattenLookups().contains(joinTableDesc.getTable())) {
                 JoinDesc join = joinTableDesc.getJoin();
                 if (!joinTableDesc.isToManyJoinRelation() || !needJoinSnapshot(join)) {
                     return;
@@ -97,8 +111,37 @@ public abstract class IndexMatcher {
         });
     }
 
+    //the integrity check is passed
+    protected abstract boolean fastValidCheckBeforeMatch();
+
     abstract MatchResult match(LayoutEntity layout);
 
+    protected abstract boolean canSkipIndexMatch(IndexEntity index);
+
+    Set<Integer> initUnmatchedColumnIds(LayoutEntity layout) {
+        Set<Integer> unmatchedCols = Sets.newHashSet();
+        unmatchedCols.addAll(sqlColumns);
+        if (isBatchFusionModel) {
+            unmatchedCols.removeAll(layout.getStreamingColumns().keySet());
+        }
+        unmatchedCols.removeAll(layout.getOrderedDimensions().keySet());
+        Set<Integer> excludedColSet = filterExcludedDims(layout);
+        if (!excludedColSet.isEmpty()) {
+            log.debug("Excluded columns of layout need to derive. The id set is: {}", excludedColSet);
+            unmatchedCols.addAll(excludedColSet);
+        }
+        return unmatchedCols;
+    }
+
+    Set<Integer> filterExcludedDims(LayoutEntity layout) {
+        if (!NProjectManager.getProjectConfig(project).isSnapshotPreferred()) {
+            return Sets.newHashSet();
+        }
+        return layout.getOrderedDimensions().entrySet().stream() //
+                .filter(entry -> excludedChecker.isExcludedCol(entry.getValue())) //
+                .map(Map.Entry::getKey).collect(Collectors.toSet());
+    }
+
     void goThruDerivedDims(final IndexEntity indexEntity, Map<Integer, DeriveInfo> needDeriveCollector,
             Set<Integer> unmatchedDims) {
         Iterator<Integer> unmatchedDimItr = unmatchedDims.iterator();
@@ -158,11 +201,10 @@ public abstract class IndexMatcher {
             Map<Integer, DeriveInfo> needDeriveCollector, Iterator<Integer> unmatchedDimItr, Integer unmatchedDim) {
         JoinDesc joinByPKSide = model.getJoinByPKSide(unmatchedDim);
         Preconditions.checkNotNull(joinByPKSide);
-        val alias = joinByPKSide.getPKSide().getAlias();
+        String alias = joinByPKSide.getPKSide().getAlias();
         List<Integer> foreignKeyColumns = foreignKeyColumnIds.get(alias);
         List<Integer> primaryKeyColumns = primaryKeyColumnIds.get(alias);
 
-        val tables = model.getAliasMap();
         if (joinByPKSide.isInnerJoin() && primaryKeyColumns.contains(unmatchedDim)) {
             Integer relatedCol = foreignKeyColumns.get(primaryKeyColumns.indexOf(unmatchedDim));
             if (indexEntity.dimensionsDerive(relatedCol)) {
@@ -171,10 +213,11 @@ public abstract class IndexMatcher {
                 unmatchedDimItr.remove();
                 return true;
             }
-        } else if (indexEntity.dimensionsDerive(foreignKeyColumns) && model.getColRef(unmatchedDim) != null
-                && Optional.ofNullable(tables.get(alias))
-                        .map(ref -> StringUtils.isNotEmpty(ref.getTableDesc().getLastSnapshotPath())).orElse(false)) {
-
+        } else if (indexEntity.dimensionsDerive(foreignKeyColumns) //
+                && model.getColRef(unmatchedDim) != null //
+                && Optional.ofNullable(model.getAliasMap().get(alias))
+                        .map(ref -> ref.getTableDesc().getLastSnapshotPath()).filter(StringUtils::isNotBlank)
+                        .isPresent()) {
             DeriveInfo.DeriveType deriveType = matchNonEquiJoinFks(indexEntity, joinByPKSide)
                     ? DeriveInfo.DeriveType.LOOKUP_NON_EQUI
                     : DeriveInfo.DeriveType.LOOKUP;
@@ -196,7 +239,6 @@ public abstract class IndexMatcher {
     @RequiredArgsConstructor
     public static class MatchResult {
 
-        @NonNull
         boolean isMatched;
 
         Map<Integer, DeriveInfo> needDerive = Maps.newHashMap();
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/cuboid/NQueryLayoutChooser.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/cuboid/NQueryLayoutChooser.java
index ccada17bd7..72b61003f3 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/cuboid/NQueryLayoutChooser.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/cuboid/NQueryLayoutChooser.java
@@ -32,13 +32,16 @@ import org.apache.commons.collections.MapUtils;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.SegmentOnlineMode;
 import org.apache.kylin.common.exception.KylinTimeoutException;
-import org.apache.kylin.metadata.MetadataExtension;
 import org.apache.kylin.metadata.cube.model.IndexEntity;
+import org.apache.kylin.metadata.cube.model.IndexPlan;
 import org.apache.kylin.metadata.cube.model.LayoutEntity;
 import org.apache.kylin.metadata.cube.model.NDataLayout;
 import org.apache.kylin.metadata.cube.model.NDataSegment;
 import org.apache.kylin.metadata.cube.model.NDataflow;
+import org.apache.kylin.metadata.model.AntiFlatChecker;
+import org.apache.kylin.metadata.model.ColExcludedChecker;
 import org.apache.kylin.metadata.model.DeriveInfo;
+import org.apache.kylin.metadata.model.NDataModel;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.project.NProjectManager;
 import org.apache.kylin.metadata.realization.CapabilityResult;
@@ -61,14 +64,14 @@ public class NQueryLayoutChooser {
     }
 
     public static NLayoutCandidate selectPartialLayoutCandidate(NDataflow dataflow, List<NDataSegment> prunedSegments,
-            SQLDigest sqlDigest, Map<String, Set<Long>> secondStorageSegmentLayoutMap) {
+            SQLDigest sqlDigest, Map<String, Set<Long>> chSegmentToLayoutsMap) {
 
         NLayoutCandidate candidate = null;
         List<NDataSegment> toRemovedSegments = Lists.newArrayList();
         for (NDataSegment segment : prunedSegments) {
             if (candidate == null) {
                 candidate = selectLayoutCandidate(dataflow, Lists.newArrayList(segment), sqlDigest,
-                        secondStorageSegmentLayoutMap);
+                        chSegmentToLayoutsMap);
                 if (candidate == null) {
                     toRemovedSegments.add(segment);
                 }
@@ -81,49 +84,58 @@ public class NQueryLayoutChooser {
     }
 
     public static NLayoutCandidate selectLayoutCandidate(NDataflow dataflow, List<NDataSegment> prunedSegments,
-            SQLDigest sqlDigest, Map<String, Set<Long>> secondStorageSegmentLayoutMap) {
+            SQLDigest sqlDigest, Map<String, Set<Long>> chSegmentToLayoutsMap) {
 
         if (CollectionUtils.isEmpty(prunedSegments)) {
             log.info("There is no segment to answer sql");
             return NLayoutCandidate.EMPTY;
         }
-        List<NLayoutCandidate> candidates = new ArrayList<>();
-        val commonLayouts = getLayoutsFromSegments(prunedSegments, dataflow, secondStorageSegmentLayoutMap);
-        val model = dataflow.getModel();
-        log.info("Matching dataflow with seg num: {} layout num: {}", prunedSegments.size(), commonLayouts.size());
-        KylinConfig config = KylinConfig.getInstanceFromEnv();
-        Set<String> excludedTables = MetadataExtension.getFactory().getQueryExcludedTablesExtension()
-                .getExcludedTables(config, model.getProject());
-        boolean isReplaceCount = config.isReplaceColCountWithCountStar();
-        val indexPlan = dataflow.getIndexPlan();
-        val chooserContext = new ChooserContext(model);
-        val aggIndexMatcher = new AggIndexMatcher(sqlDigest, chooserContext, excludedTables, isReplaceCount);
-        val tableIndexMatcher = new TableIndexMatcher(sqlDigest, chooserContext, excludedTables,
-                dataflow.getConfig().isUseTableIndexAnswerNonRawQuery());
+
+        String project = dataflow.getProject();
+        NDataModel model = dataflow.getModel();
+        KylinConfig projectConfig = NProjectManager.getProjectConfig(project);
+        ChooserContext chooserContext = new ChooserContext(model);
+        ColExcludedChecker excludedChecker = new ColExcludedChecker(projectConfig, project, model);
+        if (log.isDebugEnabled()) {
+            log.debug("When matching layouts, all deduced excluded columns are: {}",
+                    excludedChecker.getExcludedColNames());
+        }
+        AntiFlatChecker antiFlatChecker = new AntiFlatChecker(model.getJoinTables(), model);
+        if (log.isDebugEnabled()) {
+            log.debug("When matching layouts, all deduced anti-flatten lookup tables are: {}",
+                    antiFlatChecker.getAntiFlattenLookups());
+        }
+
+        AggIndexMatcher aggIndexMatcher = new AggIndexMatcher(sqlDigest, chooserContext, dataflow, excludedChecker,
+                antiFlatChecker);
+        TableIndexMatcher tableIndexMatcher = new TableIndexMatcher(sqlDigest, chooserContext, dataflow,
+                excludedChecker, antiFlatChecker);
+
         // bail out if both agg index are invalid
-        // invalid matcher may caused by
+        //  matcher may be caused by
         // 1. cc col is not present in the model
         // 2. dynamic params ? present in query like select sum(col/?) from ...,
         //    see org.apache.kylin.query.DynamicQueryTest.testDynamicParamOnAgg
-        if (!aggIndexMatcher.valid() && !tableIndexMatcher.valid()) {
+        if (!aggIndexMatcher.isValid() && !tableIndexMatcher.isValid()) {
             return null;
         }
-        val projectInstance = NProjectManager.getInstance(KylinConfig.getInstanceFromEnv())
-                .getProject(dataflow.getProject());
+
+        IndexPlan indexPlan = dataflow.getIndexPlan();
+        List<NLayoutCandidate> candidates = new ArrayList<>();
+        Collection<NDataLayout> commonLayouts = getLayoutsFromSegments(prunedSegments, dataflow, chSegmentToLayoutsMap);
+        log.info("Matching dataflow with seg num: {} layout num: {}", prunedSegments.size(), commonLayouts.size());
         for (NDataLayout dataLayout : commonLayouts) {
             log.trace("Matching layout {}", dataLayout);
-            CapabilityResult tempResult = new CapabilityResult();
-            // check indexEntity
             IndexEntity indexEntity = indexPlan.getIndexEntity(dataLayout.getIndexId());
-            LayoutEntity layout = indexPlan.getLayoutEntity(dataLayout.getLayoutId());
             log.trace("Matching indexEntity {}", indexEntity);
 
+            LayoutEntity layout = indexPlan.getLayoutEntity(dataLayout.getLayoutId());
             NLayoutCandidate candidate = new NLayoutCandidate(layout);
-            var matchResult = tableIndexMatcher.match(layout);
+            IndexMatcher.MatchResult matchResult = tableIndexMatcher.match(layout);
             double influenceFactor = 1.0;
             if (!matchResult.isMatched()) {
                 matchResult = aggIndexMatcher.match(layout);
-            } else if (projectInstance.getConfig().useTableIndexAnswerSelectStarEnabled()) {
+            } else if (projectConfig.useTableIndexAnswerSelectStarEnabled()) {
                 influenceFactor += tableIndexMatcher.getLayoutUnmatchedColsSize();
                 candidate.setLayoutUnmatchedColsSize(tableIndexMatcher.getLayoutUnmatchedColsSize());
             }
@@ -132,6 +144,7 @@ public class NQueryLayoutChooser {
                 continue;
             }
 
+            CapabilityResult tempResult = new CapabilityResult();
             tempResult.influences = matchResult.getInfluences();
             candidate.setCost(dataLayout.getRows() * (tempResult.influences.size() + influenceFactor));
             if (!matchResult.getNeedDerive().isEmpty()) {
@@ -157,26 +170,24 @@ public class NQueryLayoutChooser {
     }
 
     private static Collection<NDataLayout> getLayoutsFromSegments(List<NDataSegment> segments, NDataflow dataflow,
-            Map<String, Set<Long>> secondStorageSegmentLayoutMap) {
-        KylinConfig config = KylinConfig.getInstanceFromEnv();
-        val projectInstance = NProjectManager.getInstance(config).getProject(dataflow.getProject());
-        if (!projectInstance.getConfig().isHeterogeneousSegmentEnabled()) {
+            Map<String, Set<Long>> chSegmentToLayoutsMap) {
+        KylinConfig projectConfig = NProjectManager.getProjectConfig(dataflow.getProject());
+        if (!projectConfig.isHeterogeneousSegmentEnabled()) {
             return dataflow.getLatestReadySegment().getLayoutsMap().values();
         }
 
-        val commonLayouts = Maps.<Long, NDataLayout> newHashMap();
+        Map<Long, NDataLayout> commonLayouts = Maps.newHashMap();
         if (CollectionUtils.isEmpty(segments)) {
             return commonLayouts.values();
         }
 
+        String segmentOnlineMode = projectConfig.getKylinEngineSegmentOnlineMode();
         for (int i = 0; i < segments.size(); i++) {
             val dataSegment = segments.get(i);
             var layoutIdMapToDataLayout = dataSegment.getLayoutsMap();
-            if (SegmentOnlineMode.ANY.toString()
-                    .equalsIgnoreCase(projectInstance.getConfig().getKylinEngineSegmentOnlineMode())
-                    && MapUtils.isNotEmpty(secondStorageSegmentLayoutMap)) {
-                Set<Long> chLayouts = secondStorageSegmentLayoutMap.getOrDefault(dataSegment.getId(),
-                        Sets.newHashSet());
+            if (SegmentOnlineMode.ANY.toString().equalsIgnoreCase(segmentOnlineMode)
+                    && MapUtils.isNotEmpty(chSegmentToLayoutsMap)) {
+                Set<Long> chLayouts = chSegmentToLayoutsMap.getOrDefault(dataSegment.getId(), Sets.newHashSet());
                 Map<Long, NDataLayout> nDataLayoutMap = chLayouts.stream()
                         .map(id -> NDataLayout.newDataLayout(dataflow, dataSegment.getId(), id))
                         .collect(Collectors.toMap(NDataLayout::getLayoutId, nDataLayout -> nDataLayout));
@@ -211,8 +222,10 @@ public class NQueryLayoutChooser {
                 .collect(Collectors.toList());
 
         Ordering<NLayoutCandidate> ordering = Ordering //
-                .from(priorityLayoutComparator()).compound(derivedLayoutComparator()).compound(rowSizeComparator()) // L1 comparator, compare cuboid rows
-                .compound(filterColumnComparator(filterColIds, chooserContext)) // L2 comparator, order filter columns
+                .from(priorityLayoutComparator()) //
+                .compound(derivedLayoutComparator()) //
+                .compound(rowSizeComparator()) // L1 comparator, compare cuboid rows
+                .compound(filterColumnComparator(filterColIds)) // L2 comparator, order filter columns
                 .compound(dimensionSizeComparator()) // the lower dimension the best
                 .compound(measureSizeComparator()) // L3 comparator, order size of cuboid columns
                 .compound(nonFilterColumnComparator(nonFilterColIds)); // L4 comparator, order non-filter columns
@@ -236,15 +249,20 @@ public class NQueryLayoutChooser {
     }
 
     private static Comparator<NLayoutCandidate> derivedLayoutComparator() {
-        return (layoutCandidate1, layoutCandidate2) -> {
-            if (layoutCandidate1.getDerivedToHostMap().isEmpty() && !layoutCandidate2.getDerivedToHostMap().isEmpty()) {
-                return -1;
-            } else if (!layoutCandidate1.getDerivedToHostMap().isEmpty()
-                    && layoutCandidate2.getDerivedToHostMap().isEmpty()) {
-                return 1;
+        return (candidate1, candidate2) -> {
+            int result = 0;
+            if (candidate1.getDerivedToHostMap().isEmpty() && !candidate2.getDerivedToHostMap().isEmpty()) {
+                result = -1;
+            } else if (!candidate1.getDerivedToHostMap().isEmpty() && candidate2.getDerivedToHostMap().isEmpty()) {
+                result = 1;
             }
 
-            return 0;
+            IndexPlan indexPlan = candidate1.getLayoutEntity().getIndex().getIndexPlan();
+            KylinConfig config = indexPlan.getConfig();
+            if (config.isTableExclusionEnabled() && config.isSnapshotPreferred()) {
+                result = -1 * result;
+            }
+            return result;
         };
     }
 
@@ -263,10 +281,9 @@ public class NQueryLayoutChooser {
     /**
      * compare filters in SQL with layout dims
      * 1. choose the layout if its shardby column is found in filters
-     * 2. otherwise compare position of filter columns appear in the layout dims
+     * 2. otherwise, compare position of filter columns appear in the layout dims
      */
-    private static Comparator<NLayoutCandidate> filterColumnComparator(List<Integer> sortedFilters,
-            ChooserContext chooserContext) {
+    private static Comparator<NLayoutCandidate> filterColumnComparator(List<Integer> sortedFilters) {
         return Ordering.from(shardByComparator(sortedFilters)).compound(colComparator(sortedFilters));
     }
 
@@ -304,32 +321,25 @@ public class NQueryLayoutChooser {
      */
     private static Comparator<NLayoutCandidate> shardByComparator(List<Integer> columns) {
         return (candidate1, candidate2) -> {
-            int shardByCol1Idx = Integer.MAX_VALUE;
-            List<Integer> shardByCols1 = candidate1.getLayoutEntity().getShardByColumns();
-            if (CollectionUtils.isNotEmpty(shardByCols1)) {
-                int tmpCol = shardByCols1.get(0);
-                for (int i = 0; i < columns.size(); i++) {
-                    if (columns.get(i) == tmpCol) {
-                        shardByCol1Idx = i;
-                        break;
-                    }
-                }
-            }
+            int shardByCol1Idx = getShardByColIndex(candidate1, columns);
+            int shardByCol2Idx = getShardByColIndex(candidate2, columns);
+            return shardByCol1Idx - shardByCol2Idx;
+        };
+    }
 
-            int shardByCol2Idx = Integer.MAX_VALUE;
-            List<Integer> shardByCols2 = candidate2.getLayoutEntity().getShardByColumns();
-            if (CollectionUtils.isNotEmpty(shardByCols2)) {
-                int tmpCol = shardByCols2.get(0);
-                for (int i = 0; i < columns.size(); i++) {
-                    if (columns.get(i) == tmpCol) {
-                        shardByCol2Idx = i;
-                        break;
-                    }
+    private static int getShardByColIndex(NLayoutCandidate candidate1, List<Integer> columns) {
+        int shardByCol1Idx = Integer.MAX_VALUE;
+        List<Integer> shardByCols1 = candidate1.getLayoutEntity().getShardByColumns();
+        if (CollectionUtils.isNotEmpty(shardByCols1)) {
+            int tmpCol = shardByCols1.get(0);
+            for (int i = 0; i < columns.size(); i++) {
+                if (columns.get(i) == tmpCol) {
+                    shardByCol1Idx = i;
+                    break;
                 }
             }
-
-            return shardByCol1Idx - shardByCol2Idx;
-        };
+        }
+        return shardByCol1Idx;
     }
 
     private static List<Integer> getColumnsPos(final NLayoutCandidate candidate, List<Integer> sortedColumns) {
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/cuboid/TableIndexMatcher.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/cuboid/TableIndexMatcher.java
index f3eaee1b36..f8ebcd1cef 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/cuboid/TableIndexMatcher.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/cuboid/TableIndexMatcher.java
@@ -23,73 +23,61 @@ import java.util.Map;
 import java.util.Set;
 import java.util.stream.Collectors;
 
-import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.metadata.cube.model.IndexEntity;
 import org.apache.kylin.metadata.cube.model.LayoutEntity;
 import org.apache.kylin.metadata.cube.model.NDataflow;
-import org.apache.kylin.metadata.cube.model.NDataflowManager;
+import org.apache.kylin.metadata.model.AntiFlatChecker;
+import org.apache.kylin.metadata.model.ColExcludedChecker;
 import org.apache.kylin.metadata.model.DeriveInfo;
 import org.apache.kylin.metadata.project.NProjectManager;
-import org.apache.kylin.metadata.project.ProjectInstance;
 import org.apache.kylin.metadata.realization.CapabilityResult;
 import org.apache.kylin.metadata.realization.SQLDigest;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
 
 import lombok.extern.slf4j.Slf4j;
 
 @Slf4j
 public class TableIndexMatcher extends IndexMatcher {
 
-    private final boolean isUseTableIndexAnswerNonRawQuery;
-    private Set<Integer> sqlColumns;
-    private final boolean valid;
     private int layoutUnmatchedColsSize;
 
-    public TableIndexMatcher(SQLDigest sqlDigest, ChooserContext chooserContext, Set<String> excludedTables,
-            boolean isUseTableIndexAnswerNonRawQuery) {
-        super(sqlDigest, chooserContext, excludedTables);
-        this.isUseTableIndexAnswerNonRawQuery = isUseTableIndexAnswerNonRawQuery;
-        valid = init();
+    public TableIndexMatcher(SQLDigest sqlDigest, ChooserContext chooserContext, NDataflow dataflow,
+            ColExcludedChecker excludedChecker, AntiFlatChecker antiFlatChecker) {
+        super(sqlDigest, chooserContext, dataflow, excludedChecker, antiFlatChecker);
         this.layoutUnmatchedColsSize = 0;
+        this.valid = fastValidCheckBeforeMatch();
     }
 
-    private boolean init() {
+    @Override
+    protected boolean fastValidCheckBeforeMatch() {
         // cols may have null values as the CC col in query may not present in the model
         sqlColumns = sqlDigest.allColumns.stream().map(tblColMap::get).collect(Collectors.toSet());
         return !sqlColumns.contains(null);
     }
 
-    public boolean valid() {
-        return valid;
-    }
-
+    @Override
     public MatchResult match(LayoutEntity layout) {
-        if (!needTableIndexMatch(layout.getIndex()) || !valid) {
-            return new MatchResult(false);
+        if (canSkipIndexMatch(layout.getIndex()) || !isValid()) {
+            return new MatchResult();
         }
 
         log.trace("Matching table index");
         final Map<Integer, DeriveInfo> needDerive = Maps.newHashMap();
-        Set<Integer> unmatchedCols = Sets.newHashSet();
-        unmatchedCols.addAll(sqlColumns);
-        if (isBatchFusionModel) {
-            unmatchedCols.removeAll(layout.getStreamingColumns().keySet());
-        }
-        unmatchedCols.removeAll(layout.getOrderedDimensions().keySet());
-        ProjectInstance projectInstance = NProjectManager.getInstance(KylinConfig.getInstanceFromEnv())
-                .getProject(model.getProject());
-        if (projectInstance.getConfig().useTableIndexAnswerSelectStarEnabled()) {
+        Set<Integer> unmatchedCols = initUnmatchedColumnIds(layout);
+        if (NProjectManager.getProjectConfig(project).useTableIndexAnswerSelectStarEnabled()) {
             layoutUnmatchedColsSize = unmatchedCols.size();
-            NDataflowManager dataflowManager = NDataflowManager.getInstance(KylinConfig.getInstanceFromEnv(),
-                    model.getProject());
-            NDataflow dataflow = dataflowManager.getDataflow(layout.getModel().getId());
             unmatchedCols.removeAll(dataflow.getAllColumnsIndex());
         }
         goThruDerivedDims(layout.getIndex(), needDerive, unmatchedCols);
-        if (!unmatchedCols.isEmpty()) {
+        boolean isMatch = unmatchedCols.isEmpty();
+        if (!isMatch) {
+            unmatchedCols.removeAll(filterExcludedDims(layout));
+            log.debug("After rolling back to TableIndex to match, the unmatched columns are: {}", unmatchedCols);
+            isMatch = unmatchedCols.isEmpty();
+        }
+        if (!isMatch) {
             if (log.isDebugEnabled()) {
                 log.debug("Table index {} with unmatched columns {}", layout, unmatchedCols);
             }
@@ -100,9 +88,11 @@ public class TableIndexMatcher extends IndexMatcher {
         return new MatchResult(true, needDerive);
     }
 
-    private boolean needTableIndexMatch(IndexEntity index) {
-        boolean isUseTableIndex = isUseTableIndexAnswerNonRawQuery && !nonSupportFunTableIndex(sqlDigest.aggregations);
-        return index.isTableIndex() && (sqlDigest.isRawQuery || isUseTableIndex);
+    @Override
+    protected boolean canSkipIndexMatch(IndexEntity index) {
+        boolean isUseTableIndex = dataflow.getConfig().isUseTableIndexAnswerNonRawQuery()
+                && !nonSupportFunTableIndex(sqlDigest.aggregations);
+        return !index.isTableIndex() || (!sqlDigest.isRawQuery && !isUseTableIndex);
     }
 
     public int getLayoutUnmatchedColsSize() {
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/NBatchConstants.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/NBatchConstants.java
index 636936a66d..a7b1260ce8 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/NBatchConstants.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/NBatchConstants.java
@@ -35,7 +35,6 @@ public interface NBatchConstants {
     String P_TARGET_MODEL = "targetModel";
     String P_DATA_RANGE_START = "dataRangeStart";
     String P_DATA_RANGE_END = "dataRangeEnd";
-    String P_EXCLUDED_TABLES = "excludedTables";
 
     String P_IGNORED_SNAPSHOT_TABLES = "ignoredSnapshotTables";
     String P_NEED_BUILD_SNAPSHOTS = "needBuildSnapshots";
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/NDataSegment.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/NDataSegment.java
index a3abc92b37..eb7585347f 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/NDataSegment.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/NDataSegment.java
@@ -49,7 +49,6 @@ import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
 
 import lombok.Getter;
 import lombok.Setter;
@@ -139,8 +138,6 @@ public class NDataSegment implements ISegment, Serializable {
     @Getter
     private long maxBucketId = -1L;
 
-    private transient Set<String> excludedTables = Sets.newHashSet();
-
     // computed fields below
     // transient,generated by multiPartitionData
     @Getter
@@ -174,7 +171,6 @@ public class NDataSegment implements ISegment, Serializable {
         this.lastBuildTime = other.lastBuildTime;
         this.sourceCount = other.sourceCount;
         this.additionalInfo = other.additionalInfo;
-        this.excludedTables = other.excludedTables;
         this.isSnapshotReady = other.isSnapshotReady;
         this.isDictReady = other.isDictReady;
         this.isFlatTableReady = other.isFlatTableReady;
@@ -536,16 +532,6 @@ public class NDataSegment implements ISegment, Serializable {
         this.additionalInfo = additionalInfo;
     }
 
-    public Set<String> getExcludedTables() {
-        return excludedTables;
-    }
-
-    public void setExcludedTables(Set<String> excludedTables) {
-        if (!KylinConfig.getInstanceFromEnv().isBuildExcludedTableEnabled()) {
-            this.excludedTables = excludedTables;
-        }
-    }
-
     public long getSourceCount() {
         if (CollectionUtils.isEmpty(multiPartitions)) {
             return sourceCount;
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/favorite/FavoriteRuleManager.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/favorite/FavoriteRuleManager.java
index b7888eaf14..3fbcd7eb63 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/favorite/FavoriteRuleManager.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/favorite/FavoriteRuleManager.java
@@ -24,11 +24,12 @@ import java.util.Locale;
 import java.util.Set;
 import java.util.stream.Collectors;
 
+import org.apache.commons.collections.CollectionUtils;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.annotation.Clarification;
 import org.apache.kylin.common.persistence.ResourceStore;
-import org.apache.kylin.metadata.cachesync.CachedCrudAssist;
 import org.apache.kylin.common.persistence.transaction.UnitOfWork;
+import org.apache.kylin.metadata.cachesync.CachedCrudAssist;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -158,12 +159,14 @@ public class FavoriteRuleManager {
         return enabledRules;
     }
 
+    // only used by upgrade tool
     public Set<String> getExcludedTables() {
         FavoriteRule favoriteRule = getOrDefaultByName(FavoriteRule.EXCLUDED_TABLES_RULE);
-        if (!favoriteRule.isEnabled()) {
+        List<FavoriteRule.AbstractCondition> conditions = favoriteRule.getConds();
+        if (CollectionUtils.isEmpty(conditions)) {
             return Sets.newHashSet();
         }
-        FavoriteRule.Condition condition = (FavoriteRule.Condition) favoriteRule.getConds().get(0);
+        FavoriteRule.Condition condition = (FavoriteRule.Condition) conditions.get(0);
         return Arrays.stream(condition.getRightThreshold().split(",")) //
                 .map(table -> table.toUpperCase(Locale.ROOT)).collect(Collectors.toSet());
     }
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/ExcludedLookupChecker.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/AntiFlatChecker.java
similarity index 59%
rename from src/core-metadata/src/main/java/org/apache/kylin/metadata/model/ExcludedLookupChecker.java
rename to src/core-metadata/src/main/java/org/apache/kylin/metadata/model/AntiFlatChecker.java
index aa32c8d4c0..4aedb5bd2f 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/ExcludedLookupChecker.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/AntiFlatChecker.java
@@ -23,7 +23,6 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.commons.collections.CollectionUtils;
-import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.JsonUtil;
 import org.apache.kylin.metadata.cube.model.IndexPlan;
 
@@ -31,32 +30,25 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
-/**
- * There are two kinds of excluded lookup tables.
- * One kind is explicit excluded by project settings,
- * another is excluded by model join relations.
- */
-public class ExcludedLookupChecker {
-    private final String factTable;
-    private final Set<String> excludedLookups = Sets.newHashSet();
+import lombok.Getter;
+
+public class AntiFlatChecker {
+    @Getter
     private final Set<String> antiFlattenLookups = Sets.newHashSet();
-    private final Map<String, String> excludedLookupCCs = Maps.newHashMap();
     private final Map<String, String> antiFlattenLookupCCs = Maps.newHashMap();
     private final Map<String, Set<String>> joinTableAliasMap = Maps.newHashMap();
 
-    public ExcludedLookupChecker(Set<String> excludedTables, List<JoinTableDesc> joinTables, NDataModel model) {
-        if (KylinConfig.getInstanceFromEnv().isUTEnv() && model == null) {
-            factTable = null;
-            return;
-        }
-        factTable = model.getRootFactTableName();
-        excludedTables.forEach(table -> {
-            if (table.equalsIgnoreCase(factTable)) {
-                return;
-            }
-            excludedLookups.add(table);
-        });
-        if (model.isBroken() || CollectionUtils.isEmpty(model.getJoinTables())) {
+    /**
+     * Initialize a checker to handle lookup table which has not been built into index.
+     * 
+     * @param joinTables joinTables must get from a model which has been initialized.
+     *                   This parameter equals to {@link NDataModel#getJoinTables()} of the
+     *                   second parameter model at most case, however, if we add/update/remove
+     *                   some join tables by editing the model, they are different.
+     * @param model the model to check.
+     */
+    public AntiFlatChecker(List<JoinTableDesc> joinTables, NDataModel model) {
+        if (model == null || model.isBroken() || CollectionUtils.isEmpty(model.getJoinTables())) {
             return;
         }
 
@@ -65,112 +57,108 @@ public class ExcludedLookupChecker {
             joinTableAliasMap.get(join.getTable()).add(join.getAlias());
         });
 
-        Map<String, String> aliasToIdentityMap = Maps.newHashMap();
         if (joinTables == null) {
             return;
         }
+        Map<String, String> aliasToIdentityMap = Maps.newHashMap();
         joinTables.forEach(joinTable -> {
             aliasToIdentityMap.put(joinTable.getAlias(), joinTable.getTable());
             if (!joinTable.isFlattenable()) {
                 antiFlattenLookups.add(joinTable.getTable());
             }
         });
-        for (JoinTableDesc joinTable : joinTables) {
+        joinTables.forEach(joinTable -> {
             TblColRef[] fkColumns = joinTable.getJoin().getForeignKeyColumns();
             TableRef foreignTableRef = joinTable.getJoin().getForeignTableRef();
-            String fkTableAlias;
-            if (fkColumns.length > 0) {
+            if (fkColumns != null && fkColumns.length > 0) {
                 TblColRef firstFK = fkColumns[0];
-                fkTableAlias = firstFK.getTableAlias();
-                if (canTreatAsAntiFlattenableLookup(aliasToIdentityMap, joinTable, firstFK.getTableAlias(),
-                        firstFK.getTableWithSchema())) {
+                String tableAlias = firstFK.getTableAlias();
+                String tableWithSchema = firstFK.getTableWithSchema();
+                if (canTreatAsAntiLookup(aliasToIdentityMap, joinTable, tableAlias, tableWithSchema)) {
                     antiFlattenLookups.add(joinTable.getTable());
-                    excludedLookups.add(joinTable.getTable());
                 }
             } else if (foreignTableRef != null) {
-                fkTableAlias = foreignTableRef.getAlias();
-                if (canTreatAsAntiFlattenableLookup(aliasToIdentityMap, joinTable, foreignTableRef.getAlias(),
-                        foreignTableRef.getTableIdentity())) {
+                String tableIdentity = foreignTableRef.getTableIdentity();
+                String tableAlias = foreignTableRef.getAlias();
+                if (canTreatAsAntiLookup(aliasToIdentityMap, joinTable, tableAlias, tableIdentity)) {
                     antiFlattenLookups.add(joinTable.getTable());
-                    excludedLookups.add(joinTable.getTable());
-                }
-            } else {
-                fkTableAlias = null;
-            }
-
-            if (aliasToIdentityMap.containsKey(fkTableAlias)) {
-                String fkTable = aliasToIdentityMap.get(fkTableAlias);
-                if (excludedLookups.contains(fkTable)) {
-                    excludedLookups.add(joinTable.getTable());
                 }
             }
-        }
+        });
     }
 
-    private boolean canTreatAsAntiFlattenableLookup(Map<String, String> aliasToIdentityMap, JoinTableDesc joinTable,
+    private boolean canTreatAsAntiLookup(Map<String, String> aliasToIdentityMap, JoinTableDesc joinTable,
             String fkTableAlias, String fkTableIdentity) {
         return !joinTable.isFlattenable() //
                 || (aliasToIdentityMap.containsKey(fkTableAlias) && antiFlattenLookups.contains(fkTableIdentity));
     }
 
-    /**
-     * not very efficient for cc inner expression is a string
-     */
-    public boolean isColRefDependsLookupTable(TblColRef tblColRef) {
-        return isColDependsLookups(tblColRef, excludedLookups, excludedLookupCCs);
+    public boolean isColOfAntiLookup(TblColRef colRef) {
+        if (!colRef.getColumnDesc().isComputedColumn()) {
+            return antiFlattenLookups.contains(colRef.getTableWithSchema());
+        }
+        String innerExpression = colRef.getColumnDesc().getComputedColumnExpr();
+        return isCCOfAntiLookup(innerExpression);
     }
 
-    public boolean isCCDependsLookupTable(TblColRef tblColRef) {
+    public boolean isCCOfAntiLookup(TblColRef tblColRef) {
         List<TblColRef> operands = tblColRef.getOperands();
         if (operands == null) {
             if (tblColRef.getTable() == null) {
                 return false;
             } else {
-                return excludedLookups.contains(tblColRef.getTableWithSchema());
+                return antiFlattenLookups.contains(tblColRef.getTableWithSchema());
             }
         }
         for (TblColRef colRef : operands) {
-            if (isCCDependsLookupTable(colRef)) {
+            if (isCCOfAntiLookup(colRef)) {
                 return true;
             }
         }
         return false;
     }
 
-    public boolean isMeasureOnLookupTable(FunctionDesc functionDesc) {
+    public boolean isCCOfAntiLookup(String innerExp) {
+        if (antiFlattenLookupCCs.containsKey(innerExp)) {
+            return true;
+        }
+
+        for (String table : antiFlattenLookups) {
+            Set<String> aliasSet = joinTableAliasMap.get(table);
+            if (aliasSet == null) {
+                continue;
+            }
+            for (String alias : aliasSet) {
+                String aliasWithBacktick = String.format(Locale.ROOT, "`%s`", alias);
+                if (innerExp.contains(aliasWithBacktick)) {
+                    antiFlattenLookupCCs.putIfAbsent(innerExp, alias);
+                    return true;
+                }
+            }
+        }
+        return false;
+    }
+
+    public boolean isMeasureOfAntiLookup(FunctionDesc functionDesc) {
         List<TblColRef> colRefs = functionDesc.getColRefs();
         if (colRefs == null || colRefs.isEmpty()) {
             return false;
         }
         for (TblColRef colRef : colRefs) {
             if (colRef.isInnerColumn()) {
-                if (isCCDependsLookupTable(colRef)) {
+                if (isCCOfAntiLookup(colRef)) {
                     return true;
                 }
-            } else if (isColRefDependsLookupTable(colRef)) {
+            } else if (isColOfAntiLookup(colRef)) {
                 return true;
             }
         }
         return false;
     }
 
-    /**
-     * For computed column is very difficult to get the excluded lookup table, so handle
-     * it in the step of IndexSuggester#replaceDimOfLookupTableWithFK.
-     */
-    public Set<String> getUsedExcludedLookupTable(Set<TblColRef> colRefs) {
-        Set<String> usedExcludedLookupTables = Sets.newHashSet();
-        for (TblColRef column : colRefs) {
-            if (excludedLookups.contains(column.getTableWithSchema())) {
-                usedExcludedLookupTables.add(column.getTableWithSchema());
-            }
-        }
-        return usedExcludedLookupTables;
-    }
-
     public String detectAntiFlattenLookup(ComputedColumnDesc computedColumn) {
         String innerExp = computedColumn.getInnerExpression();
-        if (isInnerExpDependsLookups(innerExp, antiFlattenLookups, antiFlattenLookupCCs)) {
+        if (isCCOfAntiLookup(innerExp)) {
             return antiFlattenLookupCCs.get(innerExp);
         }
         return null;
@@ -183,7 +171,7 @@ public class ExcludedLookupChecker {
 
         List<ComputedColumnDesc> ccList = Lists.newArrayList();
         for (ComputedColumnDesc cc : model.getComputedColumnDescs()) {
-            if (isInnerExpDependsLookups(cc.getInnerExpression(), antiFlattenLookups, antiFlattenLookupCCs)) {
+            if (isCCOfAntiLookup(cc.getInnerExpression())) {
                 ccList.add(JsonUtil.deepCopyQuietly(cc, ComputedColumnDesc.class));
             }
         }
@@ -201,7 +189,7 @@ public class ExcludedLookupChecker {
                 continue;
             }
             TblColRef colRef = model.getEffectiveCols().get(column.getId());
-            if (isColDependsAntiFlattenLookup(colRef)) {
+            if (isColOfAntiLookup(colRef)) {
                 dimensions.add(column.getId());
             }
         }
@@ -220,10 +208,7 @@ public class ExcludedLookupChecker {
             }
             List<ParameterDesc> parameters = measure.getFunction().getParameters();
             for (ParameterDesc parameter : parameters) {
-                if (parameter.isConstant()) {
-                    continue;
-                }
-                if (isColDependsAntiFlattenLookup(parameter.getColRef())) {
+                if (!parameter.isConstant() && isColOfAntiLookup(parameter.getColRef())) {
                     measures.add(measure.getId());
                     break;
                 }
@@ -248,49 +233,8 @@ public class ExcludedLookupChecker {
         return indexes;
     }
 
-    public List<String> getAntiFlattenLookups() {
-        return Lists.newArrayList(antiFlattenLookups);
-    }
-
-    public Set<String> getExcludedLookups() {
-        return excludedLookups;
-    }
-
-    private boolean isColDependsAntiFlattenLookup(TblColRef colRef) {
-        return isColDependsLookups(colRef, antiFlattenLookups, antiFlattenLookupCCs);
-    }
-
-    private boolean isColDependsLookups(TblColRef colRef, Set<String> lookupTables, Map<String, String> cachedCC) {
-        if (!colRef.getColumnDesc().isComputedColumn()) {
-            return lookupTables.contains(colRef.getTableWithSchema());
-        }
-        String innerExpression = colRef.getColumnDesc().getComputedColumnExpr();
-        return isInnerExpDependsLookups(innerExpression, lookupTables, cachedCC);
-    }
-
-    private boolean isInnerExpDependsLookups(String innerExp, Set<String> lookupTables, Map<String, String> cachedCC) {
-        if (cachedCC.containsKey(innerExp)) {
-            return true;
-        }
-
-        for (String table : lookupTables) {
-            Set<String> aliasSet = joinTableAliasMap.get(table);
-            if (aliasSet == null) {
-                continue;
-            }
-            for (String alias : aliasSet) {
-                String aliasWithBacktick = String.format(Locale.ROOT, "`%s`", alias);
-                if (innerExp.contains(aliasWithBacktick)) {
-                    cachedCC.putIfAbsent(innerExp, alias);
-                    return true;
-                }
-            }
-        }
-        return false;
-    }
-
-    public String detectFilterConditionDependsLookups(String exp, Set<String> lookupTables) {
-        for (String table : lookupTables) {
+    public String detectFilterCondition(String exp) {
+        for (String table : antiFlattenLookups) {
             Set<String> aliasSet = joinTableAliasMap.get(table);
             if (aliasSet == null) {
                 continue;
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/ColExcludedChecker.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/ColExcludedChecker.java
new file mode 100644
index 0000000000..8fd68f26af
--- /dev/null
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/ColExcludedChecker.java
@@ -0,0 +1,177 @@
+/*
+ * 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.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.metadata.project.NProjectManager;
+import org.apache.kylin.metadata.project.ProjectInstance;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import lombok.Getter;
+import lombok.NoArgsConstructor;
+import lombok.NonNull;
+
+@NoArgsConstructor
+public class ColExcludedChecker {
+
+    @Getter
+    private final Set<ColumnDesc> excludedCols = Sets.newHashSet();
+    @Getter
+    private final Set<String> excludedColNames = Sets.newHashSet();
+    private final Map<String, TblColRef> colIdentityMap = Maps.newHashMap();
+
+    public ColExcludedChecker(KylinConfig config, String project, NDataModel model) {
+        ProjectInstance prjInstance = NProjectManager.getInstance(config).getProject(project);
+        if (!prjInstance.getConfig().isTableExclusionEnabled()) {
+            return;
+        }
+        NTableMetadataManager tableMgr = NTableMetadataManager.getInstance(config, project);
+        List<TableDesc> allTables = tableMgr.listAllTables();
+        Map<String, TableRef> identityToRefMap = Maps.newHashMap();
+        if (model != null && !model.isBroken()) {
+            for (JoinTableDesc joinTableDesc : model.getJoinTables()) {
+                TableRef tableRef = joinTableDesc.getTableRef();
+                identityToRefMap.put(tableRef.getTableIdentity(), tableRef);
+            }
+        }
+        Set<TableDesc> desiredTables = allTables.stream()
+                .filter(table -> model == null || identityToRefMap.containsKey(table.getIdentity()))
+                .collect(Collectors.toSet());
+        for (TableDesc table : desiredTables) {
+            TableExtDesc tableExt = tableMgr.getTableExtIfExists(table);
+            String factTable = model == null ? "" : model.getRootFactTableName();
+            if (StringUtils.equalsIgnoreCase(table.getIdentity(), factTable) || tableExt == null) {
+                continue;
+            }
+
+            Set<String> excludedSet = tableExt.getExcludedColumns();
+            List<ColumnDesc> list = Arrays.stream(table.getColumns()).filter(Objects::nonNull)
+                    .filter(col -> tableExt.isExcluded() || excludedSet.contains(col.getName()))
+                    .collect(Collectors.toList());
+            excludedCols.addAll(list);
+        }
+
+        // add excluded column from cc
+        collectExcludedComputedColumns(config, project, model);
+    }
+
+    private void collectExcludedComputedColumns(KylinConfig config, String project, NDataModel model) {
+        if (model == null || model.isBroken()) {
+            return;
+        }
+        model.init(config, project, Lists.newArrayList());
+        model.getAllTables().stream().filter(Objects::nonNull) //
+                .flatMap(tableRef -> tableRef.getColumns().stream())
+                .filter(tblColRef -> excludedCols.contains(tblColRef.getColumnDesc()))
+                .map(TblColRef::getBackTickIdentity).forEach(excludedColNames::add);
+        model.getEffectiveCols().forEach((id, colRef) -> colIdentityMap.put(colRef.getIdentity(), colRef));
+        model.getComputedColumnDescs().forEach(cc -> {
+            TblColRef tblColRef = colIdentityMap.get(cc.getFullName());
+            if (tblColRef == null) {
+                return;
+            }
+            ColumnDesc columnDesc = tblColRef.getColumnDesc();
+            if (isExcludedCC(cc.getInnerExpression())) {
+                excludedCols.add(columnDesc);
+                excludedColNames.add(columnDesc.getBackTickIdentity());
+            }
+        });
+    }
+
+    /**
+     * This method gives all the excluded columns without considering columns of computed columns.
+     * It is useful when using org.apache.kylin.query.util.ConvertToComputedColumn to transform query statement.
+     * If ConvertToComputedColumn is removed, this method is useless.
+     */
+    public Set<String> filterRelatedExcludedColumn(NDataModel model) {
+        String fact = model == null ? "" : model.getRootFactTableName();
+        return getExcludedCols().stream() //
+                .filter(col -> !col.getTable().getIdentity().equalsIgnoreCase(fact)) //
+                .map(ColumnDesc::getIdentity).collect(Collectors.toSet());
+    }
+
+    public boolean anyExcludedColMatch(Collection<TblColRef> tblColRefs) {
+        return tblColRefs.stream().anyMatch(this::isExcludedCol);
+    }
+
+    public boolean isExcludedCol(@NonNull TblColRef tblColRef) {
+        if (excludedCols.contains(tblColRef.getColumnDesc())) {
+            return true;
+        }
+        return isExcludedCC(tblColRef);
+    }
+
+    private boolean isExcludedCC(@NonNull TblColRef tblColRef) {
+        List<TblColRef> operands = tblColRef.getOperands();
+        if (operands == null) {
+            return false;
+        }
+        for (TblColRef colRef : operands) {
+            if (isExcludedCol(colRef)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    public boolean isExcludedCC(ComputedColumnDesc cc) {
+        return isExcludedCC(cc.getInnerExpression());
+    }
+
+    public boolean isExcludedCC(String innerExpression) {
+        if (StringUtils.isBlank(innerExpression)) {
+            return false;
+        }
+        String[] splits = innerExpression.split("\\s");
+        for (String split : splits) {
+            int begin = split.indexOf('`');
+            int end = split.lastIndexOf('`');
+            if (begin != -1 && excludedColNames.contains(split.substring(begin, end + 1))) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    public boolean isExcludedMeasure(@NonNull FunctionDesc functionDesc) {
+        List<TblColRef> colRefs = functionDesc.getColRefs();
+        if (CollectionUtils.isEmpty(colRefs)) {
+            return false;
+        }
+        for (TblColRef colRef : colRefs) {
+            if (isExcludedCol(colRef)) {
+                return true;
+            }
+        }
+        return false;
+    }
+}
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/ComputedColumnDesc.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/ComputedColumnDesc.java
index f807dea94d..6d642c4743 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/ComputedColumnDesc.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/ComputedColumnDesc.java
@@ -39,6 +39,7 @@ import org.apache.kylin.common.exception.KylinException;
 import org.apache.kylin.common.msg.MsgPicker;
 import org.apache.kylin.measure.MeasureTypeFactory;
 import org.apache.kylin.metadata.model.tool.CalciteParser;
+import org.apache.kylin.metadata.model.util.ComputedColumnUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -209,4 +210,7 @@ public class ComputedColumnDesc implements Serializable {
         return innerExpression;
     }
 
+    public boolean isAutoCC() {
+        return getColumnName().startsWith(ComputedColumnUtil.CC_NAME_PREFIX);
+    }
 }
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/NTableMetadataManager.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/NTableMetadataManager.java
index 05a28bca36..1556f6be66 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/NTableMetadataManager.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/NTableMetadataManager.java
@@ -212,6 +212,10 @@ public class NTableMetadataManager {
         return getOrCreateTableExt(t);
     }
 
+    public boolean isTableExtExist(String tableIdentity) {
+        return getTableExtIfExists(getTableDesc(tableIdentity)) != null;
+    }
+
     public TableExtDesc getOrCreateTableExt(TableDesc t) {
         TableExtDesc result = srcExtCrud.get(t.getIdentity());
 
@@ -262,6 +266,14 @@ public class NTableMetadataManager {
         saveTableExt(copyForWrite);
     }
 
+    public void saveOrUpdateTableExt(boolean isUpdate, TableExtDesc tableExt) {
+        if (isUpdate) {
+            mergeAndUpdateTableExt(tableExt, tableExt);
+        } else {
+            saveTableExt(tableExt);
+        }
+    }
+
     public void removeTableExt(String tableName) {
         TableExtDesc t = getTableExtIfExists(getTableDesc(tableName));
         if (t == null)
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableExtDesc.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableExtDesc.java
index 55aa40a75b..293dbd4646 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableExtDesc.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableExtDesc.java
@@ -22,16 +22,16 @@ import java.io.Serializable;
 import java.net.URI;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
+import java.util.Set;
 
+import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.persistence.RootPersistentEntity;
-import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.measure.hllc.HLLCounter;
 import org.apache.kylin.metadata.MetadataConstants;
 
@@ -41,6 +41,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
 
 import io.kyligence.kap.guava20.shaded.common.base.Strings;
 import lombok.Data;
@@ -49,7 +50,6 @@ import lombok.Getter;
 import lombok.Setter;
 import lombok.extern.slf4j.Slf4j;
 
-@SuppressWarnings("serial")
 @JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE)
 @Slf4j
 public class TableExtDesc extends RootPersistentEntity implements Serializable {
@@ -57,14 +57,10 @@ public class TableExtDesc extends RootPersistentEntity implements Serializable {
     public static final String S3_ROLE_PROPERTY_KEY = "s3_role";
     public static final String LOCATION_PROPERTY_KEY = "location";
     public static final String S3_ENDPOINT_KEY = "s3_endpoint";
+    public static final String SEPARATOR = "/";
 
     public static String concatRawResourcePath(String nameOnPath) {
-        return ResourceStore.TABLE_EXD_RESOURCE_ROOT + "/" + nameOnPath + ".json";
-    }
-
-    // returns <table, project>
-    public static Pair<String, String> parseResourcePath(String path) {
-        return TableDesc.parseResourcePath(path);
+        return ResourceStore.TABLE_EXD_RESOURCE_ROOT + SEPARATOR + nameOnPath + ".json";
     }
 
     // ============================================================================
@@ -79,11 +75,6 @@ public class TableExtDesc extends RootPersistentEntity implements Serializable {
     @JsonProperty("last_build_job_id")
     private String jodID;
 
-    @Getter
-    @Setter
-    @JsonProperty("frequency")
-    private int frequency;
-
     @Setter
     @JsonProperty("columns_stats")
     private List<ColumnStats> columnStats = new ArrayList<>(); // should not expose getter
@@ -93,20 +84,11 @@ public class TableExtDesc extends RootPersistentEntity implements Serializable {
     @JsonProperty("sample_rows")
     private List<String[]> sampleRows = new ArrayList<>();
 
-    @Getter
-    @Setter
-    @JsonProperty("last_modified_time")
-    private long lastModifiedTime;
-
     @Getter
     @Setter
     @JsonProperty("total_rows")
     private long totalRows;
 
-    @Setter
-    @JsonProperty("mapper_rows")
-    private List<Long> mapRecords = new ArrayList<>();
-
     @Getter
     @JsonProperty("data_source_properties")
     private Map<String, String> dataSourceProps = new HashMap<>();
@@ -114,16 +96,6 @@ public class TableExtDesc extends RootPersistentEntity implements Serializable {
     @Getter
     private String project;
 
-    @Getter
-    @Setter
-    @JsonProperty("loading_range")
-    private List<SegmentRange> loadingRange = new ArrayList<>();
-
-    @Setter
-    @Getter
-    @JsonProperty("col_stats_path")
-    private String colStatsPath;
-
     @Getter
     @Setter
     @JsonProperty("row_count_status")
@@ -139,6 +111,16 @@ public class TableExtDesc extends RootPersistentEntity implements Serializable {
     @JsonProperty("query_hit_count")
     private int snapshotHitCount = 0;
 
+    @Getter
+    @Setter
+    @JsonProperty("excluded")
+    private boolean excluded;
+
+    @Getter
+    @Setter
+    @JsonProperty("excluded_columns")
+    private Set<String> excludedColumns = Sets.newLinkedHashSet();
+
     public TableExtDesc() {
     }
 
@@ -147,16 +129,20 @@ public class TableExtDesc extends RootPersistentEntity implements Serializable {
         this.lastModified = other.lastModified;
         this.identity = other.identity;
         this.jodID = other.jodID;
-        this.frequency = other.frequency;
         this.columnStats = other.columnStats;
         this.sampleRows = other.sampleRows;
-        this.lastModifiedTime = other.lastModifiedTime;
         this.totalRows = other.totalRows;
-        this.mapRecords = other.mapRecords;
         this.dataSourceProps = other.dataSourceProps;
         this.project = other.project;
         this.originalSize = other.originalSize;
         this.snapshotHitCount = other.snapshotHitCount;
+        this.excluded = other.excluded;
+        this.excludedColumns = other.excludedColumns;
+        this.rowCountStatus = other.rowCountStatus; // no need any more, will be deleted later.
+    }
+
+    public boolean testExcluded(ColumnDesc column) {
+        return excluded || excludedColumns.contains(column.getName());
     }
 
     @Override
@@ -166,13 +152,8 @@ public class TableExtDesc extends RootPersistentEntity implements Serializable {
 
     @Override
     public String getResourcePath() {
-        return new StringBuilder().append("/").append(getProject()).append(ResourceStore.TABLE_EXD_RESOURCE_ROOT)
-                .append("/").append(getIdentity()).append(MetadataConstants.FILE_SURFIX).toString();
-    }
-
-    public void updateLoadingRange(final SegmentRange segmentRange) {
-        loadingRange.add(segmentRange);
-        Collections.sort(loadingRange);
+        return SEPARATOR + getProject() + ResourceStore.TABLE_EXD_RESOURCE_ROOT + SEPARATOR + getIdentity()
+                + MetadataConstants.FILE_SURFIX;
     }
 
     public void addDataSourceProp(String key, String value) {
@@ -213,7 +194,7 @@ public class TableExtDesc extends RootPersistentEntity implements Serializable {
     public enum RowCountStatus {
         OK("ok"), TENTATIVE("tentative");
 
-        private String status;
+        private final String status;
 
         RowCountStatus(String status) {
             this.status = status;
@@ -246,17 +227,28 @@ public class TableExtDesc extends RootPersistentEntity implements Serializable {
         return columnStatsMap.getOrDefault(colName, null);
     }
 
+    public boolean isExcludedCol(String colName) {
+        return excluded || getExcludedColumns().contains(colName);
+    }
+
+    public int countExcludedColSize() {
+        if (!isExcluded()) {
+            return getExcludedColumns().size();
+        }
+        return getTableDesc().getColumns().length;
+    }
+
+    private TableDesc getTableDesc() {
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        return NTableMetadataManager.getInstance(kylinConfig, project).getTableDesc(getIdentity());
+    }
+
     public void init(String project) {
         this.project = project;
         if (this.identity != null)
             this.identity = this.identity.toUpperCase(Locale.ROOT);
     }
 
-    public boolean isPartitioned() {
-        return this.dataSourceProps.get("partition_column") != null
-                && !this.dataSourceProps.get("partition_column").isEmpty();
-    }
-
     public S3RoleCredentialInfo getS3RoleCredentialInfo() {
         String location = this.dataSourceProps.get(LOCATION_PROPERTY_KEY);
         String s3Role = this.dataSourceProps.get(S3_ROLE_PROPERTY_KEY);
@@ -388,15 +380,6 @@ public class TableExtDesc extends RootPersistentEntity implements Serializable {
             cardinality = totalCardinality;
         }
 
-        public void addRangeHLLC(SegmentRange segRange, HLLCounter hllc) {
-            final String key = segRange.getStart() + "_" + segRange.getEnd();
-            rangeHLLC.put(key, hllc);
-        }
-
-        public void addRangeHLLC(String segRange, HLLCounter hllc) {
-            rangeHLLC.put(segRange, hllc);
-        }
-
         public void updateBasicStats(double maxNumeral, double minNumeral, int maxLength, int minLength,
                 String maxLengthValue, String minLengthValue) {
             if (Double.isNaN(this.maxNumeral) || maxNumeral > this.maxNumeral) {
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/alias/AliasMapping.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/alias/AliasMapping.java
index b79c8198cc..8473761064 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/alias/AliasMapping.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/alias/AliasMapping.java
@@ -17,17 +17,25 @@
  */
 package org.apache.kylin.metadata.model.alias;
 
+import java.util.Set;
+
 import com.google.common.collect.BiMap;
+import com.google.common.collect.Sets;
+
+import lombok.Getter;
 
 public class AliasMapping {
 
-    private BiMap<String, String> aliasMapping;
+    private final BiMap<String, String> aliasMap;
+
+    @Getter
+    private final Set<String> excludedColumns = Sets.newHashSet();
 
-    public AliasMapping(BiMap<String, String> aliasMapping) {
-        this.aliasMapping = aliasMapping;
+    public AliasMapping(BiMap<String, String> aliasMap) {
+        this.aliasMap = aliasMap;
     }
 
-    public BiMap<String, String> getAliasMapping() {
-        return aliasMapping;
+    public BiMap<String, String> getAliasMap() {
+        return aliasMap;
     }
 }
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/alias/ExpressionComparator.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/alias/ExpressionComparator.java
index febf268de8..1c4d692ebc 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/alias/ExpressionComparator.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/alias/ExpressionComparator.java
@@ -43,6 +43,9 @@ public class ExpressionComparator {
 
     private static final Logger logger = LoggerFactory.getLogger(ExpressionComparator.class);
 
+    private ExpressionComparator() {
+    }
+
     /**
      *
      * @param queryNode
@@ -56,7 +59,7 @@ public class ExpressionComparator {
         if (aliasMapping == null) {
             return false;
         }
-        return isNodeEqual(queryNode, exprNode, new AliasMachingSqlNodeComparator(aliasMapping, aliasDeduce));
+        return isNodeEqual(queryNode, exprNode, new AliasMatchingSqlNodeComparator(aliasMapping, aliasDeduce));
     }
 
     public static boolean isNodeEqual(SqlNode queryNode, SqlNode exprNode, SqlNodeComparator nodeComparator) {
@@ -69,17 +72,17 @@ public class ExpressionComparator {
         }
     }
 
-    public static class AliasMachingSqlNodeComparator extends SqlNodeComparator {
+    public static class AliasMatchingSqlNodeComparator extends SqlNodeComparator {
         private final AliasMapping aliasMapping;
         private final AliasDeduce aliasDeduce;
 
-        public AliasMachingSqlNodeComparator(AliasMapping aliasMapping, AliasDeduce aliasDeduce) {
+        public AliasMatchingSqlNodeComparator(AliasMapping aliasMapping, AliasDeduce aliasDeduce) {
             this.aliasMapping = aliasMapping;
             this.aliasDeduce = aliasDeduce;
         }
 
         protected boolean isSqlIdentifierEqual(SqlIdentifier querySqlIdentifier, SqlIdentifier exprSqlIdentifier) {
-            if (aliasMapping == null || aliasMapping.getAliasMapping() == null) {
+            if (aliasMapping == null || aliasMapping.getAliasMap() == null) {
                 return false;
             }
             Preconditions.checkState(exprSqlIdentifier.names.size() == 2);
@@ -100,10 +103,15 @@ public class ExpressionComparator {
                 }
 
                 //translate user alias to alias in model
-                String modelAlias = aliasMapping.getAliasMapping().get(queryAlias);
+                String modelAlias = aliasMapping.getAliasMap().get(queryAlias);
                 Preconditions.checkNotNull(modelAlias);
                 Preconditions.checkNotNull(queryCol);
 
+                String identity = modelAlias + "." + queryCol;
+                if (aliasMapping.getExcludedColumns().contains(identity)) {
+                    return false;
+                }
+
                 return StringUtils.equalsIgnoreCase(modelAlias, exprSqlIdentifier.names.get(0))
                         && StringUtils.equalsIgnoreCase(queryCol, exprSqlIdentifier.names.get(1));
             } catch (NullPointerException | IllegalStateException e) {
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/tool/CalciteParser.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/tool/CalciteParser.java
index ce504847aa..b09da6b5fa 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/tool/CalciteParser.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/tool/CalciteParser.java
@@ -74,8 +74,8 @@ public class CalciteParser {
     }
 
     public static SqlNode parse(String sql, String project) throws SqlParseException {
-        KylinConfig kylinConfig = StringUtils.isNotEmpty(project)
-                ? NProjectManager.getInstance(KylinConfig.getInstanceFromEnv()).getProject(project).getConfig()
+        KylinConfig kylinConfig = StringUtils.isNotEmpty(project) //
+                ? NProjectManager.getProjectConfig(project) //
                 : KylinConfig.getInstanceFromEnv();
         SqlParser.ConfigBuilder parserBuilder = SqlParser.configBuilder()
                 .setIdentifierMaxLength(kylinConfig.getMaxModelDimensionMeasureNameLength());
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 ead8a3e8fd..492a05723e 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
@@ -393,7 +393,7 @@ public class ComputedColumnUtil {
         }
         String existingAlias = existingCC.getTableAlias();
         String newAlias = newCC.getTableAlias();
-        return StringUtils.equals(newAlias, adviceAliasMapping.getAliasMapping().get(existingAlias));
+        return StringUtils.equals(newAlias, adviceAliasMapping.getAliasMap().get(existingAlias));
     }
 
     public interface CCConflictHandler {
@@ -471,7 +471,7 @@ public class ComputedColumnUtil {
             JoinsGraph ccJoinsGraph = getCCExprRelatedSubgraph(existingCC, existingModel);
             AliasMapping aliasMapping = getAliasMappingFromJoinsGraph(ccJoinsGraph, newModel.getJoinsGraph());
             String advisedExpr = aliasMapping == null ? null
-                    : CalciteParser.replaceAliasInExpr(existingCC.getExpression(), aliasMapping.getAliasMapping());
+                    : CalciteParser.replaceAliasInExpr(existingCC.getExpression(), aliasMapping.getAliasMap());
 
             String finalExpr = advisedExpr != null ? advisedExpr : existingCC.getExpression();
             String msg = String.format(Locale.ROOT, MsgPicker.getMsg().getComputedColumnNameDuplicated(),
@@ -485,7 +485,7 @@ public class ComputedColumnUtil {
         public void handleOnWrongPositionName(NDataModel existingModel, ComputedColumnDesc existingCC,
                 ComputedColumnDesc newCC, AliasMapping positionAliasMapping) {
             String advice = positionAliasMapping == null ? null
-                    : positionAliasMapping.getAliasMapping().get(existingCC.getTableAlias());
+                    : positionAliasMapping.getAliasMap().get(existingCC.getTableAlias());
 
             String msg = null;
 
@@ -507,7 +507,7 @@ public class ComputedColumnUtil {
         public void handleOnWrongPositionExpr(NDataModel existingModel, ComputedColumnDesc existingCC,
                 ComputedColumnDesc newCC, AliasMapping positionAliasMapping) {
             String advice = positionAliasMapping == null ? null
-                    : positionAliasMapping.getAliasMapping().get(existingCC.getTableAlias());
+                    : positionAliasMapping.getAliasMap().get(existingCC.getTableAlias());
 
             String msg = null;
 
diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/recommendation/ref/OptRecV2.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/recommendation/ref/OptRecV2.java
index 24f738affd..41c9d6100f 100644
--- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/recommendation/ref/OptRecV2.java
+++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/recommendation/ref/OptRecV2.java
@@ -30,17 +30,18 @@ import java.util.stream.Collectors;
 import org.apache.commons.lang.StringUtils;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.JsonUtil;
-import org.apache.kylin.metadata.model.MeasureDesc;
-import org.apache.kylin.metadata.model.ParameterDesc;
-import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.cube.model.LayoutEntity;
 import org.apache.kylin.metadata.cube.model.NIndexPlanManager;
 import org.apache.kylin.metadata.favorite.FavoriteRule;
 import org.apache.kylin.metadata.favorite.FavoriteRuleManager;
+import org.apache.kylin.metadata.model.AntiFlatChecker;
+import org.apache.kylin.metadata.model.ColExcludedChecker;
 import org.apache.kylin.metadata.model.ComputedColumnDesc;
-import org.apache.kylin.metadata.model.ExcludedLookupChecker;
+import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.NDataModel;
 import org.apache.kylin.metadata.model.NDataModelManager;
+import org.apache.kylin.metadata.model.ParameterDesc;
+import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.model.util.ComputedColumnUtil;
 import org.apache.kylin.metadata.recommendation.candidate.RawRecItem;
 import org.apache.kylin.metadata.recommendation.candidate.RawRecManager;
@@ -92,7 +93,8 @@ public class OptRecV2 {
     private final NDataModel model = initModel();
     @Getter(lazy = true)
     private final Map<String, ComputedColumnDesc> projectCCMap = initAllCCMap();
-    private final ExcludedLookupChecker checker;
+    private final AntiFlatChecker antiFlatChecker;
+    private final ColExcludedChecker excludedChecker;
     private final boolean needLog;
 
     public OptRecV2(String project, String uuid, boolean needLog) {
@@ -103,8 +105,8 @@ public class OptRecV2 {
 
         uniqueFlagToRecItemMap = RawRecManager.getInstance(project).queryNonLayoutRecItems(Sets.newHashSet(uuid));
         uniqueFlagToRecItemMap.forEach((k, recItem) -> uniqueFlagToId.put(k, recItem.getId()));
-        Set<String> excludedTables = FavoriteRuleManager.getInstance(config, project).getExcludedTables();
-        checker = new ExcludedLookupChecker(excludedTables, getModel().getJoinTables(), getModel());
+        antiFlatChecker = new AntiFlatChecker(getModel().getJoinTables(), getModel());
+        excludedChecker = new ColExcludedChecker(config, project, getModel());
         if (!getModel().isBroken()) {
             initModelColumnRefs(getModel());
             initModelMeasureRefs(getModel());
@@ -196,7 +198,7 @@ public class OptRecV2 {
             String content = ccNameToExpressionMap.getOrDefault(columnName, columnName);
             TblColRef tblColRef = model.getEffectiveCols().get(column.getId());
             RecommendationRef columnRef = new ModelColumnRef(column, tblColRef.getDatatype(), content);
-            if (checker.isColRefDependsLookupTable(tblColRef)) {
+            if (antiFlatChecker.isColOfAntiLookup(tblColRef) || excludedChecker.isExcludedCol(tblColRef)) {
                 columnRef.setExcluded(true);
             }
             columnRefs.put(id, columnRef);
@@ -218,7 +220,7 @@ public class OptRecV2 {
             MeasureRef measureRef = new MeasureRef(measure, measure.getId(), true);
             measure.getFunction().getParameters().stream().filter(ParameterDesc::isColumnType).forEach(p -> {
                 int id = model.getColumnIdByColumnName(p.getValue());
-                if (checker.isColRefDependsLookupTable(p.getColRef())) {
+                if (antiFlatChecker.isColOfAntiLookup(p.getColRef()) || excludedChecker.isExcludedCol(p.getColRef())) {
                     measureRef.setExcluded(true);
                 }
                 measureRef.getDependencies().add(columnRefs.get(id));
diff --git a/src/core-metadata/src/test/java/org/apache/kylin/model/TableExtDescTest.java b/src/core-metadata/src/test/java/org/apache/kylin/model/TableExtDescTest.java
index c13cb64b8b..fb3242189c 100644
--- a/src/core-metadata/src/test/java/org/apache/kylin/model/TableExtDescTest.java
+++ b/src/core-metadata/src/test/java/org/apache/kylin/model/TableExtDescTest.java
@@ -23,13 +23,10 @@ import static org.apache.kylin.metadata.model.NTableMetadataManager.getInstance;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.commons.lang3.RandomStringUtils;
-import org.apache.kylin.measure.hllc.HLLCounter;
-import org.apache.kylin.metadata.model.SegmentRange;
-import org.apache.kylin.metadata.model.TableDesc;
-import org.apache.kylin.metadata.model.TableExtDesc;
 import org.apache.kylin.common.util.NLocalFileMetadataTestCase;
 import org.apache.kylin.metadata.model.NTableMetadataManager;
+import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.metadata.model.TableExtDesc;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -60,11 +57,9 @@ public class TableExtDescTest extends NLocalFileMetadataTestCase {
 
         final String colName = "col_1";
         final List<TableExtDesc.ColumnStats> columnStatsList = new ArrayList<>(tableDesc.getColumnCount());
-        final SegmentRange segRange_1 = new SegmentRange.TimePartitionedSegmentRange(0L, 10L);
         TableExtDesc.ColumnStats colStats = new TableExtDesc.ColumnStats();
         colStats.setColumnName(colName);
-        HLLCounter col_hllc = mockHLLCounter(2, 5);
-        columnStatsList.add(updateColStats(colStats, 10, segRange_1, col_hllc, 1000d, -1000d, 4, 2, "9999", "99"));
+        columnStatsList.add(updateColStats(colStats, 10, 1000d, -1000d, 4, 2, "9999", "99"));
 
         tableExtDesc.setColumnStats(columnStatsList);
         tableMetadataManager.saveTableExt(tableExtDesc);
@@ -76,9 +71,7 @@ public class TableExtDescTest extends NLocalFileMetadataTestCase {
         Assert.assertEquals(colName, colStats.getColumnName());
         Assert.assertEquals(10, colStats.getNullCount());
 
-        final SegmentRange segRange_2 = new SegmentRange.TimePartitionedSegmentRange(10L, 20L);
-        col_hllc = mockHLLCounter(6, 10);
-        columnStatsList.add(updateColStats(colStats, 11, segRange_2, col_hllc, 9999d, -9999d, 5, 1, "99999", "9"));
+        columnStatsList.add(updateColStats(colStats, 11, 9999d, -9999d, 5, 1, "99999", "9"));
 
         tableExtDesc.setColumnStats(columnStatsList);
         tableMetadataManager.saveTableExt(tableExtDesc);
@@ -114,25 +107,12 @@ public class TableExtDescTest extends NLocalFileMetadataTestCase {
 
     }
 
-    private TableExtDesc.ColumnStats updateColStats(TableExtDesc.ColumnStats colStats, long nullCount,
-            SegmentRange segRange, HLLCounter hllc, double maxValue, double minValue, int maxLength, int minLength,
-            String maxLengthValue, String minLengthValue) {
+    private TableExtDesc.ColumnStats updateColStats(TableExtDesc.ColumnStats colStats, long nullCount, double maxValue,
+            double minValue, int maxLength, int minLength, String maxLengthValue, String minLengthValue) {
 
         colStats.addNullCount(nullCount);
-
-        colStats.addRangeHLLC(segRange, hllc);
-
         colStats.updateBasicStats(maxValue, minValue, maxLength, minLength, maxLengthValue, minLengthValue);
 
         return colStats;
     }
-
-    private HLLCounter mockHLLCounter(int min, int max) {
-        final HLLCounter hllCounter = new HLLCounter(14);
-        for (int i = min; i <= max; i++) {
-            hllCounter.add(RandomStringUtils.randomAlphanumeric(i));
-        }
-
-        return hllCounter;
-    }
 }
diff --git a/src/core-metadata/src/test/java/org/apache/kylin/util/MetadataTestUtils.java b/src/core-metadata/src/test/java/org/apache/kylin/util/MetadataTestUtils.java
new file mode 100644
index 0000000000..5f8606d558
--- /dev/null
+++ b/src/core-metadata/src/test/java/org/apache/kylin/util/MetadataTestUtils.java
@@ -0,0 +1,214 @@
+/*
+ * 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.util;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.JsonUtil;
+import org.apache.kylin.metadata.model.ColumnDesc;
+import org.apache.kylin.metadata.model.NTableMetadataManager;
+import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.metadata.model.TableExtDesc;
+import org.apache.kylin.metadata.project.EnhancedUnitOfWork;
+import org.apache.kylin.metadata.project.NProjectManager;
+
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+public class MetadataTestUtils {
+
+    public static final String TABLE_EXCLUSION_SETTING = "kylin.metadata.table-exclusion-enabled";
+
+    private MetadataTestUtils() {
+    }
+
+    public static Set<String> getExcludedTables(String project) {
+        NTableMetadataManager tableMgr = NTableMetadataManager.getInstance(KylinConfig.getInstanceFromEnv(), project);
+        return tableMgr.listAllTables().stream() //
+                .map(tableMgr::getTableExtIfExists).filter(Objects::nonNull) //
+                .filter(TableExtDesc::isExcluded) //
+                .map(TableExtDesc::getIdentity) //
+                .collect(Collectors.toSet());
+    }
+
+    public static Set<String> getExcludedColumns(String project, String tableIdentity) {
+        NTableMetadataManager tableMgr = NTableMetadataManager.getInstance(KylinConfig.getInstanceFromEnv(), project);
+        TableDesc tableDesc = tableMgr.getTableDesc(StringUtils.upperCase(tableIdentity));
+        TableExtDesc tableExt = tableMgr.getOrCreateTableExt(tableDesc);
+        if (tableExt.isExcluded()) {
+            return Arrays.stream(tableDesc.getColumns()).map(ColumnDesc::getName).collect(Collectors.toSet());
+        }
+        return tableExt.getExcludedColumns();
+    }
+
+    /**
+     * Mock an excluded table.
+     * @param project name of project
+     * @param tableIdentity qualified name of table
+     */
+    public static void mockExcludedTable(String project, String tableIdentity) {
+        mockExcludedCols(project, tableIdentity, Sets.newHashSet());
+    }
+
+    /**
+     * Mock excluded columns of give table.
+     * @param project name of project
+     * @param tableIdentity qualified name of table
+     * @param excludedColumns columns to be excluded
+     */
+    public static void mockExcludedCols(String project, String tableIdentity, Set<String> excludedColumns) {
+        EnhancedUnitOfWork.doInTransactionWithCheckAndRetry(() -> {
+            innerUpdate(project, TABLE_EXCLUSION_SETTING, "true");
+            KylinConfig config = KylinConfig.getInstanceFromEnv();
+            NTableMetadataManager tableMgr = NTableMetadataManager.getInstance(config, project);
+            TableDesc tableDesc = tableMgr.getTableDesc(StringUtils.upperCase(tableIdentity));
+            boolean tableExtExisted = tableMgr.isTableExtExist(StringUtils.upperCase(tableIdentity));
+            TableExtDesc tableExt = tableMgr.getOrCreateTableExt(tableDesc);
+            tableExt.setExcluded(excludedColumns.isEmpty());
+            Set<String> toBeExcludedColumns = excludedColumns.stream() //
+                    .map(StringUtils::upperCase) //
+                    .collect(Collectors.toSet());
+            tableExt.getExcludedColumns().addAll(toBeExcludedColumns);
+            tableMgr.saveOrUpdateTableExt(tableExtExisted, tableExt);
+            return null;
+        }, project);
+    }
+
+    public static void mockExcludedTables(String project, List<String> tableIdentityList) {
+        EnhancedUnitOfWork.doInTransactionWithCheckAndRetry(() -> {
+            innerUpdate(project, TABLE_EXCLUSION_SETTING, "true");
+            tableIdentityList.forEach(table -> mockExcludedTable(project, table));
+            return null;
+        }, project);
+    }
+
+    public static void toSemiAutoProjectMode(String project) {
+        updateProjectConfig(project, "kylin.metadata.semi-automatic-mode", "true");
+    }
+
+    public static KylinConfig turnOnExcludedTable(KylinConfig config) {
+        config.setProperty("kylin.metadata.table-exclusion-enabled", "true");
+        return config;
+    }
+
+    public static KylinConfig setOnlyReuseUseDefinedCC(KylinConfig config) {
+        config.setProperty("kylin.metadata.only-reuse-user-defined-computed-column", "true");
+        return config;
+    }
+
+    public static void updateProjectConfig(String project, String property, String value) {
+        EnhancedUnitOfWork.doInTransactionWithCheckAndRetry(() -> {
+            innerUpdate(project, property, value);
+            return null;
+        }, project);
+    }
+
+    private static void innerUpdate(String project, String property, String value) {
+        NProjectManager projectManager = NProjectManager.getInstance(KylinConfig.getInstanceFromEnv());
+        projectManager.updateProject(project, copyForWrite -> {
+            LinkedHashMap<String, String> overrideKylinProps = copyForWrite.getOverrideKylinProps();
+            if (overrideKylinProps == null) {
+                overrideKylinProps = Maps.newLinkedHashMap();
+            }
+            overrideKylinProps.put(property, value);
+            copyForWrite.setOverrideKylinProps(overrideKylinProps);
+        });
+    }
+
+    public static void updateProjectConfig(String project, Map<String, String> properties) {
+        EnhancedUnitOfWork.doInTransactionWithCheckAndRetry(() -> {
+            NProjectManager projectManager = NProjectManager.getInstance(KylinConfig.getInstanceFromEnv());
+            projectManager.updateProject(project, copyForWrite -> {
+                LinkedHashMap<String, String> overrideKylinProps = copyForWrite.getOverrideKylinProps();
+                overrideKylinProps.putAll(properties);
+            });
+            return null;
+        }, project);
+    }
+
+    public static void createTable(String project, Class<?> clazz, String srcTableDir, String tableIdentity)
+            throws IOException {
+        EnhancedUnitOfWork.doInTransactionWithCheckAndRetry(() -> {
+            String tableJsonPath = concatTablePath(srcTableDir, tableIdentity);
+            String fullPath = Objects.requireNonNull(clazz.getResource(tableJsonPath)).getPath();
+            TableDesc newTable = JsonUtil.readValue(new File(fullPath), TableDesc.class);
+            NTableMetadataManager mgr = NTableMetadataManager.getInstance(KylinConfig.getInstanceFromEnv(), project);
+            newTable.setMvcc(-1);
+            mgr.saveSourceTable(newTable);
+            return null;
+        }, project);
+    }
+
+    /**
+     * Replace the table in destTableDir with the table in srcTableDir.
+     * For example:
+     * --- srcTableDir is `/data/tableDesc`
+     * --- tableIdentity is `SSB.CUSTOMER`
+     */
+    public static void replaceTable(String project, Class<?> clazz, String srcTableDir, String tableIdentity)
+            throws IOException {
+        EnhancedUnitOfWork.doInTransactionWithCheckAndRetry(() -> {
+            String tableJsonPath = concatTablePath(srcTableDir, tableIdentity);
+            String fullPath = Objects.requireNonNull(clazz.getResource(tableJsonPath)).getPath();
+            TableDesc srcTable = JsonUtil.readValue(new File(fullPath), TableDesc.class);
+            NTableMetadataManager mgr = NTableMetadataManager.getInstance(KylinConfig.getInstanceFromEnv(), project);
+            TableDesc oldTable = mgr.getTableDesc(tableIdentity);
+            srcTable.setMvcc(oldTable.getMvcc());
+            mgr.updateTableDesc(srcTable);
+            return null;
+        }, project);
+    }
+
+    private static String concatTablePath(String tableDir, String tableIdentity) {
+        return concat(tableDir, tableIdentity, false);
+    }
+
+    /**
+     * Put the data from source directory to the kylin home directory.
+     */
+    public static void putTableCSVData(String srcDir, Class<?> clazz, String tableIdentity) throws IOException {
+        String dataPath = concatTableDataPath(srcDir, tableIdentity);
+        String fullPath = Objects.requireNonNull(clazz.getResource(dataPath)).getPath();
+        String data = FileUtils.readFileToString(new File(fullPath), Charset.defaultCharset());
+        FileUtils.write(new File(concatTableDataPath(KylinConfig.getKylinHome(), tableIdentity)), data,
+                Charset.defaultCharset());
+    }
+
+    private static String concatTableDataPath(String dataDir, String tableIdentity) {
+        return concat(dataDir, tableIdentity, true);
+    }
+
+    private static String concat(String dir, String table, boolean isData) {
+        String parentPath = dir.endsWith("/") ? dir : dir + "/";
+        String postfix = isData ? ".csv" : ".json";
+        return parentPath + table + postfix;
+    }
+}
diff --git a/src/datasource-service/pom.xml b/src/datasource-service/pom.xml
index acd5eacfa9..423bb64f6f 100644
--- a/src/datasource-service/pom.xml
+++ b/src/datasource-service/pom.xml
@@ -86,6 +86,12 @@
         </dependency>
 
         <!-- test -->
+        <dependency>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-core-metadata</artifactId>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
         <dependency>
             <groupId>org.junit.vintage</groupId>
             <artifactId>junit-vintage-engine</artifactId>
diff --git a/src/datasource-service/src/main/java/org/apache/kylin/rest/response/TableDescResponse.java b/src/datasource-service/src/main/java/org/apache/kylin/rest/response/TableDescResponse.java
index 5519386bb6..2e05cb02f6 100644
--- a/src/datasource-service/src/main/java/org/apache/kylin/rest/response/TableDescResponse.java
+++ b/src/datasource-service/src/main/java/org/apache/kylin/rest/response/TableDescResponse.java
@@ -64,6 +64,8 @@ public class TableDescResponse extends TableDesc {
     private ColumnDescResponse[] extColumns;
     @JsonProperty("last_build_job_id")
     private String jodID;
+    @JsonProperty("excluded")
+    private boolean excluded;
 
     @JsonProperty("kafka_bootstrap_servers")
     private String kafkaBootstrapServers;
@@ -98,7 +100,7 @@ public class TableDescResponse extends TableDesc {
 
     @Getter
     @Setter
-    public class ColumnDescResponse extends ColumnDesc {
+    public static class ColumnDescResponse extends ColumnDesc {
         @JsonProperty("cardinality")
         private Long cardinality;
         @JsonProperty("min_value")
@@ -107,6 +109,8 @@ public class TableDescResponse extends TableDesc {
         private String maxValue;
         @JsonProperty("null_count")
         private Long nullCount;
+        @JsonProperty("excluded")
+        private boolean excluded;
 
         ColumnDescResponse(ColumnDesc col) {
             super(col);
diff --git a/src/datasource-service/src/main/java/org/apache/kylin/rest/service/TableExtService.java b/src/datasource-service/src/main/java/org/apache/kylin/rest/service/TableExtService.java
index 13f2c7a7e7..e64d75264b 100644
--- a/src/datasource-service/src/main/java/org/apache/kylin/rest/service/TableExtService.java
+++ b/src/datasource-service/src/main/java/org/apache/kylin/rest/service/TableExtService.java
@@ -19,10 +19,12 @@
 package org.apache.kylin.rest.service;
 
 import static org.apache.kylin.common.exception.ServerErrorCode.INVALID_TABLE_NAME;
+import static org.apache.kylin.common.exception.code.ErrorCodeServer.EXCLUDED_TABLE_REQUEST_NOT_ALLOWED;
 
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -32,12 +34,14 @@ import java.util.Objects;
 import java.util.Set;
 import java.util.stream.Collectors;
 
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 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.HadoopUtil;
 import org.apache.kylin.common.util.Pair;
+import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.ISourceAware;
 import org.apache.kylin.metadata.model.NTableMetadataManager;
 import org.apache.kylin.metadata.model.TableDesc;
@@ -47,7 +51,12 @@ import org.apache.kylin.metadata.project.NProjectManager;
 import org.apache.kylin.metadata.project.ProjectInstance;
 import org.apache.kylin.rest.aspect.Transaction;
 import org.apache.kylin.rest.request.S3TableExtInfo;
+import org.apache.kylin.rest.request.TableExclusionRequest;
 import org.apache.kylin.rest.request.UpdateAWSTableExtDescRequest;
+import org.apache.kylin.rest.response.DataResult;
+import org.apache.kylin.rest.response.ExcludedColumnResponse;
+import org.apache.kylin.rest.response.ExcludedTableDetailResponse;
+import org.apache.kylin.rest.response.ExcludedTableResponse;
 import org.apache.kylin.rest.response.LoadTableResponse;
 import org.apache.kylin.rest.response.UpdateAWSTableExtDescResponse;
 import org.apache.kylin.rest.security.KerberosLoginManager;
@@ -68,6 +77,8 @@ import com.google.common.collect.Sets;
 public class TableExtService extends BasicService {
     private static final Logger logger = LoggerFactory.getLogger(TableExtService.class);
 
+    public static final int DEFAULT_EXCLUDED_COLUMN_SIZE = 15;
+
     @Autowired
     @Qualifier("tableService")
     private TableService tableService;
@@ -260,11 +271,11 @@ public class TableExtService extends BasicService {
             String db;
             String table = null;
             if (isDb) {
-                db = str.toUpperCase(Locale.ROOT);
+                db = StringUtils.upperCase(str);
             } else {
                 String[] dbTableName = HadoopUtil.parseHiveTableName(str);
-                db = dbTableName[0].toUpperCase(Locale.ROOT);
-                table = dbTableName[1].toUpperCase(Locale.ROOT);
+                db = StringUtils.upperCase(dbTableName[0]);
+                table = StringUtils.upperCase(dbTableName[1]);
             }
             Set<String> tables = dbTableMap.getOrDefault(db, Sets.newHashSet());
             if (table != null) {
@@ -321,4 +332,118 @@ public class TableExtService extends BasicService {
                     String.format(Locale.ROOT, MsgPicker.getMsg().getSameTableNameExist(), tableDesc.getIdentity()));
         }
     }
+
+    public List<ExcludedTableResponse> getExcludedTables(String project, boolean viewPartialCols, String tablePattern) {
+        aclEvaluate.checkProjectReadPermission(project);
+        checkTableExclusionEnabled(project);
+        NTableMetadataManager tableMgr = getManager(NTableMetadataManager.class, project);
+        return tableMgr.listAllTables().stream().map(tableMgr::getTableExtIfExists) //
+                .filter(Objects::nonNull) //
+                .filter(tableExt -> tableExt.isExcluded() || !tableExt.getExcludedColumns().isEmpty()) //
+                .filter(table -> StringUtils.isBlank(tablePattern)
+                        || StringUtils.containsIgnoreCase(table.getIdentity(), tablePattern))
+                .map(tableExt -> {
+                    TableDesc table = tableMgr.getTableDesc(tableExt.getIdentity());
+                    ExcludedTableResponse response = new ExcludedTableResponse();
+                    response.setExcluded(tableExt.isExcluded());
+                    response.setTable(tableExt.getIdentity());
+                    response.setExcludedColSize(tableExt.countExcludedColSize());
+                    response.setExcludedColumns(seekExcludedColumns(viewPartialCols, tableExt, table));
+                    return response;
+                }).collect(Collectors.toList());
+    }
+
+    private void checkTableExclusionEnabled(String project) {
+        ProjectInstance prjInstance = getManager(NProjectManager.class).getProject(project);
+        if (!prjInstance.getConfig().isTableExclusionEnabled()) {
+            throw new KylinException(EXCLUDED_TABLE_REQUEST_NOT_ALLOWED, project);
+        }
+    }
+
+    // excluded columns subject to the order of table columns
+    private List<String> seekExcludedColumns(boolean viewPartialCols, TableExtDesc tableExt, TableDesc table) {
+        ColumnDesc[] columns = table.getColumns();
+        int limit = viewPartialCols ? DEFAULT_EXCLUDED_COLUMN_SIZE : columns.length;
+        return Arrays.stream(columns).filter(tableExt::testExcluded) //
+                .limit(limit).map(ColumnDesc::getName) //
+                .collect(Collectors.toList());
+    }
+
+    public ExcludedTableDetailResponse getExcludedTable(String project, String table, int pageOffset, int pageSize,
+            String colPattern, boolean matchExcludedCols) {
+        aclEvaluate.checkProjectReadPermission(project);
+        checkTableExclusionEnabled(project);
+        NTableMetadataManager tableMgr = getManager(NTableMetadataManager.class, project);
+        TableExtDesc tableExt = tableMgr.getOrCreateTableExt(table);
+        List<ColumnDesc> columnList = Arrays.stream(tableMgr.getTableDesc(table).getColumns())
+                .filter(column -> StringUtils.isBlank(colPattern)
+                        || StringUtils.containsIgnoreCase(column.getName(), colPattern))
+                .filter(column -> matchExcludedCols == tableExt.testExcluded(column)) //
+                .collect(Collectors.toList());
+        DataResult<List<ColumnDesc>> dataResult = DataResult.get(columnList, pageOffset, pageSize);
+        List<ExcludedColumnResponse> columns = dataResult.getValue().stream()
+                .map(column -> new ExcludedColumnResponse(column, matchExcludedCols)) //
+                .collect(Collectors.toList());
+        ExcludedTableDetailResponse response = new ExcludedTableDetailResponse();
+        response.setTable(tableExt.getIdentity());
+        response.setExcluded(tableExt.isExcluded());
+        response.setTotalSize(dataResult.getTotalSize());
+        response.setOffset(dataResult.getOffset());
+        response.setLimit(dataResult.getLimit());
+        if (matchExcludedCols) {
+            response.setExcludedColumns(columns);
+        } else {
+            response.setAdmittedColumns(columns);
+        }
+        return response;
+    }
+
+    @Transaction(project = 0)
+    public void updateExcludedTables(String project, TableExclusionRequest request) {
+        aclEvaluate.checkProjectReadPermission(project);
+        checkTableExclusionEnabled(project);
+
+        // update table ext
+        NTableMetadataManager tableMgr = getManager(NTableMetadataManager.class, project);
+        request.getCanceledTables().stream().map(StringUtils::upperCase).forEach(identity -> {
+            boolean tableExtExist = tableMgr.isTableExtExist(identity);
+            TableExtDesc tableExt = tableMgr.getOrCreateTableExt(identity);
+            tableExt.setIdentity(identity);
+            tableExt.setExcluded(false);
+            tableExt.getExcludedColumns().clear();
+            tableMgr.saveOrUpdateTableExt(tableExtExist, tableExt);
+        });
+        request.getExcludedTables().forEach(excludedTable -> {
+            String excludedTableName = StringUtils.upperCase(excludedTable.getTable());
+            boolean tableExtExist = tableMgr.isTableExtExist(excludedTableName);
+            TableExtDesc tableExt = tableMgr.getOrCreateTableExt(excludedTableName);
+            tableExt.setIdentity(excludedTableName);
+            if (excludedTable.isExcluded()) {
+                tableExt.getExcludedColumns().clear();
+                tableExt.setExcluded(excludedTable.isExcluded());
+            } else {
+                TableDesc table = tableMgr.getTableDesc(tableExt.getIdentity());
+                List<String> toBeRemovedColumns = excludedTable.getRemovedColumns().stream() //
+                        .map(StringUtils::upperCase).collect(Collectors.toList());
+                List<String> toBeAddedColumns = excludedTable.getAddedColumns().stream() //
+                        .map(StringUtils::upperCase).collect(Collectors.toList());
+                if (tableExt.isExcluded()) {
+                    Set<String> colNameSet = Arrays.stream(table.getColumns()).map(ColumnDesc::getName)
+                            .collect(Collectors.toSet());
+                    toBeRemovedColumns.forEach(colNameSet::remove);
+                    tableExt.getExcludedColumns().addAll(colNameSet);
+                    tableExt.setExcluded(excludedTable.isExcluded());
+                } else {
+                    toBeAddedColumns.stream().map(StringUtils::upperCase).forEach(tableExt.getExcludedColumns()::add);
+                    toBeRemovedColumns.forEach(tableExt.getExcludedColumns()::remove);
+                    tableExt.setExcluded(excludedTable.isExcluded());
+                }
+                if (tableExt.getExcludedColumns().size() == table.getColumns().length) {
+                    tableExt.getExcludedColumns().clear();
+                    tableExt.setExcluded(true);
+                }
+            }
+            tableMgr.saveOrUpdateTableExt(tableExtExist, tableExt);
+        });
+    }
 }
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 f4c94be684..00379246e7 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
@@ -189,6 +189,7 @@ public class TableService extends BasicService {
 
     private static final Logger logger = LoggerFactory.getLogger(TableService.class);
     private static final String REFRESH_SINGLE_CATALOG_PATH = "/kylin/api/tables/single_catalog_cache";
+    private static final String SSB_ERROR_MSG = "import ssb data error.";
 
     @Autowired
     private TableModelSupporter modelService;
@@ -228,7 +229,7 @@ public class TableService extends BasicService {
     }
 
     public List<TableDesc> getTableDescByTypes(String project, boolean withExt, final String tableName,
-            final String database, boolean isFuzzy, List sourceType) throws IOException {
+            final String database, boolean isFuzzy, List<Integer> sourceType) throws IOException {
         return getTableDesc(project, withExt, tableName, database, isFuzzy).stream()
                 .filter(tableDesc -> sourceType.contains(tableDesc.getSourceType())).collect(Collectors.toList());
     }
@@ -282,8 +283,7 @@ public class TableService extends BasicService {
         final NTableMetadataManager tableMetaMgr = getManager(NTableMetadataManager.class, project);
         // save table meta
         List<String> saved = Lists.newArrayList();
-        List<TableDesc> savedTables = Lists.newArrayList();
-        Set<TableExtDesc.S3RoleCredentialInfo> brodcasttedS3Conf = new HashSet<>();
+        Set<TableExtDesc.S3RoleCredentialInfo> broadcastS3Conf = new HashSet<>();
         for (Pair<TableDesc, TableExtDesc> pair : allMeta) {
             TableDesc tableDesc = pair.getFirst();
             TableExtDesc extDesc = pair.getSecond();
@@ -324,17 +324,15 @@ public class TableService extends BasicService {
                 nTableExtDesc.init(project);
 
                 tableMetaMgr.saveTableExt(nTableExtDesc);
-                if (!brodcasttedS3Conf.contains(extDesc.getS3RoleCredentialInfo())) {
+                if (!broadcastS3Conf.contains(extDesc.getS3RoleCredentialInfo())) {
                     addAndBroadcastSparkSession(extDesc.getS3RoleCredentialInfo());
-                    brodcasttedS3Conf.add(extDesc.getS3RoleCredentialInfo());
+                    broadcastS3Conf.add(extDesc.getS3RoleCredentialInfo());
                 }
             }
 
             saved.add(tableDesc.getIdentity());
-            savedTables.add(tableDesc);
         }
-        String[] result = saved.toArray(new String[saved.size()]);
-        return result;
+        return saved.toArray(new String[0]);
     }
 
     public List<Pair<TableDesc, TableExtDesc>> extractTableMeta(String[] tables, String project) {
@@ -428,22 +426,23 @@ public class TableService extends BasicService {
             return tableDescResponse;
         }
 
-        for (TableDescResponse.ColumnDescResponse colDescRes : tableDescResponse.getExtColumns()) {
-            final TableExtDesc.ColumnStats columnStats = tableExtDesc.getColumnStatsByName(colDescRes.getName());
+        for (TableDescResponse.ColumnDescResponse colDescResponse : tableDescResponse.getExtColumns()) {
+            TableExtDesc.ColumnStats columnStats = tableExtDesc.getColumnStatsByName(colDescResponse.getName());
+            colDescResponse.setExcluded(tableExtDesc.isExcludedCol(colDescResponse.getName()));
             if (columnStats != null) {
-                colDescRes.setCardinality(columnStats.getCardinality());
-                colDescRes.setMaxValue(columnStats.getMaxValue());
-                colDescRes.setMinValue(columnStats.getMinValue());
-                colDescRes.setNullCount(columnStats.getNullCount());
+                colDescResponse.setCardinality(columnStats.getCardinality());
+                colDescResponse.setMaxValue(columnStats.getMaxValue());
+                colDescResponse.setMinValue(columnStats.getMinValue());
+                colDescResponse.setNullCount(columnStats.getNullCount());
             }
         }
         tableDescResponse.setDescExd(tableExtDesc.getDataSourceProps());
         tableDescResponse.setCreateTime(tableExtDesc.getCreateTime());
+        tableDescResponse.setExcluded(tableExtDesc.isExcluded());
         return tableDescResponse;
     }
 
-    private List<TableDesc> getTablesResponse(List<TableDesc> tables, String project, boolean withExt)
-            throws IOException {
+    private List<TableDesc> getTablesResponse(List<TableDesc> tables, String project, boolean withExt) {
         List<TableDesc> descs = new ArrayList<>();
         val projectManager = getManager(NProjectManager.class);
         val groups = getCurrentUserGroups();
@@ -458,17 +457,10 @@ public class TableService extends BasicService {
                 continue;
             }
             TableDescResponse tableDescResponse;
-            val modelsUsingRootTable = Lists.<NDataModel> newArrayList();
-            val modelsUsingTable = Lists.<NDataModel> newArrayList();
-            for (NDataModel model : healthyModels) {
-                if (model.containsTable(table)) {
-                    modelsUsingTable.add(model);
-                }
-
-                if (model.isRootFactTable(table)) {
-                    modelsUsingRootTable.add(model);
-                }
-            }
+            List<NDataModel> modelsUsingTable = healthyModels.stream() //
+                    .filter(model -> model.containsTable(table)).collect(Collectors.toList());
+            List<NDataModel> modelsUsingRootTable = healthyModels.stream() //
+                    .filter(model -> model.isRootFactTable(table)).collect(Collectors.toList());
 
             if (withExt) {
                 tableDescResponse = getTableResponse(table, project);
@@ -492,13 +484,6 @@ public class TableService extends BasicService {
                 tableDescResponse.setStorageSize(getSnapshotSize(project, table.getIdentity(), fs));
             }
             Pair<Set<String>, Set<String>> tableColumnType = getTableColumnType(project, table, modelsUsingTable);
-            NDataLoadingRange dataLoadingRange = getManager(NDataLoadingRangeManager.class, project)
-                    .getDataLoadingRange(table.getIdentity());
-            if (null != dataLoadingRange) {
-                tableDescResponse.setPartitionedColumn(dataLoadingRange.getColumnName());
-                tableDescResponse.setPartitionedColumnFormat(dataLoadingRange.getPartitionDateFormat());
-                tableDescResponse.setSegmentRange(dataLoadingRange.getCoveredRange());
-            }
             tableDescResponse.setForeignKey(tableColumnType.getSecond());
             tableDescResponse.setPrimaryKey(tableColumnType.getFirst());
             descs.add(tableDescResponse);
@@ -514,7 +499,7 @@ public class TableService extends BasicService {
         }
         List<String[]> result = Lists.newArrayList();
         final String dbTblName = rtableDesc.getIdentity();
-        Map columnRows = Arrays.stream(rtableDesc.getExtColumns()).map(cdr -> {
+        Map<Integer, AclTCR.ColumnRealRows> columnRows = Arrays.stream(rtableDesc.getExtColumns()).map(cdr -> {
             int id = Integer.parseInt(cdr.getId());
             val columnRealRows = getManager(AclTCRManager.class, project).getAuthorizedRows(dbTblName, cdr.getName(),
                     aclTCRS);
@@ -531,7 +516,7 @@ public class TableService extends BasicService {
                 if (!columnRows.containsKey(i)) {
                     continue;
                 }
-                val columnRealRows = (AclTCR.ColumnRealRows) columnRows.get(i);
+                val columnRealRows = columnRows.get(i);
                 if (Objects.isNull(columnRealRows)) {
                     jumpThisSample = true;
                     break;
@@ -803,9 +788,7 @@ public class TableService extends BasicService {
                 .filter(exec -> table.equalsIgnoreCase(exec.getParam(NBatchConstants.P_TABLE_NAME)))
                 .collect(Collectors.toList());
 
-        conflictJobs.forEach(job -> {
-            execManager.discardJob(job.getId());
-        });
+        conflictJobs.forEach(job -> execManager.discardJob(job.getId()));
         return conflictJobs;
     }
 
@@ -1076,7 +1059,8 @@ public class TableService extends BasicService {
         aclEvaluate.checkProjectWritePermission(project);
         val context = calcReloadContext(project, tableIdentity.toUpperCase(Locale.ROOT), false);
         removeFusionModelBatchPart(project, context);
-        PreReloadTableResponse preReloadTableResponse = preProcessBeforeReloadWithContext(project, context, needDetails);
+        PreReloadTableResponse preReloadTableResponse = preProcessBeforeReloadWithContext(project, context,
+                needDetails);
 
         OpenPreReloadTableResponse openPreReloadTableResponse = new OpenPreReloadTableResponse(preReloadTableResponse);
         openPreReloadTableResponse.setDuplicatedColumns(Lists.newArrayList(context.getDuplicatedColumns()));
@@ -1503,7 +1487,7 @@ public class TableService extends BasicService {
 
     private void checkNewColumn(String project, String tableName, Set<String> addColumns) {
         Multimap<String, String> duplicatedColumns = getDuplicatedColumns(project, tableName, addColumns);
-        if (Objects.nonNull(duplicatedColumns) && !duplicatedColumns.isEmpty()) {
+        if (!duplicatedColumns.isEmpty()) {
             Map.Entry<String, String> entry = duplicatedColumns.entries().iterator().next();
             throw new KylinException(DUPLICATED_COLUMN_NAME,
                     MsgPicker.getMsg().getTableReloadAddColumnExist(entry.getKey(), entry.getValue()));
@@ -1548,10 +1532,11 @@ public class TableService extends BasicService {
                 .collect(Collectors.toList());
 
         List<String> effectedJobs = Lists.newArrayList();
-        for (AbstractExecutable job : notFinalStateJobs) {
+        notFinalStateJobs.forEach(job -> {
             if (JobTypeEnum.TABLE_SAMPLING == job.getJobType()) {
                 if (newTableDesc.getIdentity().equalsIgnoreCase(job.getTargetSubject())) {
-                    effectedJobs.add(JobInfoEnum.JOB_ID == jobInfoType ? job.getId() : job.getTargetSubject());
+                    String jobId = JobInfoEnum.JOB_ID == jobInfoType ? job.getId() : job.getTargetSubject();
+                    effectedJobs.add(jobId);
                 }
             } else {
                 try {
@@ -1564,7 +1549,7 @@ public class TableService extends BasicService {
                     logger.warn("Get model by Job target subject failed!", e);
                 }
             }
-        }
+        });
         return effectedJobs;
     }
 
@@ -1578,15 +1563,16 @@ public class TableService extends BasicService {
                 return extractTableMeta(new String[] { tableIdentity }, project).get(0);
             }
         });
-        val newTableDesc = new TableDesc(tableMeta.getFirst());
+        TableDesc newTableDesc = new TableDesc(tableMeta.getFirst());
         context.setTableDesc(newTableDesc);
         context.setTableExtDesc(tableMeta.getSecond());
 
-        val originTableDesc = getManager(NTableMetadataManager.class, project).getTableDesc(tableIdentity);
+        handleExcludedColumns(project, context, newTableDesc, tableIdentity);
+
+        TableDesc originTableDesc = getManager(NTableMetadataManager.class, project).getTableDesc(tableIdentity);
         val collector = Collectors.toMap(ColumnDesc::getName, col -> Pair.newPair(col.getName(), col.getDatatype()));
         val originCols = Stream.of(originTableDesc.getColumns()).collect(collector);
         val newCols = Stream.of(newTableDesc.getColumns()).collect(collector);
-
         val diff = Maps.difference(newCols, originCols);
         context.setAddColumns(diff.entriesOnlyOnLeft().keySet());
         context.setRemoveColumns(diff.entriesOnlyOnRight().keySet());
@@ -1649,6 +1635,40 @@ public class TableService extends BasicService {
         return context;
     }
 
+    /**
+     * Handle excluded column when reloading table.
+     * 1. Add column to the excluded table, the added column will be treated as an excluded column automatically.
+     * 2. If the last un-excluded column is removed, this table will be treated as an excluded table.
+     */
+    private void handleExcludedColumns(String project, ReloadTableContext context, TableDesc newTable,
+            String tableIdentity) {
+        NTableMetadataManager tableManager = getManager(NTableMetadataManager.class, project);
+        TableDesc originTable = tableManager.getTableDesc(tableIdentity);
+        TableExtDesc originExt = tableManager.getTableExtIfExists(originTable);
+        if (originExt == null) {
+            return;
+        }
+
+        boolean excluded = originExt.isExcluded();
+        context.getTableExtDesc().setExcluded(excluded);
+        if (excluded) {
+            context.getTableExtDesc().getExcludedColumns().clear();
+        } else {
+            Set<String> excludedColumns = Sets.newHashSet(originExt.getExcludedColumns());
+            Set<String> newColNameSet = Arrays.stream(newTable.getColumns()).map(ColumnDesc::getName)
+                    .collect(Collectors.toSet());
+            excludedColumns.removeIf(col -> !newColNameSet.contains(col));
+            logger.debug("reserved excluded columns are: {}", excludedColumns);
+            if (newColNameSet.equals(excludedColumns)) {
+                context.getTableExtDesc().setExcluded(true);
+                context.getTableExtDesc().getExcludedColumns().clear();
+                logger.debug("Set the table to excluded table for all columns is excluded.");
+            } else {
+                context.getTableExtDesc().getExcludedColumns().addAll(excludedColumns);
+            }
+        }
+    }
+
     /**
      * get suitable measures when column type change
      * and remove old measure add new measure with suitable function return type
@@ -1750,10 +1770,11 @@ public class TableService extends BasicService {
         List process(String database, String table) throws Exception;
     }
 
-    public NInitTablesResponse getProjectTables(String project, String table, Integer offset, Integer limit,
-            Boolean useHiveDatabase, ProjectTablesFilter projectTablesFilter) throws Exception {
+    public NInitTablesResponse getProjectTables(String project, String table, int offset, int limit,
+            boolean withExcluded, boolean useHiveDatabase, ProjectTablesFilter projectTablesFilter) throws Exception {
         aclEvaluate.checkProjectReadPermission(project);
         NInitTablesResponse response = new NInitTablesResponse();
+        logger.debug("only get project tables of excluded: {}", withExcluded);
         if (table == null)
             table = "";
         String exceptDatabase = null;
@@ -1761,13 +1782,12 @@ public class TableService extends BasicService {
             exceptDatabase = table.split("\\.", 2)[0].trim();
             table = table.split("\\.", 2)[1].trim();
         }
-        String notAllowedModifyTableName = table;
         Collection<String> databases = useHiveDatabase ? getSourceDbNames(project) : getLoadedDatabases(project);
         val projectInstance = getManager(NProjectManager.class).getProject(project);
         List<String> tableFilterList = DataSourceState.getInstance().getHiveFilterList(projectInstance);
         for (String database : databases) {
             if ((exceptDatabase != null && !exceptDatabase.equalsIgnoreCase(database))
-                    ||(!tableFilterList.isEmpty() && !tableFilterList.contains(database))) {
+                    || (!tableFilterList.isEmpty() && !tableFilterList.contains(database))) {
                 continue;
             }
             List<?> tables;
@@ -1827,8 +1847,7 @@ public class TableService extends BasicService {
         }
     }
 
-    public List<TableNameResponse> getTableNameResponsesInCache(String project, String database, final String table)
-            throws Exception {
+    public List<TableNameResponse> getTableNameResponsesInCache(String project, String database, final String table) {
         aclEvaluate.checkProjectReadPermission(project);
         List<TableNameResponse> responses = new ArrayList<>();
         NTableMetadataManager tableManager = getManager(NTableMetadataManager.class, project);
@@ -1864,14 +1883,11 @@ public class TableService extends BasicService {
         DataSourceState.getInstance().loadAllSourceInfoToCache();
     }
 
-    public NHiveTableNameResponse loadProjectHiveTableNameToCacheImmediately(String project, boolean force)
-            throws Exception {
+    public NHiveTableNameResponse loadProjectHiveTableNameToCacheImmediately(String project, boolean force) {
         aclEvaluate.checkProjectWritePermission(project);
         return DataSourceState.getInstance().loadAllSourceInfoToCacheForced(project, force);
     }
 
-    private static final String SSB_ERROR_MSG = "import ssb data error.";
-
     public void importSSBDataBase() {
         aclEvaluate.checkIsGlobalAdmin();
         if (checkSSBDataBase()) {
@@ -1887,7 +1903,6 @@ public class TableService extends BasicService {
             try {
                 exec.execute(sampleSh, patternedLogger);
             } catch (ShellException e) {
-                logger.error(SSB_ERROR_MSG, e);
                 throw new KylinException(FAILED_IMPORT_SSB_DATA, SSB_ERROR_MSG, e);
             }
             if (!checkSSBDataBase()) {
diff --git a/src/datasource-service/src/test/java/org/apache/kylin/rest/service/TableExtServiceTest.java b/src/datasource-service/src/test/java/org/apache/kylin/rest/service/TableExtServiceTest.java
index b255df44a3..5d7dfa889e 100644
--- a/src/datasource-service/src/test/java/org/apache/kylin/rest/service/TableExtServiceTest.java
+++ b/src/datasource-service/src/test/java/org/apache/kylin/rest/service/TableExtServiceTest.java
@@ -21,8 +21,12 @@ package org.apache.kylin.rest.service;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
 
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.exception.KylinException;
+import org.apache.kylin.common.exception.code.ErrorCodeServer;
 import org.apache.kylin.common.util.NLocalFileMetadataTestCase;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.common.util.RandomUtil;
@@ -31,11 +35,16 @@ import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TableExtDesc;
 import org.apache.kylin.rest.constant.Constant;
 import org.apache.kylin.rest.request.S3TableExtInfo;
+import org.apache.kylin.rest.request.TableExclusionRequest;
 import org.apache.kylin.rest.request.UpdateAWSTableExtDescRequest;
+import org.apache.kylin.rest.response.ExcludedColumnResponse;
+import org.apache.kylin.rest.response.ExcludedTableDetailResponse;
+import org.apache.kylin.rest.response.ExcludedTableResponse;
 import org.apache.kylin.rest.response.LoadTableResponse;
 import org.apache.kylin.rest.response.UpdateAWSTableExtDescResponse;
 import org.apache.kylin.rest.util.AclEvaluate;
 import org.apache.kylin.rest.util.AclUtil;
+import org.apache.kylin.util.MetadataTestUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -217,8 +226,7 @@ public class TableExtServiceTest extends NLocalFileMetadataTestCase {
         tableDesc.setName("TEST_REMOVE");
         tableDesc.setDatabase("DEFAULT");
         tableDesc.setUuid(RandomUtil.randomUUIDStr());
-        NTableMetadataManager tableMetadataManager = NTableMetadataManager.getInstance(KylinConfig.getInstanceFromEnv(),
-                "default");
+        NTableMetadataManager tableMetadataManager = NTableMetadataManager.getInstance(getTestConfig(), "default");
         tableMetadataManager.saveTableExt(tableExtDesc);
         tableMetadataManager.saveSourceTable(tableDesc);
         tableExtService.removeJobIdFromTableExt("test", "default");
@@ -226,6 +234,311 @@ public class TableExtServiceTest extends NLocalFileMetadataTestCase {
         Assert.assertNull(tableExtDesc1.getJodID());
     }
 
+    @Test
+    public void testGetExcludedTableFailed() {
+        try {
+            tableExtService.getExcludedTable("default", "default.test_account", 0, 10, "", false);
+            Assert.fail();
+        } catch (KylinException e) {
+            Assert.assertEquals(ErrorCodeServer.EXCLUDED_TABLE_REQUEST_NOT_ALLOWED.getErrorCode().getCode(),
+                    e.getErrorCode().getCodeString());
+        }
+    }
+
+    @Test
+    public void testGetExcludedTableWithExcludedColumns() {
+        String project = "default";
+        String table = "DEFAULT.TEST_ACCOUNT";
+        MetadataTestUtils.mockExcludedTable(project, table);
+
+        // get all excluded columns
+        {
+            ExcludedTableDetailResponse response = tableExtService.getExcludedTable(project, table, 0, 10, "", true);
+            Assert.assertEquals(5, response.getTotalSize());
+            Assert.assertEquals(5, response.getExcludedColumns().size());
+            Assert.assertTrue(response.getAdmittedColumns().isEmpty());
+        }
+
+        // get excluded columns with pattern
+        {
+            ExcludedTableDetailResponse response = tableExtService.getExcludedTable(project, table, 0, 10, "level",
+                    true);
+            Assert.assertEquals(2, response.getTotalSize());
+            Assert.assertEquals(2, response.getExcludedColumns().size());
+            Assert.assertTrue(response.getAdmittedColumns().isEmpty());
+        }
+
+        // get admitted columns, the result is empty list [this is abnormal usage]
+        {
+            ExcludedTableDetailResponse response = tableExtService.getExcludedTable(project, table, 0, 10, "", false);
+            Assert.assertEquals(0, response.getTotalSize());
+            Assert.assertTrue(response.getAdmittedColumns().isEmpty());
+            Assert.assertTrue(response.getExcludedColumns().isEmpty());
+        }
+
+        // get excluded columns with pageSize is 2
+        {
+            ExcludedTableDetailResponse response = tableExtService.getExcludedTable(project, table, 0, 2, "", true);
+            Assert.assertEquals(5, response.getTotalSize());
+            Assert.assertEquals(2, response.getExcludedColumns().size());
+            Assert.assertTrue(response.getAdmittedColumns().isEmpty());
+        }
+    }
+
+    @Test
+    public void testGetExcludedTableWithToBeExcludedColumns() {
+        String project = "default";
+        String table = "DEFAULT.TEST_ACCOUNT";
+        MetadataTestUtils.turnOnExcludedTable(getTestConfig());
+
+        // get all columns to be excluded
+        {
+            ExcludedTableDetailResponse response = tableExtService.getExcludedTable(project, table, 0, 10, "", false);
+            Assert.assertEquals(5, response.getTotalSize());
+            Assert.assertEquals(5, response.getAdmittedColumns().size());
+            Assert.assertTrue(response.getExcludedColumns().isEmpty());
+        }
+
+        // get columns to be excluded with pattern
+        {
+            ExcludedTableDetailResponse response = tableExtService.getExcludedTable(project, table, 0, 10, "level",
+                    false);
+            Assert.assertEquals(2, response.getTotalSize());
+            Assert.assertEquals(2, response.getAdmittedColumns().size());
+            Assert.assertTrue(response.getExcludedColumns().isEmpty());
+        }
+
+        // get columns of excluded, the admitted columns is empty list [this is abnormal usage]
+        {
+            ExcludedTableDetailResponse response = tableExtService.getExcludedTable(project, table, 0, 10, "", true);
+            Assert.assertEquals(0, response.getTotalSize());
+            Assert.assertTrue(response.getAdmittedColumns().isEmpty());
+            Assert.assertTrue(response.getExcludedColumns().isEmpty());
+        }
+
+        // get columns of admitted columns with pageSize is 2
+        {
+            ExcludedTableDetailResponse response = tableExtService.getExcludedTable(project, table, 0, 2, "", false);
+            Assert.assertEquals(5, response.getTotalSize());
+            Assert.assertEquals(2, response.getAdmittedColumns().size());
+            Assert.assertTrue(response.getExcludedColumns().isEmpty());
+        }
+    }
+
+    @Test
+    public void testGetExcludedColumnsWhenSettingSomeExcludedColumns() {
+        String project = "default";
+        String table = "DEFAULT.TEST_ACCOUNT";
+        Set<String> excludedColumns = Sets.newHashSet("ACCOUNT_BUYER_LEVEL", "ACCOUNT_SELLER_LEVEL");
+        MetadataTestUtils.mockExcludedCols(project, table, excludedColumns);
+
+        // get all columns to be excluded
+        {
+            ExcludedTableDetailResponse response = tableExtService.getExcludedTable(project, table, 0, 10, "", false);
+            Assert.assertEquals(3, response.getTotalSize());
+            List<ExcludedColumnResponse> admittedColumnList = response.getAdmittedColumns();
+            Assert.assertEquals(3, admittedColumnList.size());
+            Assert.assertTrue(response.getExcludedColumns().isEmpty());
+            admittedColumnList.forEach(admittedColumn -> Assert.assertFalse(admittedColumn.isExcluded()));
+        }
+
+        // get all excluded columns
+        {
+            ExcludedTableDetailResponse response = tableExtService.getExcludedTable(project, table, 0, 10, "", true);
+            Assert.assertEquals(2, response.getTotalSize());
+            List<ExcludedColumnResponse> excludedColumnList = response.getExcludedColumns();
+            Assert.assertEquals(2, excludedColumnList.size());
+            Assert.assertTrue(response.getAdmittedColumns().isEmpty());
+
+            excludedColumnList.forEach(excludedColResp -> Assert.assertTrue(excludedColResp.isExcluded()));
+            Set<String> excludedColNameSet = excludedColumnList.stream().map(ExcludedColumnResponse::getName)
+                    .collect(Collectors.toSet());
+            Assert.assertEquals(excludedColumns, excludedColNameSet);
+        }
+    }
+
+    @Test
+    public void testGetExcludedTables() {
+        String project = "default";
+        String table = "DEFAULT.TEST_ACCOUNT";
+        Set<String> excludedColumns = Sets.newHashSet("ACCOUNT_BUYER_LEVEL", "ACCOUNT_SELLER_LEVEL");
+        MetadataTestUtils.mockExcludedCols(project, table, excludedColumns);
+        MetadataTestUtils.mockExcludedTable(project, "DEFAULT.TEST_COUNTRY");
+
+        // get without table pattern
+        {
+            List<ExcludedTableResponse> excludedTables = tableExtService.getExcludedTables(project, false, "");
+            Assert.assertEquals(2, excludedTables.size());
+            excludedTables.forEach(excludedTableResponse -> {
+                if (excludedTableResponse.getTable().equals(table)) {
+                    Assert.assertFalse(excludedTableResponse.isExcluded());
+                    Assert.assertEquals(2, excludedTableResponse.getExcludedColSize());
+                    Assert.assertEquals(excludedColumns, Sets.newHashSet(excludedTableResponse.getExcludedColumns()));
+                } else {
+                    Assert.assertEquals("DEFAULT.TEST_COUNTRY", excludedTableResponse.getTable());
+                    Assert.assertEquals(4, excludedTableResponse.getExcludedColSize());
+                    Assert.assertTrue(excludedTableResponse.isExcluded());
+                }
+            });
+        }
+
+        // get with table pattern 
+        {
+            List<ExcludedTableResponse> excludedTables = tableExtService.getExcludedTables(project, false,
+                    "TEST_COUNTRY");
+            Assert.assertEquals(1, excludedTables.size());
+            ExcludedTableResponse excludedTableResponse = excludedTables.get(0);
+
+            Assert.assertEquals("DEFAULT.TEST_COUNTRY", excludedTableResponse.getTable());
+            Assert.assertEquals(4, excludedTableResponse.getExcludedColSize());
+            Assert.assertTrue(excludedTableResponse.isExcluded());
+        }
+
+        // get with viewPartialCols
+        {
+            MetadataTestUtils.mockExcludedTable(project, "DEFAULT.TEST_MEASURE");
+            List<ExcludedTableResponse> excludedTables = tableExtService.getExcludedTables(project, true,
+                    "TEST_MEASURE");
+            Assert.assertEquals(1, excludedTables.size());
+            ExcludedTableResponse excludedTableResponse = excludedTables.get(0);
+            Assert.assertEquals("DEFAULT.TEST_MEASURE", excludedTableResponse.getTable());
+            Assert.assertEquals(17, excludedTableResponse.getExcludedColSize());
+            Assert.assertTrue(excludedTableResponse.isExcluded());
+            Assert.assertEquals(TableExtService.DEFAULT_EXCLUDED_COLUMN_SIZE,
+                    excludedTableResponse.getExcludedColumns().size());
+        }
+    }
+
+    @Test
+    public void updateExcludedTablesOfExcludeOneTable() {
+        String project = "default";
+        String table = "DEFAULT.TEST_ACCOUNT";
+        MetadataTestUtils.turnOnExcludedTable(getTestConfig());
+
+        TableExclusionRequest request = new TableExclusionRequest();
+        TableExclusionRequest.ExcludedTable excludedTable = new TableExclusionRequest.ExcludedTable();
+        excludedTable.setExcluded(true);
+        excludedTable.setTable(table);
+        request.setProject(project);
+        request.setExcludedTables(Lists.newArrayList(excludedTable));
+
+        tableExtService.updateExcludedTables(project, request);
+        List<ExcludedTableResponse> excludedTables = tableExtService.getExcludedTables(project, false, "");
+        Assert.assertEquals(1, excludedTables.size());
+        ExcludedTableResponse excludedTableResponse = excludedTables.get(0);
+        Assert.assertTrue(excludedTableResponse.isExcluded());
+        Assert.assertEquals(table, excludedTableResponse.getTable());
+        Assert.assertEquals(5, excludedTableResponse.getExcludedColSize());
+        String expectedExcludedCols = "[ACCOUNT_ID, ACCOUNT_BUYER_LEVEL, ACCOUNT_SELLER_LEVEL, ACCOUNT_COUNTRY, ACCOUNT_CONTACT]";
+        Assert.assertEquals(expectedExcludedCols, excludedTableResponse.getExcludedColumns().toString());
+    }
+
+    @Test
+    public void updateExcludedTablesOfCancelOneTable() {
+        String project = "default";
+        String table = "DEFAULT.TEST_ACCOUNT";
+        MetadataTestUtils.mockExcludedTable(project, table);
+        Set<String> tables = MetadataTestUtils.getExcludedTables(project);
+        Assert.assertEquals(1, tables.size());
+        Assert.assertEquals(table, tables.iterator().next());
+
+        TableExclusionRequest request = new TableExclusionRequest();
+        request.setProject(project);
+        request.setCanceledTables(Lists.newArrayList(table));
+
+        tableExtService.updateExcludedTables(project, request);
+        List<ExcludedTableResponse> excludedTables = tableExtService.getExcludedTables(project, false, "");
+        Assert.assertTrue(excludedTables.isEmpty());
+    }
+
+    @Test
+    public void updateExcludedTablesOfCancelAllExcludedColumns() {
+        String project = "default";
+        String table = "DEFAULT.TEST_ACCOUNT";
+        MetadataTestUtils.mockExcludedTable(project, table);
+        Set<String> tables = MetadataTestUtils.getExcludedTables(project);
+        Assert.assertEquals(1, tables.size());
+        Assert.assertEquals(table, tables.iterator().next());
+
+        // cancel all table columns, but not set excluded, assert the table is excluded.
+        TableExclusionRequest request = new TableExclusionRequest();
+        TableExclusionRequest.ExcludedTable excludedTable = new TableExclusionRequest.ExcludedTable();
+        excludedTable.setExcluded(false);
+        excludedTable.setTable(table);
+        excludedTable.setRemovedColumns(Lists.newArrayList("ACCOUNT_ID", "ACCOUNT_BUYER_LEVEL", "ACCOUNT_SELLER_LEVEL",
+                "ACCOUNT_COUNTRY", "ACCOUNT_CONTACT"));
+        request.setProject(project);
+        request.setExcludedTables(Lists.newArrayList(excludedTable));
+
+        tableExtService.updateExcludedTables(project, request);
+        List<ExcludedTableResponse> excludedTables = tableExtService.getExcludedTables(project, false, "");
+        Assert.assertTrue(excludedTables.isEmpty());
+    }
+
+    @Test
+    public void updateExcludedTablesOfCancelSomeColumns() {
+        String project = "default";
+        String table = "DEFAULT.TEST_ACCOUNT";
+        MetadataTestUtils.turnOnExcludedTable(getTestConfig());
+
+        // add some columns
+        {
+            TableExclusionRequest request = new TableExclusionRequest();
+            TableExclusionRequest.ExcludedTable excludedTable = new TableExclusionRequest.ExcludedTable();
+            excludedTable.setExcluded(false);
+            excludedTable.setTable(table);
+            excludedTable.setAddedColumns(Lists.newArrayList("ACCOUNT_ID", "ACCOUNT_BUYER_LEVEL"));
+            request.setProject(project);
+            request.setExcludedTables(Lists.newArrayList(excludedTable));
+
+            tableExtService.updateExcludedTables(project, request);
+            List<ExcludedTableResponse> excludedTables = tableExtService.getExcludedTables(project, false, "");
+            Assert.assertEquals(1, excludedTables.size());
+            ExcludedTableResponse excludedTableResponse = excludedTables.get(0);
+            Assert.assertFalse(excludedTableResponse.isExcluded());
+            Assert.assertEquals(table, excludedTableResponse.getTable());
+            Assert.assertEquals(2, excludedTableResponse.getExcludedColSize());
+            String expectedExcludedCols = "[ACCOUNT_ID, ACCOUNT_BUYER_LEVEL]";
+            Assert.assertEquals(expectedExcludedCols, excludedTableResponse.getExcludedColumns().toString());
+        }
+
+        // cancel some columns
+        {
+            TableExclusionRequest request = new TableExclusionRequest();
+            TableExclusionRequest.ExcludedTable excludedTable = new TableExclusionRequest.ExcludedTable();
+            excludedTable.setExcluded(false);
+            excludedTable.setTable(table);
+            excludedTable.setRemovedColumns(Lists.newArrayList("ACCOUNT_BUYER_LEVEL"));
+            request.setProject(project);
+            request.setExcludedTables(Lists.newArrayList(excludedTable));
+
+            tableExtService.updateExcludedTables(project, request);
+            List<ExcludedTableResponse> excludedTables = tableExtService.getExcludedTables(project, false, "");
+            Assert.assertEquals(1, excludedTables.size());
+            ExcludedTableResponse excludedTableResponse = excludedTables.get(0);
+            Assert.assertFalse(excludedTableResponse.isExcluded());
+            Assert.assertEquals(table, excludedTableResponse.getTable());
+            Assert.assertEquals(1, excludedTableResponse.getExcludedColSize());
+            String expectedExcludedCols = "[ACCOUNT_ID]";
+            Assert.assertEquals(expectedExcludedCols, excludedTableResponse.getExcludedColumns().toString());
+        }
+
+        // cancel other columns
+        {
+            TableExclusionRequest request = new TableExclusionRequest();
+            TableExclusionRequest.ExcludedTable excludedTable = new TableExclusionRequest.ExcludedTable();
+            excludedTable.setExcluded(false);
+            excludedTable.setTable(table);
+            excludedTable.setRemovedColumns(Lists.newArrayList("ACCOUNT_ID"));
+            request.setProject(project);
+            request.setExcludedTables(Lists.newArrayList(excludedTable));
+
+            tableExtService.updateExcludedTables(project, request);
+            List<ExcludedTableResponse> excludedTables = tableExtService.getExcludedTables(project, false, "");
+            Assert.assertTrue(excludedTables.isEmpty());
+        }
+    }
+
     private List<Pair<TableDesc, TableExtDesc>> mockTablePair(int size, String tableName) {
         List<Pair<TableDesc, TableExtDesc>> result = new ArrayList<>();
         for (int i = 0; i < size; i++) {
diff --git a/src/examples/test_case_data/sandbox/kylin.properties b/src/examples/test_case_data/sandbox/kylin.properties
index f7005a85ec..73426da7ae 100644
--- a/src/examples/test_case_data/sandbox/kylin.properties
+++ b/src/examples/test_case_data/sandbox/kylin.properties
@@ -70,7 +70,7 @@ kylin.engine.spark-conf.spark.sql.hive.metastore.version=1.2.2
 kylin.engine.spark-conf.spark.sql.hive.metastore.jars=${KYLIN_HOME}/build/spark/hive_1_2_2/*
 kylin.engine.spark-conf.spark.yarn.dist.files=../../build/conf/spark-executor-log4j.xml
 
-kylin.engine.spark.job-jar=../assembly/target/kylin-assembly-4.0.0-SNAPSHOT-job.jar
+kylin.engine.spark.job-jar=../assembly/target/kylin-assembly-5.0.0-SNAPSHOT-job.jar
 
 
 
@@ -182,7 +182,7 @@ kylin.smart.conf.propose-runner-type=in-memory
 #streaming
 kylin.streaming.spark-conf.spark.yarn.am.extraJavaOptions=-Dhdp.version=current -Dlog4j.configuration=spark-appmaster-log4j.properties
 kylin.streaming.spark-conf.spark.sql.hive.metastore.jars=${KYLIN_HOME}/build/spark/hive_1_2_2/*
-kylin.streaming.spark.job-jar=../assembly/target/kylin-assembly-4.0.0-SNAPSHOT-job.jar
+kylin.streaming.spark.job-jar=../assembly/target/kylin-assembly-5.0.0-SNAPSHOT-job.jar
 
 kylin.query.escape-default-keyword=true
 kylin.monitor.enabled=false
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
new file mode 100644
index 0000000000..02d0f0f185
--- /dev/null
+++ b/src/kylin-it/src/test/java/org/apache/kylin/metadata/model/AntiFlatCheckerTest.java
@@ -0,0 +1,167 @@
+/*
+ * 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.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.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import com.google.common.collect.Lists;
+
+@MetadataInfo
+class AntiFlatCheckerTest {
+
+    String modelId = "89af4ee2-2cdb-4b07-b39e-4c29856309aa";
+
+    String getProject() {
+        return "default";
+    }
+
+    private NDataModel prepareModel(KylinConfig kylinConfig) {
+        NDataModelManager modelManager = NDataModelManager.getInstance(kylinConfig, getProject());
+        NDataModel model = modelManager.getDataModelDesc(modelId);
+        model.getComputedColumnDescs().forEach(cc -> {
+            String innerExp = QueryUtil.massageComputedColumn(model, getProject(), cc, null);
+            cc.setInnerExpression(innerExp);
+        });
+        model.init(kylinConfig, getProject(), Lists.newArrayList());
+        return model;
+    }
+
+    @Test
+    void testJoinTablesWithNullValue() {
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        NDataModel model = prepareModel(kylinConfig);
+        AntiFlatChecker checker = new AntiFlatChecker(null, model);
+        Assertions.assertTrue(checker.getAntiFlattenLookups().isEmpty());
+        Assertions.assertTrue(checker.getInvalidComputedColumns(model).isEmpty());
+        Assertions.assertTrue(checker.getInvalidDimensions(model).isEmpty());
+        Assertions.assertTrue(checker.getInvalidMeasures(model).isEmpty());
+    }
+
+    @Test
+    void testWithBrokenModel() {
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        NDataModel model = prepareModel(kylinConfig);
+
+        NDataModel copyModel = JsonUtil.deepCopyQuietly(model, NDataModel.class);
+        copyModel.init(kylinConfig, getProject(), Lists.newArrayList());
+        List<JoinTableDesc> joinTables = copyModel.getJoinTables();
+        joinTables.forEach(joinTableDesc -> {
+            if ("DEFAULT.TEST_ORDER".equals(joinTableDesc.getTable())
+                    || "EDW.TEST_CAL_DT".equals(joinTableDesc.getTable())) {
+                joinTableDesc.setFlattenable(JoinTableDesc.NORMALIZED);
+            }
+        });
+
+        {
+            AntiFlatChecker checker = new AntiFlatChecker(joinTables, null);
+            Assertions.assertTrue(checker.getAntiFlattenLookups().isEmpty());
+        }
+
+        {
+            model.setBroken(true);
+            AntiFlatChecker checker = new AntiFlatChecker(null, model);
+            Assertions.assertTrue(checker.getAntiFlattenLookups().isEmpty());
+        }
+    }
+
+    @Test
+    void testNormal() {
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        NDataModel model = prepareModel(kylinConfig);
+
+        NDataModel copyModel = JsonUtil.deepCopyQuietly(model, NDataModel.class);
+        copyModel.init(kylinConfig, getProject(), Lists.newArrayList());
+        List<JoinTableDesc> joinTables = copyModel.getJoinTables();
+        joinTables.forEach(joinTableDesc -> {
+            if ("DEFAULT.TEST_ORDER".equals(joinTableDesc.getTable())
+                    || "EDW.TEST_CAL_DT".equals(joinTableDesc.getTable())) {
+                joinTableDesc.setFlattenable(JoinTableDesc.NORMALIZED);
+            }
+        });
+
+        AntiFlatChecker checker = new AntiFlatChecker(joinTables, model);
+
+        // =========== column ==================
+        /* 2 -> TblColRef: "DEFAULT.TEST_KYLIN_FACT.CAL_DT" */
+        Assertions.assertFalse(checker.isColOfAntiLookup(model.getEffectiveCols().get(2)));
+        /* 14 -> TblColRef: "DEFAULT.TEST_ORDER.TEST_DATE_ENC" */
+        Assertions.assertTrue(checker.isColOfAntiLookup(model.getEffectiveCols().get(14)));
+        /* 58 -> TblColRef: "EDW.TEST_CAL_DT.RETAIL_WEEK" */
+        Assertions.assertTrue(checker.isColOfAntiLookup(model.getEffectiveCols().get(58)));
+
+        // ============ measure =================
+
+        /* 100000 FunctionDesc [expression=COUNT, parameter=1, returnType=bigint] */
+        NDataModel.Measure measure1 = model.getEffectiveMeasures().get(100000);
+        Assertions.assertFalse(checker.isMeasureOfAntiLookup(measure1.getFunction()));
+
+        /* 100014 FunctionDesc [expression=COUNT, parameter=EDW.TEST_CAL_DT.CAL_DT, returnType=bigint] */
+        NDataModel.Measure measure2 = model.getEffectiveMeasures().get(100014);
+        Assertions.assertTrue(checker.isMeasureOfAntiLookup(measure2.getFunction()));
+
+        // ============ cc =================
+        /* CC {tableIdentity=DEFAULT.TEST_KYLIN_FACT, columnName=DEAL_AMOUNT,
+                    expression=TEST_KYLIN_FACT.PRICE * TEST_KYLIN_FACT.ITEM_COUNT}
+           27 -> TblColRef: "DEFAULT.TEST_KYLIN_FACT.DEAL_AMOUNT" , from CC */
+        Assertions.assertFalse(checker.isColOfAntiLookup(model.getEffectiveCols().get(27)));
+
+        /* CC {tableIdentity=DEFAULT.TEST_KYLIN_FACT, columnName=DEAL_YEAR,
+                    expression=year(TEST_KYLIN_FACT.CAL_DT)}
+           28 -> TblColRef: "DEFAULT.TEST_KYLIN_FACT.DEAL_YEAR" */
+        Assertions.assertFalse(checker.isColOfAntiLookup(model.getEffectiveCols().get(28)));
+
+        /* CC {tableIdentity=DEFAULT.TEST_KYLIN_FACT, columnName=LEFTJOIN_BUYER_COUNTRY_ABBR,
+                    expression=SUBSTR(BUYER_ACCOUNT.ACCOUNT_COUNTRY,0,1)}
+           31 -> TblColRef: "DEFAULT.TEST_KYLIN_FACT.LEFTJOIN_BUYER_COUNTRY_ABBR" */
+        Assertions.assertTrue(checker.isColOfAntiLookup(model.getEffectiveCols().get(31)));
+    }
+
+    /**
+     * This case shows a wrong result of using a non-initialized joinTables to check.
+     */
+    @Test
+    void exception() {
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        NDataModel model = prepareModel(kylinConfig);
+
+        NDataModel copyModel = JsonUtil.deepCopyQuietly(model, NDataModel.class);
+        List<JoinTableDesc> joinTables = copyModel.getJoinTables();
+        joinTables.forEach(joinTableDesc -> {
+            if ("DEFAULT.TEST_ORDER".equals(joinTableDesc.getTable())
+                    || "EDW.TEST_CAL_DT".equals(joinTableDesc.getTable())) {
+                joinTableDesc.setFlattenable(JoinTableDesc.NORMALIZED);
+            }
+        });
+
+        AntiFlatChecker checker = new AntiFlatChecker(joinTables, model);
+
+        //============ The result should be true, but it is false. ======================
+        /* CC {tableIdentity=DEFAULT.TEST_KYLIN_FACT, columnName=LEFTJOIN_BUYER_COUNTRY_ABBR,
+                    expression=SUBSTR(BUYER_ACCOUNT.ACCOUNT_COUNTRY,0,1)}
+           31 -> TblColRef: "DEFAULT.TEST_KYLIN_FACT.LEFTJOIN_BUYER_COUNTRY_ABBR" */
+        Assertions.assertFalse(checker.isColOfAntiLookup(model.getEffectiveCols().get(31)));
+    }
+}
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
new file mode 100644
index 0000000000..82c7bac737
--- /dev/null
+++ b/src/kylin-it/src/test/java/org/apache/kylin/metadata/model/ColExcludedCheckerTest.java
@@ -0,0 +1,205 @@
+/*
+ * 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.util.List;
+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.util.MetadataTestUtils;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+
+@MetadataInfo
+class ColExcludedCheckerTest {
+    String modelId = "89af4ee2-2cdb-4b07-b39e-4c29856309aa";
+
+    public String getProject() {
+        return "default";
+    }
+
+    private NDataModel prepareModel(KylinConfig kylinConfig) {
+        NDataModelManager modelManager = NDataModelManager.getInstance(kylinConfig, getProject());
+        NDataModel model = modelManager.getDataModelDesc(modelId);
+        model.getComputedColumnDescs().forEach(cc -> {
+            String innerExp = QueryUtil.massageComputedColumn(model, getProject(), cc, null);
+            cc.setInnerExpression(innerExp);
+        });
+        model.init(kylinConfig, getProject(), Lists.newArrayList());
+        return model;
+    }
+
+    @Test
+    void testWithoutTurnOnExcludedTableSettings() {
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        NDataModel model = prepareModel(kylinConfig);
+        ColExcludedChecker checker = new ColExcludedChecker(kylinConfig, getProject(), model);
+        Set<ColumnDesc> excludedCols = checker.getExcludedCols();
+        Assertions.assertTrue(excludedCols.isEmpty());
+    }
+
+    @Test
+    void testIsExcludedComputedColumn() {
+
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        NDataModel model = prepareModel(kylinConfig);
+
+        String factTable = "DEFAULT.TEST_KYLIN_FACT";
+        String lookupTable = "DEFAULT.TEST_ACCOUNT";
+        MetadataTestUtils.mockExcludedTable(getProject(), factTable);
+        MetadataTestUtils.mockExcludedCols(getProject(), lookupTable, Sets.newHashSet("ACCOUNT_ID", "ACCOUNT_COUNTRY"));
+        ColExcludedChecker checker = new ColExcludedChecker(kylinConfig, getProject(), model);
+        Assertions.assertEquals(6, checker.getExcludedCols().size());
+
+        List<ComputedColumnDesc> ccList = model.getComputedColumnDescs();
+        Assertions.assertFalse(checker.isExcludedCC(ccList.get(0)));
+        Assertions.assertFalse(checker.isExcludedCC(ccList.get(1)));
+        Assertions.assertTrue(checker.isExcludedCC(ccList.get(2)));
+        Assertions.assertTrue(checker.isExcludedCC(ccList.get(3)));
+        Assertions.assertTrue(checker.isExcludedCC(ccList.get(4)));
+        Assertions.assertTrue(checker.isExcludedCC(ccList.get(5)));
+        Assertions.assertFalse(checker.isExcludedCC(ccList.get(6)));
+        Assertions.assertFalse(checker.isExcludedCC(ccList.get(7)));
+        Assertions.assertFalse(checker.isExcludedCC(ccList.get(8)));
+        Assertions.assertFalse(checker.isExcludedCC(ccList.get(9)));
+
+        Assertions.assertFalse(checker.isExcludedCC(""));
+    }
+
+    @Test
+    void testIsExcludedMeasure() {
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        NDataModel model = prepareModel(kylinConfig);
+
+        String factTable = "DEFAULT.TEST_KYLIN_FACT";
+        String lookupTable = "DEFAULT.TEST_ACCOUNT";
+        MetadataTestUtils.mockExcludedTable(getProject(), factTable);
+        MetadataTestUtils.mockExcludedCols(getProject(), lookupTable, Sets.newHashSet("ACCOUNT_ID", "ACCOUNT_COUNTRY"));
+        ColExcludedChecker checker = new ColExcludedChecker(kylinConfig, getProject(), model);
+        Assertions.assertEquals(6, checker.getExcludedCols().size());
+
+        // assert measure
+        {
+            ParameterDesc param = new ParameterDesc();
+            param.setColRef(model.getEffectiveCols().get(29));
+            param.setValue("TEST_KYLIN_FACT.LEFTJOIN_BUYER_ID_AND_COUNTRY_NAME");
+            param.setType("column");
+            FunctionDesc function = FunctionDesc.newInstance("sum", ImmutableList.of(param), "int");
+            Assertions.assertTrue(checker.isExcludedMeasure(function));
+        }
+
+        {
+            ParameterDesc param = new ParameterDesc();
+            param.setColRef(model.getEffectiveCols().get(28));
+            param.setValue("");
+            param.setType("column");
+            FunctionDesc function = FunctionDesc.newInstance("sum", ImmutableList.of(param), "int");
+            Assertions.assertFalse(checker.isExcludedMeasure(function));
+        }
+
+        {
+            FunctionDesc function = FunctionDesc.newInstance("sum", ImmutableList.of(), "int");
+            Assertions.assertFalse(checker.isExcludedMeasure(function));
+        }
+    }
+
+    @Test
+    void testFilterRelatedExcludedColumn() {
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        NDataModel model = prepareModel(kylinConfig);
+
+        String factTable = "DEFAULT.TEST_KYLIN_FACT";
+        String lookupTable = "DEFAULT.TEST_ACCOUNT";
+        MetadataTestUtils.mockExcludedTable(getProject(), factTable);
+        MetadataTestUtils.mockExcludedCols(getProject(), lookupTable, Sets.newHashSet("ACCOUNT_ID", "ACCOUNT_COUNTRY"));
+
+        // init without model, usually used by QueryUtil
+        ColExcludedChecker checker = new ColExcludedChecker(kylinConfig, getProject(), null);
+        Assertions.assertEquals(14, checker.getExcludedCols().size());
+
+        // return the given model's excluded columns without considering columns from cc
+        Set<String> excludedCols = checker.filterRelatedExcludedColumn(model);
+        Assertions.assertEquals(2, excludedCols.size());
+        Assertions.assertEquals(Sets.newHashSet("TEST_ACCOUNT.ACCOUNT_ID", "TEST_ACCOUNT.ACCOUNT_COUNTRY"),
+                excludedCols);
+
+        // if model is null, return all project excluded columns
+        Assertions.assertEquals(14, checker.filterRelatedExcludedColumn(null).size());
+    }
+
+    @Test
+    void testNewColExcludedCheckerWithBrokenModel() {
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        NDataModel model = prepareModel(kylinConfig);
+        model.setBroken(true);
+        model.setBrokenReason(NDataModel.BrokenReason.NULL);
+
+        String factTable = "DEFAULT.TEST_KYLIN_FACT";
+        String lookupTable = "DEFAULT.TEST_ACCOUNT";
+        MetadataTestUtils.mockExcludedTable(getProject(), factTable);
+        MetadataTestUtils.mockExcludedCols(getProject(), lookupTable, Sets.newHashSet("ACCOUNT_ID", "ACCOUNT_COUNTRY"));
+
+        ColExcludedChecker checker = new ColExcludedChecker(kylinConfig, getProject(), model);
+        Assertions.assertTrue(checker.getExcludedCols().isEmpty());
+    }
+
+    @Test
+    void testNewColExcludedCheckerWithModelMissingExcludedTable() {
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        NDataModel model = prepareModel(kylinConfig);
+        model.setBroken(true);
+        model.setBrokenReason(NDataModel.BrokenReason.NULL);
+
+        String factTable = "DEFAULT.TEST_KYLIN_FACT";
+        String lookupTable = "DEFAULT.TEST_ACCOUNT";
+        MetadataTestUtils.mockExcludedTable(getProject(), factTable);
+        MetadataTestUtils.mockExcludedCols(getProject(), lookupTable, Sets.newHashSet("ACCOUNT_ID", "ACCOUNT_COUNTRY"));
+
+        NTableMetadataManager tableMgr = NTableMetadataManager.getInstance(kylinConfig, getProject());
+        tableMgr.removeSourceTable("DEFAULT.TEST_ACCOUNT");
+
+        ColExcludedChecker checker = new ColExcludedChecker(kylinConfig, getProject(), model);
+        Assertions.assertTrue(checker.getExcludedCols().isEmpty());
+    }
+
+    @Test
+    void testNewColExcludedCheckerWithModelMissingNonExcludedTable() {
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        NDataModel model = prepareModel(kylinConfig);
+        model.setBroken(true);
+        model.setBrokenReason(NDataModel.BrokenReason.NULL);
+
+        String factTable = "DEFAULT.TEST_KYLIN_FACT";
+        String lookupTable = "DEFAULT.TEST_ACCOUNT";
+        MetadataTestUtils.mockExcludedTable(getProject(), factTable);
+        MetadataTestUtils.mockExcludedCols(getProject(), lookupTable, Sets.newHashSet("ACCOUNT_ID", "ACCOUNT_COUNTRY"));
+
+        NTableMetadataManager tableMgr = NTableMetadataManager.getInstance(kylinConfig, getProject());
+        tableMgr.removeSourceTable("DEFAULT.TEST_ORDER");
+
+        ColExcludedChecker checker = new ColExcludedChecker(kylinConfig, getProject(), model);
+        Assertions.assertTrue(checker.getExcludedCols().isEmpty());
+    }
+}
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 87f4eab000..c3fd6c4df3 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
@@ -138,7 +138,7 @@ public class ExecAndComp {
             schemaFilePath = sqlPath.substring(0, index) + "/result-" + joinType + sqlPath.substring(index) + ".schema";
         }
         try {
-            if (Files.exists(Paths.get(resultFilePath)) && Files.exists(Paths.get(schemaFilePath))) {
+            if (index > 0 && Files.exists(Paths.get(resultFilePath)) && Files.exists(Paths.get(schemaFilePath))) {
                 StructType schema = StructType.fromDDL(new String(Files.readAllBytes(Paths.get(schemaFilePath))));
                 val structs = Arrays.stream(schema.fields()).map(SparderTypeUtil::convertSparkFieldToJavaField)
                         .collect(Collectors.toList());
@@ -360,8 +360,7 @@ public class ExecAndComp {
         // SQLS like "where 1<>1" will be optimized and run locally and no dataset will be returned
         String prevRunLocalConf = Unsafe.setProperty("kylin.query.engine.run-constant-query-locally", "FALSE");
         try {
-            QueryExec queryExec = new QueryExec(prj,
-                    NProjectManager.getInstance(KylinConfig.getInstanceFromEnv()).getProject(prj).getConfig(), true);
+            QueryExec queryExec = new QueryExec(prj, NProjectManager.getProjectConfig(prj), true);
             if (parameters != null) {
                 for (int i = 0; i < parameters.size(); i++) {
                     queryExec.setPrepareParam(i, parameters.get(i));
diff --git a/src/metadata-server/src/main/java/org/apache/kylin/rest/controller/NProjectController.java b/src/metadata-server/src/main/java/org/apache/kylin/rest/controller/NProjectController.java
index 5827b5d6fe..74b85153c3 100644
--- a/src/metadata-server/src/main/java/org/apache/kylin/rest/controller/NProjectController.java
+++ b/src/metadata-server/src/main/java/org/apache/kylin/rest/controller/NProjectController.java
@@ -58,6 +58,7 @@ import org.apache.kylin.rest.request.MultiPartitionConfigRequest;
 import org.apache.kylin.rest.request.OwnerChangeRequest;
 import org.apache.kylin.rest.request.ProjectConfigRequest;
 import org.apache.kylin.rest.request.ProjectConfigResetRequest;
+import org.apache.kylin.rest.request.ProjectExclusionRequest;
 import org.apache.kylin.rest.request.ProjectGeneralInfoRequest;
 import org.apache.kylin.rest.request.ProjectKerberosInfoRequest;
 import org.apache.kylin.rest.request.ProjectRequest;
@@ -285,10 +286,8 @@ public class NProjectController extends NBasicController {
             @RequestBody JobNotificationConfigRequest jobNotificationConfigRequest) {
         checkRequiredArg("data_load_empty_notification_enabled",
                 jobNotificationConfigRequest.getDataLoadEmptyNotificationEnabled());
-        checkRequiredArg("job_notification_states",
-                jobNotificationConfigRequest.getJobNotificationStates());
-        checkRequiredArg("job_notification_emails",
-                jobNotificationConfigRequest.getJobNotificationEmails());
+        checkRequiredArg("job_notification_states", jobNotificationConfigRequest.getJobNotificationStates());
+        checkRequiredArg("job_notification_emails", jobNotificationConfigRequest.getJobNotificationEmails());
         checkRequiredArg("metadata_persist_notification_enabled",
                 jobNotificationConfigRequest.getMetadataPersistNotificationEnabled());
         projectService.updateJobNotificationConfig(project, jobNotificationConfigRequest);
@@ -333,8 +332,7 @@ public class NProjectController extends NBasicController {
             @RequestBody SnapshotConfigRequest snapshotConfigRequest) {
         checkBooleanArg("snapshot_manual_management_enabled",
                 snapshotConfigRequest.getSnapshotManualManagementEnabled());
-        checkBooleanArg("snapshot_automatic_refresh_enabled",
-                snapshotConfigRequest.getSnapshotAutoRefreshEnabled());
+        checkBooleanArg("snapshot_automatic_refresh_enabled", snapshotConfigRequest.getSnapshotAutoRefreshEnabled());
         projectService.updateSnapshotConfig(project, snapshotConfigRequest);
         return new EnvelopeResponse<>(KylinException.CODE_SUCCESS, "", "");
     }
@@ -498,4 +496,13 @@ public class NProjectController extends NBasicController {
         projectService.updateJdbcInfo(project, request);
         return new EnvelopeResponse<>(KylinException.CODE_SUCCESS, "", "");
     }
+
+    @ApiOperation(value = "updateProjectExclusionInfo", notes = "Add URL: {project}; ")
+    @PutMapping(value = "/{project:.+}/exclusion_enabled")
+    @ResponseBody
+    public EnvelopeResponse<String> updateTableExclusionConfig(@PathVariable("project") String project,
+            @RequestBody ProjectExclusionRequest request) {
+        projectService.updateTableExclusionRule(project, request);
+        return new EnvelopeResponse<>(KylinException.CODE_SUCCESS, "", "");
+    }
 }
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 6b63490708..128a9ebb21 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
@@ -48,12 +48,15 @@ import org.apache.kylin.rest.request.AutoMergeRequest;
 import org.apache.kylin.rest.request.PartitionKeyRequest;
 import org.apache.kylin.rest.request.PushDownModeRequest;
 import org.apache.kylin.rest.request.ReloadTableRequest;
+import org.apache.kylin.rest.request.TableExclusionRequest;
 import org.apache.kylin.rest.request.TableLoadRequest;
 import org.apache.kylin.rest.request.TopTableRequest;
 import org.apache.kylin.rest.request.UpdateAWSTableExtDescRequest;
 import org.apache.kylin.rest.response.AutoMergeConfigResponse;
 import org.apache.kylin.rest.response.DataResult;
 import org.apache.kylin.rest.response.EnvelopeResponse;
+import org.apache.kylin.rest.response.ExcludedTableDetailResponse;
+import org.apache.kylin.rest.response.ExcludedTableResponse;
 import org.apache.kylin.rest.response.LoadTableResponse;
 import org.apache.kylin.rest.response.NHiveTableNameResponse;
 import org.apache.kylin.rest.response.NInitTablesResponse;
@@ -65,7 +68,6 @@ import org.apache.kylin.rest.response.TableRefresh;
 import org.apache.kylin.rest.response.TableRefreshAll;
 import org.apache.kylin.rest.response.TablesAndColumnsResponse;
 import org.apache.kylin.rest.response.UpdateAWSTableExtDescResponse;
-import org.apache.kylin.rest.service.ModelBuildSupporter;
 import org.apache.kylin.rest.service.ModelService;
 import org.apache.kylin.rest.service.TableExtService;
 import org.apache.kylin.rest.service.TableSamplingService;
@@ -104,10 +106,6 @@ public class NTableController extends NBasicController {
     @Qualifier("modelService")
     private ModelService modelService;
 
-    @Autowired
-    @Qualifier("modelBuildService")
-    private ModelBuildSupporter modelBuildService;
-
     @Autowired
     @Qualifier("tableSamplingService")
     private TableSamplingService tableSamplingService;
@@ -145,15 +143,17 @@ public class NTableController extends NBasicController {
             @RequestParam(value = "is_fuzzy", required = false, defaultValue = "false") boolean isFuzzy,
             @RequestParam(value = "page_offset", required = false, defaultValue = "0") Integer offset,
             @RequestParam(value = "page_size", required = false, defaultValue = "10") Integer limit,
+            @RequestParam(value = "with_excluded", required = false, defaultValue = "true") boolean withExcluded,
             @RequestParam(value = "source_type", required = false, defaultValue = "9") List<Integer> sourceType)
             throws Exception {
 
         String projectName = checkProjectName(project);
-        return new EnvelopeResponse<>(KylinException.CODE_SUCCESS,
-                tableService.getProjectTables(projectName, table, offset, limit, false, (databaseName, tableName) -> {
+        NInitTablesResponse projectTables = tableService.getProjectTables(projectName, table, offset, limit,
+                withExcluded, false, (databaseName, tableName) -> {
                     return tableService.getTableDescByTypes(projectName, withExt, tableName, databaseName, isFuzzy,
                             sourceType);
-                }), "");
+                });
+        return new EnvelopeResponse<>(KylinException.CODE_SUCCESS, projectTables, "");
     }
 
     @ApiOperation(value = "unloadTable", tags = { "AI" }, notes = "Update URL: {project}; Update Param: project")
@@ -319,9 +319,6 @@ public class NTableController extends NBasicController {
 
     /**
      * Show all tablesNames
-     *
-     * @return String[]
-     * @throws IOException
      */
     @ApiOperation(value = "showTables", tags = {
             "AI" }, notes = "Update Param: data_source_type, page_offset, page_size; Update Response: total_size")
@@ -333,7 +330,7 @@ public class NTableController extends NBasicController {
             @RequestParam(value = "table", required = false) String table,
             @RequestParam(value = "page_offset", required = false, defaultValue = "0") Integer offset,
             @RequestParam(value = "page_size", required = false, defaultValue = "10") Integer limit,
-            @RequestParam(value = "database", required = true) String database) throws Exception {
+            @RequestParam(value = "database") String database) throws Exception {
         checkProjectName(project);
         List<TableNameResponse> tables = tableService.getTableNameResponses(project, database, table);
         return new EnvelopeResponse<>(KylinException.CODE_SUCCESS, DataResult.get(tables, offset, limit), "");
@@ -350,7 +347,7 @@ public class NTableController extends NBasicController {
             @RequestParam(value = "page_size", required = false, defaultValue = "10") Integer limit) throws Exception {
         String projectName = checkProjectName(project);
         return new EnvelopeResponse<>(KylinException.CODE_SUCCESS,
-                tableService.getProjectTables(projectName, table, offset, limit, true, (databaseName,
+                tableService.getProjectTables(projectName, table, offset, limit, true, true, (databaseName,
                         tableName) -> tableService.getHiveTableNameResponses(projectName, databaseName, tableName)),
                 "");
     }
@@ -514,7 +511,7 @@ public class NTableController extends NBasicController {
     @ApiOperation(value = "catalogCache", tags = { "DW" })
     @PutMapping(value = "catalog_cache", produces = { HTTP_VND_APACHE_KYLIN_V4_PUBLIC_JSON })
     @ResponseBody
-    public EnvelopeResponse refreshCatalogCache(final HttpServletRequest refreshRequest) {
+    public EnvelopeResponse<TableRefreshAll> refreshCatalogCache(final HttpServletRequest refreshRequest) {
         TableRefreshAll response = tableService.refreshAllCatalogCache(refreshRequest);
         return new EnvelopeResponse<>(response.getCode(), response, response.getMsg());
     }
@@ -522,7 +519,7 @@ public class NTableController extends NBasicController {
     @ApiOperation(value = "modelTables", tags = { "AI" })
     @GetMapping(value = "/model_tables")
     @ResponseBody
-    public EnvelopeResponse getModelTables(@RequestParam("project") String project,
+    public EnvelopeResponse<List<TableDesc>> getModelTables(@RequestParam("project") String project,
             @RequestParam("model_name") String modelName) {
         checkProjectName(project);
         val res = tableService.getTablesOfModel(project, modelName);
@@ -540,4 +537,63 @@ public class NTableController extends NBasicController {
             throw new KylinException(INVALID_TABLE_REFRESH_PARAMETER, message.getTableRefreshParamInvalid(), false);
         }
     }
+
+    /**
+     * Get excluded tables.
+     * @param project project name
+     * @param pageOffset page offset of tables
+     * @param pageSize page size of tables
+     * @param viewPartialCols view partial columns of each excluded table
+     * @param searchKey fuzzy search related excluded tables
+     * @return Return excluded tables.
+     */
+    @ApiOperation(value = "getExcludedTables", notes = "Add URL: {project}; ")
+    @GetMapping(value = "/excluded_tables")
+    @ResponseBody
+    public EnvelopeResponse<DataResult<List<ExcludedTableResponse>>> getExcludedTables(
+            @RequestParam(value = "project") String project,
+            @RequestParam(value = "page_offset", required = false, defaultValue = "0") int pageOffset,
+            @RequestParam(value = "page_size", required = false, defaultValue = "10") int pageSize,
+            @RequestParam(value = "view_partial_cols", required = false, defaultValue = "true") boolean viewPartialCols,
+            @RequestParam(value = "key", required = false, defaultValue = "") String searchKey) {
+        List<ExcludedTableResponse> list = tableExtService.getExcludedTables(project, true, searchKey);
+        return new EnvelopeResponse<>(KylinException.CODE_SUCCESS, DataResult.get(list, pageOffset, pageSize), "");
+    }
+
+    /**
+     * Get info of specified excluded table.
+     * @param project project name
+     * @param table excluded table name
+     * @param pageOffset page offset of columns
+     * @param pageSize page size of columns
+     * @param searchKey fuzzy search columns
+     * @param colType column types: 0 - columns to be excluded, 1- excluded columns
+     */
+    @ApiOperation(value = "getExcludedTable", notes = "Add URL: {project}; ")
+    @GetMapping(value = "/excluded_table")
+    @ResponseBody
+    public EnvelopeResponse<ExcludedTableDetailResponse> getExcludedTable(
+            @RequestParam(value = "project") String project, @RequestParam(value = "table") String table,
+            @RequestParam(value = "page_offset", required = false, defaultValue = "0") int pageOffset,
+            @RequestParam(value = "page_size", required = false, defaultValue = "10") int pageSize,
+            @RequestParam(value = "key", required = false, defaultValue = "") String searchKey,
+            @RequestParam(value = "col_type", required = false, defaultValue = "0") int colType) {
+        String projectName = checkProjectName(project);
+        ExcludedTableDetailResponse response = tableExtService.getExcludedTable(projectName, table, pageOffset,
+                pageSize, searchKey, colType == 1);
+        return new EnvelopeResponse<>(KylinException.CODE_SUCCESS, response, "");
+    }
+
+    /**
+     * Update excluded table info.
+     */
+    @ApiOperation(value = "updateExcludedTables", tags = { "RR" }, notes = "Add URL: {project}; ")
+    @PutMapping(value = "/excluded_tables")
+    @ResponseBody
+    public EnvelopeResponse<String> updateExcludedTables(@RequestBody TableExclusionRequest request) {
+        String projectName = checkProjectName(request.getProject());
+        request.setProject(projectName);
+        tableExtService.updateExcludedTables(request.getProject(), request);
+        return new EnvelopeResponse<>(KylinException.CODE_SUCCESS, "", "");
+    }
 }
diff --git a/src/metadata-server/src/test/java/org/apache/kylin/rest/controller/NProjectControllerTest.java b/src/metadata-server/src/test/java/org/apache/kylin/rest/controller/NProjectControllerTest.java
index d8235a9983..8c01828c82 100644
--- a/src/metadata-server/src/test/java/org/apache/kylin/rest/controller/NProjectControllerTest.java
+++ b/src/metadata-server/src/test/java/org/apache/kylin/rest/controller/NProjectControllerTest.java
@@ -46,6 +46,7 @@ import org.apache.kylin.rest.request.JdbcSourceInfoRequest;
 import org.apache.kylin.rest.request.JobNotificationConfigRequest;
 import org.apache.kylin.rest.request.OwnerChangeRequest;
 import org.apache.kylin.rest.request.ProjectConfigRequest;
+import org.apache.kylin.rest.request.ProjectExclusionRequest;
 import org.apache.kylin.rest.request.ProjectGeneralInfoRequest;
 import org.apache.kylin.rest.request.ProjectRequest;
 import org.apache.kylin.rest.request.PushDownConfigRequest;
@@ -104,7 +105,7 @@ public class NProjectControllerTest extends NLocalFileMetadataTestCase {
     @Before
     public void setup() {
         createTestMetadata();
-        MockitoAnnotations.initMocks(this);
+        MockitoAnnotations.openMocks(this);
 
         mockMvc = MockMvcBuilders.standaloneSetup(nProjectController).defaultRequest(MockMvcRequestBuilders.get("/"))
                 .build();
@@ -533,4 +534,16 @@ public class NProjectControllerTest extends NLocalFileMetadataTestCase {
 
         Mockito.verify(projectService).updateJdbcInfo(any(), Mockito.any());
     }
+
+    @Test
+    public void testUpdateTableExclusionRule() throws Exception {
+        ProjectExclusionRequest request = new ProjectExclusionRequest();
+        request.setTableExclusionEnabled(true);
+        Mockito.doNothing().when(projectService).updateTableExclusionRule("project", request);
+        mockMvc.perform(MockMvcRequestBuilders.put("/api/projects/{project}/exclusion_enabled", "project")
+                .contentType(MediaType.APPLICATION_JSON).content(JsonUtil.writeValueAsString(request))
+                .accept(MediaType.parseMediaType(HTTP_VND_APACHE_KYLIN_JSON)))
+                .andExpect(MockMvcResultMatchers.status().isOk()).andReturn();
+        Mockito.verify(nProjectController).updateTableExclusionConfig("project", request);
+    }
 }
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 31be7be76f..3301be1d06 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
@@ -37,14 +37,15 @@ import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.rest.constant.Constant;
 import org.apache.kylin.rest.request.AWSTableLoadRequest;
 import org.apache.kylin.rest.request.AutoMergeRequest;
-import org.apache.kylin.rest.request.DateRangeRequest;
 import org.apache.kylin.rest.request.PartitionKeyRequest;
 import org.apache.kylin.rest.request.PushDownModeRequest;
 import org.apache.kylin.rest.request.ReloadTableRequest;
 import org.apache.kylin.rest.request.S3TableExtInfo;
+import org.apache.kylin.rest.request.TableExclusionRequest;
 import org.apache.kylin.rest.request.TableLoadRequest;
 import org.apache.kylin.rest.request.TopTableRequest;
 import org.apache.kylin.rest.request.UpdateAWSTableExtDescRequest;
+import org.apache.kylin.rest.response.ExcludedTableDetailResponse;
 import org.apache.kylin.rest.response.LoadTableResponse;
 import org.apache.kylin.rest.response.TableNameResponse;
 import org.apache.kylin.rest.response.TableRefresh;
@@ -283,15 +284,6 @@ public class NTableControllerTest extends NLocalFileMetadataTestCase {
         return topTableRequest;
     }
 
-    private DateRangeRequest mockDateRangeRequest() {
-        DateRangeRequest request = new DateRangeRequest();
-        request.setStart("0");
-        request.setEnd("" + Long.MAX_VALUE);
-        request.setProject("default");
-        request.setTable("TEST_KYLIN_FACT");
-        return request;
-    }
-
     private void initMockito(LoadTableResponse loadTableResponse, TableLoadRequest tableLoadRequest) throws Exception {
         StringUtil.toUpperCaseArray(tableLoadRequest.getTables(), tableLoadRequest.getTables());
         StringUtil.toUpperCaseArray(tableLoadRequest.getDatabases(), tableLoadRequest.getDatabases());
@@ -420,8 +412,8 @@ public class NTableControllerTest extends NLocalFileMetadataTestCase {
         loadTableResponse.setLoaded(loaded);
         loadTableResponse.setFailed(failed);
 
-        Mockito.when(tableExtService.loadAWSTablesCompatibleCrossAccount(tableLoadRequest.getTables(),
-                        "default")).thenReturn(loadTableResponse);
+        Mockito.when(tableExtService.loadAWSTablesCompatibleCrossAccount(tableLoadRequest.getTables(), "default"))
+                .thenReturn(loadTableResponse);
 
         mockMvc.perform(MockMvcRequestBuilders.post("/api/tables/compatibility/aws") //
                 .contentType(MediaType.APPLICATION_JSON) //
@@ -454,8 +446,7 @@ public class NTableControllerTest extends NLocalFileMetadataTestCase {
         updateTableExeDescResponse.setSucceed(succeed);
         updateTableExeDescResponse.setFailed(failed);
 
-        Mockito.when(tableExtService.updateAWSLoadedTableExtProp(request))
-                .thenReturn(updateTableExeDescResponse);
+        Mockito.when(tableExtService.updateAWSLoadedTableExtProp(request)).thenReturn(updateTableExeDescResponse);
 
         mockMvc.perform(MockMvcRequestBuilders.put("/api/tables/ext/prop/aws") //
                 .contentType(MediaType.APPLICATION_JSON) //
@@ -784,4 +775,56 @@ public class NTableControllerTest extends NLocalFileMetadataTestCase {
                 .andExpect(MockMvcResultMatchers.status().isOk()).andReturn();
         Mockito.verify(nTableController).reloadTable(Mockito.any());
     }
+
+    @Test
+    public void updateExcludedTables() throws Exception {
+        TableExclusionRequest request = new TableExclusionRequest();
+        request.setProject("default");
+        TableExclusionRequest.ExcludedTable excludedTable = new TableExclusionRequest.ExcludedTable();
+        excludedTable.setExcluded(true);
+        excludedTable.setTable("default.test_order");
+        request.setExcludedTables(Lists.newArrayList(excludedTable));
+        List<String> cancelTables = Lists.newArrayList("default.test_account");
+        request.setCanceledTables(cancelTables);
+
+        Mockito.doNothing().when(tableExtService).updateExcludedTables("default", request);
+        mockMvc.perform(MockMvcRequestBuilders.put("/api/tables/excluded_tables")
+                .contentType(MediaType.APPLICATION_JSON).content(JsonUtil.writeValueAsString(request))
+                .accept(MediaType.parseMediaType(HTTP_VND_APACHE_KYLIN_JSON)))
+                .andExpect(MockMvcResultMatchers.status().isOk()).andReturn();
+        Mockito.verify(nTableController).updateExcludedTables(request);
+    }
+
+    @Test
+    public void testGetExcludedTable() throws Exception {
+        Mockito.when(tableExtService.getExcludedTable("default", "test_kylin_fact", 0, 10, "", true)) //
+                .thenReturn(Mockito.mock(ExcludedTableDetailResponse.class));
+        mockMvc.perform(MockMvcRequestBuilders.get("/api/tables/excluded_table") //
+                .contentType(MediaType.APPLICATION_JSON) //
+                .param("project", "default") //
+                .param("table", "test_kylin_fact") //
+                .param("page_offset", "0") //
+                .param("page_size", "10") //
+                .param("key", "") //
+                .param("col_type", "0") //
+                .accept(MediaType.parseMediaType(APPLICATION_JSON))) //
+                .andExpect(MockMvcResultMatchers.status().isOk()).andReturn();
+        Mockito.verify(nTableController).getExcludedTable("default", "test_kylin_fact", 0, 10, "", 0);
+    }
+
+    @Test
+    public void testGetExcludedTables() throws Exception {
+        Mockito.when(tableExtService.getExcludedTables("default", true, "")) //
+                .thenReturn(Lists.newArrayList());
+        mockMvc.perform(MockMvcRequestBuilders.get("/api/tables/excluded_tables") //
+                .contentType(MediaType.APPLICATION_JSON) //
+                .param("project", "default") //
+                .param("page_offset", "0") //
+                .param("page_size", "10") //
+                .param("view_partial_cols", "true") //
+                .param("search_key", "") //
+                .accept(MediaType.parseMediaType(APPLICATION_JSON))) //
+                .andExpect(MockMvcResultMatchers.status().isOk()).andReturn();
+        Mockito.verify(nTableController).getExcludedTables("default", 0, 10, true, "");
+    }
 }
diff --git a/src/modeling-service/pom.xml b/src/modeling-service/pom.xml
index 355240107c..fbc7303886 100644
--- a/src/modeling-service/pom.xml
+++ b/src/modeling-service/pom.xml
@@ -67,6 +67,12 @@
             <type>test-jar</type>
             <scope>test</scope>
         </dependency>
+        <dependency>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-core-metadata</artifactId>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
         <dependency>
             <groupId>org.apache.kylin</groupId>
             <artifactId>kylin-datasource-service</artifactId>
@@ -143,6 +149,21 @@
             <artifactId>h2</artifactId>
             <scope>test</scope>
         </dependency>
+        <dependency>
+            <groupId>org.junit.jupiter</groupId>
+            <artifactId>junit-jupiter-api</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.junit.vintage</groupId>
+            <artifactId>junit-vintage-engine</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.junit-pioneer</groupId>
+            <artifactId>junit-pioneer</artifactId>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
 
 </project>
diff --git a/src/modeling-service/src/main/java/org/apache/kylin/rest/response/NDataModelResponse.java b/src/modeling-service/src/main/java/org/apache/kylin/rest/response/NDataModelResponse.java
index 3ef3a69a9f..846bdc7855 100644
--- a/src/modeling-service/src/main/java/org/apache/kylin/rest/response/NDataModelResponse.java
+++ b/src/modeling-service/src/main/java/org/apache/kylin/rest/response/NDataModelResponse.java
@@ -25,27 +25,25 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
-import java.util.Set;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 
 import org.apache.commons.lang3.builder.HashCodeExclude;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.metadata.model.ColumnDesc;
-import org.apache.kylin.metadata.model.JoinTableDesc;
-import org.apache.kylin.metadata.model.SegmentRange;
-import org.apache.kylin.metadata.model.TableExtDesc;
-import org.apache.kylin.metadata.model.TableRef;
-import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.acl.NDataModelAclParams;
 import org.apache.kylin.metadata.cube.model.IndexPlan;
 import org.apache.kylin.metadata.cube.model.NDataflowManager;
 import org.apache.kylin.metadata.cube.model.NIndexPlanManager;
-import org.apache.kylin.metadata.favorite.FavoriteRuleManager;
-import org.apache.kylin.metadata.model.ExcludedLookupChecker;
+import org.apache.kylin.metadata.model.ColExcludedChecker;
+import org.apache.kylin.metadata.model.ColumnDesc;
+import org.apache.kylin.metadata.model.JoinTableDesc;
 import org.apache.kylin.metadata.model.NDataModel;
 import org.apache.kylin.metadata.model.NDataModelManager;
 import org.apache.kylin.metadata.model.NTableMetadataManager;
+import org.apache.kylin.metadata.model.SegmentRange;
+import org.apache.kylin.metadata.model.TableExtDesc;
+import org.apache.kylin.metadata.model.TableRef;
+import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.model.util.scd2.SimplifiedJoinTableDesc;
 import org.apache.kylin.rest.constant.ModelStatusToDisplayEnum;
 import org.apache.kylin.rest.util.ModelUtils;
@@ -57,7 +55,6 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonUnwrapped;
 import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
 
 import io.kyligence.kap.secondstorage.response.SecondStorageNode;
 import lombok.Data;
@@ -152,6 +149,11 @@ public class NDataModelResponse extends NDataModel {
         super();
     }
 
+    @Override
+    public KylinConfig getConfig() {
+        return super.getConfig() == null ? KylinConfig.getInstanceFromEnv() : super.getConfig();
+    }
+
     public NDataModelResponse(NDataModel dataModel) {
         super(dataModel);
         this.setConfig(dataModel.getConfig());
@@ -186,13 +188,11 @@ public class NDataModelResponse extends NDataModel {
             tableMetadata = NTableMetadataManager.getInstance(getConfig(), this.getProject());
         }
 
-        Set<String> excludedTables = loadExcludedTables();
-        List<JoinTableDesc> joinTables = getLazyModel().getJoinTables();
-        ExcludedLookupChecker checker = new ExcludedLookupChecker(excludedTables, joinTables, getLazyModel());
+        ColExcludedChecker excludedChecker = new ColExcludedChecker(getConfig(), getProject(), this);
         List<SimplifiedNamedColumn> dimList = Lists.newArrayList();
         for (NamedColumn col : getAllNamedColumns()) {
             if (col.isDimension()) {
-                dimList.add(transColumnToDim(checker, col, tableMetadata));
+                dimList.add(transColumnToDim(excludedChecker, col, tableMetadata));
             }
         }
         if (!onlyNormalDim) {
@@ -208,7 +208,7 @@ public class NDataModelResponse extends NDataModel {
                     for (TblColRef col : joinTable.getTableRef().getColumns()) {
                         NamedColumn namedColumn = columnMap.get(col.getAliasDotName());
                         if (!namedColumn.isDimension()) {
-                            dimList.add(transColumnToDim(checker, namedColumn, tableMetadata));
+                            dimList.add(transColumnToDim(excludedChecker, namedColumn, tableMetadata));
                             namedColumn.setStatus(DIMENSION);
                         }
                     }
@@ -237,7 +237,7 @@ public class NDataModelResponse extends NDataModel {
                 .getDataModelDesc(this.getUuid());
     }
 
-    public SimplifiedNamedColumn transColumnToDim(ExcludedLookupChecker checker, NamedColumn col,
+    public SimplifiedNamedColumn transColumnToDim(ColExcludedChecker excludedChecker, NamedColumn col,
             NTableMetadataManager tableMetadata) {
         SimplifiedNamedColumn simplifiedDimension = new SimplifiedNamedColumn(col);
         simplifiedDimension.setStatus(DIMENSION);
@@ -245,8 +245,9 @@ public class NDataModelResponse extends NDataModel {
         if (colRef == null || tableMetadata == null) {
             return simplifiedDimension;
         }
-        if (checker.isColRefDependsLookupTable(colRef)) {
-            simplifiedDimension.setDependLookupTable(true);
+        if (excludedChecker.isExcludedCol(colRef)
+                && !colRef.getTableRef().getTableIdentity().equals(getLazyModel().getRootFactTableName())) {
+            simplifiedDimension.setExcluded(true);
         }
         TableExtDesc tableExt = tableMetadata.getTableExtIfExists(colRef.getTableRef().getTableDesc());
         if (tableExt != null) {
@@ -344,8 +345,8 @@ public class NDataModelResponse extends NDataModel {
         }
 
         @HashCodeExclude
-        @JsonProperty("depend_lookup_table")
-        private boolean dependLookupTable;
+        @JsonProperty("excluded")
+        private boolean excluded;
 
         @JsonProperty("cardinality")
         private Long cardinality;
@@ -392,18 +393,18 @@ public class NDataModelResponse extends NDataModel {
     public List<SimplifiedNamedColumn> getSelectedColumns() {
         List<SimplifiedNamedColumn> selectedColumns = Lists.newArrayList();
         NTableMetadataManager tableMetadata = null;
+        KylinConfig config = getConfig();
         if (!isBroken()) {
-            tableMetadata = NTableMetadataManager.getInstance(getConfig(), this.getProject());
+            tableMetadata = NTableMetadataManager.getInstance(config, this.getProject());
         }
-        Set<String> excludedTables = loadExcludedTables();
-        List<JoinTableDesc> joinTables = getLazyModel().getJoinTables();
-        ExcludedLookupChecker checker = new ExcludedLookupChecker(excludedTables, joinTables, getLazyModel());
+        ColExcludedChecker excludedChecker = new ColExcludedChecker(config, getProject(), this);
         for (NamedColumn namedColumn : getAllSelectedColumns()) {
             SimplifiedNamedColumn simplifiedNamedColumn = new SimplifiedNamedColumn(namedColumn);
             TblColRef colRef = findColumnByAlias(simplifiedNamedColumn.getAliasDotColumn());
             if (simplifiedNamedColumn.getStatus() == DIMENSION && colRef != null && tableMetadata != null) {
-                if (checker.isColRefDependsLookupTable(colRef)) {
-                    simplifiedNamedColumn.setDependLookupTable(true);
+                if (excludedChecker.isExcludedCol(colRef)
+                        && !colRef.getTableRef().getTableIdentity().equals(getLazyModel().getRootFactTableName())) {
+                    simplifiedNamedColumn.setExcluded(true);
                 }
                 TableExtDesc tableExt = tableMetadata.getTableExtIfExists(colRef.getTableRef().getTableDesc());
                 TableExtDesc.ColumnStats columnStats = Objects.isNull(tableExt) ? null
@@ -419,18 +420,6 @@ public class NDataModelResponse extends NDataModel {
         return selectedColumns;
     }
 
-    private Set<String> loadExcludedTables() {
-        FavoriteRuleManager favoriteRuleManager = null;
-        if (!isBroken()) {
-            favoriteRuleManager = FavoriteRuleManager.getInstance(getConfig(), getProject());
-        }
-        Set<String> excludedTables = Sets.newHashSet();
-        if (favoriteRuleManager != null) {
-            excludedTables.addAll(favoriteRuleManager.getExcludedTables());
-        }
-        return excludedTables;
-    }
-
     public void computedInfo(long inconsistentCount, ModelStatusToDisplayEnum status, boolean isScd2,
             NDataModel modelDesc, boolean onlyNormalDim) {
         if (!onlyNormalDim) {
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 1f915c3dfa..8a0beec63b 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
@@ -157,12 +157,11 @@ import org.apache.kylin.metadata.cube.model.NDataflowManager;
 import org.apache.kylin.metadata.cube.model.NDataflowUpdate;
 import org.apache.kylin.metadata.cube.model.NIndexPlanManager;
 import org.apache.kylin.metadata.cube.model.RuleBasedIndex;
-import org.apache.kylin.metadata.favorite.FavoriteRuleManager;
+import org.apache.kylin.metadata.model.AntiFlatChecker;
 import org.apache.kylin.metadata.model.AutoMergeTimeEnum;
 import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.ComputedColumnDesc;
 import org.apache.kylin.metadata.model.DataCheckDesc;
-import org.apache.kylin.metadata.model.ExcludedLookupChecker;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.FusionModel;
 import org.apache.kylin.metadata.model.FusionModelManager;
@@ -2642,8 +2641,7 @@ public class ModelService extends AbstractModelService implements TableModelSupp
         ComputedColumnDesc checkedCC = null;
 
         QueryContext.AclInfo aclInfo = AclPermissionUtil.createAclInfo(project, getCurrentUserGroups());
-        Set<String> excludedTables = getManager(FavoriteRuleManager.class, project).getExcludedTables();
-        ExcludedLookupChecker checker = new ExcludedLookupChecker(excludedTables, model.getJoinTables(), model);
+        AntiFlatChecker checker = new AntiFlatChecker(model.getJoinTables(), model);
         for (ComputedColumnDesc cc : model.getComputedColumnDescs()) {
             checkCCName(cc.getColumnName());
 
@@ -3101,8 +3099,7 @@ public class ModelService extends AbstractModelService implements TableModelSupp
         List<JoinTableDesc> joinTables = convertedModel.getJoinTables();
 
         IndexPlan indexPlan = indexPlanManager.getIndexPlan(uuid);
-        Set<String> excludedTables = getManager(FavoriteRuleManager.class, project).getExcludedTables();
-        ExcludedLookupChecker checker = new ExcludedLookupChecker(excludedTables, joinTables, convertedModel);
+        AntiFlatChecker checker = new AntiFlatChecker(joinTables, convertedModel);
         List<ComputedColumnDesc> invalidCCList = checker.getInvalidComputedColumns(convertedModel);
         Set<Integer> invalidDimensions = checker.getInvalidDimensions(convertedModel);
         Set<Integer> invalidMeasures = checker.getInvalidMeasures(convertedModel);
@@ -3643,14 +3640,12 @@ public class ModelService extends AbstractModelService implements TableModelSupp
 
         sqlNode.accept(sqlVisitor);
 
-        if (!KylinConfig.getInstanceFromEnv().isBuildExcludedTableEnabled()) {
-            Set<String> excludedTables = getManager(FavoriteRuleManager.class, model.getProject()).getExcludedTables();
-            ExcludedLookupChecker checker = new ExcludedLookupChecker(excludedTables, model.getJoinTables(), model);
-            String antiFlattenLookup = checker.detectFilterConditionDependsLookups(sqlNode.toString(),
-                    checker.getExcludedLookups());
-            if (antiFlattenLookup != null) {
+        if (!NProjectManager.getProjectConfig(model.getProject()).isBuildExcludedTableEnabled()) {
+            AntiFlatChecker checker = new AntiFlatChecker(model.getJoinTables(), model);
+            String antiFlatLookup = checker.detectFilterCondition(sqlNode.toString());
+            if (antiFlatLookup != null) {
                 throw new KylinException(FILTER_CONDITION_DEPENDS_ANTI_FLATTEN_LOOKUP, String.format(Locale.ROOT,
-                        MsgPicker.getMsg().getFilterConditionOnAntiFlattenLookup(), antiFlattenLookup));
+                        MsgPicker.getMsg().getFilterConditionOnAntiFlattenLookup(), antiFlatLookup));
             }
         }
         String exp = sqlNode
@@ -4075,8 +4070,7 @@ public class ModelService extends AbstractModelService implements TableModelSupp
         String uuid = model.getUuid();
         List<JoinTableDesc> joinTables = model.getJoinTables();
         IndexPlan indexPlan = getManager(NIndexPlanManager.class, project).getIndexPlan(uuid);
-        Set<String> excludedTables = getManager(FavoriteRuleManager.class, project).getExcludedTables();
-        ExcludedLookupChecker checker = new ExcludedLookupChecker(excludedTables, joinTables, model);
+        AntiFlatChecker checker = new AntiFlatChecker(joinTables, model);
         List<ComputedColumnDesc> invalidCCList = checker.getInvalidComputedColumns(model);
         Set<Integer> invalidDimensions = checker.getInvalidDimensions(model);
         Set<Integer> invalidMeasures = checker.getInvalidMeasures(model);
@@ -4093,7 +4087,6 @@ public class ModelService extends AbstractModelService implements TableModelSupp
                 aggIndexCount.getAndIncrement();
             }
         });
-        List<String> antiFlattenLookupTables = checker.getAntiFlattenLookups();
 
         List<String> invalidDimensionNames = model.getAllNamedColumns().stream()
                 .filter(col -> invalidDimensions.contains(col.getId())).map(NDataModel.NamedColumn::getAliasDotColumn)
@@ -4109,7 +4102,7 @@ public class ModelService extends AbstractModelService implements TableModelSupp
         response.setIndexes(Lists.newArrayList(invalidIndexes));
         response.setInvalidAggIndexCount(aggIndexCount.get());
         response.setInvalidTableIndexCount(tableIndexCount.get());
-        response.setAntiFlattenLookups(antiFlattenLookupTables);
+        response.setAntiFlattenLookups(Lists.newArrayList(checker.getAntiFlattenLookups()));
         return response;
     }
 
diff --git a/src/modeling-service/src/test/java/org/apache/kylin/rest/response/NDataModelResponseTest.java b/src/modeling-service/src/test/java/org/apache/kylin/rest/response/NDataModelResponseTest.java
index 5949b3fd77..231baf8f89 100644
--- a/src/modeling-service/src/test/java/org/apache/kylin/rest/response/NDataModelResponseTest.java
+++ b/src/modeling-service/src/test/java/org/apache/kylin/rest/response/NDataModelResponseTest.java
@@ -19,39 +19,104 @@
 package org.apache.kylin.rest.response;
 
 import static com.google.common.collect.Lists.newArrayList;
+import static org.apache.kylin.common.util.TestUtils.getTestConfig;
 import static org.apache.kylin.metadata.model.FunctionDesc.FUNC_COUNT;
 
 import java.util.List;
 
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.junit.annotation.MetadataInfo;
 import org.apache.kylin.metadata.model.FunctionDesc;
-import org.apache.kylin.metadata.model.ParameterDesc;
-import org.apache.kylin.common.util.NLocalFileMetadataTestCase;
 import org.apache.kylin.metadata.model.NDataModel;
 import org.apache.kylin.metadata.model.NDataModelManager;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.apache.kylin.metadata.model.ParameterDesc;
+import org.apache.kylin.metadata.project.EnhancedUnitOfWork;
+import org.apache.kylin.util.MetadataTestUtils;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
 
 import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
 
-public class NDataModelResponseTest extends NLocalFileMetadataTestCase {
+@MetadataInfo(project = "default")
+class NDataModelResponseTest {
 
     private static final String PROJECT = "default";
 
-    @Before
-    public void setup() {
-        createTestMetadata();
+    @Test
+    void testGetSelectedColumnsAndSimplifiedDimensionsNormal() {
+        List<NDataModel.NamedColumn> allNamedColumns = Lists.newArrayList();
+        NDataModel.NamedColumn namedColumn = new NDataModel.NamedColumn();
+        namedColumn.setName("PRICE1");
+        namedColumn.setAliasDotColumn("TEST_KYLIN_FACT.PRICE");
+        namedColumn.setStatus(NDataModel.ColumnStatus.DIMENSION);
+        allNamedColumns.add(namedColumn);
+
+        NDataModel model = new NDataModel();
+        model.setUuid("model");
+        model.setProject(PROJECT);
+        model.setAllNamedColumns(allNamedColumns);
+        model.setAllMeasures(Lists.newArrayList(createMeasure()));
+        model.setRootFactTableName("DEFAULT.TEST_KYLIN_FACT");
+
+        createModel(model);
+
+        NDataModelResponse modelResponse = new NDataModelResponse(model);
+        modelResponse.setConfig(KylinConfig.getInstanceFromEnv());
+        modelResponse.setProject(PROJECT);
+        List<NDataModelResponse.SimplifiedNamedColumn> selectedColumns = modelResponse.getSelectedColumns();
+        Assertions.assertEquals(1, selectedColumns.size());
+        List<NDataModelResponse.SimplifiedNamedColumn> namedColumns = modelResponse.getNamedColumns();
+        Assertions.assertEquals(1, namedColumns.size());
     }
 
-    @After
-    public void cleanup() {
-        cleanupTestMetadata();
+    @Test
+    void testGetSelectedColumnAndSimplifiedDimensionsWithExcludedColumn() {
+        MetadataTestUtils.mockExcludedCols(PROJECT, "DEFAULT.TEST_MEASURE1", Sets.newHashSet("PRICE6"));
+
+        String modelId = "cb596712-3a09-46f8-aea1-988b43fe9b6c";
+        NDataModelManager modelManager = NDataModelManager.getInstance(getTestConfig(), PROJECT);
+
+        // prepare lookup dimension
+        EnhancedUnitOfWork.doInTransactionWithCheckAndRetry(() -> {
+            NDataModelManager modelMgr = NDataModelManager.getInstance(KylinConfig.getInstanceFromEnv(), PROJECT);
+            modelMgr.updateDataModel(modelId, copyForWrite -> copyForWrite.getAllNamedColumns().get(33)
+                    .setStatus(NDataModel.ColumnStatus.DIMENSION));
+            return null;
+        }, PROJECT);
+
+        NDataModel model = modelManager.getDataModelDesc(modelId);
+        NDataModel.NamedColumn namedColumn = model.getAllNamedColumns().get(33);
+        Assertions.assertTrue(namedColumn.isDimension());
+        model.init(getTestConfig(), PROJECT, Lists.newArrayList());
+
+        NDataModelResponse modelResponse = new NDataModelResponse(model);
+        modelResponse.setConfig(KylinConfig.getInstanceFromEnv());
+        modelResponse.setProject(PROJECT);
+        List<NDataModelResponse.SimplifiedNamedColumn> selectedColumns = modelResponse.getSelectedColumns();
+        Assertions.assertEquals(2, selectedColumns.size());
+        selectedColumns.forEach(simplifiedNamedColumn -> {
+            if (simplifiedNamedColumn.getName().equalsIgnoreCase("price6")) {
+                Assertions.assertTrue(simplifiedNamedColumn.isExcluded());
+            } else {
+                Assertions.assertFalse(simplifiedNamedColumn.isExcluded());
+            }
+        });
+        List<NDataModelResponse.SimplifiedNamedColumn> namedColumns = modelResponse.getNamedColumns();
+        Assertions.assertEquals(2, namedColumns.size());
+        for (NDataModelResponse.SimplifiedNamedColumn column : namedColumns) {
+            if (column.getAliasDotColumn().equals("TEST_MEASURE1.PRICE6")) {
+                Assertions.assertTrue(column.isExcluded());
+            } else {
+                Assertions.assertFalse(column.isExcluded());
+            }
+        }
     }
 
+    //transColumnToDim
     @Test
-    public void testGetSelectedColumnsAndSimplifiedDimensionsNormal() throws Exception {
+    void testGetSelectedColumnsWithExcluded() {
+        MetadataTestUtils.mockExcludedTable(PROJECT, "DEFAULT.TEST_KYLIN_FACT");
         List<NDataModel.NamedColumn> allNamedColumns = Lists.newArrayList();
         NDataModel.NamedColumn namedColumn = new NDataModel.NamedColumn();
         namedColumn.setName("PRICE1");
@@ -72,13 +137,15 @@ public class NDataModelResponseTest extends NLocalFileMetadataTestCase {
         modelResponse.setConfig(KylinConfig.getInstanceFromEnv());
         modelResponse.setProject(PROJECT);
         List<NDataModelResponse.SimplifiedNamedColumn> selectedColumns = modelResponse.getSelectedColumns();
-        Assert.assertEquals(1, selectedColumns.size());
+        Assertions.assertEquals(1, selectedColumns.size());
         List<NDataModelResponse.SimplifiedNamedColumn> namedColumns = modelResponse.getNamedColumns();
-        Assert.assertEquals(1, namedColumns.size());
+        Assertions.assertEquals(1, namedColumns.size());
+        Assertions.assertFalse(selectedColumns.get(0).isExcluded());
+        Assertions.assertFalse(namedColumns.get(0).isExcluded());
     }
 
     @Test
-    public void testGetSelectedColumnsAndSimplifiedDimensionsWhenModelBroken() throws Exception {
+    void testGetSelectedColumnsAndSimplifiedDimensionsWhenModelBroken() {
         List<NDataModel.NamedColumn> allNamedColumns = Lists.newArrayList();
         NDataModel.NamedColumn namedColumn = new NDataModel.NamedColumn();
         namedColumn.setName("PRICE1");
@@ -98,9 +165,9 @@ public class NDataModelResponseTest extends NLocalFileMetadataTestCase {
         NDataModelResponse modelResponse = new NDataModelResponse(model);
         modelResponse.setBroken(true);
         List<NDataModelResponse.SimplifiedNamedColumn> selectedColumns = modelResponse.getSelectedColumns();
-        Assert.assertEquals(1, selectedColumns.size());
+        Assertions.assertEquals(1, selectedColumns.size());
         List<NDataModelResponse.SimplifiedNamedColumn> namedColumns = modelResponse.getNamedColumns();
-        Assert.assertEquals(1, namedColumns.size());
+        Assertions.assertEquals(1, namedColumns.size());
     }
 
     private NDataModel.Measure createMeasure() {
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 26a2cf1eaf..eb859e0ed9 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
@@ -124,8 +124,6 @@ import org.apache.kylin.metadata.cube.model.PartitionStatusEnum;
 import org.apache.kylin.metadata.cube.model.PartitionStatusEnumToDisplay;
 import org.apache.kylin.metadata.cube.model.RuleBasedIndex;
 import org.apache.kylin.metadata.cube.optimization.FrequencyMap;
-import org.apache.kylin.metadata.favorite.FavoriteRule;
-import org.apache.kylin.metadata.favorite.FavoriteRuleManager;
 import org.apache.kylin.metadata.model.AutoMergeTimeEnum;
 import org.apache.kylin.metadata.model.BadModelException;
 import org.apache.kylin.metadata.model.BadModelException.CauseType;
@@ -196,6 +194,7 @@ import org.apache.kylin.rest.util.SCD2SimplificationConvertUtil;
 import org.apache.kylin.streaming.jobs.StreamingJobListener;
 import org.apache.kylin.streaming.manager.StreamingJobManager;
 import org.apache.kylin.util.BrokenEntityProxy;
+import org.apache.kylin.util.MetadataTestUtils;
 import org.apache.kylin.util.PasswordEncodeFactory;
 import org.hamcrest.BaseMatcher;
 import org.hamcrest.Description;
@@ -271,8 +270,6 @@ public class ModelServiceTest extends SourceTestCase {
 
     private final StreamingJobListener eventListener = new StreamingJobListener();
 
-    private FavoriteRuleManager favoriteRuleManager;
-
     protected String getProject() {
         return "default";
     }
@@ -305,7 +302,6 @@ public class ModelServiceTest extends SourceTestCase {
         val result1 = new QueryTimesResponse();
         result1.setModel("89af4ee2-2cdb-4b07-b39e-4c29856309aa");
         result1.setQueryTimes(10);
-        favoriteRuleManager = FavoriteRuleManager.getInstance(getTestConfig(), getProject());
 
         try {
             new JdbcRawRecStore(getTestConfig());
@@ -3591,8 +3587,8 @@ public class ModelServiceTest extends SourceTestCase {
     @Test
     public void testMassageModelFilterConditionWithExcludedTable() {
         overwriteSystemProp("kylin.engine.build-excluded-table", "true");
-        mockExcludeTableRule("DEFAULT.TEST_ORDER");
         String project = "default";
+        MetadataTestUtils.mockExcludedTable(project, "DEFAULT.TEST_ORDER");
         NDataModelManager modelManager = NDataModelManager.getInstance(KylinConfig.getInstanceFromEnv(), project);
         NDataModel model = modelManager
                 .copyForWrite(modelManager.getDataModelDesc("89af4ee2-2cdb-4b07-b39e-4c29856309aa"));
@@ -3607,8 +3603,8 @@ public class ModelServiceTest extends SourceTestCase {
 
     @Test
     public void testMassageModelFilterConditionWithExcludedTableException() {
-        mockExcludeTableRule("DEFAULT.TEST_ORDER");
         String project = "default";
+        MetadataTestUtils.mockExcludedTable(project, "DEFAULT.TEST_ORDER");
         NDataModelManager modelManager = NDataModelManager.getInstance(KylinConfig.getInstanceFromEnv(), project);
         NDataModel model = modelManager
                 .copyForWrite(modelManager.getDataModelDesc("89af4ee2-2cdb-4b07-b39e-4c29856309aa"));
@@ -3623,15 +3619,6 @@ public class ModelServiceTest extends SourceTestCase {
         }
     }
 
-    private void mockExcludeTableRule(String excludedTables) {
-        List<FavoriteRule.AbstractCondition> conditions = com.clearspring.analytics.util.Lists.newArrayList();
-        FavoriteRule.Condition condition = new FavoriteRule.Condition();
-        condition.setLeftThreshold(null);
-        condition.setRightThreshold(excludedTables);
-        conditions.add(condition);
-        favoriteRuleManager.updateRule(conditions, true, FavoriteRule.EXCLUDED_TABLES_RULE);
-    }
-
     @Test
     public void testAddTableNameIfNotExist() {
         String project = "default";
diff --git a/src/modeling-service/src/test/java/org/apache/kylin/rest/service/ProjectServiceTest.java b/src/modeling-service/src/test/java/org/apache/kylin/rest/service/ProjectServiceTest.java
index 6018a7465d..62ded88b63 100644
--- a/src/modeling-service/src/test/java/org/apache/kylin/rest/service/ProjectServiceTest.java
+++ b/src/modeling-service/src/test/java/org/apache/kylin/rest/service/ProjectServiceTest.java
@@ -25,7 +25,6 @@ import static org.apache.kylin.metadata.model.MaintainModelType.MANUAL_MAINTAIN;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -65,11 +64,13 @@ import org.apache.kylin.rest.request.JdbcSourceInfoRequest;
 import org.apache.kylin.rest.request.JobNotificationConfigRequest;
 import org.apache.kylin.rest.request.MultiPartitionConfigRequest;
 import org.apache.kylin.rest.request.OwnerChangeRequest;
+import org.apache.kylin.rest.request.ProjectExclusionRequest;
 import org.apache.kylin.rest.request.ProjectGeneralInfoRequest;
 import org.apache.kylin.rest.request.PushDownConfigRequest;
 import org.apache.kylin.rest.request.PushDownProjectConfigRequest;
 import org.apache.kylin.rest.request.SegmentConfigRequest;
 import org.apache.kylin.rest.request.ShardNumConfigRequest;
+import org.apache.kylin.rest.response.ProjectConfigResponse;
 import org.apache.kylin.rest.response.StorageVolumeInfoResponse;
 import org.apache.kylin.rest.response.UserProjectPermissionResponse;
 import org.apache.kylin.rest.security.AclPermissionEnum;
@@ -106,7 +107,6 @@ import lombok.extern.slf4j.Slf4j;
 public class ProjectServiceTest extends NLocalFileMetadataTestCase {
     private static final String PROJECT = "default";
     private static final String PROJECT_JDBC = "jdbc";
-    private static final String PROJECT_ID = "a8f4da94-a8a4-464b-ab6f-b3012aba04d5";
     private static final String MODEL_ID = "89af4ee2-2cdb-4b07-b39e-4c29856309aa";
 
     @InjectMocks
@@ -359,6 +359,7 @@ public class ProjectServiceTest extends NLocalFileMetadataTestCase {
         Assert.assertFalse(response.isAutoMergeEnabled());
         Assert.assertFalse(response.getRetentionRange().isRetentionRangeEnabled());
         Assert.assertFalse(response.isExposeComputedColumn());
+        Assert.assertFalse(response.isTableExclusionEnabled());
     }
 
     @Test
@@ -367,8 +368,7 @@ public class ProjectServiceTest extends NLocalFileMetadataTestCase {
         var response = projectService.getProjectConfig(project);
         val jobNotificationConfigRequest = new JobNotificationConfigRequest();
         jobNotificationConfigRequest.setDataLoadEmptyNotificationEnabled(false);
-        jobNotificationConfigRequest.setJobNotificationStates(
-                Lists.newArrayList("Succeed", "Error", "Discard"));
+        jobNotificationConfigRequest.setJobNotificationStates(Lists.newArrayList("Succeed", "Error", "Discard"));
         jobNotificationConfigRequest.setJobNotificationEmails(
                 Lists.newArrayList("user1@kyligence.io", "user2@kyligence.io", "user2@kyligence.io"));
         jobNotificationConfigRequest.setMetadataPersistNotificationEnabled(false);
@@ -725,8 +725,9 @@ public class ProjectServiceTest extends NLocalFileMetadataTestCase {
 
     @Test
     public void testClearManagerCache() throws Exception {
-        val config = getTestConfig();
-        val modelManager = NDataModelManager.getInstance(config, "default");
+        // this invoke is used for add project manager cache
+        NDataModelManager.getInstance(getTestConfig(), "default");
+
         ConcurrentHashMap<Class, Object> managersCache = getInstances();
         ConcurrentHashMap<Class, ConcurrentHashMap<String, Object>> managersByPrjCache = getInstanceByProject();
 
@@ -739,9 +740,8 @@ public class ProjectServiceTest extends NLocalFileMetadataTestCase {
         managersCache = getInstances();
         managersByPrjCache = getInstanceByProject();
         //cleared
-        Assert.assertTrue(!managersCache.containsKey(NProjectManager.class));
-        Assert.assertTrue(!managersByPrjCache.get(NDataModelManager.class).containsKey("default"));
-
+        Assert.assertFalse(managersCache.containsKey(NProjectManager.class));
+        Assert.assertFalse(managersByPrjCache.get(NDataModelManager.class).containsKey("default"));
     }
 
     @Test
@@ -767,13 +767,12 @@ public class ProjectServiceTest extends NLocalFileMetadataTestCase {
     private void updateProject() {
         val segmentConfigRequest = new SegmentConfigRequest();
         segmentConfigRequest.setAutoMergeEnabled(false);
-        segmentConfigRequest.setAutoMergeTimeRanges(Arrays.asList(AutoMergeTimeEnum.YEAR));
+        segmentConfigRequest.setAutoMergeTimeRanges(Collections.singletonList(AutoMergeTimeEnum.YEAR));
         projectService.updateSegmentConfig(PROJECT, segmentConfigRequest);
 
         val jobNotificationConfigRequest = new JobNotificationConfigRequest();
         jobNotificationConfigRequest.setDataLoadEmptyNotificationEnabled(true);
-        jobNotificationConfigRequest.setJobNotificationStates(
-                Lists.newArrayList("Succeed", "Error", "Discard"));
+        jobNotificationConfigRequest.setJobNotificationStates(Lists.newArrayList("Succeed", "Error", "Discard"));
         jobNotificationConfigRequest.setJobNotificationEmails(
                 Lists.newArrayList("user1@kyligence.io", "user2@kyligence.io", "user2@kyligence.io"));
         jobNotificationConfigRequest.setMetadataPersistNotificationEnabled(false);
@@ -822,10 +821,13 @@ public class ProjectServiceTest extends NLocalFileMetadataTestCase {
 
         response = projectService.resetProjectConfig(PROJECT, "storage_quota_config");
         Assert.assertEquals(10995116277760L, response.getStorageQuotaSize());
+
+        response = projectService.resetProjectConfig(PROJECT, "table_exclusion_config");
+        Assert.assertFalse(response.isTableExclusionEnabled());
     }
 
     @Test
-    public void testUpdateYarnQueue() throws Exception {
+    public void testUpdateYarnQueue() {
         final String updateTo = "q.queue";
         Assert.assertEquals("default", projectService.getProjectConfig(PROJECT).getYarnQueue());
         projectService.updateYarnQueue(PROJECT, updateTo);
@@ -835,7 +837,7 @@ public class ProjectServiceTest extends NLocalFileMetadataTestCase {
     }
 
     @Test
-    public void testCreateProjectComputedColumnConfig() throws Exception {
+    public void testCreateProjectComputedColumnConfig() {
         // auto
         {
             getTestConfig().setProperty("kylin.metadata.semi-automatic-mode", "true");
@@ -1009,4 +1011,17 @@ public class ProjectServiceTest extends NLocalFileMetadataTestCase {
         Assert.assertTrue(aclEvaluate.hasProjectAdminPermission(PROJECT));
         aclEvaluate.checkProjectQueryPermission(PROJECT);
     }
+
+    @Test
+    public void testUpdateTableExclusionRule() {
+        Assert.assertFalse(NProjectManager.getProjectConfig(PROJECT).isTableExclusionEnabled());
+
+        // update & validate
+        ProjectExclusionRequest request = new ProjectExclusionRequest();
+        request.setTableExclusionEnabled(true);
+        projectService.updateTableExclusionRule(PROJECT, request);
+        Assert.assertTrue(NProjectManager.getProjectConfig(PROJECT).isTableExclusionEnabled());
+        ProjectConfigResponse projectConfig = projectService.getProjectConfig(PROJECT);
+        Assert.assertTrue(projectConfig.isTableExclusionEnabled());
+    }
 }
diff --git a/src/modeling-service/src/test/java/org/apache/kylin/rest/service/TableReloadServiceTest.java b/src/modeling-service/src/test/java/org/apache/kylin/rest/service/TableReloadServiceTest.java
index b252929a93..e2e17efe0d 100644
--- a/src/modeling-service/src/test/java/org/apache/kylin/rest/service/TableReloadServiceTest.java
+++ b/src/modeling-service/src/test/java/org/apache/kylin/rest/service/TableReloadServiceTest.java
@@ -19,7 +19,6 @@ package org.apache.kylin.rest.service;
 
 import static org.apache.kylin.common.exception.code.ErrorCodeServer.TABLE_RELOAD_HAVING_NOT_FINAL_JOB;
 import static org.awaitility.Awaitility.await;
-import static org.hamcrest.CoreMatchers.is;
 
 import java.io.File;
 import java.io.FileOutputStream;
@@ -31,6 +30,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
+import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.function.Predicate;
@@ -81,6 +81,7 @@ import org.apache.kylin.rest.request.ModelRequest;
 import org.apache.kylin.rest.request.S3TableExtInfo;
 import org.apache.kylin.rest.response.OpenPreReloadTableResponse;
 import org.apache.kylin.rest.response.SimplifiedMeasure;
+import org.apache.kylin.util.MetadataTestUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -160,7 +161,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testPreProcess_AffectTwoTables() throws Exception {
+    public void testPreProcessAffectTwoTables() throws Exception {
         removeColumn("DEFAULT.TEST_COUNTRY", "NAME");
 
         val response = tableService.preProcessBeforeReloadWithFailFast(PROJECT, "DEFAULT.TEST_COUNTRY");
@@ -175,7 +176,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testPreProcess_AffectByCC() throws Exception {
+    public void testPreProcessAffectByCC() throws Exception {
         removeColumn("DEFAULT.TEST_KYLIN_FACT", "PRICE");
 
         val response = tableService.preProcessBeforeReloadWithFailFast(PROJECT, "DEFAULT.TEST_KYLIN_FACT");
@@ -200,7 +201,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testPreProcess_RefreshCount() throws Exception {
+    public void testPreProcessRefreshCount() throws Exception {
         changeTypeColumn("DEFAULT.TEST_KYLIN_FACT", new HashMap<String, String>() {
             {
                 put("PRICE", "bigint");
@@ -212,7 +213,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testPreProcess_ChangeCCType() throws Exception {
+    public void testPreProcessChangeCCType() throws Exception {
         val modelManager = NDataModelManager.getInstance(getTestConfig(), PROJECT);
         var model = modelManager.getDataModelDescByAlias("nmodel_basic");
         Assert.assertEquals("decimal(30,4)", model.getComputedColumnDescs().get(0).getDatatype());
@@ -227,7 +228,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testPreProcess_UseCaseSensitiveTableIdentity() throws Exception {
+    public void testPreProcessUseCaseSensitiveTableIdentity() throws Exception {
         NTableMetadataManager manager = NTableMetadataManager.getInstance(getTestConfig(), PROJECT);
         TableDesc tableDesc = manager.getTableDesc("DEFAULT.TEST_KYLIN_FACT");
         Assert.assertNotNull(tableDesc);
@@ -246,7 +247,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testReload_RemoveCC() throws Exception {
+    public void testReloadRemoveCC() throws Exception {
         val modelManager = NDataModelManager.getInstance(getTestConfig(), PROJECT);
         var originModel = modelManager.getDataModelDescByAlias("nmodel_basic");
         val originSize = originModel.getComputedColumnDescs().size();
@@ -279,7 +280,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testReload_RemoveMeasureAffectedAggGroup() throws Exception {
+    public void testReloadRemoveMeasureAffectedAggGroup() throws Exception {
         val MODEL_ID = "741ca86a-1f13-46da-a59f-95fb68615e3a";
         val dfManager = NDataflowManager.getInstance(getTestConfig(), PROJECT);
         val modelManager = NDataModelManager.getInstance(getTestConfig(), PROJECT);
@@ -339,7 +340,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testCleanupToBeDeleted_afterChangeType() throws Exception {
+    public void testCleanupToBeDeletedAfterChangeType() throws Exception {
         val modelId = "741ca86a-1f13-46da-a59f-95fb68615e3a";
         val project = "default";
 
@@ -399,7 +400,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testReload_AddIndexCount() throws Exception {
+    public void testReloadAddIndexCount() throws Exception {
 
         val newRule = new RuleBasedIndex();
         newRule.setDimensions(Arrays.asList(14, 15, 16));
@@ -427,7 +428,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testReload_AddIndexCountHierarchy() throws Exception {
+    public void testReloadAddIndexCountHierarchy() throws Exception {
 
         val newRule = new RuleBasedIndex();
         newRule.setDimensions(Arrays.asList(14, 15, 16));
@@ -456,7 +457,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testReload_AddIndexCountMandatory() throws Exception {
+    public void testReloadAddIndexCountMandatory() throws Exception {
 
         val newRule = new RuleBasedIndex();
         newRule.setDimensions(Arrays.asList(14, 15, 16, 17, 18, 19));
@@ -485,7 +486,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testReload_AddIndexCountJoint() throws Exception {
+    public void testReloadAddIndexCountJoint() throws Exception {
 
         val newRule = new RuleBasedIndex();
         newRule.setDimensions(Arrays.asList(14, 15, 16, 17, 18, 19));
@@ -518,7 +519,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testReload_BrokenModelInAutoProject() throws Exception {
+    public void testReloadBrokenModelInAutoProject() throws Exception {
         removeColumn("DEFAULT.TEST_KYLIN_FACT", "ORDER_ID");
         overwriteSystemProp("kylin.metadata.broken-model-deleted-on-smart-mode", "true");
         await().atMost(10000, TimeUnit.MILLISECONDS).untilAsserted(() -> {
@@ -535,7 +536,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testReload_BrokenModelInManualProject() throws Exception {
+    public void testReloadBrokenModelInManualProject() throws Exception {
         removeColumn("DEFAULT.TEST_KYLIN_FACT", "ORDER_ID");
         tableService.innerReloadTable(PROJECT, "DEFAULT.TEST_KYLIN_FACT", true, null);
         val modelManager = NDataModelManager.getInstance(getTestConfig(), PROJECT);
@@ -549,7 +550,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testReloadLookup_RemoveFact() throws Exception {
+    public void testReloadLookupRemoveFact() throws Exception {
         modelService.listAllModelIdsInProject(PROJECT).forEach(id -> {
             if (!id.equals("89af4ee2-2cdb-4b07-b39e-4c29856309aa")) {
                 modelService.dropModel(id, PROJECT);
@@ -607,7 +608,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testReload_GetAndEditJoinBrokenModelInManualProject() throws Exception {
+    public void testReloadGetAndEditJoinBrokenModelInManualProject() throws Exception {
         prepareReload();
 
         changeColumnName("DEFAULT.TEST_KYLIN_FACT", "ORDER_ID", "ORDER_ID2");
@@ -666,7 +667,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testReload_GetAndEditPartitionBrokenModelInManualProject() throws Exception {
+    public void testReloadGetAndEditPartitionBrokenModelInManualProject() throws Exception {
         prepareReload();
 
         changeColumnName("DEFAULT.TEST_KYLIN_FACT", "CAL_DT", "CAL_DT2");
@@ -819,17 +820,97 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testReload_WhenProjectHasBrokenModel() throws Exception {
+    public void testReloadWhenProjectHasBrokenModel() throws Exception {
         val tableManager = NTableMetadataManager.getInstance(getTestConfig(), PROJECT);
         tableManager.removeSourceTable("DEFAULT.TEST_MEASURE");
         val dfManager = NDataflowManager.getInstance(getTestConfig(), PROJECT);
         Assert.assertEquals(7, dfManager.listUnderliningDataModels().size());
+        testPreProcessAffectTwoTables();
+    }
+
+    @Test
+    public void testTableReloadWithExcludedColumns() throws Exception {
+        NDataModelManager modelManager = NDataModelManager.getInstance(getTestConfig(), PROJECT);
+        modelManager.listAllModelIds().forEach(modelManager::dropModel);
+
+        String table = "DEFAULT.TEST_ORDER";
+        prepareTableExt(table);
+        MetadataTestUtils.mockExcludedTable(PROJECT, table);
+        Assert.assertEquals(5, MetadataTestUtils.getExcludedColumns(PROJECT, table).size());
+
+        // keep other excluded columns
+        removeColumn(table, "TEST_TIME_ENC");
+        EnhancedUnitOfWork.doInTransactionWithCheckAndRetry(() -> {
+            tableService.innerReloadTable(PROJECT, table, false, null);
+            return null;
+        }, PROJECT);
+        Assert.assertEquals(4, MetadataTestUtils.getExcludedColumns(PROJECT, table).size());
+
+        // if the table is excluded, then new added column is excluded
+        addColumn(table, true, new ColumnDesc("", "DEAL_YEAR", "int", "", "", "", null));
+        EnhancedUnitOfWork.doInTransactionWithCheckAndRetry(() -> {
+            tableService.innerReloadTable(PROJECT, table, false, null);
+            return null;
+        }, PROJECT);
+        Assert.assertEquals(5, MetadataTestUtils.getExcludedColumns(PROJECT, table).size());
+        Set<String> tables = MetadataTestUtils.getExcludedTables(PROJECT);
+        Assert.assertEquals(1, tables.size());
+        Assert.assertEquals(table, tables.iterator().next());
+    }
+
+    @Test
+    public void testTableReloadWithDropExcludedColumn() throws Exception {
+        NDataModelManager modelManager = NDataModelManager.getInstance(getTestConfig(), PROJECT);
+        modelManager.listAllModelIds().forEach(modelManager::dropModel);
+
+        String table = "DEFAULT.TEST_ORDER";
+        prepareTableExt(table);
+
+        MetadataTestUtils.mockExcludedCols(PROJECT, table, Sets.newHashSet("TEST_TIME_ENC"));
+        Set<String> excludedColumns = MetadataTestUtils.getExcludedColumns(PROJECT, table);
+        Assert.assertEquals(1, excludedColumns.size());
+        Assert.assertTrue(excludedColumns.contains("TEST_TIME_ENC"));
+
+        removeColumn(table, "TEST_TIME_ENC");
+        EnhancedUnitOfWork.doInTransactionWithCheckAndRetry(() -> {
+            tableService.innerReloadTable(PROJECT, table, false, null);
+            return null;
+        }, PROJECT);
+
+        Assert.assertTrue(MetadataTestUtils.getExcludedColumns(PROJECT, table).isEmpty());
+        Assert.assertTrue(MetadataTestUtils.getExcludedTables(PROJECT).isEmpty());
+    }
+
+    @Test
+    public void testTableReloadWithDropTheLastUnExcludedColumn() throws IOException {
+        NDataModelManager modelManager = NDataModelManager.getInstance(getTestConfig(), PROJECT);
+        modelManager.listAllModelIds().forEach(modelManager::dropModel);
+
+        String table = "DEFAULT.TEST_ORDER";
+        prepareTableExt(table);
 
-        testPreProcess_AffectTwoTables();
+        MetadataTestUtils.mockExcludedCols(PROJECT, table,
+                Sets.newHashSet("ORDER_ID", "BUYER_ID", "TEST_EXTENDED_COLUMN", "TEST_DATE_ENC"));
+        Set<String> excludedColumns = MetadataTestUtils.getExcludedColumns(PROJECT, table);
+        Assert.assertEquals(4, excludedColumns.size());
+        Assert.assertFalse(excludedColumns.contains("TEST_TIME_ENC"));
+
+        removeColumn(table, "TEST_TIME_ENC");
+        EnhancedUnitOfWork.doInTransactionWithCheckAndRetry(() -> {
+            tableService.innerReloadTable(PROJECT, table, false, null);
+            return null;
+        }, PROJECT);
+
+        Set<String> excludedColumnsAfterTableReload = MetadataTestUtils.getExcludedColumns(PROJECT, table);
+        Assert.assertEquals(4, excludedColumnsAfterTableReload.size());
+        Assert.assertEquals(excludedColumns, excludedColumnsAfterTableReload);
+        Set<String> tables = MetadataTestUtils.getExcludedTables(PROJECT);
+        Assert.assertEquals(1, tables.size());
+        Assert.assertEquals(table, tables.iterator().next());
     }
 
     @Test
-    public void testReload_RemoveDimensionsAndIndexes() throws Exception {
+    public void testReloadRemoveDimensionsAndIndexes() throws Exception {
         val indexManager = NIndexPlanManager.getInstance(getTestConfig(), PROJECT);
         val originIndexPlan = indexManager.getIndexPlanByModelAlias("nmodel_basic");
         val originTable = NTableMetadataManager.getInstance(getTestConfig(), PROJECT)
@@ -860,8 +941,10 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
 
         // index_plan without rule
         val model2 = modelManager.getDataModelDescByAlias("nmodel_basic");
-        Assert.assertEquals(NDataModel.ColumnStatus.TOMB,
-                model2.getAllNamedColumns().stream().filter(n -> n.getId() == 15).findAny().get().getStatus());
+        Optional<NDataModel.NamedColumn> optionalNamedColumn = model2.getAllNamedColumns().stream()
+                .filter(n -> n.getId() == 15).findAny();
+        Assert.assertTrue(optionalNamedColumn.isPresent());
+        Assert.assertEquals(NDataModel.ColumnStatus.TOMB, optionalNamedColumn.get().getStatus());
         val indexPlan2 = indexManager.getIndexPlan(model2.getId());
         Assert.assertEquals(
                 originIndexPlan.getAllIndexes().stream().filter(index -> !index.getDimensions().contains(15)).count(),
@@ -898,30 +981,38 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testReload_RemoveAggShardByColumns() throws Exception {
+    public void testReloadRemoveAggShardByColumns() throws Exception {
         val newRule = new RuleBasedIndex();
         newRule.setDimensions(Arrays.asList(14, 15, 16));
-        val group1 = JsonUtil.readValue("{\n" + "        \"includes\": [14,15,16],\n" + "        \"select_rule\": {\n"
-                + "          \"hierarchy_dims\": [],\n" + "          \"mandatory_dims\": [],\n"
-                + "          \"joint_dims\": []\n" + "        }\n" + "}", NAggregationGroup.class);
+        val group1 = JsonUtil.readValue("{\n" //
+                + "        \"includes\": [14,15,16],\n" //
+                + "        \"select_rule\": {\n" //
+                + "          \"hierarchy_dims\": [],\n" //
+                + "          \"mandatory_dims\": [],\n" //
+                + "          \"joint_dims\": []\n" //
+                + "        }\n" //
+                + "}", NAggregationGroup.class);
         newRule.setAggregationGroups(Lists.newArrayList(group1));
-        testReload_AggShardByColumns(newRule, Lists.newArrayList(14, 15), Lists.newArrayList());
-
+        testReloadAggShardByColumns(newRule, Lists.newArrayList(14, 15), Lists.newArrayList());
     }
 
     @Test
-    public void testReload_KeepAggShardByColumns() throws Exception {
+    public void testReloadKeepAggShardByColumns() throws Exception {
         val newRule = new RuleBasedIndex();
         newRule.setDimensions(Arrays.asList(13, 14, 15));
-        val group1 = JsonUtil.readValue("{\n" + "        \"includes\": [13,14,15],\n" + "        \"select_rule\": {\n"
-                + "          \"hierarchy_dims\": [],\n" + "          \"mandatory_dims\": [],\n"
-                + "          \"joint_dims\": []\n" + "        }\n" + "}", NAggregationGroup.class);
+        val group1 = JsonUtil.readValue("{\n" //
+                + "        \"includes\": [13,14,15],\n" //
+                + "        \"select_rule\": {\n" //
+                + "          \"hierarchy_dims\": [],\n" //
+                + "          \"mandatory_dims\": [],\n" //
+                + "          \"joint_dims\": []\n" //
+                + "        }\n" //
+                + "}", NAggregationGroup.class);
         newRule.setAggregationGroups(Lists.newArrayList(group1));
-        testReload_AggShardByColumns(newRule, Lists.newArrayList(13, 14), Lists.newArrayList(13, 14));
-
+        testReloadAggShardByColumns(newRule, Lists.newArrayList(13, 14), Lists.newArrayList(13, 14));
     }
 
-    private void testReload_AggShardByColumns(RuleBasedIndex ruleBasedIndex, List<Integer> beforeAggShardBy,
+    private void testReloadAggShardByColumns(RuleBasedIndex ruleBasedIndex, List<Integer> beforeAggShardBy,
             List<Integer> endAggShardBy) throws Exception {
         val indexManager = NIndexPlanManager.getInstance(getTestConfig(), PROJECT);
         var originIndexPlan = indexManager.getIndexPlanByModelAlias("nmodel_basic");
@@ -953,7 +1044,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testReload_addColumn_blacklistNotEmpty() throws Exception {
+    public void testReloadAddColumnBlacklistNotEmpty() throws Exception {
         val dataflowManager = NDataflowManager.getInstance(getTestConfig(), PROJECT);
         val dataflow1 = dataflowManager.getDataflowByModelAlias("nmodel_basic_inner");
         int layoutSize = dataflow1.getIndexPlan().getRuleBaseLayouts().size();
@@ -971,7 +1062,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testReload_AddColumn() throws Exception {
+    public void testReloadAddColumn() throws Exception {
         String mockPath = "default/table_snapshot/mock";
         NTableMetadataManager tableMetadataManager = NTableMetadataManager.getInstance(getTestConfig(), PROJECT);
         TableDesc tableDesc = tableMetadataManager.getTableDesc("DEFAULT.TEST_COUNTRY");
@@ -1034,7 +1125,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testReload_AddLookupColumn() throws Exception {
+    public void testReloadAddLookupColumn() throws Exception {
         addColumn("EDW.TEST_CAL_DT", true, new ColumnDesc("", "DEAL_YEAR", "int", "", "", "", null));
 
         OpenPreReloadTableResponse response = tableService.preProcessBeforeReloadWithoutFailFast(PROJECT,
@@ -1116,7 +1207,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testReload_ChangeColumn() throws Exception {
+    public void testReloadChangeColumn() throws Exception {
         removeColumn("EDW.TEST_CAL_DT", "CAL_DT_UPD_USER");
         tableService.innerReloadTable(PROJECT, "EDW.TEST_CAL_DT", true, null);
 
@@ -1179,7 +1270,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testReload_ChangeTypeAndRemoveDimension() throws Exception {
+    public void testReloadChangeTypeAndRemoveDimension() throws Exception {
         removeColumn("EDW.TEST_CAL_DT", "CAL_DT_UPD_USER");
         tableService.innerReloadTable(PROJECT, "EDW.TEST_CAL_DT", true, null);
 
@@ -1214,7 +1305,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testReload_ChangeColumnInAggManual() throws Exception {
+    public void testReloadChangeColumnInAggManual() throws Exception {
         val newRule = new RuleBasedIndex();
         newRule.setDimensions(Arrays.asList(14, 15, 16));
         val group1 = JsonUtil.readValue("{\n" //
@@ -1255,7 +1346,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testReload_WithNoBlacklistLayoutRestore() throws Exception {
+    public void testReloadWithNoBlacklistLayoutRestore() throws Exception {
         val newRule = new RuleBasedIndex();
         newRule.setDimensions(Arrays.asList(14, 15, 16));
         val group1 = JsonUtil.readValue("{\n" //
@@ -1297,11 +1388,11 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
         val layouts2 = originIndexPlan.getAllLayouts().stream().filter(LayoutEntity::isManual)
                 .filter(l -> l.getId() < IndexEntity.TABLE_INDEX_START_ID).filter(l -> l.getColOrder().contains(16))
                 .collect(Collectors.toList());
-        Assert.assertThat(layouts2.size(), is(0));
+        Assert.assertTrue(layouts2.isEmpty());
     }
 
     @Test
-    public void testReload_ChangeColumnInAggManualUnsuitable() throws Exception {
+    public void testReloadChangeColumnInAggManualUnsuitable() throws Exception {
         // add TEST_KYLIN_FACT.ITEM_COUNT as dimension
 
         var indexManager = NIndexPlanManager.getInstance(getTestConfig(), PROJECT);
@@ -1330,9 +1421,14 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
         val newRule = new RuleBasedIndex();
         // TEST_KYLIN_FACT.ITEM_COUNT, TEST_ORDER.TEST_TIME_ENC, TEST_KYLIN_FACT.SLR_SEGMENT_CD
         newRule.setDimensions(Arrays.asList(12, 15, 16));
-        val group1 = JsonUtil.readValue("{\n" + "        \"includes\": [12,15,16],\n" + "        \"select_rule\": {\n"
-                + "          \"hierarchy_dims\": [],\n" + "          \"mandatory_dims\": [],\n"
-                + "          \"joint_dims\": []\n" + "        }\n" + "}", NAggregationGroup.class);
+        val group1 = JsonUtil.readValue("{\n" //
+                + "        \"includes\": [12,15,16],\n" //
+                + "        \"select_rule\": {\n" //
+                + "          \"hierarchy_dims\": [],\n" //
+                + "          \"mandatory_dims\": [],\n" //
+                + "          \"joint_dims\": []\n" //
+                + "        }\n" //
+                + "}", NAggregationGroup.class);
         newRule.setAggregationGroups(Lists.newArrayList(group1));
         // 100000 count(1), 100004 sum(TEST_KYLIN_FACT.ITEM_COUNT)
         group1.setMeasures(new Integer[] { 100000, 100004 });
@@ -1366,7 +1462,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testReload_ChangeColumnInAggManualSuitable() throws Exception {
+    public void testReloadChangeColumnInAggManualSuitable() throws Exception {
         // add TEST_KYLIN_FACT.ITEM_COUNT as dimension
 
         var indexManager = NIndexPlanManager.getInstance(getTestConfig(), PROJECT);
@@ -1447,7 +1543,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testReload_ChangeColumnInAggManualAndAuto() throws Exception {
+    public void testReloadChangeColumnInAggManualAndAuto() throws Exception {
         String modelId = "89af4ee2-2cdb-4b07-b39e-4c29856309aa";
         val indexManager = NIndexPlanManager.getInstance(getTestConfig(), PROJECT);
         val index = new IndexEntity();
@@ -1467,13 +1563,13 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
             indexes.add(indexCopy);
             copyForWrite.setIndexes(indexes);
         });
-        testReload_ChangeColumnInAggManual();
+        testReloadChangeColumnInAggManual();
         Assert.assertTrue(indexManager.getIndexPlan(modelId).getAllLayouts().stream()
                 .anyMatch(l -> l.equals(layout) && l.isAuto() && l.isManual() && l.getId() > layout.getId()));
     }
 
     @Test
-    public void testReload_ChangeColumnInTableIndex() throws Exception {
+    public void testReloadChangeColumnInTableIndex() throws Exception {
         String modelId = "89af4ee2-2cdb-4b07-b39e-4c29856309aa";
         dropModelWhen(id -> !id.equals(modelId));
         val indexManager = NIndexPlanManager.getInstance(getTestConfig(), PROJECT);
@@ -1505,7 +1601,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testReload_ChangeColumnOrderAndDeleteColumn() throws Exception {
+    public void testReloadChangeColumnOrderAndDeleteColumn() throws Exception {
         val tableIdentity = "DEFAULT.TEST_COUNTRY";
         val originTable = NTableMetadataManager.getInstance(getTestConfig(), PROJECT).getTableDesc(tableIdentity);
         prepareTableExt(tableIdentity);
@@ -1559,7 +1655,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testReload_IndexPlanHasDictionary() throws Exception {
+    public void testReloadIndexPlanHasDictionary() throws Exception {
         val indexManager = NIndexPlanManager.getInstance(getTestConfig(), PROJECT);
         val indexPlan = indexManager.getIndexPlanByModelAlias("nmodel_basic_inner");
         indexManager.updateIndexPlan(indexPlan.getId(), copyForWrite -> {
@@ -1604,7 +1700,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testReload_NoChangeAndUpdateTableExtDesc() throws Exception {
+    public void testReloadNoChangeAndUpdateTableExtDesc() throws Exception {
         S3TableExtInfo tableExtInfo = prepareTableExtInfo("DEFAULT.TEST_ORDER", "endpoint", "role");
         prepareTableExt("DEFAULT.TEST_ORDER");
         KylinConfig.getInstanceFromEnv().setProperty("kylin.env.use-dynamic-S3-role-credential-in-table", "true");
@@ -1626,7 +1722,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
     }
 
     @Test
-    public void testReload_RemoveColumnAndUpdateTableExtDesc() throws Exception {
+    public void testReloadRemoveColumnAndUpdateTableExtDesc() throws Exception {
         removeColumn("DEFAULT.TEST_ORDER", "TEST_TIME_ENC");
         S3TableExtInfo tableExtInfo = prepareTableExtInfo("DEFAULT.TEST_ORDER", "endpoint", "role");
 
@@ -1819,7 +1915,7 @@ public class TableReloadServiceTest extends CSVSourceTestCase {
         val newColumns = Stream.of(factTable.getColumns()).filter(col -> !columns.contains(col.getName()))
                 .toArray(ColumnDesc[]::new);
         tableMeta.setColumns(newColumns);
-        JsonUtil.writeValueIndent(new FileOutputStream(new File(tablePath)), tableMeta);
+        JsonUtil.writeValueIndent(new FileOutputStream(tablePath), tableMeta);
     }
 
     private void changeColumnName(String tableIdentity, String oldName, String newName) throws IOException {
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 1747593dbb..4d1c15734c 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
@@ -1060,44 +1060,44 @@ public class TableServiceTest extends CSVSourceTestCase {
     @Test
     public void testGetProjectTables() throws Exception {
         NInitTablesResponse response;
-        response = tableService.getProjectTables("default", "SSB.SS", 0, 14, true,
+        response = tableService.getProjectTables("default", "SSB.SS", 0, 14, true, true,
                 (databaseName, tableName) -> tableService.getTableNameResponses("default", databaseName, tableName));
         Assert.assertEquals(0, response.getDatabases().size());
 
-        response = tableService.getProjectTables("default", "SSB.CU", 0, 14, true,
+        response = tableService.getProjectTables("default", "SSB.CU", 0, 14, true, true,
                 (databaseName, tableName) -> tableService.getTableNameResponses("default", databaseName, tableName));
         Assert.assertEquals(1, response.getDatabases().size());
         Assert.assertEquals(2, response.getDatabases().get(0).getTables().size());
 
-        response = tableService.getProjectTables("default", "", 0, 14, true,
+        response = tableService.getProjectTables("default", "", 0, 14, true, true,
                 (databaseName, tableName) -> tableService.getTableNameResponses("default", databaseName, tableName));
         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,
+        response = tableService.getProjectTables("default", "TEST", 0, 14, true, true,
                 (databaseName, tableName) -> tableService.getTableNameResponses("default", databaseName, tableName));
         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,
+        response = tableService.getProjectTables("default", "EDW.", 0, 14, true, true,
                 (databaseName, tableName) -> tableService.getTableNameResponses("default", databaseName, tableName));
         Assert.assertEquals(1, response.getDatabases().size());
         Assert.assertEquals(3, response.getDatabases().get(0).getTables().size());
 
-        response = tableService.getProjectTables("default", "EDW.", 0, 14, false,
+        response = tableService.getProjectTables("default", "EDW.", 0, 14, true, false,
                 (databaseName, tableName) -> tableService.getTableDesc("default", true, tableName, databaseName, true));
         Assert.assertEquals(1, response.getDatabases().size());
         Assert.assertEquals(3, response.getDatabases().get(0).getTables().size());
 
-        response = tableService.getProjectTables("default", "DEFAULT.TEST_ORDER", 0, 14, false,
+        response = tableService.getProjectTables("default", "DEFAULT.TEST_ORDER", 0, 14, true, false,
                 (databaseName, tableName) -> tableService.getTableDesc("default", true, tableName, databaseName, true));
         Assert.assertEquals(1, response.getDatabases().size());
         Assert.assertEquals(1, response.getDatabases().get(0).getTables().size());
 
-        response = tableService.getProjectTables("default", ".TEST_ORDER", 0, 14, false,
+        response = tableService.getProjectTables("default", ".TEST_ORDER", 0, 14, true, false,
                 (databaseName, tableName) -> tableService.getTableDesc("default", true, tableName, databaseName, true));
         Assert.assertEquals(0, response.getDatabases().size());
 
diff --git a/src/query-common/src/main/java/org/apache/kylin/query/relnode/OLAPContext.java b/src/query-common/src/main/java/org/apache/kylin/query/relnode/OLAPContext.java
index 609f937d57..9713500115 100644
--- a/src/query-common/src/main/java/org/apache/kylin/query/relnode/OLAPContext.java
+++ b/src/query-common/src/main/java/org/apache/kylin/query/relnode/OLAPContext.java
@@ -29,7 +29,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import io.kyligence.kap.secondstorage.SecondStorageUtil;
 import org.apache.calcite.rel.AbstractRelNode;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.TableScan;
@@ -40,31 +39,30 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.QueryContext;
+import org.apache.kylin.metadata.cube.cuboid.NLayoutCandidate;
+import org.apache.kylin.metadata.cube.realization.HybridRealization;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.JoinDesc;
 import org.apache.kylin.metadata.model.JoinsGraph;
 import org.apache.kylin.metadata.model.MeasureDesc;
+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.query.NativeQueryRealization;
+import org.apache.kylin.metadata.query.QueryMetrics;
 import org.apache.kylin.metadata.realization.IRealization;
 import org.apache.kylin.metadata.realization.SQLDigest;
 import org.apache.kylin.metadata.tuple.TupleInfo;
 import org.apache.kylin.query.routing.RealizationCheck;
 import org.apache.kylin.query.schema.OLAPSchema;
 import org.apache.kylin.storage.StorageContext;
-import org.apache.kylin.metadata.cube.cuboid.NLayoutCandidate;
-import org.apache.kylin.metadata.cube.realization.HybridRealization;
-import org.apache.kylin.metadata.model.ExcludedLookupChecker;
-import org.apache.kylin.metadata.model.NDataModel;
-import org.apache.kylin.metadata.query.NativeQueryRealization;
-import org.apache.kylin.metadata.query.QueryMetrics;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
+import io.kyligence.kap.secondstorage.SecondStorageUtil;
 import lombok.Getter;
 import lombok.Setter;
 import lombok.val;
@@ -544,24 +542,6 @@ public class OLAPContext {
         return createUniqueInputRefAmongTables(table, columnIdx, allTableScans);
     }
 
-    public Map<String, TblColRef> collectFKAsDimensionMap(ExcludedLookupChecker checker) {
-        Map<String, TblColRef> fKAsDimensionMap = Maps.newHashMap();
-        Set<String> usingExcludedLookupTables = checker.getUsedExcludedLookupTable(this.allColumns);
-        this.joins.forEach(join -> {
-            for (int i = 0; i < join.getForeignKeyColumns().length; i++) {
-                TblColRef foreignKeyColumn = join.getForeignKeyColumns()[i];
-                String derivedTable = join.getPrimaryKeyColumns()[i].getTableWithSchema();
-                if (usingExcludedLookupTables.contains(derivedTable)
-                        && !usingExcludedLookupTables.contains(foreignKeyColumn.getTableWithSchema())
-                        && !checker.getExcludedLookups().contains(foreignKeyColumn.getTableWithSchema())) {
-                    fKAsDimensionMap.putIfAbsent(foreignKeyColumn.getCanonicalName(), foreignKeyColumn);
-                }
-            }
-        });
-
-        return fKAsDimensionMap;
-    }
-
     public interface IAccessController {
         void check(List<OLAPContext> contexts, OLAPRel tree, KylinConfig config);
     }
diff --git a/src/query-common/src/main/java/org/apache/kylin/query/routing/RealizationChooser.java b/src/query-common/src/main/java/org/apache/kylin/query/routing/RealizationChooser.java
index 284a2fc190..d5cb7dc419 100644
--- a/src/query-common/src/main/java/org/apache/kylin/query/routing/RealizationChooser.java
+++ b/src/query-common/src/main/java/org/apache/kylin/query/routing/RealizationChooser.java
@@ -67,7 +67,6 @@ import org.apache.kylin.common.logging.SetLogCategory;
 import org.apache.kylin.common.msg.MsgPicker;
 import org.apache.kylin.common.util.NamedThreadFactory;
 import org.apache.kylin.common.util.SetThreadName;
-import org.apache.kylin.metadata.MetadataExtension;
 import org.apache.kylin.metadata.cube.cuboid.NLayoutCandidate;
 import org.apache.kylin.metadata.cube.cuboid.NLookupCandidate;
 import org.apache.kylin.metadata.cube.model.LayoutEntity;
@@ -277,8 +276,7 @@ public class RealizationChooser {
     private static void sortCandidate(OLAPContext context, List<Candidate> candidates) {
         ProjectInstance projectInstance = NProjectManager.getInstance(KylinConfig.getInstanceFromEnv())
                 .getProject(context.olapSchema.getProjectName());
-        if (projectInstance.getConfig().useTableIndexAnswerSelectStarEnabled()
-                && context.getSQLDigest().isRawQuery) {
+        if (projectInstance.getConfig().useTableIndexAnswerSelectStarEnabled() && context.getSQLDigest().isRawQuery) {
             candidates.sort(Candidate.COMPARATOR_TABLE_INDEX);
         } else {
             candidates.sort(Candidate.COMPARATOR);
@@ -294,7 +292,7 @@ public class RealizationChooser {
             TableDesc tableDesc = tableManager.getTableDesc(tableScan.getTableName());
             if (tableDesc.getSourceType() == ISourceAware.ID_STREAMING) {
                 throw new NoStreamingRealizationFoundException(STREAMING_MODEL_NOT_FOUND,
-                        String.format(Locale.ROOT, MsgPicker.getMsg().getNoStreamingModelFound()));
+                        MsgPicker.getMsg().getNoStreamingModelFound());
             }
         }
     }
@@ -334,15 +332,7 @@ public class RealizationChooser {
     }
 
     private static boolean needToManyDerived(NDataModel model) {
-        KylinConfig config = KylinConfig.getInstanceFromEnv();
-        Set<String> excludedTables = MetadataExtension.getFactory().getQueryExcludedTablesExtension()
-                .getExcludedTables(config, model.getProject());
-        for (JoinTableDesc joinTable : model.getJoinTables()) {
-            if (joinTable.isDerivedToManyJoinRelation() || excludedTables.contains(joinTable.getTable())) {
-                return true;
-            }
-        }
-        return false;
+        return model.getJoinTables().stream().anyMatch(JoinTableDesc::isDerivedToManyJoinRelation);
     }
 
     private static boolean hasReadySegments(NDataModel model) {
@@ -724,16 +714,16 @@ public class RealizationChooser {
     }
 
     private static KylinConfig getProjectConfig() {
-        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        String project = QueryContext.current().getProject();
         try {
-            if (QueryContext.current().getProject() != null) {
-                return NProjectManager.getInstance(kylinConfig).getProject(QueryContext.current().getProject())
-                        .getConfig();
+            if (project != null) {
+                return NProjectManager.getProjectConfig(project);
             }
         } catch (Exception e) {
-            logger.error("Fail to get project config is query match partial inner join model.", e);
+            logger.error("Failed to get config of project<{}> when matching partial inner join model. {}", //
+                    project, e.getMessage());
         }
-        return kylinConfig;
+        return KylinConfig.getInstanceFromEnv();
     }
 
 }
diff --git a/src/query-common/src/main/java/org/apache/kylin/query/routing/RealizationPruner.java b/src/query-common/src/main/java/org/apache/kylin/query/routing/RealizationPruner.java
index ff086f3b08..b429cf0407 100644
--- a/src/query-common/src/main/java/org/apache/kylin/query/routing/RealizationPruner.java
+++ b/src/query-common/src/main/java/org/apache/kylin/query/routing/RealizationPruner.java
@@ -52,21 +52,21 @@ import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.ClassUtil;
 import org.apache.kylin.common.util.DateFormat;
 import org.apache.kylin.common.util.Pair;
-import org.apache.kylin.metadata.datatype.DataType;
-import org.apache.kylin.metadata.model.ISourceAware;
-import org.apache.kylin.metadata.model.PartitionDesc;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.query.relnode.OLAPContext;
-import org.apache.kylin.query.relnode.OLAPTableScan;
 import org.apache.kylin.metadata.cube.model.NDataSegment;
 import org.apache.kylin.metadata.cube.model.NDataflow;
 import org.apache.kylin.metadata.cube.model.NDataflowManager;
+import org.apache.kylin.metadata.datatype.DataType;
+import org.apache.kylin.metadata.model.ISourceAware;
 import org.apache.kylin.metadata.model.MultiPartitionDesc;
 import org.apache.kylin.metadata.model.MultiPartitionKeyMapping;
 import org.apache.kylin.metadata.model.MultiPartitionKeyMappingProvider;
 import org.apache.kylin.metadata.model.NDataModel;
 import org.apache.kylin.metadata.model.NDataModelManager;
+import org.apache.kylin.metadata.model.PartitionDesc;
+import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.project.NProjectManager;
+import org.apache.kylin.query.relnode.OLAPContext;
+import org.apache.kylin.query.relnode.OLAPTableScan;
 
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
@@ -155,6 +155,10 @@ public class RealizationPruner {
             return allReadySegments;
         }
 
+        if (dateFormat == null) {
+            return allReadySegments;
+        }
+
         for (NDataSegment dataSegment : allReadySegments) {
             try {
                 val segmentRanges = transformSegment2RexCall(dataSegment, dateFormat, rexBuilder, partitionColInputRef,
diff --git a/src/query-common/src/main/java/org/apache/kylin/query/util/ComputedColumnRewriter.java b/src/query-common/src/main/java/org/apache/kylin/query/util/ComputedColumnRewriter.java
index d6e3be6edd..17e0ae897d 100644
--- a/src/query-common/src/main/java/org/apache/kylin/query/util/ComputedColumnRewriter.java
+++ b/src/query-common/src/main/java/org/apache/kylin/query/util/ComputedColumnRewriter.java
@@ -25,17 +25,19 @@ import java.util.Map;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.kylin.common.KapConfig;
-import org.apache.kylin.metadata.model.ParameterDesc;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.metadata.model.tool.CalciteParser;
-import org.apache.kylin.query.relnode.OLAPContext;
-import org.apache.kylin.query.relnode.TableColRefWithRel;
+import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.CollectionUtil;
 import org.apache.kylin.metadata.model.ComputedColumnDesc;
 import org.apache.kylin.metadata.model.NDataModel;
+import org.apache.kylin.metadata.model.ParameterDesc;
+import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.model.alias.ExpressionComparator;
+import org.apache.kylin.metadata.model.tool.CalciteParser;
 import org.apache.kylin.metadata.model.util.ComputedColumnUtil;
+import org.apache.kylin.metadata.project.NProjectManager;
 import org.apache.kylin.query.relnode.KapAggregateRel;
+import org.apache.kylin.query.relnode.OLAPContext;
+import org.apache.kylin.query.relnode.TableColRefWithRel;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -52,13 +54,17 @@ public class ComputedColumnRewriter {
     }
 
     public static void rewriteCcInnerCol(OLAPContext context, NDataModel model, QueryAliasMatchInfo matchInfo) {
-        rewriteAggInnerCol(context, model, matchInfo);
-        rewriteTopNInnerCol(context, model, matchInfo);
-        rewriteGroupByInnerCol(context, model, matchInfo);
-        rewriteFilterInnerCol(context, model, matchInfo);
+        KylinConfig projectConfig = NProjectManager.getProjectConfig(model.getProject());
+        rewriteAggInnerCol(projectConfig, context, model, matchInfo);
+        rewriteTopNInnerCol(projectConfig, context, model, matchInfo);
+        rewriteGroupByInnerCol(projectConfig, context, model, matchInfo);
+
+        // rewrite inner column of filter is not support yet.
+        // rewriteFilterInnerCol(projectConfig, context, model, matchInfo)
     }
 
-    private static void rewriteAggInnerCol(OLAPContext context, NDataModel model, QueryAliasMatchInfo matchInfo) {
+    private static void rewriteAggInnerCol(KylinConfig kylinConfig, OLAPContext context, NDataModel model,
+            QueryAliasMatchInfo matchInfo) {
         if (!KapConfig.getInstanceFromEnv().isAggComputedColumnRewriteEnabled()
                 || CollectionUtils.isEmpty(model.getComputedColumnDescs())) {
             return;
@@ -72,7 +78,7 @@ public class ComputedColumnRewriter {
                     continue;
                 }
 
-                TblColRef translatedInnerCol = rewriteInnerColumnToTblColRef(
+                TblColRef translatedInnerCol = rewriteInnerColumnToTblColRef(kylinConfig,
                         parameter.getColRef().getParserDescription(), model, matchInfo);
                 if (translatedInnerCol != null) {
                     parameters.add(ParameterDesc.newInstance(translatedInnerCol));
@@ -86,8 +92,8 @@ public class ComputedColumnRewriter {
         });
     }
 
-    private static TblColRef rewriteInnerColumnToTblColRef(String innerExpression, NDataModel model,
-            QueryAliasMatchInfo matchInfo) {
+    private static TblColRef rewriteInnerColumnToTblColRef(KylinConfig kylinConfig, String innerExpression,
+            NDataModel model, QueryAliasMatchInfo matchInfo) {
         SqlNode innerColExpr;
         try {
             innerColExpr = CalciteParser.getExpNode(innerExpression);
@@ -97,6 +103,10 @@ public class ComputedColumnRewriter {
         }
 
         for (ComputedColumnDesc cc : model.getComputedColumnDescs()) {
+            if (kylinConfig.isTableExclusionEnabled() && kylinConfig.onlyReuseUserDefinedCC() && cc.isAutoCC()) {
+                continue;
+            }
+
             SqlNode ccExpressionNode = CalciteParser.getExpNode(cc.getExpression());
             if (ExpressionComparator.isNodeEqual(innerColExpr, ccExpressionNode, matchInfo,
                     new AliasDeduceImpl(matchInfo))) {
@@ -110,7 +120,8 @@ public class ComputedColumnRewriter {
         return null;
     }
 
-    private static void rewriteTopNInnerCol(OLAPContext context, NDataModel model, QueryAliasMatchInfo matchInfo) {
+    private static void rewriteTopNInnerCol(KylinConfig kylinConfig, OLAPContext context, NDataModel model,
+            QueryAliasMatchInfo matchInfo) {
         if (CollectionUtils.isEmpty(model.getComputedColumnDescs()))
             return;
 
@@ -124,7 +135,7 @@ public class ComputedColumnRewriter {
                 if (innerExpression == null)
                     continue;
 
-                val translatedInnerCol = rewriteInnerColumnToTblColRef(innerExpression, model, matchInfo);
+                val translatedInnerCol = rewriteInnerColumnToTblColRef(kylinConfig, innerExpression, model, matchInfo);
                 if (translatedInnerCol != null)
                     translatedOperands.add(translatedInnerCol);
             }
@@ -133,7 +144,8 @@ public class ComputedColumnRewriter {
         });
     }
 
-    private static void rewriteGroupByInnerCol(OLAPContext ctx, NDataModel model, QueryAliasMatchInfo matchInfo) {
+    private static void rewriteGroupByInnerCol(KylinConfig kylinConfig, OLAPContext ctx, NDataModel model,
+            QueryAliasMatchInfo matchInfo) {
         if (CollectionUtils.isEmpty(model.getComputedColumnDescs())) {
             return;
         }
@@ -150,6 +162,9 @@ public class ComputedColumnRewriter {
                 continue;
             }
             for (ComputedColumnDesc cc : model.getComputedColumnDescs()) {
+                if (kylinConfig.isTableExclusionEnabled() && kylinConfig.onlyReuseUserDefinedCC() && cc.isAutoCC()) {
+                    continue;
+                }
                 SqlNode ccExpressionNode = CalciteParser.getExpNode(cc.getExpression());
                 if (ExpressionComparator.isNodeEqual(innerColExpr, ccExpressionNode, matchInfo,
                         new AliasDeduceImpl(matchInfo))) {
@@ -177,8 +192,4 @@ public class ComputedColumnRewriter {
             aggRel.reBuildGroups(colReplacementMapping);
         }
     }
-
-    private static void rewriteFilterInnerCol(OLAPContext ctx, NDataModel model, QueryAliasMatchInfo matchInfo) {
-        //empty
-    }
 }
diff --git a/src/query-common/src/main/java/org/apache/kylin/query/util/ConvertToComputedColumn.java b/src/query-common/src/main/java/org/apache/kylin/query/util/ConvertToComputedColumn.java
index 1958ccd59c..ebe5a5af3e 100644
--- a/src/query-common/src/main/java/org/apache/kylin/query/util/ConvertToComputedColumn.java
+++ b/src/query-common/src/main/java/org/apache/kylin/query/util/ConvertToComputedColumn.java
@@ -25,11 +25,13 @@ import java.util.Comparator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Objects;
+import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 
 import javax.annotation.Nullable;
 
+import org.apache.calcite.avatica.util.Quoting;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlAsOperator;
 import org.apache.calcite.sql.SqlCall;
@@ -47,6 +49,7 @@ import org.apache.calcite.sql.parser.SqlParseException;
 import org.apache.calcite.sql.util.SqlBasicVisitor;
 import org.apache.calcite.sql.util.SqlVisitor;
 import org.apache.calcite.util.Litmus;
+import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.kylin.common.KapConfig;
 import org.apache.kylin.common.KylinConfig;
@@ -57,6 +60,7 @@ import org.apache.kylin.metadata.model.ComputedColumnDesc;
 import org.apache.kylin.metadata.model.NDataModel;
 import org.apache.kylin.metadata.model.alias.ExpressionComparator;
 import org.apache.kylin.metadata.model.tool.CalciteParser;
+import org.apache.kylin.metadata.project.NProjectManager;
 import org.apache.kylin.query.IQueryTransformer;
 
 import com.google.common.base.Function;
@@ -75,6 +79,7 @@ import lombok.extern.slf4j.Slf4j;
 public class ConvertToComputedColumn implements IQueryTransformer {
 
     private static final String CONVERT_TO_CC_ERROR_MSG = "Something unexpected while ConvertToComputedColumn transforming the query, return original query.";
+    private static final String DOUBLE_QUOTE = Quoting.DOUBLE_QUOTE.string;
 
     private static final LoadingCache<String, String> transformExpressions = CacheBuilder.newBuilder()
             .maximumSize(10000).expireAfterWrite(10, TimeUnit.MINUTES).build(new CacheLoader<String, String>() {
@@ -109,7 +114,7 @@ public class ConvertToComputedColumn implements IQueryTransformer {
         // if order list is not empty and query is a select
         // then collect order list with checking on group keys
         List<SqlNode> inputNodes = new LinkedList<>();
-        if (sqlOrderBy.orderList != null && sqlOrderBy.query != null && sqlOrderBy.query instanceof SqlSelect
+        if (sqlOrderBy.orderList != null && sqlOrderBy.query instanceof SqlSelect
                 && ((SqlSelect) sqlOrderBy.query).getGroup() != null) {
             inputNodes.addAll(
                     collectCandidateInputNodes(sqlOrderBy.orderList, ((SqlSelect) sqlOrderBy.query).getGroup()));
@@ -208,12 +213,7 @@ public class ConvertToComputedColumn implements IQueryTransformer {
             }
             return alias.toString();
         }
-        if (node instanceof SqlNodeList) {
-            return StringUtils.EMPTY;
-        }
-        if (node instanceof SqlLiteral) {
-            return StringUtils.EMPTY;
-        }
+        // SqlNodeList, SqlLiteral and other return empty string
         return StringUtils.EMPTY;
     }
 
@@ -229,9 +229,8 @@ public class ConvertToComputedColumn implements IQueryTransformer {
             return Lists.newArrayList();
         }
 
-        Ordering<ComputedColumnDesc> ordering = Ordering
-                .from((Comparator<String>) (o1, o2) -> Integer.compare(o1.length(), o2.length())).reverse().nullsLast()
-                .onResultOf(new Function<ComputedColumnDesc, String>() {
+        Ordering<ComputedColumnDesc> ordering = Ordering.from(Comparator.comparingInt(String::length)).reverse() //
+                .nullsLast().onResultOf(new Function<ComputedColumnDesc, String>() {
                     @Nullable
                     @Override
                     public String apply(@Nullable ComputedColumnDesc input) {
@@ -262,11 +261,6 @@ public class ConvertToComputedColumn implements IQueryTransformer {
         return transformImpl(originSql, project, defaultSchema, dataModelDescs);
     }
 
-    public String transformImpl(String originSql, String project, NDataModel dataModelDesc, String defaultSchema)
-            throws SqlParseException {
-        return transformImpl(originSql, project, defaultSchema, Lists.newArrayList(dataModelDesc));
-    }
-
     private String transformImpl(String originSql, String project, String defaultSchema,
             List<NDataModel> dataModelDescs) throws SqlParseException {
 
@@ -334,7 +328,7 @@ public class ConvertToComputedColumn implements IQueryTransformer {
     private Pair<String, Integer> replaceComputedColumn(String inputSql, SqlCall selectOrOrderby,
             List<ComputedColumnDesc> computedColumns, QueryAliasMatchInfo queryAliasMatchInfo, boolean replaceCcName) {
 
-        if (computedColumns == null || computedColumns.isEmpty()) {
+        if (CollectionUtils.isEmpty(computedColumns)) {
             return Pair.newPair(inputSql, 0);
         }
 
@@ -361,9 +355,9 @@ public class ConvertToComputedColumn implements IQueryTransformer {
             if (queryAliasMatchInfo.isModelView()) {
                 // get alias with model alias
                 // as table of cc in model view is model view table itself
-                alias = queryAliasMatchInfo.getAliasMapping().inverse().get(queryAliasMatchInfo.getModel().getAlias());
+                alias = queryAliasMatchInfo.getAliasMap().inverse().get(queryAliasMatchInfo.getModel().getAlias());
             } else {
-                alias = queryAliasMatchInfo.getAliasMapping().inverse().get(cc.getTableAlias());
+                alias = queryAliasMatchInfo.getAliasMap().inverse().get(cc.getTableAlias());
             }
             if (alias == null) {
                 throw new IllegalStateException(cc.getExpression() + " expression of cc " + cc.getFullName()
@@ -374,8 +368,10 @@ public class ConvertToComputedColumn implements IQueryTransformer {
             String ccColumnName = replaceCcName ? cc.getInternalCcName() : cc.getColumnName();
             log.debug("Computed column: {} matching {} at [{},{}] using alias in query: {}", cc.getColumnName(), expr,
                     start, end, alias);
-            alias = Character.isAlphabetic(alias.charAt(0)) ? alias : "\"" + alias + "\"";
-            ccColumnName = Character.isAlphabetic(ccColumnName.charAt(0)) ? ccColumnName : "\"" + ccColumnName + "\"";
+
+            alias = Character.isAlphabetic(alias.charAt(0)) ? alias : DOUBLE_QUOTE + alias + DOUBLE_QUOTE;
+            ccColumnName = Character.isAlphabetic(ccColumnName.charAt(0)) ? ccColumnName
+                    : DOUBLE_QUOTE + ccColumnName + DOUBLE_QUOTE;
             result = result.substring(0, start) + alias + "." + ccColumnName + result.substring(end);
         }
         try {
@@ -415,6 +411,7 @@ public class ConvertToComputedColumn implements IQueryTransformer {
                     if (!(replaced.getFirst() >= end || replaced.getSecond() <= start)) {
                         // overlap with chosen areas
                         conflict = true;
+                        break;
                     }
                 }
                 if (conflict) {
@@ -461,13 +458,9 @@ public class ConvertToComputedColumn implements IQueryTransformer {
             }
         }
 
-        // find whether user input sql's tree node equals computed columns's define expression
-        for (SqlNode inputNode : inputNodes) {
-            if (isNodesEqual(matchInfo, ccExpressionNode, inputNode)) {
-                matchedNodes.add(inputNode);
-            }
-        }
-
+        // find whether user input tree node of sql equals defined expression of computed column
+        inputNodes.stream().filter(inputNode -> isNodesEqual(matchInfo, ccExpressionNode, inputNode))
+                .forEach(matchedNodes::add);
         return matchedNodes;
     }
 
@@ -583,24 +576,33 @@ public class ConvertToComputedColumn implements IQueryTransformer {
                     continue;
                 }
 
+                Set<String> cols = queryAliasMatcher.getChecker().filterRelatedExcludedColumn(model);
+                info.getExcludedColumns().addAll(cols);
                 List<ComputedColumnDesc> computedColumns = getSortedComputedColumnWithModel(model);
-                Pair<String, Integer> ret = replaceComputedColumn(sql, selectOrOrderby, computedColumns, info,
-                        replaceCcName);
-
-                if (replaceCcName && !sql.equals(ret.getFirst())) {
-                    choiceForCurrentSubquery = ret;
-                } else if (ret.getSecond() != 0 //
-                        && (choiceForCurrentSubquery == null
-                                || ret.getSecond() > choiceForCurrentSubquery.getSecond())) {
-                    choiceForCurrentSubquery = ret;
-                    recursionCompleted = false;
+                if (CollectionUtils.isNotEmpty(computedColumns)) {
+                    Pair<String, Integer> ret = replaceComputedColumn(sql, selectOrOrderby, computedColumns, info,
+                            replaceCcName);
+
+                    if (replaceCcName && !sql.equals(ret.getFirst())) {
+                        choiceForCurrentSubquery = ret;
+                    } else if (ret.getSecond() != 0 //
+                            && (choiceForCurrentSubquery == null
+                                    || ret.getSecond() > choiceForCurrentSubquery.getSecond())) {
+                        choiceForCurrentSubquery = ret;
+                        recursionCompleted = false;
+                    }
                 }
             }
         }
 
         //match longer expressions first
-        private List<ComputedColumnDesc> getSortedComputedColumnWithModel(NDataModel dataModelDesc) {
-            return getCCListSortByLength(dataModelDesc.getComputedColumnDescs());
+        private List<ComputedColumnDesc> getSortedComputedColumnWithModel(NDataModel model) {
+            List<ComputedColumnDesc> ccList = model.getComputedColumnDescs();
+            KylinConfig projectConfig = NProjectManager.getProjectConfig(model.getProject());
+            if (projectConfig.isTableExclusionEnabled() && projectConfig.onlyReuseUserDefinedCC()) {
+                ccList = ccList.stream().filter(cc -> !cc.isAutoCC()).collect(Collectors.toList());
+            }
+            return getCCListSortByLength(ccList);
         }
     }
 }
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 6f370d4d55..5e7bc10c46 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
@@ -47,6 +47,7 @@ import org.apache.kylin.common.KylinConfigExt;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.common.util.RandomUtil;
 import org.apache.kylin.metadata.cube.model.NDataflowManager;
+import org.apache.kylin.metadata.model.ColExcludedChecker;
 import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.JoinDesc;
 import org.apache.kylin.metadata.model.JoinsGraph;
@@ -69,20 +70,27 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
+import lombok.Getter;
+
 // match alias in query to alias in model
 // Not designed to reuse, re-new per query
 public class QueryAliasMatcher {
     static final ColumnRowType MODEL_VIEW_COLUMN_ROW_TYPE = new ColumnRowType(new ArrayList<>());
     private static final ColumnRowType SUBQUERY_TAG = new ColumnRowType(null);
     private static final String[] COLUMN_ARRAY_MARKER = new String[0];
+
     private final String project;
     private final String defaultSchema;
     private final Map<String, KapOLAPSchema> schemaMap = Maps.newHashMap();
     private final Map<String, Map<String, OLAPTable>> schemaTables = Maps.newHashMap();
 
+    @Getter
+    private final ColExcludedChecker checker;
+
     public QueryAliasMatcher(String project, String defaultSchema) {
         this.project = project;
         this.defaultSchema = defaultSchema;
+        checker = new ColExcludedChecker(KylinConfig.getInstanceFromEnv(), project, 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 0429f8a8fa..c08b4944ef 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
@@ -109,6 +109,10 @@ public class QueryUtil {
                 || (sql1.startsWith("explain") && sql1.contains(SELECT));
     }
 
+    public static boolean isSelectStarStatement(String sql) {
+        return SELECT_STAR_PTN.matcher(sql).find();
+    }
+
     public static String removeCommentInSql(String sql) {
         // match two patterns, one is "-- comment", the other is "/* comment */"
         try {
@@ -428,7 +432,7 @@ public class QueryUtil {
 
         // https://issues.apache.org/jira/browse/KYLIN-2649
         if (kylinConfig.getForceLimit() > 0 && limit <= 0 && !sql.toLowerCase(Locale.ROOT).contains("limit")
-                && SELECT_STAR_PTN.matcher(sql).find()) {
+                && isSelectStarStatement(sql)) {
             limit = kylinConfig.getForceLimit();
         }
 
diff --git a/src/query-common/src/main/java/org/apache/kylin/query/util/RestoreFromComputedColumn.java b/src/query-common/src/main/java/org/apache/kylin/query/util/RestoreFromComputedColumn.java
index 83d8a28a7e..8ae1337b11 100644
--- a/src/query-common/src/main/java/org/apache/kylin/query/util/RestoreFromComputedColumn.java
+++ b/src/query-common/src/main/java/org/apache/kylin/query/util/RestoreFromComputedColumn.java
@@ -236,7 +236,7 @@ public class RestoreFromComputedColumn implements IPushDownConverter {
             // The computed column expression is defined based on alias in model, e.g. BUYER_COUNTRY.x + BUYER_ACCOUNT.y
             // however user query might use a different alias, say bc.x + ba.y
             String ccExpression = CalciteParser.replaceAliasInExpr(computedColumnDesc.getExpression(),
-                    matchInfo.getAliasMapping().inverse());
+                    matchInfo.getAliasMap().inverse());
             // intend to handle situation like KE-15939
             String replaceExpression = columnUsage.addAlias ? ccExpression + " AS " + computedColumnDesc.getColumnName()
                     : ccExpression;
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 b28c4e96b6..3a147a4ba8 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
@@ -151,7 +151,6 @@ public class ModelServiceQueryTest extends SourceTestCase {
         cleanupTestMetadata();
     }
 
-    //    @Ignore("TODO: re-run to check.")
     @Test
     public void testQueryModels() {
         String project = "streaming_test";
@@ -233,7 +232,6 @@ public class ModelServiceQueryTest extends SourceTestCase {
         Assert.assertEquals(ModelStatusToDisplayEnum.BROKEN, nDataModelResponse.getStatus());
     }
 
-    //    @Ignore("TODO: re-run to check.")
     @Test
     public void testGetFusionModel() {
         String project = "streaming_test";
diff --git a/src/query/src/main/java/org/apache/kylin/query/engine/QueryExec.java b/src/query/src/main/java/org/apache/kylin/query/engine/QueryExec.java
index a1aecde4cb..463ac8d997 100644
--- a/src/query/src/main/java/org/apache/kylin/query/engine/QueryExec.java
+++ b/src/query/src/main/java/org/apache/kylin/query/engine/QueryExec.java
@@ -23,7 +23,6 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.LinkedHashSet;
 import java.util.List;
-import java.util.Locale;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 
@@ -50,11 +49,9 @@ import org.apache.kylin.common.QueryContext;
 import org.apache.kylin.common.QueryTrace;
 import org.apache.kylin.common.ReadFsSwitch;
 import org.apache.kylin.metadata.model.FunctionDesc;
+import org.apache.kylin.metadata.query.StructField;
 import org.apache.kylin.metadata.realization.NoRealizationFoundException;
 import org.apache.kylin.query.calcite.KylinRelDataTypeSystem;
-import org.apache.kylin.query.relnode.OLAPContext;
-import org.apache.kylin.query.util.AsyncQueryUtil;
-import org.apache.kylin.metadata.query.StructField;
 import org.apache.kylin.query.engine.data.QueryResult;
 import org.apache.kylin.query.engine.exec.ExecuteResult;
 import org.apache.kylin.query.engine.exec.calcite.CalciteQueryPlanExec;
@@ -63,8 +60,11 @@ import org.apache.kylin.query.engine.meta.SimpleDataContext;
 import org.apache.kylin.query.engine.view.ViewAnalyzer;
 import org.apache.kylin.query.mask.QueryResultMasks;
 import org.apache.kylin.query.relnode.KapAggregateRel;
+import org.apache.kylin.query.relnode.OLAPContext;
+import org.apache.kylin.query.util.AsyncQueryUtil;
 import org.apache.kylin.query.util.CalcitePlanRouterVisitor;
 import org.apache.kylin.query.util.HepUtils;
+import org.apache.kylin.query.util.QueryUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -166,8 +166,7 @@ public class QueryExec {
                 return new QueryResult();
             }
 
-            if (kylinConfig.getEmptyResultForSelectStar()
-                    && sql.toLowerCase(Locale.ROOT).matches("^select\\s+\\*\\p{all}*")
+            if (kylinConfig.getEmptyResultForSelectStar() && QueryUtil.isSelectStarStatement(sql)
                     && !QueryContext.current().getQueryTagInfo().isAsyncQuery()) {
                 return new QueryResult(Lists.newArrayList(), 0, resultFields);
             }
diff --git a/src/query/src/test/java/org/apache/kylin/query/util/ExpressionComparatorTest.java b/src/query/src/test/java/org/apache/kylin/query/util/ExpressionComparatorTest.java
index cc6a925f62..60fedb60a3 100644
--- a/src/query/src/test/java/org/apache/kylin/query/util/ExpressionComparatorTest.java
+++ b/src/query/src/test/java/org/apache/kylin/query/util/ExpressionComparatorTest.java
@@ -17,42 +17,28 @@
  */
 package org.apache.kylin.query.util;
 
-import static org.junit.Assert.assertEquals;
-
 import java.util.LinkedHashMap;
 
 import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.parser.SqlParseException;
 import org.apache.kylin.common.util.Pair;
-import org.apache.kylin.metadata.model.tool.CalciteParser;
-import org.apache.kylin.query.relnode.ColumnRowType;
-import org.apache.kylin.common.util.NLocalFileMetadataTestCase;
+import org.apache.kylin.junit.annotation.MetadataInfo;
 import org.apache.kylin.metadata.model.alias.AliasMapping;
 import org.apache.kylin.metadata.model.alias.ExpressionComparator;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.apache.kylin.metadata.model.tool.CalciteParser;
+import org.apache.kylin.query.relnode.ColumnRowType;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
 
 import com.google.common.collect.BiMap;
 import com.google.common.collect.HashBiMap;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Maps;
 
-public class ExpressionComparatorTest extends NLocalFileMetadataTestCase {
-
-    @Before
-    public void setUp() {
-        this.createTestMetadata();
-    }
-
-    @After
-    public void after() {
-        this.cleanupTestMetadata();
-    }
+@MetadataInfo(project = "default")
+class ExpressionComparatorTest {
 
     @Test
-    public void testBasicEqual() throws SqlParseException {
+    void testBasicEqual() {
         String sql0 = "select a.a + a.b + a.c from t as a";
         String sql1 = "select (((a . a +    a.b +    a.c))) from t as a";
         String sql2 = "select a.a + (a.b + a.c) from t as a";
@@ -65,13 +51,12 @@ public class ExpressionComparatorTest extends NLocalFileMetadataTestCase {
         aliasMapping.put("A", "A");
         QueryAliasMatchInfo matchInfo = new QueryAliasMatchInfo(aliasMapping, null);
 
-        assertEquals(true, ExpressionComparator.isNodeEqual(sn0, sn1, matchInfo, AliasDeduceImpl.NO_OP));
-        assertEquals(false, ExpressionComparator.isNodeEqual(sn0, sn3, matchInfo, AliasDeduceImpl.NO_OP));
-
+        Assertions.assertTrue(ExpressionComparator.isNodeEqual(sn0, sn1, matchInfo, AliasDeduceImpl.NO_OP));
+        Assertions.assertFalse(ExpressionComparator.isNodeEqual(sn0, sn3, matchInfo, AliasDeduceImpl.NO_OP));
     }
 
     @Test
-    public void testCommutativeEqual() throws SqlParseException {
+    void testCommutativeEqual() {
         String sql0 = "select a.a + a.b * a.c from t as a";
         String sql1 = "select a.c * a.b + a.a from t as a";
 
@@ -81,12 +66,11 @@ public class ExpressionComparatorTest extends NLocalFileMetadataTestCase {
         BiMap<String, String> aliasMapping = HashBiMap.create();
         aliasMapping.put("A", "A");
         QueryAliasMatchInfo matchInfo = new QueryAliasMatchInfo(aliasMapping, null);
-
-        Assert.assertTrue(ExpressionComparator.isNodeEqual(sn0, sn1, matchInfo, AliasDeduceImpl.NO_OP));
+        Assertions.assertTrue(ExpressionComparator.isNodeEqual(sn0, sn1, matchInfo, AliasDeduceImpl.NO_OP));
     }
 
     @Test
-    public void testAdvancedEqual() throws SqlParseException {
+    void testAdvancedEqual() {
         //treat sql0 as model
         String sql0 = "select a.a + a.b + a.c, cast(a.d as decimal(19,4)) from t as a";
 
@@ -112,7 +96,7 @@ public class ExpressionComparatorTest extends NLocalFileMetadataTestCase {
             mockQueryAlias.put("B", columnRowType);
 
             QueryAliasMatchInfo matchInfo = new QueryAliasMatchInfo(aliasMapping, mockQueryAlias);
-            assertEquals(true, ExpressionComparator.isNodeEqual(sn1, sn0, matchInfo, AliasDeduceImpl.NO_OP));
+            Assertions.assertTrue(ExpressionComparator.isNodeEqual(sn1, sn0, matchInfo, AliasDeduceImpl.NO_OP));
         }
 
         // when query not using alias
@@ -130,13 +114,31 @@ public class ExpressionComparatorTest extends NLocalFileMetadataTestCase {
             mockQueryAlias.put("T", columnRowType);
 
             QueryAliasMatchInfo matchInfo = new QueryAliasMatchInfo(aliasMapping, mockQueryAlias);
-            assertEquals(true, ExpressionComparator.isNodeEqual(sn2, sn0, matchInfo, new AliasDeduceImpl(matchInfo)));
+            Assertions.assertTrue(ExpressionComparator.isNodeEqual(sn2, sn0, matchInfo, //
+                    new AliasDeduceImpl(matchInfo)));
         }
 
+        // with excluded column
+        {
+            BiMap<String, String> aliasMapping = HashBiMap.create();
+            aliasMapping.put("T", "A");
+
+            ColumnRowType columnRowType = ColumnRowTypeMockUtil.mock("T", "T", //
+                    ImmutableList.of(Pair.newPair("A", "integer"), //
+                            Pair.newPair("B", "integer"), //
+                            Pair.newPair("C", "integer"), //
+                            Pair.newPair("D", "integer")));
+            LinkedHashMap<String, ColumnRowType> mockQueryAlias = Maps.newLinkedHashMap();
+            mockQueryAlias.put("T", columnRowType);
+            QueryAliasMatchInfo matchInfo = new QueryAliasMatchInfo(aliasMapping, mockQueryAlias);
+            matchInfo.getExcludedColumns().add("A.A");
+            Assertions.assertFalse(ExpressionComparator.isNodeEqual(sn2, sn0, matchInfo, //
+                    new AliasDeduceImpl(matchInfo)));
+        }
     }
 
     @Test
-    public void testNoNPE() {
+    void testNoNPE() {
         //https://github.com/Kyligence/KAP/issues/10934
         String sql0 = "select a.a + a.b + a.c from t as a";
         String sql1 = "select a.a + a.b + a.c from t as a";
@@ -148,30 +150,38 @@ public class ExpressionComparatorTest extends NLocalFileMetadataTestCase {
         SqlNode sn2 = CalciteParser.getOnlySelectNode(sql2);
         SqlNode sn3 = CalciteParser.getOnlySelectNode(sql3);
         {
-            AliasMapping aliasMapping = null;
-            ExpressionComparator.AliasMachingSqlNodeComparator matchInfo = new ExpressionComparator.AliasMachingSqlNodeComparator(
-                    aliasMapping, null);
-
-            assertEquals(false, matchInfo.isSqlNodeEqual(sn0, sn1));
+            ExpressionComparator.AliasMatchingSqlNodeComparator matchInfo = new ExpressionComparator.AliasMatchingSqlNodeComparator(
+                    null, null);
+            Assertions.assertFalse(matchInfo.isSqlNodeEqual(sn0, sn1));
         }
         {
             AliasMapping aliasMapping = new AliasMapping(null);
-            ExpressionComparator.AliasMachingSqlNodeComparator matchInfo = new ExpressionComparator.AliasMachingSqlNodeComparator(
+            ExpressionComparator.AliasMatchingSqlNodeComparator matchInfo = new ExpressionComparator.AliasMatchingSqlNodeComparator(
                     aliasMapping, null);
-            assertEquals(false, matchInfo.isSqlNodeEqual(sn0, sn1));
+            Assertions.assertFalse(matchInfo.isSqlNodeEqual(sn0, sn1));
         }
         {
-            AliasMapping aliasMapping = null;
-            ExpressionComparator.AliasMachingSqlNodeComparator matchInfo = new ExpressionComparator.AliasMachingSqlNodeComparator(
-                    aliasMapping, null);
-            assertEquals(true, matchInfo.isSqlNodeEqual(sn2, sn3));
+            ExpressionComparator.AliasMatchingSqlNodeComparator matchInfo = new ExpressionComparator.AliasMatchingSqlNodeComparator(
+                    null, null);
+            Assertions.assertTrue(matchInfo.isSqlNodeEqual(sn2, sn3));
         }
         {
             AliasMapping aliasMapping = new AliasMapping(null);
-            ExpressionComparator.AliasMachingSqlNodeComparator matchInfo = new ExpressionComparator.AliasMachingSqlNodeComparator(
+            ExpressionComparator.AliasMatchingSqlNodeComparator matchInfo = new ExpressionComparator.AliasMatchingSqlNodeComparator(
                     aliasMapping, null);
-            assertEquals(true, matchInfo.isSqlNodeEqual(sn2, sn3));
+            Assertions.assertTrue(matchInfo.isSqlNodeEqual(sn2, sn3));
+        }
+
+        {
+            ExpressionComparator.AliasMatchingSqlNodeComparator matchInfo = new ExpressionComparator.AliasMatchingSqlNodeComparator(
+                    null, null);
+            Assertions.assertFalse(matchInfo.isSqlNodeEqual(sn0, null));
         }
 
+        {
+            ExpressionComparator.AliasMatchingSqlNodeComparator matchInfo = new ExpressionComparator.AliasMatchingSqlNodeComparator(
+                    null, null);
+            Assertions.assertFalse(matchInfo.isSqlNodeEqual(null, sn1));
+        }
     }
 }
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 e5084d71ac..af76c793af 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
@@ -283,10 +283,30 @@ public class QueryUtilTest extends NLocalFileMetadataTestCase {
 
     @Test
     public void testIsSelectStatement() {
-        Assert.assertFalse(QueryUtil.isSelectStatement("INSERT INTO Person VALUES ('Li Si', 'Beijing');\n;\n"));
-        Assert.assertFalse(QueryUtil.isSelectStatement("UPDATE Person SET name = 'Fred' WHERE name = 'Li Si' "));
-        Assert.assertFalse(QueryUtil.isSelectStatement("DELETE FROM Person WHERE name = 'Wilson'"));
+        Assert.assertFalse(QueryUtil.isSelectStatement("insert into person values ('li si', 'beijing');\n;\n"));
+        Assert.assertFalse(QueryUtil.isSelectStatement("update t set name = 'fred' where name = 'lisi' "));
+        Assert.assertFalse(QueryUtil.isSelectStatement("delete from t where name = 'wilson'"));
         Assert.assertFalse(QueryUtil.isSelectStatement("drop table person"));
+        Assert.assertTrue(QueryUtil.isSelectStatement("with tempName as (select * from t) select * from tempName"));
+        Assert.assertFalse(QueryUtil.isSelectStatement("with tempName "));
+        Assert.assertTrue(QueryUtil.isSelectStatement("explain\n select * from tempName"));
+        Assert.assertFalse(QueryUtil.isSelectStatement("explain\n update t set name = 'fred' where name = 'lisi' "));
+    }
+
+    @Test
+    public void testIsSelectStarStatement() {
+        Assert.assertTrue(QueryUtil.isSelectStarStatement("select * from t"));
+        Assert.assertFalse(QueryUtil.isSelectStarStatement("select a from t"));
+        Assert.assertFalse(QueryUtil.isSelectStarStatement("update t set a = 1"));
+
+        // not support yet
+        String withClause = "with tempName as (select * from t) select * from tempName";
+        Assert.assertFalse(QueryUtil.isSelectStarStatement(withClause));
+    }
+
+    @Test
+    public void testRemoveCommentWithException() {
+        Assert.assertEquals("", QueryUtil.removeCommentInSql(""));
     }
 
     @Test
diff --git a/src/spark-project/engine-spark/pom.xml b/src/spark-project/engine-spark/pom.xml
index 2b4edf32e3..7c2f2960f4 100644
--- a/src/spark-project/engine-spark/pom.xml
+++ b/src/spark-project/engine-spark/pom.xml
@@ -66,6 +66,12 @@
             <groupId>org.apache.kylin</groupId>
             <artifactId>distributed-lock-ext</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-core-metadata</artifactId>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
 
         <dependency>
             <groupId>io.kyligence.ke</groupId>
diff --git a/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/job/NSparkCubingJob.java b/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/job/NSparkCubingJob.java
index 31f059f7a7..00dfe98882 100644
--- a/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/job/NSparkCubingJob.java
+++ b/src/spark-project/engine-spark/src/main/java/org/apache/kylin/engine/spark/job/NSparkCubingJob.java
@@ -18,16 +18,19 @@
 
 package org.apache.kylin.engine.spark.job;
 
-import com.google.common.base.Preconditions;
-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.secondstorage.SecondStorageConstants;
-import io.kyligence.kap.secondstorage.SecondStorageUtil;
-import io.kyligence.kap.secondstorage.enums.LockTypeEnum;
-import lombok.AllArgsConstructor;
-import lombok.Getter;
-import lombok.val;
+import static java.util.stream.Collectors.joining;
+import static org.apache.kylin.engine.spark.stats.utils.HiveTableRefChecker.isNeedCleanUpTransactionalTableJob;
+import static org.apache.kylin.job.factory.JobFactoryConstant.CUBE_JOB_FACTORY;
+
+import java.util.ArrayList;
+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.stream.Collectors;
+
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.kylin.common.KylinConfig;
@@ -46,24 +49,22 @@ import org.apache.kylin.metadata.cube.model.NDataflow;
 import org.apache.kylin.metadata.cube.model.NDataflowManager;
 import org.apache.kylin.metadata.cube.model.NDataflowUpdate;
 import org.apache.kylin.metadata.cube.model.PartitionStatusEnum;
-import org.apache.kylin.metadata.favorite.FavoriteRuleManager;
 import org.apache.kylin.metadata.job.JobBucket;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.ArrayList;
-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.stream.Collectors;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
 
-import static java.util.stream.Collectors.joining;
-import static org.apache.kylin.engine.spark.stats.utils.HiveTableRefChecker.isNeedCleanUpTransactionalTableJob;
-import static org.apache.kylin.job.factory.JobFactoryConstant.CUBE_JOB_FACTORY;
+import io.kyligence.kap.guava20.shaded.common.annotations.VisibleForTesting;
+import io.kyligence.kap.secondstorage.SecondStorageConstants;
+import io.kyligence.kap.secondstorage.SecondStorageUtil;
+import io.kyligence.kap.secondstorage.enums.LockTypeEnum;
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import lombok.val;
 
 public class NSparkCubingJob extends DefaultExecutableOnModel {
 
@@ -95,15 +96,15 @@ public class NSparkCubingJob extends DefaultExecutableOnModel {
 
     @VisibleForTesting
     public static NSparkCubingJob create(Set<NDataSegment> segments, Set<LayoutEntity> layouts, String submitter,
-                                         Set<JobBucket> buckets) {
+            Set<JobBucket> buckets) {
         return create(segments, layouts, submitter, JobTypeEnum.INDEX_BUILD, RandomUtil.randomUUIDStr(), null, null,
                 buckets);
     }
 
     @VisibleForTesting
     public static NSparkCubingJob create(Set<NDataSegment> segments, Set<LayoutEntity> layouts, String submitter,
-                                         JobTypeEnum jobType, String jobId, Set<String> ignoredSnapshotTables, Set<Long> partitions,
-                                         Set<JobBucket> buckets) {
+            JobTypeEnum jobType, String jobId, Set<String> ignoredSnapshotTables, Set<Long> partitions,
+            Set<JobBucket> buckets) {
         val params = new JobFactory.JobBuildParams(segments, layouts, submitter, jobType, jobId, null,
                 ignoredSnapshotTables, partitions, buckets, Maps.newHashMap());
         return innerCreate(params);
@@ -171,12 +172,6 @@ public class NSparkCubingJob extends DefaultExecutableOnModel {
         job.setParam(NBatchConstants.P_SEGMENT_IDS, String.join(",", job.getTargetSegments()));
         job.setParam(NBatchConstants.P_DATA_RANGE_START, String.valueOf(startTime));
         job.setParam(NBatchConstants.P_DATA_RANGE_END, String.valueOf(endTime));
-        FavoriteRuleManager ruleManager = FavoriteRuleManager.getInstance(kylinConfig, df.getProject());
-        Set<String> excludedTables = ruleManager.getExcludedTables();
-        // if excludedTables contains factTable, remove factTable in excludedTables
-        val rootFactTableName = df.getModel().getRootFactTableName();
-        excludedTables.remove(rootFactTableName);
-        job.setParam(NBatchConstants.P_EXCLUDED_TABLES, String.join(",", excludedTables));
         if (CollectionUtils.isNotEmpty(ignoredSnapshotTables)) {
             job.setParam(NBatchConstants.P_IGNORED_SNAPSHOT_TABLES, String.join(",", ignoredSnapshotTables));
         }
@@ -198,7 +193,7 @@ public class NSparkCubingJob extends DefaultExecutableOnModel {
     }
 
     private static AbstractExecutable initSecondStorageDeleteIndex(Set<LayoutEntity> toBeDeletedLayouts,
-                                                                   JobTypeEnum jobType, NDataflow df, NSparkCubingJob job, KylinConfigExt config) {
+            JobTypeEnum jobType, NDataflow df, NSparkCubingJob job, KylinConfigExt config) {
         if (!SecondStorageUtil.isModelEnable(df.getProject(), job.getTargetSubject())) {
             return null;
         }
@@ -211,7 +206,7 @@ public class NSparkCubingJob extends DefaultExecutableOnModel {
     }
 
     private static AbstractExecutable initSecondStorage(Set<LayoutEntity> layouts, JobTypeEnum jobType, NDataflow df,
-                                                        NSparkCubingJob job, KylinConfigExt config) {
+            NSparkCubingJob job, KylinConfigExt config) {
         AbstractExecutable secondStorage = null;
         if (SecondStorageUtil.isModelEnable(df.getProject(), job.getTargetSubject())) {
             // can't refresh segment when second storage do rebalanced
@@ -239,7 +234,7 @@ public class NSparkCubingJob extends DefaultExecutableOnModel {
     }
 
     private static AbstractExecutable initCleanUpTransactionalTable(KylinConfig kylinConfig, NDataflow df,
-                                                                    NSparkCubingJob job, KylinConfigExt config) {
+            NSparkCubingJob job, KylinConfigExt config) {
         AbstractExecutable cleanUpTransactionalTable = null;
         Boolean isRangePartitionTable = df.getModel().getAllTableRefs().stream()
                 .anyMatch(tableRef -> tableRef.getTableDesc().isRangePartition());
@@ -253,7 +248,8 @@ public class NSparkCubingJob extends DefaultExecutableOnModel {
         return cleanUpTransactionalTable;
     }
 
-    public static void setDAGRelations(AbstractExecutable job, KylinConfig config, NSparkCubingJob.NSparkCubingJobStep jobStep) {
+    public static void setDAGRelations(AbstractExecutable job, KylinConfig config,
+            NSparkCubingJob.NSparkCubingJobStep jobStep) {
         if (!StringUtils.equalsIgnoreCase(config.getJobSchedulerMode(), JobSchedulerModeEnum.CHAIN.toString())) {
             AbstractExecutable resourceDetect = jobStep.getResourceDetect();
             AbstractExecutable cubing = jobStep.getCubing();
@@ -285,14 +281,14 @@ public class NSparkCubingJob extends DefaultExecutableOnModel {
 
     public static void checkIfNeedBuildSnapshots(NSparkCubingJob job) {
         switch (job.getJobType()) {
-            case INC_BUILD:
-            case INDEX_REFRESH:
-            case INDEX_BUILD:
-                job.setParam(NBatchConstants.P_NEED_BUILD_SNAPSHOTS, "true");
-                break;
-            default:
-                job.setParam(NBatchConstants.P_NEED_BUILD_SNAPSHOTS, "false");
-                break;
+        case INC_BUILD:
+        case INDEX_REFRESH:
+        case INDEX_BUILD:
+            job.setParam(NBatchConstants.P_NEED_BUILD_SNAPSHOTS, "true");
+            break;
+        default:
+            job.setParam(NBatchConstants.P_NEED_BUILD_SNAPSHOTS, "false");
+            break;
         }
     }
 
@@ -348,41 +344,41 @@ public class NSparkCubingJob extends DefaultExecutableOnModel {
         Set<String> segmentIds = getSparkCubingStep().getSegmentIds();
         Set<Long> partitions = getSparkCubingStep().getTargetPartitions();
         switch (getJobType()) {
-            case SUB_PARTITION_BUILD:
-                for (String id : segmentIds) {
-                    NDataSegment segment = df.getSegment(id);
-                    if (segment == null) {
-                        continue;
-                    }
-                    // remove partition in layouts
-                    dfManager.removeLayoutPartition(df.getId(), partitions, Sets.newHashSet(segment.getId()));
-                    // remove partition in segments
-                    dfManager.removeSegmentPartition(df.getId(), partitions, Sets.newHashSet(segment.getId()));
-                    logger.info(String.format(Locale.ROOT, "Remove partitions [%s] in segment [%s] cause to cancel job.",
-                            partitions, id));
+        case SUB_PARTITION_BUILD:
+            for (String id : segmentIds) {
+                NDataSegment segment = df.getSegment(id);
+                if (segment == null) {
+                    continue;
                 }
-                break;
-            case SUB_PARTITION_REFRESH:
-                for (String id : segmentIds) {
-                    NDataSegment segment = df.getSegment(id);
-                    if (segment == null) {
-                        continue;
-                    }
-                    segment.getMultiPartitions().forEach(partition -> {
-                        if (partitions.contains(partition.getPartitionId())
-                                && PartitionStatusEnum.REFRESH == partition.getStatus()) {
-                            partition.setStatus(PartitionStatusEnum.READY);
-                        }
-                    });
-                    val dfUpdate = new NDataflowUpdate(df.getId());
-                    dfUpdate.setToUpdateSegs(segment);
-                    dfManager.updateDataflow(dfUpdate);
-                    logger.info(String.format(Locale.ROOT,
-                            "Change partitions [%s] in segment [%s] status to READY cause to cancel job.", partitions, id));
+                // remove partition in layouts
+                dfManager.removeLayoutPartition(df.getId(), partitions, Sets.newHashSet(segment.getId()));
+                // remove partition in segments
+                dfManager.removeSegmentPartition(df.getId(), partitions, Sets.newHashSet(segment.getId()));
+                logger.info(String.format(Locale.ROOT, "Remove partitions [%s] in segment [%s] cause to cancel job.",
+                        partitions, id));
+            }
+            break;
+        case SUB_PARTITION_REFRESH:
+            for (String id : segmentIds) {
+                NDataSegment segment = df.getSegment(id);
+                if (segment == null) {
+                    continue;
                 }
-                break;
-            default:
-                break;
+                segment.getMultiPartitions().forEach(partition -> {
+                    if (partitions.contains(partition.getPartitionId())
+                            && PartitionStatusEnum.REFRESH == partition.getStatus()) {
+                        partition.setStatus(PartitionStatusEnum.READY);
+                    }
+                });
+                val dfUpdate = new NDataflowUpdate(df.getId());
+                dfUpdate.setToUpdateSegs(segment);
+                dfManager.updateDataflow(dfUpdate);
+                logger.info(String.format(Locale.ROOT,
+                        "Change partitions [%s] in segment [%s] status to READY cause to cancel job.", partitions, id));
+            }
+            break;
+        default:
+            break;
         }
     }
 
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 d31d02f83e..308654a884 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
@@ -312,24 +312,12 @@ public class NSparkExecutable extends AbstractExecutable implements ChainedStage
         if (Objects.isNull(dataFlow) || StringUtils.isBlank(originSegments)) {
             return copied;
         }
-        removeFactTableInExcludedTables(dataFlow, copied);
         String newSegments = Stream.of(StringUtils.split(originSegments, COMMA))
                 .filter(id -> Objects.nonNull(dataFlow.getSegment(id))).collect(Collectors.joining(COMMA));
         copied.put(NBatchConstants.P_SEGMENT_IDS, newSegments);
         return copied;
     }
 
-    private void removeFactTableInExcludedTables(NDataflow dataFlow, final Map<String, String> originParams) {
-        val rootFactTableName = dataFlow.getModel().getRootFactTableName();
-        val excludedTablesString = originParams.getOrDefault(NBatchConstants.P_EXCLUDED_TABLES, "");
-        if (StringUtils.isBlank(excludedTablesString)) {
-            return;
-        }
-        val excludedTables = Sets.newHashSet(excludedTablesString.split(","));
-        excludedTables.remove(rootFactTableName);
-        originParams.put(NBatchConstants.P_EXCLUDED_TABLES, String.join(",", excludedTables));
-    }
-
     /**
      * generate the spark driver log hdfs path format, json path + timestamp + .log
      *
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 bd590e6104..29d38f641b 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,7 +18,9 @@
 
 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.util.HadoopUtil
 import org.apache.kylin.common.{KapConfig, KylinConfig}
@@ -37,8 +39,6 @@ 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 scala.collection.JavaConverters._
 import scala.collection.mutable
 import scala.collection.parallel.ForkJoinTaskSupport
@@ -46,6 +46,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 {
 
@@ -207,7 +209,7 @@ class SegmentFlatTable(private val sparkSession: SparkSession, //
 
   protected def generateLookupTables(): mutable.LinkedHashMap[JoinTableDesc, Dataset[Row]] = {
     val ret = mutable.LinkedHashMap[JoinTableDesc, Dataset[Row]]()
-    val normalizedTableSet = mutable.Set[String]()
+    val antiFlattenTableSet = mutable.Set[String]()
     dataModel.getJoinTables.asScala
       .filter(isTableToBuild)
       .foreach { joinDesc =>
@@ -216,12 +218,10 @@ class SegmentFlatTable(private val sparkSession: SparkSession, //
           throw new IllegalArgumentException("FK table cannot be null")
         }
         val fkTable = fkTableRef.getTableDesc.getIdentity
-        if (!joinDesc.isFlattenable || normalizedTableSet.contains(fkTable)) {
-          normalizedTableSet.add(joinDesc.getTable)
+        if (!joinDesc.isFlattenable || antiFlattenTableSet.contains(fkTable)) {
+          antiFlattenTableSet.add(joinDesc.getTable)
         }
-        if (joinDesc.isFlattenable && !dataSegment.getExcludedTables.contains(joinDesc.getTable)
-          && !dataSegment.getExcludedTables.contains(fkTable)
-          && !normalizedTableSet.contains(joinDesc.getTable)) {
+        if (joinDesc.isFlattenable && !antiFlattenTableSet.contains(joinDesc.getTable)) {
           val tableRef = joinDesc.getTableRef
           val tableDS = newTableDS(tableRef)
           ret.put(joinDesc, fulfillDS(tableDS, Set.empty, tableRef))
diff --git a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/SegmentJob.java b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/SegmentJob.java
index c573010c8e..479428f09a 100644
--- a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/SegmentJob.java
+++ b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/job/SegmentJob.java
@@ -44,7 +44,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableSet;
 
 public abstract class SegmentJob extends SparkApplication {
 
@@ -92,16 +91,8 @@ public abstract class SegmentJob extends SparkApplication {
 
         final Predicate<NDataSegment> notSkip = (NDataSegment dataSegment) -> !needSkipSegment(dataSegment);
 
-        String excludeTableStr = getParam(NBatchConstants.P_EXCLUDED_TABLES);
-        ImmutableSet<String> tables = StringUtils.isBlank(excludeTableStr) //
-                ? ImmutableSet.of()
-                : ImmutableSet.copyOf(excludeTableStr.split(SegmentJob.COMMA));
         readOnlySegments = Collections.unmodifiableSet((Set<? extends NDataSegment>) segmentIDs.stream() //
-                .map(segmentId -> {
-                    NDataSegment dataSegment = getSegment(segmentId);
-                    dataSegment.setExcludedTables(tables);
-                    return dataSegment;
-                }).filter(notSkip) //
+                .map(this::getSegment).filter(notSkip) //
                 .collect(Collectors.toCollection(LinkedHashSet::new)));
         runtime = new JobRuntime(config.getSegmentExecMaxThreads());
     }
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 f06bee49c2..4de752d79e 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,8 @@
 
 package org.apache.kylin.engine.spark.job.stage.build
 
-import com.google.common.collect.Sets
+import java.util.{Locale, Objects}
+
 import org.apache.commons.lang3.StringUtils
 import org.apache.hadoop.fs.Path
 import org.apache.kylin.common.util.HadoopUtil
@@ -43,7 +44,6 @@ import org.apache.spark.sql.types.StructField
 import org.apache.spark.sql.util.SparderTypeUtil
 import org.apache.spark.utils.ProxyThreadUtils
 
-import java.util.{Locale, Objects}
 import scala.collection.JavaConverters._
 import scala.collection.mutable
 import scala.collection.parallel.ForkJoinTaskSupport
@@ -51,6 +51,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)
@@ -220,7 +222,7 @@ abstract class FlatTableAndDictBase(private val jobContext: SegmentJob,
 
   def generateLookupTables(): mutable.LinkedHashMap[JoinTableDesc, Dataset[Row]] = {
     val ret = mutable.LinkedHashMap[JoinTableDesc, Dataset[Row]]()
-    val normalizedTableSet = mutable.Set[String]()
+    val antiFlattenTableSet = mutable.Set[String]()
     dataModel.getJoinTables.asScala
       .filter(isTableToBuild)
       .foreach { joinDesc =>
@@ -229,12 +231,10 @@ abstract class FlatTableAndDictBase(private val jobContext: SegmentJob,
           throw new IllegalArgumentException("FK table cannot be null")
         }
         val fkTable = fkTableRef.getTableDesc.getIdentity
-        if (!joinDesc.isFlattenable || normalizedTableSet.contains(fkTable)) {
-          normalizedTableSet.add(joinDesc.getTable)
+        if (!joinDesc.isFlattenable || antiFlattenTableSet.contains(fkTable)) {
+          antiFlattenTableSet.add(joinDesc.getTable)
         }
-        if (joinDesc.isFlattenable && !dataSegment.getExcludedTables.contains(joinDesc.getTable)
-          && !dataSegment.getExcludedTables.contains(fkTable)
-          && !normalizedTableSet.contains(joinDesc.getTable)) {
+        if (joinDesc.isFlattenable && !antiFlattenTableSet.contains(joinDesc.getTable)) {
           val tableRef = joinDesc.getTableRef
           val tableDS = newTableDS(tableRef)
           ret.put(joinDesc, fulfillDS(tableDS, Set.empty, tableRef))
diff --git a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/model/SegmentFlatTableDesc.java b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/model/SegmentFlatTableDesc.java
index 485f543da6..0cb125fcb7 100644
--- a/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/model/SegmentFlatTableDesc.java
+++ b/src/spark-project/engine-spark/src/main/scala/org/apache/kylin/engine/spark/model/SegmentFlatTableDesc.java
@@ -31,15 +31,15 @@ import org.apache.hadoop.fs.Path;
 import org.apache.kylin.common.KapConfig;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.HadoopUtil;
-import org.apache.kylin.metadata.model.JoinTableDesc;
-import org.apache.kylin.metadata.model.MeasureDesc;
-import org.apache.kylin.metadata.model.SegmentRange;
-import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.engine.spark.smarter.IndexDependencyParser;
 import org.apache.kylin.metadata.cube.cuboid.AdaptiveSpanningTree;
 import org.apache.kylin.metadata.cube.model.IndexPlan;
 import org.apache.kylin.metadata.cube.model.NDataSegment;
+import org.apache.kylin.metadata.model.JoinTableDesc;
+import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.NDataModel;
+import org.apache.kylin.metadata.model.SegmentRange;
+import org.apache.kylin.metadata.model.TblColRef;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
@@ -77,7 +77,7 @@ public class SegmentFlatTableDesc {
     }
 
     public SegmentFlatTableDesc(KylinConfig config, NDataSegment dataSegment, AdaptiveSpanningTree spanningTree,
-                                List<String> relatedTables) {
+            List<String> relatedTables) {
         this.config = config;
         this.kapConfig = KapConfig.getInstanceFromEnv();
         this.dataSegment = dataSegment;
@@ -211,7 +211,7 @@ public class SegmentFlatTableDesc {
         return spanningTree.getLevel0thIndices().stream().anyMatch(index -> index.getEffectiveDimCols().values() //
                 .stream().anyMatch(col -> !col.getTableRef().getTableIdentity().equalsIgnoreCase(factTableId)) //
                 || index.getEffectiveMeasures().values().stream().anyMatch(m -> m.getFunction().getColRefs().stream() //
-                .anyMatch(col -> !col.getTableRef().getTableIdentity().equalsIgnoreCase(factTableId))));
+                        .anyMatch(col -> !col.getTableRef().getTableIdentity().equalsIgnoreCase(factTableId))));
     }
 
     public int getFlatTableCoalescePartitionNum() {
@@ -288,9 +288,6 @@ public class SegmentFlatTableDesc {
         if (columnIdMap.containsKey(colRef.getIdentity())) {
             return;
         }
-        if (dataSegment.getExcludedTables().contains(colRef.getTable())) {
-            return;
-        }
 
         columns.add(colRef);
 
diff --git a/src/spark-project/engine-spark/src/test/java/org/apache/kylin/engine/spark/NLocalWithSparkSessionTest.java b/src/spark-project/engine-spark/src/test/java/org/apache/kylin/engine/spark/NLocalWithSparkSessionTest.java
index df13963a68..22e98dc05b 100644
--- a/src/spark-project/engine-spark/src/test/java/org/apache/kylin/engine/spark/NLocalWithSparkSessionTest.java
+++ b/src/spark-project/engine-spark/src/test/java/org/apache/kylin/engine/spark/NLocalWithSparkSessionTest.java
@@ -200,12 +200,6 @@ public class NLocalWithSparkSessionTest extends NLocalFileMetadataTestCase imple
 
     }
 
-    protected void updateProjectConfig(String property, String value) {
-        NProjectManager projectManager = NProjectManager.getInstance(getTestConfig());
-        projectManager.updateProject(getProject(),
-                copyForWrite -> copyForWrite.getOverrideKylinProps().put(property, value));
-    }
-
     private static DataType convertType(org.apache.kylin.metadata.datatype.DataType type) {
         if (type.isTimeFamily())
             return DataTypes.TimestampType;
diff --git a/src/spark-project/engine-spark/src/test/java/org/apache/kylin/engine/spark/job/NSparkCubingJobTest.java b/src/spark-project/engine-spark/src/test/java/org/apache/kylin/engine/spark/job/NSparkCubingJobTest.java
index f403a6ae77..7d01791c03 100644
--- a/src/spark-project/engine-spark/src/test/java/org/apache/kylin/engine/spark/job/NSparkCubingJobTest.java
+++ b/src/spark-project/engine-spark/src/test/java/org/apache/kylin/engine/spark/job/NSparkCubingJobTest.java
@@ -18,9 +18,22 @@
 
 package org.apache.kylin.engine.spark.job;
 
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import lombok.val;
+import static org.apache.kylin.metadata.cube.model.NBatchConstants.P_LAYOUT_IDS;
+import static org.awaitility.Awaitility.await;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang3.RandomUtils;
 import org.apache.kylin.common.KylinConfig;
@@ -58,8 +71,6 @@ import org.apache.kylin.metadata.cube.model.NDataflow;
 import org.apache.kylin.metadata.cube.model.NDataflowManager;
 import org.apache.kylin.metadata.cube.model.NDataflowUpdate;
 import org.apache.kylin.metadata.cube.model.NIndexPlanManager;
-import org.apache.kylin.metadata.favorite.FavoriteRule;
-import org.apache.kylin.metadata.favorite.FavoriteRuleManager;
 import org.apache.kylin.metadata.model.NDataModel;
 import org.apache.kylin.metadata.model.NDataModelManager;
 import org.apache.kylin.metadata.model.NTableMetadataManager;
@@ -72,6 +83,7 @@ import org.apache.kylin.metadata.project.NProjectManager;
 import org.apache.kylin.metadata.realization.IRealization;
 import org.apache.kylin.storage.IStorage;
 import org.apache.kylin.storage.IStorageQuery;
+import org.apache.kylin.util.MetadataTestUtils;
 import org.apache.spark.sql.Dataset;
 import org.apache.spark.sql.Row;
 import org.apache.spark.sql.SparkSession;
@@ -84,24 +96,12 @@ import org.junit.Ignore;
 import org.junit.Test;
 import org.mockito.Mockito;
 import org.sparkproject.guava.collect.Sets;
-import scala.Option;
-import scala.runtime.AbstractFunction1;
 
-import java.io.File;
-import java.io.IOException;
-import java.nio.charset.Charset;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.stream.Collectors;
+import com.google.common.collect.Maps;
 
-import static org.apache.kylin.metadata.cube.model.NBatchConstants.P_LAYOUT_IDS;
-import static org.awaitility.Awaitility.await;
+import lombok.val;
+import scala.Option;
+import scala.runtime.AbstractFunction1;
 
 public class NSparkCubingJobTest extends NLocalWithSparkSessionTest {
 
@@ -332,12 +332,8 @@ public class NSparkCubingJobTest extends NLocalWithSparkSessionTest {
             Assert.assertNull(layout);
         }
 
-        // add ExcludedTables
-        FavoriteRuleManager ruleManager = FavoriteRuleManager.getInstance(config, df.getProject());
-        List<FavoriteRule.AbstractCondition> conds = Lists.newArrayList();
-        //        isEnabled = request.isExcludeTablesEnable();
-        conds.add(new FavoriteRule.Condition(null, df.getModel().getRootFactTableName()));
-        ruleManager.updateRule(conds, true, FavoriteRule.EXCLUDED_TABLES_RULE);
+        // add ExcludedTable
+        MetadataTestUtils.mockExcludedTable(getProject(), df.getModel().getRootFactTableName());
 
         // Round1. Build new segment
         NSparkCubingJob job = NSparkCubingJob.create(Sets.newHashSet(oneSeg), Sets.newLinkedHashSet(round1), "ADMIN",
@@ -768,8 +764,7 @@ public class NSparkCubingJobTest extends NLocalWithSparkSessionTest {
         final String project = getProject();
         final KylinConfig config = getTestConfig();
         final String dfId = "89af4ee2-2cdb-4b07-b39e-4c29856309aa";
-        overwriteSystemProp("kylin.engine.spark.build-class-name",
-                "MockResumeBuildJob");
+        overwriteSystemProp("kylin.engine.spark.build-class-name", "MockResumeBuildJob");
         // prepare segment
         final NDataflowManager dfMgr = NDataflowManager.getInstance(config, project);
         final NExecutableManager execMgr = NExecutableManager.getInstance(config, project);
@@ -950,8 +945,7 @@ public class NSparkCubingJobTest extends NLocalWithSparkSessionTest {
                 throw new RuntimeException(e);
             }
             if (engineInterface == clz) {
-                return (I) ClassUtil
-                        .newInstance("NSparkCubingJobTest$MockParquetStorage");
+                return (I) ClassUtil.newInstance("NSparkCubingJobTest$MockParquetStorage");
             } else {
                 throw new RuntimeException("Cannot adapt to " + engineInterface);
             }