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/25 15:22:46 UTC

[shardingsphere] branch master updated: Refactor `RefreshTableMetaDataHandler` (#24261)

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 d2d039ac42f Refactor `RefreshTableMetaDataHandler` (#24261)
d2d039ac42f is described below

commit d2d039ac42f0b74a15b02ccfa1cfc050f617d3dc
Author: Zichao <57...@users.noreply.github.com>
AuthorDate: Sat Feb 25 23:22:30 2023 +0800

    Refactor `RefreshTableMetaDataHandler` (#24261)
    
    * Refactor `RefreshTableMetaDataHandler`
    
    * Fix `hint` sharding strategy in DistSQL
    
    * Refactor `RefreshTableMetaDataHandler`
    
    * Refactor `RefreshTableMetaDataHandler`
    
    * Refactor `RefreshTableMetaDataHandler`
    
    * Refactor `RefreshTableMetaDataHandler`
---
 .../distsql/ral/RALBackendHandlerFactory.java      | 40 +---------------
 .../distsql/ral/UpdatableRALBackendHandler.java    | 42 -----------------
 .../update/UpdatableScalingRALBackendHandler.java  | 54 ----------------------
 ...ndler.java => RefreshTableMetaDataUpdater.java} | 53 ++++++++++++---------
 ...ingsphere.distsql.handler.ral.update.RALUpdater |  1 +
 ...t.java => RefreshTableMetaDataUpdaterTest.java} | 18 +++-----
 6 files changed, 39 insertions(+), 169 deletions(-)

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 7a48504492b..5a156c1ddc0 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
@@ -22,33 +22,17 @@ import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.distsql.handler.ral.update.RALUpdater;
 import org.apache.shardingsphere.distsql.parser.statement.ral.QueryableRALStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.RALStatement;
-import org.apache.shardingsphere.distsql.parser.statement.ral.UpdatableGlobalRuleRALStatement;
 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.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;
 import org.apache.shardingsphere.proxy.backend.handler.ProxyBackendHandler;
-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.session.ConnectionSession;
 
-import java.util.HashMap;
-import java.util.Map;
-
 /**
  * RAL backend handler factory.
  */
 @NoArgsConstructor(access = AccessLevel.PRIVATE)
 public final class RALBackendHandlerFactory {
     
-    private static final Map<Class<? extends RALStatement>, Class<? extends RALBackendHandler<?>>> HANDLERS = new HashMap<>();
-    
-    static {
-        HANDLERS.put(RefreshTableMetaDataStatement.class, RefreshTableMetaDataHandler.class);
-    }
-    
     /**
      * Create new instance of RAL backend handler.
      *
@@ -64,28 +48,6 @@ public final class RALBackendHandlerFactory {
         if (TypedSPILoader.contains(RALUpdater.class, sqlStatement.getClass().getName())) {
             return new UpdatableRALUpdaterBackendHandler<>((UpdatableRALStatement) sqlStatement, connectionSession);
         }
-        if (sqlStatement instanceof UpdatableScalingRALStatement) {
-            return new UpdatableScalingRALBackendHandler((UpdatableScalingRALStatement) sqlStatement, connectionSession);
-        }
-        if (sqlStatement instanceof UpdatableGlobalRuleRALStatement) {
-            return new UpdatableGlobalRuleRALBackendHandler(sqlStatement);
-        }
-        return createRALBackendHandler(sqlStatement, connectionSession);
-    }
-    
-    private static RALBackendHandler<?> newInstance(final Class<? extends RALBackendHandler<?>> clazz) {
-        try {
-            return clazz.getDeclaredConstructor().newInstance();
-        } catch (final ReflectiveOperationException ex) {
-            throw new UnsupportedSQLOperationException(String.format("Can not find public constructor for class `%s`", clazz.getName()));
-        }
-    }
-    
-    private static RALBackendHandler<?> createRALBackendHandler(final RALStatement sqlStatement, final ConnectionSession connectionSession) {
-        Class<? extends RALBackendHandler<?>> clazz = HANDLERS.get(sqlStatement.getClass());
-        ShardingSpherePreconditions.checkState(null != clazz, () -> new UnsupportedSQLOperationException(String.format("Unsupported SQL statement : %s", sqlStatement.getClass().getName())));
-        RALBackendHandler<?> result = newInstance(clazz);
-        result.init(sqlStatement, connectionSession);
-        return result;
+        return new UpdatableGlobalRuleRALBackendHandler(sqlStatement);
     }
 }
diff --git a/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/UpdatableRALBackendHandler.java b/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/UpdatableRALBackendHandler.java
deleted file mode 100644
index d7dcbe947f4..00000000000
--- a/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/UpdatableRALBackendHandler.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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;
-
-import org.apache.shardingsphere.distsql.parser.statement.ral.UpdatableRALStatement;
-import org.apache.shardingsphere.mode.manager.ContextManager;
-import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
-import org.apache.shardingsphere.proxy.backend.response.header.ResponseHeader;
-import org.apache.shardingsphere.proxy.backend.response.header.update.UpdateResponseHeader;
-
-import java.sql.SQLException;
-
-/**
- * Updatable RAL backend handler.
- * 
- * @param <T> type of RAL Statement
- */
-public abstract class UpdatableRALBackendHandler<T extends UpdatableRALStatement> extends RALBackendHandler<T> {
-    
-    @Override
-    public final ResponseHeader execute() throws SQLException {
-        update(ProxyContext.getInstance().getContextManager());
-        return new UpdateResponseHeader(getSqlStatement());
-    }
-    
-    protected abstract void update(ContextManager contextManager) throws SQLException;
-}
diff --git a/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/migration/update/UpdatableScalingRALBackendHandler.java b/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/migration/update/UpdatableScalingRALBackendHandler.java
deleted file mode 100644
index 6ca4cdcf081..00000000000
--- a/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/migration/update/UpdatableScalingRALBackendHandler.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * 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.migration.update;
-
-import lombok.RequiredArgsConstructor;
-import lombok.Setter;
-import org.apache.shardingsphere.distsql.handler.ral.update.RALUpdater;
-import org.apache.shardingsphere.distsql.parser.statement.ral.scaling.UpdatableScalingRALStatement;
-import org.apache.shardingsphere.infra.util.spi.type.typed.TypedSPILoader;
-import org.apache.shardingsphere.proxy.backend.handler.distsql.DistSQLBackendHandler;
-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;
-
-import java.sql.SQLException;
-
-/**
- * Updatable scaling RAL backend handler factory.
- */
-@RequiredArgsConstructor
-@Setter
-public final class UpdatableScalingRALBackendHandler implements DistSQLBackendHandler {
-    
-    private final UpdatableScalingRALStatement sqlStatement;
-    
-    private final ConnectionSession connectionSession;
-    
-    @SuppressWarnings("unchecked")
-    @Override
-    public ResponseHeader execute() throws SQLException {
-        String databaseName = getDatabaseName(connectionSession);
-        TypedSPILoader.getService(RALUpdater.class, sqlStatement.getClass().getName()).executeUpdate(databaseName, sqlStatement);
-        return new UpdateResponseHeader(sqlStatement);
-    }
-    
-    private String getDatabaseName(final ConnectionSession connectionSession) {
-        return connectionSession.getDatabaseName();
-    }
-}
diff --git a/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/RefreshTableMetaDataHandler.java b/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/RefreshTableMetaDataUpdater.java
similarity index 61%
rename from proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/RefreshTableMetaDataHandler.java
rename to proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/RefreshTableMetaDataUpdater.java
index 14653079110..f280b03e44f 100644
--- a/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/RefreshTableMetaDataHandler.java
+++ b/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/RefreshTableMetaDataUpdater.java
@@ -20,14 +20,15 @@ package org.apache.shardingsphere.proxy.backend.handler.distsql.ral.updatable;
 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.parser.statement.ral.updatable.RefreshTableMetaDataStatement;
-import org.apache.shardingsphere.infra.database.type.DatabaseTypeEngine;
 import org.apache.shardingsphere.distsql.handler.exception.storageunit.EmptyStorageUnitException;
 import org.apache.shardingsphere.distsql.handler.exception.storageunit.MissingRequiredStorageUnitsException;
+import org.apache.shardingsphere.distsql.parser.statement.ral.updatable.RefreshTableMetaDataStatement;
+import org.apache.shardingsphere.infra.database.type.DatabaseTypeEngine;
 import org.apache.shardingsphere.infra.util.exception.ShardingSpherePreconditions;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
-import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.UpdatableRALBackendHandler;
+import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.updatable.updater.ConnectionSessionRequiredRALUpdater;
+import org.apache.shardingsphere.proxy.backend.session.ConnectionSession;
 
 import javax.sql.DataSource;
 import java.util.Collections;
@@ -36,38 +37,39 @@ import java.util.Map;
 /**
  * Refresh table meta data handler.
  */
-public final class RefreshTableMetaDataHandler extends UpdatableRALBackendHandler<RefreshTableMetaDataStatement> {
+public final class RefreshTableMetaDataUpdater implements ConnectionSessionRequiredRALUpdater<RefreshTableMetaDataStatement> {
     
     @Override
-    protected void update(final ContextManager contextManager) {
-        String databaseName = getDatabaseName();
-        checkDataSources(databaseName, contextManager.getDataSourceMap(databaseName));
-        String schemaName = getSchemaName(databaseName);
-        if (getSqlStatement().getStorageUnitName().isPresent()) {
-            if (getSqlStatement().getTableName().isPresent()) {
-                contextManager.reloadTable(databaseName, schemaName, getSqlStatement().getStorageUnitName().get(), getSqlStatement().getTableName().get());
+    public void executeUpdate(final ConnectionSession connectionSession, final RefreshTableMetaDataStatement sqlStatement) {
+        String databaseName = getDatabaseName(connectionSession);
+        ContextManager contextManager = ProxyContext.getInstance().getContextManager();
+        checkDataSources(databaseName, contextManager.getDataSourceMap(databaseName), sqlStatement);
+        String schemaName = getSchemaName(databaseName, sqlStatement, connectionSession);
+        if (sqlStatement.getStorageUnitName().isPresent()) {
+            if (sqlStatement.getTableName().isPresent()) {
+                contextManager.reloadTable(databaseName, schemaName, sqlStatement.getStorageUnitName().get(), sqlStatement.getTableName().get());
             } else {
-                contextManager.reloadSchema(databaseName, schemaName, getSqlStatement().getStorageUnitName().get());
+                contextManager.reloadSchema(databaseName, schemaName, sqlStatement.getStorageUnitName().get());
             }
             return;
         }
-        if (getSqlStatement().getTableName().isPresent()) {
-            contextManager.reloadTable(databaseName, schemaName, getSqlStatement().getTableName().get());
+        if (sqlStatement.getTableName().isPresent()) {
+            contextManager.reloadTable(databaseName, schemaName, sqlStatement.getTableName().get());
         } else {
             contextManager.reloadDatabaseMetaData(databaseName);
         }
     }
     
-    private void checkDataSources(final String databaseName, final Map<String, DataSource> dataSources) {
+    private void checkDataSources(final String databaseName, final Map<String, DataSource> dataSources, final RefreshTableMetaDataStatement sqlStatement) {
         ShardingSpherePreconditions.checkState(!dataSources.isEmpty(), () -> new EmptyStorageUnitException(databaseName));
-        if (getSqlStatement().getStorageUnitName().isPresent()) {
-            String storageUnitName = getSqlStatement().getStorageUnitName().get();
+        if (sqlStatement.getStorageUnitName().isPresent()) {
+            String storageUnitName = sqlStatement.getStorageUnitName().get();
             ShardingSpherePreconditions.checkState(dataSources.containsKey(storageUnitName), () -> new MissingRequiredStorageUnitsException(databaseName, Collections.singletonList(storageUnitName)));
         }
     }
     
-    private String getDatabaseName() {
-        String result = getConnectionSession().getDatabaseName();
+    private String getDatabaseName(final ConnectionSession connectionSession) {
+        String result = connectionSession.getDatabaseName();
         if (Strings.isNullOrEmpty(result)) {
             throw new NoDatabaseSelectedException();
         }
@@ -77,9 +79,14 @@ public final class RefreshTableMetaDataHandler extends UpdatableRALBackendHandle
         return result;
     }
     
-    private String getSchemaName(final String databaseName) {
-        return getSqlStatement().getSchemaName().isPresent()
-                ? getSqlStatement().getSchemaName().get()
-                : DatabaseTypeEngine.getDefaultSchemaName(getConnectionSession().getProtocolType(), databaseName);
+    private String getSchemaName(final String databaseName, final RefreshTableMetaDataStatement sqlStatement, final ConnectionSession connectionSession) {
+        return sqlStatement.getSchemaName().isPresent()
+                ? sqlStatement.getSchemaName().get()
+                : DatabaseTypeEngine.getDefaultSchemaName(connectionSession.getProtocolType(), databaseName);
+    }
+    
+    @Override
+    public String getType() {
+        return RefreshTableMetaDataStatement.class.getName();
     }
 }
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 acfaeaa9ade..54b05b5a171 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
@@ -23,3 +23,4 @@ org.apache.shardingsphere.proxy.backend.handler.distsql.ral.updatable.ImportData
 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
+org.apache.shardingsphere.proxy.backend.handler.distsql.ral.updatable.RefreshTableMetaDataUpdater
diff --git a/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/RefreshTableMetaDataHandlerTest.java b/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/RefreshTableMetaDataUpdaterTest.java
similarity index 84%
rename from proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/RefreshTableMetaDataHandlerTest.java
rename to proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/RefreshTableMetaDataUpdaterTest.java
index b865693dc2b..412240a4010 100644
--- a/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/RefreshTableMetaDataHandlerTest.java
+++ b/proxy/backend/core/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/updatable/RefreshTableMetaDataUpdaterTest.java
@@ -24,6 +24,7 @@ import org.apache.shardingsphere.distsql.handler.exception.storageunit.MissingRe
 import org.apache.shardingsphere.distsql.parser.statement.ral.updatable.RefreshTableMetaDataStatement;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
+import org.apache.shardingsphere.proxy.backend.handler.distsql.ral.UpdatableRALUpdaterBackendHandler;
 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;
@@ -41,12 +42,11 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.mockStatic;
 import static org.mockito.Mockito.when;
 
-public final class RefreshTableMetaDataHandlerTest {
+public final class RefreshTableMetaDataUpdaterTest {
     
     @Test(expected = NoDatabaseSelectedException.class)
     public void assertNoDatabaseSelected() throws SQLException {
-        RefreshTableMetaDataHandler backendHandler = new RefreshTableMetaDataHandler();
-        backendHandler.init(new RefreshTableMetaDataStatement(), mock(ConnectionSession.class));
+        UpdatableRALUpdaterBackendHandler backendHandler = new UpdatableRALUpdaterBackendHandler(new RefreshTableMetaDataStatement(), mock(ConnectionSession.class));
         try (MockedStatic<ProxyContext> proxyContext = mockStatic(ProxyContext.class, RETURNS_DEEP_STUBS)) {
             proxyContext.when(() -> ProxyContext.getInstance().getContextManager()).thenReturn(mock(ContextManager.class, RETURNS_DEEP_STUBS));
             backendHandler.execute();
@@ -55,8 +55,7 @@ public final class RefreshTableMetaDataHandlerTest {
     
     @Test(expected = UnknownDatabaseException.class)
     public void assertUnknownDatabaseException() throws SQLException {
-        RefreshTableMetaDataHandler backendHandler = new RefreshTableMetaDataHandler();
-        backendHandler.init(new RefreshTableMetaDataStatement(), mockConnectionSession("not_existed_db"));
+        UpdatableRALUpdaterBackendHandler backendHandler = new UpdatableRALUpdaterBackendHandler(new RefreshTableMetaDataStatement(), mockConnectionSession("not_existed_db"));
         try (MockedStatic<ProxyContext> proxyContext = mockStatic(ProxyContext.class, RETURNS_DEEP_STUBS)) {
             proxyContext.when(() -> ProxyContext.getInstance().getContextManager()).thenReturn(mock(ContextManager.class, RETURNS_DEEP_STUBS));
             backendHandler.execute();
@@ -65,8 +64,7 @@ public final class RefreshTableMetaDataHandlerTest {
     
     @Test(expected = EmptyStorageUnitException.class)
     public void assertEmptyResource() throws SQLException {
-        RefreshTableMetaDataHandler backendHandler = new RefreshTableMetaDataHandler();
-        backendHandler.init(new RefreshTableMetaDataStatement(), mockConnectionSession("sharding_db"));
+        UpdatableRALUpdaterBackendHandler backendHandler = new UpdatableRALUpdaterBackendHandler(new RefreshTableMetaDataStatement(), mockConnectionSession("sharding_db"));
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
         when(contextManager.getDataSourceMap("sharding_db")).thenReturn(Collections.emptyMap());
         try (MockedStatic<ProxyContext> proxyContext = mockStatic(ProxyContext.class, RETURNS_DEEP_STUBS)) {
@@ -78,8 +76,7 @@ public final class RefreshTableMetaDataHandlerTest {
     
     @Test(expected = MissingRequiredStorageUnitsException.class)
     public void assertMissingRequiredResources() throws SQLException {
-        RefreshTableMetaDataHandler backendHandler = new RefreshTableMetaDataHandler();
-        backendHandler.init(new RefreshTableMetaDataStatement("t_order", "ds_1", null), mockConnectionSession("sharding_db"));
+        UpdatableRALUpdaterBackendHandler backendHandler = new UpdatableRALUpdaterBackendHandler(new RefreshTableMetaDataStatement("t_order", "ds_1", null), mockConnectionSession("sharding_db"));
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
         when(contextManager.getDataSourceMap("sharding_db")).thenReturn(Collections.singletonMap("ds_0", new MockedDataSource()));
         try (MockedStatic<ProxyContext> proxyContext = mockStatic(ProxyContext.class, RETURNS_DEEP_STUBS)) {
@@ -91,8 +88,7 @@ public final class RefreshTableMetaDataHandlerTest {
     
     @Test
     public void assertUpdate() throws SQLException {
-        RefreshTableMetaDataHandler backendHandler = new RefreshTableMetaDataHandler();
-        backendHandler.init(new RefreshTableMetaDataStatement(), mockConnectionSession("sharding_db"));
+        UpdatableRALUpdaterBackendHandler backendHandler = new UpdatableRALUpdaterBackendHandler(new RefreshTableMetaDataStatement(), mockConnectionSession("sharding_db"));
         ContextManager contextManager = mock(ContextManager.class, RETURNS_DEEP_STUBS);
         when(contextManager.getDataSourceMap("sharding_db")).thenReturn(Collections.singletonMap("ds_0", new MockedDataSource()));
         try (MockedStatic<ProxyContext> proxyContext = mockStatic(ProxyContext.class, RETURNS_DEEP_STUBS)) {