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 2021/05/08 04:11:10 UTC

[kylin] branch kylin-on-parquet-v2 updated (d16e7f0 -> f5ec8f3)

This is an automated email from the ASF dual-hosted git repository.

xxyu pushed a change to branch kylin-on-parquet-v2
in repository https://gitbox.apache.org/repos/asf/kylin.git.


    from d16e7f0  KYLIN-4980 Support prunning segments from complex filter conditions
     new 6d87071  [KYLIN-4554] validate filter condition on model saving
     new f5ec8f3   [KYLIN-4554] validate filter condition on model saving, fix bug added all the columns of all tables to the validate collection

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../java/org/apache/kylin/job/JoinedFormatter.java |  10 +-
 .../org/apache/kylin/metadata/util/ModelUtil.java  | 124 +++++++++++++++++++++
 .../apache/kylin/rest/service/ModelService.java    |  17 ++-
 .../kylin/rest/service/ModelServiceTest.java       |  25 ++++-
 4 files changed, 171 insertions(+), 5 deletions(-)
 create mode 100644 core-metadata/src/main/java/org/apache/kylin/metadata/util/ModelUtil.java

[kylin] 01/02: [KYLIN-4554] validate filter condition on model saving

Posted by xx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

xxyu pushed a commit to branch kylin-on-parquet-v2
in repository https://gitbox.apache.org/repos/asf/kylin.git

commit 6d87071158cb9256552c880c0c29bc995eb2cb54
Author: zheniantoushipashi <su...@gmail.com>
AuthorDate: Sun Apr 11 18:23:53 2021 +0800

    [KYLIN-4554] validate filter condition on model saving
    
    verify filter condition using the following login
    
    1 select * from tableName where {filterCondition} pass   calcite  parse
    
    2 all column in filter condition must be model table column
---
 .../java/org/apache/kylin/job/JoinedFormatter.java |  10 +-
 .../org/apache/kylin/metadata/util/ModelUtil.java  | 103 +++++++++++++++++++++
 .../apache/kylin/rest/service/ModelService.java    |  17 +++-
 .../kylin/rest/service/ModelServiceTest.java       |  25 ++++-
 4 files changed, 150 insertions(+), 5 deletions(-)

diff --git a/core-job/src/main/java/org/apache/kylin/job/JoinedFormatter.java b/core-job/src/main/java/org/apache/kylin/job/JoinedFormatter.java
index 3f19ce9..20e0f47 100644
--- a/core-job/src/main/java/org/apache/kylin/job/JoinedFormatter.java
+++ b/core-job/src/main/java/org/apache/kylin/job/JoinedFormatter.java
@@ -56,6 +56,14 @@ public class JoinedFormatter {
         setDateEnv(flatDesc);
     }
 
+    public JoinedFormatter(Boolean validateModel) {
+        // for validate model filter condition
+        String start = "20190710";
+        String end = "20190711";
+        setKeyValue(START_DATE, start);
+        setKeyValue(END_DATE, end);
+    }
+
     private void setDateEnv(IJoinedFlatTableDesc flatDesc) {
         DataModelDesc model = flatDesc.getDataModel();
         PartitionDesc partDesc = model.getPartitionDesc();
@@ -83,7 +91,7 @@ public class JoinedFormatter {
         return value == null ? "" : value;
     }
 
-    String formatSentence(String sentence) {
+    public String formatSentence(String sentence) {
         String[] cArray = REG_PATTERN.split(sentence);
         StringBuilder sbr = new StringBuilder();
         List<String> keys = getKeys(sentence);
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/util/ModelUtil.java b/core-metadata/src/main/java/org/apache/kylin/metadata/util/ModelUtil.java
new file mode 100644
index 0000000..a3dbf8d
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/util/ModelUtil.java
@@ -0,0 +1,103 @@
+/*
+ * 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.util;
+
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.kylin.metadata.model.TableDesc;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+
+public class ModelUtil {
+
+    private static final Logger logger = LoggerFactory.getLogger(ModelUtil.class);
+
+    public static void verifyFilterCondition(String factTableName, String filterCondition, TableDesc tableDesc)
+            throws Exception {
+        StringBuilder checkSql = new StringBuilder();
+        checkSql.append("select * from ").append(factTableName).append(" where ").append(filterCondition);
+
+        SqlCall inputToNode = (SqlCall) parse(doubleQuoteKeywordDefault(checkSql.toString()));
+        SqlVerify sqlVerify = new SqlVerify(tableDesc);
+        sqlVerify.visit(inputToNode);
+
+    }
+
+    public static SqlNode parse(String sql) throws Exception {
+        SqlParser.ConfigBuilder parserBuilder = SqlParser.configBuilder().setIdentifierMaxLength(300);
+        SqlParser sqlParser = SqlParser.create(sql, parserBuilder.build());
+        return sqlParser.parseQuery();
+    }
+
+    private static class SqlVerify extends SqlBasicVisitor {
+
+        private TableDesc tableDesc;
+
+        SqlVerify(TableDesc tableDesc) {
+            this.tableDesc = tableDesc;
+        }
+
+        @Override
+        public Object visit(SqlCall call) {
+            SqlSelect select = (SqlSelect) call;
+            WhereColumnVerify.verify(select.getWhere(), tableDesc);
+            return null;
+        }
+    }
+
+    private static class WhereColumnVerify extends SqlBasicVisitor {
+
+        private List<String> allSqlIdentifier = Lists.newArrayList();
+
+        static void verify(SqlNode whereNode, TableDesc tableDesc) {
+            WhereColumnVerify whereColumnVerify = new WhereColumnVerify();
+            whereNode.accept(whereColumnVerify);
+            whereColumnVerify.allSqlIdentifier.stream().forEach(col -> {
+                if (tableDesc.findColumnByName(col) == null) {
+                    String verifyError = String.format(Locale.ROOT,
+                            "filter condition col: %s is not a column in the table ", col);
+                    logger.error(verifyError);
+                    throw new IllegalArgumentException(verifyError);
+                }
+            });
+        }
+
+        public Object visit(SqlIdentifier id) {
+            allSqlIdentifier.add(id.names.get(0));
+            return null;
+        }
+    }
+
+    public static String doubleQuoteKeywordDefault(String sql) {
+        sql = sql.replaceAll("(?i)default\\.", "\"DEFAULT\".");
+        sql = sql.replace("defaultCatalog.", "");
+        sql = sql.replace("\"defaultCatalog\".", "");
+        return sql;
+    }
+
+}
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/ModelService.java b/server-base/src/main/java/org/apache/kylin/rest/service/ModelService.java
index 2bb803a..c488959 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/ModelService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/ModelService.java
@@ -31,6 +31,7 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.kylin.common.persistence.RootPersistentEntity;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.job.JoinedFormatter;
 import org.apache.kylin.metadata.ModifiedOrder;
 import org.apache.kylin.metadata.draft.Draft;
 import org.apache.kylin.metadata.model.DataModelDesc;
@@ -39,12 +40,15 @@ import org.apache.kylin.metadata.model.JoinsTree;
 import org.apache.kylin.metadata.model.ModelDimensionDesc;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.metadata.util.ModelUtil;
 import org.apache.kylin.rest.exception.BadRequestException;
 import org.apache.kylin.rest.exception.ForbiddenException;
 import org.apache.kylin.rest.msg.Message;
 import org.apache.kylin.rest.msg.MsgPicker;
 import org.apache.kylin.rest.util.AclEvaluate;
 import org.apache.kylin.rest.util.ValidateUtil;
+import org.apache.kylin.shaded.com.google.common.collect.Maps;
+import org.apache.kylin.shaded.com.google.common.collect.Sets;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
@@ -53,9 +57,6 @@ import org.springframework.security.access.AccessDeniedException;
 import org.springframework.security.core.context.SecurityContextHolder;
 import org.springframework.stereotype.Component;
 
-import org.apache.kylin.shaded.com.google.common.collect.Maps;
-import org.apache.kylin.shaded.com.google.common.collect.Sets;
-
 /**
  * @author jiazhong
  */
@@ -154,6 +155,16 @@ public class ModelService extends BasicService {
     public void validateModel(String project, DataModelDesc desc) throws IllegalArgumentException {
         String factTableName = desc.getRootFactTableName();
         TableDesc tableDesc = getTableManager().getTableDesc(factTableName, project);
+
+        if (!StringUtils.isEmpty(desc.getFilterCondition())) {
+            try {
+                JoinedFormatter formatter = new JoinedFormatter(true);
+                ModelUtil.verifyFilterCondition(factTableName, formatter.formatSentence(desc.getFilterCondition()),
+                        tableDesc);
+            } catch (Exception e) {
+                throw new BadRequestException(e.toString());
+            }
+        }
         if ((tableDesc.getSourceType() == ISourceAware.ID_STREAMING || tableDesc.isStreamingTable())
                 && (desc.getPartitionDesc() == null || desc.getPartitionDesc().getPartitionDateColumn() == null)) {
             throw new IllegalArgumentException("Must define a partition column.");
diff --git a/server/src/test/java/org/apache/kylin/rest/service/ModelServiceTest.java b/server/src/test/java/org/apache/kylin/rest/service/ModelServiceTest.java
index aeb6d79..b6ff06d 100644
--- a/server/src/test/java/org/apache/kylin/rest/service/ModelServiceTest.java
+++ b/server/src/test/java/org/apache/kylin/rest/service/ModelServiceTest.java
@@ -50,7 +50,7 @@ public class ModelServiceTest extends ServiceTestBase {
     @Test
     public void testSuccessModelUpdate() throws IOException, JobException {
         Serializer<DataModelDesc> serializer = modelService.getDataModelManager().getDataModelSerializer();
-        
+
         List<DataModelDesc> dataModelDescs = modelService.listAllModels("ci_inner_join_model", "default", true);
         Assert.assertTrue(dataModelDescs.size() == 1);
 
@@ -65,6 +65,29 @@ public class ModelServiceTest extends ServiceTestBase {
     }
 
     @Test
+    public void testVerifyFilterCondition() throws IOException {
+        Serializer<DataModelDesc> serializer = modelService.getDataModelManager()
+            .getDataModelSerializer();
+        List<DataModelDesc> dataModelDescs = modelService
+            .listAllModels("ci_inner_join_model", "default", true);
+        Assert.assertTrue(dataModelDescs.size() == 1);
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        serializer.serialize(dataModelDescs.get(0), new DataOutputStream(baos));
+        ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
+        DataModelDesc deserialize = serializer.deserialize(new DataInputStream(bais));
+        deserialize.setOwner("somebody");
+        deserialize.setFilterCondition("TRANS_ID = 1");
+        modelService.validateModel("default", deserialize);
+        try {
+            deserialize.setFilterCondition("TRANS_IDD = 1");
+            modelService.validateModel("default", deserialize);
+            Assert.fail("should throw an exception");
+        } catch (Exception e){
+            Assert.assertTrue(e.getMessage().equals("java.lang.IllegalArgumentException: filter condition col: TRANS_IDD is not a column in the table "));
+        }
+    }
+
+    @Test
     public void testRevisableModelInCaseOfDeleteMeasure() throws IOException {
         List<DataModelDesc> dataModelDescs = modelService.listAllModels("ci_left_join_model", "default", true);
         Assert.assertTrue(dataModelDescs.size() == 1);

[kylin] 02/02: [KYLIN-4554] validate filter condition on model saving, fix bug added all the columns of all tables to the validate collection

Posted by xx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

xxyu pushed a commit to branch kylin-on-parquet-v2
in repository https://gitbox.apache.org/repos/asf/kylin.git

commit f5ec8f3dd1a70c5e1125966e144d492248030de4
Author: zheniantoushipashi <su...@gmail.com>
AuthorDate: Wed Apr 14 11:06:49 2021 +0800

     [KYLIN-4554] validate filter condition on model saving, fix bug added all the columns of all tables to the validate collection
---
 .../org/apache/kylin/metadata/util/ModelUtil.java  | 47 ++++++++++++++++------
 .../apache/kylin/rest/service/ModelService.java    |  4 +-
 .../kylin/rest/service/ModelServiceTest.java       |  2 +-
 3 files changed, 37 insertions(+), 16 deletions(-)

diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/util/ModelUtil.java b/core-metadata/src/main/java/org/apache/kylin/metadata/util/ModelUtil.java
index a3dbf8d..2c09c48 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/util/ModelUtil.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/util/ModelUtil.java
@@ -18,8 +18,10 @@
 
 package org.apache.kylin.metadata.util;
 
+import java.util.Arrays;
 import java.util.List;
 import java.util.Locale;
+import java.util.stream.Collectors;
 
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlIdentifier;
@@ -27,7 +29,9 @@ import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlSelect;
 import org.apache.calcite.sql.parser.SqlParser;
 import org.apache.calcite.sql.util.SqlBasicVisitor;
-import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.metadata.TableMetadataManager;
+import org.apache.kylin.metadata.model.ColumnDesc;
+import org.apache.kylin.metadata.model.DataModelDesc;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -37,15 +41,15 @@ public class ModelUtil {
 
     private static final Logger logger = LoggerFactory.getLogger(ModelUtil.class);
 
-    public static void verifyFilterCondition(String factTableName, String filterCondition, TableDesc tableDesc)
-            throws Exception {
+    public static void verifyFilterCondition(String project, TableMetadataManager tableManager, DataModelDesc modelDesc,
+            String filterCondition) throws Exception {
         StringBuilder checkSql = new StringBuilder();
-        checkSql.append("select * from ").append(factTableName).append(" where ").append(filterCondition);
+        checkSql.append("select * from ").append(modelDesc.getRootFactTableName()).append(" where ")
+                .append(filterCondition);
 
         SqlCall inputToNode = (SqlCall) parse(doubleQuoteKeywordDefault(checkSql.toString()));
-        SqlVerify sqlVerify = new SqlVerify(tableDesc);
+        SqlVerify sqlVerify = new SqlVerify(project, tableManager, modelDesc);
         sqlVerify.visit(inputToNode);
-
     }
 
     public static SqlNode parse(String sql) throws Exception {
@@ -56,16 +60,20 @@ public class ModelUtil {
 
     private static class SqlVerify extends SqlBasicVisitor {
 
-        private TableDesc tableDesc;
+        private DataModelDesc modelDesc;
+        private TableMetadataManager tableManager;
+        private String project;
 
-        SqlVerify(TableDesc tableDesc) {
-            this.tableDesc = tableDesc;
+        SqlVerify(String project, TableMetadataManager tableManager, DataModelDesc modelDesc) {
+            this.modelDesc = modelDesc;
+            this.tableManager = tableManager;
+            this.project = project;
         }
 
         @Override
         public Object visit(SqlCall call) {
             SqlSelect select = (SqlSelect) call;
-            WhereColumnVerify.verify(select.getWhere(), tableDesc);
+            WhereColumnVerify.verify(project, select.getWhere(), modelDesc, tableManager);
             return null;
         }
     }
@@ -74,11 +82,20 @@ public class ModelUtil {
 
         private List<String> allSqlIdentifier = Lists.newArrayList();
 
-        static void verify(SqlNode whereNode, TableDesc tableDesc) {
+        static void verify(String project, SqlNode whereNode, DataModelDesc modelDesc,
+                TableMetadataManager tableManager) {
             WhereColumnVerify whereColumnVerify = new WhereColumnVerify();
             whereNode.accept(whereColumnVerify);
+
+            List<ColumnDesc> columnDesc = Arrays.stream(modelDesc.getJoinTables()).flatMap(table -> {
+                return Arrays.stream(tableManager.getTableDesc(table.getTable(), project).getColumns());
+            }).collect(Collectors.toList());
+            columnDesc.addAll(
+                    Arrays.asList(tableManager.getTableDesc(modelDesc.getRootFactTableName(), project).getColumns()));
+            List<String> allColumn = columnDesc.stream().map(cd -> cd.getName().toLowerCase(Locale.ROOT))
+                    .collect(Collectors.toList());
             whereColumnVerify.allSqlIdentifier.stream().forEach(col -> {
-                if (tableDesc.findColumnByName(col) == null) {
+                if (!allColumn.contains(col.toLowerCase(Locale.ROOT))) {
                     String verifyError = String.format(Locale.ROOT,
                             "filter condition col: %s is not a column in the table ", col);
                     logger.error(verifyError);
@@ -88,7 +105,11 @@ public class ModelUtil {
         }
 
         public Object visit(SqlIdentifier id) {
-            allSqlIdentifier.add(id.names.get(0));
+            if (id.names.size() == 1) {
+                allSqlIdentifier.add(id.names.get(0));
+            } else if (id.names.size() == 2) {
+                allSqlIdentifier.add(id.names.get(1));
+            }
             return null;
         }
     }
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/ModelService.java b/server-base/src/main/java/org/apache/kylin/rest/service/ModelService.java
index c488959..40722a8 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/ModelService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/ModelService.java
@@ -159,8 +159,8 @@ public class ModelService extends BasicService {
         if (!StringUtils.isEmpty(desc.getFilterCondition())) {
             try {
                 JoinedFormatter formatter = new JoinedFormatter(true);
-                ModelUtil.verifyFilterCondition(factTableName, formatter.formatSentence(desc.getFilterCondition()),
-                        tableDesc);
+                ModelUtil.verifyFilterCondition(project, getTableManager(), desc,
+                        formatter.formatSentence(desc.getFilterCondition()));
             } catch (Exception e) {
                 throw new BadRequestException(e.toString());
             }
diff --git a/server/src/test/java/org/apache/kylin/rest/service/ModelServiceTest.java b/server/src/test/java/org/apache/kylin/rest/service/ModelServiceTest.java
index b6ff06d..d1c48c4 100644
--- a/server/src/test/java/org/apache/kylin/rest/service/ModelServiceTest.java
+++ b/server/src/test/java/org/apache/kylin/rest/service/ModelServiceTest.java
@@ -79,7 +79,7 @@ public class ModelServiceTest extends ServiceTestBase {
         deserialize.setFilterCondition("TRANS_ID = 1");
         modelService.validateModel("default", deserialize);
         try {
-            deserialize.setFilterCondition("TRANS_IDD = 1");
+            deserialize.setFilterCondition("kylin_account.TRANS_IDD = 1");
             modelService.validateModel("default", deserialize);
             Assert.fail("should throw an exception");
         } catch (Exception e){