You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by GitBox <gi...@apache.org> on 2022/05/26 04:25:24 UTC

[GitHub] [incubator-doris] pengxiangyu opened a new pull request, #9786: Change policy

pengxiangyu opened a new pull request, #9786:
URL: https://github.com/apache/incubator-doris/pull/9786

   # Proposed changes
   
   Issue Number: close #xxx
   
   ## Problem Summary:
   
   Describe the overview of changes.
   
   ## Checklist(Required)
   
   1. Does it affect the original behavior: (Yes/No/I Don't know)
   2. Has unit tests been added: (Yes/No/No Need)
   3. Has document been added or modified: (Yes/No/No Need)
   4. Does it need to update dependencies: (Yes/No)
   5. Are there any changes that cannot be rolled back: (Yes/No)
   
   ## Further comments
   
   If this is a relatively large or complex change, kick off the discussion at [dev@doris.apache.org](mailto:dev@doris.apache.org) by explaining why you chose the solution you did and what alternatives you considered, etc...
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman merged pull request #9786: There is Policy which is not on TABLE, Policy should be a base class, and use TablePolicy to extends it.

Posted by GitBox <gi...@apache.org>.
morningman merged PR #9786:
URL: https://github.com/apache/incubator-doris/pull/9786


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] github-actions[bot] commented on pull request #9786: There is Policy which is not on TABLE, Policy should be a base class, and use TablePolicy to extends it.

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #9786:
URL: https://github.com/apache/incubator-doris/pull/9786#issuecomment-1143610203

   PR approved by at least one committer and no changes requested.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a diff in pull request #9786: There is Policy which is not on TABLE, Policy should be a base class, and use TablePolicy to extends it.

Posted by GitBox <gi...@apache.org>.
morningman commented on code in PR #9786:
URL: https://github.com/apache/incubator-doris/pull/9786#discussion_r884050084


##########
fe/fe-core/src/main/java/org/apache/doris/policy/Policy.java:
##########
@@ -18,79 +18,61 @@
 package org.apache.doris.policy;
 
 import org.apache.doris.analysis.CreatePolicyStmt;
-import org.apache.doris.analysis.Expr;
-import org.apache.doris.analysis.SqlParser;
-import org.apache.doris.analysis.SqlScanner;
 import org.apache.doris.analysis.UserIdentity;
 import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.Database;
 import org.apache.doris.catalog.Table;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.io.Text;
 import org.apache.doris.common.io.Writable;
-import org.apache.doris.common.util.SqlParserUtils;
 import org.apache.doris.persist.gson.GsonPostProcessable;
 import org.apache.doris.persist.gson.GsonUtils;
 import org.apache.doris.qe.ConnectContext;
 
-import com.google.common.collect.Lists;
 import com.google.gson.annotations.SerializedName;
-import lombok.AllArgsConstructor;
 import lombok.Data;
+import org.apache.commons.lang.StringUtils;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
-import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.io.StringReader;
 import java.util.List;
 
 /**
  * Save policy for filtering data.
  **/
 @Data
-@AllArgsConstructor
-public class Policy implements Writable, GsonPostProcessable {
-
-    public static final String ROW_POLICY = "ROW";
+public abstract class Policy implements Writable, GsonPostProcessable {

Review Comment:
   Modify the comment of this class



##########
fe/fe-core/src/main/cup/sql_parser.cup:
##########
@@ -1354,7 +1354,7 @@ create_stmt ::=
     {:
         RESULT = new CreateSqlBlockRuleStmt(ruleName, properties);
     :}
-    /* row policy */
+    /* table policy */

Review Comment:
   This is indeed a row policy, why change to table policy in comment?



##########
fe/fe-core/src/main/java/org/apache/doris/policy/TablePolicy.java:
##########
@@ -0,0 +1,166 @@
+// 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.doris.policy;
+
+import org.apache.doris.analysis.CreatePolicyStmt;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.SqlParser;
+import org.apache.doris.analysis.SqlScanner;
+import org.apache.doris.analysis.UserIdentity;
+import org.apache.doris.catalog.Catalog;
+import org.apache.doris.catalog.Database;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.io.Text;
+import org.apache.doris.common.util.SqlParserUtils;
+import org.apache.doris.persist.gson.GsonUtils;
+
+import com.google.common.collect.Lists;
+import com.google.gson.annotations.SerializedName;
+import lombok.Data;
+import org.apache.commons.lang.StringUtils;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.io.StringReader;
+import java.util.List;
+
+/**
+ * Save policy for filtering data.
+ **/
+@Data
+public class TablePolicy extends Policy {
+
+    private static final Logger LOG = LogManager.getLogger(TablePolicy.class);
+
+    /**
+     * Policy bind user.
+     **/
+    @SerializedName(value = "user")
+    private UserIdentity user = null;
+
+    @SerializedName(value = "dbId")
+    private long dbId = -1;
+
+    @SerializedName(value = "tableId")
+    private long tableId = -1;
+
+    /**
+     * PERMISSIVE | RESTRICTIVE, If multiple types exist, the last type prevails.
+     **/
+    @SerializedName(value = "filterType")
+    private FilterType filterType = null;
+
+    private Expr wherePredicate = null;
+
+    public TablePolicy() {}
+
+    /**
+     * Policy for Table. Policy of ROW or others.
+     *
+     * @param type PolicyType
+     * @param policyName policy name
+     * @param dbId database i
+     * @param user username
+     * @param originStmt origin stmt
+     * @param tableId table id
+     * @param filterType filter type
+     * @param wherePredicate where predicate
+     */
+    public TablePolicy(final PolicyTypeEnum type, final String policyName, long dbId,
+                  UserIdentity user, String originStmt, final long tableId,
+                  final FilterType filterType, final Expr wherePredicate) {
+        super(type, policyName, originStmt);
+        this.user = user;
+        this.dbId = dbId;
+        this.tableId = tableId;
+        this.filterType = filterType;
+        this.wherePredicate = wherePredicate;
+    }
+
+    /**
+     * Use for SHOW POLICY.
+     **/
+    public List<String> getShowInfo() throws AnalysisException {
+        Database database = Catalog.getCurrentCatalog().getDbOrAnalysisException(this.dbId);
+        Table table = database.getTableOrAnalysisException(this.tableId);
+        return Lists.newArrayList(this.policyName, database.getFullName(), table.getName(), this.type.name(),
+                this.filterType.name(), this.wherePredicate.toSql(), this.user.getQualifiedUser(), this.originStmt);
+    }
+
+    /**
+     * Read Table Policy from file.
+     **/
+    public static TablePolicy read(DataInput in) throws IOException {
+        String json = Text.readString(in);
+        return GsonUtils.GSON.fromJson(json, TablePolicy.class);
+    }
+
+    @Override
+    public void gsonPostProcess() throws IOException {
+        if (wherePredicate != null) {
+            return;
+        }
+        try {
+            SqlScanner input = new SqlScanner(new StringReader(originStmt), 0L);
+            SqlParser parser = new SqlParser(input);
+            CreatePolicyStmt stmt = (CreatePolicyStmt) SqlParserUtils.getFirstStmt(parser);
+            wherePredicate = stmt.getWherePredicate();
+        } catch (Exception e) {
+            throw new IOException("table policy parse originStmt error", e);
+        }
+    }
+
+    @Override
+    public TablePolicy clone() {
+        return new TablePolicy(this.type, this.policyName, this.dbId, this.user, this.originStmt, this.tableId,
+                               this.filterType, this.wherePredicate);
+    }
+
+    private boolean checkMatched(long dbId, long tableId, PolicyTypeEnum type,
+                                 String policyName, UserIdentity user) {
+        return super.checkMatched(type, policyName)
+                && (dbId == -1 || dbId == this.dbId)
+                && (tableId == -1 || tableId == this.tableId)
+                && (user == null || this.user == null
+                        || StringUtils.equals(user.getQualifiedUser(), this.user.getQualifiedUser()));
+    }
+
+    @Override
+    public boolean matchPolicy(Policy policy) {
+        if (!(policy instanceof TablePolicy)) {
+            return false;
+        }
+        TablePolicy tablePolicy = (TablePolicy) policy;
+        return checkMatched(tablePolicy.getDbId(), tablePolicy.getTableId(), tablePolicy.getType(),
+                            tablePolicy.getPolicyName(), tablePolicy.getUser());
+    }
+
+    @Override
+    public boolean matchPolicy(DropPolicyLog dropPolicyLog) {

Review Comment:
   This method name is strange



##########
fe/fe-core/src/main/java/org/apache/doris/policy/Policy.java:
##########
@@ -18,79 +18,61 @@
 package org.apache.doris.policy;
 
 import org.apache.doris.analysis.CreatePolicyStmt;
-import org.apache.doris.analysis.Expr;
-import org.apache.doris.analysis.SqlParser;
-import org.apache.doris.analysis.SqlScanner;
 import org.apache.doris.analysis.UserIdentity;
 import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.Database;
 import org.apache.doris.catalog.Table;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.io.Text;
 import org.apache.doris.common.io.Writable;
-import org.apache.doris.common.util.SqlParserUtils;
 import org.apache.doris.persist.gson.GsonPostProcessable;
 import org.apache.doris.persist.gson.GsonUtils;
 import org.apache.doris.qe.ConnectContext;
 
-import com.google.common.collect.Lists;
 import com.google.gson.annotations.SerializedName;
-import lombok.AllArgsConstructor;
 import lombok.Data;
+import org.apache.commons.lang.StringUtils;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
-import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.io.StringReader;
 import java.util.List;
 
 /**
  * Save policy for filtering data.
  **/
 @Data
-@AllArgsConstructor
-public class Policy implements Writable, GsonPostProcessable {
-
-    public static final String ROW_POLICY = "ROW";
+public abstract class Policy implements Writable, GsonPostProcessable {
 
     private static final Logger LOG = LogManager.getLogger(Policy.class);
 
-    @SerializedName(value = "dbId")
-    private long dbId;
-
-    @SerializedName(value = "tableId")
-    private long tableId;
+    @SerializedName(value = "type")
+    protected PolicyTypeEnum type = null;
 
     @SerializedName(value = "policyName")
-    private String policyName;
-
-    /**
-     * ROW.
-     **/
-    @SerializedName(value = "type")
-    private PolicyTypeEnum type;
+    protected String policyName = null;
 
     /**
-     * PERMISSIVE | RESTRICTIVE, If multiple types exist, the last type prevails.
+     * Use for Serialization/deserialization.
      **/
-    @SerializedName(value = "filterType")
-    private final FilterType filterType;
-
-    private Expr wherePredicate;
+    @SerializedName(value = "originStmt")
+    protected String originStmt;

Review Comment:
   Not all policy need to save origin stmt. I think we can move this field to TablePolicy.
   For row policy, we save origin stmt only because we can not serialize the filter expr directly.
   But for storage policy, there is only key value properties that can be serialized directly.



##########
fe/fe-core/src/main/java/org/apache/doris/policy/Policy.java:
##########
@@ -101,54 +83,42 @@ public static Policy fromCreateStmt(CreatePolicyStmt stmt) throws AnalysisExcept
             curDb = ConnectContext.get().getDatabase();
         }
         Database db = Catalog.getCurrentCatalog().getDbOrAnalysisException(curDb);
-        Table table = db.getTableOrAnalysisException(stmt.getTableName().getTbl());
         UserIdentity userIdent = stmt.getUser();
         userIdent.analyze(ConnectContext.get().getClusterName());
-        return new Policy(db.getId(), table.getId(), stmt.getPolicyName(), stmt.getType(), stmt.getFilterType(),
-                stmt.getWherePredicate(), userIdent, stmt.getOrigStmt().originStmt);
+        switch (stmt.getType()) {
+            case ROW:
+            default:
+                Table table = db.getTableOrAnalysisException(stmt.getTableName().getTbl());
+                return new TablePolicy(stmt.getType(), stmt.getPolicyName(), db.getId(), userIdent,
+                    stmt.getOrigStmt().originStmt, table.getId(), stmt.getFilterType(),
+                    stmt.getWherePredicate());
+        }
     }
 
     /**
      * Use for SHOW POLICY.
      **/
-    public List<String> getShowInfo() throws AnalysisException {
-        Database database = Catalog.getCurrentCatalog().getDbOrAnalysisException(this.dbId);
-        Table table = database.getTableOrAnalysisException(this.tableId);
-        return Lists.newArrayList(this.policyName, database.getFullName(), table.getName(), this.type.name(),
-                this.filterType.name(), this.wherePredicate.toSql(), this.user.getQualifiedUser(), this.originStmt);
-    }
+    public abstract List<String> getShowInfo() throws AnalysisException;
 
     @Override
     public void write(DataOutput out) throws IOException {
         Text.writeString(out, GsonUtils.GSON.toJson(this));
     }
 
-    /**
-     * Read policy from file.
-     **/
-    public static Policy read(DataInput in) throws IOException {
-        String json = Text.readString(in);
-        return GsonUtils.GSON.fromJson(json, Policy.class);
+    protected boolean checkMatched(PolicyTypeEnum type, String policyName) {
+        return (type == null || type.equals(this.type))

Review Comment:
   the `type` and `policyname` should not be null.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] pengxiangyu commented on a diff in pull request #9786: There is Policy which is not on TABLE, Policy should be a base class, and use TablePolicy to extends it.

Posted by GitBox <gi...@apache.org>.
pengxiangyu commented on code in PR #9786:
URL: https://github.com/apache/incubator-doris/pull/9786#discussion_r885501010


##########
fe/fe-common/src/main/java/org/apache/doris/common/FeMetaVersion.java:
##########
@@ -40,8 +40,10 @@ public final class FeMetaVersion {
     public static final int VERSION_109 = 109;
     // For routine load user info
     public static final int VERSION_110 = 110;
-    // NOTE: when increment meta version, should assign the latest version to VERSION_CURRENT
-    public static final int VERSION_CURRENT = VERSION_110;
+    // change Policy to TablePolicy
+    public static final int VERSION_111 = 111;

Review Comment:
   change it to:  VERSION_CURRENT = VERSION_111;   in next lines



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a diff in pull request #9786: There is Policy which is not on TABLE, Policy should be a base class, and use TablePolicy to extends it.

Posted by GitBox <gi...@apache.org>.
morningman commented on code in PR #9786:
URL: https://github.com/apache/incubator-doris/pull/9786#discussion_r886538597


##########
fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java:
##########
@@ -130,6 +132,12 @@ public class GsonUtils {
             .of(LoadJobStateUpdateInfo.class, "clazz")
             .registerSubtype(SparkLoadJobStateUpdateInfo.class, SparkLoadJobStateUpdateInfo.class.getSimpleName());
 
+
+    // runtime adapter for class "Resource"

Review Comment:
   ```suggestion
       // runtime adapter for class "Policy"
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] stalary commented on pull request #9786: Change policy

Posted by GitBox <gi...@apache.org>.
stalary commented on PR #9786:
URL: https://github.com/apache/incubator-doris/pull/9786#issuecomment-1138189958

   Can you submit an issue describing what you are going to do?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] stalary commented on pull request #9786: There is Policy which is not on TABLE, Policy should be a base class, and use TablePolicy to extends it.

Posted by GitBox <gi...@apache.org>.
stalary commented on PR #9786:
URL: https://github.com/apache/incubator-doris/pull/9786#issuecomment-1139489259

   LGTM


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] stalary commented on pull request #9786: There is Policy which is not on TABLE, Policy should be a base class, and use TablePolicy to extends it.

Posted by GitBox <gi...@apache.org>.
stalary commented on PR #9786:
URL: https://github.com/apache/incubator-doris/pull/9786#issuecomment-1138271405

   Need to rebase.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] github-actions[bot] commented on pull request #9786: There is Policy which is not on TABLE, Policy should be a base class, and use TablePolicy to extends it.

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #9786:
URL: https://github.com/apache/incubator-doris/pull/9786#issuecomment-1143610252

   PR approved by anyone and no changes requested.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] pengxiangyu commented on a diff in pull request #9786: There is Policy which is not on TABLE, Policy should be a base class, and use TablePolicy to extends it.

Posted by GitBox <gi...@apache.org>.
pengxiangyu commented on code in PR #9786:
URL: https://github.com/apache/incubator-doris/pull/9786#discussion_r885501010


##########
fe/fe-common/src/main/java/org/apache/doris/common/FeMetaVersion.java:
##########
@@ -40,8 +40,10 @@ public final class FeMetaVersion {
     public static final int VERSION_109 = 109;
     // For routine load user info
     public static final int VERSION_110 = 110;
-    // NOTE: when increment meta version, should assign the latest version to VERSION_CURRENT
-    public static final int VERSION_CURRENT = VERSION_110;
+    // change Policy to TablePolicy
+    public static final int VERSION_111 = 111;

Review Comment:
   change it to:  VERSION_CURRENT = VERSION_111;   in next lines



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] stalary commented on a diff in pull request #9786: There is Policy which is not on TABLE, Policy should be a base class, and use TablePolicy to extends it.

Posted by GitBox <gi...@apache.org>.
stalary commented on code in PR #9786:
URL: https://github.com/apache/incubator-doris/pull/9786#discussion_r882408928


##########
fe/fe-core/src/main/java/org/apache/doris/policy/Policy.java:
##########
@@ -17,106 +17,97 @@
 
 package org.apache.doris.policy;
 
-import org.apache.doris.analysis.CreatePolicyStmt;
-import org.apache.doris.analysis.Expr;
-import org.apache.doris.analysis.SqlParser;
-import org.apache.doris.analysis.SqlScanner;
+import org.apache.doris.analysis.CreateTablePolicyStmt;
 import org.apache.doris.analysis.UserIdentity;
 import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.Database;
 import org.apache.doris.catalog.Table;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.io.Text;
 import org.apache.doris.common.io.Writable;
-import org.apache.doris.common.util.SqlParserUtils;
 import org.apache.doris.persist.gson.GsonPostProcessable;
 import org.apache.doris.persist.gson.GsonUtils;
 import org.apache.doris.qe.ConnectContext;
 
-import com.google.common.collect.Lists;
 import com.google.gson.annotations.SerializedName;
-import lombok.AllArgsConstructor;
-import lombok.Data;
+
+import org.apache.commons.lang.StringUtils;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.io.StringReader;
 import java.util.List;
 
+import lombok.Data;
+
 /**
  * Save policy for filtering data.
  **/
 @Data
-@AllArgsConstructor
-public class Policy implements Writable, GsonPostProcessable {
-
-    public static final String ROW_POLICY = "ROW";
+public abstract class Policy implements Writable, GsonPostProcessable {
 
     private static final Logger LOG = LogManager.getLogger(Policy.class);
 
-    @SerializedName(value = "dbId")
-    private long dbId;
-
-    @SerializedName(value = "tableId")
-    private long tableId;
-
-    @SerializedName(value = "policyName")
-    private String policyName;
-
-    /**
-     * ROW.
-     **/
     @SerializedName(value = "type")
-    private PolicyTypeEnum type;
+    protected PolicyTypeEnum type;
 
-    /**
-     * PERMISSIVE | RESTRICTIVE, If multiple types exist, the last type prevails.
-     **/
-    @SerializedName(value = "filterType")
-    private final FilterType filterType;
+    @SerializedName(value = "policyName")
+    protected String policyName;
 
-    private Expr wherePredicate;
+    @SerializedName(value = "dbId")
+    protected long dbId;

Review Comment:
   Is it possible that stroage is not for DB, but for be nodes?



##########
fe/fe-core/src/main/java/org/apache/doris/policy/Policy.java:
##########
@@ -17,106 +17,97 @@
 
 package org.apache.doris.policy;
 
-import org.apache.doris.analysis.CreatePolicyStmt;
-import org.apache.doris.analysis.Expr;
-import org.apache.doris.analysis.SqlParser;
-import org.apache.doris.analysis.SqlScanner;
+import org.apache.doris.analysis.CreateTablePolicyStmt;
 import org.apache.doris.analysis.UserIdentity;
 import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.Database;
 import org.apache.doris.catalog.Table;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.io.Text;
 import org.apache.doris.common.io.Writable;
-import org.apache.doris.common.util.SqlParserUtils;
 import org.apache.doris.persist.gson.GsonPostProcessable;
 import org.apache.doris.persist.gson.GsonUtils;
 import org.apache.doris.qe.ConnectContext;
 
-import com.google.common.collect.Lists;
 import com.google.gson.annotations.SerializedName;
-import lombok.AllArgsConstructor;
-import lombok.Data;
+
+import org.apache.commons.lang.StringUtils;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.io.StringReader;
 import java.util.List;
 
+import lombok.Data;
+
 /**
  * Save policy for filtering data.
  **/
 @Data
-@AllArgsConstructor
-public class Policy implements Writable, GsonPostProcessable {
-
-    public static final String ROW_POLICY = "ROW";
+public abstract class Policy implements Writable, GsonPostProcessable {
 
     private static final Logger LOG = LogManager.getLogger(Policy.class);
 
-    @SerializedName(value = "dbId")
-    private long dbId;
-
-    @SerializedName(value = "tableId")
-    private long tableId;
-
-    @SerializedName(value = "policyName")
-    private String policyName;
-
-    /**
-     * ROW.
-     **/
     @SerializedName(value = "type")
-    private PolicyTypeEnum type;
+    protected PolicyTypeEnum type;
 
-    /**
-     * PERMISSIVE | RESTRICTIVE, If multiple types exist, the last type prevails.
-     **/
-    @SerializedName(value = "filterType")
-    private final FilterType filterType;
+    @SerializedName(value = "policyName")
+    protected String policyName;
 
-    private Expr wherePredicate;
+    @SerializedName(value = "dbId")
+    protected long dbId;
 
     /**
      * Policy bind user.
      **/
     @SerializedName(value = "user")
-    private final UserIdentity user;
+    protected final UserIdentity user;
 
     /**
      * Use for Serialization/deserialization.
      **/
     @SerializedName(value = "originStmt")
-    private String originStmt;
+    protected String originStmt;
+
+    public Policy() {
+        user = null;

Review Comment:
   Maybe we can not use final for user?



##########
fe/fe-core/src/main/java/org/apache/doris/policy/Policy.java:
##########
@@ -17,106 +17,97 @@
 
 package org.apache.doris.policy;
 
-import org.apache.doris.analysis.CreatePolicyStmt;
-import org.apache.doris.analysis.Expr;
-import org.apache.doris.analysis.SqlParser;
-import org.apache.doris.analysis.SqlScanner;
+import org.apache.doris.analysis.CreateTablePolicyStmt;
 import org.apache.doris.analysis.UserIdentity;
 import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.Database;
 import org.apache.doris.catalog.Table;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.io.Text;
 import org.apache.doris.common.io.Writable;
-import org.apache.doris.common.util.SqlParserUtils;
 import org.apache.doris.persist.gson.GsonPostProcessable;
 import org.apache.doris.persist.gson.GsonUtils;
 import org.apache.doris.qe.ConnectContext;
 
-import com.google.common.collect.Lists;
 import com.google.gson.annotations.SerializedName;
-import lombok.AllArgsConstructor;
-import lombok.Data;
+
+import org.apache.commons.lang.StringUtils;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.io.StringReader;
 import java.util.List;
 
+import lombok.Data;
+
 /**
  * Save policy for filtering data.
  **/
 @Data
-@AllArgsConstructor
-public class Policy implements Writable, GsonPostProcessable {
-
-    public static final String ROW_POLICY = "ROW";
+public abstract class Policy implements Writable, GsonPostProcessable {
 
     private static final Logger LOG = LogManager.getLogger(Policy.class);
 
-    @SerializedName(value = "dbId")
-    private long dbId;
-
-    @SerializedName(value = "tableId")
-    private long tableId;
-
-    @SerializedName(value = "policyName")
-    private String policyName;
-
-    /**
-     * ROW.
-     **/
     @SerializedName(value = "type")
-    private PolicyTypeEnum type;
+    protected PolicyTypeEnum type;
 
-    /**
-     * PERMISSIVE | RESTRICTIVE, If multiple types exist, the last type prevails.
-     **/
-    @SerializedName(value = "filterType")
-    private final FilterType filterType;
+    @SerializedName(value = "policyName")
+    protected String policyName;
 
-    private Expr wherePredicate;
+    @SerializedName(value = "dbId")
+    protected long dbId;
 
     /**
      * Policy bind user.
      **/
     @SerializedName(value = "user")
-    private final UserIdentity user;
+    protected final UserIdentity user;
 
     /**
      * Use for Serialization/deserialization.
      **/
     @SerializedName(value = "originStmt")
-    private String originStmt;
+    protected String originStmt;
+
+    public Policy() {
+        user = null;
+    }
 
+    public Policy(final PolicyTypeEnum type, final String policyName, long dbId,
+                  UserIdentity user, String originStmt) {
+        this.type = type;
+        this.policyName = policyName;
+        this.dbId = dbId;
+        this.user = user;
+        this.originStmt = originStmt;
+    }
     /**
      * Trans stmt to Policy.
      **/
-    public static Policy fromCreateStmt(CreatePolicyStmt stmt) throws AnalysisException {
+    public static Policy fromCreateStmt(CreateTablePolicyStmt stmt) throws AnalysisException {

Review Comment:
   Is it better to use CreateTablePolicyStmt in TablePolicy?



##########
fe/fe-core/src/main/java/org/apache/doris/policy/PolicyMgr.java:
##########
@@ -68,7 +68,7 @@ public class PolicyMgr implements Writable {
      * Cache merge policy for match.
      * key:dbId:tableId-type-user
      **/
-    private Map<Long, Map<String, Policy>> dbIdToMergePolicyMap = Maps.newConcurrentMap();
+    private Map<Long, Map<String, TablePolicy>> dbIdToMergePolicyMap = Maps.newConcurrentMap();

Review Comment:
   ```suggestion
       private Map<Long, Map<String, TablePolicy>> dbIdToMergeTablePolicyMap = Maps.newConcurrentMap();
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a diff in pull request #9786: There is Policy which is not on TABLE, Policy should be a base class, and use TablePolicy to extends it.

Posted by GitBox <gi...@apache.org>.
morningman commented on code in PR #9786:
URL: https://github.com/apache/incubator-doris/pull/9786#discussion_r885434959


##########
fe/fe-core/src/main/java/org/apache/doris/policy/RowPolicy.java:
##########
@@ -0,0 +1,174 @@
+// 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.doris.policy;
+
+import org.apache.doris.analysis.CreatePolicyStmt;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.SqlParser;
+import org.apache.doris.analysis.SqlScanner;
+import org.apache.doris.analysis.UserIdentity;
+import org.apache.doris.catalog.Catalog;
+import org.apache.doris.catalog.Database;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.io.Text;
+import org.apache.doris.common.util.SqlParserUtils;
+import org.apache.doris.persist.gson.GsonUtils;
+
+import com.google.common.collect.Lists;
+import com.google.gson.annotations.SerializedName;
+import lombok.Data;
+import org.apache.commons.lang.StringUtils;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.io.StringReader;
+import java.util.List;
+
+/**
+ * Save policy for filtering data.
+ **/
+@Data
+public class RowPolicy extends Policy {
+
+    private static final Logger LOG = LogManager.getLogger(RowPolicy.class);
+
+    /**
+     * Policy bind user.
+     **/
+    @SerializedName(value = "user")
+    private UserIdentity user = null;
+
+    @SerializedName(value = "dbId")
+    private long dbId = -1;
+
+    @SerializedName(value = "tableId")
+    private long tableId = -1;
+
+    /**
+     * PERMISSIVE | RESTRICTIVE, If multiple types exist, the last type prevails.
+     **/
+    @SerializedName(value = "filterType")
+    private FilterType filterType = null;
+
+    /**
+     * Use for Serialization/deserialization.
+     **/
+    @SerializedName(value = "originStmt")
+    private String originStmt;
+
+    private Expr wherePredicate = null;
+
+    public RowPolicy() {}
+
+    /**
+     * Policy for Table. Policy of ROW or others.
+     *
+     * @param type PolicyType
+     * @param policyName policy name
+     * @param dbId database i
+     * @param user username
+     * @param originStmt origin stmt
+     * @param tableId table id
+     * @param filterType filter type
+     * @param wherePredicate where predicate
+     */
+    public RowPolicy(final PolicyTypeEnum type, final String policyName, long dbId,
+                     UserIdentity user, String originStmt, final long tableId,
+                     final FilterType filterType, final Expr wherePredicate) {
+        super(type, policyName);
+        this.user = user;
+        this.dbId = dbId;
+        this.tableId = tableId;
+        this.filterType = filterType;
+        this.originStmt = originStmt;
+        this.wherePredicate = wherePredicate;
+    }
+
+    /**
+     * Use for SHOW POLICY.
+     **/
+    public List<String> getShowInfo() throws AnalysisException {
+        Database database = Catalog.getCurrentCatalog().getDbOrAnalysisException(this.dbId);
+        Table table = database.getTableOrAnalysisException(this.tableId);
+        return Lists.newArrayList(this.policyName, database.getFullName(), table.getName(), this.type.name(),
+                this.filterType.name(), this.wherePredicate.toSql(), this.user.getQualifiedUser(), this.originStmt);
+    }
+
+    /**
+     * Read Table Policy from file.
+     **/
+    public static RowPolicy read(DataInput in) throws IOException {

Review Comment:
   This `read` method should be implement in `Policy` abstract class.
   And you need to add a `RuntimeTypeAdapterFactory` in `GsonUtils` for derived class.
   You can refer to `Resource.java` and its derived class like `SparkResource` and `S3Resource`.
   
   And please add ut for the read/write method of RowPolicy, you can refer to `DropInfoTest.java`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on pull request #9786: There is Policy which is not on TABLE, Policy should be a base class, and use TablePolicy to extends it.

Posted by GitBox <gi...@apache.org>.
morningman commented on PR #9786:
URL: https://github.com/apache/incubator-doris/pull/9786#issuecomment-1140136621

   Currently, Doris has row policy and storage policy.
   But they all apply to the table granularity, actually they can both called table policy.
   So I think it is not suitable to use `TablePolicy` for row policy.
   
   We can just use an abstract class `Policy` as base class, and `RowPolicy` and `StoragePolicy` as derived classes.
   And in `Policy` class, there is `id`, `name`,  `type` fields. and an abstract `matchPolicy()` method which should be implemented in derived class. 
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a diff in pull request #9786: There is Policy which is not on TABLE, Policy should be a base class, and use TablePolicy to extends it.

Posted by GitBox <gi...@apache.org>.
morningman commented on code in PR #9786:
URL: https://github.com/apache/incubator-doris/pull/9786#discussion_r885434959


##########
fe/fe-core/src/main/java/org/apache/doris/policy/RowPolicy.java:
##########
@@ -0,0 +1,174 @@
+// 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.doris.policy;
+
+import org.apache.doris.analysis.CreatePolicyStmt;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.SqlParser;
+import org.apache.doris.analysis.SqlScanner;
+import org.apache.doris.analysis.UserIdentity;
+import org.apache.doris.catalog.Catalog;
+import org.apache.doris.catalog.Database;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.io.Text;
+import org.apache.doris.common.util.SqlParserUtils;
+import org.apache.doris.persist.gson.GsonUtils;
+
+import com.google.common.collect.Lists;
+import com.google.gson.annotations.SerializedName;
+import lombok.Data;
+import org.apache.commons.lang.StringUtils;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.io.StringReader;
+import java.util.List;
+
+/**
+ * Save policy for filtering data.
+ **/
+@Data
+public class RowPolicy extends Policy {
+
+    private static final Logger LOG = LogManager.getLogger(RowPolicy.class);
+
+    /**
+     * Policy bind user.
+     **/
+    @SerializedName(value = "user")
+    private UserIdentity user = null;
+
+    @SerializedName(value = "dbId")
+    private long dbId = -1;
+
+    @SerializedName(value = "tableId")
+    private long tableId = -1;
+
+    /**
+     * PERMISSIVE | RESTRICTIVE, If multiple types exist, the last type prevails.
+     **/
+    @SerializedName(value = "filterType")
+    private FilterType filterType = null;
+
+    /**
+     * Use for Serialization/deserialization.
+     **/
+    @SerializedName(value = "originStmt")
+    private String originStmt;
+
+    private Expr wherePredicate = null;
+
+    public RowPolicy() {}
+
+    /**
+     * Policy for Table. Policy of ROW or others.
+     *
+     * @param type PolicyType
+     * @param policyName policy name
+     * @param dbId database i
+     * @param user username
+     * @param originStmt origin stmt
+     * @param tableId table id
+     * @param filterType filter type
+     * @param wherePredicate where predicate
+     */
+    public RowPolicy(final PolicyTypeEnum type, final String policyName, long dbId,
+                     UserIdentity user, String originStmt, final long tableId,
+                     final FilterType filterType, final Expr wherePredicate) {
+        super(type, policyName);
+        this.user = user;
+        this.dbId = dbId;
+        this.tableId = tableId;
+        this.filterType = filterType;
+        this.originStmt = originStmt;
+        this.wherePredicate = wherePredicate;
+    }
+
+    /**
+     * Use for SHOW POLICY.
+     **/
+    public List<String> getShowInfo() throws AnalysisException {
+        Database database = Catalog.getCurrentCatalog().getDbOrAnalysisException(this.dbId);
+        Table table = database.getTableOrAnalysisException(this.tableId);
+        return Lists.newArrayList(this.policyName, database.getFullName(), table.getName(), this.type.name(),
+                this.filterType.name(), this.wherePredicate.toSql(), this.user.getQualifiedUser(), this.originStmt);
+    }
+
+    /**
+     * Read Table Policy from file.
+     **/
+    public static RowPolicy read(DataInput in) throws IOException {

Review Comment:
   This `read` method should be implement in `Policy` abstract class.
   And you need to add a `RuntimeTypeAdapterFactory` in `GsonUtils` for derived class.
   You can refer to `Resource.java` and its derived class like `SparkResource` and `S3Resource`.
   
   And please and ut for the read/write mothod of RowPolicy, you can refer to `DropInfoTest.java`



##########
fe/fe-common/src/main/java/org/apache/doris/common/FeMetaVersion.java:
##########
@@ -40,8 +40,10 @@ public final class FeMetaVersion {
     public static final int VERSION_109 = 109;
     // For routine load user info
     public static final int VERSION_110 = 110;
-    // NOTE: when increment meta version, should assign the latest version to VERSION_CURRENT
-    public static final int VERSION_CURRENT = VERSION_110;
+    // change Policy to TablePolicy
+    public static final int VERSION_111 = 111;

Review Comment:
   Not used?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] stalary commented on pull request #9786: There is Policy which is not on TABLE, Policy should be a base class, and use TablePolicy to extends it.

Posted by GitBox <gi...@apache.org>.
stalary commented on PR #9786:
URL: https://github.com/apache/incubator-doris/pull/9786#issuecomment-1143650299

   LGTM


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org