You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by ji...@apache.org on 2023/02/14 03:53:01 UTC

[shardingsphere] branch master updated: Make CREATE DEFAULT SHARDING STRATEGY support NoneShardingStrategy (#24151)

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

jianglongtao 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 c236c86d5db Make CREATE DEFAULT SHARDING STRATEGY support NoneShardingStrategy (#24151)
c236c86d5db is described below

commit c236c86d5db833f71602a345292fc662f34f478c
Author: Zichao <57...@users.noreply.github.com>
AuthorDate: Tue Feb 14 11:52:48 2023 +0800

    Make CREATE DEFAULT SHARDING STRATEGY support NoneShardingStrategy (#24151)
---
 ...ateDefaultShardingStrategyStatementUpdater.java | 19 ++++++++++-----
 ...efaultShardingStrategyStatementUpdaterTest.java | 27 ++++++++++++++++++++++
 .../core/ShardingDistSQLStatementVisitor.java      | 15 ++++++------
 .../MySQLAuthenticationEngineTest.java             | 11 ++++-----
 4 files changed, 53 insertions(+), 19 deletions(-)

diff --git a/features/sharding/distsql/handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/CreateDefaultShardingStrategyStatementUpdater.java b/features/sharding/distsql/handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/CreateDefaultShardingStrategyStatementUpdater.java
index 4837896ffd5..f9e9e1898be 100644
--- a/features/sharding/distsql/handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/CreateDefaultShardingStrategyStatementUpdater.java
+++ b/features/sharding/distsql/handler/src/main/java/org/apache/shardingsphere/sharding/distsql/handler/update/CreateDefaultShardingStrategyStatementUpdater.java
@@ -27,6 +27,7 @@ import org.apache.shardingsphere.infra.config.algorithm.AlgorithmConfiguration;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.util.exception.ShardingSpherePreconditions;
 import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
+import org.apache.shardingsphere.sharding.api.config.strategy.sharding.NoneShardingStrategyConfiguration;
 import org.apache.shardingsphere.sharding.api.config.strategy.sharding.ShardingStrategyConfiguration;
 import org.apache.shardingsphere.sharding.distsql.handler.converter.ShardingTableRuleStatementConverter;
 import org.apache.shardingsphere.sharding.distsql.handler.enums.ShardingStrategyLevelType;
@@ -44,7 +45,9 @@ public final class CreateDefaultShardingStrategyStatementUpdater implements Rule
     public void checkSQLStatement(final ShardingSphereDatabase database, final CreateDefaultShardingStrategyStatement sqlStatement, final ShardingRuleConfiguration currentRuleConfig) {
         String databaseName = database.getName();
         checkCurrentRuleConfiguration(databaseName, currentRuleConfig);
-        checkAlgorithm(databaseName, currentRuleConfig, sqlStatement);
+        if (!"none".equalsIgnoreCase(sqlStatement.getStrategyType())) {
+            checkAlgorithm(sqlStatement);
+        }
         if (!sqlStatement.isIfNotExists()) {
             checkExist(databaseName, sqlStatement, currentRuleConfig);
         }
@@ -54,7 +57,7 @@ public final class CreateDefaultShardingStrategyStatementUpdater implements Rule
         ShardingSpherePreconditions.checkNotNull(currentRuleConfig, () -> new MissingRequiredRuleException("Sharding", databaseName));
     }
     
-    private void checkAlgorithm(final String databaseName, final ShardingRuleConfiguration currentRuleConfig, final CreateDefaultShardingStrategyStatement sqlStatement) {
+    private void checkAlgorithm(final CreateDefaultShardingStrategyStatement sqlStatement) {
         ShardingSpherePreconditions.checkState(ShardingStrategyType.contains(sqlStatement.getStrategyType()), () -> new InvalidAlgorithmConfigurationException(sqlStatement.getStrategyType()));
         ShardingSpherePreconditions.checkState(ShardingStrategyType.getValueOf(sqlStatement.getStrategyType())
                 .isValid(sqlStatement.getShardingColumn()), () -> new InvalidAlgorithmConfigurationException(sqlStatement.getStrategyType()));
@@ -81,10 +84,14 @@ public final class CreateDefaultShardingStrategyStatementUpdater implements Rule
     @Override
     public ShardingRuleConfiguration buildToBeCreatedRuleConfiguration(final ShardingRuleConfiguration currentRuleConfig, final CreateDefaultShardingStrategyStatement sqlStatement) {
         ShardingRuleConfiguration result = new ShardingRuleConfiguration();
-        String shardingAlgorithmName = getShardingAlgorithmName(sqlStatement, result);
-        ShardingStrategyConfiguration strategyConfig = ShardingTableRuleStatementConverter.createStrategyConfiguration(
-                sqlStatement.getStrategyType(), sqlStatement.getShardingColumn(), shardingAlgorithmName);
-        setStrategyConfiguration(result, sqlStatement.getDefaultType(), strategyConfig);
+        if ("none".equalsIgnoreCase(sqlStatement.getStrategyType())) {
+            setStrategyConfiguration(result, sqlStatement.getDefaultType(), new NoneShardingStrategyConfiguration());
+        } else {
+            String shardingAlgorithmName = getShardingAlgorithmName(sqlStatement, result);
+            ShardingStrategyConfiguration strategyConfig = ShardingTableRuleStatementConverter.createStrategyConfiguration(
+                    sqlStatement.getStrategyType(), sqlStatement.getShardingColumn(), shardingAlgorithmName);
+            setStrategyConfiguration(result, sqlStatement.getDefaultType(), strategyConfig);
+        }
         return result;
     }
     
diff --git a/features/sharding/distsql/handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/CreateDefaultShardingStrategyStatementUpdaterTest.java b/features/sharding/distsql/handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/CreateDefaultShardingStrategyStatementUpdaterTest.java
index f8edc67c7c0..f36404caff5 100644
--- a/features/sharding/distsql/handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/CreateDefaultShardingStrategyStatementUpdaterTest.java
+++ b/features/sharding/distsql/handler/src/test/java/org/apache/shardingsphere/sharding/distsql/update/CreateDefaultShardingStrategyStatementUpdaterTest.java
@@ -23,6 +23,7 @@ import org.apache.shardingsphere.distsql.handler.exception.rule.MissingRequiredR
 import org.apache.shardingsphere.distsql.parser.segment.AlgorithmSegment;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
+import org.apache.shardingsphere.sharding.api.config.strategy.sharding.NoneShardingStrategyConfiguration;
 import org.apache.shardingsphere.sharding.api.config.strategy.sharding.StandardShardingStrategyConfiguration;
 import org.apache.shardingsphere.sharding.distsql.handler.update.CreateDefaultShardingStrategyStatementUpdater;
 import org.apache.shardingsphere.sharding.distsql.parser.statement.CreateDefaultShardingStrategyStatement;
@@ -150,6 +151,32 @@ public final class CreateDefaultShardingStrategyStatementUpdaterTest {
         assertThat(defaultDatabaseShardingStrategy.getShardingColumn(), is("user_id"));
     }
     
+    @Test
+    public void assertCreateDefaultTableShardingStrategyWithNoneShardingStrategyType() {
+        CreateDefaultShardingStrategyStatement statement = new CreateDefaultShardingStrategyStatement(false, "TABLE", "none", null, null);
+        ShardingRuleConfiguration currentRuleConfig = new ShardingRuleConfiguration();
+        currentRuleConfig.setDefaultDatabaseShardingStrategy(new StandardShardingStrategyConfiguration("order_id", "orderAlgorithm"));
+        currentRuleConfig.getShardingAlgorithms().put("order_id_algorithm", null);
+        updater.checkSQLStatement(database, statement, currentRuleConfig);
+        ShardingRuleConfiguration toBeCreatedRuleConfig = updater.buildToBeCreatedRuleConfiguration(currentRuleConfig, statement);
+        updater.updateCurrentRuleConfiguration(currentRuleConfig, toBeCreatedRuleConfig);
+        NoneShardingStrategyConfiguration defaultTableShardingStrategy = (NoneShardingStrategyConfiguration) currentRuleConfig.getDefaultTableShardingStrategy();
+        assertThat(defaultTableShardingStrategy.getType(), is(""));
+        assertThat(defaultTableShardingStrategy.getShardingAlgorithmName(), is(""));
+    }
+    
+    @Test
+    public void assertCreateDefaultDatabaseShardingStrategyWithNoneShardingStrategyType() {
+        CreateDefaultShardingStrategyStatement statement = new CreateDefaultShardingStrategyStatement(false, "DATABASE", "none", null, null);
+        ShardingRuleConfiguration currentRuleConfig = new ShardingRuleConfiguration();
+        updater.checkSQLStatement(database, statement, currentRuleConfig);
+        ShardingRuleConfiguration toBeCreatedRuleConfig = updater.buildToBeCreatedRuleConfiguration(currentRuleConfig, statement);
+        updater.updateCurrentRuleConfiguration(currentRuleConfig, toBeCreatedRuleConfig);
+        NoneShardingStrategyConfiguration defaultDatabaseShardingStrategy = (NoneShardingStrategyConfiguration) currentRuleConfig.getDefaultDatabaseShardingStrategy();
+        assertThat(defaultDatabaseShardingStrategy.getType(), is(""));
+        assertThat(defaultDatabaseShardingStrategy.getShardingAlgorithmName(), is(""));
+    }
+    
     private AlgorithmSegment getAutoCreativeAlgorithmSegment(final String name, final Properties props) {
         return new AlgorithmSegment(name, props);
     }
diff --git a/features/sharding/distsql/parser/src/main/java/org/apache/shardingsphere/sharding/distsql/parser/core/ShardingDistSQLStatementVisitor.java b/features/sharding/distsql/parser/src/main/java/org/apache/shardingsphere/sharding/distsql/parser/core/ShardingDistSQLStatementVisitor.java
index a59bab21c89..0c1c0d155c3 100644
--- a/features/sharding/distsql/parser/src/main/java/org/apache/shardingsphere/sharding/distsql/parser/core/ShardingDistSQLStatementVisitor.java
+++ b/features/sharding/distsql/parser/src/main/java/org/apache/shardingsphere/sharding/distsql/parser/core/ShardingDistSQLStatementVisitor.java
@@ -190,11 +190,14 @@ public final class ShardingDistSQLStatementVisitor extends ShardingDistSQLStatem
     @Override
     public ASTNode visitCreateDefaultShardingStrategy(final CreateDefaultShardingStrategyContext ctx) {
         ShardingStrategyContext shardingStrategyContext = ctx.shardingStrategy();
+        String strategyType = getIdentifierValue(shardingStrategyContext.strategyType());
+        if ("none".equalsIgnoreCase(strategyType)) {
+            return new CreateDefaultShardingStrategyStatement(null != ctx.ifNotExists(), new IdentifierValue(ctx.type.getText()).getValue(), "none", null, null);
+        }
         AlgorithmSegment algorithmSegment = null != shardingStrategyContext.shardingAlgorithm().algorithmDefinition()
                 ? (AlgorithmSegment) visitAlgorithmDefinition(shardingStrategyContext.shardingAlgorithm().algorithmDefinition())
                 : null;
         String defaultType = new IdentifierValue(ctx.type.getText()).getValue();
-        String strategyType = getIdentifierValue(shardingStrategyContext.strategyType());
         String shardingColumn = buildShardingColumn(ctx.shardingStrategy().shardingColumnDefinition());
         return new CreateDefaultShardingStrategyStatement(null != ctx.ifNotExists(), defaultType, strategyType, shardingColumn, algorithmSegment);
     }
@@ -321,14 +324,12 @@ public final class ShardingDistSQLStatementVisitor extends ShardingDistSQLStatem
     
     @Override
     public ASTNode visitShardingStrategy(final ShardingStrategyContext ctx) {
-        if (null == ctx) {
-            return null;
-        }
-        if ("none".equalsIgnoreCase(getIdentifierValue(ctx.strategyType()))) {
-            return new ShardingStrategySegment(getIdentifierValue(ctx.strategyType()), null, null);
+        String strategyType = getIdentifierValue(ctx.strategyType());
+        if ("none".equalsIgnoreCase(strategyType)) {
+            return new ShardingStrategySegment(strategyType, null, null);
         }
         AlgorithmSegment algorithmSegment = null != ctx.shardingAlgorithm().algorithmDefinition() ? (AlgorithmSegment) visitAlgorithmDefinition(ctx.shardingAlgorithm().algorithmDefinition()) : null;
-        return new ShardingStrategySegment(getIdentifierValue(ctx.strategyType()), buildShardingColumn(ctx.shardingColumnDefinition()), algorithmSegment);
+        return new ShardingStrategySegment(strategyType, buildShardingColumn(ctx.shardingColumnDefinition()), algorithmSegment);
     }
     
     private Collection<String> getResources(final StorageUnitsContext ctx) {
diff --git a/proxy/frontend/mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/authentication/MySQLAuthenticationEngineTest.java b/proxy/frontend/mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/authentication/MySQLAuthenticationEngineTest.java
index b8edbf32c20..59d9eb417d7 100644
--- a/proxy/frontend/mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/authentication/MySQLAuthenticationEngineTest.java
+++ b/proxy/frontend/mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/authentication/MySQLAuthenticationEngineTest.java
@@ -153,9 +153,8 @@ public final class MySQLAuthenticationEngineTest extends ProxyContextRestorer {
         ChannelHandlerContext context = mockChannelHandlerContext();
         try (
                 MockedConstruction<AuthenticatorFactory> mockedAuthenticatorFactory = mockConstruction(AuthenticatorFactory.class,
-                        (mock, mockContext) -> when(mock.newInstance(user)).thenReturn(mock(Authenticator.class))); 
-                MockedConstruction<MySQLErrPacket> mockedErrPacket = mockConstruction(MySQLErrPacket.class, (mock, mockContext) -> assertAuthenticationErrorPacket(mockContext.arguments()))
-        ) {
+                        (mock, mockContext) -> when(mock.newInstance(user)).thenReturn(mock(Authenticator.class)));
+                MockedConstruction<MySQLErrPacket> mockedErrPacket = mockConstruction(MySQLErrPacket.class, (mock, mockContext) -> assertAuthenticationErrorPacket(mockContext.arguments()))) {
             authenticationEngine.authenticate(context, getPayload("root", "sharding_db", authResponse));
             verify(context).writeAndFlush(any(MySQLErrPacket.class));
             verify(context).close();
@@ -164,7 +163,7 @@ public final class MySQLAuthenticationEngineTest extends ProxyContextRestorer {
     
     private void assertAuthenticationErrorPacket(final List<?> arguments) {
         assertThat(arguments.get(0), is(MySQLVendorError.ER_ACCESS_DENIED_ERROR));
-        assertThat(arguments.get(1), is(new String[] {"root", "127.0.0.1", "YES"}));
+        assertThat(arguments.get(1), is(new String[]{"root", "127.0.0.1", "YES"}));
     }
     
     @Test
@@ -185,7 +184,7 @@ public final class MySQLAuthenticationEngineTest extends ProxyContextRestorer {
     
     private void assertDatabaseAccessDeniedErrorPacket(final List<?> arguments) {
         assertThat(arguments.get(0), is(MySQLVendorError.ER_DBACCESS_DENIED_ERROR));
-        assertThat(arguments.get(1), is(new String[] {"root", "127.0.0.1", "sharding_db"}));
+        assertThat(arguments.get(1), is(new String[]{"root", "127.0.0.1", "sharding_db"}));
     }
     
     @Test
@@ -204,7 +203,7 @@ public final class MySQLAuthenticationEngineTest extends ProxyContextRestorer {
     
     private void assertInvalidDatabaseErrorPacket(final List<?> arguments) {
         assertThat(arguments.get(0), is(MySQLVendorError.ER_BAD_DB_ERROR));
-        assertThat(arguments.get(1), is(new String[] {"invalid_db"}));
+        assertThat(arguments.get(1), is(new String[]{"invalid_db"}));
     }
     
     @Test