You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by du...@apache.org on 2022/05/30 08:38:59 UTC

[shardingsphere] branch master updated: add create view validator (#18006)

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

duanzhengqiang pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/shardingsphere.git


The following commit(s) were added to refs/heads/master by this push:
     new b2fad36a125 add create view validator (#18006)
b2fad36a125 is described below

commit b2fad36a125897fc3af1a774a0e1390c9a5db2a1
Author: Chuxin Chen <ch...@qq.com>
AuthorDate: Mon May 30 16:38:53 2022 +0800

    add create view validator (#18006)
    
    * add create view validator
    
    * Add create view statement validator.
    
    * add validator for create view in sharding
---
 .../impl/ShardingCreateViewStatementValidator.java | 71 ++++++++++++++++++-
 .../ShardingCreateViewStatementValidatorTest.java  | 79 +++++++++++++++-------
 2 files changed, 125 insertions(+), 25 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingCreateViewStatementValidator.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingCreateViewStatementValidator.java
index e31b6776690..e3513c08b1f 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingCreateViewStatementValidator.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingCreateViewStatementValidator.java
@@ -19,13 +19,23 @@ package org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl;
 
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.sharding.route.engine.validator.ddl.ShardingDDLStatementValidator;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
+import org.apache.shardingsphere.sql.parser.sql.common.extractor.TableExtractor;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.AggregationProjectionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.ProjectionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateViewStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.handler.dml.SelectStatementHandler;
 
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.List;
+import java.util.Optional;
 
 /**
  * Sharding create view statement validator.
@@ -35,11 +45,70 @@ public final class ShardingCreateViewStatementValidator extends ShardingDDLState
     @Override
     public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<CreateViewStatement> sqlStatementContext,
                             final List<Object> parameters, final ShardingSphereDatabase database) {
-        // TODO prevent if has need merge select.
+        Optional<SelectStatement> selectStatement = sqlStatementContext.getSqlStatement().getSelect();
+        if (!selectStatement.isPresent()) {
+            return;
+        }
+        if (isShardingTablesWithoutBinding(shardingRule, sqlStatementContext, selectStatement.get())) {
+            throw new ShardingSphereException("View name has to bind to sharding tables!");
+        }
     }
     
     @Override
     public void postValidate(final ShardingRule shardingRule, final SQLStatementContext<CreateViewStatement> sqlStatementContext, final List<Object> parameters,
                              final ShardingSphereDatabase database, final ConfigurationProperties props, final RouteContext routeContext) {
+        Optional<SelectStatement> selectStatement = sqlStatementContext.getSqlStatement().getSelect();
+        if (!selectStatement.isPresent()) {
+            return;
+        }
+        if (isContainsNotSupportedViewStatement(selectStatement.get(), routeContext)) {
+            throw new ShardingSphereException("This view statement contains not supported query statement!");
+        }
+    }
+    
+    private boolean isShardingTablesWithoutBinding(final ShardingRule shardingRule, final SQLStatementContext<CreateViewStatement> sqlStatementContext, final SelectStatement selectStatement) {
+        TableExtractor extractor = new TableExtractor();
+        extractor.extractTablesFromSelect(selectStatement);
+        Collection<SimpleTableSegment> tableSegments = extractor.getRewriteTables();
+        for (SimpleTableSegment each : tableSegments) {
+            String logicTable = each.getTableName().getIdentifier().getValue();
+            if (shardingRule.isShardingTable(logicTable) && !isBindingTables(shardingRule, sqlStatementContext.getSqlStatement().getView().getTableName().getIdentifier().getValue(), logicTable)) {
+                return true;
+            }
+        }
+        return false;
+    }
+    
+    private boolean isBindingTables(final ShardingRule shardingRule, final String logicViewName, final String logicTable) {
+        Collection<String> bindTables = Arrays.asList(logicTable, logicViewName);
+        return shardingRule.isAllBindingTables(bindTables);
+    }
+    
+    private boolean isContainsNotSupportedViewStatement(final SelectStatement selectStatement, final RouteContext routeContext) {
+        if (routeContext.getRouteUnits().size() <= 1) {
+            return false;
+        }
+        return hasGroupBy(selectStatement) || hasAggregation(selectStatement) || hasDistinct(selectStatement) || hasLimit(selectStatement);
+    }
+    
+    private boolean hasGroupBy(final SelectStatement selectStatement) {
+        return selectStatement.getGroupBy().map(groupBySegment -> !groupBySegment.getGroupByItems().isEmpty()).orElse(false);
+    }
+    
+    private boolean hasAggregation(final SelectStatement selectStatement) {
+        for (ProjectionSegment each : selectStatement.getProjections().getProjections()) {
+            if (each instanceof AggregationProjectionSegment) {
+                return true;
+            }
+        }
+        return false;
+    }
+    
+    private boolean hasDistinct(final SelectStatement selectStatement) {
+        return selectStatement.getProjections().isDistinctRow();
+    }
+    
+    private boolean hasLimit(final SelectStatement selectStatement) {
+        return SelectStatementHandler.getLimitSegment(selectStatement).isPresent();
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateViewStatementValidatorTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateViewStatementValidatorTest.java
index c51740bc182..a8d935924dc 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateViewStatementValidatorTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateViewStatementValidatorTest.java
@@ -17,57 +17,88 @@
 
 package org.apache.shardingsphere.sharding.route.engine.validator.ddl;
 
-import org.apache.shardingsphere.infra.binder.statement.CommonSQLStatementContext;
-import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.ddl.CreateViewStatementContext;
+import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl.ShardingCreateViewStatementValidator;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.ProjectionsSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.TableNameSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateViewStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
-import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.ddl.MySQLCreateViewStatement;
-import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLSelectStatement;
-import org.junit.Ignore;
+import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
+import org.mockito.Answers;
 import org.mockito.Mock;
 import org.mockito.junit.MockitoJUnitRunner;
 
 import java.util.Collections;
+import java.util.Optional;
 
+import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
-// TODO refactor after create view statement validator is completed.
-@Ignore
 @RunWith(MockitoJUnitRunner.class)
 public final class ShardingCreateViewStatementValidatorTest {
     
+    @Mock(answer = Answers.RETURNS_DEEP_STUBS)
+    private RouteContext routeContext;
+    
     @Mock
     private ShardingRule shardingRule;
     
+    @Mock
+    private CreateViewStatementContext createViewStatementContext;
+    
+    @Mock
+    private CreateViewStatement createViewStatement;
+    
+    @Mock
+    private SelectStatement selectStatement;
+    
+    @Before
+    public void setUp() {
+        when(createViewStatementContext.getSqlStatement()).thenReturn(createViewStatement);
+        when(createViewStatement.getSelect()).thenReturn(Optional.of(selectStatement));
+        when(selectStatement.getFrom()).thenReturn(new SimpleTableSegment(new TableNameSegment(0, 0, new IdentifierValue("t_order"))));
+        when(createViewStatement.getView()).thenReturn(new SimpleTableSegment(new TableNameSegment(0, 0, new IdentifierValue("order_view"))));
+        when(routeContext.getRouteUnits().size()).thenReturn(2);
+    }
+    
+    @Test
+    public void assertPreValidateCreateView() {
+        when(shardingRule.isShardingTable(any())).thenReturn(true);
+        when(shardingRule.isAllBindingTables(any())).thenReturn(true);
+        new ShardingCreateViewStatementValidator().preValidate(shardingRule, createViewStatementContext, Collections.emptyList(), mock(ShardingSphereDatabase.class));
+    }
+    
+    @Test(expected = ShardingSphereException.class)
+    public void assertPreValidateCreateViewWithException() {
+        when(shardingRule.isShardingTable(any())).thenReturn(true);
+        when(shardingRule.isAllBindingTables(any())).thenReturn(false);
+        new ShardingCreateViewStatementValidator().preValidate(shardingRule, createViewStatementContext, Collections.emptyList(), mock(ShardingSphereDatabase.class));
+    }
+    
     @Test
-    public void assertPreValidateCreateViewForMySQL() {
-        MySQLSelectStatement selectStatement = new MySQLSelectStatement();
-        selectStatement.setFrom(new SimpleTableSegment(new TableNameSegment(0, 0, new IdentifierValue("t_order"))));
-        MySQLCreateViewStatement sqlStatement = new MySQLCreateViewStatement();
-        sqlStatement.setSelect(selectStatement);
-        SQLStatementContext<CreateViewStatement> sqlStatementContext = new CommonSQLStatementContext<>(sqlStatement);
-        when(shardingRule.isShardingTable("t_order")).thenReturn(false);
-        new ShardingCreateViewStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList(), mock(ShardingSphereDatabase.class));
+    public void assertPostValidateCreateView() {
+        ProjectionsSegment projectionsSegment = mock(ProjectionsSegment.class);
+        when(selectStatement.getProjections()).thenReturn(projectionsSegment);
+        new ShardingCreateViewStatementValidator().postValidate(shardingRule, createViewStatementContext, Collections.emptyList(), mock(ShardingSphereDatabase.class),
+                mock(ConfigurationProperties.class), routeContext);
     }
     
     @Test(expected = ShardingSphereException.class)
-    public void assertPreValidateCreateViewWithShardingTableForMySQL() {
-        MySQLSelectStatement selectStatement = new MySQLSelectStatement();
-        selectStatement.setFrom(new SimpleTableSegment(new TableNameSegment(0, 0, new IdentifierValue("t_order"))));
-        MySQLCreateViewStatement sqlStatement = new MySQLCreateViewStatement();
-        sqlStatement.setSelect(selectStatement);
-        ShardingSphereDatabase database = mock(ShardingSphereDatabase.class);
-        SQLStatementContext<CreateViewStatement> sqlStatementContext = new CommonSQLStatementContext<>(sqlStatement);
-        when(shardingRule.isShardingTable("t_order")).thenReturn(true);
-        new ShardingCreateViewStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList(), database);
+    public void assertPostValidateCreateViewWithException() {
+        ProjectionsSegment projectionsSegment = mock(ProjectionsSegment.class);
+        when(projectionsSegment.isDistinctRow()).thenReturn(true);
+        when(selectStatement.getProjections()).thenReturn(projectionsSegment);
+        new ShardingCreateViewStatementValidator().postValidate(shardingRule, createViewStatementContext, Collections.emptyList(), mock(ShardingSphereDatabase.class),
+                mock(ConfigurationProperties.class), routeContext);
     }
 }