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/07 13:21:48 UTC

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

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