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/19 05:40:07 UTC

[shardingsphere] branch master updated: Refactor `AlterReadwriteSplittingStorageUnitStatusStatementHandler` (#24229)

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 3d579ff5ed7 Refactor `AlterReadwriteSplittingStorageUnitStatusStatementHandler` (#24229)
3d579ff5ed7 is described below

commit 3d579ff5ed7bccd9bdb55bb5aa69cf301ee7a868
Author: Zichao <57...@users.noreply.github.com>
AuthorDate: Sun Feb 19 13:39:52 2023 +0800

    Refactor `AlterReadwriteSplittingStorageUnitStatusStatementHandler` (#24229)
    
    * Refactor `AlterReadwriteSplittingStorageUnitStatusStatementHandler`
    
    * Refactor `AlterReadwriteSplittingStorageUnitStatusStatementHandler`
    
    * Refactor `AlterReadwriteSplittingStorageUnitStatusStatementHandler`
    
    * Refactor `AlterReadwriteSplittingStorageUnitStatusStatementHandler`
    
    * Refactor `AlterReadwriteSplittingStorageUnitStatusStatementHandler`
    
    * Refactor `AlterReadwriteSplittingStorageUnitStatusStatementHandler`
---
 .../distsql/ral/RALBackendHandlerFactory.java      |  3 -
 ...plittingStorageUnitStatusStatementUpdater.java} | 40 ++++++++------
 ...ingsphere.distsql.handler.ral.update.RALUpdater |  1 +
 ...ttingStorageUnitStatusStatementUpdaterTest.java | 64 ++++++++++++++++++++++
 4 files changed, 88 insertions(+), 20 deletions(-)

diff --git a/proxy/backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/RALBackendHandlerFactory.java b/proxy/backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/RALBackendHandlerFactory.java
index 0cfd7330358..29308799204 100644
--- a/proxy/backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/RALBackendHandlerFactory.java
+++ b/proxy/backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/RALBackendHandlerFactory.java
@@ -35,12 +35,10 @@ import org.apache.shardingsphere.infra.util.spi.type.typed.TypedSPILoader;
 import org.apache.shardingsphere.proxy.backend.handler.ProxyBackendHandler;
 import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.hint.HintRALBackendHandler;
 import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.migration.update.UpdatableScalingRALBackendHandler;
-import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.updatable.AlterReadwriteSplittingStorageUnitStatusStatementHandler;
 import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.updatable.RefreshDatabaseMetaDataHandler;
 import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.updatable.RefreshTableMetaDataHandler;
 import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.updatable.SetDistVariableHandler;
 import org.apache.shardingsphere.proxy.backend.session.ConnectionSession;
-import org.apache.shardingsphere.readwritesplitting.distsql.parser.statement.status.AlterReadwriteSplittingStorageUnitStatusStatement;
 
 import java.util.HashMap;
 import java.util.Map;
@@ -55,7 +53,6 @@ public final class RALBackendHandlerFactory {
     
     static {
         HANDLERS.put(SetDistVariableStatement.class, SetDistVariableHandler.class);
-        HANDLERS.put(AlterReadwriteSplittingStorageUnitStatusStatement.class, AlterReadwriteSplittingStorageUnitStatusStatementHandler.class);
         HANDLERS.put(RefreshDatabaseMetaDataStatement.class, RefreshDatabaseMetaDataHandler.class);
         HANDLERS.put(RefreshTableMetaDataStatement.class, RefreshTableMetaDataHandler.class);
     }
diff --git a/proxy/backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/AlterReadwriteSplittingStorageUnitStatusStatementHandler.java b/proxy/backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/AlterReadwriteSplittingStorageUnitStatusStatementUpdater.java
similarity index 90%
rename from proxy/backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/AlterReadwriteSplittingStorageUnitStatusStatementHandler.java
rename to proxy/backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/AlterReadwriteSplittingStorageUnitStatusStatementUpdater.java
index 67152e9a148..32bc153a30c 100644
--- a/proxy/backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/AlterReadwriteSplittingStorageUnitStatusStatementHandler.java
+++ b/proxy/backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/AlterReadwriteSplittingStorageUnitStatusStatementUpdater.java
@@ -22,6 +22,7 @@ import com.google.common.base.Strings;
 import org.apache.shardingsphere.dialect.exception.syntax.database.NoDatabaseSelectedException;
 import org.apache.shardingsphere.dialect.exception.syntax.database.UnknownDatabaseException;
 import org.apache.shardingsphere.distsql.handler.exception.storageunit.MissingRequiredStorageUnitsException;
+import org.apache.shardingsphere.distsql.handler.ral.update.RALUpdater;
 import org.apache.shardingsphere.infra.datasource.state.DataSourceState;
 import org.apache.shardingsphere.infra.metadata.database.schema.QualifiedDatabase;
 import org.apache.shardingsphere.infra.rule.identifier.type.exportable.RuleExportEngine;
@@ -37,7 +38,6 @@ import org.apache.shardingsphere.mode.metadata.storage.StorageNodeRole;
 import org.apache.shardingsphere.mode.metadata.storage.event.DataSourceDisabledEvent;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
-import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.UpdatableRALBackendHandler;
 import org.apache.shardingsphere.readwritesplitting.distsql.parser.statement.status.AlterReadwriteSplittingStorageUnitStatusStatement;
 import org.apache.shardingsphere.readwritesplitting.rule.ReadwriteSplittingRule;
 
@@ -51,35 +51,36 @@ import java.util.Optional;
 import java.util.stream.Collectors;
 
 /**
- * Set readwrite-splitting storage unit status handler.
+ * Set readwrite-splitting storage unit status updater.
  */
-public final class AlterReadwriteSplittingStorageUnitStatusStatementHandler extends UpdatableRALBackendHandler<AlterReadwriteSplittingStorageUnitStatusStatement> {
+public final class AlterReadwriteSplittingStorageUnitStatusStatementUpdater implements RALUpdater<AlterReadwriteSplittingStorageUnitStatusStatement> {
     
     private static final String DISABLE = "DISABLE";
     
     @Override
-    protected void update(final ContextManager contextManager) {
-        String databaseName = getSqlStatement().getDatabase().isPresent() ? getSqlStatement().getDatabase().get().getIdentifier().getValue() : getConnectionSession().getDatabaseName();
-        String toBeUpdatedStorageUnit = getSqlStatement().getStorageUnitName();
+    public void executeUpdate(final String databaseName, final AlterReadwriteSplittingStorageUnitStatusStatement sqlStatement) {
+        String actualDatabaseName = sqlStatement.getDatabase().isPresent() ? sqlStatement.getDatabase().get().getIdentifier().getValue() : databaseName;
+        String toBeUpdatedStorageUnit = sqlStatement.getStorageUnitName();
+        ContextManager contextManager = ProxyContext.getInstance().getContextManager();
         checkModeAndPersistRepository(contextManager);
-        checkDatabaseName(databaseName);
-        checkReadwriteSplittingRule(contextManager, databaseName);
-        Map<String, String> replicaStorageUnits = getReplicaResources(contextManager, databaseName);
-        Map<String, String> disabledStorageUnits = getDisabledResources(contextManager, databaseName);
-        Map<String, String> autoAwareResources = getAutoAwareResources(contextManager, databaseName);
-        boolean isDisable = DISABLE.equals(getSqlStatement().getStatus());
+        checkDatabaseName(actualDatabaseName);
+        checkReadwriteSplittingRule(contextManager, actualDatabaseName);
+        Map<String, String> replicaStorageUnits = getReplicaResources(contextManager, actualDatabaseName);
+        Map<String, String> disabledStorageUnits = getDisabledResources(contextManager, actualDatabaseName);
+        Map<String, String> autoAwareResources = getAutoAwareResources(contextManager, actualDatabaseName);
+        boolean isDisable = DISABLE.equals(sqlStatement.getStatus());
         if (isDisable) {
-            checkDisable(contextManager, databaseName, disabledStorageUnits.keySet(), toBeUpdatedStorageUnit, replicaStorageUnits);
+            checkDisable(contextManager, actualDatabaseName, disabledStorageUnits.keySet(), toBeUpdatedStorageUnit, replicaStorageUnits);
         } else {
-            checkEnable(contextManager, databaseName, disabledStorageUnits, toBeUpdatedStorageUnit);
+            checkEnable(contextManager, actualDatabaseName, disabledStorageUnits, toBeUpdatedStorageUnit);
         }
         Collection<String> groupNames = getGroupNames(toBeUpdatedStorageUnit, replicaStorageUnits, disabledStorageUnits, autoAwareResources);
-        String groupName = getSqlStatement().getGroupName();
+        String groupName = sqlStatement.getGroupName();
         if (Strings.isNullOrEmpty(groupName)) {
-            updateStatus(databaseName, groupNames, toBeUpdatedStorageUnit, isDisable);
+            updateStatus(actualDatabaseName, groupNames, toBeUpdatedStorageUnit, isDisable);
         } else {
             checkGroupName(groupNames, groupName);
-            updateStatus(databaseName, Collections.singleton(groupName), toBeUpdatedStorageUnit, isDisable);
+            updateStatus(actualDatabaseName, Collections.singleton(groupName), toBeUpdatedStorageUnit, isDisable);
         }
     }
     
@@ -217,4 +218,9 @@ public final class AlterReadwriteSplittingStorageUnitStatusStatementHandler exte
     private void put(final Map<String, String> map, final String key, final String value) {
         map.put(key, map.containsKey(key) ? String.join(",", map.get(key), value) : value);
     }
+    
+    @Override
+    public String getType() {
+        return AlterReadwriteSplittingStorageUnitStatusStatement.class.getName();
+    }
 }
diff --git a/proxy/backend/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.ral.update.RALUpdater b/proxy/backend/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.ral.update.RALUpdater
index 1a4bf1ab11a..80413ec94a4 100644
--- a/proxy/backend/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.ral.update.RALUpdater
+++ b/proxy/backend/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.ral.update.RALUpdater
@@ -20,3 +20,4 @@ org.apache.shardingsphere.proxy.backend.handler.distsql.ral.updatable.LabelCompu
 org.apache.shardingsphere.proxy.backend.handler.distsql.ral.updatable.UnlabelComputeNodeUpdater
 org.apache.shardingsphere.proxy.backend.handler.distsql.ral.updatable.SetInstanceStatusUpdater
 org.apache.shardingsphere.proxy.backend.handler.distsql.ral.updatable.ImportDatabaseConfigurationUpdater
+org.apache.shardingsphere.proxy.backend.handler.distsql.ral.updatable.AlterReadwriteSplittingStorageUnitStatusStatementUpdater
diff --git a/proxy/backend/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/AlterReadwriteSplittingStorageUnitStatusStatementUpdaterTest.java b/proxy/backend/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/AlterReadwriteSplittingStorageUnitStatusStatementUpdaterTest.java
new file mode 100644
index 00000000000..1b4c980edb2
--- /dev/null
+++ b/proxy/backend/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/AlterReadwriteSplittingStorageUnitStatusStatementUpdaterTest.java
@@ -0,0 +1,64 @@
+/*
+ * 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.proxy.backend.handler.distsql.ral.updatable;
+
+import org.apache.shardingsphere.dialect.exception.syntax.database.UnknownDatabaseException;
+import org.apache.shardingsphere.infra.util.exception.external.sql.type.generic.UnsupportedSQLOperationException;
+import org.apache.shardingsphere.mode.manager.ContextManager;
+import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
+import org.apache.shardingsphere.proxy.backend.util.ProxyContextRestorer;
+import org.apache.shardingsphere.readwritesplitting.distsql.parser.statement.status.AlterReadwriteSplittingStorageUnitStatusStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.DatabaseSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
+import org.junit.Test;
+
+import java.sql.SQLException;
+
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public final class AlterReadwriteSplittingStorageUnitStatusStatementUpdaterTest extends ProxyContextRestorer {
+    
+    @Test(expected = UnsupportedSQLOperationException.class)
+    public void assertWithStandaloneMode() throws SQLException {
+        ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
+        ProxyContext.init(contextManager);
+        AlterReadwriteSplittingStorageUnitStatusStatementUpdater updater = new AlterReadwriteSplittingStorageUnitStatusStatementUpdater();
+        updater.executeUpdate("foo", new AlterReadwriteSplittingStorageUnitStatusStatement(new DatabaseSegment(1, 1, new IdentifierValue("db")), "group", "read_ds", "ENABLE"));
+    }
+    
+    @Test(expected = UnknownDatabaseException.class)
+    public void assertWithUnknownDatabase() throws SQLException {
+        ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
+        when(contextManager.getInstanceContext().isCluster()).thenReturn(true);
+        ProxyContext.init(contextManager);
+        AlterReadwriteSplittingStorageUnitStatusStatementUpdater updater = new AlterReadwriteSplittingStorageUnitStatusStatementUpdater();
+        updater.executeUpdate("foo", new AlterReadwriteSplittingStorageUnitStatusStatement(new DatabaseSegment(1, 1, new IdentifierValue("db")), "group", "read_ds", "ENABLE"));
+    }
+    
+    @Test(expected = UnsupportedSQLOperationException.class)
+    public void assertWithNoReadwriteSplittingRule() throws SQLException {
+        ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
+        when(contextManager.getMetaDataContexts().getMetaData().containsDatabase("db")).thenReturn(true);
+        when(contextManager.getInstanceContext().isCluster()).thenReturn(true);
+        ProxyContext.init(contextManager);
+        AlterReadwriteSplittingStorageUnitStatusStatementUpdater updater = new AlterReadwriteSplittingStorageUnitStatusStatementUpdater();
+        updater.executeUpdate("foo", new AlterReadwriteSplittingStorageUnitStatusStatement(new DatabaseSegment(1, 1, new IdentifierValue("db")), "group", "read_ds", "ENABLE"));
+    }
+}