You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by GitBox <gi...@apache.org> on 2020/10/06 12:34:17 UTC

[GitHub] [shardingsphere] strongduanmu opened a new pull request #7704: add create function and procedure validator

strongduanmu opened a new pull request #7704:
URL: https://github.com/apache/shardingsphere/pull/7704


   Fixes #6498.
   
   Changes proposed in this pull request:
   - parse create function and procedure g4 file
   - add create function and procedure validator
   


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] tristaZero commented on a change in pull request #7704: add create function and procedure validator

Posted by GitBox <gi...@apache.org>.
tristaZero commented on a change in pull request #7704:
URL: https://github.com/apache/shardingsphere/pull/7704#discussion_r501437608



##########
File path: shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/impl/MySQLDDLVisitor.java
##########
@@ -461,92 +483,256 @@ public ASTNode visitKeyParts_(final KeyParts_Context ctx) {
         }
         return result;
     }
-
+    
     @Override
     public ASTNode visitCreateProcedure(final CreateProcedureContext ctx) {
-        return new MySQLCreateProcedureStatement();
+        MySQLCreateProcedureStatement result = new MySQLCreateProcedureStatement();
+        result.setRoutineBody((RoutineBodySegment) visit(ctx.routineBody()));
+        return result;
     }
-
+    
     @Override
     public ASTNode visitAlterProcedure(final AlterProcedureContext ctx) {
         return new MySQLAlterProcedureStatement();
     }
-
+    
     @Override
     public ASTNode visitDropProcedure(final DropProcedureContext ctx) {
         return new MySQLDropProcedureStatement();
     }
-
+    
     @Override
     public ASTNode visitCreateFunction(final CreateFunctionContext ctx) {
-        return new MySQLCreateFunctionStatement();
+        MySQLCreateFunctionStatement result = new MySQLCreateFunctionStatement();
+        result.setRoutineBody((RoutineBodySegment) visit(ctx.routineBody()));
+        return result;
     }
-
+    
+    @SuppressWarnings("unchecked")
+    @Override
+    public ASTNode visitRoutineBody(final RoutineBodyContext ctx) {
+        RoutineBodySegment result = new RoutineBodySegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex());
+        CollectionValue<ValidStatementSegment> validStatements;
+        if (null != ctx.simpleStatement()) {
+            validStatements = (CollectionValue<ValidStatementSegment>) visit(ctx.simpleStatement());
+        } else {
+            validStatements = (CollectionValue<ValidStatementSegment>) visit(ctx.compoundStatement());
+        }
+        result.getValidStatements().addAll(validStatements.getValue());
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitSimpleStatement(final SimpleStatementContext ctx) {
+        return visit(ctx.validStatement());
+    }
+    
+    @Override
+    public ASTNode visitCompoundStatement(final CompoundStatementContext ctx) {
+        return visit(ctx.beginStatement());
+    }
+    
+    @SuppressWarnings("unchecked")
+    @Override
+    public ASTNode visitBeginStatement(final BeginStatementContext ctx) {
+        CollectionValue<ValidStatementSegment> result = new CollectionValue<>();
+        for (ValidStatementContext each : ctx.validStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(each));
+        }
+        return result;
+    }
+    
+    @SuppressWarnings("unchecked")
+    @Override
+    public ASTNode visitValidStatement(final ValidStatementContext ctx) {
+        CollectionValue<ValidStatementSegment> result = new CollectionValue<>();
+        ValidStatementSegment validStatement = new ValidStatementSegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex());
+        if (null != ctx.createTable()) {
+            validStatement.setCreateTable((MySQLCreateTableStatement) visit(ctx.createTable()));
+            result.getValue().add(validStatement);
+        } else if (null != ctx.alterTable()) {
+            validStatement.setAlterTable((MySQLAlterTableStatement) visit(ctx.alterTable()));
+            result.getValue().add(validStatement);
+        } else if (null != ctx.dropTable()) {
+            validStatement.setDropTable((MySQLDropTableStatement) visit(ctx.dropTable()));
+            result.getValue().add(validStatement);
+        } else if (null != ctx.truncateTable()) {
+            validStatement.setTruncate((MySQLTruncateStatement) visit(ctx.truncateTable()));
+            result.getValue().add(validStatement);
+        }
+        if (null != ctx.validDMLStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(ctx.validDMLStatement()));
+        }
+        if (null != ctx.beginStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(ctx.beginStatement()));
+        }
+        if (null != ctx.flowControlStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(ctx.flowControlStatement()));
+        }
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitValidDMLStatement(final ValidDMLStatementContext ctx) {
+        CollectionValue<ValidStatementSegment> result = new CollectionValue<>();
+        ValidStatementSegment validStatement = new ValidStatementSegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex());
+        if (null != ctx.insert()) {
+            validStatement.setInsert((MySQLInsertStatement) createParseTreeVisitor(ctx.insert().getClass()).visit(ctx.insert()));
+        } else if (null != ctx.replace()) {
+            validStatement.setReplace((MySQLInsertStatement) createParseTreeVisitor(ctx.replace().getClass()).visit(ctx.replace()));
+        } else if (null != ctx.update()) {
+            validStatement.setUpdate((MySQLUpdateStatement) createParseTreeVisitor(ctx.update().getClass()).visit(ctx.update()));
+        } else if (null != ctx.delete()) {
+            validStatement.setDelete((MySQLDeleteStatement) createParseTreeVisitor(ctx.delete().getClass()).visit(ctx.delete()));
+        } else if (null != ctx.select()) {
+            validStatement.setSelect((MySQLSelectStatement) createParseTreeVisitor(ctx.select().getClass()).visit(ctx.select()));
+        }
+        result.getValue().add(validStatement);
+        return result;
+    }
+    
+    @SuppressWarnings("rawtypes")
+    private ParseTreeVisitor createParseTreeVisitor(final Class<? extends ParseTree> parseTreeClass) {

Review comment:
       Yep, I buy that  `MySQLVisitor` is supposed to contain the generic functions and items. But if `MySQLDDLVisitor` needs to use these functions of `MySQLDMLVisitor`,  the relevant ones will become shared one for `DMLVisitor` and `DDLVisitor`, and is possible to move them to `MySQLVisitor`, though which will create a bloated father class.
   
   What's more, as you know, the `currentParameterIndex` makes `MySQLVisitor` not as a simple utility. If you create a new instance of `DMLVisitor` in `DDLVisitor`, I guess there will be an incorrect parameterIndex.
   
   Honestly speaking, I have no better method to handle this issue. :-(
   Actually, for some of the other SQL parser engines like `druid` or `hive`, they put all the visitors and g4 files together, as they do not have a complete and an awful lot of SQL definitions. 😅 
   

##########
File path: shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/ddl/routine/ValidStatementSegment.java
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.shardingsphere.sql.parser.sql.common.segment.ddl.routine;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import lombok.Setter;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.SQLSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.DropTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.TruncateStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DeleteStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.InsertStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.UpdateStatement;
+
+import java.util.Optional;
+
+/**
+ * Valid statement segment.
+ */
+@RequiredArgsConstructor
+@Getter
+@Setter
+public class ValidStatementSegment implements SQLSegment {
+    
+    private final int startIndex;
+    
+    private final int stopIndex;
+    
+    private CreateTableStatement createTable;
+    
+    private AlterTableStatement alterTable;
+    
+    private DropTableStatement dropTable;
+    
+    private TruncateStatement truncate;
+    
+    private InsertStatement insert;
+    
+    private InsertStatement replace;
+    
+    private UpdateStatement update;
+    
+    private DeleteStatement delete;
+    
+    private SelectStatement select;

Review comment:
       >  we need to extract the tables contained in the various statements during parsing. 
   I think the visitor handling could retain the same, even though here its member is `SQLStatement`. 
   We should visit out the concrete SQLStatement, IMO, since that is in line with its g4 definition. However, I wonder whether a generic SQLStatment instead of a list of possible types is adequate to reflect its composition. What's more, the following functions like `getCreateTable()` seem to help give more descriptions of a `SQLStatement` to the outer calls. 
   It seems these public functions rather than its members are more likely to be called,  doesn't it?  That way, do you think this list of member options seems a lit bloated (as you said before:-))?




----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] tristaZero merged pull request #7704: add create function and procedure validator

Posted by GitBox <gi...@apache.org>.
tristaZero merged pull request #7704:
URL: https://github.com/apache/shardingsphere/pull/7704


   


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] strongduanmu commented on a change in pull request #7704: add create function and procedure validator

Posted by GitBox <gi...@apache.org>.
strongduanmu commented on a change in pull request #7704:
URL: https://github.com/apache/shardingsphere/pull/7704#discussion_r501049348



##########
File path: shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingCreateFunctionStatementValidator.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.shardingsphere.sharding.route.engine.validator.impl;
+
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.sharding.route.engine.exception.NoSuchTableException;
+import org.apache.shardingsphere.sharding.route.engine.exception.TableExistsException;
+import org.apache.shardingsphere.sharding.route.engine.validator.ShardingStatementValidator;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
+import org.apache.shardingsphere.sql.parser.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.sql.parser.binder.statement.ddl.CreateFunctionStatementContext;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateFunctionStatement;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Sharding create function statement validator.
+ */
+public final class ShardingCreateFunctionStatementValidator implements ShardingStatementValidator<CreateFunctionStatement> {
+    
+    @Override
+    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<CreateFunctionStatement> sqlStatementContext,
+                            final List<Object> parameters, final ShardingSphereMetaData metaData) {
+        for (SimpleTableSegment each : ((CreateFunctionStatementContext) sqlStatementContext).getExistTables()) {
+            String tableName = each.getTableName().getIdentifier().getValue();
+            if (metaData.getRuleSchemaMetaData().getConfiguredSchemaMetaData().getAllTableNames().contains(tableName)) {
+                throw new ShardingSphereException("Create function statement can not support sharding table '%s'.", tableName);
+            }
+            for (Map.Entry<String, Collection<String>> entry : metaData.getRuleSchemaMetaData().getUnconfiguredSchemaMetaDataMap().entrySet()) {
+                if (!entry.getValue().contains(tableName)) {
+                    throw new NoSuchTableException(entry.getKey(), tableName);
+                }

Review comment:
       > A function will make it intelligible. :-)
   
   @tristaZero I will optimize it. 😀




----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] tristaZero commented on a change in pull request #7704: add create function and procedure validator

Posted by GitBox <gi...@apache.org>.
tristaZero commented on a change in pull request #7704:
URL: https://github.com/apache/shardingsphere/pull/7704#discussion_r501432718



##########
File path: shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/ddl/routine/ValidStatementSegment.java
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.shardingsphere.sql.parser.sql.common.segment.ddl.routine;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import lombok.Setter;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.SQLSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.DropTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.TruncateStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DeleteStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.InsertStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.UpdateStatement;
+
+import java.util.Optional;
+
+/**
+ * Valid statement segment.
+ */
+@RequiredArgsConstructor
+@Getter
+@Setter
+public class ValidStatementSegment implements SQLSegment {
+    
+    private final int startIndex;
+    
+    private final int stopIndex;
+    
+    private CreateTableStatement createTable;
+    
+    private AlterTableStatement alterTable;
+    
+    private DropTableStatement dropTable;
+    
+    private TruncateStatement truncate;
+    
+    private InsertStatement insert;
+    
+    private InsertStatement replace;
+    
+    private UpdateStatement update;
+    
+    private DeleteStatement delete;
+    
+    private SelectStatement select;

Review comment:
       >  we need to extract the tables contained in the various statements during parsing. 
   
   I think the visitor handling could retain the same, even though here its member is `SQLStatement`. 
   We should visit out the concrete SQLStatement, IMO, since that is in line with its g4 definition. However, I wonder whether a generic SQLStatment instead of a list of possible types is adequate to reflect its composition. What's more, the following functions like `getCreateTable()` seem to help give more descriptions of a `SQLStatement` to the outer calls. 
   It seems these public functions rather than its members are more likely to be called,  doesn't it?  That way, do you think this list of member options seems a lit bloated (as you said before:-))?




----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] strongduanmu commented on a change in pull request #7704: add create function and procedure validator

Posted by GitBox <gi...@apache.org>.
strongduanmu commented on a change in pull request #7704:
URL: https://github.com/apache/shardingsphere/pull/7704#discussion_r501677521



##########
File path: shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/impl/MySQLDDLVisitor.java
##########
@@ -461,92 +483,256 @@ public ASTNode visitKeyParts_(final KeyParts_Context ctx) {
         }
         return result;
     }
-
+    
     @Override
     public ASTNode visitCreateProcedure(final CreateProcedureContext ctx) {
-        return new MySQLCreateProcedureStatement();
+        MySQLCreateProcedureStatement result = new MySQLCreateProcedureStatement();
+        result.setRoutineBody((RoutineBodySegment) visit(ctx.routineBody()));
+        return result;
     }
-
+    
     @Override
     public ASTNode visitAlterProcedure(final AlterProcedureContext ctx) {
         return new MySQLAlterProcedureStatement();
     }
-
+    
     @Override
     public ASTNode visitDropProcedure(final DropProcedureContext ctx) {
         return new MySQLDropProcedureStatement();
     }
-
+    
     @Override
     public ASTNode visitCreateFunction(final CreateFunctionContext ctx) {
-        return new MySQLCreateFunctionStatement();
+        MySQLCreateFunctionStatement result = new MySQLCreateFunctionStatement();
+        result.setRoutineBody((RoutineBodySegment) visit(ctx.routineBody()));
+        return result;
     }
-
+    
+    @SuppressWarnings("unchecked")
+    @Override
+    public ASTNode visitRoutineBody(final RoutineBodyContext ctx) {
+        RoutineBodySegment result = new RoutineBodySegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex());
+        CollectionValue<ValidStatementSegment> validStatements;
+        if (null != ctx.simpleStatement()) {
+            validStatements = (CollectionValue<ValidStatementSegment>) visit(ctx.simpleStatement());
+        } else {
+            validStatements = (CollectionValue<ValidStatementSegment>) visit(ctx.compoundStatement());
+        }
+        result.getValidStatements().addAll(validStatements.getValue());
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitSimpleStatement(final SimpleStatementContext ctx) {
+        return visit(ctx.validStatement());
+    }
+    
+    @Override
+    public ASTNode visitCompoundStatement(final CompoundStatementContext ctx) {
+        return visit(ctx.beginStatement());
+    }
+    
+    @SuppressWarnings("unchecked")
+    @Override
+    public ASTNode visitBeginStatement(final BeginStatementContext ctx) {
+        CollectionValue<ValidStatementSegment> result = new CollectionValue<>();
+        for (ValidStatementContext each : ctx.validStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(each));
+        }
+        return result;
+    }
+    
+    @SuppressWarnings("unchecked")
+    @Override
+    public ASTNode visitValidStatement(final ValidStatementContext ctx) {
+        CollectionValue<ValidStatementSegment> result = new CollectionValue<>();
+        ValidStatementSegment validStatement = new ValidStatementSegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex());
+        if (null != ctx.createTable()) {
+            validStatement.setCreateTable((MySQLCreateTableStatement) visit(ctx.createTable()));
+            result.getValue().add(validStatement);
+        } else if (null != ctx.alterTable()) {
+            validStatement.setAlterTable((MySQLAlterTableStatement) visit(ctx.alterTable()));
+            result.getValue().add(validStatement);
+        } else if (null != ctx.dropTable()) {
+            validStatement.setDropTable((MySQLDropTableStatement) visit(ctx.dropTable()));
+            result.getValue().add(validStatement);
+        } else if (null != ctx.truncateTable()) {
+            validStatement.setTruncate((MySQLTruncateStatement) visit(ctx.truncateTable()));
+            result.getValue().add(validStatement);
+        }
+        if (null != ctx.validDMLStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(ctx.validDMLStatement()));
+        }
+        if (null != ctx.beginStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(ctx.beginStatement()));
+        }
+        if (null != ctx.flowControlStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(ctx.flowControlStatement()));
+        }
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitValidDMLStatement(final ValidDMLStatementContext ctx) {
+        CollectionValue<ValidStatementSegment> result = new CollectionValue<>();
+        ValidStatementSegment validStatement = new ValidStatementSegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex());
+        if (null != ctx.insert()) {
+            validStatement.setInsert((MySQLInsertStatement) createParseTreeVisitor(ctx.insert().getClass()).visit(ctx.insert()));
+        } else if (null != ctx.replace()) {
+            validStatement.setReplace((MySQLInsertStatement) createParseTreeVisitor(ctx.replace().getClass()).visit(ctx.replace()));
+        } else if (null != ctx.update()) {
+            validStatement.setUpdate((MySQLUpdateStatement) createParseTreeVisitor(ctx.update().getClass()).visit(ctx.update()));
+        } else if (null != ctx.delete()) {
+            validStatement.setDelete((MySQLDeleteStatement) createParseTreeVisitor(ctx.delete().getClass()).visit(ctx.delete()));
+        } else if (null != ctx.select()) {
+            validStatement.setSelect((MySQLSelectStatement) createParseTreeVisitor(ctx.select().getClass()).visit(ctx.select()));
+        }
+        result.getValue().add(validStatement);
+        return result;
+    }
+    
+    @SuppressWarnings("rawtypes")
+    private ParseTreeVisitor createParseTreeVisitor(final Class<? extends ParseTree> parseTreeClass) {

Review comment:
       @tristaZero Thank you very much for your advice, the setting of parameter `currentParameterIndex` is really a problem, I will optimize it by migrating the common function from the MySQLDMLVisitor to the MySQLVisitor. 😀

##########
File path: shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/ddl/routine/ValidStatementSegment.java
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.shardingsphere.sql.parser.sql.common.segment.ddl.routine;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import lombok.Setter;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.SQLSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.DropTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.TruncateStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DeleteStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.InsertStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.UpdateStatement;
+
+import java.util.Optional;
+
+/**
+ * Valid statement segment.
+ */
+@RequiredArgsConstructor
+@Getter
+@Setter
+public class ValidStatementSegment implements SQLSegment {
+    
+    private final int startIndex;
+    
+    private final int stopIndex;
+    
+    private CreateTableStatement createTable;
+    
+    private AlterTableStatement alterTable;
+    
+    private DropTableStatement dropTable;
+    
+    private TruncateStatement truncate;
+    
+    private InsertStatement insert;
+    
+    private InsertStatement replace;
+    
+    private UpdateStatement update;
+    
+    private DeleteStatement delete;
+    
+    private SelectStatement select;

Review comment:
       @tristaZero I like your suggestion that using `SQLStatement` to receive different types of statements, and then use public methods to provide external calls. 👍 




----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] strongduanmu commented on a change in pull request #7704: add create function and procedure validator

Posted by GitBox <gi...@apache.org>.
strongduanmu commented on a change in pull request #7704:
URL: https://github.com/apache/shardingsphere/pull/7704#discussion_r501677521



##########
File path: shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/impl/MySQLDDLVisitor.java
##########
@@ -461,92 +483,256 @@ public ASTNode visitKeyParts_(final KeyParts_Context ctx) {
         }
         return result;
     }
-
+    
     @Override
     public ASTNode visitCreateProcedure(final CreateProcedureContext ctx) {
-        return new MySQLCreateProcedureStatement();
+        MySQLCreateProcedureStatement result = new MySQLCreateProcedureStatement();
+        result.setRoutineBody((RoutineBodySegment) visit(ctx.routineBody()));
+        return result;
     }
-
+    
     @Override
     public ASTNode visitAlterProcedure(final AlterProcedureContext ctx) {
         return new MySQLAlterProcedureStatement();
     }
-
+    
     @Override
     public ASTNode visitDropProcedure(final DropProcedureContext ctx) {
         return new MySQLDropProcedureStatement();
     }
-
+    
     @Override
     public ASTNode visitCreateFunction(final CreateFunctionContext ctx) {
-        return new MySQLCreateFunctionStatement();
+        MySQLCreateFunctionStatement result = new MySQLCreateFunctionStatement();
+        result.setRoutineBody((RoutineBodySegment) visit(ctx.routineBody()));
+        return result;
     }
-
+    
+    @SuppressWarnings("unchecked")
+    @Override
+    public ASTNode visitRoutineBody(final RoutineBodyContext ctx) {
+        RoutineBodySegment result = new RoutineBodySegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex());
+        CollectionValue<ValidStatementSegment> validStatements;
+        if (null != ctx.simpleStatement()) {
+            validStatements = (CollectionValue<ValidStatementSegment>) visit(ctx.simpleStatement());
+        } else {
+            validStatements = (CollectionValue<ValidStatementSegment>) visit(ctx.compoundStatement());
+        }
+        result.getValidStatements().addAll(validStatements.getValue());
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitSimpleStatement(final SimpleStatementContext ctx) {
+        return visit(ctx.validStatement());
+    }
+    
+    @Override
+    public ASTNode visitCompoundStatement(final CompoundStatementContext ctx) {
+        return visit(ctx.beginStatement());
+    }
+    
+    @SuppressWarnings("unchecked")
+    @Override
+    public ASTNode visitBeginStatement(final BeginStatementContext ctx) {
+        CollectionValue<ValidStatementSegment> result = new CollectionValue<>();
+        for (ValidStatementContext each : ctx.validStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(each));
+        }
+        return result;
+    }
+    
+    @SuppressWarnings("unchecked")
+    @Override
+    public ASTNode visitValidStatement(final ValidStatementContext ctx) {
+        CollectionValue<ValidStatementSegment> result = new CollectionValue<>();
+        ValidStatementSegment validStatement = new ValidStatementSegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex());
+        if (null != ctx.createTable()) {
+            validStatement.setCreateTable((MySQLCreateTableStatement) visit(ctx.createTable()));
+            result.getValue().add(validStatement);
+        } else if (null != ctx.alterTable()) {
+            validStatement.setAlterTable((MySQLAlterTableStatement) visit(ctx.alterTable()));
+            result.getValue().add(validStatement);
+        } else if (null != ctx.dropTable()) {
+            validStatement.setDropTable((MySQLDropTableStatement) visit(ctx.dropTable()));
+            result.getValue().add(validStatement);
+        } else if (null != ctx.truncateTable()) {
+            validStatement.setTruncate((MySQLTruncateStatement) visit(ctx.truncateTable()));
+            result.getValue().add(validStatement);
+        }
+        if (null != ctx.validDMLStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(ctx.validDMLStatement()));
+        }
+        if (null != ctx.beginStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(ctx.beginStatement()));
+        }
+        if (null != ctx.flowControlStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(ctx.flowControlStatement()));
+        }
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitValidDMLStatement(final ValidDMLStatementContext ctx) {
+        CollectionValue<ValidStatementSegment> result = new CollectionValue<>();
+        ValidStatementSegment validStatement = new ValidStatementSegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex());
+        if (null != ctx.insert()) {
+            validStatement.setInsert((MySQLInsertStatement) createParseTreeVisitor(ctx.insert().getClass()).visit(ctx.insert()));
+        } else if (null != ctx.replace()) {
+            validStatement.setReplace((MySQLInsertStatement) createParseTreeVisitor(ctx.replace().getClass()).visit(ctx.replace()));
+        } else if (null != ctx.update()) {
+            validStatement.setUpdate((MySQLUpdateStatement) createParseTreeVisitor(ctx.update().getClass()).visit(ctx.update()));
+        } else if (null != ctx.delete()) {
+            validStatement.setDelete((MySQLDeleteStatement) createParseTreeVisitor(ctx.delete().getClass()).visit(ctx.delete()));
+        } else if (null != ctx.select()) {
+            validStatement.setSelect((MySQLSelectStatement) createParseTreeVisitor(ctx.select().getClass()).visit(ctx.select()));
+        }
+        result.getValue().add(validStatement);
+        return result;
+    }
+    
+    @SuppressWarnings("rawtypes")
+    private ParseTreeVisitor createParseTreeVisitor(final Class<? extends ParseTree> parseTreeClass) {

Review comment:
       @tristaZero Thank you very much for your advice, the setting of parameter `currentParameterIndex` is really a problem, I will optimize it by migrating the common function from the MySQLDMLVisitor to the MySQLVisitor. 😀




----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] tristaZero merged pull request #7704: add create function and procedure validator

Posted by GitBox <gi...@apache.org>.
tristaZero merged pull request #7704:
URL: https://github.com/apache/shardingsphere/pull/7704


   


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] tristaZero commented on a change in pull request #7704: add create function and procedure validator

Posted by GitBox <gi...@apache.org>.
tristaZero commented on a change in pull request #7704:
URL: https://github.com/apache/shardingsphere/pull/7704#discussion_r500958956



##########
File path: shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/ddl/DropIndexStatementHandlerTest.java
##########
@@ -28,7 +28,7 @@
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-public class DropIndexStatementHandlerTest {
+public final class DropIndexStatementHandlerTest {

Review comment:
       Carefully examined. 👍 

##########
File path: shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/antlr4/imports/mysql/DDLStatement.g4
##########
@@ -625,10 +625,13 @@ compoundStatement
     ;
 
 validStatement
-    : (createTable | alterTable | dropTable | truncateTable 
-    | insert | replace | update | delete | select | call
+    : (createTable | alterTable | dropTable | truncateTable | validDMLStatement
     | setVariable | beginStatement | declareStatement | flowControlStatement | cursorStatement | conditionHandlingStatement) SEMI_?
     ;
+    
+validDMLStatement

Review comment:
       I suppose `validDMLStatement` is just for your convenience.  If  `MySQLVisitor` contains the content of `MySQLDMLStatement`, this rule is unneeded, right? As there is no such concept in the SQL grammar of MySQL, I guess.

##########
File path: shardingsphere-sql-parser/shardingsphere-sql-parser-binder/src/main/java/org/apache/shardingsphere/sql/parser/binder/statement/ddl/CreateFunctionStatementContext.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.shardingsphere.sql.parser.binder.statement.ddl;
+
+import lombok.Getter;
+import org.apache.shardingsphere.sql.parser.binder.statement.CommonSQLStatementContext;
+import org.apache.shardingsphere.sql.parser.sql.common.extractor.TableExtractor;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.routine.RoutineBodySegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateFunctionStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.handler.ddl.CreateFunctionStatementHandler;
+
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.Optional;
+
+/**
+ * Create function statement context.
+ */
+@Getter
+public final class CreateFunctionStatementContext extends CommonSQLStatementContext<CreateFunctionStatement> {
+    
+    private final Collection<SimpleTableSegment> existTables = new LinkedList<>();

Review comment:
       From your changes,  I suppose two of there are no more than checking,  aren't they?
   That way, why do not we move `routineBody -> existTables.addAll(extractor.extractExistTableFromRoutineBody(routineBody)` to `validator`?

##########
File path: shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingCreateFunctionStatementValidator.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.shardingsphere.sharding.route.engine.validator.impl;
+
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.sharding.route.engine.exception.NoSuchTableException;
+import org.apache.shardingsphere.sharding.route.engine.exception.TableExistsException;
+import org.apache.shardingsphere.sharding.route.engine.validator.ShardingStatementValidator;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
+import org.apache.shardingsphere.sql.parser.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.sql.parser.binder.statement.ddl.CreateFunctionStatementContext;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateFunctionStatement;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Sharding create function statement validator.
+ */
+public final class ShardingCreateFunctionStatementValidator implements ShardingStatementValidator<CreateFunctionStatement> {
+    
+    @Override
+    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<CreateFunctionStatement> sqlStatementContext,
+                            final List<Object> parameters, final ShardingSphereMetaData metaData) {
+        for (SimpleTableSegment each : ((CreateFunctionStatementContext) sqlStatementContext).getExistTables()) {
+            String tableName = each.getTableName().getIdentifier().getValue();
+            if (metaData.getRuleSchemaMetaData().getConfiguredSchemaMetaData().getAllTableNames().contains(tableName)) {
+                throw new ShardingSphereException("Create function statement can not support sharding table '%s'.", tableName);
+            }
+            for (Map.Entry<String, Collection<String>> entry : metaData.getRuleSchemaMetaData().getUnconfiguredSchemaMetaDataMap().entrySet()) {
+                if (!entry.getValue().contains(tableName)) {
+                    throw new NoSuchTableException(entry.getKey(), tableName);
+                }

Review comment:
       A function will make it intelligible. :-)

##########
File path: shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingCreateFunctionStatementValidator.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.shardingsphere.sharding.route.engine.validator.impl;
+
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.sharding.route.engine.exception.NoSuchTableException;
+import org.apache.shardingsphere.sharding.route.engine.exception.TableExistsException;
+import org.apache.shardingsphere.sharding.route.engine.validator.ShardingStatementValidator;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
+import org.apache.shardingsphere.sql.parser.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.sql.parser.binder.statement.ddl.CreateFunctionStatementContext;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateFunctionStatement;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Sharding create function statement validator.
+ */
+public final class ShardingCreateFunctionStatementValidator implements ShardingStatementValidator<CreateFunctionStatement> {
+    
+    @Override
+    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<CreateFunctionStatement> sqlStatementContext,
+                            final List<Object> parameters, final ShardingSphereMetaData metaData) {
+        for (SimpleTableSegment each : ((CreateFunctionStatementContext) sqlStatementContext).getExistTables()) {
+            String tableName = each.getTableName().getIdentifier().getValue();
+            if (metaData.getRuleSchemaMetaData().getConfiguredSchemaMetaData().getAllTableNames().contains(tableName)) {
+                throw new ShardingSphereException("Create function statement can not support sharding table '%s'.", tableName);
+            }
+            for (Map.Entry<String, Collection<String>> entry : metaData.getRuleSchemaMetaData().getUnconfiguredSchemaMetaDataMap().entrySet()) {
+                if (!entry.getValue().contains(tableName)) {
+                    throw new NoSuchTableException(entry.getKey(), tableName);
+                }
+            }
+        }
+        for (SimpleTableSegment each : ((CreateFunctionStatementContext) sqlStatementContext).getNotExistTables()) {
+            String tableName = each.getTableName().getIdentifier().getValue();
+            if (metaData.getRuleSchemaMetaData().getAllTableNames().contains(tableName)) {
+                throw new TableExistsException(tableName);
+            }
+        }
+    }

Review comment:
       Same. But it is not a big problem.  Just an optional tip.

##########
File path: shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/impl/MySQLDDLVisitor.java
##########
@@ -461,92 +483,256 @@ public ASTNode visitKeyParts_(final KeyParts_Context ctx) {
         }
         return result;
     }
-
+    
     @Override
     public ASTNode visitCreateProcedure(final CreateProcedureContext ctx) {
-        return new MySQLCreateProcedureStatement();
+        MySQLCreateProcedureStatement result = new MySQLCreateProcedureStatement();
+        result.setRoutineBody((RoutineBodySegment) visit(ctx.routineBody()));
+        return result;
     }
-
+    
     @Override
     public ASTNode visitAlterProcedure(final AlterProcedureContext ctx) {
         return new MySQLAlterProcedureStatement();
     }
-
+    
     @Override
     public ASTNode visitDropProcedure(final DropProcedureContext ctx) {
         return new MySQLDropProcedureStatement();
     }
-
+    
     @Override
     public ASTNode visitCreateFunction(final CreateFunctionContext ctx) {
-        return new MySQLCreateFunctionStatement();
+        MySQLCreateFunctionStatement result = new MySQLCreateFunctionStatement();
+        result.setRoutineBody((RoutineBodySegment) visit(ctx.routineBody()));
+        return result;
     }
-
+    
+    @SuppressWarnings("unchecked")
+    @Override
+    public ASTNode visitRoutineBody(final RoutineBodyContext ctx) {
+        RoutineBodySegment result = new RoutineBodySegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex());
+        CollectionValue<ValidStatementSegment> validStatements;
+        if (null != ctx.simpleStatement()) {
+            validStatements = (CollectionValue<ValidStatementSegment>) visit(ctx.simpleStatement());
+        } else {
+            validStatements = (CollectionValue<ValidStatementSegment>) visit(ctx.compoundStatement());
+        }
+        result.getValidStatements().addAll(validStatements.getValue());
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitSimpleStatement(final SimpleStatementContext ctx) {
+        return visit(ctx.validStatement());
+    }
+    
+    @Override
+    public ASTNode visitCompoundStatement(final CompoundStatementContext ctx) {
+        return visit(ctx.beginStatement());
+    }
+    
+    @SuppressWarnings("unchecked")
+    @Override
+    public ASTNode visitBeginStatement(final BeginStatementContext ctx) {
+        CollectionValue<ValidStatementSegment> result = new CollectionValue<>();
+        for (ValidStatementContext each : ctx.validStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(each));
+        }
+        return result;
+    }
+    
+    @SuppressWarnings("unchecked")
+    @Override
+    public ASTNode visitValidStatement(final ValidStatementContext ctx) {
+        CollectionValue<ValidStatementSegment> result = new CollectionValue<>();
+        ValidStatementSegment validStatement = new ValidStatementSegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex());
+        if (null != ctx.createTable()) {
+            validStatement.setCreateTable((MySQLCreateTableStatement) visit(ctx.createTable()));
+            result.getValue().add(validStatement);
+        } else if (null != ctx.alterTable()) {
+            validStatement.setAlterTable((MySQLAlterTableStatement) visit(ctx.alterTable()));
+            result.getValue().add(validStatement);
+        } else if (null != ctx.dropTable()) {
+            validStatement.setDropTable((MySQLDropTableStatement) visit(ctx.dropTable()));
+            result.getValue().add(validStatement);
+        } else if (null != ctx.truncateTable()) {
+            validStatement.setTruncate((MySQLTruncateStatement) visit(ctx.truncateTable()));
+            result.getValue().add(validStatement);
+        }
+        if (null != ctx.validDMLStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(ctx.validDMLStatement()));
+        }
+        if (null != ctx.beginStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(ctx.beginStatement()));
+        }
+        if (null != ctx.flowControlStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(ctx.flowControlStatement()));
+        }
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitValidDMLStatement(final ValidDMLStatementContext ctx) {
+        CollectionValue<ValidStatementSegment> result = new CollectionValue<>();
+        ValidStatementSegment validStatement = new ValidStatementSegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex());
+        if (null != ctx.insert()) {
+            validStatement.setInsert((MySQLInsertStatement) createParseTreeVisitor(ctx.insert().getClass()).visit(ctx.insert()));
+        } else if (null != ctx.replace()) {
+            validStatement.setReplace((MySQLInsertStatement) createParseTreeVisitor(ctx.replace().getClass()).visit(ctx.replace()));
+        } else if (null != ctx.update()) {
+            validStatement.setUpdate((MySQLUpdateStatement) createParseTreeVisitor(ctx.update().getClass()).visit(ctx.update()));
+        } else if (null != ctx.delete()) {
+            validStatement.setDelete((MySQLDeleteStatement) createParseTreeVisitor(ctx.delete().getClass()).visit(ctx.delete()));
+        } else if (null != ctx.select()) {
+            validStatement.setSelect((MySQLSelectStatement) createParseTreeVisitor(ctx.select().getClass()).visit(ctx.select()));
+        }
+        result.getValue().add(validStatement);
+        return result;
+    }
+    
+    @SuppressWarnings("rawtypes")
+    private ParseTreeVisitor createParseTreeVisitor(final Class<? extends ParseTree> parseTreeClass) {

Review comment:
       if `MySQLDMLVisitor` is necessary, do you think moving the content of `MySQLDMLVisitor` to `MySQLVisitor` will be better than `createParseTreeVisitor`?

##########
File path: shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/ddl/routine/ValidStatementSegment.java
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.shardingsphere.sql.parser.sql.common.segment.ddl.routine;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import lombok.Setter;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.SQLSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.DropTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.TruncateStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DeleteStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.InsertStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.UpdateStatement;
+
+import java.util.Optional;
+
+/**
+ * Valid statement segment.
+ */
+@RequiredArgsConstructor
+@Getter
+@Setter
+public class ValidStatementSegment implements SQLSegment {
+    
+    private final int startIndex;
+    
+    private final int stopIndex;
+    
+    private CreateTableStatement createTable;
+    
+    private AlterTableStatement alterTable;
+    
+    private DropTableStatement dropTable;
+    
+    private TruncateStatement truncate;
+    
+    private InsertStatement insert;
+    
+    private InsertStatement replace;
+    
+    private UpdateStatement update;
+    
+    private DeleteStatement delete;
+    
+    private SelectStatement select;

Review comment:
       There only will be one of them at a time, right?  Plus, currently, do we indeed care about the concrete class type of this object?  I mean, do you think it is possible to merge them as a `SQLStatement`?




----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] strongduanmu commented on a change in pull request #7704: add create function and procedure validator

Posted by GitBox <gi...@apache.org>.
strongduanmu commented on a change in pull request #7704:
URL: https://github.com/apache/shardingsphere/pull/7704#discussion_r501058138



##########
File path: shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/ddl/routine/ValidStatementSegment.java
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.shardingsphere.sql.parser.sql.common.segment.ddl.routine;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import lombok.Setter;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.SQLSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.DropTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.TruncateStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DeleteStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.InsertStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.UpdateStatement;
+
+import java.util.Optional;
+
+/**
+ * Valid statement segment.
+ */
+@RequiredArgsConstructor
+@Getter
+@Setter
+public class ValidStatementSegment implements SQLSegment {
+    
+    private final int startIndex;
+    
+    private final int stopIndex;
+    
+    private CreateTableStatement createTable;
+    
+    private AlterTableStatement alterTable;
+    
+    private DropTableStatement dropTable;
+    
+    private TruncateStatement truncate;
+    
+    private InsertStatement insert;
+    
+    private InsertStatement replace;
+    
+    private UpdateStatement update;
+    
+    private DeleteStatement delete;
+    
+    private SelectStatement select;

Review comment:
       > There only will be one of them at a time, right? Plus, currently, do we indeed care about the concrete class type of this object? I mean, do you think it is possible to merge them as a `SQLStatement`?
   
   @tristaZero For now, we are only concerned with the tables contained in these statements, which are then used for validation, and perhaps `sharding routing and rewriting` will be supported in the future.
   
   If we combine these statements into a single statement, we need to extract the tables contained in the various statements during parsing. This will add a lot of logic to the visitor. Is that ok? 🤔




----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] strongduanmu commented on a change in pull request #7704: add create function and procedure validator

Posted by GitBox <gi...@apache.org>.
strongduanmu commented on a change in pull request #7704:
URL: https://github.com/apache/shardingsphere/pull/7704#discussion_r501051166



##########
File path: shardingsphere-sql-parser/shardingsphere-sql-parser-binder/src/main/java/org/apache/shardingsphere/sql/parser/binder/statement/ddl/CreateFunctionStatementContext.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.shardingsphere.sql.parser.binder.statement.ddl;
+
+import lombok.Getter;
+import org.apache.shardingsphere.sql.parser.binder.statement.CommonSQLStatementContext;
+import org.apache.shardingsphere.sql.parser.sql.common.extractor.TableExtractor;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.routine.RoutineBodySegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateFunctionStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.handler.ddl.CreateFunctionStatementHandler;
+
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.Optional;
+
+/**
+ * Create function statement context.
+ */
+@Getter
+public final class CreateFunctionStatementContext extends CommonSQLStatementContext<CreateFunctionStatement> {
+    
+    private final Collection<SimpleTableSegment> existTables = new LinkedList<>();

Review comment:
       > From your changes, I suppose two of there are no more than checking, aren't they?
   > That way, why do not we move `routineBody -> existTables.addAll(extractor.extractExistTableFromRoutineBody(routineBody)` to `validator`?
   
   @tristaZero Yes, this logic can be extracted into the Validator.




----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] tristaZero commented on a change in pull request #7704: add create function and procedure validator

Posted by GitBox <gi...@apache.org>.
tristaZero commented on a change in pull request #7704:
URL: https://github.com/apache/shardingsphere/pull/7704#discussion_r501432718



##########
File path: shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/ddl/routine/ValidStatementSegment.java
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.shardingsphere.sql.parser.sql.common.segment.ddl.routine;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import lombok.Setter;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.SQLSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.DropTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.TruncateStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DeleteStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.InsertStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.UpdateStatement;
+
+import java.util.Optional;
+
+/**
+ * Valid statement segment.
+ */
+@RequiredArgsConstructor
+@Getter
+@Setter
+public class ValidStatementSegment implements SQLSegment {
+    
+    private final int startIndex;
+    
+    private final int stopIndex;
+    
+    private CreateTableStatement createTable;
+    
+    private AlterTableStatement alterTable;
+    
+    private DropTableStatement dropTable;
+    
+    private TruncateStatement truncate;
+    
+    private InsertStatement insert;
+    
+    private InsertStatement replace;
+    
+    private UpdateStatement update;
+    
+    private DeleteStatement delete;
+    
+    private SelectStatement select;

Review comment:
       >  we need to extract the tables contained in the various statements during parsing. 
   
   I think the visitor handling could retain the same, even though here its member is `SQLStatement`. 
   We should visit out the concrete SQLStatement, IMO, since that is in line with its g4 definition. However, I wonder whether a generic SQLStatment instead of a list of possible types is adequate to reflect its composition. What's more, the following functions like `getCreateTable()` seem to help give more descriptions of a `SQLStatement` to the outer calls. 
   It seems these public functions rather than its members are more likely to be called,  doesn't it?  That way, do you think this list of member options seems a lit bloated (as you said before:-))?




----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] strongduanmu commented on a change in pull request #7704: add create function and procedure validator

Posted by GitBox <gi...@apache.org>.
strongduanmu commented on a change in pull request #7704:
URL: https://github.com/apache/shardingsphere/pull/7704#discussion_r501685166



##########
File path: shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/ddl/routine/ValidStatementSegment.java
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.shardingsphere.sql.parser.sql.common.segment.ddl.routine;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import lombok.Setter;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.SQLSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.DropTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.TruncateStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DeleteStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.InsertStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.UpdateStatement;
+
+import java.util.Optional;
+
+/**
+ * Valid statement segment.
+ */
+@RequiredArgsConstructor
+@Getter
+@Setter
+public class ValidStatementSegment implements SQLSegment {
+    
+    private final int startIndex;
+    
+    private final int stopIndex;
+    
+    private CreateTableStatement createTable;
+    
+    private AlterTableStatement alterTable;
+    
+    private DropTableStatement dropTable;
+    
+    private TruncateStatement truncate;
+    
+    private InsertStatement insert;
+    
+    private InsertStatement replace;
+    
+    private UpdateStatement update;
+    
+    private DeleteStatement delete;
+    
+    private SelectStatement select;

Review comment:
       @tristaZero I like your suggestion that using `SQLStatement` to receive different types of statements, and then use public methods to provide external calls. 👍 




----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] strongduanmu commented on a change in pull request #7704: add create function and procedure validator

Posted by GitBox <gi...@apache.org>.
strongduanmu commented on a change in pull request #7704:
URL: https://github.com/apache/shardingsphere/pull/7704#discussion_r501033363



##########
File path: shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/impl/MySQLDDLVisitor.java
##########
@@ -461,92 +483,256 @@ public ASTNode visitKeyParts_(final KeyParts_Context ctx) {
         }
         return result;
     }
-
+    
     @Override
     public ASTNode visitCreateProcedure(final CreateProcedureContext ctx) {
-        return new MySQLCreateProcedureStatement();
+        MySQLCreateProcedureStatement result = new MySQLCreateProcedureStatement();
+        result.setRoutineBody((RoutineBodySegment) visit(ctx.routineBody()));
+        return result;
     }
-
+    
     @Override
     public ASTNode visitAlterProcedure(final AlterProcedureContext ctx) {
         return new MySQLAlterProcedureStatement();
     }
-
+    
     @Override
     public ASTNode visitDropProcedure(final DropProcedureContext ctx) {
         return new MySQLDropProcedureStatement();
     }
-
+    
     @Override
     public ASTNode visitCreateFunction(final CreateFunctionContext ctx) {
-        return new MySQLCreateFunctionStatement();
+        MySQLCreateFunctionStatement result = new MySQLCreateFunctionStatement();
+        result.setRoutineBody((RoutineBodySegment) visit(ctx.routineBody()));
+        return result;
     }
-
+    
+    @SuppressWarnings("unchecked")
+    @Override
+    public ASTNode visitRoutineBody(final RoutineBodyContext ctx) {
+        RoutineBodySegment result = new RoutineBodySegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex());
+        CollectionValue<ValidStatementSegment> validStatements;
+        if (null != ctx.simpleStatement()) {
+            validStatements = (CollectionValue<ValidStatementSegment>) visit(ctx.simpleStatement());
+        } else {
+            validStatements = (CollectionValue<ValidStatementSegment>) visit(ctx.compoundStatement());
+        }
+        result.getValidStatements().addAll(validStatements.getValue());
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitSimpleStatement(final SimpleStatementContext ctx) {
+        return visit(ctx.validStatement());
+    }
+    
+    @Override
+    public ASTNode visitCompoundStatement(final CompoundStatementContext ctx) {
+        return visit(ctx.beginStatement());
+    }
+    
+    @SuppressWarnings("unchecked")
+    @Override
+    public ASTNode visitBeginStatement(final BeginStatementContext ctx) {
+        CollectionValue<ValidStatementSegment> result = new CollectionValue<>();
+        for (ValidStatementContext each : ctx.validStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(each));
+        }
+        return result;
+    }
+    
+    @SuppressWarnings("unchecked")
+    @Override
+    public ASTNode visitValidStatement(final ValidStatementContext ctx) {
+        CollectionValue<ValidStatementSegment> result = new CollectionValue<>();
+        ValidStatementSegment validStatement = new ValidStatementSegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex());
+        if (null != ctx.createTable()) {
+            validStatement.setCreateTable((MySQLCreateTableStatement) visit(ctx.createTable()));
+            result.getValue().add(validStatement);
+        } else if (null != ctx.alterTable()) {
+            validStatement.setAlterTable((MySQLAlterTableStatement) visit(ctx.alterTable()));
+            result.getValue().add(validStatement);
+        } else if (null != ctx.dropTable()) {
+            validStatement.setDropTable((MySQLDropTableStatement) visit(ctx.dropTable()));
+            result.getValue().add(validStatement);
+        } else if (null != ctx.truncateTable()) {
+            validStatement.setTruncate((MySQLTruncateStatement) visit(ctx.truncateTable()));
+            result.getValue().add(validStatement);
+        }
+        if (null != ctx.validDMLStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(ctx.validDMLStatement()));
+        }
+        if (null != ctx.beginStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(ctx.beginStatement()));
+        }
+        if (null != ctx.flowControlStatement()) {
+            result.combine((CollectionValue<ValidStatementSegment>) visit(ctx.flowControlStatement()));
+        }
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitValidDMLStatement(final ValidDMLStatementContext ctx) {
+        CollectionValue<ValidStatementSegment> result = new CollectionValue<>();
+        ValidStatementSegment validStatement = new ValidStatementSegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex());
+        if (null != ctx.insert()) {
+            validStatement.setInsert((MySQLInsertStatement) createParseTreeVisitor(ctx.insert().getClass()).visit(ctx.insert()));
+        } else if (null != ctx.replace()) {
+            validStatement.setReplace((MySQLInsertStatement) createParseTreeVisitor(ctx.replace().getClass()).visit(ctx.replace()));
+        } else if (null != ctx.update()) {
+            validStatement.setUpdate((MySQLUpdateStatement) createParseTreeVisitor(ctx.update().getClass()).visit(ctx.update()));
+        } else if (null != ctx.delete()) {
+            validStatement.setDelete((MySQLDeleteStatement) createParseTreeVisitor(ctx.delete().getClass()).visit(ctx.delete()));
+        } else if (null != ctx.select()) {
+            validStatement.setSelect((MySQLSelectStatement) createParseTreeVisitor(ctx.select().getClass()).visit(ctx.select()));
+        }
+        result.getValue().add(validStatement);
+        return result;
+    }
+    
+    @SuppressWarnings("rawtypes")
+    private ParseTreeVisitor createParseTreeVisitor(final Class<? extends ParseTree> parseTreeClass) {

Review comment:
       > if `MySQLDMLVisitor` is necessary, do you think moving the content of `MySQLDMLVisitor` to `MySQLVisitor` will be better than `createParseTreeVisitor`?
   
   @tristaZero The stored routine supports a variety of DML and DAL statements, and migrating the implementation of these statements to the `MySQLVisitor` would cause the `MySQLVisitor` to become bloated. 
   In addition, the `MySQLVisitor` seems to be more suited to processing some generic syntax fragments than the entire statement.




----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] strongduanmu commented on a change in pull request #7704: add create function and procedure validator

Posted by GitBox <gi...@apache.org>.
strongduanmu commented on a change in pull request #7704:
URL: https://github.com/apache/shardingsphere/pull/7704#discussion_r501048116



##########
File path: shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/antlr4/imports/mysql/DDLStatement.g4
##########
@@ -625,10 +625,13 @@ compoundStatement
     ;
 
 validStatement
-    : (createTable | alterTable | dropTable | truncateTable 
-    | insert | replace | update | delete | select | call
+    : (createTable | alterTable | dropTable | truncateTable | validDMLStatement
     | setVariable | beginStatement | declareStatement | flowControlStatement | cursorStatement | conditionHandlingStatement) SEMI_?
     ;
+    
+validDMLStatement

Review comment:
       > I suppose `validDMLStatement` is just for your convenience. If `MySQLVisitor` contains the content of `MySQLDMLStatement`, this rule is unneeded, right? As there is no such concept in the SQL grammar of MySQL, I guess.
   
   @tristaZero Yes, the `validDMLStatement` is simply a way to differentiate the statements in routine Body so that they can be processed during routine parsing. If `MySQLVisitor` contains the content of `MySQLDMLStatement`, this rule can be removed.




----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] strongduanmu commented on a change in pull request #7704: add create function and procedure validator

Posted by GitBox <gi...@apache.org>.
strongduanmu commented on a change in pull request #7704:
URL: https://github.com/apache/shardingsphere/pull/7704#discussion_r501049860



##########
File path: shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingCreateFunctionStatementValidator.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.shardingsphere.sharding.route.engine.validator.impl;
+
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.sharding.route.engine.exception.NoSuchTableException;
+import org.apache.shardingsphere.sharding.route.engine.exception.TableExistsException;
+import org.apache.shardingsphere.sharding.route.engine.validator.ShardingStatementValidator;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
+import org.apache.shardingsphere.sql.parser.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.sql.parser.binder.statement.ddl.CreateFunctionStatementContext;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateFunctionStatement;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Sharding create function statement validator.
+ */
+public final class ShardingCreateFunctionStatementValidator implements ShardingStatementValidator<CreateFunctionStatement> {
+    
+    @Override
+    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<CreateFunctionStatement> sqlStatementContext,
+                            final List<Object> parameters, final ShardingSphereMetaData metaData) {
+        for (SimpleTableSegment each : ((CreateFunctionStatementContext) sqlStatementContext).getExistTables()) {
+            String tableName = each.getTableName().getIdentifier().getValue();
+            if (metaData.getRuleSchemaMetaData().getConfiguredSchemaMetaData().getAllTableNames().contains(tableName)) {
+                throw new ShardingSphereException("Create function statement can not support sharding table '%s'.", tableName);
+            }
+            for (Map.Entry<String, Collection<String>> entry : metaData.getRuleSchemaMetaData().getUnconfiguredSchemaMetaDataMap().entrySet()) {
+                if (!entry.getValue().contains(tableName)) {
+                    throw new NoSuchTableException(entry.getKey(), tableName);
+                }
+            }
+        }
+        for (SimpleTableSegment each : ((CreateFunctionStatementContext) sqlStatementContext).getNotExistTables()) {
+            String tableName = each.getTableName().getIdentifier().getValue();
+            if (metaData.getRuleSchemaMetaData().getAllTableNames().contains(tableName)) {
+                throw new TableExistsException(tableName);
+            }
+        }
+    }

Review comment:
       > Same. But it is not a big problem. Just an optional tip.
   
   @tristaZero I will optimize it too.




----------------------------------------------------------------
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.

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