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/20 04:07:11 UTC

[shardingsphere] branch master updated: Refactor `SetDistVariableHandler` (#24239)

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 ab55d7fa0dc Refactor `SetDistVariableHandler` (#24239)
ab55d7fa0dc is described below

commit ab55d7fa0dc50d84f09338d74579402c359ed71c
Author: Zichao <57...@users.noreply.github.com>
AuthorDate: Mon Feb 20 12:07:02 2023 +0800

    Refactor `SetDistVariableHandler` (#24239)
    
    * Refactor `SetDistVariableHandler`
    
    * Refactor `SetDistVariableHandler`
    
    * Refactor `SetDistVariableHandler`
---
 .../distsql/handler/ral/update/RALUpdater.java     |  3 +-
 .../distsql/ral/RALBackendHandlerFactory.java      |  3 --
 .../ral/UpdatableRALUpdaterBackendHandler.java     |  8 +++++-
 ...bleHandler.java => SetDistVariableUpdater.java} | 32 +++++++++++++---------
 .../ConnectionSessionRequiredRALUpdater.java       | 17 +++++-------
 ...ingsphere.distsql.handler.ral.update.RALUpdater |  1 +
 .../handler/ProxyBackendHandlerFactoryTest.java    |  4 +--
 ...ableUpdatableRALUpdaterBackendHandlerTest.java} | 27 +++++++-----------
 ...orTest.java => SetDistVariableUpdaterTest.java} | 32 +++++++++-------------
 9 files changed, 61 insertions(+), 66 deletions(-)

diff --git a/distsql/handler/src/main/java/org/apache/shardingsphere/distsql/handler/ral/update/RALUpdater.java b/distsql/handler/src/main/java/org/apache/shardingsphere/distsql/handler/ral/update/RALUpdater.java
index 42ab27dd11a..2c62bcc77ca 100644
--- a/distsql/handler/src/main/java/org/apache/shardingsphere/distsql/handler/ral/update/RALUpdater.java
+++ b/distsql/handler/src/main/java/org/apache/shardingsphere/distsql/handler/ral/update/RALUpdater.java
@@ -38,5 +38,6 @@ public interface RALUpdater<T extends SQLStatement> extends TypedSPI {
      * @param sqlStatement updatable RAL statement
      * @throws SQLException SQL exception
      */
-    void executeUpdate(String databaseName, T sqlStatement) throws SQLException;
+    default void executeUpdate(String databaseName, T sqlStatement) throws SQLException {
+    }
 }
diff --git a/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/RALBackendHandlerFactory.java b/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/RALBackendHandlerFactory.java
index 9a384ee6fbb..af7db86fdad 100644
--- a/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/RALBackendHandlerFactory.java
+++ b/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/RALBackendHandlerFactory.java
@@ -27,7 +27,6 @@ import org.apache.shardingsphere.distsql.parser.statement.ral.UpdatableGlobalRul
 import org.apache.shardingsphere.distsql.parser.statement.ral.UpdatableRALStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.scaling.UpdatableScalingRALStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.updatable.RefreshTableMetaDataStatement;
-import org.apache.shardingsphere.distsql.parser.statement.ral.updatable.SetDistVariableStatement;
 import org.apache.shardingsphere.infra.util.exception.ShardingSpherePreconditions;
 import org.apache.shardingsphere.infra.util.exception.external.sql.type.generic.UnsupportedSQLOperationException;
 import org.apache.shardingsphere.infra.util.spi.type.typed.TypedSPILoader;
@@ -35,7 +34,6 @@ 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.RefreshTableMetaDataHandler;
-import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.updatable.SetDistVariableHandler;
 import org.apache.shardingsphere.proxy.backend.session.ConnectionSession;
 
 import java.util.HashMap;
@@ -50,7 +48,6 @@ public final class RALBackendHandlerFactory {
     private static final Map<Class<? extends RALStatement>, Class<? extends RALBackendHandler<?>>> HANDLERS = new HashMap<>();
     
     static {
-        HANDLERS.put(SetDistVariableStatement.class, SetDistVariableHandler.class);
         HANDLERS.put(RefreshTableMetaDataStatement.class, RefreshTableMetaDataHandler.class);
     }
     
diff --git a/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/UpdatableRALUpdaterBackendHandler.java b/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/UpdatableRALUpdaterBackendHandler.java
index cda4a5ed195..6a60dc01fd3 100644
--- a/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/UpdatableRALUpdaterBackendHandler.java
+++ b/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/UpdatableRALUpdaterBackendHandler.java
@@ -22,6 +22,7 @@ import lombok.Setter;
 import org.apache.shardingsphere.distsql.handler.ral.update.RALUpdater;
 import org.apache.shardingsphere.distsql.parser.statement.ral.UpdatableRALStatement;
 import org.apache.shardingsphere.infra.util.spi.type.typed.TypedSPILoader;
+import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.updatable.updater.ConnectionSessionRequiredRALUpdater;
 import org.apache.shardingsphere.proxy.backend.response.header.ResponseHeader;
 import org.apache.shardingsphere.proxy.backend.response.header.update.UpdateResponseHeader;
 import org.apache.shardingsphere.proxy.backend.session.ConnectionSession;
@@ -44,7 +45,12 @@ public final class UpdatableRALUpdaterBackendHandler<T extends UpdatableRALState
     @SuppressWarnings("unchecked")
     @Override
     public ResponseHeader execute() throws SQLException {
-        TypedSPILoader.getService(RALUpdater.class, sqlStatement.getClass().getName()).executeUpdate(connectionSession.getDatabaseName(), sqlStatement);
+        RALUpdater<T> updater = TypedSPILoader.getService(RALUpdater.class, sqlStatement.getClass().getName());
+        if (updater instanceof ConnectionSessionRequiredRALUpdater) {
+            ((ConnectionSessionRequiredRALUpdater<T>) updater).executeUpdate(connectionSession, (T) sqlStatement);
+        } else {
+            updater.executeUpdate(connectionSession.getDatabaseName(), (T) sqlStatement);
+        }
         return new UpdateResponseHeader(sqlStatement);
     }
 }
diff --git a/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/SetDistVariableHandler.java b/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/SetDistVariableUpdater.java
similarity index 85%
rename from proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/SetDistVariableHandler.java
rename to proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/SetDistVariableUpdater.java
index 15450062c6f..61d629b56aa 100644
--- a/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/SetDistVariableHandler.java
+++ b/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/SetDistVariableUpdater.java
@@ -34,8 +34,9 @@ import org.apache.shardingsphere.mode.metadata.MetaDataContexts;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.exception.InvalidValueException;
 import org.apache.shardingsphere.proxy.backend.exception.UnsupportedVariableException;
-import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.UpdatableRALBackendHandler;
 import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.common.enums.VariableEnum;
+import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.updatable.updater.ConnectionSessionRequiredRALUpdater;
+import org.apache.shardingsphere.proxy.backend.session.ConnectionSession;
 import org.apache.shardingsphere.proxy.backend.util.SystemPropertyUtil;
 import org.apache.shardingsphere.transaction.api.TransactionType;
 import org.slf4j.LoggerFactory;
@@ -43,19 +44,19 @@ import org.slf4j.LoggerFactory;
 import java.util.Properties;
 
 /**
- * Set dist variable statement handler.
+ * Set dist variable statement updater.
  */
-public final class SetDistVariableHandler extends UpdatableRALBackendHandler<SetDistVariableStatement> {
+public final class SetDistVariableUpdater implements ConnectionSessionRequiredRALUpdater<SetDistVariableStatement> {
     
     @Override
-    protected void update(final ContextManager contextManager) {
-        Enum<?> enumType = getEnumType(getSqlStatement().getName());
+    public void executeUpdate(final ConnectionSession connectionSession, final SetDistVariableStatement sqlStatement) {
+        Enum<?> enumType = getEnumType(sqlStatement.getName());
         if (enumType instanceof TypedPropertyKey) {
-            handleConfigurationProperty((TypedPropertyKey) enumType, getSqlStatement().getValue());
+            handleConfigurationProperty((TypedPropertyKey) enumType, sqlStatement.getValue());
         } else if (enumType instanceof VariableEnum) {
-            handleVariables();
+            handleVariables(connectionSession, sqlStatement);
         } else {
-            throw new UnsupportedVariableException(getSqlStatement().getName());
+            throw new UnsupportedVariableException(sqlStatement.getName());
         }
     }
     
@@ -121,18 +122,18 @@ public final class SetDistVariableHandler extends UpdatableRALBackendHandler<Set
         }
     }
     
-    private void handleVariables() {
-        VariableEnum variable = VariableEnum.getValueOf(getSqlStatement().getName());
+    private void handleVariables(final ConnectionSession connectionSession, final SetDistVariableStatement sqlStatement) {
+        VariableEnum variable = VariableEnum.getValueOf(sqlStatement.getName());
         switch (variable) {
             case AGENT_PLUGINS_ENABLED:
-                Boolean agentPluginsEnabled = BooleanUtils.toBooleanObject(getSqlStatement().getValue());
+                Boolean agentPluginsEnabled = BooleanUtils.toBooleanObject(sqlStatement.getValue());
                 SystemPropertyUtil.setSystemProperty(variable.name(), null == agentPluginsEnabled ? Boolean.FALSE.toString() : agentPluginsEnabled.toString());
                 break;
             case TRANSACTION_TYPE:
-                getConnectionSession().getTransactionStatus().setTransactionType(getTransactionType(getSqlStatement().getValue()));
+                connectionSession.getTransactionStatus().setTransactionType(getTransactionType(sqlStatement.getValue()));
                 break;
             default:
-                throw new UnsupportedVariableException(getSqlStatement().getName());
+                throw new UnsupportedVariableException(sqlStatement.getName());
         }
     }
     
@@ -143,4 +144,9 @@ public final class SetDistVariableHandler extends UpdatableRALBackendHandler<Set
             throw new UnsupportedVariableException(transactionTypeName);
         }
     }
+    
+    @Override
+    public String getType() {
+        return SetDistVariableStatement.class.getName();
+    }
 }
diff --git a/distsql/handler/src/main/java/org/apache/shardingsphere/distsql/handler/ral/update/RALUpdater.java b/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/updater/ConnectionSessionRequiredRALUpdater.java
similarity index 67%
copy from distsql/handler/src/main/java/org/apache/shardingsphere/distsql/handler/ral/update/RALUpdater.java
copy to proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/updater/ConnectionSessionRequiredRALUpdater.java
index 42ab27dd11a..23a17def2e3 100644
--- a/distsql/handler/src/main/java/org/apache/shardingsphere/distsql/handler/ral/update/RALUpdater.java
+++ b/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/updater/ConnectionSessionRequiredRALUpdater.java
@@ -15,28 +15,25 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.distsql.handler.ral.update;
+package org.apache.shardingsphere.proxy.backend.handler.distsql.ral.updatable.updater;
 
-import org.apache.shardingsphere.infra.util.spi.annotation.SingletonSPI;
-import org.apache.shardingsphere.infra.util.spi.type.typed.TypedSPI;
+import org.apache.shardingsphere.distsql.handler.ral.update.RALUpdater;
+import org.apache.shardingsphere.proxy.backend.session.ConnectionSession;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 
 import java.sql.SQLException;
 
 /**
- * RAL updater.
- * 
- * @param <T> type of updatable RAL statement
+ * Connection session required RAL updater.
  */
-@SingletonSPI
-public interface RALUpdater<T extends SQLStatement> extends TypedSPI {
+public interface ConnectionSessionRequiredRALUpdater<T extends SQLStatement> extends RALUpdater<T> {
     
     /**
      * Execute update.
      *
-     * @param databaseName database name
+     * @param connectionSession connection session
      * @param sqlStatement updatable RAL statement
      * @throws SQLException SQL exception
      */
-    void executeUpdate(String databaseName, T sqlStatement) throws SQLException;
+    void executeUpdate(ConnectionSession connectionSession, T sqlStatement) throws SQLException;
 }
diff --git a/proxy/backend/core/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.ral.update.RALUpdater b/proxy/backend/core/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.ral.update.RALUpdater
index 20eef4e9707..acfaeaa9ade 100644
--- a/proxy/backend/core/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.ral.update.RALUpdater
+++ b/proxy/backend/core/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.ral.update.RALUpdater
@@ -21,4 +21,5 @@ org.apache.shardingsphere.proxy.backend.handler.distsql.ral.updatable.UnlabelCom
 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
+org.apache.shardingsphere.proxy.backend.handler.distsql.ral.updatable.SetDistVariableUpdater
 org.apache.shardingsphere.proxy.backend.handler.distsql.ral.updatable.RefreshDatabaseMetaDataUpdater
diff --git a/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/ProxyBackendHandlerFactoryTest.java b/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/ProxyBackendHandlerFactoryTest.java
index 4955c032db9..c3a4e542064 100644
--- a/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/ProxyBackendHandlerFactoryTest.java
+++ b/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/ProxyBackendHandlerFactoryTest.java
@@ -37,8 +37,8 @@ import org.apache.shardingsphere.proxy.backend.handler.admin.DatabaseAdminQueryB
 import org.apache.shardingsphere.proxy.backend.handler.admin.DatabaseAdminUpdateBackendHandler;
 import org.apache.shardingsphere.proxy.backend.handler.data.impl.UnicastDatabaseBackendHandler;
 import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.QueryableRALBackendHandler;
+import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.UpdatableRALUpdaterBackendHandler;
 import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.hint.HintRALBackendHandler;
-import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.updatable.SetDistVariableHandler;
 import org.apache.shardingsphere.proxy.backend.handler.distsql.rql.RQLBackendHandler;
 import org.apache.shardingsphere.proxy.backend.handler.distsql.rul.SQLRULBackendHandler;
 import org.apache.shardingsphere.proxy.backend.handler.skip.SkipBackendHandler;
@@ -112,7 +112,7 @@ public final class ProxyBackendHandlerFactoryTest extends ProxyContextRestorer {
     public void assertNewInstanceWithDistSQL() throws SQLException {
         String sql = "set dist variable transaction_type='LOCAL'";
         ProxyBackendHandler actual = ProxyBackendHandlerFactory.newInstance(databaseType, sql, connectionSession);
-        assertThat(actual, instanceOf(SetDistVariableHandler.class));
+        assertThat(actual, instanceOf(UpdatableRALUpdaterBackendHandler.class));
         sql = "show dist variable where name = transaction_type";
         actual = ProxyBackendHandlerFactory.newInstance(databaseType, sql, connectionSession);
         assertThat(actual, instanceOf(QueryableRALBackendHandler.class));
diff --git a/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/SetDistVariableBackendHandlerTest.java b/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/SetDistVariableUpdatableRALUpdaterBackendHandlerTest.java
similarity index 83%
rename from proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/SetDistVariableBackendHandlerTest.java
rename to proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/SetDistVariableUpdatableRALUpdaterBackendHandlerTest.java
index 20031a2e08a..221b0787cd7 100644
--- a/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/SetDistVariableBackendHandlerTest.java
+++ b/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/SetDistVariableUpdatableRALUpdaterBackendHandlerTest.java
@@ -33,6 +33,7 @@ import org.apache.shardingsphere.mode.metadata.MetaDataContexts;
 import org.apache.shardingsphere.mode.metadata.persist.MetaDataPersistService;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.exception.UnsupportedVariableException;
+import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.UpdatableRALUpdaterBackendHandler;
 import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.common.enums.VariableEnum;
 import org.apache.shardingsphere.proxy.backend.response.header.ResponseHeader;
 import org.apache.shardingsphere.proxy.backend.response.header.update.UpdateResponseHeader;
@@ -58,7 +59,7 @@ import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
-public final class SetDistVariableBackendHandlerTest extends ProxyContextRestorer {
+public final class SetDistVariableUpdatableRALUpdaterBackendHandlerTest extends ProxyContextRestorer {
     
     private static final String DATABASE_PATTERN = "db_%s";
     
@@ -91,8 +92,7 @@ public final class SetDistVariableBackendHandlerTest extends ProxyContextRestore
     @Test
     public void assertSwitchTransactionTypeXA() throws SQLException {
         connectionSession.setCurrentDatabase(String.format(DATABASE_PATTERN, 0));
-        SetDistVariableHandler handler = new SetDistVariableHandler();
-        handler.init(new SetDistVariableStatement("transaction_type", "XA"), connectionSession);
+        UpdatableRALUpdaterBackendHandler handler = new UpdatableRALUpdaterBackendHandler(new SetDistVariableStatement("transaction_type", "XA"), connectionSession);
         ResponseHeader actual = handler.execute();
         assertThat(actual, instanceOf(UpdateResponseHeader.class));
         assertThat(connectionSession.getTransactionStatus().getTransactionType(), is(TransactionType.XA));
@@ -101,8 +101,7 @@ public final class SetDistVariableBackendHandlerTest extends ProxyContextRestore
     @Test
     public void assertSwitchTransactionTypeBASE() throws SQLException {
         connectionSession.setCurrentDatabase(String.format(DATABASE_PATTERN, 0));
-        SetDistVariableHandler handler = new SetDistVariableHandler();
-        handler.init(new SetDistVariableStatement("transaction_type", "BASE"), connectionSession);
+        UpdatableRALUpdaterBackendHandler handler = new UpdatableRALUpdaterBackendHandler(new SetDistVariableStatement("transaction_type", "BASE"), connectionSession);
         ResponseHeader actual = handler.execute();
         assertThat(actual, instanceOf(UpdateResponseHeader.class));
         assertThat(connectionSession.getTransactionStatus().getTransactionType(), is(TransactionType.BASE));
@@ -111,8 +110,7 @@ public final class SetDistVariableBackendHandlerTest extends ProxyContextRestore
     @Test
     public void assertSwitchTransactionTypeLOCAL() throws SQLException {
         connectionSession.setCurrentDatabase(String.format(DATABASE_PATTERN, 0));
-        SetDistVariableHandler handler = new SetDistVariableHandler();
-        handler.init(new SetDistVariableStatement("transaction_type", "LOCAL"), connectionSession);
+        UpdatableRALUpdaterBackendHandler handler = new UpdatableRALUpdaterBackendHandler(new SetDistVariableStatement("transaction_type", "LOCAL"), connectionSession);
         ResponseHeader actual = handler.execute();
         assertThat(actual, instanceOf(UpdateResponseHeader.class));
         assertThat(connectionSession.getTransactionStatus().getTransactionType(), is(TransactionType.LOCAL));
@@ -120,24 +118,21 @@ public final class SetDistVariableBackendHandlerTest extends ProxyContextRestore
     
     @Test(expected = UnsupportedVariableException.class)
     public void assertSwitchTransactionTypeFailed() throws SQLException {
-        SetDistVariableHandler handler = new SetDistVariableHandler();
         connectionSession.setCurrentDatabase(String.format(DATABASE_PATTERN, 0));
-        handler.init(new SetDistVariableStatement("transaction_type", "XXX"), connectionSession);
+        UpdatableRALUpdaterBackendHandler handler = new UpdatableRALUpdaterBackendHandler(new SetDistVariableStatement("transaction_type", "XXX"), connectionSession);
         handler.execute();
     }
     
     @Test(expected = UnsupportedVariableException.class)
     public void assertNotSupportedVariable() throws SQLException {
-        SetDistVariableHandler handler = new SetDistVariableHandler();
-        handler.init(new SetDistVariableStatement("@@session", "XXX"), connectionSession);
+        UpdatableRALUpdaterBackendHandler handler = new UpdatableRALUpdaterBackendHandler(new SetDistVariableStatement("@@session", "XXX"), connectionSession);
         handler.execute();
     }
     
     @Test
     public void assertSetAgentPluginsEnabledTrue() throws SQLException {
-        SetDistVariableHandler handler = new SetDistVariableHandler();
         connectionSession.setCurrentDatabase(String.format(DATABASE_PATTERN, 0));
-        handler.init(new SetDistVariableStatement(VariableEnum.AGENT_PLUGINS_ENABLED.name(), Boolean.TRUE.toString()), null);
+        UpdatableRALUpdaterBackendHandler handler = new UpdatableRALUpdaterBackendHandler(new SetDistVariableStatement(VariableEnum.AGENT_PLUGINS_ENABLED.name(), Boolean.TRUE.toString()), null);
         ResponseHeader actual = handler.execute();
         assertThat(actual, instanceOf(UpdateResponseHeader.class));
         assertThat(SystemPropertyUtil.getSystemProperty(VariableEnum.AGENT_PLUGINS_ENABLED.name(), Boolean.FALSE.toString()), is(Boolean.TRUE.toString()));
@@ -145,9 +140,8 @@ public final class SetDistVariableBackendHandlerTest extends ProxyContextRestore
     
     @Test
     public void assertSetAgentPluginsEnabledFalse() throws SQLException {
-        SetDistVariableHandler handler = new SetDistVariableHandler();
         connectionSession.setCurrentDatabase(String.format(DATABASE_PATTERN, 0));
-        handler.init(new SetDistVariableStatement(VariableEnum.AGENT_PLUGINS_ENABLED.name(), Boolean.FALSE.toString()), null);
+        UpdatableRALUpdaterBackendHandler handler = new UpdatableRALUpdaterBackendHandler(new SetDistVariableStatement(VariableEnum.AGENT_PLUGINS_ENABLED.name(), Boolean.FALSE.toString()), null);
         ResponseHeader actual = handler.execute();
         assertThat(actual, instanceOf(UpdateResponseHeader.class));
         assertThat(SystemPropertyUtil.getSystemProperty(VariableEnum.AGENT_PLUGINS_ENABLED.name(), Boolean.FALSE.toString()), is(Boolean.FALSE.toString()));
@@ -155,9 +149,8 @@ public final class SetDistVariableBackendHandlerTest extends ProxyContextRestore
     
     @Test
     public void assertSetAgentPluginsEnabledFalseWithUnknownValue() throws SQLException {
-        SetDistVariableHandler handler = new SetDistVariableHandler();
         connectionSession.setCurrentDatabase(String.format(DATABASE_PATTERN, 0));
-        handler.init(new SetDistVariableStatement(VariableEnum.AGENT_PLUGINS_ENABLED.name(), "xxx"), connectionSession);
+        UpdatableRALUpdaterBackendHandler handler = new UpdatableRALUpdaterBackendHandler(new SetDistVariableStatement(VariableEnum.AGENT_PLUGINS_ENABLED.name(), "xxx"), connectionSession);
         ResponseHeader actual = handler.execute();
         assertThat(actual, instanceOf(UpdateResponseHeader.class));
         assertThat(SystemPropertyUtil.getSystemProperty(VariableEnum.AGENT_PLUGINS_ENABLED.name(), Boolean.FALSE.toString()), is(Boolean.FALSE.toString()));
diff --git a/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/SetDistVariableExecutorTest.java b/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/SetDistVariableUpdaterTest.java
similarity index 87%
rename from proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/SetDistVariableExecutorTest.java
rename to proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/SetDistVariableUpdaterTest.java
index f1905cdc66f..a9e90cb249d 100644
--- a/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/SetDistVariableExecutorTest.java
+++ b/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/SetDistVariableUpdaterTest.java
@@ -54,7 +54,7 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 @RunWith(MockitoJUnitRunner.class)
-public final class SetDistVariableExecutorTest extends ProxyContextRestorer {
+public final class SetDistVariableUpdaterTest extends ProxyContextRestorer {
     
     @Mock
     private ConnectionSession connectionSession;
@@ -63,18 +63,16 @@ public final class SetDistVariableExecutorTest extends ProxyContextRestorer {
     public void assertExecuteWithTransactionType() throws SQLException {
         SetDistVariableStatement statement = new SetDistVariableStatement("transaction_type", "local");
         when(connectionSession.getTransactionStatus()).thenReturn(new TransactionStatus(TransactionType.XA));
-        SetDistVariableHandler handler = new SetDistVariableHandler();
-        handler.init(statement, connectionSession);
-        handler.execute();
+        SetDistVariableUpdater updater = new SetDistVariableUpdater();
+        updater.executeUpdate(connectionSession, statement);
         assertThat(connectionSession.getTransactionStatus().getTransactionType().name(), is(TransactionType.LOCAL.name()));
     }
     
     @Test
     public void assertExecuteWithAgent() throws SQLException {
         SetDistVariableStatement statement = new SetDistVariableStatement("AGENT_PLUGINS_ENABLED", Boolean.FALSE.toString());
-        SetDistVariableHandler handler = new SetDistVariableHandler();
-        handler.init(statement, connectionSession);
-        handler.execute();
+        SetDistVariableUpdater updater = new SetDistVariableUpdater();
+        updater.executeUpdate(connectionSession, statement);
         String actualValue = SystemPropertyUtil.getSystemProperty(VariableEnum.AGENT_PLUGINS_ENABLED.name(), "default");
         assertThat(actualValue, is(Boolean.FALSE.toString()));
     }
@@ -83,9 +81,8 @@ public final class SetDistVariableExecutorTest extends ProxyContextRestorer {
     public void assertExecuteWithConfigurationKey() throws SQLException {
         ContextManager contextManager = mockContextManager();
         SetDistVariableStatement statement = new SetDistVariableStatement("proxy_frontend_flush_threshold", "1024");
-        SetDistVariableHandler handler = new SetDistVariableHandler();
-        handler.init(statement, connectionSession);
-        handler.execute();
+        SetDistVariableUpdater updater = new SetDistVariableUpdater();
+        updater.executeUpdate(connectionSession, statement);
         Object actualValue = contextManager.getMetaDataContexts().getMetaData().getProps().getProps().get("proxy-frontend-flush-threshold");
         assertThat(actualValue.toString(), is("1024"));
         assertThat(contextManager.getMetaDataContexts().getMetaData().getProps().getValue(ConfigurationPropertyKey.PROXY_FRONTEND_FLUSH_THRESHOLD), is(1024));
@@ -95,9 +92,8 @@ public final class SetDistVariableExecutorTest extends ProxyContextRestorer {
     public void assertExecuteWithInternalConfigurationKey() throws SQLException {
         ContextManager contextManager = mockContextManager();
         SetDistVariableStatement statement = new SetDistVariableStatement("proxy_meta_data_collector_enabled", "false");
-        SetDistVariableHandler handler = new SetDistVariableHandler();
-        handler.init(statement, connectionSession);
-        handler.execute();
+        SetDistVariableUpdater updater = new SetDistVariableUpdater();
+        updater.executeUpdate(connectionSession, statement);
         Object actualValue = contextManager.getMetaDataContexts().getMetaData().getInternalProps().getProps().get("proxy-meta-data-collector-enabled");
         assertThat(actualValue.toString(), is("false"));
         assertThat(contextManager.getMetaDataContexts().getMetaData().getInternalProps().getValue(InternalConfigurationPropertyKey.PROXY_META_DATA_COLLECTOR_ENABLED), is(false));
@@ -107,9 +103,8 @@ public final class SetDistVariableExecutorTest extends ProxyContextRestorer {
     public void assertExecuteWithSystemLogLevel() throws SQLException {
         ContextManager contextManager = mockContextManager();
         SetDistVariableStatement statement = new SetDistVariableStatement("system_log_level", "debug");
-        SetDistVariableHandler handler = new SetDistVariableHandler();
-        handler.init(statement, connectionSession);
-        handler.execute();
+        SetDistVariableUpdater updater = new SetDistVariableUpdater();
+        updater.executeUpdate(connectionSession, statement);
         Object actualValue = contextManager.getMetaDataContexts().getMetaData().getProps().getProps().get("system-log-level");
         assertThat(actualValue.toString(), is("DEBUG"));
         assertThat(contextManager.getMetaDataContexts().getMetaData().getProps().getValue(ConfigurationPropertyKey.SYSTEM_LOG_LEVEL), is(LoggerLevel.DEBUG));
@@ -119,9 +114,8 @@ public final class SetDistVariableExecutorTest extends ProxyContextRestorer {
     public void assertExecuteWithWrongSystemLogLevel() throws SQLException {
         mockContextManager();
         SetDistVariableStatement statement = new SetDistVariableStatement("system_log_level", "invalid");
-        SetDistVariableHandler handler = new SetDistVariableHandler();
-        handler.init(statement, connectionSession);
-        handler.execute();
+        SetDistVariableUpdater updater = new SetDistVariableUpdater();
+        updater.executeUpdate(connectionSession, statement);
     }
     
     private ContextManager mockContextManager() {